diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index 0dff3745a375..3c450ea215d5 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -2685,7 +2685,7 @@ func TestRestoreAsOfSystemTimeGCBounds(t *testing.T) { ctx, tc, sqlDB, _, cleanupFn := BackupRestoreTestSetup(t, singleNode, numAccounts, InitNone) defer cleanupFn() const dir = "nodelocal://0/" - preGC := tree.TimestampToDecimal(tc.Server(0).Clock().Now()).String() + preGC := tree.TimestampToDecimalDatum(tc.Server(0).Clock().Now()).String() gcr := roachpb.GCRequest{ // Bogus span to make it a valid request. @@ -2701,7 +2701,7 @@ func TestRestoreAsOfSystemTimeGCBounds(t *testing.T) { t.Fatal(err) } - postGC := tree.TimestampToDecimal(tc.Server(0).Clock().Now()).String() + postGC := tree.TimestampToDecimalDatum(tc.Server(0).Clock().Now()).String() lateFullTableBackup := dir + "/tbl-after-gc" sqlDB.Exec(t, `BACKUP data.bank TO $1 WITH revision_history`, lateFullTableBackup) diff --git a/pkg/ccl/changefeedccl/encoder.go b/pkg/ccl/changefeedccl/encoder.go index 80416ccf2f75..629842951628 100644 --- a/pkg/ccl/changefeedccl/encoder.go +++ b/pkg/ccl/changefeedccl/encoder.go @@ -251,7 +251,7 @@ func (e *jsonEncoder) EncodeResolvedTimestamp( _ context.Context, _ string, resolved hlc.Timestamp, ) ([]byte, error) { meta := map[string]interface{}{ - `resolved`: tree.TimestampToDecimal(resolved).Decimal.String(), + `resolved`: tree.TimestampToDecimalDatum(resolved).Decimal.String(), } var jsonEntries interface{} if e.wrapped { diff --git a/pkg/ccl/logictestccl/testdata/logic_test/partitioning b/pkg/ccl/logictestccl/testdata/logic_test/partitioning index 08ac596c9331..b49f81bc8e2a 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/partitioning +++ b/pkg/ccl/logictestccl/testdata/logic_test/partitioning @@ -423,6 +423,7 @@ TABLE ok1 ├── a int not null ├── b int not null ├── c int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] └── INDEX primary ├── a int not null ├── b int not null @@ -465,6 +466,7 @@ TABLE ok2 ├── a int not null ├── b int not null ├── c int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] └── INDEX primary ├── a int not null ├── b int not null @@ -507,6 +509,7 @@ TABLE ok3 ├── a int not null ├── b int not null ├── c int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] └── INDEX primary ├── a int not null ├── b int not null @@ -552,6 +555,7 @@ TABLE ok4 ├── a int not null ├── b int not null ├── c int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] └── INDEX primary ├── a int not null ├── b int not null @@ -590,6 +594,7 @@ TABLE ok5 ├── a int not null ├── b int not null ├── c int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] └── INDEX primary ├── a int not null ├── b int not null @@ -653,6 +658,7 @@ TABLE ok6 ├── a int not null ├── b int not null ├── c int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] └── INDEX primary ├── a int not null └── b int not null @@ -688,6 +694,7 @@ TABLE ok7 ├── a int not null ├── b int not null ├── c int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] └── INDEX primary ├── a int not null └── b int not null @@ -729,6 +736,7 @@ TABLE ok8 ├── a int not null ├── b int not null ├── c int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] └── INDEX primary ├── a int not null └── b int not null @@ -772,6 +780,7 @@ TABLE ok9 ├── a int not null ├── b int not null ├── c int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] └── INDEX primary ├── a int not null └── b int not null @@ -817,6 +826,7 @@ TABLE ok10 ├── a int not null ├── b int not null ├── c int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] └── INDEX primary ├── a int not null └── b int not null @@ -871,6 +881,7 @@ TABLE ok11 ├── a int not null ├── b int not null ├── c int not null + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] └── INDEX primary ├── a int not null ├── b int not null @@ -916,6 +927,7 @@ TABLE ok12 ├── a int not null ├── b int not null ├── c int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] └── INDEX primary ├── a int not null ├── b int not null diff --git a/pkg/ccl/logictestccl/testdata/logic_test/zone b/pkg/ccl/logictestccl/testdata/logic_test/zone index b2915eaac4f6..878f3f87079d 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/zone +++ b/pkg/ccl/logictestccl/testdata/logic_test/zone @@ -39,6 +39,7 @@ EXPLAIN (OPT, CATALOG) SELECT * FROM t TABLE t ├── k int not null ├── v string + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── FAMILY fam_0_k_v (k, v) ├── INDEX primary │ ├── k int not null @@ -82,6 +83,7 @@ EXPLAIN (OPT, CATALOG) SELECT * FROM t TABLE t ├── k int not null ├── v string + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── FAMILY fam_0_k_v (k, v) ├── INDEX primary │ ├── k int not null @@ -145,6 +147,7 @@ EXPLAIN (OPT, CATALOG) SELECT * FROM t TABLE t ├── k int not null ├── v string + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── FAMILY fam_0_k_v (k, v) ├── INDEX primary │ ├── k int not null @@ -247,6 +250,7 @@ EXPLAIN (OPT, CATALOG) SELECT * FROM t TABLE t ├── k int not null ├── v string + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── FAMILY fam_0_k_v (k, v) ├── INDEX primary │ ├── k int not null @@ -300,6 +304,7 @@ EXPLAIN (OPT, CATALOG) SELECT * FROM t TABLE t ├── k int not null ├── v string + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── FAMILY fam_0_k_v (k, v) ├── INDEX primary │ ├── k int not null @@ -401,6 +406,7 @@ EXPLAIN (OPT, CATALOG) SELECT * FROM t TABLE t ├── k int not null ├── v string + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── FAMILY fam_0_k_v (k, v) ├── INDEX primary │ ├── k int not null diff --git a/pkg/sql/colexec/colbuilder/execplan.go b/pkg/sql/colexec/colbuilder/execplan.go index 4c28526967a9..ae9bf2fbb83a 100644 --- a/pkg/sql/colexec/colbuilder/execplan.go +++ b/pkg/sql/colexec/colbuilder/execplan.go @@ -632,8 +632,7 @@ func NewColOperator( // still responsible for doing the cancellation check on their own while // performing long operations. result.Op = colexec.NewCancelChecker(result.Op) - returnMutations := core.TableReader.Visibility == execinfra.ScanVisibilityPublicAndNotPublic - result.ColumnTypes = core.TableReader.Table.ColumnTypesWithMutations(returnMutations) + result.ColumnTypes = scanOp.ResultTypes case core.Aggregator != nil: if err := checkNumIn(inputs, 1); err != nil { return r, err diff --git a/pkg/sql/colfetcher/cfetcher.go b/pkg/sql/colfetcher/cfetcher.go index 433f3d87b2af..8fea03e16137 100644 --- a/pkg/sql/colfetcher/cfetcher.go +++ b/pkg/sql/colfetcher/cfetcher.go @@ -17,6 +17,7 @@ import ( "sort" "strings" + "github.com/cockroachdb/apd/v2" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" @@ -32,6 +33,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/encoding" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -93,6 +95,15 @@ type cTableInfo struct { // id pair at the start of the key. knownPrefixLength int + // The following fields contain MVCC metadata for each row and may be + // returned to users of cFetcher immediately after NextBatch returns. + // + // rowLastModified is the timestamp of the last time any family in the row + // was modified in any way. + rowLastModified hlc.Timestamp + // timestampOutputIdx controls at what row ordinal to write the timestamp. + timestampOutputIdx int + keyValTypes []*types.T extraTypes []*types.T @@ -143,6 +154,10 @@ func (m colIdxMap) get(c sqlbase.ColumnID) (int, bool) { return 0, false } +// noTimestampColumn is a sentinel value to denote that the MVCC timestamp +// column is not part of the output. +const noTimestampColumn = -1 + // cFetcher handles fetching kvs and forming table rows for an // arbitrary number of tables. // Usage: @@ -186,6 +201,11 @@ type cFetcher struct { // when beginning a new scan. traceKV bool + // mvccDecodeStrategy controls whether or not MVCC timestamps should + // be decoded from KV's fetched. It is set if any of the requested tables + // are required to produce an MVCC timestamp system column. + mvccDecodeStrategy row.MVCCDecodingStrategy + // fetcher is the underlying fetcher that provides KVs. fetcher *row.KVFetcher @@ -221,6 +241,11 @@ type cFetcher struct { // colvecs is a slice of the ColVecs within batch, pulled out to avoid // having to call batch.Vec too often in the tight loop. colvecs []coldata.Vec + + // timestampCol is the underlying ColVec for the timestamp output column, + // or nil if the timestamp column was not requested. It is pulled out from + // colvecs to avoid having to cast the vec to decimal on every write. + timestampCol []apd.Decimal } // adapter is a utility struct that helps with memory accounting. @@ -267,12 +292,13 @@ func (rf *cFetcher) Init( sort.Sort(m) colDescriptors := tableArgs.Cols table := &cTableInfo{ - spans: tableArgs.Spans, - desc: tableArgs.Desc, - colIdxMap: m, - index: tableArgs.Index, - isSecondaryIndex: tableArgs.IsSecondaryIndex, - cols: colDescriptors, + spans: tableArgs.Spans, + desc: tableArgs.Desc, + colIdxMap: m, + index: tableArgs.Index, + isSecondaryIndex: tableArgs.IsSecondaryIndex, + cols: colDescriptors, + timestampOutputIdx: noTimestampColumn, } typs := make([]*types.T, len(colDescriptors)) @@ -280,9 +306,6 @@ func (rf *cFetcher) Init( typs[i] = colDescriptors[i].Type } - rf.machine.batch = allocator.NewMemBatch(typs) - rf.machine.colvecs = rf.machine.batch.ColVecs() - var err error var neededCols util.FastIntSet @@ -294,10 +317,24 @@ func (rf *cFetcher) Init( // The idx-th column is required. neededCols.Add(int(col)) table.neededColsList = append(table.neededColsList, int(col)) + // If this column is the timestamp column, set up the output index. + sysColKind := sqlbase.GetSystemColumnKindFromColumnID(col) + if sysColKind == sqlbase.SystemColumnKind_MVCCTIMESTAMP { + table.timestampOutputIdx = idx + rf.mvccDecodeStrategy = row.MVCCDecodingRequired + } } } sort.Ints(table.neededColsList) + rf.machine.batch = allocator.NewMemBatch(typs) + rf.machine.colvecs = rf.machine.batch.ColVecs() + // If the fetcher is requested to produce a timestamp column, pull out the + // column as a decimal and save it. + if table.timestampOutputIdx != noTimestampColumn { + rf.machine.timestampCol = rf.machine.colvecs[table.timestampOutputIdx].Decimal() + } + table.knownPrefixLength = len(sqlbase.MakeIndexKeyPrefix(codec, table.desc.TableDesc(), table.index.ID)) var indexColumnIDs []sqlbase.ColumnID @@ -309,6 +346,15 @@ func (rf *cFetcher) Init( } table.neededValueColsByIdx = tableArgs.ValNeededForCol.Copy() + + // If system columns are requested, they are present in ValNeededForCol. + // However, we don't want to include them in neededValueColsByIdx, because + // the handling of system columns is separate from the standard value + // decoding process. + if table.timestampOutputIdx != noTimestampColumn { + table.neededValueColsByIdx.Remove(table.timestampOutputIdx) + } + neededIndexCols := 0 nIndexCols := len(indexColumnIDs) if cap(table.indexColOrdinals) >= nIndexCols { @@ -590,7 +636,7 @@ func (rf *cFetcher) nextBatch(ctx context.Context) (coldata.Batch, error) { case stateInvalid: return nil, errors.New("invalid fetcher state") case stateInitFetch: - moreKeys, kv, newSpan, err := rf.fetcher.NextKV(ctx) + moreKeys, kv, newSpan, err := rf.fetcher.NextKV(ctx, rf.mvccDecodeStrategy) if err != nil { return nil, colexecerror.NewStorageError(err) } @@ -628,6 +674,9 @@ func (rf *cFetcher) nextBatch(ctx context.Context) (coldata.Batch, error) { rf.machine.batch.ResetInternalBatch() rf.shiftState() case stateDecodeFirstKVOfRow: + // Reset MVCC metadata for the table, since this is the first KV of a row. + rf.table.rowLastModified = hlc.Timestamp{} + // 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 // to tell whether they have extra columns appended to the key. @@ -732,6 +781,10 @@ func (rf *cFetcher) nextBatch(ctx context.Context) (coldata.Batch, error) { if rf.traceKV { log.VEventf(ctx, 2, "fetched: %s -> %s", prettyKey, prettyVal) } + // Update the MVCC values for this row. + if rf.table.rowLastModified.Less(rf.machine.nextKV.Value.Timestamp) { + rf.table.rowLastModified = rf.machine.nextKV.Value.Timestamp + } if len(rf.table.desc.Families) == 1 { rf.machine.state[0] = stateFinalizeRow rf.machine.state[1] = stateInitFetch @@ -740,7 +793,7 @@ func (rf *cFetcher) nextBatch(ctx context.Context) (coldata.Batch, error) { rf.machine.state[0] = stateFetchNextKVWithUnfinishedRow case stateSeekPrefix: for { - moreRows, kv, _, err := rf.fetcher.NextKV(ctx) + moreRows, kv, _, err := rf.fetcher.NextKV(ctx, rf.mvccDecodeStrategy) if err != nil { return nil, colexecerror.NewStorageError(err) } @@ -772,7 +825,7 @@ func (rf *cFetcher) nextBatch(ctx context.Context) (coldata.Batch, error) { rf.shiftState() case stateFetchNextKVWithUnfinishedRow: - moreKVs, kv, _, err := rf.fetcher.NextKV(ctx) + moreKVs, kv, _, err := rf.fetcher.NextKV(ctx, rf.mvccDecodeStrategy) if err != nil { return nil, colexecerror.NewStorageError(err) } @@ -823,6 +876,11 @@ func (rf *cFetcher) nextBatch(ctx context.Context) (coldata.Batch, error) { log.VEventf(ctx, 2, "fetched: %s -> %s", prettyKey, prettyVal) } + // Update the MVCC values for this row. + if rf.table.rowLastModified.Less(rf.machine.nextKV.Value.Timestamp) { + rf.table.rowLastModified = rf.machine.nextKV.Value.Timestamp + } + if familyID == rf.table.maxColumnFamilyID { // We know the row can't have any more keys, so finalize the row. rf.machine.state[0] = stateFinalizeRow @@ -833,6 +891,10 @@ func (rf *cFetcher) nextBatch(ctx context.Context) (coldata.Batch, error) { } case stateFinalizeRow: + // Populate the row with the buffered MVCC information. + if rf.table.timestampOutputIdx != noTimestampColumn { + rf.machine.timestampCol[rf.machine.rowIdx] = tree.TimestampToDecimal(rf.table.rowLastModified) + } // We're finished with a row. Bump the row index, fill the row in with // nulls if necessary, emit the batch if necessary, and move to the next // state. diff --git a/pkg/sql/colfetcher/colbatch_scan.go b/pkg/sql/colfetcher/colbatch_scan.go index 0bbb81957fd7..c04f9aabde19 100644 --- a/pkg/sql/colfetcher/colbatch_scan.go +++ b/pkg/sql/colfetcher/colbatch_scan.go @@ -23,11 +23,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" + "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/errors" ) -// TODO(yuzefovich): reading the data through a pair of colBatchScan and +// 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 @@ -35,9 +36,9 @@ import ( // 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 +// 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 { +type ColBatchScan struct { colexecbase.ZeroInputNode spans roachpb.Spans flowCtx *execinfra.FlowCtx @@ -50,11 +51,16 @@ type colBatchScan struct { rowsRead int // init is true after Init() has been called. init bool + // 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 } -var _ colexecbase.Operator = &colBatchScan{} +var _ colexecbase.Operator = &ColBatchScan{} -func (s *colBatchScan) Init() { +// Init initializes a ColBatchScan. +func (s *ColBatchScan) Init() { s.ctx = context.Background() s.init = true @@ -67,7 +73,8 @@ func (s *colBatchScan) Init() { } } -func (s *colBatchScan) Next(ctx context.Context) coldata.Batch { +// Next is part of the Operator interface. +func (s *ColBatchScan) Next(ctx context.Context) coldata.Batch { bat, err := s.rf.NextBatch(ctx) if err != nil { colexecerror.InternalError(err) @@ -80,7 +87,7 @@ func (s *colBatchScan) Next(ctx context.Context) coldata.Batch { } // DrainMeta is part of the MetadataSource interface. -func (s *colBatchScan) DrainMeta(ctx context.Context) []execinfrapb.ProducerMetadata { +func (s *ColBatchScan) DrainMeta(ctx context.Context) []execinfrapb.ProducerMetadata { if !s.init { // In some pathological queries like `SELECT 1 FROM t HAVING true`, Init() // and Next() may never get called. Return early to avoid using an @@ -108,26 +115,37 @@ func (s *colBatchScan) DrainMeta(ctx context.Context) []execinfrapb.ProducerMeta return trailingMeta } -// NewColBatchScan creates a new colBatchScan operator. +// NewColBatchScan creates a new ColBatchScan operator. func NewColBatchScan( allocator *colmem.Allocator, flowCtx *execinfra.FlowCtx, spec *execinfrapb.TableReaderSpec, post *execinfrapb.PostProcessSpec, -) (colexecbase.DrainableOperator, error) { +) (*ColBatchScan, 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, errors.Errorf("attempting to create a ColBatchScan with uninitialized NodeID") } limitHint := execinfra.LimitHint(spec.LimitHint, post) returnMutations := spec.Visibility == execinfra.ScanVisibilityPublicAndNotPublic typs := spec.Table.ColumnTypesWithMutations(returnMutations) + columnIdxMap := spec.Table.ColumnIdxMapWithMutations(returnMutations) + // Add all requested system columns to the output. + sysColTypes, sysColDescs, err := sqlbase.GetSystemColumnTypesAndDescriptors(&spec.Table, spec.SystemColumns) + if err != nil { + return nil, err + } + typs = append(typs, sysColTypes...) + for i := range sysColDescs { + columnIdxMap[sysColDescs[i].ID] = len(columnIdxMap) + } + evalCtx := flowCtx.NewEvalCtx() // Before we can safely use types from the table descriptor, we need to // make sure they are hydrated. In row execution engine it is done during - // the processor initialization, but neither colBatchScan nor cFetcher are + // the processor initialization, but neither ColBatchScan nor cFetcher are // processors, so we need to do the hydration ourselves. if err := execinfrapb.HydrateTypeSlice(evalCtx, typs); err != nil { return nil, err @@ -144,7 +162,6 @@ func NewColBatchScan( neededColumns := helper.NeededColumns() - columnIdxMap := spec.Table.ColumnIdxMapWithMutations(returnMutations) fetcher := cFetcher{} if spec.IsCheck { // cFetchers don't support these checks. @@ -152,7 +169,7 @@ func NewColBatchScan( } if _, _, err := initCRowFetcher( flowCtx.Codec(), allocator, &fetcher, &spec.Table, int(spec.IndexIdx), columnIdxMap, - spec.Reverse, neededColumns, spec.Visibility, spec.LockingStrength, + spec.Reverse, neededColumns, spec.Visibility, spec.LockingStrength, sysColDescs, ); err != nil { return nil, err } @@ -162,7 +179,7 @@ func NewColBatchScan( for i := range spans { spans[i] = spec.Spans[i].Span } - return &colBatchScan{ + return &ColBatchScan{ spans: spans, flowCtx: flowCtx, rf: &fetcher, @@ -170,6 +187,7 @@ func NewColBatchScan( // Parallelize shouldn't be set when there's a limit hint, but double-check // just in case. parallelize: spec.Parallelize && limitHint == 0, + ResultTypes: typs, }, nil } @@ -185,6 +203,7 @@ func initCRowFetcher( valNeededForCol util.FastIntSet, scanVisibility execinfrapb.ScanVisibility, lockStr sqlbase.ScanLockingStrength, + systemColumnDescs []sqlbase.ColumnDescriptor, ) (index *sqlbase.IndexDescriptor, isSecondaryIndex bool, err error) { immutDesc := sqlbase.NewImmutableTableDescriptor(*desc) index, isSecondaryIndex, err = immutDesc.FindIndexByIndexIdx(indexIdx) @@ -196,6 +215,9 @@ func initCRowFetcher( if scanVisibility == execinfra.ScanVisibilityPublicAndNotPublic { cols = immutDesc.ReadableColumns } + // Add on any requested system columns. We slice cols to avoid modifying + // the underlying table descriptor. + cols = append(cols[:len(cols):len(cols)], systemColumnDescs...) tableArgs := row.FetcherTableArgs{ Desc: immutDesc, Index: index, diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index bdbf7279bb09..64360f84a080 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -286,7 +286,7 @@ CREATE TABLE crdb_internal.tables ( dbName, tree.NewDInt(tree.DInt(int64(table.Version))), tree.TimestampToInexactDTimestamp(table.ModificationTime), - tree.TimestampToDecimal(table.ModificationTime), + tree.TimestampToDecimalDatum(table.ModificationTime), tree.NewDString(table.FormatVersion.String()), tree.NewDString(table.State.String()), leaseNodeDatum, @@ -578,7 +578,7 @@ CREATE TABLE crdb_internal.jobs ( // Progress contains either fractionCompleted for traditional jobs, // or the highWaterTimestamp for change feeds. if highwater := progress.GetHighWater(); highwater != nil { - highWaterTimestamp = tree.TimestampToDecimal(*highwater) + highWaterTimestamp = tree.TimestampToDecimalDatum(*highwater) } else { fractionCompleted = tree.NewDFloat(tree.DFloat(progress.GetFractionCompleted())) } diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index c25445fa11cb..976a5cd83d84 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -936,7 +936,8 @@ func initTableReaderSpec( LockingWaitPolicy: n.lockingWaitPolicy, // Retain the capacity of the spans slice. - Spans: s.Spans[:0], + Spans: s.Spans[:0], + SystemColumns: n.systemColumns, } indexIdx, err := getIndexIdx(n.index, n.desc) if err != nil { @@ -979,6 +980,13 @@ func tableOrdinal( } } } + + // The column is an implicit system column, so give it an ordinal based + // on its ID that is larger than physical columns. + if sqlbase.IsColIDSystemColumn(colID) { + return len(desc.Columns) + len(desc.MutationColumns()) + int(colID-sqlbase.MVCCTimestampColumnID) + } + panic(fmt.Sprintf("column %d not in desc.Columns", colID)) } @@ -1121,17 +1129,19 @@ func (dsp *DistSQLPlanner) createTableReaders( planCtx, &p, &tableReaderPlanningInfo{ - spec: spec, - post: post, - desc: n.desc, - spans: n.spans, - reverse: n.reverse, - scanVisibility: n.colCfg.visibility, - parallelize: n.parallelize, - estimatedRowCount: n.estimatedRowCount, - reqOrdering: n.reqOrdering, - cols: n.cols, - colsToTableOrdrinalMap: scanNodeToTableOrdinalMap, + spec: spec, + post: post, + desc: n.desc, + spans: n.spans, + reverse: n.reverse, + scanVisibility: n.colCfg.visibility, + parallelize: n.parallelize, + estimatedRowCount: n.estimatedRowCount, + reqOrdering: n.reqOrdering, + cols: n.cols, + colsToTableOrdinalMap: scanNodeToTableOrdinalMap, + systemColumns: n.systemColumns, + systemColumnOrdinals: n.systemColumnOrdinals, }, ) return &p, err @@ -1141,17 +1151,19 @@ func (dsp *DistSQLPlanner) createTableReaders( // needed to perform the physical planning of table readers once the specs have // been created. See scanNode to get more context on some of the fields. type tableReaderPlanningInfo struct { - spec *execinfrapb.TableReaderSpec - post execinfrapb.PostProcessSpec - desc *sqlbase.ImmutableTableDescriptor - spans []roachpb.Span - reverse bool - scanVisibility execinfrapb.ScanVisibility - parallelize bool - estimatedRowCount uint64 - reqOrdering ReqOrdering - cols []*sqlbase.ColumnDescriptor - colsToTableOrdrinalMap []int + spec *execinfrapb.TableReaderSpec + post execinfrapb.PostProcessSpec + desc *sqlbase.ImmutableTableDescriptor + spans []roachpb.Span + reverse bool + scanVisibility execinfrapb.ScanVisibility + parallelize bool + estimatedRowCount uint64 + reqOrdering ReqOrdering + cols []*sqlbase.ColumnDescriptor + colsToTableOrdinalMap []int + systemColumns []sqlbase.SystemColumnKind + systemColumnOrdinals []int } func (dsp *DistSQLPlanner) planTableReaders( @@ -1228,12 +1240,16 @@ func (dsp *DistSQLPlanner) planTableReaders( typs = append(typs, col.Type) } } + // Append all system column types to the output. + for _, kind := range info.systemColumns { + typs = append(typs, sqlbase.GetSystemColumnTypeForKind(kind)) + } p.AddNoInputStage( - corePlacement, info.post, typs, dsp.convertOrdering(info.reqOrdering, info.colsToTableOrdrinalMap), + corePlacement, info.post, typs, dsp.convertOrdering(info.reqOrdering, info.colsToTableOrdinalMap), ) - outCols := getOutputColumnsFromColsForScan(info.cols, info.colsToTableOrdrinalMap) + outCols := getOutputColumnsFromColsForScan(info.cols, info.colsToTableOrdinalMap) planToStreamColMap := make([]int, len(info.cols)) descColumnIDs := make([]sqlbase.ColumnID, 0, len(info.desc.Columns)) for i := range info.desc.Columns { @@ -1244,6 +1260,11 @@ func (dsp *DistSQLPlanner) planTableReaders( descColumnIDs = append(descColumnIDs, c.ID) } } + // Add all system column IDs to the projection. + for _, ord := range info.systemColumnOrdinals { + descColumnIDs = append(descColumnIDs, info.cols[ord].ID) + } + for i := range planToStreamColMap { planToStreamColMap[i] = -1 for j, c := range outCols { @@ -1915,6 +1936,7 @@ func (dsp *DistSQLPlanner) createPlanForIndexJoin( Visibility: n.table.colCfg.visibility, LockingStrength: n.table.lockingStrength, LockingWaitPolicy: n.table.lockingWaitPolicy, + SystemColumns: n.table.systemColumns, } post := execinfrapb.PostProcessSpec{ diff --git a/pkg/sql/distsql_spec_exec_factory.go b/pkg/sql/distsql_spec_exec_factory.go index 17298ddf614a..8fd780a67d8e 100644 --- a/pkg/sql/distsql_spec_exec_factory.go +++ b/pkg/sql/distsql_spec_exec_factory.go @@ -189,6 +189,10 @@ func (e *distSQLSpecExecFactory) ConstructScan( tabDesc := table.(*optTable).desc indexDesc := index.(*optIndex).desc colCfg := makeScanColumnsConfig(table, params.NeededCols) + + // Check if any system columns are requested, as they need special handling. + systemColumns, systemColumnOrdinals := collectSystemColumnsFromCfg(&colCfg, tabDesc.TableDesc()) + sb := span.MakeBuilder(e.planner.ExecCfg().Codec, tabDesc.TableDesc(), indexDesc) // Note that initColsForScan and setting ResultColumns below are equivalent @@ -238,7 +242,8 @@ func (e *distSQLSpecExecFactory) ConstructScan( IsCheck: false, Visibility: colCfg.visibility, // Retain the capacity of the spans slice. - Spans: trSpec.Spans[:0], + Spans: trSpec.Spans[:0], + SystemColumns: systemColumns, } trSpec.IndexIdx, err = getIndexIdx(indexDesc, tabDesc) if err != nil { @@ -270,17 +275,19 @@ func (e *distSQLSpecExecFactory) ConstructScan( e.getPlanCtx(recommendation), &p, &tableReaderPlanningInfo{ - spec: trSpec, - post: post, - desc: tabDesc, - spans: spans, - reverse: params.Reverse, - scanVisibility: colCfg.visibility, - parallelize: params.Parallelize, - estimatedRowCount: uint64(params.EstimatedRowCount), - reqOrdering: ReqOrdering(reqOrdering), - cols: cols, - colsToTableOrdrinalMap: colsToTableOrdinalMap, + spec: trSpec, + post: post, + desc: tabDesc, + spans: spans, + reverse: params.Reverse, + scanVisibility: colCfg.visibility, + parallelize: params.Parallelize, + estimatedRowCount: uint64(params.EstimatedRowCount), + reqOrdering: ReqOrdering(reqOrdering), + cols: cols, + colsToTableOrdinalMap: colsToTableOrdinalMap, + systemColumns: systemColumns, + systemColumnOrdinals: systemColumnOrdinals, }, ) @@ -628,6 +635,7 @@ func (e *distSQLSpecExecFactory) ConstructLookupJoin( onCond tree.TypedExpr, reqOrdering exec.OutputOrdering, ) (exec.Node, error) { + // TODO (rohany): Implement production of system columns by the underlying scan here. return nil, unimplemented.NewWithIssue(47473, "experimental opt-driven distsql planning: lookup join") } diff --git a/pkg/sql/exec_factory_util.go b/pkg/sql/exec_factory_util.go index 41b269f075bf..a4e381780415 100644 --- a/pkg/sql/exec_factory_util.go +++ b/pkg/sql/exec_factory_util.go @@ -365,3 +365,17 @@ func constructVirtualScan( } return n, nil } + +func collectSystemColumnsFromCfg( + colCfg *scanColumnsConfig, table *sqlbase.TableDescriptor, +) (systemColumns []sqlbase.SystemColumnKind, systemColumnOrdinals []int) { + for i, id := range colCfg.wantedColumns { + sysColKind := sqlbase.GetSystemColumnKindFromColumnID(sqlbase.ColumnID(id)) + if sysColKind != sqlbase.SystemColumnKind_NONE { + // The scan is requested to produce a system column. + systemColumns = append(systemColumns, sysColKind) + systemColumnOrdinals = append(systemColumnOrdinals, i) + } + } + return systemColumns, systemColumnOrdinals +} diff --git a/pkg/sql/execinfrapb/processors_sql.pb.go b/pkg/sql/execinfrapb/processors_sql.pb.go index 450ef63e972e..d25f0dbe3be3 100644 --- a/pkg/sql/execinfrapb/processors_sql.pb.go +++ b/pkg/sql/execinfrapb/processors_sql.pb.go @@ -64,7 +64,7 @@ func (x *ScanVisibility) UnmarshalJSON(data []byte) error { return nil } func (ScanVisibility) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_1d68febe4779e414, []int{0} + return fileDescriptor_processors_sql_8dc98b9258a161ae, []int{0} } // These mirror the aggregate functions supported by sql/parser. See @@ -187,7 +187,7 @@ func (x *AggregatorSpec_Func) UnmarshalJSON(data []byte) error { return nil } func (AggregatorSpec_Func) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_1d68febe4779e414, []int{12, 0} + return fileDescriptor_processors_sql_8dc98b9258a161ae, []int{12, 0} } type AggregatorSpec_Type int32 @@ -233,7 +233,7 @@ func (x *AggregatorSpec_Type) UnmarshalJSON(data []byte) error { return nil } func (AggregatorSpec_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_1d68febe4779e414, []int{12, 1} + return fileDescriptor_processors_sql_8dc98b9258a161ae, []int{12, 1} } type WindowerSpec_WindowFunc int32 @@ -297,7 +297,7 @@ func (x *WindowerSpec_WindowFunc) UnmarshalJSON(data []byte) error { return nil } func (WindowerSpec_WindowFunc) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_1d68febe4779e414, []int{15, 0} + return fileDescriptor_processors_sql_8dc98b9258a161ae, []int{15, 0} } // Mode indicates which mode of framing is used. @@ -341,7 +341,7 @@ func (x *WindowerSpec_Frame_Mode) UnmarshalJSON(data []byte) error { return nil } func (WindowerSpec_Frame_Mode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_1d68febe4779e414, []int{15, 1, 0} + return fileDescriptor_processors_sql_8dc98b9258a161ae, []int{15, 1, 0} } // BoundType indicates which type of boundary is used. @@ -388,7 +388,7 @@ func (x *WindowerSpec_Frame_BoundType) UnmarshalJSON(data []byte) error { return nil } func (WindowerSpec_Frame_BoundType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_1d68febe4779e414, []int{15, 1, 1} + return fileDescriptor_processors_sql_8dc98b9258a161ae, []int{15, 1, 1} } // Exclusion specifies the type of frame exclusion. @@ -431,7 +431,7 @@ func (x *WindowerSpec_Frame_Exclusion) UnmarshalJSON(data []byte) error { return nil } func (WindowerSpec_Frame_Exclusion) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_1d68febe4779e414, []int{15, 1, 2} + return fileDescriptor_processors_sql_8dc98b9258a161ae, []int{15, 1, 2} } // ValuesCoreSpec is the core of a processor that has no inputs and generates @@ -451,7 +451,7 @@ func (m *ValuesCoreSpec) Reset() { *m = ValuesCoreSpec{} } func (m *ValuesCoreSpec) String() string { return proto.CompactTextString(m) } func (*ValuesCoreSpec) ProtoMessage() {} func (*ValuesCoreSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_1d68febe4779e414, []int{0} + return fileDescriptor_processors_sql_8dc98b9258a161ae, []int{0} } func (m *ValuesCoreSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -552,13 +552,17 @@ type TableReaderSpec struct { // makes it out of the SQL optimizer without throwing an error. If/when other // wait policies are supported, this field will be plumbed further. LockingWaitPolicy sqlbase.ScanLockingWaitPolicy `protobuf:"varint,11,opt,name=locking_wait_policy,json=lockingWaitPolicy,enum=cockroach.sql.sqlbase.ScanLockingWaitPolicy" json:"locking_wait_policy"` + // Indicates what implicit system columns this TableReader is expected to + // synthesize. These system columns will be placed at the end of the row + // output by the TableReader, in the order specified. + SystemColumns []sqlbase.SystemColumnKind `protobuf:"varint,13,rep,name=system_columns,json=systemColumns,enum=cockroach.sql.sqlbase.SystemColumnKind" json:"system_columns,omitempty"` } func (m *TableReaderSpec) Reset() { *m = TableReaderSpec{} } func (m *TableReaderSpec) String() string { return proto.CompactTextString(m) } func (*TableReaderSpec) ProtoMessage() {} func (*TableReaderSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_1d68febe4779e414, []int{1} + return fileDescriptor_processors_sql_8dc98b9258a161ae, []int{1} } func (m *TableReaderSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -616,7 +620,7 @@ func (m *IndexSkipTableReaderSpec) Reset() { *m = IndexSkipTableReaderSp func (m *IndexSkipTableReaderSpec) String() string { return proto.CompactTextString(m) } func (*IndexSkipTableReaderSpec) ProtoMessage() {} func (*IndexSkipTableReaderSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_1d68febe4779e414, []int{2} + return fileDescriptor_processors_sql_8dc98b9258a161ae, []int{2} } func (m *IndexSkipTableReaderSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -712,13 +716,21 @@ type JoinReaderSpec struct { // stream. This is only applicable to lookup joins, where doing so is // expensive. Index joins do this by default regardless of the parameter. MaintainOrdering bool `protobuf:"varint,11,opt,name=maintain_ordering,json=maintainOrdering" json:"maintain_ordering"` + // Indicates what implicit system columns this JoinReader is expected to + // synthesize. These system columns will be placed at the end of the row + // output by the JoinReader, in the order specified. + // + // This is only used in the special case of index joins, where the final + // result of the secondary index joined against the primary index is + // expected to contain the materialized system columns. + SystemColumns []sqlbase.SystemColumnKind `protobuf:"varint,12,rep,name=system_columns,json=systemColumns,enum=cockroach.sql.sqlbase.SystemColumnKind" json:"system_columns,omitempty"` } func (m *JoinReaderSpec) Reset() { *m = JoinReaderSpec{} } func (m *JoinReaderSpec) String() string { return proto.CompactTextString(m) } func (*JoinReaderSpec) ProtoMessage() {} func (*JoinReaderSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_1d68febe4779e414, []int{3} + return fileDescriptor_processors_sql_8dc98b9258a161ae, []int{3} } func (m *JoinReaderSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -762,7 +774,7 @@ func (m *SorterSpec) Reset() { *m = SorterSpec{} } func (m *SorterSpec) String() string { return proto.CompactTextString(m) } func (*SorterSpec) ProtoMessage() {} func (*SorterSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_1d68febe4779e414, []int{4} + return fileDescriptor_processors_sql_8dc98b9258a161ae, []int{4} } func (m *SorterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -824,7 +836,7 @@ func (m *DistinctSpec) Reset() { *m = DistinctSpec{} } func (m *DistinctSpec) String() string { return proto.CompactTextString(m) } func (*DistinctSpec) ProtoMessage() {} func (*DistinctSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_1d68febe4779e414, []int{5} + return fileDescriptor_processors_sql_8dc98b9258a161ae, []int{5} } func (m *DistinctSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -859,7 +871,7 @@ func (m *OrdinalitySpec) Reset() { *m = OrdinalitySpec{} } func (m *OrdinalitySpec) String() string { return proto.CompactTextString(m) } func (*OrdinalitySpec) ProtoMessage() {} func (*OrdinalitySpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_1d68febe4779e414, []int{6} + return fileDescriptor_processors_sql_8dc98b9258a161ae, []int{6} } func (m *OrdinalitySpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -917,7 +929,7 @@ func (m *ZigzagJoinerSpec) Reset() { *m = ZigzagJoinerSpec{} } func (m *ZigzagJoinerSpec) String() string { return proto.CompactTextString(m) } func (*ZigzagJoinerSpec) ProtoMessage() {} func (*ZigzagJoinerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_1d68febe4779e414, []int{7} + return fileDescriptor_processors_sql_8dc98b9258a161ae, []int{7} } func (m *ZigzagJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -993,7 +1005,7 @@ func (m *MergeJoinerSpec) Reset() { *m = MergeJoinerSpec{} } func (m *MergeJoinerSpec) String() string { return proto.CompactTextString(m) } func (*MergeJoinerSpec) ProtoMessage() {} func (*MergeJoinerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_1d68febe4779e414, []int{8} + return fileDescriptor_processors_sql_8dc98b9258a161ae, []int{8} } func (m *MergeJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1080,7 +1092,7 @@ func (m *HashJoinerSpec) Reset() { *m = HashJoinerSpec{} } func (m *HashJoinerSpec) String() string { return proto.CompactTextString(m) } func (*HashJoinerSpec) ProtoMessage() {} func (*HashJoinerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_1d68febe4779e414, []int{9} + return fileDescriptor_processors_sql_8dc98b9258a161ae, []int{9} } func (m *HashJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1177,7 +1189,7 @@ func (m *InvertedJoinerSpec) Reset() { *m = InvertedJoinerSpec{} } func (m *InvertedJoinerSpec) String() string { return proto.CompactTextString(m) } func (*InvertedJoinerSpec) ProtoMessage() {} func (*InvertedJoinerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_1d68febe4779e414, []int{10} + return fileDescriptor_processors_sql_8dc98b9258a161ae, []int{10} } func (m *InvertedJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1230,7 +1242,7 @@ func (m *InvertedFiltererSpec) Reset() { *m = InvertedFiltererSpec{} } func (m *InvertedFiltererSpec) String() string { return proto.CompactTextString(m) } func (*InvertedFiltererSpec) ProtoMessage() {} func (*InvertedFiltererSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_1d68febe4779e414, []int{11} + return fileDescriptor_processors_sql_8dc98b9258a161ae, []int{11} } func (m *InvertedFiltererSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1276,7 +1288,7 @@ func (m *AggregatorSpec) Reset() { *m = AggregatorSpec{} } func (m *AggregatorSpec) String() string { return proto.CompactTextString(m) } func (*AggregatorSpec) ProtoMessage() {} func (*AggregatorSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_1d68febe4779e414, []int{12} + return fileDescriptor_processors_sql_8dc98b9258a161ae, []int{12} } func (m *AggregatorSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1327,7 +1339,7 @@ func (m *AggregatorSpec_Aggregation) Reset() { *m = AggregatorSpec_Aggre func (m *AggregatorSpec_Aggregation) String() string { return proto.CompactTextString(m) } func (*AggregatorSpec_Aggregation) ProtoMessage() {} func (*AggregatorSpec_Aggregation) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_1d68febe4779e414, []int{12, 0} + return fileDescriptor_processors_sql_8dc98b9258a161ae, []int{12, 0} } func (m *AggregatorSpec_Aggregation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1402,7 +1414,7 @@ func (m *InterleavedReaderJoinerSpec) Reset() { *m = InterleavedReaderJo func (m *InterleavedReaderJoinerSpec) String() string { return proto.CompactTextString(m) } func (*InterleavedReaderJoinerSpec) ProtoMessage() {} func (*InterleavedReaderJoinerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_1d68febe4779e414, []int{13} + return fileDescriptor_processors_sql_8dc98b9258a161ae, []int{13} } func (m *InterleavedReaderJoinerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1458,7 +1470,7 @@ func (m *InterleavedReaderJoinerSpec_Table) Reset() { *m = InterleavedRe func (m *InterleavedReaderJoinerSpec_Table) String() string { return proto.CompactTextString(m) } func (*InterleavedReaderJoinerSpec_Table) ProtoMessage() {} func (*InterleavedReaderJoinerSpec_Table) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_1d68febe4779e414, []int{13, 0} + return fileDescriptor_processors_sql_8dc98b9258a161ae, []int{13, 0} } func (m *InterleavedReaderJoinerSpec_Table) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1498,7 +1510,7 @@ func (m *ProjectSetSpec) Reset() { *m = ProjectSetSpec{} } func (m *ProjectSetSpec) String() string { return proto.CompactTextString(m) } func (*ProjectSetSpec) ProtoMessage() {} func (*ProjectSetSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_1d68febe4779e414, []int{14} + return fileDescriptor_processors_sql_8dc98b9258a161ae, []int{14} } func (m *ProjectSetSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1540,7 +1552,7 @@ func (m *WindowerSpec) Reset() { *m = WindowerSpec{} } func (m *WindowerSpec) String() string { return proto.CompactTextString(m) } func (*WindowerSpec) ProtoMessage() {} func (*WindowerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_1d68febe4779e414, []int{15} + return fileDescriptor_processors_sql_8dc98b9258a161ae, []int{15} } func (m *WindowerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1576,7 +1588,7 @@ func (m *WindowerSpec_Func) Reset() { *m = WindowerSpec_Func{} } func (m *WindowerSpec_Func) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_Func) ProtoMessage() {} func (*WindowerSpec_Func) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_1d68febe4779e414, []int{15, 0} + return fileDescriptor_processors_sql_8dc98b9258a161ae, []int{15, 0} } func (m *WindowerSpec_Func) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1612,7 +1624,7 @@ func (m *WindowerSpec_Frame) Reset() { *m = WindowerSpec_Frame{} } func (m *WindowerSpec_Frame) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_Frame) ProtoMessage() {} func (*WindowerSpec_Frame) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_1d68febe4779e414, []int{15, 1} + return fileDescriptor_processors_sql_8dc98b9258a161ae, []int{15, 1} } func (m *WindowerSpec_Frame) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1653,7 +1665,7 @@ func (m *WindowerSpec_Frame_Bound) Reset() { *m = WindowerSpec_Frame_Bou func (m *WindowerSpec_Frame_Bound) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_Frame_Bound) ProtoMessage() {} func (*WindowerSpec_Frame_Bound) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_1d68febe4779e414, []int{15, 1, 0} + return fileDescriptor_processors_sql_8dc98b9258a161ae, []int{15, 1, 0} } func (m *WindowerSpec_Frame_Bound) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1689,7 +1701,7 @@ func (m *WindowerSpec_Frame_Bounds) Reset() { *m = WindowerSpec_Frame_Bo func (m *WindowerSpec_Frame_Bounds) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_Frame_Bounds) ProtoMessage() {} func (*WindowerSpec_Frame_Bounds) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_1d68febe4779e414, []int{15, 1, 1} + return fileDescriptor_processors_sql_8dc98b9258a161ae, []int{15, 1, 1} } func (m *WindowerSpec_Frame_Bounds) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1739,7 +1751,7 @@ func (m *WindowerSpec_WindowFn) Reset() { *m = WindowerSpec_WindowFn{} } func (m *WindowerSpec_WindowFn) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_WindowFn) ProtoMessage() {} func (*WindowerSpec_WindowFn) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_sql_1d68febe4779e414, []int{15, 2} + return fileDescriptor_processors_sql_8dc98b9258a161ae, []int{15, 2} } func (m *WindowerSpec_WindowFn) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1914,6 +1926,13 @@ func (m *TableReaderSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0 } i++ + if len(m.SystemColumns) > 0 { + for _, num := range m.SystemColumns { + dAtA[i] = 0x68 + i++ + i = encodeVarintProcessorsSql(dAtA, i, uint64(num)) + } + } return i, nil } @@ -2054,6 +2073,13 @@ func (m *JoinReaderSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0 } i++ + if len(m.SystemColumns) > 0 { + for _, num := range m.SystemColumns { + dAtA[i] = 0x60 + i++ + i = encodeVarintProcessorsSql(dAtA, i, uint64(num)) + } + } return i, nil } @@ -3032,6 +3058,11 @@ func (m *TableReaderSpec) Size() (n int) { n += 1 + sovProcessorsSql(uint64(m.LockingStrength)) n += 1 + sovProcessorsSql(uint64(m.LockingWaitPolicy)) n += 2 + if len(m.SystemColumns) > 0 { + for _, e := range m.SystemColumns { + n += 1 + sovProcessorsSql(uint64(e)) + } + } return n } @@ -3081,6 +3112,11 @@ func (m *JoinReaderSpec) Size() (n int) { n += 1 + sovProcessorsSql(uint64(m.LockingStrength)) n += 1 + sovProcessorsSql(uint64(m.LockingWaitPolicy)) n += 2 + if len(m.SystemColumns) > 0 { + for _, e := range m.SystemColumns { + n += 1 + sovProcessorsSql(uint64(e)) + } + } return n } @@ -3898,6 +3934,72 @@ func (m *TableReaderSpec) Unmarshal(dAtA []byte) error { } } m.Parallelize = bool(v != 0) + case 13: + if wireType == 0 { + var v sqlbase.SystemColumnKind + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsSql + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (sqlbase.SystemColumnKind(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.SystemColumns = append(m.SystemColumns, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsSql + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthProcessorsSql + } + postIndex := iNdEx + packedLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + if elementCount != 0 && len(m.SystemColumns) == 0 { + m.SystemColumns = make([]sqlbase.SystemColumnKind, 0, elementCount) + } + for iNdEx < postIndex { + var v sqlbase.SystemColumnKind + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsSql + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (sqlbase.SystemColumnKind(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.SystemColumns = append(m.SystemColumns, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field SystemColumns", wireType) + } default: iNdEx = preIndex skippy, err := skipProcessorsSql(dAtA[iNdEx:]) @@ -4423,6 +4525,72 @@ func (m *JoinReaderSpec) Unmarshal(dAtA []byte) error { } } m.MaintainOrdering = bool(v != 0) + case 12: + if wireType == 0 { + var v sqlbase.SystemColumnKind + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsSql + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (sqlbase.SystemColumnKind(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.SystemColumns = append(m.SystemColumns, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsSql + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthProcessorsSql + } + postIndex := iNdEx + packedLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + if elementCount != 0 && len(m.SystemColumns) == 0 { + m.SystemColumns = make([]sqlbase.SystemColumnKind, 0, elementCount) + } + for iNdEx < postIndex { + var v sqlbase.SystemColumnKind + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsSql + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (sqlbase.SystemColumnKind(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.SystemColumns = append(m.SystemColumns, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field SystemColumns", wireType) + } default: iNdEx = preIndex skippy, err := skipProcessorsSql(dAtA[iNdEx:]) @@ -7948,176 +8116,178 @@ var ( ) func init() { - proto.RegisterFile("sql/execinfrapb/processors_sql.proto", fileDescriptor_processors_sql_1d68febe4779e414) -} - -var fileDescriptor_processors_sql_1d68febe4779e414 = []byte{ - // 2663 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x5a, 0xcd, 0x73, 0xdb, 0xc6, - 0x15, 0x17, 0xf8, 0x21, 0x91, 0x8f, 0x1f, 0x5a, 0xaf, 0x95, 0x98, 0xa1, 0x53, 0x59, 0x66, 0xd2, - 0x44, 0x76, 0x1c, 0xa9, 0x51, 0x3b, 0xe9, 0x24, 0xe9, 0x21, 0xfc, 0x00, 0x65, 0xca, 0x14, 0x40, - 0x83, 0xa4, 0x9c, 0x8f, 0x99, 0x60, 0x20, 0x72, 0x45, 0x21, 0x06, 0x01, 0x0a, 0x00, 0x2d, 0x29, - 0xff, 0x40, 0x4f, 0x9d, 0x69, 0xa7, 0x97, 0x9e, 0x3a, 0xe9, 0xa1, 0xed, 0x3f, 0xd0, 0x73, 0x7a, - 0xc9, 0xc1, 0xc7, 0x9c, 0xda, 0x9c, 0x3a, 0x8d, 0xfd, 0x47, 0x64, 0x7a, 0xeb, 0xec, 0x07, 0x20, - 0x90, 0x23, 0xba, 0xa6, 0xad, 0xc4, 0xd3, 0x8b, 0x86, 0xfb, 0xde, 0xbe, 0xdf, 0xbe, 0xaf, 0x7d, - 0xfb, 0x76, 0x21, 0x78, 0xdd, 0x3b, 0xb2, 0x36, 0xc9, 0x09, 0xe9, 0x99, 0xf6, 0x81, 0x6b, 0x8c, - 0xf6, 0x37, 0x47, 0xae, 0xd3, 0x23, 0x9e, 0xe7, 0xb8, 0x9e, 0xee, 0x1d, 0x59, 0x1b, 0x23, 0xd7, - 0xf1, 0x1d, 0x5c, 0xe8, 0x39, 0xbd, 0xfb, 0xae, 0x63, 0xf4, 0x0e, 0x37, 0x28, 0xb1, 0x6f, 0x7a, - 0xbe, 0x77, 0x64, 0xb9, 0x63, 0xbb, 0xf8, 0x2a, 0x95, 0xf7, 0x8e, 0xac, 0x7d, 0xc3, 0x23, 0x9b, - 0x9e, 0xef, 0x8e, 0x7b, 0xfe, 0xd8, 0x25, 0x7d, 0x2e, 0x57, 0xbc, 0x1a, 0xe5, 0x7e, 0xee, 0x98, - 0xb6, 0xee, 0x9f, 0x8e, 0x88, 0x60, 0xbe, 0x12, 0x65, 0x5a, 0x4e, 0xef, 0xbe, 0x69, 0x0f, 0x04, - 0xeb, 0x25, 0xca, 0xa2, 0x53, 0x3d, 0xfe, 0x57, 0x90, 0x8b, 0xd3, 0xca, 0xf6, 0x0d, 0xdf, 0x10, - 0xbc, 0x9f, 0x3e, 0xc1, 0x10, 0xba, 0x82, 0x98, 0x76, 0x93, 0x4e, 0x73, 0x46, 0xfe, 0xa6, 0x69, - 0x3f, 0x20, 0xae, 0x4f, 0xfa, 0xe4, 0x64, 0xe4, 0x6e, 0x7a, 0x23, 0xc3, 0xd6, 0xe9, 0x2f, 0xe2, - 0x79, 0xa6, 0x63, 0x8b, 0xb9, 0x2b, 0x03, 0x67, 0xe0, 0xb0, 0x9f, 0x9b, 0xf4, 0x17, 0xa7, 0x96, - 0x7e, 0x2f, 0x41, 0x7e, 0xcf, 0xb0, 0xc6, 0xc4, 0xab, 0x3a, 0x2e, 0x69, 0x8f, 0x48, 0x0f, 0x57, - 0x61, 0xa9, 0xe7, 0x58, 0xe3, 0xa1, 0xed, 0x15, 0xa4, 0xb5, 0xf8, 0x7a, 0x66, 0xeb, 0xb5, 0x8d, - 0x59, 0x0e, 0xdb, 0xa8, 0x19, 0xfe, 0x78, 0xd8, 0xb0, 0x0f, 0x9c, 0x4a, 0xe2, 0xe1, 0xbf, 0xae, - 0x2d, 0x68, 0x81, 0x24, 0xbe, 0x0a, 0x69, 0xd7, 0x38, 0xd6, 0xf7, 0x4f, 0x7d, 0xe2, 0x15, 0x62, - 0x6b, 0xf1, 0xf5, 0xac, 0x96, 0x72, 0x8d, 0xe3, 0x0a, 0x1d, 0xe3, 0x6b, 0x90, 0xb2, 0xc7, 0x43, - 0xdd, 0x75, 0x8e, 0xbd, 0x42, 0x7c, 0x4d, 0x5a, 0x4f, 0x04, 0xd2, 0xf6, 0x78, 0xa8, 0x39, 0xc7, - 0x5e, 0xe9, 0x4f, 0x49, 0x58, 0xee, 0x18, 0xfb, 0x16, 0xd1, 0x88, 0xd1, 0x27, 0x2e, 0x53, 0xab, - 0x02, 0x49, 0x9f, 0x92, 0x0a, 0xd2, 0x9a, 0xb4, 0x9e, 0xd9, 0x7a, 0x63, 0x4a, 0x29, 0xe1, 0xfa, - 0x0d, 0x26, 0x56, 0x23, 0x5e, 0xcf, 0x35, 0x47, 0xbe, 0xe3, 0x0a, 0x64, 0x2e, 0x8a, 0xaf, 0x43, - 0xda, 0xb4, 0xfb, 0xe4, 0x44, 0x37, 0xfb, 0x27, 0x85, 0xd8, 0x9a, 0xb4, 0x9e, 0x13, 0xfc, 0x14, - 0x23, 0x37, 0xfa, 0x27, 0x78, 0x15, 0x96, 0x5c, 0xf2, 0x80, 0xb8, 0x1e, 0x61, 0xaa, 0xa5, 0x02, - 0xd5, 0x04, 0x11, 0xcb, 0x90, 0xa4, 0xfe, 0xf5, 0x0a, 0x09, 0xe6, 0x9b, 0x1b, 0xb3, 0x7d, 0x33, - 0x61, 0x80, 0x61, 0x07, 0x9a, 0x30, 0x69, 0xfc, 0x1a, 0x80, 0x65, 0x0e, 0x4d, 0x5f, 0x3f, 0x34, - 0x6d, 0xbf, 0x90, 0x5c, 0x93, 0xd6, 0xe3, 0x62, 0x42, 0x9a, 0xd1, 0x6f, 0x9b, 0xb6, 0x4f, 0xfd, - 0x64, 0x7a, 0x7a, 0xef, 0x90, 0xf4, 0xee, 0x17, 0x16, 0xa3, 0xca, 0x98, 0x5e, 0x95, 0x12, 0xb1, - 0x02, 0xf0, 0xc0, 0xf4, 0xcc, 0x7d, 0xd3, 0x32, 0xfd, 0xd3, 0xc2, 0xd2, 0x9a, 0xb4, 0x9e, 0xdf, - 0x5a, 0x9f, 0xad, 0x51, 0xbb, 0x67, 0xd8, 0x7b, 0xe1, 0x7c, 0x01, 0x16, 0x41, 0xc0, 0x1f, 0xc0, - 0x95, 0xa1, 0x71, 0xa2, 0xfb, 0xe6, 0x90, 0x78, 0xbe, 0x31, 0x1c, 0xe9, 0xc6, 0x80, 0xe8, 0xb6, - 0x61, 0x3b, 0x5e, 0x21, 0x1d, 0x89, 0xd3, 0xca, 0xd0, 0x38, 0xe9, 0x04, 0x73, 0xca, 0x03, 0xa2, - 0xd0, 0x19, 0xf8, 0x53, 0x40, 0x22, 0xef, 0x75, 0xcf, 0x77, 0x89, 0x3d, 0xf0, 0x0f, 0x0b, 0xc0, - 0x54, 0xba, 0x39, 0x23, 0x56, 0x54, 0x9f, 0x26, 0x17, 0x69, 0x0b, 0x09, 0xb1, 0xc2, 0xb2, 0x35, - 0x49, 0xc6, 0xfb, 0x70, 0x39, 0x00, 0x3f, 0x36, 0x4c, 0x5f, 0x1f, 0x39, 0x96, 0xd9, 0x3b, 0x2d, - 0x64, 0x18, 0xfe, 0xad, 0xff, 0x8d, 0x7f, 0xcf, 0x30, 0xfd, 0x16, 0x93, 0x11, 0x2b, 0x5c, 0xb2, - 0xa6, 0x19, 0xf8, 0x0d, 0xc8, 0x8c, 0x0c, 0xd7, 0xb0, 0x2c, 0x62, 0x99, 0x5f, 0x90, 0x42, 0x36, - 0xe2, 0xf1, 0x28, 0x63, 0x27, 0x91, 0x4a, 0xa1, 0x74, 0xe9, 0x3f, 0x71, 0x28, 0x34, 0x68, 0xd6, - 0xb4, 0xef, 0x9b, 0xa3, 0x17, 0x94, 0xac, 0x61, 0x32, 0xc6, 0x9f, 0x2b, 0x19, 0x27, 0xd3, 0x28, - 0xf1, 0xdc, 0x69, 0x14, 0xd9, 0x43, 0xc9, 0xf3, 0xf6, 0xd0, 0x79, 0x99, 0xb2, 0xf8, 0x03, 0x67, - 0xca, 0xd2, 0x05, 0x66, 0x4a, 0xe9, 0xaf, 0x49, 0xc8, 0xef, 0x38, 0xa6, 0xfd, 0xe3, 0x47, 0xfc, - 0x06, 0xe4, 0x2d, 0xc7, 0xb9, 0x3f, 0x1e, 0xe9, 0x41, 0x8d, 0xa6, 0xa1, 0xcf, 0x55, 0x62, 0x48, - 0xd2, 0x72, 0x9c, 0x53, 0x15, 0x25, 0xb8, 0x0a, 0x4b, 0x0e, 0x3f, 0x07, 0x58, 0x48, 0x33, 0x5b, - 0xaf, 0xcf, 0x0e, 0xa9, 0x1c, 0x9e, 0x16, 0x62, 0xc5, 0x45, 0xc7, 0xa6, 0x34, 0xfc, 0x1e, 0x24, - 0xe8, 0x99, 0x25, 0xc2, 0x73, 0x6d, 0x86, 0x55, 0xd4, 0x17, 0x9d, 0xd3, 0x11, 0x11, 0xc2, 0x4c, - 0xe4, 0xc2, 0x8b, 0xd3, 0x7b, 0xf0, 0xf2, 0xa4, 0xe9, 0xba, 0xe1, 0x12, 0xfd, 0x3e, 0x39, 0x2d, - 0xa4, 0x22, 0x49, 0x76, 0x79, 0xc2, 0x09, 0x65, 0x97, 0xdc, 0x21, 0xa7, 0xe7, 0x26, 0x5c, 0xfa, - 0x07, 0x4e, 0x38, 0xb8, 0xc8, 0xd2, 0xf4, 0x0e, 0x5c, 0x1a, 0x1a, 0xa6, 0xed, 0x1b, 0xa6, 0xad, - 0x3b, 0x6e, 0x9f, 0xb8, 0xa6, 0x3d, 0x60, 0xc5, 0x2f, 0x30, 0x1b, 0x05, 0x6c, 0x55, 0x70, 0x77, - 0x12, 0xa9, 0x24, 0x5a, 0xa4, 0xe7, 0x3b, 0xb4, 0x1d, 0xd7, 0x17, 0x59, 0x7a, 0x17, 0x96, 0x9d, - 0xb1, 0x3f, 0x1a, 0xfb, 0x67, 0x28, 0x3c, 0x5f, 0x4b, 0xb3, 0x03, 0x13, 0x20, 0x8a, 0x95, 0xf2, - 0x1c, 0x20, 0xa0, 0xe2, 0x2d, 0xc0, 0x01, 0x96, 0x3e, 0x34, 0xfc, 0xde, 0xa1, 0x6e, 0x11, 0x7b, - 0x22, 0x7b, 0x51, 0xc0, 0xdf, 0xa5, 0xec, 0x26, 0xb1, 0x4b, 0x5f, 0x49, 0x90, 0xad, 0x99, 0x9e, - 0x6f, 0xda, 0x3d, 0x9f, 0xe9, 0xf5, 0x26, 0x2c, 0xb3, 0x49, 0xa4, 0xaf, 0x47, 0x7b, 0x8f, 0x9c, - 0x96, 0x17, 0xe4, 0x20, 0xa9, 0x6f, 0x00, 0xea, 0x0b, 0xc1, 0x70, 0x66, 0x8c, 0xcd, 0x5c, 0x0e, - 0xe8, 0xc1, 0xd4, 0x2d, 0xc0, 0xf6, 0xd8, 0xb2, 0x78, 0x9a, 0x04, 0xcc, 0x89, 0x43, 0x1d, 0x31, - 0x7e, 0xd9, 0x25, 0x81, 0x2e, 0xf8, 0x0d, 0xc8, 0x12, 0xd7, 0x75, 0x5c, 0xdd, 0xb1, 0xf5, 0xfe, - 0x78, 0xc4, 0x36, 0x4e, 0x3a, 0xc8, 0x45, 0xc6, 0x51, 0xed, 0xda, 0x78, 0x54, 0x42, 0x90, 0x57, - 0xdd, 0xbe, 0x69, 0x1b, 0x34, 0x33, 0xa9, 0x05, 0xa5, 0x3f, 0xc4, 0x01, 0x7d, 0x62, 0x0e, 0xbe, - 0x30, 0x06, 0x74, 0x33, 0x08, 0x77, 0xd7, 0x60, 0x91, 0xed, 0xec, 0xa0, 0x93, 0x9a, 0xaf, 0x2a, - 0x08, 0x59, 0x5c, 0x07, 0x20, 0x47, 0x13, 0xd6, 0x66, 0xb6, 0xae, 0xcf, 0x8e, 0x97, 0xb0, 0x3f, - 0x68, 0x27, 0xc8, 0xd1, 0x99, 0xef, 0xf2, 0xbc, 0xbc, 0x38, 0x5c, 0xf5, 0x89, 0xda, 0xc1, 0x38, - 0xc2, 0xa6, 0x0b, 0xaa, 0x1d, 0x77, 0x20, 0x7b, 0x60, 0x9e, 0x90, 0xbe, 0xfe, 0x80, 0x35, 0x98, - 0x85, 0x24, 0xd3, 0xfc, 0x09, 0x25, 0x60, 0xb2, 0x11, 0xd5, 0x32, 0x4c, 0x9a, 0x13, 0x9f, 0xa3, - 0x10, 0x95, 0xfe, 0x11, 0x87, 0xe5, 0x5d, 0xe2, 0x0e, 0x48, 0x24, 0x32, 0xbb, 0x90, 0xb3, 0xc8, - 0xc1, 0x73, 0x6c, 0x83, 0x2c, 0x15, 0x0f, 0x37, 0x81, 0x0a, 0x79, 0xd7, 0x1c, 0x1c, 0x46, 0xf0, - 0x62, 0x73, 0xe2, 0xe5, 0x98, 0x7c, 0x08, 0x18, 0x09, 0x40, 0xf2, 0x45, 0x14, 0xef, 0x1b, 0x90, - 0xa3, 0x9b, 0x43, 0x27, 0x47, 0x63, 0x23, 0xac, 0xdf, 0xc1, 0xbe, 0xc9, 0x52, 0x96, 0x2c, 0x38, - 0xf8, 0x7d, 0xb8, 0xc2, 0x5c, 0x79, 0x96, 0xa3, 0x33, 0x0a, 0x33, 0x39, 0xf0, 0xe5, 0xa3, 0xc9, - 0xc2, 0xfc, 0x2b, 0x28, 0x70, 0xbf, 0x9d, 0x23, 0x9c, 0x8e, 0x08, 0xaf, 0xb0, 0x59, 0x53, 0xd2, - 0xa5, 0xdf, 0xc5, 0x21, 0x7f, 0xdb, 0xf0, 0x0e, 0x23, 0x71, 0xbd, 0x09, 0xcb, 0x53, 0xca, 0xf0, - 0x42, 0x22, 0x0e, 0xc8, 0xa8, 0x0a, 0xf8, 0x16, 0xa0, 0xe9, 0xc5, 0x79, 0x2d, 0x61, 0x93, 0xf3, - 0x93, 0x4b, 0xbe, 0xf0, 0x88, 0xbc, 0x05, 0xf9, 0x21, 0x4d, 0xe2, 0xb3, 0x0a, 0x19, 0x0d, 0x49, - 0x8e, 0xf3, 0x02, 0x65, 0x5f, 0x5c, 0x4c, 0xbe, 0x8f, 0x01, 0x6e, 0x88, 0xdb, 0x68, 0x24, 0x2e, - 0x3f, 0x5a, 0x7b, 0x94, 0x9b, 0xe8, 0x11, 0x58, 0xb9, 0xcf, 0x85, 0x5b, 0x36, 0xd2, 0x1a, 0x60, - 0x15, 0x72, 0xc1, 0xad, 0xf9, 0x59, 0x0b, 0x5d, 0x36, 0x00, 0x60, 0xb1, 0x7d, 0xc1, 0x09, 0x52, - 0xfa, 0x8b, 0x04, 0x2b, 0x81, 0xe7, 0xeb, 0xa6, 0xe5, 0x13, 0x57, 0xf8, 0x7e, 0x03, 0x50, 0x68, - 0x69, 0xcf, 0xb1, 0x98, 0xfb, 0xa4, 0x88, 0x5f, 0xf2, 0x01, 0xb7, 0xea, 0x58, 0xd4, 0x89, 0x9f, - 0x4d, 0x7b, 0x86, 0xd7, 0xb2, 0x9f, 0x4f, 0x29, 0xe3, 0x8c, 0xfc, 0x8d, 0xe8, 0xbb, 0xc3, 0x06, - 0xbd, 0x58, 0x9c, 0x19, 0xd6, 0x72, 0x1d, 0xdf, 0x39, 0xcf, 0x51, 0xa5, 0xdf, 0xa4, 0x20, 0x5f, - 0x1e, 0x0c, 0x5c, 0x32, 0x30, 0x7c, 0x87, 0xab, 0x78, 0x1d, 0x60, 0xe0, 0x3a, 0x3c, 0x6c, 0xd1, - 0x4d, 0x98, 0x66, 0xd4, 0xaa, 0x63, 0x79, 0xf8, 0x33, 0xc8, 0x1a, 0x42, 0xc8, 0x74, 0xc2, 0x2b, - 0xcf, 0x2f, 0x66, 0xfb, 0x78, 0x72, 0x89, 0x70, 0x18, 0x09, 0x5f, 0x14, 0x0f, 0xff, 0x4c, 0xf4, - 0x31, 0xa4, 0xaf, 0x47, 0x54, 0x49, 0x84, 0xaa, 0x20, 0xc1, 0xdd, 0x0e, 0x35, 0xda, 0x16, 0xb1, - 0x4a, 0xb2, 0x58, 0xbd, 0xfd, 0xd4, 0x9a, 0x4c, 0x47, 0xae, 0xf8, 0xeb, 0x18, 0x64, 0x22, 0xea, - 0x51, 0xe0, 0x83, 0xb1, 0xdd, 0x63, 0x41, 0x9a, 0x07, 0xb8, 0x3e, 0xb6, 0x7b, 0x01, 0x30, 0x05, - 0xc0, 0x6b, 0x90, 0x0a, 0x1b, 0x9f, 0x58, 0x64, 0xeb, 0x86, 0x54, 0xfc, 0x3a, 0xe4, 0x0f, 0x58, - 0xae, 0x84, 0x99, 0x41, 0xb7, 0x41, 0x4e, 0xcb, 0x72, 0xaa, 0xc8, 0x88, 0x2b, 0xec, 0x49, 0x88, - 0xb1, 0x93, 0xac, 0xd9, 0x5a, 0xec, 0x71, 0xc6, 0x6d, 0x48, 0x1b, 0xee, 0x60, 0x3c, 0x24, 0xb6, - 0xef, 0x15, 0x16, 0x59, 0x44, 0xe6, 0xc9, 0xfa, 0x33, 0xe1, 0x9d, 0x44, 0x2a, 0x8e, 0x12, 0xa5, - 0xaf, 0xe3, 0x90, 0xa0, 0x56, 0x60, 0x04, 0xd9, 0xb2, 0xf2, 0xb1, 0xae, 0xa8, 0x1d, 0x5d, 0xe9, - 0x36, 0x9b, 0x68, 0x01, 0x2f, 0x41, 0xbc, 0xbc, 0xb7, 0x8d, 0x24, 0x9c, 0x85, 0x54, 0x45, 0x55, - 0x9b, 0x7a, 0x59, 0xa9, 0xa1, 0x18, 0xce, 0xc0, 0x12, 0x1b, 0xa9, 0x1a, 0x8a, 0xe3, 0x3c, 0x40, - 0x55, 0x55, 0xaa, 0xe5, 0x8e, 0x5e, 0xde, 0xde, 0x46, 0x09, 0x9c, 0x86, 0x64, 0x55, 0xed, 0x2a, - 0x1d, 0x94, 0xa4, 0xe2, 0xbb, 0xe5, 0x8f, 0xd0, 0x12, 0xfb, 0xd1, 0x50, 0x50, 0x0a, 0x03, 0x2c, - 0xb6, 0x3b, 0xb5, 0x9a, 0xbc, 0x87, 0xd2, 0x94, 0xd8, 0xee, 0xee, 0x22, 0xa0, 0x70, 0xed, 0xee, - 0xae, 0xde, 0x50, 0x3a, 0x28, 0x43, 0x57, 0xda, 0x2b, 0x6b, 0x8d, 0xb2, 0x52, 0x95, 0x51, 0x96, - 0xb2, 0x3e, 0x52, 0x35, 0x86, 0x9c, 0xe3, 0x2b, 0x75, 0x95, 0x8e, 0xae, 0xa9, 0xf7, 0xda, 0x28, - 0xcf, 0xe4, 0xee, 0x6a, 0xb5, 0x46, 0xbd, 0x8e, 0x96, 0x31, 0x86, 0x7c, 0xbd, 0xa1, 0x94, 0x9b, - 0x7a, 0x28, 0x8d, 0xa8, 0x41, 0x9c, 0x26, 0xd6, 0xbc, 0x84, 0x73, 0x90, 0x2e, 0x6b, 0x5a, 0xf9, - 0x63, 0x86, 0x88, 0xe9, 0x62, 0x3b, 0x6d, 0x55, 0x61, 0xa3, 0xcb, 0x94, 0x49, 0x47, 0x15, 0x36, - 0x5c, 0xa1, 0xcb, 0xb5, 0x3b, 0x5a, 0x43, 0xd9, 0x66, 0xe3, 0x97, 0x98, 0xd5, 0x8d, 0x0e, 0x73, - 0xc1, 0xcb, 0xd4, 0x10, 0x3a, 0x50, 0x35, 0x74, 0x05, 0xa7, 0x20, 0x51, 0x55, 0x35, 0x0d, 0x15, - 0x70, 0x01, 0x56, 0x5a, 0xb2, 0x56, 0x95, 0x95, 0x4e, 0xa3, 0x29, 0xeb, 0xb5, 0x46, 0xbb, 0xaa, - 0x37, 0x76, 0x5b, 0x4d, 0xf4, 0xca, 0x14, 0xa7, 0xaa, 0x2a, 0x1d, 0xce, 0x29, 0xe2, 0xcb, 0xb0, - 0xcc, 0x74, 0x50, 0x2b, 0x3b, 0x72, 0x95, 0x3b, 0xf1, 0x2a, 0x5e, 0x01, 0xc4, 0x55, 0x89, 0x50, - 0x5f, 0xa5, 0x1a, 0xec, 0x95, 0x35, 0xbd, 0xa5, 0xb6, 0xd0, 0x4f, 0xb8, 0x7a, 0xd4, 0x2c, 0x36, - 0x5e, 0x2d, 0xdd, 0x82, 0x04, 0x4d, 0x72, 0xaa, 0x4d, 0xb9, 0xdb, 0x51, 0xd1, 0x02, 0x73, 0x76, - 0xb5, 0xdc, 0x2c, 0x6b, 0x48, 0xa2, 0xb3, 0x15, 0x55, 0xd1, 0xc5, 0x38, 0x56, 0xfa, 0xfb, 0x22, - 0x5c, 0x6d, 0xd8, 0x3e, 0x71, 0x2d, 0x62, 0x3c, 0x20, 0x7d, 0x7e, 0xa9, 0x8e, 0x9c, 0x1d, 0x1f, - 0x4f, 0x75, 0xd1, 0x1f, 0xcc, 0xce, 0xb0, 0x27, 0xc0, 0xf0, 0xb3, 0x65, 0xaa, 0xb5, 0x8e, 0xbc, - 0x54, 0xc4, 0xce, 0x7b, 0xa9, 0x98, 0x7c, 0xa6, 0x8b, 0x9f, 0xff, 0x4c, 0x77, 0xa1, 0x17, 0xed, - 0xe4, 0xfc, 0x9d, 0xc1, 0xff, 0xfb, 0x73, 0x4a, 0xf1, 0xab, 0x18, 0x24, 0x59, 0x74, 0xf0, 0x87, - 0x90, 0xe8, 0x13, 0xaf, 0xf7, 0x4c, 0x5d, 0x02, 0x93, 0x7c, 0x9a, 0x26, 0xa1, 0x0a, 0x89, 0x91, - 0xe3, 0xf1, 0x70, 0x3e, 0xf1, 0xd1, 0xac, 0xe5, 0x78, 0x7e, 0x8b, 0x3f, 0xbc, 0xd3, 0x0c, 0x0a, - 0xd6, 0xa1, 0xc2, 0xb8, 0x06, 0xa9, 0xb0, 0xd7, 0x4f, 0xcc, 0xd9, 0xeb, 0x87, 0x92, 0x67, 0x0f, - 0x78, 0xc9, 0xe7, 0x79, 0xc0, 0x2b, 0x7d, 0x2d, 0x41, 0xbe, 0xe5, 0x3a, 0x9f, 0x93, 0x9e, 0xdf, - 0x26, 0xfc, 0x46, 0xfd, 0x21, 0x24, 0x69, 0x46, 0x06, 0x7b, 0x66, 0x9e, 0x94, 0xe4, 0x82, 0x78, - 0x1b, 0x2e, 0x0d, 0x88, 0x4d, 0x5c, 0xc3, 0x8f, 0xf4, 0x9c, 0xfc, 0xf6, 0x59, 0x9c, 0x0e, 0x0c, - 0x19, 0x6e, 0xf0, 0x4f, 0x1b, 0x1d, 0x0d, 0x85, 0x42, 0x41, 0x33, 0xfa, 0x26, 0x20, 0x7b, 0x3c, - 0x64, 0xe7, 0xa9, 0x3e, 0x22, 0xae, 0x3e, 0x20, 0x36, 0xbf, 0x79, 0x6a, 0x39, 0x7b, 0x3c, 0xa4, - 0x47, 0x69, 0x8b, 0xb8, 0xdb, 0xc4, 0x2e, 0x7d, 0x97, 0x83, 0xec, 0x3d, 0xd3, 0xee, 0x3b, 0xc7, - 0x62, 0xe7, 0xaf, 0xb1, 0x17, 0x59, 0xdf, 0x64, 0x87, 0xf6, 0xa9, 0x78, 0x12, 0x88, 0x92, 0x70, - 0x1b, 0xd2, 0xc7, 0x4c, 0xa2, 0x1e, 0x2a, 0xb7, 0x39, 0xdb, 0xd4, 0x28, 0xb8, 0x18, 0xd4, 0xc3, - 0xb3, 0x28, 0xc4, 0x29, 0xfe, 0x4d, 0x12, 0xa7, 0x50, 0x1b, 0x72, 0x41, 0x8f, 0x40, 0xea, 0xcf, - 0x7a, 0x22, 0x6b, 0x93, 0x18, 0xf8, 0x2e, 0x80, 0x58, 0x8a, 0x22, 0xc6, 0x18, 0xe2, 0x3b, 0xf3, - 0xe9, 0x4c, 0x51, 0x23, 0x20, 0xef, 0x27, 0x1e, 0x7e, 0x79, 0x4d, 0x2a, 0x7e, 0xb9, 0x04, 0xc9, - 0xba, 0x6b, 0x0c, 0x09, 0xbe, 0x03, 0x89, 0xa1, 0xd3, 0x27, 0x42, 0xdd, 0xa7, 0x05, 0x67, 0xb2, - 0x1b, 0xbb, 0x4e, 0x3f, 0x2c, 0x2f, 0x14, 0x04, 0xdf, 0x85, 0xc5, 0x7d, 0x67, 0x6c, 0xf7, 0xbd, - 0x19, 0x7d, 0xe0, 0x93, 0xe1, 0x2a, 0x4c, 0x34, 0x28, 0x76, 0x1c, 0x08, 0x7f, 0x02, 0x69, 0x72, - 0xd2, 0xb3, 0xc6, 0x34, 0xe9, 0xd8, 0x36, 0xcc, 0x6f, 0xbd, 0x3b, 0x17, 0xaa, 0x1c, 0x48, 0x87, - 0xaf, 0x1c, 0x01, 0xa1, 0xf8, 0xbd, 0x04, 0x49, 0xb6, 0x28, 0x5d, 0x85, 0xad, 0x47, 0x0b, 0xa6, - 0x70, 0xc5, 0xbb, 0xf3, 0xeb, 0x1e, 0x29, 0xb7, 0x67, 0x70, 0xf4, 0x60, 0x30, 0x6d, 0x5f, 0x77, - 0x0e, 0x0e, 0x3c, 0xc2, 0x7b, 0xab, 0xe0, 0xe3, 0x48, 0xda, 0xb4, 0x7d, 0x95, 0x91, 0xf1, 0x75, - 0xc8, 0xd2, 0x5d, 0xd1, 0x0f, 0xa6, 0x51, 0x4b, 0xb3, 0x5a, 0x86, 0xd1, 0xc4, 0x94, 0x1d, 0xc8, - 0x70, 0x26, 0xfb, 0x96, 0x28, 0x2a, 0xc9, 0x1c, 0x1f, 0xdc, 0x80, 0x4b, 0x53, 0x9d, 0x8a, 0x7f, - 0x94, 0x60, 0x91, 0xbb, 0x1b, 0x2b, 0x90, 0xf4, 0x7c, 0xc3, 0xf5, 0x45, 0x21, 0xdd, 0x9a, 0xdf, - 0xec, 0xb0, 0xc0, 0x50, 0x18, 0x5c, 0x83, 0x38, 0xb1, 0xfb, 0x22, 0x01, 0x9e, 0x01, 0x4d, 0xa3, - 0xe2, 0xa5, 0x37, 0x21, 0x41, 0xb3, 0x8b, 0xb6, 0x65, 0x5a, 0x59, 0xd9, 0x96, 0xd1, 0x02, 0xed, - 0x10, 0x58, 0x07, 0x25, 0xd1, 0x0e, 0x61, 0x5b, 0x53, 0xbb, 0xad, 0x36, 0x8a, 0x95, 0xbe, 0x80, - 0x74, 0xe8, 0x7b, 0x7c, 0x05, 0x2e, 0x77, 0x95, 0x8a, 0xda, 0x55, 0x6a, 0x72, 0x4d, 0x6f, 0x69, - 0x72, 0x55, 0xae, 0x35, 0x94, 0x6d, 0xb4, 0x30, 0xc9, 0xa8, 0xab, 0xcd, 0xa6, 0x7a, 0x8f, 0x32, - 0x24, 0xda, 0xb1, 0xa8, 0xf5, 0x7a, 0x5b, 0xee, 0x44, 0xa6, 0xc7, 0x22, 0xd4, 0xb3, 0xb9, 0x71, - 0xbc, 0x0c, 0x99, 0x6a, 0x57, 0xd3, 0x64, 0xde, 0xca, 0xa1, 0x44, 0xe9, 0x53, 0x48, 0x87, 0xd9, - 0x45, 0xbb, 0x36, 0x45, 0xd5, 0xe5, 0x8f, 0xaa, 0xcd, 0x6e, 0xbb, 0xa1, 0x2a, 0x7c, 0x51, 0x36, - 0xac, 0xc9, 0x7a, 0x54, 0x4e, 0xc2, 0x97, 0x20, 0x17, 0x30, 0x98, 0x1d, 0x28, 0x46, 0xa5, 0x03, - 0x52, 0xa7, 0x21, 0xb7, 0x51, 0xbc, 0xf8, 0xcf, 0x18, 0xa4, 0x82, 0xba, 0x83, 0xe5, 0x48, 0x9b, - 0x9f, 0xd9, 0x7a, 0xeb, 0x69, 0xbd, 0x3a, 0xdd, 0xe4, 0x5f, 0xcc, 0x49, 0x54, 0x81, 0xe4, 0x01, - 0x8d, 0x97, 0xb8, 0xbb, 0xde, 0x9a, 0x27, 0xc6, 0x1a, 0x17, 0xc5, 0xeb, 0x30, 0x71, 0x6d, 0x60, - 0x4d, 0x48, 0x32, 0xb8, 0x6c, 0x4d, 0x5c, 0x28, 0x8a, 0x90, 0x32, 0xdc, 0x81, 0xd7, 0xe8, 0x9f, - 0x78, 0x85, 0x25, 0x56, 0xd5, 0xc3, 0x31, 0x45, 0xe1, 0x4f, 0xcc, 0x02, 0x25, 0x15, 0xbd, 0xc2, - 0x47, 0x39, 0x3b, 0x89, 0x54, 0x0c, 0xc5, 0xc5, 0xcd, 0xe1, 0xcf, 0x12, 0xc0, 0x59, 0x75, 0xa4, - 0x3d, 0xa6, 0xa6, 0xde, 0xd3, 0x95, 0xee, 0x6e, 0x45, 0xd6, 0x44, 0x9e, 0x95, 0x95, 0x3b, 0xbc, - 0xfb, 0xac, 0xc9, 0x4a, 0x5b, 0xd6, 0xd9, 0x98, 0x05, 0x49, 0x74, 0xc3, 0x9c, 0x12, 0xa7, 0xbd, - 0x77, 0xb5, 0xbb, 0xcb, 0x7a, 0xe6, 0x0e, 0xbf, 0x44, 0xb0, 0x4e, 0x99, 0x5f, 0x22, 0x9a, 0xe5, - 0x6d, 0xb4, 0x48, 0xe1, 0x9a, 0x72, 0xb9, 0x86, 0x96, 0x68, 0xfe, 0xd4, 0x1b, 0x5a, 0xbb, 0xa3, - 0xef, 0x95, 0x9b, 0x5d, 0x19, 0xa5, 0x28, 0x7e, 0xb3, 0x1c, 0x8e, 0xd3, 0x14, 0x4d, 0xe9, 0xdc, - 0x16, 0x43, 0xb8, 0xf9, 0x4b, 0xc8, 0x4f, 0x7e, 0xe9, 0xa0, 0x89, 0xdf, 0xea, 0x56, 0x9a, 0x8d, - 0x2a, 0x5a, 0xc0, 0xaf, 0xc0, 0x4b, 0xfc, 0x37, 0x6d, 0xed, 0xd9, 0xed, 0x47, 0xb0, 0xa4, 0xca, - 0xdb, 0x0f, 0xbf, 0x5b, 0x5d, 0x78, 0xf8, 0x68, 0x55, 0xfa, 0xe6, 0xd1, 0xaa, 0xf4, 0xed, 0xa3, - 0x55, 0xe9, 0xdf, 0x8f, 0x56, 0xa5, 0xdf, 0x3e, 0x5e, 0x5d, 0xf8, 0xe6, 0xf1, 0xea, 0xc2, 0xb7, - 0x8f, 0x57, 0x17, 0x3e, 0xc9, 0x44, 0xfe, 0x61, 0xe0, 0xbf, 0x01, 0x00, 0x00, 0xff, 0xff, 0x40, - 0xc0, 0x61, 0xb0, 0x0b, 0x21, 0x00, 0x00, + proto.RegisterFile("sql/execinfrapb/processors_sql.proto", fileDescriptor_processors_sql_8dc98b9258a161ae) +} + +var fileDescriptor_processors_sql_8dc98b9258a161ae = []byte{ + // 2700 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x5a, 0x4b, 0x73, 0xdb, 0xd6, + 0xf5, 0x17, 0xf8, 0x90, 0xc8, 0xc3, 0x87, 0xae, 0xaf, 0x95, 0x98, 0xa1, 0xf3, 0x97, 0x65, 0x26, + 0xff, 0x58, 0x76, 0x1c, 0xa9, 0x51, 0x3b, 0xe9, 0x24, 0xe9, 0x22, 0x7c, 0x80, 0x32, 0x65, 0x0a, + 0xa0, 0x41, 0x52, 0xce, 0x63, 0x26, 0x18, 0x88, 0xbc, 0xa2, 0x10, 0x83, 0x00, 0x05, 0x80, 0x96, + 0x94, 0x2f, 0xd0, 0x55, 0x67, 0xda, 0xe9, 0xa6, 0xab, 0x4e, 0x36, 0xed, 0x27, 0xe8, 0x3a, 0xdd, + 0x64, 0xe1, 0x65, 0x56, 0x6d, 0x56, 0x9d, 0xc6, 0xf9, 0x04, 0x5d, 0x79, 0xba, 0xeb, 0xdc, 0x07, + 0x20, 0x90, 0x23, 0xba, 0xa6, 0xad, 0xc4, 0xd3, 0x8d, 0x86, 0xf7, 0xbc, 0xee, 0xb9, 0xe7, 0x9c, + 0x7b, 0xee, 0xef, 0x5e, 0x08, 0xde, 0xf4, 0x8e, 0xac, 0x4d, 0x72, 0x42, 0x7a, 0xa6, 0x7d, 0xe0, + 0x1a, 0xa3, 0xfd, 0xcd, 0x91, 0xeb, 0xf4, 0x88, 0xe7, 0x39, 0xae, 0xa7, 0x7b, 0x47, 0xd6, 0xc6, + 0xc8, 0x75, 0x7c, 0x07, 0x17, 0x7a, 0x4e, 0xef, 0x81, 0xeb, 0x18, 0xbd, 0xc3, 0x0d, 0x4a, 0xec, + 0x9b, 0x9e, 0xef, 0x1d, 0x59, 0xee, 0xd8, 0x2e, 0xbe, 0x4e, 0xf5, 0xbd, 0x23, 0x6b, 0xdf, 0xf0, + 0xc8, 0xa6, 0xe7, 0xbb, 0xe3, 0x9e, 0x3f, 0x76, 0x49, 0x9f, 0xeb, 0x15, 0xaf, 0x46, 0xb9, 0x5f, + 0x38, 0xa6, 0xad, 0xfb, 0xa7, 0x23, 0x22, 0x98, 0xaf, 0x45, 0x99, 0x96, 0xd3, 0x7b, 0x60, 0xda, + 0x03, 0xc1, 0x7a, 0x85, 0xb2, 0xa8, 0xa8, 0xc7, 0xff, 0x0a, 0x72, 0x71, 0xda, 0xd9, 0xbe, 0xe1, + 0x1b, 0x82, 0xf7, 0xff, 0x4f, 0x59, 0x08, 0x9d, 0x41, 0x88, 0xdd, 0xa2, 0x62, 0xce, 0xc8, 0xdf, + 0x34, 0xed, 0x87, 0xc4, 0xf5, 0x49, 0x9f, 0x9c, 0x8c, 0xdc, 0x4d, 0x6f, 0x64, 0xd8, 0x3a, 0xfd, + 0x45, 0x3c, 0xcf, 0x74, 0x6c, 0x21, 0xbb, 0x32, 0x70, 0x06, 0x0e, 0xfb, 0xb9, 0x49, 0x7f, 0x71, + 0x6a, 0xe9, 0xf7, 0x12, 0xe4, 0xf7, 0x0c, 0x6b, 0x4c, 0xbc, 0xaa, 0xe3, 0x92, 0xf6, 0x88, 0xf4, + 0x70, 0x15, 0x96, 0x7a, 0x8e, 0x35, 0x1e, 0xda, 0x5e, 0x41, 0x5a, 0x8b, 0xaf, 0x67, 0xb6, 0xde, + 0xd8, 0x98, 0x15, 0xb0, 0x8d, 0x9a, 0xe1, 0x8f, 0x87, 0x0d, 0xfb, 0xc0, 0xa9, 0x24, 0x1e, 0xfd, + 0xe3, 0xda, 0x82, 0x16, 0x68, 0xe2, 0xab, 0x90, 0x76, 0x8d, 0x63, 0x7d, 0xff, 0xd4, 0x27, 0x5e, + 0x21, 0xb6, 0x16, 0x5f, 0xcf, 0x6a, 0x29, 0xd7, 0x38, 0xae, 0xd0, 0x31, 0xbe, 0x06, 0x29, 0x7b, + 0x3c, 0xd4, 0x5d, 0xe7, 0xd8, 0x2b, 0xc4, 0xd7, 0xa4, 0xf5, 0x44, 0xa0, 0x6d, 0x8f, 0x87, 0x9a, + 0x73, 0xec, 0x95, 0xfe, 0x95, 0x84, 0xe5, 0x8e, 0xb1, 0x6f, 0x11, 0x8d, 0x18, 0x7d, 0xe2, 0x32, + 0xb7, 0x2a, 0x90, 0xf4, 0x29, 0xa9, 0x20, 0xad, 0x49, 0xeb, 0x99, 0xad, 0xb7, 0xa6, 0x9c, 0x12, + 0xa1, 0xdf, 0x60, 0x6a, 0x35, 0xe2, 0xf5, 0x5c, 0x73, 0xe4, 0x3b, 0xae, 0xb0, 0xcc, 0x55, 0xf1, + 0x75, 0x48, 0x9b, 0x76, 0x9f, 0x9c, 0xe8, 0x66, 0xff, 0xa4, 0x10, 0x5b, 0x93, 0xd6, 0x73, 0x82, + 0x9f, 0x62, 0xe4, 0x46, 0xff, 0x04, 0xaf, 0xc2, 0x92, 0x4b, 0x1e, 0x12, 0xd7, 0x23, 0xcc, 0xb5, + 0x54, 0xe0, 0x9a, 0x20, 0x62, 0x19, 0x92, 0x34, 0xbe, 0x5e, 0x21, 0xc1, 0x62, 0x73, 0x73, 0x76, + 0x6c, 0x26, 0x16, 0x60, 0xd8, 0x81, 0x27, 0x4c, 0x1b, 0xbf, 0x01, 0x60, 0x99, 0x43, 0xd3, 0xd7, + 0x0f, 0x4d, 0xdb, 0x2f, 0x24, 0xd7, 0xa4, 0xf5, 0xb8, 0x10, 0x48, 0x33, 0xfa, 0x1d, 0xd3, 0xf6, + 0x69, 0x9c, 0x4c, 0x4f, 0xef, 0x1d, 0x92, 0xde, 0x83, 0xc2, 0x62, 0xd4, 0x19, 0xd3, 0xab, 0x52, + 0x22, 0x56, 0x00, 0x1e, 0x9a, 0x9e, 0xb9, 0x6f, 0x5a, 0xa6, 0x7f, 0x5a, 0x58, 0x5a, 0x93, 0xd6, + 0xf3, 0x5b, 0xeb, 0xb3, 0x3d, 0x6a, 0xf7, 0x0c, 0x7b, 0x2f, 0x94, 0x17, 0xc6, 0x22, 0x16, 0xf0, + 0x87, 0x70, 0x65, 0x68, 0x9c, 0xe8, 0xbe, 0x39, 0x24, 0x9e, 0x6f, 0x0c, 0x47, 0xba, 0x31, 0x20, + 0xba, 0x6d, 0xd8, 0x8e, 0x57, 0x48, 0x47, 0xf2, 0xb4, 0x32, 0x34, 0x4e, 0x3a, 0x81, 0x4c, 0x79, + 0x40, 0x14, 0x2a, 0x81, 0x3f, 0x03, 0x24, 0xea, 0x5e, 0xf7, 0x7c, 0x97, 0xd8, 0x03, 0xff, 0xb0, + 0x00, 0xcc, 0xa5, 0x5b, 0x33, 0x72, 0x45, 0xfd, 0x69, 0x72, 0x95, 0xb6, 0xd0, 0x10, 0x33, 0x2c, + 0x5b, 0x93, 0x64, 0xbc, 0x0f, 0x97, 0x03, 0xe3, 0xc7, 0x86, 0xe9, 0xeb, 0x23, 0xc7, 0x32, 0x7b, + 0xa7, 0x85, 0x0c, 0xb3, 0x7f, 0xfb, 0xbf, 0xdb, 0xbf, 0x6f, 0x98, 0x7e, 0x8b, 0xe9, 0x88, 0x19, + 0x2e, 0x59, 0xd3, 0x0c, 0xfc, 0x16, 0x64, 0x46, 0x86, 0x6b, 0x58, 0x16, 0xb1, 0xcc, 0x2f, 0x49, + 0x21, 0x1b, 0x89, 0x78, 0x94, 0x81, 0x15, 0xc8, 0x7b, 0xa7, 0x9e, 0x4f, 0x86, 0x7a, 0xb0, 0x4f, + 0x72, 0x6b, 0xf1, 0xf5, 0xfc, 0xd6, 0x8d, 0x59, 0x6e, 0x30, 0xe1, 0x2a, 0x93, 0xbd, 0x6b, 0xda, + 0x7d, 0x2d, 0xe7, 0x45, 0x28, 0xde, 0x4e, 0x22, 0x95, 0x42, 0xe9, 0xd2, 0xbf, 0xe3, 0x50, 0x68, + 0xd0, 0x2a, 0x6c, 0x3f, 0x30, 0x47, 0x2f, 0xa9, 0xf8, 0xc3, 0xe2, 0x8e, 0xbf, 0x50, 0x71, 0x4f, + 0x96, 0x65, 0xe2, 0x85, 0xcb, 0x32, 0xb2, 0x27, 0x93, 0xe7, 0xed, 0xc9, 0xf3, 0x2a, 0x6f, 0xf1, + 0x47, 0xae, 0xbc, 0xa5, 0x0b, 0xac, 0xbc, 0xd2, 0x93, 0x24, 0xe4, 0x77, 0x1c, 0xd3, 0xfe, 0xe9, + 0x33, 0x7e, 0x13, 0xf2, 0x96, 0xe3, 0x3c, 0x18, 0x8f, 0xc2, 0x5a, 0xa6, 0xa9, 0xcf, 0x55, 0x62, + 0x48, 0xd2, 0x72, 0x9c, 0x23, 0xca, 0x94, 0x9e, 0x0b, 0x0e, 0x3f, 0x57, 0x58, 0x4a, 0x33, 0x5b, + 0x6f, 0xce, 0x4e, 0xa9, 0x1c, 0x9e, 0x3e, 0x62, 0xc6, 0x45, 0xc7, 0xa6, 0x34, 0xfc, 0x3e, 0x24, + 0xe8, 0x19, 0x28, 0xd2, 0x73, 0x6d, 0xc6, 0xaa, 0x68, 0x2c, 0x3a, 0xa7, 0x23, 0x22, 0x94, 0x99, + 0xca, 0x85, 0x37, 0xbb, 0xf7, 0xe1, 0xd5, 0xc9, 0xa5, 0xeb, 0x86, 0x4b, 0xf4, 0x07, 0xe4, 0xb4, + 0x90, 0x8a, 0x14, 0xd9, 0xe5, 0x89, 0x20, 0x94, 0x5d, 0x72, 0x97, 0x9c, 0x9e, 0x5b, 0x70, 0xe9, + 0x1f, 0xb9, 0xe0, 0xe0, 0x22, 0x5b, 0xdd, 0xbb, 0x70, 0x69, 0x68, 0x98, 0xb6, 0x6f, 0x98, 0xb6, + 0xee, 0xb8, 0x7d, 0xe2, 0x9a, 0xf6, 0x80, 0x35, 0xd3, 0x60, 0xd9, 0x28, 0x60, 0xab, 0x82, 0x7b, + 0x4e, 0xd7, 0xcb, 0xbe, 0x60, 0xd7, 0x4b, 0xa2, 0x45, 0x8a, 0x3f, 0xa0, 0xed, 0xb8, 0xbe, 0xa8, + 0xfa, 0x7b, 0xb0, 0xec, 0x8c, 0xfd, 0xd1, 0xd8, 0x3f, 0xf3, 0x8a, 0xd7, 0x7f, 0x69, 0x76, 0xa2, + 0x03, 0x0f, 0x85, 0xe7, 0x79, 0x6e, 0x20, 0xf4, 0x7b, 0x0b, 0x70, 0x60, 0x4b, 0x1f, 0x1a, 0x7e, + 0xef, 0x50, 0xb7, 0x88, 0x3d, 0xb1, 0x1b, 0x50, 0xc0, 0xdf, 0xa5, 0xec, 0x26, 0xb1, 0x4b, 0x5f, + 0x4b, 0x90, 0xad, 0x99, 0x9e, 0x6f, 0xda, 0x3d, 0x9f, 0xf9, 0x75, 0x03, 0x96, 0x99, 0x10, 0xe9, + 0xeb, 0x51, 0x6c, 0x94, 0xd3, 0xf2, 0x82, 0x1c, 0x6c, 0x92, 0x9b, 0x80, 0xfa, 0x42, 0x31, 0x94, + 0x8c, 0x31, 0xc9, 0xe5, 0x80, 0x1e, 0x88, 0x6e, 0x01, 0xb6, 0xc7, 0x96, 0xc5, 0xcb, 0x2e, 0x60, + 0x4e, 0x80, 0x0e, 0xc4, 0xf8, 0x65, 0x97, 0x04, 0xbe, 0xe0, 0xb7, 0x20, 0x4b, 0x5c, 0xd7, 0x71, + 0x75, 0xc7, 0xd6, 0xfb, 0xe3, 0x11, 0xdb, 0x88, 0xe9, 0xa0, 0xb6, 0x19, 0x47, 0xb5, 0x6b, 0xe3, + 0x51, 0x09, 0x41, 0x5e, 0x75, 0xfb, 0xa6, 0x6d, 0xd0, 0x4a, 0xa7, 0x2b, 0x28, 0xfd, 0x21, 0x0e, + 0xe8, 0x53, 0x73, 0xf0, 0xa5, 0x31, 0xa0, 0x9b, 0x4b, 0x84, 0xbb, 0x06, 0x8b, 0xac, 0x53, 0x04, + 0x48, 0x6f, 0xbe, 0x2e, 0x23, 0x74, 0x71, 0x1d, 0x80, 0x1c, 0x4d, 0xac, 0x36, 0xb3, 0x75, 0x7d, + 0x76, 0xbe, 0xc4, 0xfa, 0x03, 0xb8, 0x43, 0x8e, 0xce, 0x62, 0x97, 0xe7, 0xed, 0xca, 0xe1, 0xae, + 0x4f, 0xf4, 0x22, 0xc6, 0x11, 0x6b, 0xba, 0xa0, 0x5e, 0x74, 0x17, 0xb2, 0x07, 0xe6, 0x09, 0xe9, + 0xeb, 0x0f, 0x19, 0x00, 0x2e, 0x24, 0x99, 0xe7, 0x4f, 0x69, 0x29, 0x93, 0x40, 0x59, 0xcb, 0x30, + 0x6d, 0x4e, 0x7c, 0x81, 0xc6, 0x56, 0xfa, 0x5b, 0x1c, 0x96, 0x77, 0x89, 0x3b, 0x20, 0x91, 0xcc, + 0xec, 0x42, 0xce, 0x22, 0x07, 0x2f, 0xb0, 0x0d, 0xb2, 0x54, 0x3d, 0xdc, 0x04, 0x2a, 0xe4, 0x5d, + 0x73, 0x70, 0x18, 0xb1, 0x17, 0x9b, 0xd3, 0x5e, 0x8e, 0xe9, 0x87, 0x06, 0x23, 0x09, 0x48, 0xbe, + 0x8c, 0xc3, 0xe0, 0x26, 0xe4, 0xe8, 0xe6, 0xd0, 0xc9, 0xd1, 0xd8, 0x08, 0xcf, 0x83, 0x60, 0xdf, + 0x64, 0x29, 0x4b, 0x16, 0x1c, 0xfc, 0x01, 0x5c, 0x61, 0xa1, 0x3c, 0xab, 0xd1, 0x19, 0x8d, 0x9e, + 0x1c, 0xf8, 0xf2, 0xd1, 0x64, 0xa3, 0xff, 0x15, 0x14, 0x78, 0xdc, 0xce, 0x51, 0x4e, 0x47, 0x94, + 0x57, 0x98, 0xd4, 0x94, 0x76, 0xe9, 0x77, 0x71, 0xc8, 0xdf, 0x31, 0xbc, 0xc3, 0x48, 0x5e, 0x6f, + 0xc1, 0xf2, 0x94, 0x33, 0xbc, 0x91, 0x88, 0x03, 0x37, 0xea, 0x02, 0xbe, 0x0d, 0x68, 0x7a, 0x72, + 0xde, 0x4b, 0x98, 0x70, 0x7e, 0x72, 0xca, 0x97, 0x9e, 0x91, 0xb7, 0x21, 0x3f, 0xa4, 0x45, 0x7c, + 0xd6, 0x21, 0xa3, 0x29, 0xc9, 0x71, 0x5e, 0xe0, 0xec, 0xcb, 0xcb, 0xc9, 0x93, 0x18, 0xe0, 0x86, + 0xb8, 0x2d, 0x47, 0xf2, 0xf2, 0x93, 0xc1, 0xad, 0xdc, 0x04, 0xe6, 0x60, 0xed, 0x3e, 0x17, 0x6e, + 0xd9, 0x08, 0xd4, 0xc0, 0x2a, 0xe4, 0x82, 0x5b, 0xfd, 0xf3, 0x36, 0xba, 0x6c, 0x60, 0x80, 0xe5, + 0xf6, 0x25, 0x17, 0x48, 0xe9, 0xcf, 0x12, 0xac, 0x04, 0x91, 0xaf, 0x9b, 0x96, 0x4f, 0x5c, 0x11, + 0xfb, 0x0d, 0x40, 0xe1, 0x4a, 0x7b, 0x8e, 0xc5, 0xc2, 0x27, 0x45, 0xe2, 0x92, 0x0f, 0xb8, 0x55, + 0xc7, 0xa2, 0x41, 0xfc, 0x7c, 0x3a, 0x32, 0xbc, 0x97, 0xfd, 0x7c, 0xca, 0x19, 0x67, 0xe4, 0x6f, + 0x44, 0xdf, 0x45, 0x36, 0xe8, 0x45, 0xe5, 0x6c, 0x61, 0x2d, 0xd7, 0xf1, 0x9d, 0xf3, 0x02, 0x55, + 0xfa, 0x4d, 0x0a, 0xf2, 0xe5, 0xc1, 0xc0, 0x25, 0x03, 0xc3, 0x77, 0xb8, 0x8b, 0xd7, 0x01, 0x06, + 0xae, 0xc3, 0xd3, 0x16, 0xdd, 0x84, 0x69, 0x46, 0xad, 0x3a, 0x96, 0x87, 0x3f, 0x87, 0xac, 0x21, + 0x94, 0x4c, 0x27, 0xbc, 0x42, 0xfd, 0x62, 0x76, 0x8c, 0x27, 0xa7, 0x08, 0x87, 0x91, 0xf4, 0x45, + 0xed, 0xe1, 0x9f, 0x09, 0x1c, 0x43, 0xfa, 0x7a, 0xc4, 0x95, 0x44, 0xe8, 0x0a, 0x12, 0xdc, 0xed, + 0xd0, 0xa3, 0x6d, 0x91, 0xab, 0x24, 0xcb, 0xd5, 0x3b, 0xcf, 0xec, 0xc9, 0x74, 0xe6, 0x8a, 0xbf, + 0x8e, 0x41, 0x26, 0xe2, 0x1e, 0x35, 0x7c, 0x30, 0xb6, 0x7b, 0x2c, 0x49, 0xf3, 0x18, 0xae, 0x8f, + 0xed, 0x5e, 0x60, 0x98, 0x1a, 0xc0, 0x6b, 0x90, 0x0a, 0x81, 0x4f, 0x2c, 0xb2, 0x75, 0x43, 0x2a, + 0x7e, 0x13, 0xf2, 0x07, 0xac, 0x56, 0xc2, 0xca, 0xa0, 0xdb, 0x20, 0xa7, 0x65, 0x39, 0x55, 0x54, + 0xc4, 0x15, 0xf6, 0x64, 0xc5, 0xd8, 0x49, 0x06, 0xb6, 0x16, 0x7b, 0x9c, 0x71, 0x07, 0xd2, 0x86, + 0x3b, 0x18, 0x0f, 0x89, 0xed, 0x7b, 0x85, 0x45, 0x96, 0x91, 0x79, 0xaa, 0xfe, 0x4c, 0x79, 0x27, + 0x91, 0x8a, 0xa3, 0x44, 0xe9, 0x9b, 0x38, 0x24, 0xe8, 0x2a, 0x30, 0x82, 0x6c, 0x59, 0xf9, 0x44, + 0x57, 0xd4, 0x8e, 0xae, 0x74, 0x9b, 0x4d, 0xb4, 0x80, 0x97, 0x20, 0x5e, 0xde, 0xdb, 0x46, 0x12, + 0xce, 0x42, 0xaa, 0xa2, 0xaa, 0x4d, 0xbd, 0xac, 0xd4, 0x50, 0x0c, 0x67, 0x60, 0x89, 0x8d, 0x54, + 0x0d, 0xc5, 0x71, 0x1e, 0xa0, 0xaa, 0x2a, 0xd5, 0x72, 0x47, 0x2f, 0x6f, 0x6f, 0xa3, 0x04, 0x4e, + 0x43, 0xb2, 0xaa, 0x76, 0x95, 0x0e, 0x4a, 0x52, 0xf5, 0xdd, 0xf2, 0xc7, 0x68, 0x89, 0xfd, 0x68, + 0x28, 0x28, 0x85, 0x01, 0x16, 0xdb, 0x9d, 0x5a, 0x4d, 0xde, 0x43, 0x69, 0x4a, 0x6c, 0x77, 0x77, + 0x11, 0x50, 0x73, 0xed, 0xee, 0xae, 0xde, 0x50, 0x3a, 0x28, 0x43, 0x67, 0xda, 0x2b, 0x6b, 0x8d, + 0xb2, 0x52, 0x95, 0x51, 0x96, 0xb2, 0x3e, 0x56, 0x35, 0x66, 0x39, 0xc7, 0x67, 0xea, 0x2a, 0x1d, + 0x5d, 0x53, 0xef, 0xb7, 0x51, 0x9e, 0xe9, 0xdd, 0xd3, 0x6a, 0x8d, 0x7a, 0x1d, 0x2d, 0x63, 0x0c, + 0xf9, 0x7a, 0x43, 0x29, 0x37, 0xf5, 0x50, 0x1b, 0xd1, 0x05, 0x71, 0x9a, 0x98, 0xf3, 0x12, 0xce, + 0x41, 0xba, 0xac, 0x69, 0xe5, 0x4f, 0x98, 0x45, 0x4c, 0x27, 0xdb, 0x69, 0xab, 0x0a, 0x1b, 0x5d, + 0xa6, 0x4c, 0x3a, 0xaa, 0xb0, 0xe1, 0x0a, 0x9d, 0xae, 0xdd, 0xd1, 0x1a, 0xca, 0x36, 0x1b, 0xbf, + 0xc2, 0x56, 0xdd, 0xe8, 0xb0, 0x10, 0xbc, 0x4a, 0x17, 0x42, 0x07, 0xaa, 0x86, 0xae, 0xe0, 0x14, + 0x24, 0xaa, 0xaa, 0xa6, 0xa1, 0x02, 0x2e, 0xc0, 0x4a, 0x4b, 0xd6, 0xaa, 0xb2, 0xd2, 0x69, 0x34, + 0x65, 0xbd, 0xd6, 0x68, 0x57, 0xf5, 0xc6, 0x6e, 0xab, 0x89, 0x5e, 0x9b, 0xe2, 0x54, 0x55, 0xa5, + 0xc3, 0x39, 0x45, 0x7c, 0x19, 0x96, 0x99, 0x0f, 0x6a, 0x65, 0x47, 0xae, 0xf2, 0x20, 0x5e, 0xc5, + 0x2b, 0x80, 0xb8, 0x2b, 0x11, 0xea, 0xeb, 0xd4, 0x83, 0xbd, 0xb2, 0xa6, 0xb7, 0xd4, 0x16, 0xfa, + 0x3f, 0xee, 0x1e, 0x5d, 0x16, 0x1b, 0xaf, 0x96, 0x6e, 0x43, 0x82, 0x16, 0x39, 0xf5, 0xa6, 0xdc, + 0xed, 0xa8, 0x68, 0x81, 0x05, 0xbb, 0x5a, 0x6e, 0x96, 0x35, 0x24, 0x51, 0x69, 0x45, 0x55, 0x74, + 0x31, 0x8e, 0x95, 0xfe, 0xba, 0x08, 0x57, 0x1b, 0xb6, 0x4f, 0x5c, 0x8b, 0x18, 0x0f, 0x49, 0x9f, + 0x5f, 0xd2, 0x23, 0x67, 0xc7, 0x27, 0x53, 0x28, 0xfa, 0xc3, 0xd9, 0x15, 0xf6, 0x14, 0x33, 0xfc, + 0x6c, 0x99, 0x82, 0xd6, 0x91, 0x97, 0x8f, 0xd8, 0x79, 0x2f, 0x1f, 0x93, 0xcf, 0x88, 0xf1, 0xf3, + 0x9f, 0x11, 0x2f, 0xf4, 0xe2, 0x9e, 0x9c, 0x1f, 0x19, 0xfc, 0xaf, 0x3f, 0xcf, 0x14, 0xbf, 0x8e, + 0x41, 0x92, 0x65, 0x07, 0x7f, 0x04, 0x89, 0x3e, 0xf1, 0x7a, 0xcf, 0x85, 0x12, 0x98, 0xe6, 0xb3, + 0x80, 0x84, 0x2a, 0x24, 0x46, 0x8e, 0xc7, 0xd3, 0xf9, 0xd4, 0x47, 0xb8, 0x96, 0xe3, 0xf9, 0x2d, + 0xfe, 0x61, 0x80, 0x56, 0x50, 0x30, 0x0f, 0x55, 0xc6, 0x35, 0x48, 0x85, 0x58, 0x3f, 0x31, 0x27, + 0xd6, 0x0f, 0x35, 0xcf, 0x1e, 0x04, 0x93, 0x2f, 0xf2, 0x20, 0x58, 0xfa, 0x46, 0x82, 0x7c, 0xcb, + 0x75, 0xbe, 0x20, 0x3d, 0xbf, 0x4d, 0xf8, 0x8d, 0xfa, 0x23, 0x48, 0xd2, 0x8a, 0x0c, 0xf6, 0xcc, + 0x3c, 0x25, 0xc9, 0x15, 0xf1, 0x36, 0x5c, 0x1a, 0x10, 0x9b, 0xb8, 0x86, 0x1f, 0xc1, 0x9c, 0xfc, + 0xf6, 0x59, 0x9c, 0x4e, 0x0c, 0x19, 0x6e, 0xf0, 0x4f, 0x2f, 0x1d, 0x0d, 0x85, 0x4a, 0x01, 0x18, + 0xbd, 0x01, 0xc8, 0x1e, 0xb3, 0x67, 0x0d, 0x4f, 0x1f, 0x11, 0x57, 0x1f, 0x10, 0x9b, 0xdf, 0x3c, + 0xb5, 0x9c, 0x3d, 0x1e, 0xd2, 0xa3, 0xb4, 0x45, 0xdc, 0x6d, 0x62, 0x97, 0xbe, 0xcf, 0x41, 0xf6, + 0xbe, 0x69, 0xf7, 0x9d, 0x63, 0xb1, 0xf3, 0xd7, 0xd8, 0x8b, 0xb1, 0x6f, 0xb2, 0x43, 0xfb, 0x54, + 0x3c, 0x09, 0x44, 0x49, 0xb8, 0x0d, 0xe9, 0x63, 0xa6, 0x51, 0x0f, 0x9d, 0xdb, 0x9c, 0xbd, 0xd4, + 0xa8, 0x71, 0x31, 0xa8, 0x87, 0x67, 0x51, 0x68, 0xa7, 0xf8, 0x17, 0x49, 0x9c, 0x42, 0x6d, 0xc8, + 0x05, 0x18, 0x81, 0xd4, 0x9f, 0xf7, 0x44, 0xd6, 0x26, 0x6d, 0xe0, 0x7b, 0x00, 0x62, 0x2a, 0x6a, + 0x31, 0xc6, 0x2c, 0xbe, 0x3b, 0x9f, 0xcf, 0xd4, 0x6a, 0xc4, 0xc8, 0x07, 0x89, 0x47, 0x5f, 0x5d, + 0x93, 0x8a, 0x5f, 0x2d, 0x41, 0xb2, 0xee, 0x1a, 0x43, 0x82, 0xef, 0x42, 0x62, 0xe8, 0xf4, 0x89, + 0x70, 0xf7, 0x59, 0x8d, 0x33, 0xdd, 0x8d, 0x5d, 0xa7, 0x1f, 0xb6, 0x17, 0x6a, 0x04, 0xdf, 0x83, + 0xc5, 0x7d, 0x67, 0x6c, 0xf7, 0xbd, 0x19, 0x38, 0xf0, 0xe9, 0xe6, 0x2a, 0x4c, 0x35, 0x68, 0x76, + 0xdc, 0x10, 0xfe, 0x14, 0xd2, 0xe4, 0xa4, 0x67, 0x8d, 0x69, 0xd1, 0xb1, 0x6d, 0x98, 0xdf, 0x7a, + 0x6f, 0x2e, 0xab, 0x72, 0xa0, 0x1d, 0xbe, 0x72, 0x04, 0x84, 0xe2, 0x13, 0x09, 0x92, 0x6c, 0x52, + 0x3a, 0x0b, 0x9b, 0x8f, 0x36, 0x4c, 0x11, 0x8a, 0xf7, 0xe6, 0xf7, 0x3d, 0xd2, 0x6e, 0xcf, 0xcc, + 0xd1, 0x83, 0xc1, 0xb4, 0x7d, 0xdd, 0x39, 0x38, 0xf0, 0x08, 0xc7, 0x56, 0xc1, 0xc7, 0x9b, 0xb4, + 0x69, 0xfb, 0x2a, 0x23, 0xe3, 0xeb, 0x90, 0xa5, 0xbb, 0xa2, 0x1f, 0x88, 0xd1, 0x95, 0x66, 0xb5, + 0x0c, 0xa3, 0x09, 0x91, 0x1d, 0xc8, 0x70, 0x26, 0xfb, 0xd6, 0x29, 0x3a, 0xc9, 0x1c, 0x1f, 0x04, + 0x81, 0x6b, 0x53, 0x9f, 0x8a, 0x7f, 0x94, 0x60, 0x91, 0x87, 0x1b, 0x2b, 0x90, 0xf4, 0x7c, 0xc3, + 0xf5, 0x45, 0x23, 0xdd, 0x9a, 0x7f, 0xd9, 0x61, 0x83, 0xa1, 0x66, 0x70, 0x0d, 0xe2, 0xc4, 0xee, + 0x8b, 0x02, 0x78, 0x0e, 0x6b, 0x1a, 0x55, 0x2f, 0xdd, 0x80, 0x04, 0xad, 0x2e, 0x0a, 0xcb, 0xb4, + 0xb2, 0xb2, 0x2d, 0xa3, 0x05, 0x8a, 0x10, 0x18, 0x82, 0x92, 0x28, 0x42, 0xd8, 0xd6, 0xd4, 0x6e, + 0xab, 0x8d, 0x62, 0xa5, 0x2f, 0x21, 0x1d, 0xc6, 0x1e, 0x5f, 0x81, 0xcb, 0x5d, 0xa5, 0xa2, 0x76, + 0x95, 0x9a, 0x5c, 0xd3, 0x5b, 0x9a, 0x5c, 0x95, 0x6b, 0x0d, 0x65, 0x1b, 0x2d, 0x4c, 0x32, 0xea, + 0x6a, 0xb3, 0xa9, 0xde, 0xa7, 0x0c, 0x89, 0x22, 0x16, 0xb5, 0x5e, 0x6f, 0xcb, 0x9d, 0x88, 0x78, + 0x2c, 0x42, 0x3d, 0x93, 0x8d, 0xe3, 0x65, 0xc8, 0x54, 0xbb, 0x9a, 0x26, 0x73, 0x28, 0x87, 0x12, + 0xa5, 0xcf, 0x20, 0x1d, 0x56, 0x17, 0x45, 0x6d, 0x8a, 0xaa, 0xcb, 0x1f, 0x57, 0x9b, 0xdd, 0x76, + 0x43, 0x55, 0xf8, 0xa4, 0x6c, 0x58, 0x93, 0xf5, 0xa8, 0x9e, 0x84, 0x2f, 0x41, 0x2e, 0x60, 0xb0, + 0x75, 0xa0, 0x18, 0xd5, 0x0e, 0x48, 0x9d, 0x86, 0xdc, 0x46, 0xf1, 0xe2, 0xdf, 0x63, 0x90, 0x0a, + 0xfa, 0x0e, 0x96, 0x23, 0x30, 0x3f, 0xb3, 0xf5, 0xf6, 0xb3, 0x46, 0x75, 0x1a, 0xe4, 0x5f, 0xcc, + 0x49, 0x54, 0x81, 0xe4, 0x01, 0xcd, 0x97, 0xb8, 0xbb, 0xde, 0x9e, 0x27, 0xc7, 0x1a, 0x57, 0xc5, + 0xeb, 0x30, 0x71, 0x6d, 0x60, 0x20, 0x24, 0x19, 0x5c, 0xb6, 0x26, 0x2e, 0x14, 0x45, 0x48, 0x19, + 0xee, 0xc0, 0x6b, 0xf4, 0x4f, 0xbc, 0xc2, 0x12, 0xeb, 0xea, 0xe1, 0x98, 0x5a, 0xe1, 0x4f, 0xcc, + 0xc2, 0x4a, 0x2a, 0x7a, 0x85, 0x8f, 0x72, 0x76, 0x12, 0xa9, 0x18, 0x8a, 0x8b, 0x9b, 0xc3, 0x9f, + 0x24, 0x80, 0xb3, 0xee, 0x48, 0x31, 0xa6, 0xa6, 0xde, 0xd7, 0x95, 0xee, 0x6e, 0x45, 0xd6, 0x44, + 0x9d, 0x95, 0x95, 0xbb, 0x1c, 0x7d, 0xd6, 0x64, 0xa5, 0x2d, 0xeb, 0x6c, 0xcc, 0x92, 0x24, 0xd0, + 0x30, 0xa7, 0xc4, 0x29, 0xf6, 0xae, 0x76, 0x77, 0x19, 0x66, 0xee, 0xf0, 0x4b, 0x04, 0x43, 0xca, + 0xfc, 0x12, 0xd1, 0x2c, 0x6f, 0xa3, 0x45, 0x6a, 0xae, 0x29, 0x97, 0x6b, 0x68, 0x89, 0xd6, 0x4f, + 0xbd, 0xa1, 0xb5, 0x3b, 0xfa, 0x5e, 0xb9, 0xd9, 0x95, 0x51, 0x8a, 0xda, 0x6f, 0x96, 0xc3, 0x71, + 0x9a, 0x5a, 0x53, 0x3a, 0x77, 0xc4, 0x10, 0x6e, 0xfd, 0x12, 0xf2, 0x93, 0x5f, 0x4e, 0x68, 0xe1, + 0xb7, 0xba, 0x95, 0x66, 0xa3, 0x8a, 0x16, 0xf0, 0x6b, 0xf0, 0x0a, 0xff, 0x4d, 0xa1, 0x3d, 0xbb, + 0xfd, 0x08, 0x96, 0x54, 0x79, 0xe7, 0xd1, 0xf7, 0xab, 0x0b, 0x8f, 0x1e, 0xaf, 0x4a, 0xdf, 0x3e, + 0x5e, 0x95, 0xbe, 0x7b, 0xbc, 0x2a, 0xfd, 0xf3, 0xf1, 0xaa, 0xf4, 0xdb, 0x1f, 0x56, 0x17, 0xbe, + 0xfd, 0x61, 0x75, 0xe1, 0xbb, 0x1f, 0x56, 0x17, 0x3e, 0xcd, 0x44, 0xfe, 0xa1, 0xe1, 0x3f, 0x01, + 0x00, 0x00, 0xff, 0xff, 0xa2, 0x3f, 0xd6, 0xb4, 0xab, 0x21, 0x00, 0x00, } diff --git a/pkg/sql/execinfrapb/processors_sql.proto b/pkg/sql/execinfrapb/processors_sql.proto index 902687aa25b6..6ececab9bd31 100644 --- a/pkg/sql/execinfrapb/processors_sql.proto +++ b/pkg/sql/execinfrapb/processors_sql.proto @@ -137,6 +137,11 @@ message TableReaderSpec { // makes it out of the SQL optimizer without throwing an error. If/when other // wait policies are supported, this field will be plumbed further. optional sqlbase.ScanLockingWaitPolicy locking_wait_policy = 11 [(gogoproto.nullable) = false]; + + // Indicates what implicit system columns this TableReader is expected to + // synthesize. These system columns will be placed at the end of the row + // output by the TableReader, in the order specified. + repeated sqlbase.SystemColumnKind system_columns = 13; } // IndexSkipTableReaderSpec is the specification for a table reader that @@ -257,6 +262,15 @@ message JoinReaderSpec { // stream. This is only applicable to lookup joins, where doing so is // expensive. Index joins do this by default regardless of the parameter. optional bool maintain_ordering = 11 [(gogoproto.nullable) = false]; + + // Indicates what implicit system columns this JoinReader is expected to + // synthesize. These system columns will be placed at the end of the row + // output by the JoinReader, in the order specified. + // + // This is only used in the special case of index joins, where the final + // result of the secondary index joined against the primary index is + // expected to contain the materialized system columns. + repeated sqlbase.SystemColumnKind system_columns = 12; } // SorterSpec is the specification for a "sorting aggregator". A sorting diff --git a/pkg/sql/logictest/testdata/logic_test/dist_vectorize b/pkg/sql/logictest/testdata/logic_test/dist_vectorize index 737004efec59..e8153bbf2218 100644 --- a/pkg/sql/logictest/testdata/logic_test/dist_vectorize +++ b/pkg/sql/logictest/testdata/logic_test/dist_vectorize @@ -99,7 +99,7 @@ EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv │ ├ *colexec.countOp │ │ └ *colexec.simpleProjectOp │ │ └ *colexec.CancelChecker -│ │ └ *colfetcher.colBatchScan +│ │ └ *colfetcher.ColBatchScan │ ├ *colrpc.Inbox │ ├ *colrpc.Inbox │ ├ *colrpc.Inbox @@ -110,28 +110,28 @@ EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv │ └ *colexec.countOp │ └ *colexec.simpleProjectOp │ └ *colexec.CancelChecker -│ └ *colfetcher.colBatchScan +│ └ *colfetcher.ColBatchScan ├ Node 3 │ └ *colrpc.Outbox │ └ *colexec.deselectorOp │ └ *colexec.countOp │ └ *colexec.simpleProjectOp │ └ *colexec.CancelChecker -│ └ *colfetcher.colBatchScan +│ └ *colfetcher.ColBatchScan ├ Node 4 │ └ *colrpc.Outbox │ └ *colexec.deselectorOp │ └ *colexec.countOp │ └ *colexec.simpleProjectOp │ └ *colexec.CancelChecker -│ └ *colfetcher.colBatchScan +│ └ *colfetcher.ColBatchScan └ Node 5 └ *colrpc.Outbox └ *colexec.deselectorOp └ *colexec.countOp └ *colexec.simpleProjectOp └ *colexec.CancelChecker - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv NATURAL INNER HASH JOIN kv kv2 @@ -151,7 +151,7 @@ EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv NATURAL INNER HASH JOIN kv kv2 │ │ │ │ ├ *colexec.routerOutputOp │ │ │ │ │ └ *colexec.HashRouter │ │ │ │ │ └ *colexec.CancelChecker -│ │ │ │ │ └ *colfetcher.colBatchScan +│ │ │ │ │ └ *colfetcher.ColBatchScan │ │ │ │ ├ *colrpc.Inbox │ │ │ │ ├ *colrpc.Inbox │ │ │ │ ├ *colrpc.Inbox @@ -160,7 +160,7 @@ EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv NATURAL INNER HASH JOIN kv kv2 │ │ │ ├ *colexec.routerOutputOp │ │ │ │ └ *colexec.HashRouter │ │ │ │ └ *colexec.CancelChecker -│ │ │ │ └ *colfetcher.colBatchScan +│ │ │ │ └ *colfetcher.ColBatchScan │ │ │ ├ *colrpc.Inbox │ │ │ ├ *colrpc.Inbox │ │ │ ├ *colrpc.Inbox @@ -186,7 +186,7 @@ EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv NATURAL INNER HASH JOIN kv kv2 │ │ │ ├ *colexec.routerOutputOp │ │ │ │ └ *colexec.HashRouter │ │ │ │ └ *colexec.CancelChecker -│ │ │ │ └ *colfetcher.colBatchScan +│ │ │ │ └ *colfetcher.ColBatchScan │ │ │ ├ *colrpc.Inbox │ │ │ ├ *colrpc.Inbox │ │ │ └ *colrpc.Inbox @@ -195,7 +195,7 @@ EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv NATURAL INNER HASH JOIN kv kv2 │ │ ├ *colexec.routerOutputOp │ │ │ └ *colexec.HashRouter │ │ │ └ *colexec.CancelChecker -│ │ │ └ *colfetcher.colBatchScan +│ │ │ └ *colfetcher.ColBatchScan │ │ ├ *colrpc.Inbox │ │ ├ *colrpc.Inbox │ │ └ *colrpc.Inbox @@ -217,7 +217,7 @@ EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv NATURAL INNER HASH JOIN kv kv2 │ │ │ ├ *colexec.routerOutputOp │ │ │ │ └ *colexec.HashRouter │ │ │ │ └ *colexec.CancelChecker -│ │ │ │ └ *colfetcher.colBatchScan +│ │ │ │ └ *colfetcher.ColBatchScan │ │ │ ├ *colrpc.Inbox │ │ │ └ *colrpc.Inbox │ │ └ *colexec.ParallelUnorderedSynchronizer @@ -226,7 +226,7 @@ EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv NATURAL INNER HASH JOIN kv kv2 │ │ ├ *colexec.routerOutputOp │ │ │ └ *colexec.HashRouter │ │ │ └ *colexec.CancelChecker -│ │ │ └ *colfetcher.colBatchScan +│ │ │ └ *colfetcher.ColBatchScan │ │ ├ *colrpc.Inbox │ │ └ *colrpc.Inbox │ ├ *colexec.ParallelUnorderedSynchronizer @@ -248,7 +248,7 @@ EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv NATURAL INNER HASH JOIN kv kv2 │ │ │ ├ *colexec.routerOutputOp │ │ │ │ └ *colexec.HashRouter │ │ │ │ └ *colexec.CancelChecker -│ │ │ │ └ *colfetcher.colBatchScan +│ │ │ │ └ *colfetcher.ColBatchScan │ │ │ └ *colrpc.Inbox │ │ └ *colexec.ParallelUnorderedSynchronizer │ │ ├ *colrpc.Inbox @@ -257,7 +257,7 @@ EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv NATURAL INNER HASH JOIN kv kv2 │ │ ├ *colexec.routerOutputOp │ │ │ └ *colexec.HashRouter │ │ │ └ *colexec.CancelChecker -│ │ │ └ *colfetcher.colBatchScan +│ │ │ └ *colfetcher.ColBatchScan │ │ └ *colrpc.Inbox │ ├ *colexec.ParallelUnorderedSynchronizer │ ├ *colexec.ParallelUnorderedSynchronizer @@ -279,7 +279,7 @@ EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv NATURAL INNER HASH JOIN kv kv2 │ │ └ *colexec.routerOutputOp │ │ └ *colexec.HashRouter │ │ └ *colexec.CancelChecker - │ │ └ *colfetcher.colBatchScan + │ │ └ *colfetcher.ColBatchScan │ └ *colexec.ParallelUnorderedSynchronizer │ ├ *colrpc.Inbox │ ├ *colrpc.Inbox @@ -288,7 +288,7 @@ EXPLAIN (VEC, VERBOSE) SELECT count(*) FROM kv NATURAL INNER HASH JOIN kv kv2 │ └ *colexec.routerOutputOp │ └ *colexec.HashRouter │ └ *colexec.CancelChecker - │ └ *colfetcher.colBatchScan + │ └ *colfetcher.ColBatchScan ├ *colexec.ParallelUnorderedSynchronizer ├ *colexec.ParallelUnorderedSynchronizer └ *colexec.externalHashJoiner diff --git a/pkg/sql/logictest/testdata/logic_test/experimental_distsql_planning_5node b/pkg/sql/logictest/testdata/logic_test/experimental_distsql_planning_5node index 67aa647970ec..9351fe3294db 100644 --- a/pkg/sql/logictest/testdata/logic_test/experimental_distsql_planning_5node +++ b/pkg/sql/logictest/testdata/logic_test/experimental_distsql_planning_5node @@ -75,23 +75,23 @@ EXPLAIN (VEC) SELECT * FROM kv WHERE k::REGCLASS IS NOT NULL │ └ *colexec.isNullSelOp │ └ *colexec.castInt64Int64Op │ └ *colexec.ParallelUnorderedSynchronizer -│ ├ *colfetcher.colBatchScan +│ ├ *colfetcher.ColBatchScan │ ├ *colrpc.Inbox │ ├ *colrpc.Inbox │ ├ *colrpc.Inbox │ └ *colrpc.Inbox ├ Node 2 │ └ *colrpc.Outbox -│ └ *colfetcher.colBatchScan +│ └ *colfetcher.ColBatchScan ├ Node 3 │ └ *colrpc.Outbox -│ └ *colfetcher.colBatchScan +│ └ *colfetcher.ColBatchScan ├ Node 4 │ └ *colrpc.Outbox -│ └ *colfetcher.colBatchScan +│ └ *colfetcher.ColBatchScan └ Node 5 └ *colrpc.Outbox - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan # Check that the plan is local when experimental DistSQL planning is disabled. statement ok @@ -104,7 +104,7 @@ EXPLAIN (VEC) SELECT * FROM kv WHERE k::REGCLASS IS NOT NULL └ Node 1 └ *colexec.isNullSelOp └ *colexec.castInt64Int64Op - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan statement ok SET experimental_distsql_planning = always @@ -118,23 +118,23 @@ EXPLAIN (VEC) SELECT k::REGCLASS FROM kv ├ Node 1 │ └ *colexec.castInt64Int64Op │ └ *colexec.ParallelUnorderedSynchronizer -│ ├ *colfetcher.colBatchScan +│ ├ *colfetcher.ColBatchScan │ ├ *colrpc.Inbox │ ├ *colrpc.Inbox │ ├ *colrpc.Inbox │ └ *colrpc.Inbox ├ Node 2 │ └ *colrpc.Outbox -│ └ *colfetcher.colBatchScan +│ └ *colfetcher.ColBatchScan ├ Node 3 │ └ *colrpc.Outbox -│ └ *colfetcher.colBatchScan +│ └ *colfetcher.ColBatchScan ├ Node 4 │ └ *colrpc.Outbox -│ └ *colfetcher.colBatchScan +│ └ *colfetcher.ColBatchScan └ Node 5 └ *colrpc.Outbox - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query TTT EXPLAIN SELECT k::REGCLASS FROM kv @@ -173,7 +173,14 @@ EXPLAIN (VEC) SELECT k::REGCLASS FROM kv │ └ Node 1 └ *colexec.castInt64Int64Op - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan statement ok SET disable_partially_distributed_plans = false + +# Ensure that the new factory can materialize system columns. +# TODO (rohany): The factory can't plan lookup joins yet. Add a test when it can. +query B +SELECT crdb_internal_mvcc_timestamp IS NOT NULL FROM kv LIMIT 1 +---- +true diff --git a/pkg/sql/logictest/testdata/logic_test/merge_join_dist_vec b/pkg/sql/logictest/testdata/logic_test/merge_join_dist_vec index 89dfd0deffca..265496ccee6c 100644 --- a/pkg/sql/logictest/testdata/logic_test/merge_join_dist_vec +++ b/pkg/sql/logictest/testdata/logic_test/merge_join_dist_vec @@ -65,9 +65,9 @@ EXPLAIN (VEC) SELECT * FROM numbers AS t1 INNER MERGE JOIN numbers AS t2 ON t1._ └ *colexec.mergeJoinInnerOp ├ *colexec.castInt16Int32Op │ └ *colexec.sortOp - │ └ *colfetcher.colBatchScan + │ └ *colfetcher.ColBatchScan └ *colexec.sortOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T EXPLAIN (VEC) SELECT * FROM numbers AS t1 INNER MERGE JOIN numbers AS t2 ON t1._int8 = t2._int2 @@ -76,10 +76,10 @@ EXPLAIN (VEC) SELECT * FROM numbers AS t1 INNER MERGE JOIN numbers AS t2 ON t1._ └ Node 1 └ *colexec.mergeJoinInnerOp ├ *colexec.sortOp - │ └ *colfetcher.colBatchScan + │ └ *colfetcher.ColBatchScan └ *colexec.castInt16Int64Op └ *colexec.sortOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan # Also check that we cannot plan a merge join with other numeric types. statement error could not produce a query plan conforming to the MERGE JOIN hint diff --git a/pkg/sql/logictest/testdata/logic_test/mvcc b/pkg/sql/logictest/testdata/logic_test/mvcc new file mode 100644 index 000000000000..1fc2a79212fb --- /dev/null +++ b/pkg/sql/logictest/testdata/logic_test/mvcc @@ -0,0 +1,122 @@ +statement ok +CREATE TABLE t (x INT PRIMARY KEY, y INT, z INT, INDEX i (z)); +INSERT INTO t VALUES (1, 2, 3) + +# Get the timestamp for row (1, 2, 3). +query B +SELECT crdb_internal_mvcc_timestamp IS NOT NULL FROM t +---- +true + +let $base_ts +SELECT crdb_internal_mvcc_timestamp FROM t + +# Insert a new value into t. +statement ok +INSERT INTO t VALUES (2, 3, 4) + +# Its timestamp should be bigger than the timestamp of the first row. +query B +SELECT crdb_internal_mvcc_timestamp > $base_ts FROM t WHERE x = 2 +---- +true + +# Check that trying to get the timestamp from t@i succeeds too. +query B +SELECT crdb_internal_mvcc_timestamp = $base_ts FROM t@i WHERE x = 1 +---- +true + +# We should be able to produce the mvcc column as many times as needed. +query BBBB +SELECT crdb_internal_mvcc_timestamp = $base_ts, crdb_internal_mvcc_timestamp = $base_ts, crdb_internal_mvcc_timestamp = $base_ts, crdb_internal_mvcc_timestamp = $base_ts FROM t@i WHERE x = 1 +---- +true true true true + +# Update the original row. +statement ok +UPDATE t SET z = 5 WHERE x = 1 + +query B +SELECT crdb_internal_mvcc_timestamp > $base_ts FROM t +---- +true +true + +# Ensure that we can use the timestamp column in different places. +query III rowsort +SELECT * FROM t WHERE crdb_internal_mvcc_timestamp IS NOT NULL +---- +1 2 5 +2 3 4 + +query IIIIII rowsort +SELECT t1.*, t2.* FROM t t1 JOIN t t2 ON t1.crdb_internal_mvcc_timestamp = t2.crdb_internal_mvcc_timestamp +---- +1 2 5 1 2 5 +2 3 4 2 3 4 + +let $update_ts +SELECT crdb_internal_mvcc_timestamp FROM t WHERE x = 2 + +statement ok +UPDATE t SET z = 6 WHERE crdb_internal_mvcc_timestamp = $update_ts + +query III rowsort +SELECT * FROM t +---- +1 2 5 +2 3 6 + +let $del_ts +SELECT crdb_internal_mvcc_timestamp FROM t WHERE x = 2 + +statement ok +DELETE FROM t WHERE crdb_internal_mvcc_timestamp = $del_ts + +query III +SELECT * FROM t +---- +1 2 5 + +let $update_ts +SELECT crdb_internal_mvcc_timestamp FROM t WHERE x = 1 + +statement ok +UPDATE t SET z = (crdb_internal_mvcc_timestamp + 1.0)::INT WHERE x = 1 + +query B +SELECT z = ($update_ts + 1.0)::INT FROM t WHERE x = 1 +---- +true + +query IIB +SELECT x, y, crdb_internal_mvcc_timestamp IS NOT NULL AS foo FROM t ORDER BY foo +---- +1 2 true + +# We shouldn't be able to insert into the MVCC column. +statement error pq: cannot modify system column \"crdb_internal_mvcc_timestamp\" +INSERT INTO t (x, crdb_internal_mvcc_timestamp) VALUES (1, 0) + +statement error pq: cannot modify system column \"crdb_internal_mvcc_timestamp\" +UPDATE t SET crdb_internal_mvcc_timestamp = 1.0 WHERE x = 1 + +statement error pq: cannot modify system column \"crdb_internal_mvcc_timestamp\" +UPSERT INTO t (x, crdb_internal_mvcc_timestamp) VALUES (1, 0) + +statement error pq: INSERT has more expressions than target columns, 4 expressions for 3 targets +INSERT INTO t VALUES (7, 8, 9, 1.0) + +statement error pq: column \"crdb_internal_mvcc_timestamp\" does not exist +INSERT INTO t VALUES (1, 2, 3) RETURNING crdb_internal_mvcc_timestamp + +# Ensure that we can't create columns that conflict with system column names. +statement error pq: column name "crdb_internal_mvcc_timestamp" conflicts with a system column name +CREATE TABLE bad (crdb_internal_mvcc_timestamp int) + +statement error pq: column name "crdb_internal_mvcc_timestamp" conflicts with a system column name +ALTER TABLE t ADD COLUMN crdb_internal_mvcc_timestamp INT + +statement error pq: column name "crdb_internal_mvcc_timestamp" conflicts with a system column name +ALTER TABLE t RENAME COLUMN x TO crdb_internal_mvcc_timestamp diff --git a/pkg/sql/logictest/testdata/logic_test/tpch_vec b/pkg/sql/logictest/testdata/logic_test/tpch_vec index 6e04073976b1..354052ccc63c 100644 --- a/pkg/sql/logictest/testdata/logic_test/tpch_vec +++ b/pkg/sql/logictest/testdata/logic_test/tpch_vec @@ -541,7 +541,7 @@ EXPLAIN (VEC) SELECT l_returnflag, l_linestatus, sum(l_quantity) AS sum_qty, sum └ *colexec.projMultFloat64Float64Op └ *colexec.projMinusFloat64ConstFloat64Op └ *colexec.selLEInt64Int64ConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan # Query 2 query T @@ -559,18 +559,18 @@ EXPLAIN (VEC) SELECT s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_ │ ├ *rowexec.joinReader │ │ └ *colexec.selSuffixBytesBytesConstOp │ │ └ *colexec.selEQInt64Int64ConstOp - │ │ └ *colfetcher.colBatchScan + │ │ └ *colfetcher.ColBatchScan │ └ *colexec.hashJoiner - │ ├ *colfetcher.colBatchScan + │ ├ *colfetcher.ColBatchScan │ └ *colexec.hashJoiner - │ ├ *colfetcher.colBatchScan + │ ├ *colfetcher.ColBatchScan │ └ *colexec.selEQBytesBytesConstOp - │ └ *colfetcher.colBatchScan + │ └ *colfetcher.ColBatchScan └ *rowexec.joinReader └ *colexec.mergeJoinInnerOp - ├ *colfetcher.colBatchScan + ├ *colfetcher.ColBatchScan └ *colexec.selEQBytesBytesConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan # Query 3 query T @@ -583,12 +583,12 @@ EXPLAIN (VEC) SELECT l_orderkey, sum(l_extendedprice * (1 - l_discount)) AS reve └ *colexec.hashAggregator └ *rowexec.joinReader └ *colexec.hashJoiner - ├ *colfetcher.colBatchScan + ├ *colfetcher.ColBatchScan └ *colexec.hashJoiner ├ *colexec.selLTInt64Int64ConstOp - │ └ *colfetcher.colBatchScan + │ └ *colfetcher.ColBatchScan └ *colexec.selEQBytesBytesConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan # Query 4 query T @@ -600,7 +600,7 @@ EXPLAIN (VEC) SELECT o_orderpriority, count(*) AS order_count FROM orders WHERE └ *colexec.hashAggregator └ *rowexec.joinReader └ *rowexec.indexJoiner - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan # Query 5 query T @@ -616,13 +616,13 @@ EXPLAIN (VEC) SELECT n_name, sum(l_extendedprice * (1 - l_discount)) AS revenue ├ *rowexec.joinReader │ └ *colexec.hashJoiner │ ├ *rowexec.indexJoiner - │ │ └ *colfetcher.colBatchScan + │ │ └ *colfetcher.ColBatchScan │ └ *rowexec.joinReader │ └ *colexec.hashJoiner - │ ├ *colfetcher.colBatchScan + │ ├ *colfetcher.ColBatchScan │ └ *colexec.selEQBytesBytesConstOp - │ └ *colfetcher.colBatchScan - └ *colfetcher.colBatchScan + │ └ *colfetcher.ColBatchScan + └ *colfetcher.ColBatchScan # Query 6 query T @@ -633,7 +633,7 @@ EXPLAIN (VEC) SELECT sum(l_extendedprice * l_discount) AS revenue FROM lineitem └ *colexec.orderedAggregator └ *colexec.distinctChainOps └ *rowexec.indexJoiner - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan # Query 7 query T @@ -648,7 +648,7 @@ EXPLAIN (VEC) SELECT supp_nation, cust_nation, l_year, sum(volume) AS revenue FR └ *colexec.defaultBuiltinFuncOperator └ *colexec.constBytesOp └ *colexec.hashJoiner - ├ *colfetcher.colBatchScan + ├ *colfetcher.ColBatchScan └ *rowexec.joinReader └ *rowexec.joinReader └ *rowexec.joinReader @@ -656,8 +656,8 @@ EXPLAIN (VEC) SELECT supp_nation, cust_nation, l_year, sum(volume) AS revenue FR └ *colexec.caseOp ├ *colexec.bufferOp │ └ *colexec.hashJoiner - │ ├ *colfetcher.colBatchScan - │ └ *colfetcher.colBatchScan + │ ├ *colfetcher.ColBatchScan + │ └ *colfetcher.ColBatchScan ├ *colexec.constBoolOp │ └ *colexec.andProjOp │ ├ *colexec.bufferOp @@ -688,19 +688,19 @@ EXPLAIN (VEC) SELECT o_year, sum(CASE WHEN nation = 'BRAZIL' THEN volume ELSE 0 │ └ *colexec.constBytesOp │ └ *colexec.hashJoiner │ ├ *colexec.hashJoiner - │ │ ├ *colfetcher.colBatchScan + │ │ ├ *colfetcher.ColBatchScan │ │ └ *colexec.hashJoiner │ │ ├ *rowexec.joinReader │ │ │ └ *colexec.mergeJoinInnerOp - │ │ │ ├ *colfetcher.colBatchScan + │ │ │ ├ *colfetcher.ColBatchScan │ │ │ └ *colexec.selEQBytesBytesConstOp - │ │ │ └ *colfetcher.colBatchScan + │ │ │ └ *colfetcher.ColBatchScan │ │ └ *rowexec.joinReader │ │ └ *rowexec.joinReader │ │ └ *rowexec.joinReader │ │ └ *colexec.selEQBytesBytesConstOp - │ │ └ *colfetcher.colBatchScan - │ └ *colfetcher.colBatchScan + │ │ └ *colfetcher.ColBatchScan + │ └ *colfetcher.ColBatchScan ├ *colexec.projEQBytesBytesConstOp │ └ *colexec.bufferOp └ *colexec.constFloat64Op @@ -722,15 +722,15 @@ EXPLAIN (VEC) SELECT nation, o_year, sum(amount) AS sum_profit FROM ( SELECT n_n └ *colexec.constBytesOp └ *colexec.hashJoiner ├ *colexec.hashJoiner - │ ├ *colfetcher.colBatchScan + │ ├ *colfetcher.ColBatchScan │ └ *rowexec.joinReader │ └ *rowexec.joinReader │ └ *rowexec.joinReader │ └ *colexec.mergeJoinInnerOp │ ├ *colexec.selRegexpBytesBytesConstOp - │ │ └ *colfetcher.colBatchScan - │ └ *colfetcher.colBatchScan - └ *colfetcher.colBatchScan + │ │ └ *colfetcher.ColBatchScan + │ └ *colfetcher.ColBatchScan + └ *colfetcher.ColBatchScan # Query 10 query T @@ -746,10 +746,10 @@ EXPLAIN (VEC) SELECT c_custkey, c_name, sum(l_extendedprice * (1 - l_discount)) └ *colexec.hashJoiner ├ *rowexec.joinReader │ └ *colexec.hashJoiner - │ ├ *colfetcher.colBatchScan + │ ├ *colfetcher.ColBatchScan │ └ *rowexec.indexJoiner - │ └ *colfetcher.colBatchScan - └ *colfetcher.colBatchScan + │ └ *colfetcher.ColBatchScan + └ *colfetcher.ColBatchScan # Query 11 query T @@ -766,7 +766,7 @@ EXPLAIN (VEC) SELECT ps_partkey, sum(ps_supplycost * ps_availqty::float) AS valu └ *rowexec.joinReader └ *rowexec.joinReader └ *colexec.selEQBytesBytesConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan # Query 12 query T @@ -778,7 +778,7 @@ EXPLAIN (VEC) SELECT l_shipmode, sum(CASE WHEN o_orderpriority = '1-URGENT' or o └ *colexec.hashAggregator └ *rowexec.joinReader └ *rowexec.indexJoiner - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan # Query 13 query T @@ -791,8 +791,8 @@ EXPLAIN (VEC) SELECT c_count, count(*) AS custdist FROM ( SELECT c_custkey, coun └ *colexec.hashAggregator └ *colexec.hashJoiner ├ *colexec.selNotRegexpBytesBytesConstOp - │ └ *colfetcher.colBatchScan - └ *colfetcher.colBatchScan + │ └ *colfetcher.ColBatchScan + └ *colfetcher.ColBatchScan # Query 14 query T @@ -809,9 +809,9 @@ EXPLAIN (VEC) SELECT 100.00 * sum(CASE WHEN p_type LIKE 'PROMO%' THEN l_extended └ *colexec.caseOp ├ *colexec.bufferOp │ └ *colexec.hashJoiner - │ ├ *colfetcher.colBatchScan + │ ├ *colfetcher.ColBatchScan │ └ *rowexec.indexJoiner - │ └ *colfetcher.colBatchScan + │ └ *colfetcher.ColBatchScan ├ *colexec.projMultFloat64Float64Op │ └ *colexec.projMinusFloat64ConstFloat64Op │ └ *colexec.projPrefixBytesBytesConstOp @@ -829,14 +829,14 @@ EXPLAIN (VEC) SELECT s_suppkey, s_name, s_address, s_phone, total_revenue FROM s │ └ Node 1 └ *colexec.mergeJoinInnerOp - ├ *colfetcher.colBatchScan + ├ *colfetcher.ColBatchScan └ *colexec.sortOp └ *colexec.selEQFloat64Float64Op └ *colexec.castOpNullAny └ *colexec.constNullOp └ *colexec.hashAggregator └ *rowexec.indexJoiner - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan statement ok DROP VIEW revenue0 @@ -852,13 +852,13 @@ EXPLAIN (VEC) SELECT p_brand, p_type, p_size, count(distinct ps_suppkey) AS supp └ *colexec.unorderedDistinct └ *colexec.hashJoiner ├ *colexec.mergeJoinLeftAntiOp - │ ├ *colfetcher.colBatchScan + │ ├ *colfetcher.ColBatchScan │ └ *colexec.selRegexpBytesBytesConstOp - │ └ *colfetcher.colBatchScan + │ └ *colfetcher.ColBatchScan └ *colexec.selectInOpInt64 └ *colexec.selNotPrefixBytesBytesConstOp └ *colexec.selNEBytesBytesConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan # Query 17 query T @@ -878,7 +878,7 @@ EXPLAIN (VEC) SELECT sum(l_extendedprice) / 7.0 AS avg_yearly FROM lineitem, par └ *rowexec.joinReader └ *colexec.selEQBytesBytesConstOp └ *colexec.selEQBytesBytesConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan # Query 18 query T @@ -890,15 +890,15 @@ EXPLAIN (VEC) SELECT c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, s └ *colexec.topKSorter └ *colexec.hashAggregator └ *colexec.hashJoiner - ├ *colfetcher.colBatchScan + ├ *colfetcher.ColBatchScan └ *colexec.hashJoiner ├ *colexec.mergeJoinLeftSemiOp - │ ├ *colfetcher.colBatchScan + │ ├ *colfetcher.ColBatchScan │ └ *colexec.selGTFloat64Float64ConstOp │ └ *colexec.orderedAggregator │ └ *colexec.distinctChainOps - │ └ *colfetcher.colBatchScan - └ *colfetcher.colBatchScan + │ └ *colfetcher.ColBatchScan + └ *colfetcher.ColBatchScan # Query 19 query T @@ -915,9 +915,9 @@ EXPLAIN (VEC) SELECT sum(l_extendedprice* (1 - l_discount)) AS revenue FROM line │ └ *colexec.hashJoiner │ ├ *colexec.selEQBytesBytesConstOp │ │ └ *colexec.selectInOpBytes - │ │ └ *colfetcher.colBatchScan + │ │ └ *colfetcher.ColBatchScan │ └ *colexec.selGEInt64Int64ConstOp - │ └ *colfetcher.colBatchScan + │ └ *colfetcher.ColBatchScan ├ *colexec.constBoolOp │ └ *colexec.orProjOp │ ├ *colexec.bufferOp @@ -962,19 +962,19 @@ EXPLAIN (VEC) SELECT s_name, s_address FROM supplier, nation WHERE s_suppkey IN └ *colexec.sortOp └ *colexec.hashJoiner ├ *colexec.hashJoiner - │ ├ *colfetcher.colBatchScan + │ ├ *colfetcher.ColBatchScan │ └ *colexec.hashJoiner │ ├ *colexec.selGTInt64Float64Op │ │ └ *colexec.projMultFloat64Float64ConstOp │ │ └ *colexec.hashAggregator │ │ └ *colexec.hashJoiner │ │ ├ *rowexec.indexJoiner - │ │ │ └ *colfetcher.colBatchScan - │ │ └ *colfetcher.colBatchScan + │ │ │ └ *colfetcher.ColBatchScan + │ │ └ *colfetcher.ColBatchScan │ └ *colexec.selPrefixBytesBytesConstOp - │ └ *colfetcher.colBatchScan + │ └ *colfetcher.ColBatchScan └ *colexec.selEQBytesBytesConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan # Query 21 query T @@ -990,14 +990,14 @@ EXPLAIN (VEC) SELECT s_name, count(*) AS numwait FROM supplier, lineitem l1, ord ├ *rowexec.hashJoiner │ ├ *rowexec.mergeJoiner │ │ ├ *colexec.selGTInt64Int64Op - │ │ │ └ *colfetcher.colBatchScan + │ │ │ └ *colfetcher.ColBatchScan │ │ └ *colexec.selGTInt64Int64Op - │ │ └ *colfetcher.colBatchScan - │ └ *colfetcher.colBatchScan + │ │ └ *colfetcher.ColBatchScan + │ └ *colfetcher.ColBatchScan └ *rowexec.joinReader └ *rowexec.joinReader └ *colexec.selEQBytesBytesConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan # Query 22 query T @@ -1015,4 +1015,4 @@ EXPLAIN (VEC) SELECT cntrycode, count(*) AS numcust, sum(c_acctbal) AS totacctba └ *colexec.substringInt64Int64Operator └ *colexec.constInt64Op └ *colexec.constInt64Op - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan diff --git a/pkg/sql/logictest/testdata/logic_test/vectorize b/pkg/sql/logictest/testdata/logic_test/vectorize index 41e1f6424391..45dff41f72c7 100644 --- a/pkg/sql/logictest/testdata/logic_test/vectorize +++ b/pkg/sql/logictest/testdata/logic_test/vectorize @@ -1241,3 +1241,22 @@ query T SELECT c0 FROM t47715 ORDER by c1 ---- 1819487610 + +# Ensure that the vectorized engine can plan and produce the MVCC system column. +statement ok +RESET vectorize; +CREATE TABLE mvcc (x INT PRIMARY KEY, y INT, z INT, INDEX i (z)); +INSERT INTO mvcc VALUES (1, 2, 3) + +statement ok +SET vectorize = experimental_always + +query B +SELECT crdb_internal_mvcc_timestamp IS NOT NULL FROM mvcc +---- +true + +query B +SELECT crdb_internal_mvcc_timestamp IS NOT NULL FROM mvcc@i +---- +true diff --git a/pkg/sql/logictest/testdata/logic_test/vectorize_agg b/pkg/sql/logictest/testdata/logic_test/vectorize_agg index 347b616e1cfb..764e0647aa34 100644 --- a/pkg/sql/logictest/testdata/logic_test/vectorize_agg +++ b/pkg/sql/logictest/testdata/logic_test/vectorize_agg @@ -40,7 +40,7 @@ EXPLAIN (VEC) SELECT concat_agg(_bytes), concat_agg(_string) FROM bytes_string G │ └ Node 1 └ *colexec.hashAggregator - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T EXPLAIN (VEC) SELECT concat_agg(_bytes), concat_agg(_string) FROM bytes_string @@ -49,7 +49,7 @@ EXPLAIN (VEC) SELECT concat_agg(_bytes), concat_agg(_string) FROM bytes_string └ Node 1 └ *colexec.orderedAggregator └ *colexec.distinctChainOps - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan statement ok SET vectorize=experimental_always diff --git a/pkg/sql/logictest/testdata/logic_test/vectorize_local b/pkg/sql/logictest/testdata/logic_test/vectorize_local index 4296ea912401..4e55cc0aac0d 100644 --- a/pkg/sql/logictest/testdata/logic_test/vectorize_local +++ b/pkg/sql/logictest/testdata/logic_test/vectorize_local @@ -115,9 +115,9 @@ EXPLAIN (VEC) SELECT t46122_0.c0 FROM t46122_0, t46122_1 ---- │ └ Node 1 -└ *rowexec.hashJoiner - ├ *colfetcher.colBatchScan - └ *colfetcher.colBatchScan + └ *rowexec.hashJoiner + ├ *colfetcher.ColBatchScan + └ *colfetcher.ColBatchScan # Regression test for #46404 (rowexec.noopProcessor not implementing # execinfra.OpNode interface). @@ -129,11 +129,11 @@ EXPLAIN (VEC) SELECT stddev((t46404_1.c0 > ANY (0, 0))::INT) FROM t46404_0, t464 ---- │ └ Node 1 -└ *rowexec.hashAggregator - └ *rowexec.noopProcessor - └ *colexec.hashJoiner - ├ *colfetcher.colBatchScan - └ *colfetcher.colBatchScan + └ *rowexec.hashAggregator + └ *rowexec.noopProcessor + └ *colexec.hashJoiner + ├ *colfetcher.ColBatchScan + └ *colfetcher.ColBatchScan statement ok CREATE TABLE xyz ( @@ -150,8 +150,8 @@ EXPLAIN (VEC) SELECT * FROM xyz AS t1 FULL OUTER JOIN xyz AS t2 ON t1.x = t2.x A │ └ Node 1 └ *rowexec.hashJoiner - ├ *colfetcher.colBatchScan - └ *colfetcher.colBatchScan + ├ *colfetcher.ColBatchScan + └ *colfetcher.ColBatchScan # Verify that the vectorized engine is used (there is a mismatch between # argument type width and the result). @@ -162,7 +162,7 @@ EXPLAIN (VEC) SELECT max(c) FROM a └ Node 1 └ *colexec.orderedAggregator └ *colexec.distinctChainOps - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan # Verify that binary operations on integers of any width return INT8. statement ok @@ -187,7 +187,7 @@ EXPLAIN (VEC) SELECT _int2 * _int2 FROM ints WHERE _int4 + _int4 = _int8 + 2 └ *colexec.projPlusInt64Int64Op └ *colexec.castInt32Int64Op └ *colexec.castInt32Int64Op - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query I SELECT _int2 * _int2 FROM ints WHERE _int4 + _int4 = _int8 + 2 @@ -204,7 +204,7 @@ EXPLAIN (VEC) SELECT c.a FROM c JOIN d ON d.b = c.b │ └ Node 1 └ *rowexec.joinReader - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan statement ok SET vectorize = experimental_always diff --git a/pkg/sql/logictest/testdata/logic_test/vectorize_overloads b/pkg/sql/logictest/testdata/logic_test/vectorize_overloads index c2b0fe9b5a12..0a5c1bc4416d 100644 --- a/pkg/sql/logictest/testdata/logic_test/vectorize_overloads +++ b/pkg/sql/logictest/testdata/logic_test/vectorize_overloads @@ -112,7 +112,7 @@ EXPLAIN (VEC) SELECT _inet & _inet FROM many_types │ └ Node 1 └ *colexec.projBitandDatumDatumOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T rowsort SELECT _inet & _inet FROM many_types @@ -129,7 +129,7 @@ EXPLAIN (VEC) SELECT _inet - _int2 FROM many_types └ Node 1 └ *colexec.projMinusDatumInt64Op └ *colexec.castInt16Int64Op - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T EXPLAIN (VEC) SELECT _int2^_int4 FROM many_types @@ -139,7 +139,7 @@ EXPLAIN (VEC) SELECT _int2^_int4 FROM many_types └ *colexec.projPowInt64Int64Op └ *colexec.castInt32Int64Op └ *colexec.castInt16Int64Op - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T EXPLAIN (VEC) SELECT _int2^_int FROM many_types @@ -148,7 +148,7 @@ EXPLAIN (VEC) SELECT _int2^_int FROM many_types └ Node 1 └ *colexec.projPowInt64Int64Op └ *colexec.castInt16Int64Op - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T EXPLAIN (VEC) SELECT _float^_float FROM many_types @@ -156,7 +156,7 @@ EXPLAIN (VEC) SELECT _float^_float FROM many_types │ └ Node 1 └ *colexec.projPowFloat64Float64Op - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T EXPLAIN (VEC) SELECT _decimal^_int4 FROM many_types @@ -165,7 +165,7 @@ EXPLAIN (VEC) SELECT _decimal^_int4 FROM many_types └ Node 1 └ *colexec.projPowDecimalInt64Op └ *colexec.castInt32Int64Op - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query R rowsort SELECT _float^_float FROM many_types @@ -197,7 +197,7 @@ EXPLAIN (VEC) SELECT _inet - 1 FROM many_types │ └ Node 1 └ *colexec.projMinusDatumInt64ConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T rowsort SELECT _inet - 1 FROM many_types @@ -214,7 +214,7 @@ EXPLAIN (VEC) SELECT _int4 + _inet FROM many_types └ Node 1 └ *colexec.projPlusDatumInt64Op └ *colexec.castInt32Int64Op - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T rowsort SELECT _int4 + _inet FROM many_types @@ -230,7 +230,7 @@ EXPLAIN (VEC) SELECT 2 + _inet FROM many_types │ └ Node 1 └ *colexec.projPlusDatumInt64ConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T rowsort SELECT 2 + _inet FROM many_types @@ -246,7 +246,7 @@ EXPLAIN (VEC) SELECT _time + _interval FROM many_types │ └ Node 1 └ *colexec.projPlusDatumIntervalOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T rowsort SELECT _time + _interval FROM many_types @@ -262,7 +262,7 @@ EXPLAIN (VEC) SELECT _json - _int FROM many_types │ └ Node 1 └ *colexec.projMinusDatumInt64Op - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T rowsort SELECT _json - _int FROM many_types @@ -278,7 +278,7 @@ EXPLAIN (VEC) SELECT _bytes || _bytes FROM many_types │ └ Node 1 └ *colexec.projConcatBytesBytesOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T SELECT _bytes || _bytes FROM many_types @@ -294,7 +294,7 @@ EXPLAIN (VEC) SELECT _string || _string FROM many_types │ └ Node 1 └ *colexec.projConcatBytesBytesOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T SELECT _string || _string FROM many_types @@ -310,7 +310,7 @@ EXPLAIN (VEC) SELECT _json || _json FROM many_types │ └ Node 1 └ *colexec.projConcatDatumDatumOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T SELECT _json || _json FROM many_types @@ -326,7 +326,7 @@ EXPLAIN (VEC) SELECT _varbit || _varbit FROM many_types │ └ Node 1 └ *colexec.projConcatDatumDatumOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T SELECT _varbit || _varbit FROM many_types @@ -342,7 +342,7 @@ EXPLAIN (VEC) SELECT _int << 1 FROM many_types │ └ Node 1 └ *colexec.projLShiftInt64Int64ConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T EXPLAIN (VEC) SELECT _int >> 1 FROM many_types @@ -350,7 +350,7 @@ EXPLAIN (VEC) SELECT _int >> 1 FROM many_types │ └ Node 1 └ *colexec.projRShiftInt64Int64ConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query I rowsort SELECT _int2 >> 1 FROM many_types @@ -388,7 +388,7 @@ EXPLAIN (VEC) SELECT _varbit << 4 FROM many_types │ └ Node 1 └ *colexec.projLShiftDatumInt64ConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T EXPLAIN (VEC) SELECT _varbit << _int2 FROM many_types @@ -397,7 +397,7 @@ EXPLAIN (VEC) SELECT _varbit << _int2 FROM many_types └ Node 1 └ *colexec.projLShiftDatumInt64Op └ *colexec.castInt16Int64Op - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T EXPLAIN (VEC) SELECT _varbit << _int4 FROM many_types @@ -406,7 +406,7 @@ EXPLAIN (VEC) SELECT _varbit << _int4 FROM many_types └ Node 1 └ *colexec.projLShiftDatumInt64Op └ *colexec.castInt32Int64Op - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T EXPLAIN (VEC) SELECT _varbit << _int FROM many_types @@ -414,7 +414,7 @@ EXPLAIN (VEC) SELECT _varbit << _int FROM many_types │ └ Node 1 └ *colexec.projLShiftDatumInt64Op - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T @@ -423,7 +423,7 @@ EXPLAIN (VEC) SELECT _varbit >> 4 FROM many_types │ └ Node 1 └ *colexec.projRShiftDatumInt64ConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T EXPLAIN (VEC) SELECT _varbit >> _int2 FROM many_types @@ -432,7 +432,7 @@ EXPLAIN (VEC) SELECT _varbit >> _int2 FROM many_types └ Node 1 └ *colexec.projRShiftDatumInt64Op └ *colexec.castInt16Int64Op - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T EXPLAIN (VEC) SELECT _varbit >> _int4 FROM many_types @@ -441,7 +441,7 @@ EXPLAIN (VEC) SELECT _varbit >> _int4 FROM many_types └ Node 1 └ *colexec.projRShiftDatumInt64Op └ *colexec.castInt32Int64Op - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T EXPLAIN (VEC) SELECT _varbit >> _int FROM many_types @@ -449,7 +449,7 @@ EXPLAIN (VEC) SELECT _varbit >> _int FROM many_types │ └ Node 1 └ *colexec.projRShiftDatumInt64Op - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T SELECT _varbit >> 1 FROM many_types @@ -482,7 +482,7 @@ EXPLAIN (VEC) SELECT _json -> _int2 FROM many_types └ Node 1 └ *colexec.projJSONFetchValDatumInt64Op └ *colexec.castInt16Int64Op - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query I rowsort SELECT _int2^_int FROM many_types WHERE _int2 < 10 AND _int < 10 @@ -514,7 +514,7 @@ EXPLAIN (VEC) SELECT _json -> _int4 FROM many_types └ Node 1 └ *colexec.projJSONFetchValDatumInt64Op └ *colexec.castInt32Int64Op - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T SELECT _json -> _int4 FROM many_types @@ -530,7 +530,7 @@ EXPLAIN (VEC) SELECT _json -> _int FROM many_types │ └ Node 1 └ *colexec.projJSONFetchValDatumInt64Op - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T SELECT _json -> _int FROM many_types @@ -546,7 +546,7 @@ EXPLAIN (VEC) SELECT _json -> 2 FROM many_types │ └ Node 1 └ *colexec.projJSONFetchValDatumInt64ConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T SELECT _json -> 2 FROM many_types @@ -563,7 +563,7 @@ EXPLAIN (VEC) SELECT _json -> 2 -> 'a' FROM many_types └ Node 1 └ *colexec.projJSONFetchValDatumBytesConstOp └ *colexec.projJSONFetchValDatumInt64ConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T SELECT _json -> 2 -> 'a' FROM many_types @@ -594,7 +594,7 @@ EXPLAIN (VEC) SELECT '[2, "hi", {"b": ["bar", {"c": 4}]}]'::jsonb -> _int FROM m │ └ Node 1 └ *colexec.projJSONFetchValDatumConstInt64Op - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query T rowsort SELECT '[2, "hi", {"b": ["bar", {"c": 4}]}]'::jsonb -> _int FROM many_types @@ -613,7 +613,7 @@ EXPLAIN (VEC) SELECT B'11' <> _varbit FROM many_types │ └ Node 1 └ *colexec.projNEDatumDatumConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query B rowsort SELECT B'11' >= _varbit FROM many_types @@ -631,7 +631,7 @@ EXPLAIN (VEC) SELECT _int4 // _int FROM many_types WHERE _int <> 0 └ *colexec.projFloorDivInt64Int64Op └ *colexec.castInt32Int64Op └ *colexec.selNEInt64Int64ConstOp - └ *colfetcher.colBatchScan + └ *colfetcher.ColBatchScan query III rowsort SELECT _int, _int2, _int // _int2 FROM many_types WHERE _int2 <> 0 diff --git a/pkg/sql/opt/cat/column.go b/pkg/sql/opt/cat/column.go index 8a8168968da4..f96b23cbfc9c 100644 --- a/pkg/sql/opt/cat/column.go +++ b/pkg/sql/opt/cat/column.go @@ -28,6 +28,11 @@ const ( // DeleteOnly columns are mutation columns that have to be updated only on // deletes and cannot be otherwise accessed. DeleteOnly + // System columns are implicit columns that every physical table + // contains. These columns can only be read from and must not be included + // as part of mutations. They also cannot be part of the lax or key columns + // for indexes. System columns are not members of any column family. + System ) // Column is an interface to a table column, exposing only the information @@ -103,3 +108,9 @@ func IsMutationColumn(table Table, ord int) bool { kind := table.ColumnKind(ord) return kind == WriteOnly || kind == DeleteOnly } + +// IsSystemColumn is a convenience function that returns true if the column at +// the given ordinal position is a system column. +func IsSystemColumn(table Table, ord int) bool { + return table.ColumnKind(ord) == System +} diff --git a/pkg/sql/opt/cat/index.go b/pkg/sql/opt/cat/index.go index a6f37acc592e..8a39f7bcc8bd 100644 --- a/pkg/sql/opt/cat/index.go +++ b/pkg/sql/opt/cat/index.go @@ -54,7 +54,9 @@ type Index interface { // ColumnCount returns the number of columns in the index. This includes // columns that were part of the index definition (including the STORING - // clause), as well as implicitly added primary key columns. + // clause), as well as implicitly added primary key columns. It also contains + // implicit system columns, which are placed after all physical columns in + // the table. ColumnCount() int // Predicate returns the partial index predicate expression and true if the diff --git a/pkg/sql/opt/cat/utils.go b/pkg/sql/opt/cat/utils.go index 61d967adcc18..5f3ebc2f6ae7 100644 --- a/pkg/sql/opt/cat/utils.go +++ b/pkg/sql/opt/cat/utils.go @@ -132,7 +132,7 @@ func FormatTable(cat Catalog, tab Table, tp treeprinter.Node) { var buf bytes.Buffer for i := 0; i < tab.ColumnCount(); i++ { buf.Reset() - formatColumn(tab.Column(i), IsMutationColumn(tab, i), &buf) + formatColumn(tab.Column(i), tab.ColumnKind(i), &buf) child.Child(buf.String()) } @@ -189,7 +189,7 @@ func formatCatalogIndex(tab Table, ord int, tp treeprinter.Node) { buf.Reset() idxCol := idx.Column(i) - formatColumn(idxCol.Column, false /* isMutationCol */, &buf) + formatColumn(idxCol.Column, tab.ColumnKind(i), &buf) if idxCol.Descending { fmt.Fprintf(&buf, " desc") } @@ -284,7 +284,7 @@ func formatCatalogFKRef( ) } -func formatColumn(col Column, isMutationCol bool, buf *bytes.Buffer) { +func formatColumn(col Column, kind ColumnKind, buf *bytes.Buffer) { fmt.Fprintf(buf, "%s %s", col.ColName(), col.DatumType()) if !col.IsNullable() { fmt.Fprintf(buf, " not null") @@ -298,8 +298,11 @@ func formatColumn(col Column, isMutationCol bool, buf *bytes.Buffer) { if col.IsHidden() { fmt.Fprintf(buf, " [hidden]") } - if isMutationCol { + switch kind { + case WriteOnly, DeleteOnly: fmt.Fprintf(buf, " [mutation]") + case System: + fmt.Fprintf(buf, " [system]") } } diff --git a/pkg/sql/opt/constraint/columns_test.go b/pkg/sql/opt/constraint/columns_test.go index 51396b6faf7c..e7ff10a7fd54 100644 --- a/pkg/sql/opt/constraint/columns_test.go +++ b/pkg/sql/opt/constraint/columns_test.go @@ -45,7 +45,7 @@ func TestColumns_RemapColumns(t *testing.T) { t.Errorf("\noriginal Columns were changed: %s", originalColumns.String()) } - expected = "/5/-7/8" + expected = "/6/-8/9" if remappedColumns.String() != expected { t.Errorf("\nexpected: %s\nactual: %s\n", expected, remappedColumns.String()) } diff --git a/pkg/sql/opt/exec/execbuilder/mutation.go b/pkg/sql/opt/exec/execbuilder/mutation.go index 755035016e46..831763192175 100644 --- a/pkg/sql/opt/exec/execbuilder/mutation.go +++ b/pkg/sql/opt/exec/execbuilder/mutation.go @@ -704,7 +704,8 @@ func mutationOutputColMap(mutation memo.RelExpr) opt.ColMap { ord := 0 for i, n := 0, tab.ColumnCount(); i < n; i++ { colID := private.Table.ColumnID(i) - if outCols.Contains(colID) { + // System columns should not be included in mutations. + if outCols.Contains(colID) && !cat.IsSystemColumn(tab, i) { colMap.Set(int(colID), ord) ord++ } diff --git a/pkg/sql/opt/exec/execbuilder/testdata/aggregate b/pkg/sql/opt/exec/execbuilder/testdata/aggregate index cdd1de4b6697..5369ffe77d45 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/aggregate +++ b/pkg/sql/opt/exec/execbuilder/testdata/aggregate @@ -14,19 +14,19 @@ EXPLAIN (TYPES) SELECT min(1), max(1), count(NULL), sum_int(1), avg(1), sum(1), · distribution local · · · vectorized true · · group · · (min int, max int, count int, sum_int int, avg decimal, sum decimal, stddev decimal, variance decimal, bool_and bool, bool_or bool, xor_agg bytes) · - │ aggregate 0 min(column5) · · - │ aggregate 1 max(column5) · · - │ aggregate 2 count(column8) · · - │ aggregate 3 sum_int(column5) · · - │ aggregate 4 avg(column5) · · - │ aggregate 5 sum(column5) · · - │ aggregate 6 stddev(column5) · · - │ aggregate 7 variance(column5) · · - │ aggregate 8 bool_and(column15) · · - │ aggregate 9 bool_or(column17) · · - │ aggregate 10 xor_agg(column19) · · + │ aggregate 0 min(column6) · · + │ aggregate 1 max(column6) · · + │ aggregate 2 count(column9) · · + │ aggregate 3 sum_int(column6) · · + │ aggregate 4 avg(column6) · · + │ aggregate 5 sum(column6) · · + │ aggregate 6 stddev(column6) · · + │ aggregate 7 variance(column6) · · + │ aggregate 8 bool_and(column16) · · + │ aggregate 9 bool_or(column18) · · + │ aggregate 10 xor_agg(column20) · · │ scalar · · · - └── render · · (column5 int, column8 unknown, column15 bool, column17 bool, column19 bytes) · + └── render · · (column6 int, column9 unknown, column16 bool, column18 bool, column20 bytes) · │ render 0 (1)[int] · · │ render 1 (NULL)[unknown] · · │ render 2 (true)[bool] · · @@ -57,15 +57,15 @@ render · · (min int, max │ aggregate 0 min(v) · · │ aggregate 1 max(v) · · │ aggregate 2 count(v) · · - │ aggregate 3 sum_int(column8) · · + │ aggregate 3 sum_int(column9) · · │ aggregate 4 avg(v) · · │ aggregate 5 sum(v) · · │ aggregate 6 stddev(v) · · │ aggregate 7 variance(v) · · - │ aggregate 8 bool_and(column14) · · - │ aggregate 9 xor_agg(column16) · · + │ aggregate 8 bool_and(column15) · · + │ aggregate 9 xor_agg(column17) · · │ scalar · · · - └── render · · (column8 int, column14 bool, column16 bytes, v int) · + └── render · · (column9 int, column15 bool, column17 bytes, v int) · │ render 0 (1)[int] · · │ render 1 ((v)[int] = (1)[int])[bool] · · │ render 2 ((s)[string]::BYTES)[bytes] · · @@ -137,12 +137,12 @@ EXPLAIN (TYPES) SELECT count(*), k+v AS r FROM kv GROUP BY k+v · vectorized true · · render · · (count int, r int) · │ render 0 (count_rows)[int] · · - │ render 1 (column6)[int] · · - └── group · · (column6 int, count_rows int) · - │ aggregate 0 column6 · · + │ render 1 (column7)[int] · · + └── group · · (column7 int, count_rows int) · + │ aggregate 0 column7 · · │ aggregate 1 count_rows() · · - │ group by column6 · · - └── render · · (column6 int) · + │ group by column7 · · + └── render · · (column7 int) · │ render 0 ((k)[int] + (v)[int])[int] · · └── scan · · (k int, v int) · · table kv@primary · · @@ -216,11 +216,11 @@ EXPLAIN (VERBOSE) SELECT count(DISTINCT a.*) FROM kv a, kv b · distribution local · · · vectorized true · · group · · (count) · - │ aggregate 0 count(column9) · · + │ aggregate 0 count(column11) · · │ scalar · · · - └── distinct · · (column9) · - │ distinct on column9 · · - └── render · · (column9) · + └── distinct · · (column11) · + │ distinct on column11 · · + └── render · · (column11) · │ render 0 ((k, v, w, s) AS k, v, w, s) · · └── cross-join · · (k, v, w, s) · │ type cross · · @@ -624,10 +624,10 @@ render · · (v int, count int) │ order +count_rows · · └── group · · (v int, count int, count_rows int) · │ aggregate 0 v · · - │ aggregate 1 count(column5) · · + │ aggregate 1 count(column6) · · │ aggregate 2 count_rows() · · │ group by v · · - └── render · · (column5 unknown, v int) · + └── render · · (column6 unknown, v int) · │ render 0 (NULL)[unknown] · · │ render 1 (v)[int] · · └── scan · · (v int) · @@ -642,9 +642,9 @@ EXPLAIN (VERBOSE) SELECT * FROM (SELECT v, count(NULL) FROM kv GROUP BY v) WHERE · vectorized true · · group · · (v, count) · │ aggregate 0 v · · - │ aggregate 1 count(column5) · · + │ aggregate 1 count(column6) · · │ group by v · · - └── render · · (column5, v) · + └── render · · (column6, v) · │ render 0 NULL · · │ render 1 v · · └── filter · · (v) · @@ -673,10 +673,10 @@ render · · (cou │ render 1 max · · └── group · · (v, count, max) · │ aggregate 0 v · · - │ aggregate 1 count(column5) FILTER (WHERE column6) · · - │ aggregate 2 max(column6) FILTER (WHERE column6) · · + │ aggregate 1 count(column6) FILTER (WHERE column7) · · + │ aggregate 2 max(column7) FILTER (WHERE column7) · · │ group by v · · - └── render · · (column5, column6, v) · + └── render · · (column6, column7, v) · │ render 0 true · · │ render 1 k > 5 · · │ render 2 v · · @@ -693,9 +693,9 @@ render · · (cou │ render 0 count · · └── group · · (v, count) · │ aggregate 0 v · · - │ aggregate 1 count(column5) FILTER (WHERE column6) · · + │ aggregate 1 count(column6) FILTER (WHERE column7) · · │ group by v · · - └── render · · (column5, column6, v) · + └── render · · (column6, column7, v) · │ render 0 true · · │ render 1 k > 5 · · │ render 2 v · · @@ -783,9 +783,9 @@ EXPLAIN (TYPES) SELECT min(v+1) FROM opt_test · distribution local · · · vectorized true · · group · · (min int) · - │ aggregate 0 min(column3) · · + │ aggregate 0 min(column4) · · │ scalar · · · - └── render · · (column3 int) · + └── render · · (column4 int) · │ render 0 ((v)[int] + (1)[int])[int] · · └── scan · · (v int) · · table opt_test@primary · · @@ -967,11 +967,11 @@ EXPLAIN (TYPES) SELECT 1 a FROM kv GROUP BY v, w::DECIMAL HAVING w::DECIMAL > 1; · vectorized true · · render · · (a int) · │ render 0 (1)[int] · · - └── distinct · · (column5 decimal, v int) · - │ distinct on column5, v · · - └── filter · · (column5 decimal, v int) · - │ filter ((column5)[decimal] > (1)[decimal])[bool] · · - └── render · · (column5 decimal, v int) · + └── distinct · · (column6 decimal, v int) · + │ distinct on column6, v · · + └── filter · · (column6 decimal, v int) · + │ filter ((column6)[decimal] > (1)[decimal])[bool] · · + └── render · · (column6 decimal, v int) · │ render 0 ((w)[int]::DECIMAL)[decimal] · · │ render 1 (v)[int] · · └── scan · · (v int, w int) · @@ -989,11 +989,11 @@ EXPLAIN (VERBOSE) SELECT min(a) AS m FROM foo GROUP BY @1 · vectorized true · · render · · (m) · │ render 0 min · · - └── group · · (column5, min) · - │ aggregate 0 column5 · · + └── group · · (column6, min) · + │ aggregate 0 column6 · · │ aggregate 1 min(a) · · - │ group by column5 · · - └── render · · (column5, a) · + │ group by column6 · · + └── render · · (column6, a) · │ render 0 a · · │ render 1 a · · └── scan · · (a) · @@ -1007,11 +1007,11 @@ EXPLAIN (VERBOSE) SELECT min(a) AS m FROM foo GROUP BY @2 · vectorized true · · render · · (m) · │ render 0 min · · - └── group · · (column5, min) · - │ aggregate 0 column5 · · + └── group · · (column6, min) · + │ aggregate 0 column6 · · │ aggregate 1 min(a) · · - │ group by column5 · · - └── render · · (column5, a) · + │ group by column6 · · + └── render · · (column6, a) · │ render 0 b · · │ render 1 a · · └── scan · · (a, b) · @@ -1077,9 +1077,9 @@ EXPLAIN (VERBOSE) SELECT string_agg(s, ',') FROM (SELECT s FROM kv ORDER BY k) · distribution local · · · vectorized true · · group · · (string_agg) · - │ aggregate 0 string_agg(s, column5) · · + │ aggregate 0 string_agg(s, column6) · · │ scalar · · · - └── render · · (column5, k, s) +k + └── render · · (column6, k, s) +k │ render 0 ',' · · │ render 1 k · · │ render 2 s · · @@ -1134,9 +1134,9 @@ EXPLAIN (VERBOSE) SELECT string_agg(s, ', ') FROM kv · distribution local · · · vectorized true · · group · · (string_agg) · - │ aggregate 0 string_agg(s, column5) · · + │ aggregate 0 string_agg(s, column6) · · │ scalar · · · - └── render · · (column5, s) · + └── render · · (column6, s) · │ render 0 ', ' · · │ render 1 s · · └── scan · · (s) · @@ -1167,9 +1167,9 @@ sort · · (company_id, │ order +company_id · · └── group · · (company_id, string_agg) · │ aggregate 0 company_id · · - │ aggregate 1 string_agg(employee, column4) · · + │ aggregate 1 string_agg(employee, column5) · · │ group by company_id · · - └── render · · (column4, company_id, employee) · + └── render · · (column5, company_id, employee) · │ render 0 ',' · · │ render 1 company_id · · │ render 2 employee · · @@ -1194,9 +1194,9 @@ sort · · (company_id, │ order +company_id · · └── group · · (company_id, string_agg) · │ aggregate 0 company_id · · - │ aggregate 1 string_agg(column4, column5) · · + │ aggregate 1 string_agg(column5, column6) · · │ group by company_id · · - └── render · · (column4, column5, company_id) · + └── render · · (column5, column6, company_id) · │ render 0 employee::BYTES · · │ render 1 '\x2c' · · │ render 2 company_id · · @@ -1221,9 +1221,9 @@ sort · · (company_id, │ order +company_id · · └── group · · (company_id, string_agg) · │ aggregate 0 company_id · · - │ aggregate 1 string_agg(employee, column4) · · + │ aggregate 1 string_agg(employee, column5) · · │ group by company_id · · - └── render · · (column4, company_id, employee) · + └── render · · (column5, company_id, employee) · │ render 0 NULL · · │ render 1 company_id · · │ render 2 employee · · @@ -1248,9 +1248,9 @@ sort · · (company_id, │ order +company_id · · └── group · · (company_id, string_agg) · │ aggregate 0 company_id · · - │ aggregate 1 string_agg(column4, column5) · · + │ aggregate 1 string_agg(column5, column6) · · │ group by company_id · · - └── render · · (column4, column5, company_id) · + └── render · · (column5, column6, company_id) · │ render 0 employee::BYTES · · │ render 1 NULL · · │ render 2 company_id · · diff --git a/pkg/sql/opt/exec/execbuilder/testdata/catalog b/pkg/sql/opt/exec/execbuilder/testdata/catalog index 1f916f0fda84..e1c1fe411548 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/catalog +++ b/pkg/sql/opt/exec/execbuilder/testdata/catalog @@ -16,6 +16,7 @@ TABLE xyz ├── x int not null ├── y int ├── z int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── INDEX primary │ └── x int not null └── INDEX foo @@ -48,6 +49,7 @@ TABLE abcdef ├── e int as (a) stored ├── f int not null ├── rowid int not null default (unique_rowid()) [hidden] + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── CHECK (f > 2:::INT8) └── INDEX primary └── rowid int not null default (unique_rowid()) [hidden] @@ -82,6 +84,7 @@ TABLE uvwxy ├── w int ├── x int ├── y int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── FAMILY fam_0_u_v_w (u, v, w) ├── FAMILY fam_1_x (x) ├── FAMILY fam_2_y (y) @@ -111,6 +114,7 @@ TABLE child ├── p int ├── q int ├── r int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── INDEX primary │ └── c int not null └── CONSTRAINT fk FOREIGN KEY child (p, q, r) REFERENCES parent (p, q, r) @@ -124,6 +128,7 @@ TABLE parent ├── q int not null ├── r int not null ├── other int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── INDEX primary │ ├── p int not null │ ├── q int not null @@ -149,6 +154,7 @@ TABLE child2 ├── p int ├── q int ├── r int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── INDEX primary │ └── c int not null └── CONSTRAINT fk FOREIGN KEY child2 (p, q, r) REFERENCES parent (p, q, r) MATCH FULL ON DELETE SET NULL @@ -162,6 +168,7 @@ TABLE parent ├── q int not null ├── r int not null ├── other int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── INDEX primary │ ├── p int not null │ ├── q int not null @@ -196,6 +203,7 @@ EXPLAIN (OPT, CATALOG) SELECT * FROM a TABLE a ├── a int not null ├── other int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── FAMILY fam_0_a_other (a, other) └── INDEX primary ├── a int not null @@ -209,6 +217,7 @@ EXPLAIN (OPT, CATALOG) SELECT * FROM ab TABLE ab ├── a int not null ├── b int not null + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── FAMILY fam_0_a_b (a, b) └── INDEX primary ├── a int not null @@ -228,6 +237,7 @@ TABLE abc ├── a int ├── b int ├── c int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── FAMILY fam_0_a_b_c_k (a, b, c, k) ├── INDEX primary │ └── k int not null @@ -248,6 +258,7 @@ TABLE abx ├── a int not null ├── b int not null ├── x int not null + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── FAMILY fam_0_a_b_x (a, b, x) └── INDEX primary ├── a int not null diff --git a/pkg/sql/opt/exec/execbuilder/testdata/distsql_agg b/pkg/sql/opt/exec/execbuilder/testdata/distsql_agg index 316b14d60733..196c4d902cef 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/distsql_agg +++ b/pkg/sql/opt/exec/execbuilder/testdata/distsql_agg @@ -329,14 +329,14 @@ query T EXPLAIN (opt,verbose) SELECT b, count(*) FROM data2 WHERE a=1 GROUP BY b ---- group-by - ├── columns: b:2 count:3 + ├── columns: b:2 count:4 ├── grouping columns: b:2 ├── internal-ordering: +2 opt(1) ├── stats: [rows=9.5617925, distinct(2)=9.5617925, null(2)=0] ├── cost: 11.1256179 ├── key: (2) - ├── fd: (2)-->(3) - ├── prune: (3) + ├── fd: (2)-->(4) + ├── prune: (4) ├── scan data2 │ ├── columns: a:1 b:2 │ ├── constraint: /1/2: [/1 - /1] @@ -348,7 +348,7 @@ group-by │ ├── prune: (2) │ └── interesting orderings: (+1,+2) (+2,+1) └── aggregations - └── count-rows [as=count_rows:3] + └── count-rows [as=count_rows:4] query TTTTT EXPLAIN (verbose) SELECT b, count(*), corr(a, b) FROM data2 WHERE a=1 GROUP BY b diff --git a/pkg/sql/opt/exec/execbuilder/testdata/explain b/pkg/sql/opt/exec/execbuilder/testdata/explain index 80890efad09f..456c41716c98 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/explain +++ b/pkg/sql/opt/exec/execbuilder/testdata/explain @@ -912,6 +912,7 @@ TABLE tc ├── a int ├── b int ├── rowid int not null default (unique_rowid()) [hidden] + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── INDEX primary │ └── rowid int not null default (unique_rowid()) [hidden] └── INDEX c @@ -929,6 +930,7 @@ TABLE tc ├── a int ├── b int ├── rowid int not null default (unique_rowid()) [hidden] + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── INDEX primary │ └── rowid int not null default (unique_rowid()) [hidden] └── INDEX c @@ -937,33 +939,34 @@ TABLE tc TABLE t ├── k int not null ├── v int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] └── INDEX primary └── k int not null inner-join (hash) - ├── columns: a:1 b:2 k:4 v:5 + ├── columns: a:1 b:2 k:5 v:6 ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - ├── stats: [rows=990, distinct(1)=99, null(1)=0, distinct(4)=99, null(4)=0] + ├── stats: [rows=990, distinct(1)=99, null(1)=0, distinct(5)=99, null(5)=0] ├── cost: 2219.96 - ├── fd: (4)-->(5), (1)==(4), (4)==(1) - ├── prune: (2,5) + ├── fd: (5)-->(6), (1)==(5), (5)==(1) + ├── prune: (2,6) ├── scan tc │ ├── columns: a:1 b:2 │ ├── stats: [rows=1000, distinct(1)=100, null(1)=10] │ ├── cost: 1100.02 │ ├── prune: (1,2) │ ├── interesting orderings: (+1) - │ └── unfiltered-cols: (1-3) + │ └── unfiltered-cols: (1-4) ├── scan t - │ ├── columns: k:4 v:5 - │ ├── stats: [rows=1000, distinct(4)=1000, null(4)=0] + │ ├── columns: k:5 v:6 + │ ├── stats: [rows=1000, distinct(5)=1000, null(5)=0] │ ├── cost: 1080.02 - │ ├── key: (4) - │ ├── fd: (4)-->(5) - │ ├── prune: (4,5) - │ ├── interesting orderings: (+4) - │ └── unfiltered-cols: (4,5) + │ ├── key: (5) + │ ├── fd: (5)-->(6) + │ ├── prune: (5,6) + │ ├── interesting orderings: (+5) + │ └── unfiltered-cols: (5-7) └── filters - └── k:4 = a:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + └── k:5 = a:1 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] query T EXPLAIN (OPT) SELECT * FROM tc WHERE a + 2 * b > 1 ORDER BY a*b @@ -981,21 +984,21 @@ query T EXPLAIN (OPT, VERBOSE) SELECT * FROM tc WHERE a + 2 * b > 1 ORDER BY a*b ---- sort - ├── columns: a:1 b:2 [hidden: column4:4] + ├── columns: a:1 b:2 [hidden: column5:5] ├── immutable ├── stats: [rows=333.333333] ├── cost: 1179.26548 - ├── fd: (1,2)-->(4) - ├── ordering: +4 - ├── prune: (1,2,4) + ├── fd: (1,2)-->(5) + ├── ordering: +5 + ├── prune: (1,2,5) ├── interesting orderings: (+1) └── project - ├── columns: column4:4 a:1 b:2 + ├── columns: column5:5 a:1 b:2 ├── immutable ├── stats: [rows=333.333333] ├── cost: 1116.71667 - ├── fd: (1,2)-->(4) - ├── prune: (1,2,4) + ├── fd: (1,2)-->(5) + ├── prune: (1,2,5) ├── interesting orderings: (+1) ├── select │ ├── columns: a:1 b:2 @@ -1012,27 +1015,27 @@ sort │ └── filters │ └── (a:1 + (b:2 * 2)) > 1 [outer=(1,2), immutable] └── projections - └── a:1 * b:2 [as=column4:4, outer=(1,2), immutable] + └── a:1 * b:2 [as=column5:5, outer=(1,2), immutable] query T EXPLAIN (OPT, TYPES) SELECT * FROM tc WHERE a + 2 * b > 1 ORDER BY a*b ---- sort - ├── columns: a:1(int) b:2(int) [hidden: column4:4(int)] + ├── columns: a:1(int) b:2(int) [hidden: column5:5(int)] ├── immutable ├── stats: [rows=333.333333] ├── cost: 1179.26548 - ├── fd: (1,2)-->(4) - ├── ordering: +4 - ├── prune: (1,2,4) + ├── fd: (1,2)-->(5) + ├── ordering: +5 + ├── prune: (1,2,5) ├── interesting orderings: (+1) └── project - ├── columns: column4:4(int) a:1(int) b:2(int) + ├── columns: column5:5(int) a:1(int) b:2(int) ├── immutable ├── stats: [rows=333.333333] ├── cost: 1116.71667 - ├── fd: (1,2)-->(4) - ├── prune: (1,2,4) + ├── fd: (1,2)-->(5) + ├── prune: (1,2,5) ├── interesting orderings: (+1) ├── select │ ├── columns: a:1(int) b:2(int) @@ -1055,7 +1058,7 @@ sort │ │ └── const: 2 [type=int] │ └── const: 1 [type=int] └── projections - └── mult [as=column4:4, type=int, outer=(1,2), immutable] + └── mult [as=column5:5, type=int, outer=(1,2), immutable] ├── variable: a:1 [type=int] └── variable: b:2 [type=int] diff --git a/pkg/sql/opt/exec/execbuilder/testdata/hash_sharded_index b/pkg/sql/opt/exec/execbuilder/testdata/hash_sharded_index index cdee4c5eca02..91bcb6c5e310 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/hash_sharded_index +++ b/pkg/sql/opt/exec/execbuilder/testdata/hash_sharded_index @@ -16,11 +16,11 @@ count · · │ into sharded_primary(crdb_internal_a_shard_11, a) · · │ strategy inserter · · │ auto commit · · · - └── render · · (column4, column1, check1) · - │ render 0 column4 · · + └── render · · (column5, column1, check1) · + │ render 0 column5 · · │ render 1 column1 · · - │ render 2 column4 IN (0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10) · · - └── render · · (column4, column1) · + │ render 2 column5 IN (0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10) · · + └── render · · (column5, column1) · │ render 0 mod(fnv32(COALESCE(column1::STRING, '')), 11) · · │ render 1 column1 · · └── values · · (column1) · @@ -41,12 +41,12 @@ count · · │ into sharded_secondary(a, crdb_internal_a_shard_12, rowid) · · │ strategy inserter · · │ auto commit · · · - └── render · · (column1, column6, column5, check1) · + └── render · · (column1, column7, column6, check1) · │ render 0 column1 · · - │ render 1 column6 · · - │ render 2 column5 · · - │ render 3 column6 IN (0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11) · · - └── render · · (column6, column5, column1) · + │ render 1 column7 · · + │ render 2 column6 · · + │ render 3 column7 IN (0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11) · · + └── render · · (column7, column6, column1) · │ render 0 mod(fnv32(COALESCE(column1::STRING, '')), 12) · · │ render 1 unique_rowid() · · │ render 2 column1 · · diff --git a/pkg/sql/opt/exec/execbuilder/testdata/insert b/pkg/sql/opt/exec/execbuilder/testdata/insert index 878bae16f710..071ef7c1b116 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/insert +++ b/pkg/sql/opt/exec/execbuilder/testdata/insert @@ -424,29 +424,29 @@ query TTTTT EXPLAIN (VERBOSE) INSERT INTO insert_t (SELECT length(k), 2 FROM kv ORDER BY k || v LIMIT 10) RETURNING x+v ---- -· distribution local · · -· vectorized false · · -render · · ("?column?") · - │ render 0 x + v · · - └── run · · (x, v, rowid[hidden]) · - └── insert · · (x, v, rowid[hidden]) · - │ into insert_t(x, v, rowid) · · - │ strategy inserter · · - └── render · · (length, "?column?", column9) · - │ render 0 length · · - │ render 1 "?column?" · · - │ render 2 unique_rowid() · · - └── limit · · (length, "?column?", column8) +column8 - │ count 10 · · - └── sort · · (length, "?column?", column8) +column8 - │ order +column8 · · - └── render · · (length, "?column?", column8) · - │ render 0 length(k) · · - │ render 1 2 · · - │ render 2 k::STRING || v::STRING · · - └── scan · · (k, v) · -· table kv@primary · · -· spans FULL SCAN · · +· distribution local · · +· vectorized false · · +render · · ("?column?") · + │ render 0 x + v · · + └── run · · (x, v, rowid[hidden]) · + └── insert · · (x, v, rowid[hidden]) · + │ into insert_t(x, v, rowid) · · + │ strategy inserter · · + └── render · · (length, "?column?", column11) · + │ render 0 length · · + │ render 1 "?column?" · · + │ render 2 unique_rowid() · · + └── limit · · (length, "?column?", column10) +column10 + │ count 10 · · + └── sort · · (length, "?column?", column10) +column10 + │ order +column10 · · + └── render · · (length, "?column?", column10) · + │ render 0 length(k) · · + │ render 1 2 · · + │ render 2 k::STRING || v::STRING · · + └── scan · · (k, v) · +· table kv@primary · · +· spans FULL SCAN · · # ------------------------------------------------------------------------------ # Insert rows into table during schema changes. @@ -511,34 +511,34 @@ CREATE TABLE xyz (x INT, y INT, z INT) query TTTTT EXPLAIN (VERBOSE) SELECT * FROM [INSERT INTO xyz SELECT a, b, c FROM abc RETURNING z] ORDER BY z ---- -· distribution local · · -· vectorized false · · -root · · (z) +z - ├── sort · · (z) +z - │ │ order +z · · - │ └── scan buffer node · · (z) · - │ label buffer 1 · · - └── subquery · · · · - │ id @S1 · · - │ original sql INSERT INTO xyz SELECT a, b, c FROM abc RETURNING z · · - │ exec mode all rows · · - └── buffer node · · (z) · - │ label buffer 1 · · - └── spool · · (z) · - └── render · · (z) · - │ render 0 z · · - └── run · · (z, rowid[hidden]) · - └── insert · · (z, rowid[hidden]) · - │ into xyz(x, y, z, rowid) · · - │ strategy inserter · · - └── render · · (a, b, c, column9) · - │ render 0 a · · - │ render 1 b · · - │ render 2 c · · - │ render 3 unique_rowid() · · - └── scan · · (a, b, c) · -· table abc@primary · · -· spans FULL SCAN · · +· distribution local · · +· vectorized false · · +root · · (z) +z + ├── sort · · (z) +z + │ │ order +z · · + │ └── scan buffer node · · (z) · + │ label buffer 1 · · + └── subquery · · · · + │ id @S1 · · + │ original sql INSERT INTO xyz SELECT a, b, c FROM abc RETURNING z · · + │ exec mode all rows · · + └── buffer node · · (z) · + │ label buffer 1 · · + └── spool · · (z) · + └── render · · (z) · + │ render 0 z · · + └── run · · (z, rowid[hidden]) · + └── insert · · (z, rowid[hidden]) · + │ into xyz(x, y, z, rowid) · · + │ strategy inserter · · + └── render · · (a, b, c, column11) · + │ render 0 a · · + │ render 1 b · · + │ render 2 c · · + │ render 3 unique_rowid() · · + └── scan · · (a, b, c) · +· table abc@primary · · +· spans FULL SCAN · · # ------------------------------------------------------------------------------ # Regression for #35364. This tests behavior that is different between the CBO diff --git a/pkg/sql/opt/exec/execbuilder/testdata/join b/pkg/sql/opt/exec/execbuilder/testdata/join index 8afe679d84b2..bf4683fc1641 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/join +++ b/pkg/sql/opt/exec/execbuilder/testdata/join @@ -473,10 +473,10 @@ render · · (a, b, n, sq) │ render 1 b · · │ render 2 n · · │ render 3 sq · · - └── hash-join · · (column6, a, b, n, sq) · + └── hash-join · · (column8, a, b, n, sq) · │ type inner · · - │ equality (column6) = (sq) · · - ├── render · · (column6, a, b) · + │ equality (column8) = (sq) · · + ├── render · · (column8, a, b) · │ │ render 0 a + b · · │ │ render 1 a · · │ │ render 2 b · · @@ -527,10 +527,10 @@ render · · (a, b, n, sq) │ render 1 b · · │ render 2 n · · │ render 3 sq · · - └── hash-join · · (column6, a, b, n, sq) · + └── hash-join · · (column8, a, b, n, sq) · │ type full outer · · - │ equality (column6) = (sq) · · - ├── render · · (column6, a, b) · + │ equality (column8) = (sq) · · + ├── render · · (column8, a, b) · │ │ render 0 a + b · · │ │ render 1 a · · │ │ render 2 b · · @@ -551,12 +551,12 @@ render · · (a, b, n, sq) │ render 1 b · · │ render 2 n · · │ render 3 sq · · - └── filter · · (column6, a, b, n, sq) · + └── filter · · (column8, a, b, n, sq) · │ filter (b % 2) != (sq % 2) · · - └── hash-join · · (column6, a, b, n, sq) · + └── hash-join · · (column8, a, b, n, sq) · │ type full outer · · - │ equality (column6) = (sq) · · - ├── render · · (column6, a, b) · + │ equality (column8) = (sq) · · + ├── render · · (column8, a, b) · │ │ render 0 a + b · · │ │ render 1 a · · │ │ render 2 b · · diff --git a/pkg/sql/opt/exec/execbuilder/testdata/lookup_join b/pkg/sql/opt/exec/execbuilder/testdata/lookup_join index 803bcaab68fa..5bb36e6f31b4 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/lookup_join +++ b/pkg/sql/opt/exec/execbuilder/testdata/lookup_join @@ -1495,13 +1495,13 @@ EXPLAIN (VERBOSE) · vectorized true · · render · · (pk) · │ render 0 pk · · - └── lookup-join · · ("project_const_col_@13", pk, col0, col3) · + └── lookup-join · · ("project_const_col_@14", pk, col0, col3) · │ table tab4@tab4_col3_col4_key · · │ type semi · · - │ equality (col0, project_const_col_@13) = (col3, col4) · · + │ equality (col0, project_const_col_@14) = (col3, col4) · · │ equality cols are key · · · │ parallel · · · - └── render · · ("project_const_col_@13", pk, col0, col3) · + └── render · · ("project_const_col_@14", pk, col0, col3) · │ render 0 495.6 · · │ render 1 pk · · │ render 2 col0 · · diff --git a/pkg/sql/opt/exec/execbuilder/testdata/mvcc b/pkg/sql/opt/exec/execbuilder/testdata/mvcc new file mode 100644 index 000000000000..7ccc1ba1897f --- /dev/null +++ b/pkg/sql/opt/exec/execbuilder/testdata/mvcc @@ -0,0 +1,58 @@ +# LogicTest: local + +# Create a table with different families. +statement ok +CREATE TABLE t (x INT PRIMARY KEY, y INT, z INT, FAMILY (x), FAMILY (y), FAMILY (z)) + +# When doing a lookup where we could split families, but the MVCC column is +# requested, we shouldn't split the family. +query TTT +EXPLAIN SELECT z FROM t WHERE x = 1 +---- +· distribution local +· vectorized true +render · · + └── scan · · +· table t@primary +· spans /1/0-/1/1 /1/2/1-/1/2/2 + +query TTT +EXPLAIN SELECT crdb_internal_mvcc_timestamp, z FROM t WHERE x = 1 +---- +· distribution local +· vectorized true +render · · + └── scan · · +· table t@primary +· spans /1-/1/# + +# Ensure that the presence of mutation columns doesn't affect accessing system +# columns. +statement ok +BEGIN; +ALTER TABLE t ADD COLUMN w INT + +query TTT +EXPLAIN SELECT x, crdb_internal_mvcc_timestamp FROM t +---- +· distribution local +· vectorized true +scan · · +· table t@primary +· spans FULL SCAN + +statement ok +ROLLBACK + +# Test that we can access system columns by ID. +let $t_id +SELECT id FROM system.namespace WHERE name = 't' + +query TTT +EXPLAIN SELECT * FROM [$t_id(4294967295) AS _] +---- +· distribution local +· vectorized true +scan · · +· table t@primary +· spans FULL SCAN diff --git a/pkg/sql/opt/exec/execbuilder/testdata/orderby b/pkg/sql/opt/exec/execbuilder/testdata/orderby index e8cdaeb49d19..c3ff806c8e5c 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/orderby +++ b/pkg/sql/opt/exec/execbuilder/testdata/orderby @@ -692,9 +692,9 @@ EXPLAIN (VERBOSE) SELECT b, a FROM foo ORDER BY @1 render · · (b, a) · │ render 0 b · · │ render 1 a · · - └── sort · · (column4, a, b) +a + └── sort · · (column5, a, b) +a │ order +a · · - └── render · · (column4, a, b) · + └── render · · (column5, a, b) · │ render 0 a · · │ render 1 a · · │ render 2 b · · @@ -710,9 +710,9 @@ EXPLAIN (VERBOSE) SELECT b, a FROM foo ORDER BY @2 render · · (b, a) · │ render 0 b · · │ render 1 a · · - └── sort · · (column4, a, b) +b + └── sort · · (column5, a, b) +b │ order +b · · - └── render · · (column4, a, b) · + └── render · · (column5, a, b) · │ render 0 b · · │ render 1 a · · │ render 2 b · · diff --git a/pkg/sql/opt/exec/execbuilder/testdata/select_for_update b/pkg/sql/opt/exec/execbuilder/testdata/select_for_update index 8a750915fc16..1fd3606bdfc5 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/select_for_update +++ b/pkg/sql/opt/exec/execbuilder/testdata/select_for_update @@ -523,80 +523,78 @@ root · · · locking strength for update query TTT -EXPLAIN SELECT * FROM t WHERE a IN (SELECT a FROM t) FOR UPDATE ----- -· distribution local -· vectorized true -merge-join · · - │ type semi - │ equality (a) = (a) - │ left cols are key · - │ right cols are key · - │ mergeJoinOrder +"(a=a)" - ├── scan · · - │ table t@primary - │ spans FULL SCAN - │ locking strength for update - └── scan · · -· table t@primary -· spans FULL SCAN - -query TTT -EXPLAIN SELECT * FROM t WHERE a IN (SELECT a FROM t FOR UPDATE) ----- -· distribution local -· vectorized true -merge-join · · - │ type semi - │ equality (a) = (a) - │ left cols are key · - │ right cols are key · - │ mergeJoinOrder +"(a=a)" - ├── scan · · - │ table t@primary - │ spans FULL SCAN - └── scan · · -· table t@primary -· spans FULL SCAN -· locking strength for update - -query TTT -EXPLAIN SELECT * FROM t WHERE a IN (SELECT a FROM t) FOR UPDATE OF t ----- -· distribution local -· vectorized true -merge-join · · - │ type semi - │ equality (a) = (a) - │ left cols are key · - │ right cols are key · - │ mergeJoinOrder +"(a=a)" - ├── scan · · - │ table t@primary - │ spans FULL SCAN - │ locking strength for update - └── scan · · -· table t@primary -· spans FULL SCAN - -query TTT -EXPLAIN SELECT * FROM t WHERE a IN (SELECT a FROM t FOR UPDATE OF t) ----- -· distribution local -· vectorized true -merge-join · · - │ type semi - │ equality (a) = (a) - │ left cols are key · - │ right cols are key · - │ mergeJoinOrder +"(a=a)" - ├── scan · · - │ table t@primary - │ spans FULL SCAN - └── scan · · -· table t@primary -· spans FULL SCAN -· locking strength for update +EXPLAIN SELECT * FROM t WHERE a IN (SELECT b FROM t) FOR UPDATE +---- +· distribution local +· vectorized true +render · · + └── lookup-join · · + │ table t@primary + │ type inner + │ equality (b) = (a) + │ equality cols are key · + │ parallel · + └── distinct · · + │ distinct on b + └── scan · · +· table t@primary +· spans FULL SCAN + +query TTT +EXPLAIN SELECT * FROM t WHERE a IN (SELECT b FROM t FOR UPDATE) +---- +· distribution local +· vectorized true +render · · + └── lookup-join · · + │ table t@primary + │ type inner + │ equality (b) = (a) + │ equality cols are key · + │ parallel · + └── distinct · · + │ distinct on b + └── scan · · +· table t@primary +· spans FULL SCAN +· locking strength for update + +query TTT +EXPLAIN SELECT * FROM t WHERE a IN (SELECT b FROM t) FOR UPDATE OF t +---- +· distribution local +· vectorized true +render · · + └── lookup-join · · + │ table t@primary + │ type inner + │ equality (b) = (a) + │ equality cols are key · + │ parallel · + └── distinct · · + │ distinct on b + └── scan · · +· table t@primary +· spans FULL SCAN + +query TTT +EXPLAIN SELECT * FROM t WHERE a IN (SELECT b FROM t FOR UPDATE OF t) +---- +· distribution local +· vectorized true +render · · + └── lookup-join · · + │ table t@primary + │ type inner + │ equality (b) = (a) + │ equality cols are key · + │ parallel · + └── distinct · · + │ distinct on b + └── scan · · +· table t@primary +· spans FULL SCAN +· locking strength for update # ------------------------------------------------------------------------------ # Tests with common-table expressions. diff --git a/pkg/sql/opt/exec/execbuilder/testdata/srfs b/pkg/sql/opt/exec/execbuilder/testdata/srfs index 68d8c0d14d35..f2ad71acb60c 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/srfs +++ b/pkg/sql/opt/exec/execbuilder/testdata/srfs @@ -238,17 +238,17 @@ render · · │ render 0 id · · │ render 1 data · · │ render 2 "?column?" · · - └── hash-join · · (column8, id, data, column9, "?column?") · + └── hash-join · · (column10, id, data, column11, "?column?") · │ type left outer · · - │ equality (column8) = (column9) · · - ├── render · · (column8, id, data) · + │ equality (column10) = (column11) · · + ├── render · · (column10, id, data) · │ │ render 0 data->>'name' · · │ │ render 1 id · · │ │ render 2 data · · │ └── scan · · (id, data) · │ table groups@primary · · │ spans FULL SCAN · · - └── render · · (column9, "?column?") · + └── render · · (column11, "?column?") · │ render 0 data->>'name' · · │ render 1 data->'members' · · └── scan · · (data) · diff --git a/pkg/sql/opt/exec/execbuilder/testdata/subquery b/pkg/sql/opt/exec/execbuilder/testdata/subquery index ff59a876f819..46e1364323c4 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/subquery +++ b/pkg/sql/opt/exec/execbuilder/testdata/subquery @@ -189,12 +189,12 @@ EXPLAIN (VERBOSE) SELECT * FROM a WHERE EXISTS(SELECT * FROM b WHERE b.x-1 = a.x · vectorized true · · hash-join · · (x, y) · │ type semi · · - │ equality (x) = (column5) · · + │ equality (x) = (column7) · · │ left cols are key · · · ├── scan · · (x, y) · │ table a@primary · · │ spans FULL SCAN · · - └── render · · (column5) · + └── render · · (column7) · │ render 0 x - 1 · · └── scan · · (x) · · table b@primary · · @@ -225,12 +225,12 @@ EXPLAIN (VERBOSE) SELECT * FROM b WHERE NOT EXISTS(SELECT * FROM a WHERE x-1 = b · vectorized true · · hash-join · · (x, z) · │ type anti · · - │ equality (x) = (column5) · · + │ equality (x) = (column7) · · │ left cols are key · · · ├── scan · · (x, z) · │ table b@primary · · │ spans FULL SCAN · · - └── render · · (column5) · + └── render · · (column7) · │ render 0 x - 1 · · └── scan · · (x) · · table a@primary · · diff --git a/pkg/sql/opt/exec/execbuilder/testdata/upsert b/pkg/sql/opt/exec/execbuilder/testdata/upsert index ff354ac3b36c..fb778e6f2bfe 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/upsert +++ b/pkg/sql/opt/exec/execbuilder/testdata/upsert @@ -96,7 +96,7 @@ count · · │ auto commit · └── render · · │ render 0 k - │ render 1 column5 + │ render 1 column7 │ render 2 k └── lookup-join · · │ table kv@primary @@ -147,24 +147,24 @@ count · · │ auto commit · └── render · · │ render 0 column1 - │ render 1 column6 - │ render 2 column7 - │ render 3 column8 + │ render 1 column7 + │ render 2 column8 + │ render 3 column9 │ render 4 a │ render 5 b │ render 6 c │ render 7 d - │ render 8 column6 - │ render 9 column7 - │ render 10 column8 + │ render 8 column7 + │ render 9 column8 + │ render 10 column9 │ render 11 a │ render 12 check1 └── render · · - │ render 0 column7 > 0 + │ render 0 column8 > 0 │ render 1 column1 - │ render 2 column6 - │ render 3 column7 - │ render 4 column8 + │ render 2 column7 + │ render 3 column8 + │ render 4 column9 │ render 5 a │ render 6 b │ render 7 c @@ -199,12 +199,12 @@ run · · │ auto commit · └── render · · │ render 0 column1 - │ render 1 column6 - │ render 2 column7 - │ render 3 column8 - │ render 4 column6 - │ render 5 column7 - │ render 6 column8 + │ render 1 column7 + │ render 2 column8 + │ render 3 column9 + │ render 4 column7 + │ render 5 column8 + │ render 6 column9 │ render 7 check1 └── values · · · size 5 columns, 1 row @@ -349,42 +349,42 @@ CREATE TABLE xyz (x INT, y INT, z INT) query TTTTT EXPLAIN (VERBOSE) SELECT * FROM [UPSERT INTO xyz SELECT a, b, c FROM abc RETURNING z] ORDER BY z ---- -· distribution local · · -· vectorized false · · -root · · (z) +z - ├── sort · · (z) +z - │ │ order +z · · - │ └── scan buffer node · · (z) · - │ label buffer 1 · · - └── subquery · · · · - │ id @S1 · · - │ original sql UPSERT INTO xyz SELECT a, b, c FROM abc RETURNING z · · - │ exec mode all rows · · - └── buffer node · · (z) · - │ label buffer 1 · · - └── spool · · (z) · - └── render · · (z) · - │ render 0 z · · - └── run · · (z, rowid[hidden]) · - └── upsert · · (z, rowid[hidden]) · - │ into xyz(x, y, z, rowid) · · - │ strategy opt upserter · · - └── render · · (a, b, c, column9, a, b, c) · - │ render 0 a · · - │ render 1 b · · - │ render 2 c · · - │ render 3 column9 · · - │ render 4 a · · - │ render 5 b · · - │ render 6 c · · - └── render · · (column9, a, b, c) · - │ render 0 unique_rowid() · · - │ render 1 a · · - │ render 2 b · · - │ render 3 c · · - └── scan · · (a, b, c) · -· table abc@primary · · -· spans FULL SCAN · · +· distribution local · · +· vectorized false · · +root · · (z) +z + ├── sort · · (z) +z + │ │ order +z · · + │ └── scan buffer node · · (z) · + │ label buffer 1 · · + └── subquery · · · · + │ id @S1 · · + │ original sql UPSERT INTO xyz SELECT a, b, c FROM abc RETURNING z · · + │ exec mode all rows · · + └── buffer node · · (z) · + │ label buffer 1 · · + └── spool · · (z) · + └── render · · (z) · + │ render 0 z · · + └── run · · (z, rowid[hidden]) · + └── upsert · · (z, rowid[hidden]) · + │ into xyz(x, y, z, rowid) · · + │ strategy opt upserter · · + └── render · · (a, b, c, column11, a, b, c) · + │ render 0 a · · + │ render 1 b · · + │ render 2 c · · + │ render 3 column11 · · + │ render 4 a · · + │ render 5 b · · + │ render 6 c · · + └── render · · (column11, a, b, c) · + │ render 0 unique_rowid() · · + │ render 1 a · · + │ render 2 b · · + │ render 3 c · · + └── scan · · (a, b, c) · +· table abc@primary · · +· spans FULL SCAN · · # ------------------------------------------------------------------------------ # Regression for #35364. This tests behavior that is different between the CBO diff --git a/pkg/sql/opt/exec/execbuilder/testdata/with b/pkg/sql/opt/exec/execbuilder/testdata/with index 52c98a448ea4..07757f8f9974 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/with +++ b/pkg/sql/opt/exec/execbuilder/testdata/with @@ -63,7 +63,7 @@ root · · └── insert · · (a, rowid[hidden]) · │ into x(a, rowid) · · │ strategy inserter · · - └── values · · (column1, column4) · + └── values · · (column1, column5) · · size 2 columns, 1 row · · · row 0, expr 0 1 · · · row 0, expr 1 unique_rowid() · · diff --git a/pkg/sql/opt/memo/check_expr.go b/pkg/sql/opt/memo/check_expr.go index c17b9d08d0e9..1e5630757722 100644 --- a/pkg/sql/opt/memo/check_expr.go +++ b/pkg/sql/opt/memo/check_expr.go @@ -183,6 +183,9 @@ func (m *Memo) CheckExpr(e opt.Expr) { if tab.ColumnKind(i) != cat.DeleteOnly && t.InsertCols[i] == 0 { panic(errors.AssertionFailedf("insert values not provided for all table columns")) } + if cat.IsSystemColumn(tab, i) && t.InsertCols[i] != 0 { + panic(errors.AssertionFailedf("system column found in insertion columns")) + } } m.checkMutationExpr(t, &t.MutationPrivate) diff --git a/pkg/sql/opt/memo/testdata/format b/pkg/sql/opt/memo/testdata/format index 7a7b32146eb1..de15987385db 100644 --- a/pkg/sql/opt/memo/testdata/format +++ b/pkg/sql/opt/memo/testdata/format @@ -6,31 +6,31 @@ opt format=show-all SELECT a + 1, min(b) FROM t WHERE k + a > b GROUP BY a ORDER BY a ---- sort - ├── columns: "?column?":5(int) min:4(int!null) [hidden: t.public.t.a:1(int)] + ├── columns: "?column?":6(int) min:5(int!null) [hidden: t.public.t.a:1(int)] ├── immutable ├── stats: [rows=98.1771622] ├── cost: 1097.88224 ├── key: (1) - ├── fd: (1)-->(4,5) + ├── fd: (1)-->(5,6) ├── ordering: +1 - ├── prune: (1,4,5) + ├── prune: (1,5,6) └── project - ├── columns: "?column?":5(int) t.public.t.a:1(int) min:4(int!null) + ├── columns: "?column?":6(int) t.public.t.a:1(int) min:5(int!null) ├── immutable ├── stats: [rows=98.1771622] ├── cost: 1082.91531 ├── key: (1) - ├── fd: (1)-->(4,5) - ├── prune: (1,4,5) + ├── fd: (1)-->(5,6) + ├── prune: (1,5,6) ├── group-by - │ ├── columns: t.public.t.a:1(int) min:4(int!null) + │ ├── columns: t.public.t.a:1(int) min:5(int!null) │ ├── grouping columns: t.public.t.a:1(int) │ ├── immutable │ ├── stats: [rows=98.1771622, distinct(1)=98.1771622, null(1)=1] │ ├── cost: 1080.94177 │ ├── key: (1) - │ ├── fd: (1)-->(4) - │ ├── prune: (4) + │ ├── fd: (1)-->(5) + │ ├── prune: (5) │ ├── select │ │ ├── columns: t.public.t.a:1(int) t.public.t.b:2(int!null) t.public.t.k:3(int!null) │ │ ├── immutable @@ -54,10 +54,10 @@ sort │ │ ├── variable: t.public.t.k:3 [type=int] │ │ └── variable: t.public.t.a:1 [type=int] │ └── aggregations - │ └── min [as=min:4, type=int, outer=(2)] + │ └── min [as=min:5, type=int, outer=(2)] │ └── variable: t.public.t.b:2 [type=int] └── projections - └── plus [as="?column?":5, type=int, outer=(1), immutable] + └── plus [as="?column?":6, type=int, outer=(1), immutable] ├── variable: t.public.t.a:1 [type=int] └── const: 1 [type=int] @@ -65,16 +65,16 @@ opt format=(hide-miscprops,hide-constraints,hide-funcdeps,hide-ruleprops) SELECT a + 1, min(b) FROM t WHERE k + a > b GROUP BY a ORDER BY a ---- sort - ├── columns: "?column?":5(int) min:4(int!null) [hidden: t.public.t.a:1(int)] + ├── columns: "?column?":6(int) min:5(int!null) [hidden: t.public.t.a:1(int)] ├── stats: [rows=98.1771622] ├── cost: 1097.88224 ├── ordering: +1 └── project - ├── columns: "?column?":5(int) t.public.t.a:1(int) min:4(int!null) + ├── columns: "?column?":6(int) t.public.t.a:1(int) min:5(int!null) ├── stats: [rows=98.1771622] ├── cost: 1082.91531 ├── group-by - │ ├── columns: t.public.t.a:1(int) min:4(int!null) + │ ├── columns: t.public.t.a:1(int) min:5(int!null) │ ├── grouping columns: t.public.t.a:1(int) │ ├── stats: [rows=98.1771622, distinct(1)=98.1771622, null(1)=1] │ ├── cost: 1080.94177 @@ -93,10 +93,10 @@ sort │ │ ├── variable: t.public.t.k:3 [type=int] │ │ └── variable: t.public.t.a:1 [type=int] │ └── aggregations - │ └── min [as=min:4, type=int] + │ └── min [as=min:5, type=int] │ └── variable: t.public.t.b:2 [type=int] └── projections - └── plus [as="?column?":5, type=int] + └── plus [as="?column?":6, type=int] ├── variable: t.public.t.a:1 [type=int] └── const: 1 [type=int] @@ -104,25 +104,25 @@ opt format=(hide-stats,hide-cost,hide-qual,hide-scalars) SELECT a + 1, min(b) FROM t WHERE k + a > b GROUP BY a ORDER BY a ---- sort - ├── columns: "?column?":5(int) min:4(int!null) [hidden: a:1(int)] + ├── columns: "?column?":6(int) min:5(int!null) [hidden: a:1(int)] ├── immutable ├── key: (1) - ├── fd: (1)-->(4,5) + ├── fd: (1)-->(5,6) ├── ordering: +1 - ├── prune: (1,4,5) + ├── prune: (1,5,6) └── project - ├── columns: "?column?":5(int) a:1(int) min:4(int!null) + ├── columns: "?column?":6(int) a:1(int) min:5(int!null) ├── immutable ├── key: (1) - ├── fd: (1)-->(4,5) - ├── prune: (1,4,5) + ├── fd: (1)-->(5,6) + ├── prune: (1,5,6) ├── group-by - │ ├── columns: a:1(int) min:4(int!null) + │ ├── columns: a:1(int) min:5(int!null) │ ├── grouping columns: a:1(int) │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(4) - │ ├── prune: (4) + │ ├── fd: (1)-->(5) + │ ├── prune: (5) │ ├── select │ │ ├── columns: a:1(int) b:2(int!null) k:3(int!null) │ │ ├── immutable @@ -138,34 +138,34 @@ sort │ │ └── filters │ │ └── b:2 < (k:3 + a:1) [type=bool, outer=(1-3), immutable, constraints=(/2: (/NULL - ])] │ └── aggregations - │ └── min [as=min:4, type=int, outer=(2)] + │ └── min [as=min:5, type=int, outer=(2)] │ └── b:2 [type=int] └── projections - └── a:1 + 1 [as="?column?":5, type=int, outer=(1), immutable] + └── a:1 + 1 [as="?column?":6, type=int, outer=(1), immutable] opt format=(hide-stats,hide-cost,hide-qual,hide-scalars,hide-types) SELECT a + 1, min(b) FROM t WHERE k + a > b GROUP BY a ORDER BY a ---- sort - ├── columns: "?column?":5 min:4!null [hidden: a:1] + ├── columns: "?column?":6 min:5!null [hidden: a:1] ├── immutable ├── key: (1) - ├── fd: (1)-->(4,5) + ├── fd: (1)-->(5,6) ├── ordering: +1 - ├── prune: (1,4,5) + ├── prune: (1,5,6) └── project - ├── columns: "?column?":5 a:1 min:4!null + ├── columns: "?column?":6 a:1 min:5!null ├── immutable ├── key: (1) - ├── fd: (1)-->(4,5) - ├── prune: (1,4,5) + ├── fd: (1)-->(5,6) + ├── prune: (1,5,6) ├── group-by - │ ├── columns: a:1 min:4!null + │ ├── columns: a:1 min:5!null │ ├── grouping columns: a:1 │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(4) - │ ├── prune: (4) + │ ├── fd: (1)-->(5) + │ ├── prune: (5) │ ├── select │ │ ├── columns: a:1 b:2!null k:3!null │ │ ├── immutable @@ -181,34 +181,34 @@ sort │ │ └── filters │ │ └── b:2 < (k:3 + a:1) [outer=(1-3), immutable, constraints=(/2: (/NULL - ])] │ └── aggregations - │ └── min [as=min:4, outer=(2)] + │ └── min [as=min:5, outer=(2)] │ └── b:2 └── projections - └── a:1 + 1 [as="?column?":5, outer=(1), immutable] + └── a:1 + 1 [as="?column?":6, outer=(1), immutable] opt format=(hide-stats,hide-cost,hide-qual,hide-scalars,hide-notnull) SELECT a + 1, min(b) FROM t WHERE k + a > b GROUP BY a ORDER BY a ---- sort - ├── columns: "?column?":5(int) min:4(int) [hidden: a:1(int)] + ├── columns: "?column?":6(int) min:5(int) [hidden: a:1(int)] ├── immutable ├── key: (1) - ├── fd: (1)-->(4,5) + ├── fd: (1)-->(5,6) ├── ordering: +1 - ├── prune: (1,4,5) + ├── prune: (1,5,6) └── project - ├── columns: "?column?":5(int) a:1(int) min:4(int) + ├── columns: "?column?":6(int) a:1(int) min:5(int) ├── immutable ├── key: (1) - ├── fd: (1)-->(4,5) - ├── prune: (1,4,5) + ├── fd: (1)-->(5,6) + ├── prune: (1,5,6) ├── group-by - │ ├── columns: a:1(int) min:4(int) + │ ├── columns: a:1(int) min:5(int) │ ├── grouping columns: a:1(int) │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(4) - │ ├── prune: (4) + │ ├── fd: (1)-->(5) + │ ├── prune: (5) │ ├── select │ │ ├── columns: a:1(int) b:2(int) k:3(int) │ │ ├── immutable @@ -224,34 +224,34 @@ sort │ │ └── filters │ │ └── b:2 < (k:3 + a:1) [type=bool, outer=(1-3), immutable, constraints=(/2: (/NULL - ])] │ └── aggregations - │ └── min [as=min:4, type=int, outer=(2)] + │ └── min [as=min:5, type=int, outer=(2)] │ └── b:2 [type=int] └── projections - └── a:1 + 1 [as="?column?":5, type=int, outer=(1), immutable] + └── a:1 + 1 [as="?column?":6, type=int, outer=(1), immutable] opt format=(hide-stats,hide-cost,hide-qual,hide-scalars,hide-types,hide-notnull) SELECT a + 1, min(b) FROM t WHERE k + a > b GROUP BY a ORDER BY a ---- sort - ├── columns: "?column?":5 min:4 [hidden: a:1] + ├── columns: "?column?":6 min:5 [hidden: a:1] ├── immutable ├── key: (1) - ├── fd: (1)-->(4,5) + ├── fd: (1)-->(5,6) ├── ordering: +1 - ├── prune: (1,4,5) + ├── prune: (1,5,6) └── project - ├── columns: "?column?":5 a:1 min:4 + ├── columns: "?column?":6 a:1 min:5 ├── immutable ├── key: (1) - ├── fd: (1)-->(4,5) - ├── prune: (1,4,5) + ├── fd: (1)-->(5,6) + ├── prune: (1,5,6) ├── group-by - │ ├── columns: a:1 min:4 + │ ├── columns: a:1 min:5 │ ├── grouping columns: a:1 │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(4) - │ ├── prune: (4) + │ ├── fd: (1)-->(5) + │ ├── prune: (5) │ ├── select │ │ ├── columns: a:1 b:2 k:3 │ │ ├── immutable @@ -267,10 +267,10 @@ sort │ │ └── filters │ │ └── b:2 < (k:3 + a:1) [outer=(1-3), immutable, constraints=(/2: (/NULL - ])] │ └── aggregations - │ └── min [as=min:4, outer=(2)] + │ └── min [as=min:5, outer=(2)] │ └── b:2 └── projections - └── a:1 + 1 [as="?column?":5, outer=(1), immutable] + └── a:1 + 1 [as="?column?":6, outer=(1), immutable] opt format=(hide-miscprops,hide-physprops,hide-columns) SELECT a + 1, min(b) FROM t WHERE k + a > b GROUP BY a ORDER BY a @@ -279,20 +279,20 @@ sort ├── stats: [rows=98.1771622] ├── cost: 1097.88224 ├── key: (1) - ├── fd: (1)-->(4,5) - ├── prune: (1,4,5) + ├── fd: (1)-->(5,6) + ├── prune: (1,5,6) └── project ├── stats: [rows=98.1771622] ├── cost: 1082.91531 ├── key: (1) - ├── fd: (1)-->(4,5) - ├── prune: (1,4,5) + ├── fd: (1)-->(5,6) + ├── prune: (1,5,6) ├── group-by │ ├── stats: [rows=98.1771622, distinct(1)=98.1771622, null(1)=1] │ ├── cost: 1080.94177 │ ├── key: (1) - │ ├── fd: (1)-->(4) - │ ├── prune: (4) + │ ├── fd: (1)-->(5) + │ ├── prune: (5) │ ├── select │ │ ├── stats: [rows=330, distinct(1)=98.1771622, null(1)=3.3, distinct(2)=100, null(2)=0] │ │ ├── cost: 1070.04 diff --git a/pkg/sql/opt/memo/testdata/logprops/constraints b/pkg/sql/opt/memo/testdata/logprops/constraints index 008b96817f04..932324d0b215 100644 --- a/pkg/sql/opt/memo/testdata/logprops/constraints +++ b/pkg/sql/opt/memo/testdata/logprops/constraints @@ -648,21 +648,21 @@ opt SELECT * FROM (SELECT (x, y) AS col FROM a) WHERE col > (1, 2) ---- select - ├── columns: col:4(tuple{int, int}!null) + ├── columns: col:5(tuple{int, int}!null) ├── immutable ├── project - │ ├── columns: col:4(tuple{int, int}) - │ ├── prune: (4) + │ ├── columns: col:5(tuple{int, int}) + │ ├── prune: (5) │ ├── scan a │ │ ├── columns: x:1(int) y:2(int) │ │ └── prune: (1,2) │ └── projections - │ └── tuple [as=col:4, type=tuple{int, int}, outer=(1,2)] + │ └── tuple [as=col:5, type=tuple{int, int}, outer=(1,2)] │ ├── variable: x:1 [type=int] │ └── variable: y:2 [type=int] └── filters - └── gt [type=bool, outer=(4), immutable, constraints=(/4: [/(1, 3) - ]; tight)] - ├── variable: col:4 [type=tuple{int, int}] + └── gt [type=bool, outer=(5), immutable, constraints=(/5: [/(1, 3) - ]; tight)] + ├── variable: col:5 [type=tuple{int, int}] └── tuple [type=tuple{int, int}] ├── const: 1 [type=int] └── const: 2 [type=int] @@ -988,20 +988,20 @@ opt SELECT * FROM (SELECT (x, y) AS foo FROM a) WHERE foo IN ((3, 4), (1, 2)) ---- select - ├── columns: foo:4(tuple{int, int}!null) + ├── columns: foo:5(tuple{int, int}!null) ├── project - │ ├── columns: foo:4(tuple{int, int}) - │ ├── prune: (4) + │ ├── columns: foo:5(tuple{int, int}) + │ ├── prune: (5) │ ├── scan a │ │ ├── columns: x:1(int) y:2(int) │ │ └── prune: (1,2) │ └── projections - │ └── tuple [as=foo:4, type=tuple{int, int}, outer=(1,2)] + │ └── tuple [as=foo:5, type=tuple{int, int}, outer=(1,2)] │ ├── variable: x:1 [type=int] │ └── variable: y:2 [type=int] └── filters - └── in [type=bool, outer=(4), constraints=(/4: [/(1, 2) - /(1, 2)] [/(3, 4) - /(3, 4)]; tight)] - ├── variable: foo:4 [type=tuple{int, int}] + └── in [type=bool, outer=(5), constraints=(/5: [/(1, 2) - /(1, 2)] [/(3, 4) - /(3, 4)]; tight)] + ├── variable: foo:5 [type=tuple{int, int}] └── tuple [type=tuple{tuple{int, int}, tuple{int, int}}] ├── tuple [type=tuple{int, int}] │ ├── const: 3 [type=int] diff --git a/pkg/sql/opt/memo/testdata/logprops/delete b/pkg/sql/opt/memo/testdata/logprops/delete index 2e3b1daf8e74..a499a2b85e91 100644 --- a/pkg/sql/opt/memo/testdata/logprops/delete +++ b/pkg/sql/opt/memo/testdata/logprops/delete @@ -22,31 +22,31 @@ DELETE FROM abcde WHERE a=1 ---- delete abcde ├── columns: - ├── fetch columns: a:7(int) b:8(int) c:9(int) d:10(int) rowid:11(int) e:12(int) + ├── fetch columns: a:8(int) b:9(int) c:10(int) d:11(int) rowid:12(int) e:13(int) ├── cardinality: [0 - 0] ├── volatile, mutations └── select - ├── columns: a:7(int!null) b:8(int) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) - ├── key: (11) - ├── fd: ()-->(7), (11)-->(8-10,12) - ├── prune: (8-12) - ├── interesting orderings: (+11) + ├── columns: a:8(int!null) b:9(int) c:10(int!null) d:11(int) rowid:12(int!null) e:13(int) crdb_internal_mvcc_timestamp:14(decimal) + ├── key: (12) + ├── fd: ()-->(8), (12)-->(9-11,13,14) + ├── prune: (9-14) + ├── interesting orderings: (+12) ├── scan abcde - │ ├── columns: a:7(int!null) b:8(int) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) + │ ├── columns: a:8(int!null) b:9(int) c:10(int!null) d:11(int) rowid:12(int!null) e:13(int) crdb_internal_mvcc_timestamp:14(decimal) │ ├── computed column expressions - │ │ └── d:10 + │ │ └── d:11 │ │ └── plus [type=int] │ │ ├── plus [type=int] - │ │ │ ├── variable: b:8 [type=int] - │ │ │ └── variable: c:9 [type=int] + │ │ │ ├── variable: b:9 [type=int] + │ │ │ └── variable: c:10 [type=int] │ │ └── const: 1 [type=int] - │ ├── key: (11) - │ ├── fd: (11)-->(7-10,12) - │ ├── prune: (7-12) - │ └── interesting orderings: (+11) + │ ├── key: (12) + │ ├── fd: (12)-->(8-11,13,14) + │ ├── prune: (8-14) + │ └── interesting orderings: (+12) └── filters - └── eq [type=bool, outer=(7), constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] - ├── variable: a:7 [type=int] + └── eq [type=bool, outer=(8), constraints=(/8: [/1 - /1]; tight), fd=()-->(8)] + ├── variable: a:8 [type=int] └── const: 1 [type=int] # Properties with RETURNING clause. @@ -60,33 +60,33 @@ project ├── prune: (1-4) └── delete abcde ├── columns: a:1(int!null) b:2(int) c:3(int!null) d:4(int) rowid:5(int!null) - ├── fetch columns: a:7(int) b:8(int) c:9(int) d:10(int) rowid:11(int) e:12(int) + ├── fetch columns: a:8(int) b:9(int) c:10(int) d:11(int) rowid:12(int) e:13(int) ├── volatile, mutations ├── key: (5) ├── fd: ()-->(1), (5)-->(2-4) ├── prune: (1-4) └── select - ├── columns: a:7(int!null) b:8(int) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) - ├── key: (11) - ├── fd: ()-->(7), (11)-->(8-10,12) - ├── prune: (8-12) - ├── interesting orderings: (+11) + ├── columns: a:8(int!null) b:9(int) c:10(int!null) d:11(int) rowid:12(int!null) e:13(int) crdb_internal_mvcc_timestamp:14(decimal) + ├── key: (12) + ├── fd: ()-->(8), (12)-->(9-11,13,14) + ├── prune: (9-14) + ├── interesting orderings: (+12) ├── scan abcde - │ ├── columns: a:7(int!null) b:8(int) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) + │ ├── columns: a:8(int!null) b:9(int) c:10(int!null) d:11(int) rowid:12(int!null) e:13(int) crdb_internal_mvcc_timestamp:14(decimal) │ ├── computed column expressions - │ │ └── d:10 + │ │ └── d:11 │ │ └── plus [type=int] │ │ ├── plus [type=int] - │ │ │ ├── variable: b:8 [type=int] - │ │ │ └── variable: c:9 [type=int] + │ │ │ ├── variable: b:9 [type=int] + │ │ │ └── variable: c:10 [type=int] │ │ └── const: 1 [type=int] - │ ├── key: (11) - │ ├── fd: (11)-->(7-10,12) - │ ├── prune: (7-12) - │ └── interesting orderings: (+11) + │ ├── key: (12) + │ ├── fd: (12)-->(8-11,13,14) + │ ├── prune: (8-14) + │ └── interesting orderings: (+12) └── filters - └── eq [type=bool, outer=(7), constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] - ├── variable: a:7 [type=int] + └── eq [type=bool, outer=(8), constraints=(/8: [/1 - /1]; tight), fd=()-->(8)] + ├── variable: a:8 [type=int] └── const: 1 [type=int] # Input is cardinality 0-1 expression. @@ -102,35 +102,35 @@ project ├── prune: (1-4) └── delete abcde ├── columns: a:1(int!null) b:2(int) c:3(int!null) d:4(int) rowid:5(int!null) - ├── fetch columns: a:7(int) b:8(int) c:9(int) d:10(int) rowid:11(int) e:12(int) + ├── fetch columns: a:8(int) b:9(int) c:10(int) d:11(int) rowid:12(int) e:13(int) ├── cardinality: [0 - 1] ├── volatile, mutations ├── key: () ├── fd: ()-->(1-5) ├── prune: (1-4) └── select - ├── columns: a:7(int!null) b:8(int) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) + ├── columns: a:8(int!null) b:9(int) c:10(int!null) d:11(int) rowid:12(int!null) e:13(int) crdb_internal_mvcc_timestamp:14(decimal) ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(7-12) - ├── prune: (7-10,12) - ├── interesting orderings: (+11) + ├── fd: ()-->(8-14) + ├── prune: (8-11,13,14) + ├── interesting orderings: (+12) ├── scan abcde - │ ├── columns: a:7(int!null) b:8(int) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) + │ ├── columns: a:8(int!null) b:9(int) c:10(int!null) d:11(int) rowid:12(int!null) e:13(int) crdb_internal_mvcc_timestamp:14(decimal) │ ├── computed column expressions - │ │ └── d:10 + │ │ └── d:11 │ │ └── plus [type=int] │ │ ├── plus [type=int] - │ │ │ ├── variable: b:8 [type=int] - │ │ │ └── variable: c:9 [type=int] + │ │ │ ├── variable: b:9 [type=int] + │ │ │ └── variable: c:10 [type=int] │ │ └── const: 1 [type=int] - │ ├── key: (11) - │ ├── fd: (11)-->(7-10,12) - │ ├── prune: (7-12) - │ └── interesting orderings: (+11) + │ ├── key: (12) + │ ├── fd: (12)-->(8-11,13,14) + │ ├── prune: (8-14) + │ └── interesting orderings: (+12) └── filters - └── eq [type=bool, outer=(11), constraints=(/11: [/1 - /1]; tight), fd=()-->(11)] - ├── variable: rowid:11 [type=int] + └── eq [type=bool, outer=(12), constraints=(/12: [/1 - /1]; tight), fd=()-->(12)] + ├── variable: rowid:12 [type=int] └── const: 1 [type=int] # Filter FD set. @@ -144,31 +144,31 @@ project ├── prune: (1-4) └── delete abcde ├── columns: a:1(int!null) b:2(int!null) c:3(int!null) d:4(int) rowid:5(int!null) - ├── fetch columns: a:7(int) b:8(int) c:9(int) d:10(int) rowid:11(int) e:12(int) + ├── fetch columns: a:8(int) b:9(int) c:10(int) d:11(int) rowid:12(int) e:13(int) ├── volatile, mutations ├── key: (5) ├── fd: (2)==(3), (3)==(2), (5)-->(1-4) ├── prune: (1-4) └── select - ├── columns: a:7(int!null) b:8(int!null) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) - ├── key: (11) - ├── fd: (11)-->(7-10,12), (8)==(9), (9)==(8) - ├── prune: (7,10-12) - ├── interesting orderings: (+11) + ├── columns: a:8(int!null) b:9(int!null) c:10(int!null) d:11(int) rowid:12(int!null) e:13(int) crdb_internal_mvcc_timestamp:14(decimal) + ├── key: (12) + ├── fd: (12)-->(8-11,13,14), (9)==(10), (10)==(9) + ├── prune: (8,11-14) + ├── interesting orderings: (+12) ├── scan abcde - │ ├── columns: a:7(int!null) b:8(int) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) + │ ├── columns: a:8(int!null) b:9(int) c:10(int!null) d:11(int) rowid:12(int!null) e:13(int) crdb_internal_mvcc_timestamp:14(decimal) │ ├── computed column expressions - │ │ └── d:10 + │ │ └── d:11 │ │ └── plus [type=int] │ │ ├── plus [type=int] - │ │ │ ├── variable: b:8 [type=int] - │ │ │ └── variable: c:9 [type=int] + │ │ │ ├── variable: b:9 [type=int] + │ │ │ └── variable: c:10 [type=int] │ │ └── const: 1 [type=int] - │ ├── key: (11) - │ ├── fd: (11)-->(7-10,12) - │ ├── prune: (7-12) - │ └── interesting orderings: (+11) + │ ├── key: (12) + │ ├── fd: (12)-->(8-11,13,14) + │ ├── prune: (8-14) + │ └── interesting orderings: (+12) └── filters - └── eq [type=bool, outer=(8,9), constraints=(/8: (/NULL - ]; /9: (/NULL - ]), fd=(8)==(9), (9)==(8)] - ├── variable: b:8 [type=int] - └── variable: c:9 [type=int] + └── eq [type=bool, outer=(9,10), constraints=(/9: (/NULL - ]; /10: (/NULL - ]), fd=(9)==(10), (10)==(9)] + ├── variable: b:9 [type=int] + └── variable: c:10 [type=int] diff --git a/pkg/sql/opt/memo/testdata/logprops/groupby b/pkg/sql/opt/memo/testdata/logprops/groupby index 7a6ebdc48d4f..b38f13ddf433 100644 --- a/pkg/sql/opt/memo/testdata/logprops/groupby +++ b/pkg/sql/opt/memo/testdata/logprops/groupby @@ -14,69 +14,69 @@ WHERE s IS NOT NULL GROUP BY x, y ---- project - ├── columns: y:2(int) sum:5(float!null) x:1(int!null) bool:10(bool!null) avg:7(float) string_agg:9(string!null) + ├── columns: y:2(int) sum:6(float!null) x:1(int!null) bool:11(bool!null) avg:8(float) string_agg:10(string!null) ├── key: (1) - ├── fd: ()-->(10), (1)-->(2,5,7,9) - ├── prune: (1,2,5,7,9,10) + ├── fd: ()-->(11), (1)-->(2,6,8,10) + ├── prune: (1,2,6,8,10,11) ├── interesting orderings: (+1) ├── group-by - │ ├── columns: x:1(int!null) y:2(int) sum:5(float!null) avg:7(float) string_agg:9(string!null) + │ ├── columns: x:1(int!null) y:2(int) sum:6(float!null) avg:8(float) string_agg:10(string!null) │ ├── grouping columns: x:1(int!null) y:2(int) │ ├── key: (1) - │ ├── fd: (1)-->(2,5,7,9) - │ ├── prune: (5,7,9) + │ ├── fd: (1)-->(2,6,8,10) + │ ├── prune: (6,8,10) │ ├── interesting orderings: (+1) │ ├── project - │ │ ├── columns: column6:6(bool!null) column8:8(string!null) x:1(int!null) y:2(int) z:3(float!null) s:4(string!null) + │ │ ├── columns: column7:7(bool!null) column9:9(string!null) x:1(int!null) y:2(int) z:3(float!null) s:4(string!null) │ │ ├── key: (1) - │ │ ├── fd: ()-->(8), (1)-->(2-4,6), (3,4)-->(1,2) - │ │ ├── prune: (1-4,6,8) + │ │ ├── fd: ()-->(9), (1)-->(2-4,7), (3,4)-->(1,2) + │ │ ├── prune: (1-4,7,9) │ │ ├── interesting orderings: (+1) (-4,+3,+1) │ │ ├── select - │ │ │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string!null) + │ │ │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string!null) crdb_internal_mvcc_timestamp:5(decimal) │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(2-4), (3,4)-->(1,2) - │ │ │ ├── prune: (1-3) + │ │ │ ├── fd: (1)-->(2-5), (3,4)-->(1,2,5) + │ │ │ ├── prune: (1-3,5) │ │ │ ├── interesting orderings: (+1) (-4,+3,+1) │ │ │ ├── scan xyzs - │ │ │ │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) + │ │ │ │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) crdb_internal_mvcc_timestamp:5(decimal) │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ │ │ │ ├── prune: (1-4) + │ │ │ │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ │ │ │ ├── prune: (1-5) │ │ │ │ └── interesting orderings: (+1) (-4,+3,+1) │ │ │ └── filters │ │ │ └── is-not [type=bool, outer=(4), constraints=(/4: (/NULL - ]; tight)] │ │ │ ├── variable: s:4 [type=string] │ │ │ └── null [type=unknown] │ │ └── projections - │ │ ├── gt [as=column6:6, type=bool, outer=(3)] + │ │ ├── gt [as=column7:7, type=bool, outer=(3)] │ │ │ ├── variable: z:3 [type=float] │ │ │ └── const: 0.0 [type=float] - │ │ └── const: ',' [as=column8:8, type=string] + │ │ └── const: ',' [as=column9:9, type=string] │ └── aggregations - │ ├── sum [as=sum:5, type=float, outer=(3)] + │ ├── sum [as=sum:6, type=float, outer=(3)] │ │ └── variable: z:3 [type=float] - │ ├── agg-filter [as=avg:7, type=float, outer=(3,6)] + │ ├── agg-filter [as=avg:8, type=float, outer=(3,7)] │ │ ├── avg [type=float] │ │ │ └── variable: z:3 [type=float] - │ │ └── variable: column6:6 [type=bool] - │ └── agg-distinct [as=string_agg:9, type=string, outer=(4,8)] + │ │ └── variable: column7:7 [type=bool] + │ └── agg-distinct [as=string_agg:10, type=string, outer=(4,9)] │ └── string-agg [type=string] │ ├── variable: s:4 [type=string] - │ └── variable: column8:8 [type=string] + │ └── variable: column9:9 [type=string] └── projections - └── false [as=bool:10, type=bool] + └── false [as=bool:11, type=bool] # Scalar groupby. build SELECT sum(x), max(y), count(x) FROM xyzs ---- scalar-group-by - ├── columns: sum:5(decimal) max:6(int) count:7(int!null) + ├── columns: sum:6(decimal) max:7(int) count:8(int!null) ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(5-7) - ├── prune: (5-7) + ├── fd: ()-->(6-8) + ├── prune: (6-8) ├── project │ ├── columns: x:1(int!null) y:2(int) │ ├── key: (1) @@ -84,17 +84,17 @@ scalar-group-by │ ├── prune: (1,2) │ ├── interesting orderings: (+1) │ └── scan xyzs - │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) + │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) crdb_internal_mvcc_timestamp:5(decimal) │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ ├── prune: (1-4) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── prune: (1-5) │ └── interesting orderings: (+1) (-4,+3,+1) └── aggregations - ├── sum [as=sum:5, type=decimal, outer=(1)] + ├── sum [as=sum:6, type=decimal, outer=(1)] │ └── variable: x:1 [type=int] - ├── max [as=max:6, type=int, outer=(2)] + ├── max [as=max:7, type=int, outer=(2)] │ └── variable: y:2 [type=int] - └── count [as=count:7, type=int, outer=(1)] + └── count [as=count:8, type=int, outer=(1)] └── variable: x:1 [type=int] # Group by unique index columns. @@ -116,10 +116,10 @@ project ├── prune: (3,4) ├── interesting orderings: (-4,+3) └── scan xyzs - ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) + ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) crdb_internal_mvcc_timestamp:5(decimal) ├── key: (1) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - ├── prune: (1-4) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + ├── prune: (1-5) └── interesting orderings: (+1) (-4,+3,+1) # Group by columns that otherwise wouldn't be weak key. @@ -127,25 +127,25 @@ build SELECT y, sum(z) FROM xyzs GROUP BY z, y ---- project - ├── columns: y:2(int) sum:5(float!null) - ├── prune: (2,5) + ├── columns: y:2(int) sum:6(float!null) + ├── prune: (2,6) └── group-by - ├── columns: y:2(int) z:3(float!null) sum:5(float!null) + ├── columns: y:2(int) z:3(float!null) sum:6(float!null) ├── grouping columns: y:2(int) z:3(float!null) ├── key: (2,3) - ├── fd: (2,3)-->(5) - ├── prune: (5) + ├── fd: (2,3)-->(6) + ├── prune: (6) ├── project │ ├── columns: y:2(int) z:3(float!null) │ ├── prune: (2,3) │ └── scan xyzs - │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) + │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) crdb_internal_mvcc_timestamp:5(decimal) │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ ├── prune: (1-4) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── prune: (1-5) │ └── interesting orderings: (+1) (-4,+3,+1) └── aggregations - └── sum [as=sum:5, type=float, outer=(3)] + └── sum [as=sum:6, type=float, outer=(3)] └── variable: z:3 [type=float] # Group by column that is subset of unique index. @@ -153,24 +153,24 @@ build SELECT z, max(s) FROM xyzs GROUP BY z ---- group-by - ├── columns: z:3(float!null) max:5(string) + ├── columns: z:3(float!null) max:6(string) ├── grouping columns: z:3(float!null) ├── key: (3) - ├── fd: (3)-->(5) - ├── prune: (5) + ├── fd: (3)-->(6) + ├── prune: (6) ├── project │ ├── columns: z:3(float!null) s:4(string) │ ├── lax-key: (3,4) │ ├── prune: (3,4) │ ├── interesting orderings: (-4,+3) │ └── scan xyzs - │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) + │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) crdb_internal_mvcc_timestamp:5(decimal) │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ ├── prune: (1-4) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── prune: (1-5) │ └── interesting orderings: (+1) (-4,+3,+1) └── aggregations - └── max [as=max:5, type=string, outer=(4)] + └── max [as=max:6, type=string, outer=(4)] └── variable: s:4 [type=string] # Group by all columns. @@ -187,80 +187,86 @@ project ├── key: (1) ├── fd: (1)-->(2-4), (3,4)~~>(1,2) ├── interesting orderings: (+1) (-4,+3,+1) - └── scan xyzs + └── project ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) ├── key: (1) ├── fd: (1)-->(2-4), (3,4)~~>(1,2) ├── prune: (1-4) - └── interesting orderings: (+1) (-4,+3,+1) + ├── interesting orderings: (+1) (-4,+3,+1) + └── scan xyzs + ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) crdb_internal_mvcc_timestamp:5(decimal) + ├── key: (1) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + ├── prune: (1-5) + └── interesting orderings: (+1) (-4,+3,+1) # Propagate outer columns. build SELECT (SELECT sum(x) FROM (SELECT y, u FROM kuv) GROUP BY u) FROM xyzs GROUP BY y ---- project - ├── columns: sum:12(decimal) - ├── prune: (12) + ├── columns: sum:14(decimal) + ├── prune: (14) ├── group-by - │ ├── columns: xyzs.y:2(int) sum:10(decimal!null) + │ ├── columns: xyzs.y:2(int) sum:12(decimal!null) │ ├── grouping columns: xyzs.y:2(int) │ ├── key: (2) - │ ├── fd: (2)-->(10) - │ ├── prune: (10) + │ ├── fd: (2)-->(12) + │ ├── prune: (12) │ ├── project - │ │ ├── columns: x:9(int!null) xyzs.y:2(int) - │ │ ├── key: (9) - │ │ ├── fd: (9)-->(2) - │ │ ├── prune: (2,9) + │ │ ├── columns: x:11(int!null) xyzs.y:2(int) + │ │ ├── key: (11) + │ │ ├── fd: (11)-->(2) + │ │ ├── prune: (2,11) │ │ ├── scan xyzs - │ │ │ ├── columns: xyzs.x:1(int!null) xyzs.y:2(int) z:3(float!null) s:4(string) + │ │ │ ├── columns: xyzs.x:1(int!null) xyzs.y:2(int) z:3(float!null) s:4(string) xyzs.crdb_internal_mvcc_timestamp:5(decimal) │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ │ │ ├── prune: (1-4) + │ │ │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ │ │ ├── prune: (1-5) │ │ │ └── interesting orderings: (+1) (-4,+3,+1) │ │ └── projections - │ │ └── variable: xyzs.x:1 [as=x:9, type=int, outer=(1)] + │ │ └── variable: xyzs.x:1 [as=x:11, type=int, outer=(1)] │ └── aggregations - │ └── sum [as=sum:10, type=decimal, outer=(9)] - │ └── variable: x:9 [type=int] + │ └── sum [as=sum:12, type=decimal, outer=(11)] + │ └── variable: x:11 [type=int] └── projections - └── subquery [as=sum:12, type=decimal, outer=(2,10), correlated-subquery] + └── subquery [as=sum:14, type=decimal, outer=(2,12), correlated-subquery] └── max1-row - ├── columns: sum:11(decimal) + ├── columns: sum:13(decimal) ├── error: "more than one row returned by a subquery used as an expression" - ├── outer: (2,10) + ├── outer: (2,12) ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(11) + ├── fd: ()-->(13) └── project - ├── columns: sum:11(decimal) - ├── outer: (2,10) - ├── fd: ()-->(11) - ├── prune: (11) + ├── columns: sum:13(decimal) + ├── outer: (2,12) + ├── fd: ()-->(13) + ├── prune: (13) ├── group-by - │ ├── columns: u:6(float) - │ ├── grouping columns: u:6(float) + │ ├── columns: u:7(float) + │ ├── grouping columns: u:7(float) │ ├── outer: (2) - │ ├── key: (6) + │ ├── key: (7) │ └── project - │ ├── columns: u:6(float) + │ ├── columns: u:7(float) │ ├── outer: (2) - │ ├── prune: (6) + │ ├── prune: (7) │ └── project - │ ├── columns: y:8(int) u:6(float) + │ ├── columns: y:10(int) u:7(float) │ ├── outer: (2) - │ ├── fd: ()-->(8) - │ ├── prune: (6,8) + │ ├── fd: ()-->(10) + │ ├── prune: (7,10) │ ├── scan kuv - │ │ ├── columns: k:5(int!null) u:6(float) v:7(string) - │ │ ├── key: (5) - │ │ ├── fd: (5)-->(6,7) - │ │ ├── prune: (5-7) - │ │ └── interesting orderings: (+5) + │ │ ├── columns: k:6(int!null) u:7(float) v:8(string) kuv.crdb_internal_mvcc_timestamp:9(decimal) + │ │ ├── key: (6) + │ │ ├── fd: (6)-->(7-9) + │ │ ├── prune: (6-9) + │ │ └── interesting orderings: (+6) │ └── projections - │ └── variable: xyzs.y:2 [as=y:8, type=int, outer=(2)] + │ └── variable: xyzs.y:2 [as=y:10, type=int, outer=(2)] └── projections - └── variable: sum:10 [as=sum:11, type=decimal, outer=(10)] + └── variable: sum:12 [as=sum:13, type=decimal, outer=(12)] # Calculate groupby cardinality. build @@ -292,11 +298,11 @@ opt SELECT x, count(y) FROM xyzs GROUP BY x HAVING x=1 ---- group-by - ├── columns: x:1(int!null) count:5(int!null) + ├── columns: x:1(int!null) count:6(int!null) ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(1,5) - ├── prune: (1,5) + ├── fd: ()-->(1,6) + ├── prune: (1,6) ├── scan xyzs │ ├── columns: x:1(int!null) y:2(int) │ ├── constraint: /1: [/1 - /1] @@ -306,7 +312,7 @@ group-by │ ├── prune: (2) │ └── interesting orderings: (+1) └── aggregations - ├── count [as=count:5, type=int, outer=(2)] + ├── count [as=count:6, type=int, outer=(2)] │ └── variable: y:2 [type=int] └── const-agg [as=x:1, type=int, outer=(1)] └── variable: x:1 [type=int] @@ -319,14 +325,14 @@ FROM xyzs GROUP BY x, y ---- project - ├── columns: variance:5(decimal) stddev:6(decimal) corr:7(float) - ├── prune: (5-7) + ├── columns: variance:6(decimal) stddev:7(decimal) corr:8(float) + ├── prune: (6-8) └── group-by - ├── columns: x:1(int!null) y:2(int) variance:5(decimal) stddev:6(decimal) corr:7(float) + ├── columns: x:1(int!null) y:2(int) variance:6(decimal) stddev:7(decimal) corr:8(float) ├── grouping columns: x:1(int!null) y:2(int) ├── key: (1) - ├── fd: (1)-->(2,5-7) - ├── prune: (5-7) + ├── fd: (1)-->(2,6-8) + ├── prune: (6-8) ├── interesting orderings: (+1) ├── project │ ├── columns: x:1(int!null) y:2(int) @@ -335,16 +341,16 @@ project │ ├── prune: (1,2) │ ├── interesting orderings: (+1) │ └── scan xyzs - │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) + │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) crdb_internal_mvcc_timestamp:5(decimal) │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ ├── prune: (1-4) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── prune: (1-5) │ └── interesting orderings: (+1) (-4,+3,+1) └── aggregations - ├── variance [as=variance:5, type=decimal, outer=(1)] + ├── variance [as=variance:6, type=decimal, outer=(1)] │ └── variable: x:1 [type=int] - ├── std-dev [as=stddev:6, type=decimal, outer=(1)] + ├── std-dev [as=stddev:7, type=decimal, outer=(1)] │ └── variable: x:1 [type=int] - └── corr [as=corr:7, type=float, outer=(1,2)] + └── corr [as=corr:8, type=float, outer=(1,2)] ├── variable: x:1 [type=int] └── variable: y:2 [type=int] diff --git a/pkg/sql/opt/memo/testdata/logprops/insert b/pkg/sql/opt/memo/testdata/logprops/insert index 370e43c147e7..a19856c8f943 100644 --- a/pkg/sql/opt/memo/testdata/logprops/insert +++ b/pkg/sql/opt/memo/testdata/logprops/insert @@ -23,58 +23,58 @@ INSERT INTO abcde (a, b) SELECT y, y FROM xyz ORDER BY y, z LIMIT 10 insert abcde ├── columns: ├── insert-mapping: - │ ├── y:8 => a:1 - │ ├── y:8 => b:2 - │ ├── column10:10 => c:3 - │ ├── column13:13 => d:4 - │ ├── column11:11 => rowid:5 - │ └── column12:12 => e:6 + │ ├── y:9 => a:1 + │ ├── y:9 => b:2 + │ ├── column12:12 => c:3 + │ ├── column15:15 => d:4 + │ ├── column13:13 => rowid:5 + │ └── column14:14 => e:6 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: column13:13(int!null) y:8(int!null) column10:10(int!null) column11:11(int) column12:12(int) + ├── columns: column15:15(int!null) y:9(int!null) column12:12(int!null) column13:13(int) column14:14(int) ├── cardinality: [0 - 10] ├── volatile - ├── fd: ()-->(10,12), (8)-->(13) - ├── prune: (8,10-13) - ├── interesting orderings: (+8) + ├── fd: ()-->(12,14), (9)-->(15) + ├── prune: (9,12-15) + ├── interesting orderings: (+9) ├── project - │ ├── columns: column10:10(int!null) column11:11(int) column12:12(int) y:8(int!null) + │ ├── columns: column12:12(int!null) column13:13(int) column14:14(int) y:9(int!null) │ ├── cardinality: [0 - 10] │ ├── volatile - │ ├── fd: ()-->(10,12) - │ ├── prune: (8,10-12) - │ ├── interesting orderings: (+8) + │ ├── fd: ()-->(12,14) + │ ├── prune: (9,12-14) + │ ├── interesting orderings: (+9) │ ├── limit - │ │ ├── columns: y:8(int!null) z:9(float) - │ │ ├── internal-ordering: +8,+9 + │ │ ├── columns: y:9(int!null) z:10(float) + │ │ ├── internal-ordering: +9,+10 │ │ ├── cardinality: [0 - 10] - │ │ ├── interesting orderings: (+8,+9) + │ │ ├── interesting orderings: (+9,+10) │ │ ├── sort - │ │ │ ├── columns: y:8(int!null) z:9(float) - │ │ │ ├── ordering: +8,+9 + │ │ │ ├── columns: y:9(int!null) z:10(float) + │ │ │ ├── ordering: +9,+10 │ │ │ ├── limit hint: 10.00 - │ │ │ ├── prune: (8,9) + │ │ │ ├── prune: (9,10) │ │ │ └── project - │ │ │ ├── columns: y:8(int!null) z:9(float) - │ │ │ ├── prune: (8,9) + │ │ │ ├── columns: y:9(int!null) z:10(float) + │ │ │ ├── prune: (9,10) │ │ │ └── scan xyz - │ │ │ ├── columns: x:7(string!null) y:8(int!null) z:9(float) - │ │ │ ├── key: (7) - │ │ │ ├── fd: (7)-->(8,9) - │ │ │ ├── prune: (7-9) - │ │ │ └── interesting orderings: (+7) + │ │ │ ├── columns: x:8(string!null) y:9(int!null) z:10(float) xyz.crdb_internal_mvcc_timestamp:11(decimal) + │ │ │ ├── key: (8) + │ │ │ ├── fd: (8)-->(9-11) + │ │ │ ├── prune: (8-11) + │ │ │ └── interesting orderings: (+8) │ │ └── const: 10 [type=int] │ └── projections - │ ├── const: 10 [as=column10:10, type=int] - │ ├── function: unique_rowid [as=column11:11, type=int, volatile] - │ └── cast: INT8 [as=column12:12, type=int, immutable] + │ ├── const: 10 [as=column12:12, type=int] + │ ├── function: unique_rowid [as=column13:13, type=int, volatile] + │ └── cast: INT8 [as=column14:14, type=int, immutable] │ └── null [type=unknown] └── projections - └── plus [as=column13:13, type=int, outer=(8,10), immutable] + └── plus [as=column15:15, type=int, outer=(9,12), immutable] ├── plus [type=int] - │ ├── variable: y:8 [type=int] - │ └── variable: column10:10 [type=int] + │ ├── variable: y:9 [type=int] + │ └── variable: column12:12 [type=int] └── const: 1 [type=int] # Properties with RETURNING clause. @@ -90,59 +90,59 @@ project └── insert abcde ├── columns: a:1(int!null) b:2(int!null) c:3(int!null) d:4(int!null) rowid:5(int!null) ├── insert-mapping: - │ ├── y:8 => a:1 - │ ├── y:8 => b:2 - │ ├── column10:10 => c:3 - │ ├── column13:13 => d:4 - │ ├── column11:11 => rowid:5 - │ └── column12:12 => e:6 + │ ├── y:9 => a:1 + │ ├── y:9 => b:2 + │ ├── column12:12 => c:3 + │ ├── column15:15 => d:4 + │ ├── column13:13 => rowid:5 + │ └── column14:14 => e:6 ├── cardinality: [0 - 10] ├── volatile, mutations ├── fd: ()-->(3), (1)==(2), (2)==(1), (1)-->(4) └── project - ├── columns: column13:13(int!null) y:8(int!null) column10:10(int!null) column11:11(int) column12:12(int) + ├── columns: column15:15(int!null) y:9(int!null) column12:12(int!null) column13:13(int) column14:14(int) ├── cardinality: [0 - 10] ├── volatile - ├── fd: ()-->(10,12), (8)-->(13) - ├── prune: (8,10-13) - ├── interesting orderings: (+8) + ├── fd: ()-->(12,14), (9)-->(15) + ├── prune: (9,12-15) + ├── interesting orderings: (+9) ├── project - │ ├── columns: column10:10(int!null) column11:11(int) column12:12(int) y:8(int!null) + │ ├── columns: column12:12(int!null) column13:13(int) column14:14(int) y:9(int!null) │ ├── cardinality: [0 - 10] │ ├── volatile - │ ├── fd: ()-->(10,12) - │ ├── prune: (8,10-12) - │ ├── interesting orderings: (+8) + │ ├── fd: ()-->(12,14) + │ ├── prune: (9,12-14) + │ ├── interesting orderings: (+9) │ ├── limit - │ │ ├── columns: y:8(int!null) z:9(float) - │ │ ├── internal-ordering: +8,+9 + │ │ ├── columns: y:9(int!null) z:10(float) + │ │ ├── internal-ordering: +9,+10 │ │ ├── cardinality: [0 - 10] - │ │ ├── interesting orderings: (+8,+9) + │ │ ├── interesting orderings: (+9,+10) │ │ ├── sort - │ │ │ ├── columns: y:8(int!null) z:9(float) - │ │ │ ├── ordering: +8,+9 + │ │ │ ├── columns: y:9(int!null) z:10(float) + │ │ │ ├── ordering: +9,+10 │ │ │ ├── limit hint: 10.00 - │ │ │ ├── prune: (8,9) + │ │ │ ├── prune: (9,10) │ │ │ └── project - │ │ │ ├── columns: y:8(int!null) z:9(float) - │ │ │ ├── prune: (8,9) + │ │ │ ├── columns: y:9(int!null) z:10(float) + │ │ │ ├── prune: (9,10) │ │ │ └── scan xyz - │ │ │ ├── columns: x:7(string!null) y:8(int!null) z:9(float) - │ │ │ ├── key: (7) - │ │ │ ├── fd: (7)-->(8,9) - │ │ │ ├── prune: (7-9) - │ │ │ └── interesting orderings: (+7) + │ │ │ ├── columns: x:8(string!null) y:9(int!null) z:10(float) xyz.crdb_internal_mvcc_timestamp:11(decimal) + │ │ │ ├── key: (8) + │ │ │ ├── fd: (8)-->(9-11) + │ │ │ ├── prune: (8-11) + │ │ │ └── interesting orderings: (+8) │ │ └── const: 10 [type=int] │ └── projections - │ ├── const: 10 [as=column10:10, type=int] - │ ├── function: unique_rowid [as=column11:11, type=int, volatile] - │ └── cast: INT8 [as=column12:12, type=int, immutable] + │ ├── const: 10 [as=column12:12, type=int] + │ ├── function: unique_rowid [as=column13:13, type=int, volatile] + │ └── cast: INT8 [as=column14:14, type=int, immutable] │ └── null [type=unknown] └── projections - └── plus [as=column13:13, type=int, outer=(8,10), immutable] + └── plus [as=column15:15, type=int, outer=(9,12), immutable] ├── plus [type=int] - │ ├── variable: y:8 [type=int] - │ └── variable: column10:10 [type=int] + │ ├── variable: y:9 [type=int] + │ └── variable: column12:12 [type=int] └── const: 1 [type=int] # Properties with RETURNING clause. @@ -157,43 +157,43 @@ project └── insert abcde ├── columns: a:1(int!null) b:2(int!null) c:3(int!null) d:4(int!null) rowid:5(int!null) ├── insert-mapping: - │ ├── y:8 => a:1 - │ ├── y:8 => b:2 - │ ├── column10:10 => c:3 - │ ├── column13:13 => d:4 - │ ├── column11:11 => rowid:5 - │ └── column12:12 => e:6 + │ ├── y:9 => a:1 + │ ├── y:9 => b:2 + │ ├── column12:12 => c:3 + │ ├── column15:15 => d:4 + │ ├── column13:13 => rowid:5 + │ └── column14:14 => e:6 ├── volatile, mutations ├── fd: ()-->(3), (1)==(2), (2)==(1), (1)-->(4) └── project - ├── columns: column13:13(int!null) y:8(int!null) column10:10(int!null) column11:11(int) column12:12(int) + ├── columns: column15:15(int!null) y:9(int!null) column12:12(int!null) column13:13(int) column14:14(int) ├── volatile - ├── fd: ()-->(10,12), (8)-->(13) - ├── prune: (8,10-13) + ├── fd: ()-->(12,14), (9)-->(15) + ├── prune: (9,12-15) ├── project - │ ├── columns: column10:10(int!null) column11:11(int) column12:12(int) y:8(int!null) + │ ├── columns: column12:12(int!null) column13:13(int) column14:14(int) y:9(int!null) │ ├── volatile - │ ├── fd: ()-->(10,12) - │ ├── prune: (8,10-12) + │ ├── fd: ()-->(12,14) + │ ├── prune: (9,12-14) │ ├── project - │ │ ├── columns: y:8(int!null) z:9(float) - │ │ ├── prune: (8,9) + │ │ ├── columns: y:9(int!null) z:10(float) + │ │ ├── prune: (9,10) │ │ └── scan xyz - │ │ ├── columns: x:7(string!null) y:8(int!null) z:9(float) - │ │ ├── key: (7) - │ │ ├── fd: (7)-->(8,9) - │ │ ├── prune: (7-9) - │ │ └── interesting orderings: (+7) + │ │ ├── columns: x:8(string!null) y:9(int!null) z:10(float) xyz.crdb_internal_mvcc_timestamp:11(decimal) + │ │ ├── key: (8) + │ │ ├── fd: (8)-->(9-11) + │ │ ├── prune: (8-11) + │ │ └── interesting orderings: (+8) │ └── projections - │ ├── const: 10 [as=column10:10, type=int] - │ ├── function: unique_rowid [as=column11:11, type=int, volatile] - │ └── cast: INT8 [as=column12:12, type=int, immutable] + │ ├── const: 10 [as=column12:12, type=int] + │ ├── function: unique_rowid [as=column13:13, type=int, volatile] + │ └── cast: INT8 [as=column14:14, type=int, immutable] │ └── null [type=unknown] └── projections - └── plus [as=column13:13, type=int, outer=(8,10), immutable] + └── plus [as=column15:15, type=int, outer=(9,12), immutable] ├── plus [type=int] - │ ├── variable: y:8 [type=int] - │ └── variable: column10:10 [type=int] + │ ├── variable: y:9 [type=int] + │ └── variable: column12:12 [type=int] └── const: 1 [type=int] # Input is cardinality 1 VALUES expression. @@ -203,49 +203,49 @@ INSERT INTO abcde (a, b) (VALUES (1, 2)) RETURNING *, rowid; insert abcde ├── columns: a:1(int!null) b:2(int!null) c:3(int!null) d:4(int!null) rowid:5(int!null) ├── insert-mapping: - │ ├── column1:7 => a:1 - │ ├── column2:8 => b:2 - │ ├── column9:9 => c:3 - │ ├── column12:12 => d:4 - │ ├── column10:10 => rowid:5 - │ └── column11:11 => e:6 + │ ├── column1:8 => a:1 + │ ├── column2:9 => b:2 + │ ├── column10:10 => c:3 + │ ├── column13:13 => d:4 + │ ├── column11:11 => rowid:5 + │ └── column12:12 => e:6 ├── cardinality: [1 - 1] ├── volatile, mutations ├── key: () ├── fd: ()-->(1-5) └── project - ├── columns: column12:12(int!null) column1:7(int!null) column2:8(int!null) column9:9(int!null) column10:10(int) column11:11(int) + ├── columns: column13:13(int!null) column1:8(int!null) column2:9(int!null) column10:10(int!null) column11:11(int) column12:12(int) ├── cardinality: [1 - 1] ├── volatile ├── key: () - ├── fd: ()-->(7-12) - ├── prune: (7-12) + ├── fd: ()-->(8-13) + ├── prune: (8-13) ├── project - │ ├── columns: column9:9(int!null) column10:10(int) column11:11(int) column1:7(int!null) column2:8(int!null) + │ ├── columns: column10:10(int!null) column11:11(int) column12:12(int) column1:8(int!null) column2:9(int!null) │ ├── cardinality: [1 - 1] │ ├── volatile │ ├── key: () - │ ├── fd: ()-->(7-11) - │ ├── prune: (7-11) + │ ├── fd: ()-->(8-12) + │ ├── prune: (8-12) │ ├── values - │ │ ├── columns: column1:7(int!null) column2:8(int!null) + │ │ ├── columns: column1:8(int!null) column2:9(int!null) │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(7,8) - │ │ ├── prune: (7,8) + │ │ ├── fd: ()-->(8,9) + │ │ ├── prune: (8,9) │ │ └── tuple [type=tuple{int, int}] │ │ ├── const: 1 [type=int] │ │ └── const: 2 [type=int] │ └── projections - │ ├── const: 10 [as=column9:9, type=int] - │ ├── function: unique_rowid [as=column10:10, type=int, volatile] - │ └── cast: INT8 [as=column11:11, type=int, immutable] + │ ├── const: 10 [as=column10:10, type=int] + │ ├── function: unique_rowid [as=column11:11, type=int, volatile] + │ └── cast: INT8 [as=column12:12, type=int, immutable] │ └── null [type=unknown] └── projections - └── plus [as=column12:12, type=int, outer=(8,9), immutable] + └── plus [as=column13:13, type=int, outer=(9,10), immutable] ├── plus [type=int] - │ ├── variable: column2:8 [type=int] - │ └── variable: column9:9 [type=int] + │ ├── variable: column2:9 [type=int] + │ └── variable: column10:10 [type=int] └── const: 1 [type=int] # Filter FD set. @@ -260,58 +260,58 @@ project └── insert abcde ├── columns: a:1(int!null) b:2(int) c:3(int!null) d:4(int) rowid:5(int!null) ├── insert-mapping: - │ ├── y:8 => a:1 - │ ├── int8:10 => b:2 - │ ├── column11:11 => c:3 - │ ├── column14:14 => d:4 - │ ├── column12:12 => rowid:5 - │ └── column13:13 => e:6 + │ ├── y:9 => a:1 + │ ├── int8:12 => b:2 + │ ├── column13:13 => c:3 + │ ├── column16:16 => d:4 + │ ├── column14:14 => rowid:5 + │ └── column15:15 => e:6 ├── volatile, mutations ├── fd: ()-->(1,3), (2)-->(4) └── project - ├── columns: column14:14(int) y:8(int!null) int8:10(int) column11:11(int!null) column12:12(int) column13:13(int) + ├── columns: column16:16(int) y:9(int!null) int8:12(int) column13:13(int!null) column14:14(int) column15:15(int) ├── volatile - ├── fd: ()-->(8,11,13), (10)-->(14) - ├── prune: (8,10-14) + ├── fd: ()-->(9,13,15), (12)-->(16) + ├── prune: (9,12-16) ├── project - │ ├── columns: column11:11(int!null) column12:12(int) column13:13(int) y:8(int!null) int8:10(int) + │ ├── columns: column13:13(int!null) column14:14(int) column15:15(int) y:9(int!null) int8:12(int) │ ├── volatile - │ ├── fd: ()-->(8,11,13) - │ ├── prune: (8,10-13) + │ ├── fd: ()-->(9,13,15) + │ ├── prune: (9,12-15) │ ├── project - │ │ ├── columns: int8:10(int) y:8(int!null) + │ │ ├── columns: int8:12(int) y:9(int!null) │ │ ├── immutable - │ │ ├── fd: ()-->(8) - │ │ ├── prune: (8,10) + │ │ ├── fd: ()-->(9) + │ │ ├── prune: (9,12) │ │ ├── select - │ │ │ ├── columns: x:7(string!null) y:8(int!null) z:9(float) - │ │ │ ├── key: (7) - │ │ │ ├── fd: ()-->(8), (7)-->(9) - │ │ │ ├── prune: (7,9) - │ │ │ ├── interesting orderings: (+7) + │ │ │ ├── columns: x:8(string!null) y:9(int!null) z:10(float) xyz.crdb_internal_mvcc_timestamp:11(decimal) + │ │ │ ├── key: (8) + │ │ │ ├── fd: ()-->(9), (8)-->(10,11) + │ │ │ ├── prune: (8,10,11) + │ │ │ ├── interesting orderings: (+8) │ │ │ ├── scan xyz - │ │ │ │ ├── columns: x:7(string!null) y:8(int!null) z:9(float) - │ │ │ │ ├── key: (7) - │ │ │ │ ├── fd: (7)-->(8,9) - │ │ │ │ ├── prune: (7-9) - │ │ │ │ └── interesting orderings: (+7) + │ │ │ │ ├── columns: x:8(string!null) y:9(int!null) z:10(float) xyz.crdb_internal_mvcc_timestamp:11(decimal) + │ │ │ │ ├── key: (8) + │ │ │ │ ├── fd: (8)-->(9-11) + │ │ │ │ ├── prune: (8-11) + │ │ │ │ └── interesting orderings: (+8) │ │ │ └── filters - │ │ │ └── eq [type=bool, outer=(8), constraints=(/8: [/1 - /1]; tight), fd=()-->(8)] - │ │ │ ├── variable: y:8 [type=int] + │ │ │ └── eq [type=bool, outer=(9), constraints=(/9: [/1 - /1]; tight), fd=()-->(9)] + │ │ │ ├── variable: y:9 [type=int] │ │ │ └── const: 1 [type=int] │ │ └── projections - │ │ └── cast: INT8 [as=int8:10, type=int, outer=(9), immutable] + │ │ └── cast: INT8 [as=int8:12, type=int, outer=(10), immutable] │ │ └── plus [type=float] - │ │ ├── variable: z:9 [type=float] + │ │ ├── variable: z:10 [type=float] │ │ └── const: 1.0 [type=float] │ └── projections - │ ├── const: 10 [as=column11:11, type=int] - │ ├── function: unique_rowid [as=column12:12, type=int, volatile] - │ └── cast: INT8 [as=column13:13, type=int, immutable] + │ ├── const: 10 [as=column13:13, type=int] + │ ├── function: unique_rowid [as=column14:14, type=int, volatile] + │ └── cast: INT8 [as=column15:15, type=int, immutable] │ └── null [type=unknown] └── projections - └── plus [as=column14:14, type=int, outer=(10,11), immutable] + └── plus [as=column16:16, type=int, outer=(12,13), immutable] ├── plus [type=int] - │ ├── variable: int8:10 [type=int] - │ └── variable: column11:11 [type=int] + │ ├── variable: int8:12 [type=int] + │ └── variable: column13:13 [type=int] └── const: 1 [type=int] diff --git a/pkg/sql/opt/memo/testdata/logprops/join b/pkg/sql/opt/memo/testdata/logprops/join index 4c2940696dfa..c7091ae25e81 100644 --- a/pkg/sql/opt/memo/testdata/logprops/join +++ b/pkg/sql/opt/memo/testdata/logprops/join @@ -36,44 +36,50 @@ CREATE TABLE ref ( build SELECT *, rowid FROM xysd INNER JOIN uv ON x=u ---- -inner-join (hash) - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:5(int!null) v:6(int!null) rowid:7(int!null) - ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) - ├── key: (7) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (7)-->(5,6), (1)==(5), (5)==(1) - ├── prune: (2-4,6,7) - ├── interesting orderings: (+1) (-3,+4,+1) (+7) - ├── scan xysd - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) - │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ ├── prune: (1-4) - │ ├── interesting orderings: (+1) (-3,+4,+1) - │ └── unfiltered-cols: (1-4) - ├── scan uv - │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) - │ ├── key: (7) - │ ├── fd: (7)-->(5,6) - │ ├── prune: (5-7) - │ ├── interesting orderings: (+7) - │ └── unfiltered-cols: (5-7) - └── filters - └── eq [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - ├── variable: x:1 [type=int] - └── variable: u:5 [type=int] +project + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:6(int!null) v:7(int!null) rowid:8(int!null) + ├── key: (8) + ├── fd: (1)-->(2-4), (3,4)~~>(1,2,6), (8)-->(1-4,6,7), (1)==(6), (6)==(1) + ├── prune: (1-4,6-8) + ├── interesting orderings: (+1) (-3,+4,+1) (+8) + └── inner-join (hash) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) u:6(int!null) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) + ├── key: (8) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (8)-->(6,7,9), (1)==(6), (6)==(1) + ├── prune: (2-5,7-9) + ├── interesting orderings: (+1) (-3,+4,+1) (+8) + ├── scan xysd + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) + │ ├── key: (1) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── prune: (1-5) + │ ├── interesting orderings: (+1) (-3,+4,+1) + │ └── unfiltered-cols: (1-5) + ├── scan uv + │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + │ ├── key: (8) + │ ├── fd: (8)-->(6,7,9) + │ ├── prune: (6-9) + │ ├── interesting orderings: (+8) + │ └── unfiltered-cols: (6-9) + └── filters + └── eq [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + ├── variable: x:1 [type=int] + └── variable: u:6 [type=int] # Inner-join-apply. opt SELECT (SELECT (VALUES (x), (y))) FROM xysd ---- project - ├── columns: column1:7(int) - ├── prune: (7) + ├── columns: column1:8(int) + ├── prune: (8) ├── inner-join-apply - │ ├── columns: x:1(int!null) y:2(int) column1:5(int) column1:6(int) + │ ├── columns: x:1(int!null) y:2(int) column1:6(int) column1:7(int) │ ├── key: (1) - │ ├── fd: (1)-->(2,5,6) - │ ├── prune: (6) + │ ├── fd: (1)-->(2,6,7) + │ ├── prune: (7) │ ├── interesting orderings: (+1) │ ├── scan xysd │ │ ├── columns: x:1(int!null) y:2(int) @@ -82,41 +88,41 @@ project │ │ ├── prune: (1,2) │ │ └── interesting orderings: (+1) │ ├── inner-join-apply - │ │ ├── columns: column1:5(int) column1:6(int) + │ │ ├── columns: column1:6(int) column1:7(int) │ │ ├── outer: (1,2) │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(5,6) - │ │ ├── prune: (6) + │ │ ├── fd: ()-->(6,7) + │ │ ├── prune: (7) │ │ ├── max1-row - │ │ │ ├── columns: column1:5(int) + │ │ │ ├── columns: column1:6(int) │ │ │ ├── error: "more than one row returned by a subquery used as an expression" │ │ │ ├── outer: (1,2) │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(5) + │ │ │ ├── fd: ()-->(6) │ │ │ └── values - │ │ │ ├── columns: column1:5(int) + │ │ │ ├── columns: column1:6(int) │ │ │ ├── outer: (1,2) │ │ │ ├── cardinality: [2 - 2] - │ │ │ ├── prune: (5) + │ │ │ ├── prune: (6) │ │ │ ├── tuple [type=tuple{int}] │ │ │ │ └── variable: x:1 [type=int] │ │ │ └── tuple [type=tuple{int}] │ │ │ └── variable: y:2 [type=int] │ │ ├── values - │ │ │ ├── columns: column1:6(int) - │ │ │ ├── outer: (5) + │ │ │ ├── columns: column1:7(int) + │ │ │ ├── outer: (6) │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(6) - │ │ │ ├── prune: (6) + │ │ │ ├── fd: ()-->(7) + │ │ │ ├── prune: (7) │ │ │ └── tuple [type=tuple{int}] - │ │ │ └── variable: column1:5 [type=int] + │ │ │ └── variable: column1:6 [type=int] │ │ └── filters (true) │ └── filters (true) └── projections - └── variable: column1:6 [as=column1:7, type=int, outer=(6)] + └── variable: column1:7 [as=column1:8, type=int, outer=(7)] # Inner-join-apply nested in inner-join-apply with outer column references to # each parent. @@ -129,54 +135,54 @@ project ├── fd: (1)-->(2-4), (3,4)~~>(1,2) ├── prune: (1-4) └── select - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) v:6(int!null) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) v:7(int!null) ├── key: (1) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (1)==(6), (6)==(1) + ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (1)==(7), (7)==(1) ├── prune: (2-4) ├── ensure-distinct-on - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) v:6(int) + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) v:7(int) │ ├── grouping columns: x:1(int!null) │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── key: (1) - │ ├── fd: (1)-->(2-4,6), (3,4)~~>(1,2) - │ ├── prune: (2-4,6) + │ ├── fd: (1)-->(2-4,7), (3,4)~~>(1,2) + │ ├── prune: (2-4,7) │ ├── left-join (hash) - │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) v:6(int) n:9(int) + │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) v:7(int) n:11(int) │ │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) - │ │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (6)==(9), (9)==(6) + │ │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (7)==(11), (11)==(7) │ │ ├── prune: (2-4) - │ │ ├── reject-nulls: (6,9) - │ │ ├── interesting orderings: (+1) (-3,+4,+1) (+9) + │ │ ├── reject-nulls: (7,11) + │ │ ├── interesting orderings: (+1) (-3,+4,+1) (+11) │ │ ├── scan xysd │ │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) │ │ │ ├── key: (1) │ │ │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) │ │ │ ├── prune: (1-4) │ │ │ ├── interesting orderings: (+1) (-3,+4,+1) - │ │ │ └── unfiltered-cols: (1-4) + │ │ │ └── unfiltered-cols: (1-5) │ │ ├── inner-join (hash) - │ │ │ ├── columns: v:6(int!null) n:9(int!null) + │ │ │ ├── columns: v:7(int!null) n:11(int!null) │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ ├── fd: (6)==(9), (9)==(6) - │ │ │ ├── interesting orderings: (+9) + │ │ │ ├── fd: (7)==(11), (11)==(7) + │ │ │ ├── interesting orderings: (+11) │ │ │ ├── scan uv - │ │ │ │ ├── columns: v:6(int!null) - │ │ │ │ ├── prune: (6) - │ │ │ │ └── unfiltered-cols: (5-7) + │ │ │ │ ├── columns: v:7(int!null) + │ │ │ │ ├── prune: (7) + │ │ │ │ └── unfiltered-cols: (6-9) │ │ │ ├── scan mn - │ │ │ │ ├── columns: n:9(int) - │ │ │ │ ├── lax-key: (9) - │ │ │ │ ├── prune: (9) - │ │ │ │ ├── interesting orderings: (+9) - │ │ │ │ └── unfiltered-cols: (8,9) + │ │ │ │ ├── columns: n:11(int) + │ │ │ │ ├── lax-key: (11) + │ │ │ │ ├── prune: (11) + │ │ │ │ ├── interesting orderings: (+11) + │ │ │ │ └── unfiltered-cols: (10-12) │ │ │ └── filters - │ │ │ └── eq [type=bool, outer=(6,9), constraints=(/6: (/NULL - ]; /9: (/NULL - ]), fd=(6)==(9), (9)==(6)] - │ │ │ ├── variable: n:9 [type=int] - │ │ │ └── variable: v:6 [type=int] + │ │ │ └── eq [type=bool, outer=(7,11), constraints=(/7: (/NULL - ]; /11: (/NULL - ]), fd=(7)==(11), (11)==(7)] + │ │ │ ├── variable: n:11 [type=int] + │ │ │ └── variable: v:7 [type=int] │ │ └── filters - │ │ └── eq [type=bool, outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ]), fd=(1)==(9), (9)==(1)] + │ │ └── eq [type=bool, outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ]), fd=(1)==(11), (11)==(1)] │ │ ├── variable: x:1 [type=int] - │ │ └── variable: n:9 [type=int] + │ │ └── variable: n:11 [type=int] │ └── aggregations │ ├── const-agg [as=y:2, type=int, outer=(2)] │ │ └── variable: y:2 [type=int] @@ -184,12 +190,12 @@ project │ │ └── variable: s:3 [type=string] │ ├── const-agg [as=d:4, type=decimal, outer=(4)] │ │ └── variable: d:4 [type=decimal] - │ └── const-agg [as=v:6, type=int, outer=(6)] - │ └── variable: v:6 [type=int] + │ └── const-agg [as=v:7, type=int, outer=(7)] + │ └── variable: v:7 [type=int] └── filters - └── eq [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── eq [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] ├── variable: x:1 [type=int] - └── variable: v:6 [type=int] + └── variable: v:7 [type=int] # Inner-join nested in inner-join-apply with outer column reference to top-level # inner-join-apply. @@ -202,37 +208,37 @@ project ├── fd: (1)-->(2-4), (3,4)~~>(1,2) ├── prune: (1-4) └── select - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) v:6(int!null) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) v:7(int!null) ├── key: (1) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (1)==(6), (6)==(1) + ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (1)==(7), (7)==(1) ├── prune: (2-4) ├── ensure-distinct-on - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) v:6(int) + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) v:7(int) │ ├── grouping columns: x:1(int!null) │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── key: (1) - │ ├── fd: (1)-->(2-4,6), (3,4)~~>(1,2) - │ ├── prune: (2-4,6) + │ ├── fd: (1)-->(2-4,7), (3,4)~~>(1,2) + │ ├── prune: (2-4,7) │ ├── right-join (hash) - │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) v:6(int) m:8(int) + │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) v:7(int) m:10(int) │ │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ │ ├── prune: (3,4,6) - │ │ ├── reject-nulls: (6,8) - │ │ ├── interesting orderings: (+1) (-3,+4,+1) (+8) + │ │ ├── prune: (3,4,7) + │ │ ├── reject-nulls: (7,10) + │ │ ├── interesting orderings: (+1) (-3,+4,+1) (+10) │ │ ├── inner-join (cross) - │ │ │ ├── columns: v:6(int!null) m:8(int!null) - │ │ │ ├── prune: (6,8) - │ │ │ ├── interesting orderings: (+8) + │ │ │ ├── columns: v:7(int!null) m:10(int!null) + │ │ │ ├── prune: (7,10) + │ │ │ ├── interesting orderings: (+10) │ │ │ ├── scan uv - │ │ │ │ ├── columns: v:6(int!null) - │ │ │ │ ├── prune: (6) - │ │ │ │ └── unfiltered-cols: (5-7) + │ │ │ │ ├── columns: v:7(int!null) + │ │ │ │ ├── prune: (7) + │ │ │ │ └── unfiltered-cols: (6-9) │ │ │ ├── scan mn - │ │ │ │ ├── columns: m:8(int!null) - │ │ │ │ ├── key: (8) - │ │ │ │ ├── prune: (8) - │ │ │ │ ├── interesting orderings: (+8) - │ │ │ │ └── unfiltered-cols: (8,9) + │ │ │ │ ├── columns: m:10(int!null) + │ │ │ │ ├── key: (10) + │ │ │ │ ├── prune: (10) + │ │ │ │ ├── interesting orderings: (+10) + │ │ │ │ └── unfiltered-cols: (10-12) │ │ │ └── filters (true) │ │ ├── scan xysd │ │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) @@ -240,14 +246,14 @@ project │ │ │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) │ │ │ ├── prune: (1-4) │ │ │ ├── interesting orderings: (+1) (-3,+4,+1) - │ │ │ └── unfiltered-cols: (1-4) + │ │ │ └── unfiltered-cols: (1-5) │ │ └── filters │ │ ├── eq [type=bool, outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ]), fd=(1)==(2), (2)==(1)] │ │ │ ├── variable: x:1 [type=int] │ │ │ └── variable: y:2 [type=int] - │ │ └── eq [type=bool, outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] + │ │ └── eq [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] │ │ ├── variable: x:1 [type=int] - │ │ └── variable: m:8 [type=int] + │ │ └── variable: m:10 [type=int] │ └── aggregations │ ├── const-agg [as=y:2, type=int, outer=(2)] │ │ └── variable: y:2 [type=int] @@ -255,43 +261,50 @@ project │ │ └── variable: s:3 [type=string] │ ├── const-agg [as=d:4, type=decimal, outer=(4)] │ │ └── variable: d:4 [type=decimal] - │ └── const-agg [as=v:6, type=int, outer=(6)] - │ └── variable: v:6 [type=int] + │ └── const-agg [as=v:7, type=int, outer=(7)] + │ └── variable: v:7 [type=int] └── filters - └── eq [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── eq [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] ├── variable: x:1 [type=int] - └── variable: v:6 [type=int] + └── variable: v:7 [type=int] # Left-join. build SELECT *, rowid FROM xysd LEFT JOIN uv ON x=u ---- -left-join (hash) - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:5(int) v:6(int) rowid:7(int) - ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) - ├── key: (1,7) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (7)-->(5,6) - ├── prune: (2-4,6,7) - ├── reject-nulls: (5-7) - ├── interesting orderings: (+1) (-3,+4,+1) (+7) - ├── scan xysd - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) - │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ ├── prune: (1-4) - │ ├── interesting orderings: (+1) (-3,+4,+1) - │ └── unfiltered-cols: (1-4) - ├── scan uv - │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) - │ ├── key: (7) - │ ├── fd: (7)-->(5,6) - │ ├── prune: (5-7) - │ ├── interesting orderings: (+7) - │ └── unfiltered-cols: (5-7) - └── filters - └── eq [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - ├── variable: x:1 [type=int] - └── variable: u:5 [type=int] +project + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:6(int) v:7(int) rowid:8(int) + ├── key: (1,8) + ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (8)-->(6,7) + ├── prune: (1-4,6-8) + ├── reject-nulls: (6-8) + ├── interesting orderings: (+1) (-3,+4,+1) (+8) + └── left-join (hash) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) u:6(int) v:7(int) rowid:8(int) uv.crdb_internal_mvcc_timestamp:9(decimal) + ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) + ├── key: (1,8) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (8)-->(6,7,9) + ├── prune: (2-5,7-9) + ├── reject-nulls: (6-9) + ├── interesting orderings: (+1) (-3,+4,+1) (+8) + ├── scan xysd + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) + │ ├── key: (1) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── prune: (1-5) + │ ├── interesting orderings: (+1) (-3,+4,+1) + │ └── unfiltered-cols: (1-5) + ├── scan uv + │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + │ ├── key: (8) + │ ├── fd: (8)-->(6,7,9) + │ ├── prune: (6-9) + │ ├── interesting orderings: (+8) + │ └── unfiltered-cols: (6-9) + └── filters + └── eq [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + ├── variable: x:1 [type=int] + └── variable: u:6 [type=int] # Left-join-apply. opt @@ -303,23 +316,23 @@ project ├── fd: (1)-->(2-4), (3,4)~~>(1,2) ├── prune: (1-4) └── select - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:5(int) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:6(int) ├── key: (1) - ├── fd: ()-->(5), (1)-->(2-4), (3,4)~~>(1,2) + ├── fd: ()-->(6), (1)-->(2-4), (3,4)~~>(1,2) ├── prune: (2-4) ├── ensure-distinct-on - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:5(int) + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:6(int) │ ├── grouping columns: x:1(int!null) │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── key: (1) - │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2) - │ ├── prune: (2-5) + │ ├── fd: (1)-->(2-4,6), (3,4)~~>(1,2) + │ ├── prune: (2-4,6) │ ├── left-join (hash) - │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:5(int) + │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:6(int) │ │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) │ │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) │ │ ├── prune: (2-4) - │ │ ├── reject-nulls: (5) + │ │ ├── reject-nulls: (6) │ │ ├── interesting orderings: (+1) (-3,+4,+1) │ │ ├── scan xysd │ │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) @@ -327,14 +340,14 @@ project │ │ │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) │ │ │ ├── prune: (1-4) │ │ │ ├── interesting orderings: (+1) (-3,+4,+1) - │ │ │ └── unfiltered-cols: (1-4) + │ │ │ └── unfiltered-cols: (1-5) │ │ ├── scan uv - │ │ │ ├── columns: u:5(int) - │ │ │ ├── prune: (5) - │ │ │ └── unfiltered-cols: (5-7) + │ │ │ ├── columns: u:6(int) + │ │ │ ├── prune: (6) + │ │ │ └── unfiltered-cols: (6-9) │ │ └── filters - │ │ └── eq [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - │ │ ├── variable: u:5 [type=int] + │ │ └── eq [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ │ ├── variable: u:6 [type=int] │ │ └── variable: x:1 [type=int] │ └── aggregations │ ├── const-agg [as=y:2, type=int, outer=(2)] @@ -343,71 +356,85 @@ project │ │ └── variable: s:3 [type=string] │ ├── const-agg [as=d:4, type=decimal, outer=(4)] │ │ └── variable: d:4 [type=decimal] - │ └── const-agg [as=u:5, type=int, outer=(5)] - │ └── variable: u:5 [type=int] + │ └── const-agg [as=u:6, type=int, outer=(6)] + │ └── variable: u:6 [type=int] └── filters - └── is [type=bool, outer=(5), constraints=(/5: [/NULL - /NULL]; tight), fd=()-->(5)] - ├── variable: u:5 [type=int] + └── is [type=bool, outer=(6), constraints=(/6: [/NULL - /NULL]; tight), fd=()-->(6)] + ├── variable: u:6 [type=int] └── null [type=unknown] # Right-join. build SELECT *, rowid FROM xysd RIGHT JOIN uv ON x=u ---- -right-join (hash) - ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) u:5(int) v:6(int!null) rowid:7(int!null) - ├── key: (7) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (7)-->(1-6) - ├── prune: (2-4,6,7) +project + ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) u:6(int) v:7(int!null) rowid:8(int!null) + ├── key: (8) + ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (8)-->(1-4,6,7) + ├── prune: (1-4,6-8) ├── reject-nulls: (1-4) - ├── interesting orderings: (+1) (-3,+4,+1) (+7) - ├── scan xysd - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) - │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ ├── prune: (1-4) - │ └── interesting orderings: (+1) (-3,+4,+1) - ├── scan uv - │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) - │ ├── key: (7) - │ ├── fd: (7)-->(5,6) - │ ├── prune: (5-7) - │ └── interesting orderings: (+7) - └── filters - └── eq [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - ├── variable: x:1 [type=int] - └── variable: u:5 [type=int] + ├── interesting orderings: (+1) (-3,+4,+1) (+8) + └── right-join (hash) + ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) xysd.crdb_internal_mvcc_timestamp:5(decimal) u:6(int) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + ├── key: (8) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (8)-->(1-7,9) + ├── prune: (2-5,7-9) + ├── reject-nulls: (1-5) + ├── interesting orderings: (+1) (-3,+4,+1) (+8) + ├── scan xysd + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) + │ ├── key: (1) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── prune: (1-5) + │ └── interesting orderings: (+1) (-3,+4,+1) + ├── scan uv + │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + │ ├── key: (8) + │ ├── fd: (8)-->(6,7,9) + │ ├── prune: (6-9) + │ └── interesting orderings: (+8) + └── filters + └── eq [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + ├── variable: x:1 [type=int] + └── variable: u:6 [type=int] # Full-join. build SELECT *, rowid FROM xysd FULL JOIN uv ON x=u ---- -full-join (hash) - ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) u:5(int) v:6(int) rowid:7(int) - ├── multiplicity: left-rows(one-or-more), right-rows(exactly-one) - ├── key: (1,7) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (7)-->(5,6) - ├── prune: (2-4,6,7) - ├── reject-nulls: (1-7) - ├── interesting orderings: (+1) (-3,+4,+1) (+7) - ├── scan xysd - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) - │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ ├── prune: (1-4) - │ ├── interesting orderings: (+1) (-3,+4,+1) - │ └── unfiltered-cols: (1-4) - ├── scan uv - │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) - │ ├── key: (7) - │ ├── fd: (7)-->(5,6) - │ ├── prune: (5-7) - │ ├── interesting orderings: (+7) - │ └── unfiltered-cols: (5-7) - └── filters - └── eq [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - ├── variable: x:1 [type=int] - └── variable: u:5 [type=int] +project + ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) u:6(int) v:7(int) rowid:8(int) + ├── key: (1,8) + ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (8)-->(6,7) + ├── prune: (1-4,6-8) + ├── reject-nulls: (1-4,6-8) + ├── interesting orderings: (+1) (-3,+4,+1) (+8) + └── full-join (hash) + ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) xysd.crdb_internal_mvcc_timestamp:5(decimal) u:6(int) v:7(int) rowid:8(int) uv.crdb_internal_mvcc_timestamp:9(decimal) + ├── multiplicity: left-rows(one-or-more), right-rows(exactly-one) + ├── key: (1,8) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (8)-->(6,7,9) + ├── prune: (2-5,7-9) + ├── reject-nulls: (1-9) + ├── interesting orderings: (+1) (-3,+4,+1) (+8) + ├── scan xysd + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) + │ ├── key: (1) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── prune: (1-5) + │ ├── interesting orderings: (+1) (-3,+4,+1) + │ └── unfiltered-cols: (1-5) + ├── scan uv + │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + │ ├── key: (8) + │ ├── fd: (8)-->(6,7,9) + │ ├── prune: (6-9) + │ ├── interesting orderings: (+8) + │ └── unfiltered-cols: (6-9) + └── filters + └── eq [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + ├── variable: x:1 [type=int] + └── variable: u:6 [type=int] # Semi-join. opt @@ -420,21 +447,21 @@ project ├── prune: (2-4) ├── interesting orderings: (+1) (-3,+4,+1) └── inner-join (lookup xysd) - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:5(int!null) - ├── key columns: [5] = [1] + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:6(int!null) + ├── key columns: [6] = [1] ├── lookup columns are key - ├── key: (5) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (1)==(5), (5)==(1) + ├── key: (6) + ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (1)==(6), (6)==(1) ├── prune: (2-4) ├── interesting orderings: (+1) (-3,+4,+1) ├── distinct-on - │ ├── columns: u:5(int) - │ ├── grouping columns: u:5(int) - │ ├── key: (5) + │ ├── columns: u:6(int) + │ ├── grouping columns: u:6(int) + │ ├── key: (6) │ └── scan uv - │ ├── columns: u:5(int) - │ ├── prune: (5) - │ └── unfiltered-cols: (5-7) + │ ├── columns: u:6(int) + │ ├── prune: (6) + │ └── unfiltered-cols: (6-9) └── filters (true) # Semi-join-apply. @@ -454,19 +481,19 @@ semi-join-apply │ ├── prune: (1-4) │ └── interesting orderings: (+1) (-3,+4,+1) ├── offset - │ ├── columns: v:6(int!null) + │ ├── columns: v:7(int!null) │ ├── outer: (1) - │ ├── fd: ()-->(6) + │ ├── fd: ()-->(7) │ ├── select - │ │ ├── columns: v:6(int!null) + │ │ ├── columns: v:7(int!null) │ │ ├── outer: (1) - │ │ ├── fd: ()-->(6) + │ │ ├── fd: ()-->(7) │ │ ├── scan uv - │ │ │ ├── columns: v:6(int!null) - │ │ │ └── prune: (6) + │ │ │ ├── columns: v:7(int!null) + │ │ │ └── prune: (7) │ │ └── filters - │ │ └── eq [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ │ ├── variable: v:6 [type=int] + │ │ └── eq [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + │ │ ├── variable: v:7 [type=int] │ │ └── variable: x:1 [type=int] │ └── const: 1 [type=int] └── filters (true) @@ -488,26 +515,26 @@ semi-join-apply │ ├── prune: (1-4) │ └── interesting orderings: (+1) (-3,+4,+1) ├── semi-join (cross) - │ ├── columns: v:6(int!null) + │ ├── columns: v:7(int!null) │ ├── outer: (1) - │ ├── fd: ()-->(6) + │ ├── fd: ()-->(7) │ ├── scan uv - │ │ ├── columns: v:6(int!null) - │ │ ├── prune: (6) - │ │ └── unfiltered-cols: (5-7) + │ │ ├── columns: v:7(int!null) + │ │ ├── prune: (7) + │ │ └── unfiltered-cols: (6-9) │ ├── scan mn - │ │ ├── columns: m:8(int!null) - │ │ ├── key: (8) - │ │ ├── prune: (8) - │ │ ├── interesting orderings: (+8) - │ │ └── unfiltered-cols: (8,9) + │ │ ├── columns: m:10(int!null) + │ │ ├── key: (10) + │ │ ├── prune: (10) + │ │ ├── interesting orderings: (+10) + │ │ └── unfiltered-cols: (10-12) │ └── filters - │ ├── eq [type=bool, outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] + │ ├── eq [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] │ │ ├── variable: x:1 [type=int] - │ │ └── variable: m:8 [type=int] - │ └── eq [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ │ └── variable: m:10 [type=int] + │ └── eq [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] │ ├── variable: x:1 [type=int] - │ └── variable: v:6 [type=int] + │ └── variable: v:7 [type=int] └── filters (true) # Anti-join. @@ -526,15 +553,15 @@ anti-join (hash) │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) │ ├── prune: (1-4) │ ├── interesting orderings: (+1) (-3,+4,+1) - │ └── unfiltered-cols: (1-4) + │ └── unfiltered-cols: (1-5) ├── scan uv - │ ├── columns: u:5(int) - │ ├── prune: (5) - │ └── unfiltered-cols: (5-7) + │ ├── columns: u:6(int) + │ ├── prune: (6) + │ └── unfiltered-cols: (6-9) └── filters - └── eq [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + └── eq [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] ├── variable: x:1 [type=int] - └── variable: u:5 [type=int] + └── variable: u:6 [type=int] # Anti-join-apply. opt @@ -553,19 +580,19 @@ anti-join-apply │ ├── prune: (1-4) │ └── interesting orderings: (+1) (-3,+4,+1) ├── offset - │ ├── columns: v:6(int!null) + │ ├── columns: v:7(int!null) │ ├── outer: (1) - │ ├── fd: ()-->(6) + │ ├── fd: ()-->(7) │ ├── select - │ │ ├── columns: v:6(int!null) + │ │ ├── columns: v:7(int!null) │ │ ├── outer: (1) - │ │ ├── fd: ()-->(6) + │ │ ├── fd: ()-->(7) │ │ ├── scan uv - │ │ │ ├── columns: v:6(int!null) - │ │ │ └── prune: (6) + │ │ │ ├── columns: v:7(int!null) + │ │ │ └── prune: (7) │ │ └── filters - │ │ └── eq [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ │ ├── variable: v:6 [type=int] + │ │ └── eq [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + │ │ ├── variable: v:7 [type=int] │ │ └── variable: x:1 [type=int] │ └── const: 1 [type=int] └── filters (true) @@ -575,150 +602,163 @@ build SELECT * FROM xysd, uv ---- project - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:5(int) v:6(int!null) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:6(int) v:7(int!null) ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - ├── prune: (1-6) + ├── prune: (1-4,6,7) ├── interesting orderings: (+1) (-3,+4,+1) └── inner-join (cross) - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:5(int) v:6(int!null) rowid:7(int!null) - ├── key: (1,7) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (7)-->(5,6) - ├── prune: (1-7) - ├── interesting orderings: (+1) (-3,+4,+1) (+7) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) u:6(int) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + ├── key: (1,8) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (8)-->(6,7,9) + ├── prune: (1-9) + ├── interesting orderings: (+1) (-3,+4,+1) (+8) ├── scan xysd - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ ├── prune: (1-4) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── prune: (1-5) │ ├── interesting orderings: (+1) (-3,+4,+1) - │ └── unfiltered-cols: (1-4) + │ └── unfiltered-cols: (1-5) ├── scan uv - │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) - │ ├── key: (7) - │ ├── fd: (7)-->(5,6) - │ ├── prune: (5-7) - │ ├── interesting orderings: (+7) - │ └── unfiltered-cols: (5-7) + │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + │ ├── key: (8) + │ ├── fd: (8)-->(6,7,9) + │ ├── prune: (6-9) + │ ├── interesting orderings: (+8) + │ └── unfiltered-cols: (6-9) └── filters (true) # Self-join. build SELECT * FROM xysd, xysd AS xysd ---- -inner-join (cross) - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) x:5(int!null) y:6(int) s:7(string) d:8(decimal!null) - ├── key: (1,5) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (5)-->(6-8), (7,8)~~>(5,6) - ├── prune: (1-8) - ├── interesting orderings: (+1) (-3,+4,+1) (+5) (-7,+8,+5) - ├── scan xysd - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) - │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ ├── prune: (1-4) - │ ├── interesting orderings: (+1) (-3,+4,+1) - │ └── unfiltered-cols: (1-4) - ├── scan xysd - │ ├── columns: x:5(int!null) y:6(int) s:7(string) d:8(decimal!null) - │ ├── key: (5) - │ ├── fd: (5)-->(6-8), (7,8)~~>(5,6) - │ ├── prune: (5-8) - │ ├── interesting orderings: (+5) (-7,+8,+5) - │ └── unfiltered-cols: (5-8) - └── filters (true) +project + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) x:6(int!null) y:7(int) s:8(string) d:9(decimal!null) + ├── key: (1,6) + ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (6)-->(7-9), (8,9)~~>(6,7) + ├── prune: (1-4,6-9) + ├── interesting orderings: (+1) (-3,+4,+1) (+6) (-8,+9,+6) + └── inner-join (cross) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) x:6(int!null) y:7(int) s:8(string) d:9(decimal!null) crdb_internal_mvcc_timestamp:10(decimal) + ├── key: (1,6) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (6)-->(7-10), (8,9)~~>(6,7,10) + ├── prune: (1-10) + ├── interesting orderings: (+1) (-3,+4,+1) (+6) (-8,+9,+6) + ├── scan xysd + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) + │ ├── key: (1) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── prune: (1-5) + │ ├── interesting orderings: (+1) (-3,+4,+1) + │ └── unfiltered-cols: (1-5) + ├── scan xysd + │ ├── columns: x:6(int!null) y:7(int) s:8(string) d:9(decimal!null) crdb_internal_mvcc_timestamp:10(decimal) + │ ├── key: (6) + │ ├── fd: (6)-->(7-10), (8,9)~~>(6,7,10) + │ ├── prune: (6-10) + │ ├── interesting orderings: (+6) (-8,+9,+6) + │ └── unfiltered-cols: (6-10) + └── filters (true) # Propagate outer columns. build SELECT * FROM xysd WHERE EXISTS(SELECT * FROM (SELECT x) INNER JOIN (SELECT y) ON x::string = s) ---- -select +project ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) ├── immutable ├── key: (1) ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - ├── prune: (4) + ├── prune: (1-4) ├── interesting orderings: (+1) (-3,+4,+1) - ├── scan xysd - │ ├── columns: xysd.x:1(int!null) xysd.y:2(int) s:3(string) d:4(decimal!null) - │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ ├── prune: (1-4) - │ └── interesting orderings: (+1) (-3,+4,+1) - └── filters - └── exists [type=bool, outer=(1-3), immutable, correlated-subquery] - └── inner-join (cross) - ├── columns: x:5(int) y:6(int) - ├── outer: (1-3) - ├── cardinality: [0 - 1] - ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── immutable - ├── key: () - ├── fd: ()-->(5,6) - ├── prune: (6) - ├── project - │ ├── columns: x:5(int) - │ ├── outer: (1) - │ ├── cardinality: [1 - 1] - │ ├── key: () - │ ├── fd: ()-->(5) - │ ├── prune: (5) - │ ├── values - │ │ ├── cardinality: [1 - 1] - │ │ ├── key: () - │ │ └── tuple [type=tuple] - │ └── projections - │ └── variable: xysd.x:1 [as=x:5, type=int, outer=(1)] - ├── project - │ ├── columns: y:6(int) - │ ├── outer: (2) - │ ├── cardinality: [1 - 1] - │ ├── key: () - │ ├── fd: ()-->(6) - │ ├── prune: (6) - │ ├── values - │ │ ├── cardinality: [1 - 1] - │ │ ├── key: () - │ │ └── tuple [type=tuple] - │ └── projections - │ └── variable: xysd.y:2 [as=y:6, type=int, outer=(2)] - └── filters - └── eq [type=bool, outer=(3,5), immutable] - ├── cast: STRING [type=string] - │ └── variable: x:5 [type=int] - └── variable: s:3 [type=string] + └── select + ├── columns: xysd.x:1(int!null) xysd.y:2(int) s:3(string) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) + ├── immutable + ├── key: (1) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + ├── prune: (4,5) + ├── interesting orderings: (+1) (-3,+4,+1) + ├── scan xysd + │ ├── columns: xysd.x:1(int!null) xysd.y:2(int) s:3(string) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) + │ ├── key: (1) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── prune: (1-5) + │ └── interesting orderings: (+1) (-3,+4,+1) + └── filters + └── exists [type=bool, outer=(1-3), immutable, correlated-subquery] + └── inner-join (cross) + ├── columns: x:6(int) y:7(int) + ├── outer: (1-3) + ├── cardinality: [0 - 1] + ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) + ├── immutable + ├── key: () + ├── fd: ()-->(6,7) + ├── prune: (7) + ├── project + │ ├── columns: x:6(int) + │ ├── outer: (1) + │ ├── cardinality: [1 - 1] + │ ├── key: () + │ ├── fd: ()-->(6) + │ ├── prune: (6) + │ ├── values + │ │ ├── cardinality: [1 - 1] + │ │ ├── key: () + │ │ └── tuple [type=tuple] + │ └── projections + │ └── variable: xysd.x:1 [as=x:6, type=int, outer=(1)] + ├── project + │ ├── columns: y:7(int) + │ ├── outer: (2) + │ ├── cardinality: [1 - 1] + │ ├── key: () + │ ├── fd: ()-->(7) + │ ├── prune: (7) + │ ├── values + │ │ ├── cardinality: [1 - 1] + │ │ ├── key: () + │ │ └── tuple [type=tuple] + │ └── projections + │ └── variable: xysd.y:2 [as=y:7, type=int, outer=(2)] + └── filters + └── eq [type=bool, outer=(3,6), immutable] + ├── cast: STRING [type=string] + │ └── variable: x:6 [type=int] + └── variable: s:3 [type=string] # Calculate semi-join cardinality when left side has non-zero cardinality. opt SELECT * FROM (SELECT count(*) cnt FROM xysd) WHERE EXISTS(SELECT * FROM uv WHERE cnt=1) ---- project - ├── columns: cnt:5(int!null) + ├── columns: cnt:6(int!null) ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(5) + ├── fd: ()-->(6) └── inner-join (cross) - ├── columns: count_rows:5(int!null) + ├── columns: count_rows:6(int!null) ├── cardinality: [0 - 1] ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) ├── key: () - ├── fd: ()-->(5) + ├── fd: ()-->(6) ├── select - │ ├── columns: count_rows:5(int!null) + │ ├── columns: count_rows:6(int!null) │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(5) + │ ├── fd: ()-->(6) │ ├── scalar-group-by - │ │ ├── columns: count_rows:5(int!null) + │ │ ├── columns: count_rows:6(int!null) │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(5) - │ │ ├── prune: (5) + │ │ ├── fd: ()-->(6) + │ │ ├── prune: (6) │ │ ├── scan xysd@secondary │ │ └── aggregations - │ │ └── count-rows [as=count_rows:5, type=int] + │ │ └── count-rows [as=count_rows:6, type=int] │ └── filters - │ └── eq [type=bool, outer=(5), constraints=(/5: [/1 - /1]; tight), fd=()-->(5)] - │ ├── variable: count_rows:5 [type=int] + │ └── eq [type=bool, outer=(6), constraints=(/6: [/1 - /1]; tight), fd=()-->(6)] + │ ├── variable: count_rows:6 [type=int] │ └── const: 1 [type=int] ├── scan uv │ ├── limit: 1 @@ -772,7 +812,7 @@ anti-join (hash) ├── scan uv │ ├── columns: u:2(int) │ ├── prune: (2) - │ └── unfiltered-cols: (2-4) + │ └── unfiltered-cols: (2-5) └── filters └── eq [type=bool, outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ]), fd=(1)==(2), (2)==(1)] ├── variable: u:2 [type=int] @@ -832,11 +872,11 @@ inner-join (cross) │ │ ├── limit hint: 2.00 │ │ ├── prune: (2,3) │ │ └── scan uv - │ │ ├── columns: u:2(int) v:3(int!null) rowid:4(int!null) + │ │ ├── columns: u:2(int) v:3(int!null) rowid:4(int!null) crdb_internal_mvcc_timestamp:5(decimal) │ │ ├── key: (4) - │ │ ├── fd: (4)-->(2,3) + │ │ ├── fd: (4)-->(2,3,5) │ │ ├── limit hint: 2.00 - │ │ ├── prune: (2-4) + │ │ ├── prune: (2-5) │ │ └── interesting orderings: (+4) │ └── const: 2 [type=int] └── filters @@ -870,11 +910,11 @@ left-join (cross) │ │ ├── limit hint: 2.00 │ │ ├── prune: (2,3) │ │ └── scan uv - │ │ ├── columns: u:2(int) v:3(int!null) rowid:4(int!null) + │ │ ├── columns: u:2(int) v:3(int!null) rowid:4(int!null) crdb_internal_mvcc_timestamp:5(decimal) │ │ ├── key: (4) - │ │ ├── fd: (4)-->(2,3) + │ │ ├── fd: (4)-->(2,3,5) │ │ ├── limit hint: 2.00 - │ │ ├── prune: (2-4) + │ │ ├── prune: (2-5) │ │ └── interesting orderings: (+4) │ └── const: 2 [type=int] └── filters @@ -885,9 +925,9 @@ build SELECT * FROM (SELECT * FROM uv LIMIT 2) RIGHT JOIN (VALUES (1), (2), (3)) ON True ---- right-join (cross) - ├── columns: u:1(int) v:2(int) column1:4(int!null) + ├── columns: u:1(int) v:2(int) column1:5(int!null) ├── cardinality: [3 - 6] - ├── prune: (1,2,4) + ├── prune: (1,2,5) ├── reject-nulls: (1,2) ├── limit │ ├── columns: u:1(int) v:2(int!null) @@ -898,17 +938,17 @@ right-join (cross) │ │ ├── limit hint: 2.00 │ │ ├── prune: (1,2) │ │ └── scan uv - │ │ ├── columns: u:1(int) v:2(int!null) rowid:3(int!null) + │ │ ├── columns: u:1(int) v:2(int!null) rowid:3(int!null) crdb_internal_mvcc_timestamp:4(decimal) │ │ ├── key: (3) - │ │ ├── fd: (3)-->(1,2) + │ │ ├── fd: (3)-->(1,2,4) │ │ ├── limit hint: 2.00 - │ │ ├── prune: (1-3) + │ │ ├── prune: (1-4) │ │ └── interesting orderings: (+3) │ └── const: 2 [type=int] ├── values - │ ├── columns: column1:4(int!null) + │ ├── columns: column1:5(int!null) │ ├── cardinality: [3 - 3] - │ ├── prune: (4) + │ ├── prune: (5) │ ├── tuple [type=tuple{int}] │ │ └── const: 1 [type=int] │ ├── tuple [type=tuple{int}] @@ -1013,42 +1053,49 @@ full-join (cross) build SELECT * FROM xysd FULL JOIN (SELECT * FROM (VALUES (1), (2))) ON True ---- -full-join (cross) - ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) column1:5(int) +project + ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) column1:6(int) ├── cardinality: [2 - ] ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - ├── prune: (1-5) - ├── reject-nulls: (1-5) + ├── prune: (1-4,6) + ├── reject-nulls: (1-4,6) ├── interesting orderings: (+1) (-3,+4,+1) - ├── scan xysd - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) - │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ ├── prune: (1-4) - │ ├── interesting orderings: (+1) (-3,+4,+1) - │ └── unfiltered-cols: (1-4) - ├── values - │ ├── columns: column1:5(int!null) - │ ├── cardinality: [2 - 2] - │ ├── prune: (5) - │ ├── tuple [type=tuple{int}] - │ │ └── const: 1 [type=int] - │ └── tuple [type=tuple{int}] - │ └── const: 2 [type=int] - └── filters - └── true [type=bool] + └── full-join (cross) + ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) crdb_internal_mvcc_timestamp:5(decimal) column1:6(int) + ├── cardinality: [2 - ] + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + ├── prune: (1-6) + ├── reject-nulls: (1-6) + ├── interesting orderings: (+1) (-3,+4,+1) + ├── scan xysd + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) + │ ├── key: (1) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── prune: (1-5) + │ ├── interesting orderings: (+1) (-3,+4,+1) + │ └── unfiltered-cols: (1-5) + ├── values + │ ├── columns: column1:6(int!null) + │ ├── cardinality: [2 - 2] + │ ├── prune: (6) + │ ├── tuple [type=tuple{int}] + │ │ └── const: 1 [type=int] + │ └── tuple [type=tuple{int}] + │ └── const: 2 [type=int] + └── filters + └── true [type=bool] # Keys on both sides of full-join. build SELECT * FROM (SELECT * FROM xysd LIMIT 1) FULL JOIN (SELECT * FROM xysd LIMIT 1) ON True ---- full-join (cross) - ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) x:5(int) y:6(int) s:7(string) d:8(decimal) + ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) x:6(int) y:7(int) s:8(string) d:9(decimal) ├── cardinality: [0 - 2] ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) - ├── prune: (1-8) - ├── reject-nulls: (1-8) - ├── interesting orderings: (+1) (-3,+4,+1) (+5) (-7,+8,+5) + ├── prune: (1-4,6-9) + ├── reject-nulls: (1-4,6-9) + ├── interesting orderings: (+1) (-3,+4,+1) (+6) (-8,+9,+6) ├── limit │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) │ ├── cardinality: [0 - 1] @@ -1056,28 +1103,42 @@ full-join (cross) │ ├── fd: ()-->(1-4) │ ├── prune: (1-4) │ ├── interesting orderings: (+1) (-3,+4,+1) - │ ├── scan xysd + │ ├── project │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) │ │ ├── key: (1) │ │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) │ │ ├── limit hint: 1.00 │ │ ├── prune: (1-4) - │ │ └── interesting orderings: (+1) (-3,+4,+1) + │ │ ├── interesting orderings: (+1) (-3,+4,+1) + │ │ └── scan xysd + │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) + │ │ ├── key: (1) + │ │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ │ ├── limit hint: 1.00 + │ │ ├── prune: (1-5) + │ │ └── interesting orderings: (+1) (-3,+4,+1) │ └── const: 1 [type=int] ├── limit - │ ├── columns: x:5(int!null) y:6(int) s:7(string) d:8(decimal!null) + │ ├── columns: x:6(int!null) y:7(int) s:8(string) d:9(decimal!null) │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(5-8) - │ ├── prune: (5-8) - │ ├── interesting orderings: (+5) (-7,+8,+5) - │ ├── scan xysd - │ │ ├── columns: x:5(int!null) y:6(int) s:7(string) d:8(decimal!null) - │ │ ├── key: (5) - │ │ ├── fd: (5)-->(6-8), (7,8)~~>(5,6) + │ ├── fd: ()-->(6-9) + │ ├── prune: (6-9) + │ ├── interesting orderings: (+6) (-8,+9,+6) + │ ├── project + │ │ ├── columns: x:6(int!null) y:7(int) s:8(string) d:9(decimal!null) + │ │ ├── key: (6) + │ │ ├── fd: (6)-->(7-9), (8,9)~~>(6,7) │ │ ├── limit hint: 1.00 - │ │ ├── prune: (5-8) - │ │ └── interesting orderings: (+5) (-7,+8,+5) + │ │ ├── prune: (6-9) + │ │ ├── interesting orderings: (+6) (-8,+9,+6) + │ │ └── scan xysd + │ │ ├── columns: x:6(int!null) y:7(int) s:8(string) d:9(decimal!null) crdb_internal_mvcc_timestamp:10(decimal) + │ │ ├── key: (6) + │ │ ├── fd: (6)-->(7-10), (8,9)~~>(6,7,10) + │ │ ├── limit hint: 1.00 + │ │ ├── prune: (6-10) + │ │ └── interesting orderings: (+6) (-8,+9,+6) │ └── const: 1 [type=int] └── filters └── true [type=bool] @@ -1086,261 +1147,303 @@ full-join (cross) build SELECT * FROM xysd LEFT JOIN (SELECT u, sum(v) FROM uv GROUP BY u) ON u IS NOT NULL ---- -left-join (cross) - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:5(int) sum:8(decimal) - ├── key: (1,5) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (5)~~>(8), (1,5)-->(8) - ├── prune: (1-4,8) - ├── reject-nulls: (5,8) +project + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:6(int) sum:10(decimal) + ├── key: (1,6) + ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (6)~~>(10), (1,6)-->(10) + ├── prune: (1-4,6,10) + ├── reject-nulls: (6,10) ├── interesting orderings: (+1) (-3,+4,+1) - ├── scan xysd - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) - │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ ├── prune: (1-4) - │ ├── interesting orderings: (+1) (-3,+4,+1) - │ └── unfiltered-cols: (1-4) - ├── group-by - │ ├── columns: u:5(int) sum:8(decimal!null) - │ ├── grouping columns: u:5(int) - │ ├── key: (5) - │ ├── fd: (5)-->(8) - │ ├── prune: (8) - │ ├── project - │ │ ├── columns: u:5(int) v:6(int!null) - │ │ ├── prune: (5,6) - │ │ └── scan uv - │ │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) - │ │ ├── key: (7) - │ │ ├── fd: (7)-->(5,6) - │ │ ├── prune: (5-7) - │ │ └── interesting orderings: (+7) - │ └── aggregations - │ └── sum [as=sum:8, type=decimal, outer=(6)] - │ └── variable: v:6 [type=int] - └── filters - └── is-not [type=bool, outer=(5), constraints=(/5: (/NULL - ]; tight)] - ├── variable: u:5 [type=int] - └── null [type=unknown] + └── left-join (cross) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) u:6(int) sum:10(decimal) + ├── key: (1,6) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (6)~~>(10), (1,6)-->(10) + ├── prune: (1-5,10) + ├── reject-nulls: (6,10) + ├── interesting orderings: (+1) (-3,+4,+1) + ├── scan xysd + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) + │ ├── key: (1) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── prune: (1-5) + │ ├── interesting orderings: (+1) (-3,+4,+1) + │ └── unfiltered-cols: (1-5) + ├── group-by + │ ├── columns: u:6(int) sum:10(decimal!null) + │ ├── grouping columns: u:6(int) + │ ├── key: (6) + │ ├── fd: (6)-->(10) + │ ├── prune: (10) + │ ├── project + │ │ ├── columns: u:6(int) v:7(int!null) + │ │ ├── prune: (6,7) + │ │ └── scan uv + │ │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + │ │ ├── key: (8) + │ │ ├── fd: (8)-->(6,7,9) + │ │ ├── prune: (6-9) + │ │ └── interesting orderings: (+8) + │ └── aggregations + │ └── sum [as=sum:10, type=decimal, outer=(7)] + │ └── variable: v:7 [type=int] + └── filters + └── is-not [type=bool, outer=(6), constraints=(/6: (/NULL - ]; tight)] + ├── variable: u:6 [type=int] + └── null [type=unknown] # Not-null FD determinant on right side of left-join stays strict. build SELECT * FROM xysd LEFT JOIN (SELECT u, sum(v) FROM uv WHERE u IS NOT NULL GROUP BY u) ON True ---- -left-join (cross) - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:5(int) sum:8(decimal) - ├── key: (1,5) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (5)-->(8) - ├── prune: (1-4,8) - ├── reject-nulls: (5,8) +project + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:6(int) sum:10(decimal) + ├── key: (1,6) + ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (6)-->(10) + ├── prune: (1-4,6,10) + ├── reject-nulls: (6,10) ├── interesting orderings: (+1) (-3,+4,+1) - ├── scan xysd - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) - │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ ├── prune: (1-4) - │ ├── interesting orderings: (+1) (-3,+4,+1) - │ └── unfiltered-cols: (1-4) - ├── group-by - │ ├── columns: u:5(int!null) sum:8(decimal!null) - │ ├── grouping columns: u:5(int!null) - │ ├── key: (5) - │ ├── fd: (5)-->(8) - │ ├── prune: (8) - │ ├── project - │ │ ├── columns: u:5(int!null) v:6(int!null) - │ │ ├── prune: (5,6) - │ │ └── select - │ │ ├── columns: u:5(int!null) v:6(int!null) rowid:7(int!null) - │ │ ├── key: (7) - │ │ ├── fd: (7)-->(5,6) - │ │ ├── prune: (6,7) - │ │ ├── interesting orderings: (+7) - │ │ ├── scan uv - │ │ │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) - │ │ │ ├── key: (7) - │ │ │ ├── fd: (7)-->(5,6) - │ │ │ ├── prune: (5-7) - │ │ │ └── interesting orderings: (+7) - │ │ └── filters - │ │ └── is-not [type=bool, outer=(5), constraints=(/5: (/NULL - ]; tight)] - │ │ ├── variable: u:5 [type=int] - │ │ └── null [type=unknown] - │ └── aggregations - │ └── sum [as=sum:8, type=decimal, outer=(6)] - │ └── variable: v:6 [type=int] - └── filters - └── true [type=bool] + └── left-join (cross) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) u:6(int) sum:10(decimal) + ├── key: (1,6) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (6)-->(10) + ├── prune: (1-5,10) + ├── reject-nulls: (6,10) + ├── interesting orderings: (+1) (-3,+4,+1) + ├── scan xysd + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) + │ ├── key: (1) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── prune: (1-5) + │ ├── interesting orderings: (+1) (-3,+4,+1) + │ └── unfiltered-cols: (1-5) + ├── group-by + │ ├── columns: u:6(int!null) sum:10(decimal!null) + │ ├── grouping columns: u:6(int!null) + │ ├── key: (6) + │ ├── fd: (6)-->(10) + │ ├── prune: (10) + │ ├── project + │ │ ├── columns: u:6(int!null) v:7(int!null) + │ │ ├── prune: (6,7) + │ │ └── select + │ │ ├── columns: u:6(int!null) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + │ │ ├── key: (8) + │ │ ├── fd: (8)-->(6,7,9) + │ │ ├── prune: (7-9) + │ │ ├── interesting orderings: (+8) + │ │ ├── scan uv + │ │ │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + │ │ │ ├── key: (8) + │ │ │ ├── fd: (8)-->(6,7,9) + │ │ │ ├── prune: (6-9) + │ │ │ └── interesting orderings: (+8) + │ │ └── filters + │ │ └── is-not [type=bool, outer=(6), constraints=(/6: (/NULL - ]; tight)] + │ │ ├── variable: u:6 [type=int] + │ │ └── null [type=unknown] + │ └── aggregations + │ └── sum [as=sum:10, type=decimal, outer=(7)] + │ └── variable: v:7 [type=int] + └── filters + └── true [type=bool] # Nullable FD determinant on left side of right-join becomes lax. build SELECT * FROM (SELECT u, sum(v) FROM uv GROUP BY u) RIGHT JOIN xysd ON u IS NOT NULL ---- -right-join (cross) - ├── columns: u:1(int) sum:4(decimal) x:5(int!null) y:6(int) s:7(string) d:8(decimal!null) - ├── key: (1,5) - ├── fd: (5)-->(6-8), (7,8)~~>(5,6), (1)~~>(4), (1,5)-->(4) - ├── prune: (4-8) - ├── reject-nulls: (1,4) - ├── interesting orderings: (+5) (-7,+8,+5) - ├── group-by - │ ├── columns: u:1(int) sum:4(decimal!null) - │ ├── grouping columns: u:1(int) - │ ├── key: (1) - │ ├── fd: (1)-->(4) - │ ├── prune: (4) - │ ├── project - │ │ ├── columns: u:1(int) v:2(int!null) - │ │ ├── prune: (1,2) - │ │ └── scan uv - │ │ ├── columns: u:1(int) v:2(int!null) rowid:3(int!null) - │ │ ├── key: (3) - │ │ ├── fd: (3)-->(1,2) - │ │ ├── prune: (1-3) - │ │ └── interesting orderings: (+3) - │ └── aggregations - │ └── sum [as=sum:4, type=decimal, outer=(2)] - │ └── variable: v:2 [type=int] - ├── scan xysd - │ ├── columns: x:5(int!null) y:6(int) s:7(string) d:8(decimal!null) - │ ├── key: (5) - │ ├── fd: (5)-->(6-8), (7,8)~~>(5,6) - │ ├── prune: (5-8) - │ └── interesting orderings: (+5) (-7,+8,+5) - └── filters - └── is-not [type=bool, outer=(1), constraints=(/1: (/NULL - ]; tight)] - ├── variable: u:1 [type=int] - └── null [type=unknown] +project + ├── columns: u:1(int) sum:5(decimal) x:6(int!null) y:7(int) s:8(string) d:9(decimal!null) + ├── key: (1,6) + ├── fd: (6)-->(7-9), (8,9)~~>(6,7), (1)~~>(5), (1,6)-->(5) + ├── prune: (1,5-9) + ├── reject-nulls: (1,5) + ├── interesting orderings: (+6) (-8,+9,+6) + └── right-join (cross) + ├── columns: u:1(int) sum:5(decimal) x:6(int!null) y:7(int) s:8(string) d:9(decimal!null) xysd.crdb_internal_mvcc_timestamp:10(decimal) + ├── key: (1,6) + ├── fd: (6)-->(7-10), (8,9)~~>(6,7,10), (1)~~>(5), (1,6)-->(5) + ├── prune: (5-10) + ├── reject-nulls: (1,5) + ├── interesting orderings: (+6) (-8,+9,+6) + ├── group-by + │ ├── columns: u:1(int) sum:5(decimal!null) + │ ├── grouping columns: u:1(int) + │ ├── key: (1) + │ ├── fd: (1)-->(5) + │ ├── prune: (5) + │ ├── project + │ │ ├── columns: u:1(int) v:2(int!null) + │ │ ├── prune: (1,2) + │ │ └── scan uv + │ │ ├── columns: u:1(int) v:2(int!null) rowid:3(int!null) uv.crdb_internal_mvcc_timestamp:4(decimal) + │ │ ├── key: (3) + │ │ ├── fd: (3)-->(1,2,4) + │ │ ├── prune: (1-4) + │ │ └── interesting orderings: (+3) + │ └── aggregations + │ └── sum [as=sum:5, type=decimal, outer=(2)] + │ └── variable: v:2 [type=int] + ├── scan xysd + │ ├── columns: x:6(int!null) y:7(int) s:8(string) d:9(decimal!null) xysd.crdb_internal_mvcc_timestamp:10(decimal) + │ ├── key: (6) + │ ├── fd: (6)-->(7-10), (8,9)~~>(6,7,10) + │ ├── prune: (6-10) + │ └── interesting orderings: (+6) (-8,+9,+6) + └── filters + └── is-not [type=bool, outer=(1), constraints=(/1: (/NULL - ]; tight)] + ├── variable: u:1 [type=int] + └── null [type=unknown] # Not-null FD determinant on left side of right-join stays strict. build SELECT * FROM (SELECT u, sum(v) FROM uv WHERE u IS NOT NULL GROUP BY u) RIGHT JOIN xysd ON True ---- -right-join (cross) - ├── columns: u:1(int) sum:4(decimal) x:5(int!null) y:6(int) s:7(string) d:8(decimal!null) - ├── key: (1,5) - ├── fd: (1)-->(4), (5)-->(6-8), (7,8)~~>(5,6) - ├── prune: (4-8) - ├── reject-nulls: (1,4) - ├── interesting orderings: (+5) (-7,+8,+5) - ├── group-by - │ ├── columns: u:1(int!null) sum:4(decimal!null) - │ ├── grouping columns: u:1(int!null) - │ ├── key: (1) - │ ├── fd: (1)-->(4) - │ ├── prune: (4) - │ ├── project - │ │ ├── columns: u:1(int!null) v:2(int!null) - │ │ ├── prune: (1,2) - │ │ └── select - │ │ ├── columns: u:1(int!null) v:2(int!null) rowid:3(int!null) - │ │ ├── key: (3) - │ │ ├── fd: (3)-->(1,2) - │ │ ├── prune: (2,3) - │ │ ├── interesting orderings: (+3) - │ │ ├── scan uv - │ │ │ ├── columns: u:1(int) v:2(int!null) rowid:3(int!null) - │ │ │ ├── key: (3) - │ │ │ ├── fd: (3)-->(1,2) - │ │ │ ├── prune: (1-3) - │ │ │ └── interesting orderings: (+3) - │ │ └── filters - │ │ └── is-not [type=bool, outer=(1), constraints=(/1: (/NULL - ]; tight)] - │ │ ├── variable: u:1 [type=int] - │ │ └── null [type=unknown] - │ └── aggregations - │ └── sum [as=sum:4, type=decimal, outer=(2)] - │ └── variable: v:2 [type=int] - ├── scan xysd - │ ├── columns: x:5(int!null) y:6(int) s:7(string) d:8(decimal!null) - │ ├── key: (5) - │ ├── fd: (5)-->(6-8), (7,8)~~>(5,6) - │ ├── prune: (5-8) - │ └── interesting orderings: (+5) (-7,+8,+5) - └── filters - └── true [type=bool] +project + ├── columns: u:1(int) sum:5(decimal) x:6(int!null) y:7(int) s:8(string) d:9(decimal!null) + ├── key: (1,6) + ├── fd: (1)-->(5), (6)-->(7-9), (8,9)~~>(6,7) + ├── prune: (1,5-9) + ├── reject-nulls: (1,5) + ├── interesting orderings: (+6) (-8,+9,+6) + └── right-join (cross) + ├── columns: u:1(int) sum:5(decimal) x:6(int!null) y:7(int) s:8(string) d:9(decimal!null) xysd.crdb_internal_mvcc_timestamp:10(decimal) + ├── key: (1,6) + ├── fd: (1)-->(5), (6)-->(7-10), (8,9)~~>(6,7,10) + ├── prune: (5-10) + ├── reject-nulls: (1,5) + ├── interesting orderings: (+6) (-8,+9,+6) + ├── group-by + │ ├── columns: u:1(int!null) sum:5(decimal!null) + │ ├── grouping columns: u:1(int!null) + │ ├── key: (1) + │ ├── fd: (1)-->(5) + │ ├── prune: (5) + │ ├── project + │ │ ├── columns: u:1(int!null) v:2(int!null) + │ │ ├── prune: (1,2) + │ │ └── select + │ │ ├── columns: u:1(int!null) v:2(int!null) rowid:3(int!null) uv.crdb_internal_mvcc_timestamp:4(decimal) + │ │ ├── key: (3) + │ │ ├── fd: (3)-->(1,2,4) + │ │ ├── prune: (2-4) + │ │ ├── interesting orderings: (+3) + │ │ ├── scan uv + │ │ │ ├── columns: u:1(int) v:2(int!null) rowid:3(int!null) uv.crdb_internal_mvcc_timestamp:4(decimal) + │ │ │ ├── key: (3) + │ │ │ ├── fd: (3)-->(1,2,4) + │ │ │ ├── prune: (1-4) + │ │ │ └── interesting orderings: (+3) + │ │ └── filters + │ │ └── is-not [type=bool, outer=(1), constraints=(/1: (/NULL - ]; tight)] + │ │ ├── variable: u:1 [type=int] + │ │ └── null [type=unknown] + │ └── aggregations + │ └── sum [as=sum:5, type=decimal, outer=(2)] + │ └── variable: v:2 [type=int] + ├── scan xysd + │ ├── columns: x:6(int!null) y:7(int) s:8(string) d:9(decimal!null) xysd.crdb_internal_mvcc_timestamp:10(decimal) + │ ├── key: (6) + │ ├── fd: (6)-->(7-10), (8,9)~~>(6,7,10) + │ ├── prune: (6-10) + │ └── interesting orderings: (+6) (-8,+9,+6) + └── filters + └── true [type=bool] # Nullable FD determinant on right side of full-join becomes lax. build SELECT * FROM xysd FULL JOIN (SELECT u, sum(v) FROM uv GROUP BY u) ON u IS NOT NULL ---- -full-join (cross) - ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) u:5(int) sum:8(decimal) - ├── key: (1,5) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (5)~~>(8), (1,5)-->(8) - ├── prune: (1-4,8) - ├── reject-nulls: (1-5,8) +project + ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) u:6(int) sum:10(decimal) + ├── key: (1,6) + ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (6)~~>(10), (1,6)-->(10) + ├── prune: (1-4,6,10) + ├── reject-nulls: (1-4,6,10) ├── interesting orderings: (+1) (-3,+4,+1) - ├── scan xysd - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) - │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ ├── prune: (1-4) - │ ├── interesting orderings: (+1) (-3,+4,+1) - │ └── unfiltered-cols: (1-4) - ├── group-by - │ ├── columns: u:5(int) sum:8(decimal!null) - │ ├── grouping columns: u:5(int) - │ ├── key: (5) - │ ├── fd: (5)-->(8) - │ ├── prune: (8) - │ ├── project - │ │ ├── columns: u:5(int) v:6(int!null) - │ │ ├── prune: (5,6) - │ │ └── scan uv - │ │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) - │ │ ├── key: (7) - │ │ ├── fd: (7)-->(5,6) - │ │ ├── prune: (5-7) - │ │ └── interesting orderings: (+7) - │ └── aggregations - │ └── sum [as=sum:8, type=decimal, outer=(6)] - │ └── variable: v:6 [type=int] - └── filters - └── is-not [type=bool, outer=(5), constraints=(/5: (/NULL - ]; tight)] - ├── variable: u:5 [type=int] - └── null [type=unknown] + └── full-join (cross) + ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) xysd.crdb_internal_mvcc_timestamp:5(decimal) u:6(int) sum:10(decimal) + ├── key: (1,6) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (6)~~>(10), (1,6)-->(10) + ├── prune: (1-5,10) + ├── reject-nulls: (1-6,10) + ├── interesting orderings: (+1) (-3,+4,+1) + ├── scan xysd + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) + │ ├── key: (1) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── prune: (1-5) + │ ├── interesting orderings: (+1) (-3,+4,+1) + │ └── unfiltered-cols: (1-5) + ├── group-by + │ ├── columns: u:6(int) sum:10(decimal!null) + │ ├── grouping columns: u:6(int) + │ ├── key: (6) + │ ├── fd: (6)-->(10) + │ ├── prune: (10) + │ ├── project + │ │ ├── columns: u:6(int) v:7(int!null) + │ │ ├── prune: (6,7) + │ │ └── scan uv + │ │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + │ │ ├── key: (8) + │ │ ├── fd: (8)-->(6,7,9) + │ │ ├── prune: (6-9) + │ │ └── interesting orderings: (+8) + │ └── aggregations + │ └── sum [as=sum:10, type=decimal, outer=(7)] + │ └── variable: v:7 [type=int] + └── filters + └── is-not [type=bool, outer=(6), constraints=(/6: (/NULL - ]; tight)] + ├── variable: u:6 [type=int] + └── null [type=unknown] # Nullable FD determinant on left side of full-join becomes lax. build SELECT * FROM (SELECT u, sum(v) FROM uv GROUP BY u) FULL JOIN xysd ON u IS NOT NULL ---- -full-join (cross) - ├── columns: u:1(int) sum:4(decimal) x:5(int) y:6(int) s:7(string) d:8(decimal) - ├── key: (1,5) - ├── fd: (5)-->(6-8), (7,8)~~>(5,6), (1)~~>(4), (1,5)-->(4) - ├── prune: (4-8) - ├── reject-nulls: (1,4-8) - ├── interesting orderings: (+5) (-7,+8,+5) - ├── group-by - │ ├── columns: u:1(int) sum:4(decimal!null) - │ ├── grouping columns: u:1(int) - │ ├── key: (1) - │ ├── fd: (1)-->(4) - │ ├── prune: (4) - │ ├── project - │ │ ├── columns: u:1(int) v:2(int!null) - │ │ ├── prune: (1,2) - │ │ └── scan uv - │ │ ├── columns: u:1(int) v:2(int!null) rowid:3(int!null) - │ │ ├── key: (3) - │ │ ├── fd: (3)-->(1,2) - │ │ ├── prune: (1-3) - │ │ └── interesting orderings: (+3) - │ └── aggregations - │ └── sum [as=sum:4, type=decimal, outer=(2)] - │ └── variable: v:2 [type=int] - ├── scan xysd - │ ├── columns: x:5(int!null) y:6(int) s:7(string) d:8(decimal!null) - │ ├── key: (5) - │ ├── fd: (5)-->(6-8), (7,8)~~>(5,6) - │ ├── prune: (5-8) - │ ├── interesting orderings: (+5) (-7,+8,+5) - │ └── unfiltered-cols: (5-8) - └── filters - └── is-not [type=bool, outer=(1), constraints=(/1: (/NULL - ]; tight)] - ├── variable: u:1 [type=int] - └── null [type=unknown] +project + ├── columns: u:1(int) sum:5(decimal) x:6(int) y:7(int) s:8(string) d:9(decimal) + ├── key: (1,6) + ├── fd: (6)-->(7-9), (8,9)~~>(6,7), (1)~~>(5), (1,6)-->(5) + ├── prune: (1,5-9) + ├── reject-nulls: (1,5-9) + ├── interesting orderings: (+6) (-8,+9,+6) + └── full-join (cross) + ├── columns: u:1(int) sum:5(decimal) x:6(int) y:7(int) s:8(string) d:9(decimal) xysd.crdb_internal_mvcc_timestamp:10(decimal) + ├── key: (1,6) + ├── fd: (6)-->(7-10), (8,9)~~>(6,7,10), (1)~~>(5), (1,6)-->(5) + ├── prune: (5-10) + ├── reject-nulls: (1,5-10) + ├── interesting orderings: (+6) (-8,+9,+6) + ├── group-by + │ ├── columns: u:1(int) sum:5(decimal!null) + │ ├── grouping columns: u:1(int) + │ ├── key: (1) + │ ├── fd: (1)-->(5) + │ ├── prune: (5) + │ ├── project + │ │ ├── columns: u:1(int) v:2(int!null) + │ │ ├── prune: (1,2) + │ │ └── scan uv + │ │ ├── columns: u:1(int) v:2(int!null) rowid:3(int!null) uv.crdb_internal_mvcc_timestamp:4(decimal) + │ │ ├── key: (3) + │ │ ├── fd: (3)-->(1,2,4) + │ │ ├── prune: (1-4) + │ │ └── interesting orderings: (+3) + │ └── aggregations + │ └── sum [as=sum:5, type=decimal, outer=(2)] + │ └── variable: v:2 [type=int] + ├── scan xysd + │ ├── columns: x:6(int!null) y:7(int) s:8(string) d:9(decimal!null) xysd.crdb_internal_mvcc_timestamp:10(decimal) + │ ├── key: (6) + │ ├── fd: (6)-->(7-10), (8,9)~~>(6,7,10) + │ ├── prune: (6-10) + │ ├── interesting orderings: (+6) (-8,+9,+6) + │ └── unfiltered-cols: (6-10) + └── filters + └── is-not [type=bool, outer=(1), constraints=(/1: (/NULL - ]; tight)] + ├── variable: u:1 [type=int] + └── null [type=unknown] # Merge join (inner). expr @@ -1358,11 +1461,11 @@ expr ) ---- inner-join (merge) - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:5(int!null) v:6(int!null) rowid:7(int!null) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:6(int!null) v:7(int!null) rowid:8(int!null) ├── left ordering: +1 - ├── right ordering: +5 - ├── key: (7) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (7)-->(5,6), (1)==(5), (5)==(1) + ├── right ordering: +6 + ├── key: (8) + ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (8)-->(6,7), (1)==(6), (6)==(1) ├── scan xysd │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) │ ├── key: (1) @@ -1371,18 +1474,18 @@ inner-join (merge) │ ├── prune: (1-4) │ └── interesting orderings: (+1) (-3,+4,+1) ├── sort - │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) - │ ├── key: (7) - │ ├── fd: (7)-->(5,6) - │ ├── ordering: +5 - │ ├── prune: (5-7) - │ ├── interesting orderings: (+7) + │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) + │ ├── key: (8) + │ ├── fd: (8)-->(6,7) + │ ├── ordering: +6 + │ ├── prune: (6-8) + │ ├── interesting orderings: (+8) │ └── scan uv - │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) - │ ├── key: (7) - │ ├── fd: (7)-->(5,6) - │ ├── prune: (5-7) - │ └── interesting orderings: (+7) + │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) + │ ├── key: (8) + │ ├── fd: (8)-->(6,7) + │ ├── prune: (6-8) + │ └── interesting orderings: (+8) └── filters (true) # Merge join (left). @@ -1401,11 +1504,11 @@ expr ) ---- left-join (merge) - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:5(int) v:6(int) rowid:7(int) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:6(int) v:7(int) rowid:8(int) ├── left ordering: +1 - ├── right ordering: +5 - ├── key: (1,7) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (7)-->(5,6) + ├── right ordering: +6 + ├── key: (1,8) + ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (8)-->(6,7) ├── scan xysd │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) │ ├── key: (1) @@ -1414,18 +1517,18 @@ left-join (merge) │ ├── prune: (1-4) │ └── interesting orderings: (+1) (-3,+4,+1) ├── sort - │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) - │ ├── key: (7) - │ ├── fd: (7)-->(5,6) - │ ├── ordering: +5 - │ ├── prune: (5-7) - │ ├── interesting orderings: (+7) + │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) + │ ├── key: (8) + │ ├── fd: (8)-->(6,7) + │ ├── ordering: +6 + │ ├── prune: (6-8) + │ ├── interesting orderings: (+8) │ └── scan uv - │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) - │ ├── key: (7) - │ ├── fd: (7)-->(5,6) - │ ├── prune: (5-7) - │ └── interesting orderings: (+7) + │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) + │ ├── key: (8) + │ ├── fd: (8)-->(6,7) + │ ├── prune: (6-8) + │ └── interesting orderings: (+8) └── filters (true) # Merge join (right) with remaining ON condition. @@ -1444,11 +1547,11 @@ expr ) ---- right-join (merge) - ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) u:5(int) v:6(int!null) rowid:7(int!null) + ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) u:6(int) v:7(int!null) rowid:8(int!null) ├── left ordering: +1 - ├── right ordering: +5 - ├── key: (7) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (7)-->(1-6) + ├── right ordering: +6 + ├── key: (8) + ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (8)-->(1-4,6,7) ├── scan xysd │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) │ ├── key: (1) @@ -1457,22 +1560,22 @@ right-join (merge) │ ├── prune: (1-4) │ └── interesting orderings: (+1) (-3,+4,+1) ├── sort - │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) - │ ├── key: (7) - │ ├── fd: (7)-->(5,6) - │ ├── ordering: +5 - │ ├── prune: (5-7) - │ ├── interesting orderings: (+7) + │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) + │ ├── key: (8) + │ ├── fd: (8)-->(6,7) + │ ├── ordering: +6 + │ ├── prune: (6-8) + │ ├── interesting orderings: (+8) │ └── scan uv - │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) - │ ├── key: (7) - │ ├── fd: (7)-->(5,6) - │ ├── prune: (5-7) - │ └── interesting orderings: (+7) + │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) + │ ├── key: (8) + │ ├── fd: (8)-->(6,7) + │ ├── prune: (6-8) + │ └── interesting orderings: (+8) └── filters - └── gt [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + └── gt [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] ├── variable: y:2 [type=int] - └── variable: v:6 [type=int] + └── variable: v:7 [type=int] # Regression test #36183. opt @@ -1482,51 +1585,51 @@ SELECT (SELECT m FROM ) FROM uv ---- with &1 - ├── columns: m:19(int) + ├── columns: m:24(int) ├── volatile, mutations - ├── fd: ()-->(19) - ├── prune: (19) + ├── fd: ()-->(24) + ├── prune: (24) ├── project - │ ├── columns: uv.u:4(int!null) uv.v:5(int!null) + │ ├── columns: uv.u:5(int!null) uv.v:6(int!null) │ ├── cardinality: [1 - 1] │ ├── volatile, mutations │ ├── key: () - │ ├── fd: ()-->(4,5) - │ ├── prune: (4,5) + │ ├── fd: ()-->(5,6) + │ ├── prune: (5,6) │ └── insert uv - │ ├── columns: uv.u:4(int!null) uv.v:5(int!null) rowid:6(int!null) + │ ├── columns: uv.u:5(int!null) uv.v:6(int!null) rowid:7(int!null) │ ├── insert-mapping: - │ │ ├── column1:7 => uv.u:4 - │ │ ├── column2:8 => uv.v:5 - │ │ └── column9:9 => rowid:6 + │ │ ├── column1:9 => uv.u:5 + │ │ ├── column2:10 => uv.v:6 + │ │ └── column11:11 => rowid:7 │ ├── cardinality: [1 - 1] │ ├── volatile, mutations │ ├── key: () - │ ├── fd: ()-->(4-6) + │ ├── fd: ()-->(5-7) │ └── values - │ ├── columns: column1:7(int!null) column2:8(int!null) column9:9(int) + │ ├── columns: column1:9(int!null) column2:10(int!null) column11:11(int) │ ├── cardinality: [1 - 1] │ ├── volatile │ ├── key: () - │ ├── fd: ()-->(7-9) - │ ├── prune: (7-9) + │ ├── fd: ()-->(9-11) + │ ├── prune: (9-11) │ └── tuple [type=tuple{int, int, int}] │ ├── const: 1 [type=int] │ ├── const: 2 [type=int] │ └── function: unique_rowid [type=int] └── project - ├── columns: m:19(int) - ├── fd: ()-->(19) - ├── prune: (19) + ├── columns: m:24(int) + ├── fd: ()-->(24) + ├── prune: (24) ├── scan uv └── projections - └── subquery [as=m:19, type=int, subquery] + └── subquery [as=m:24, type=int, subquery] └── values - ├── columns: mn.m:15(int!null) + ├── columns: mn.m:18(int!null) ├── cardinality: [0 - 0] ├── key: () - ├── fd: ()-->(15) - └── prune: (15) + ├── fd: ()-->(18) + └── prune: (18) # Regression test #40456. opt @@ -1535,11 +1638,11 @@ FROM uv WHERE NOT EXISTS(SELECT uv.u); ---- values - ├── columns: "?column?":5(unknown!null) + ├── columns: "?column?":6(unknown!null) ├── cardinality: [0 - 0] ├── key: () - ├── fd: ()-->(5) - └── prune: (5) + ├── fd: ()-->(6) + └── prune: (6) # Regression test #43651: outer join with empty key. opt @@ -1595,11 +1698,11 @@ FULL JOIN ON t1.x = t2.x ---- full-join (hash) - ├── columns: x:1(int) y:2(int) x:4(int) y:5(int) + ├── columns: x:1(int) y:2(int) x:5(int) y:6(int) ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) - ├── lax-key: (1,4) - ├── fd: (1)~~>(2), (4)~~>(5), (1,4)~~>(2,5) - ├── reject-nulls: (1,2,4,5) + ├── lax-key: (1,5) + ├── fd: (1)~~>(2), (5)~~>(6), (1,5)~~>(2,6) + ├── reject-nulls: (1,2,5,6) ├── select │ ├── columns: t1.x:1(int) t1.y:2(int!null) │ ├── key: (1) @@ -1621,29 +1724,29 @@ full-join (hash) │ ├── variable: t1.y:2 [type=int] │ └── null [type=unknown] ├── select - │ ├── columns: t2.x:4(int) t2.y:5(int!null) - │ ├── key: (4) - │ ├── fd: (4)-->(5) + │ ├── columns: t2.x:5(int) t2.y:6(int!null) + │ ├── key: (5) + │ ├── fd: (5)-->(6) │ ├── distinct-on - │ │ ├── columns: t2.x:4(int) t2.y:5(int) - │ │ ├── grouping columns: t2.x:4(int) - │ │ ├── key: (4) - │ │ ├── fd: (4)-->(5) - │ │ ├── prune: (5) + │ │ ├── columns: t2.x:5(int) t2.y:6(int) + │ │ ├── grouping columns: t2.x:5(int) + │ │ ├── key: (5) + │ │ ├── fd: (5)-->(6) + │ │ ├── prune: (6) │ │ ├── scan t2 - │ │ │ ├── columns: t2.x:4(int) t2.y:5(int) - │ │ │ └── prune: (4,5) + │ │ │ ├── columns: t2.x:5(int) t2.y:6(int) + │ │ │ └── prune: (5,6) │ │ └── aggregations - │ │ └── first-agg [as=t2.y:5, type=int, outer=(5)] - │ │ └── variable: t2.y:5 [type=int] + │ │ └── first-agg [as=t2.y:6, type=int, outer=(6)] + │ │ └── variable: t2.y:6 [type=int] │ └── filters - │ └── is-not [type=bool, outer=(5), constraints=(/5: (/NULL - ]; tight)] - │ ├── variable: t2.y:5 [type=int] + │ └── is-not [type=bool, outer=(6), constraints=(/6: (/NULL - ]; tight)] + │ ├── variable: t2.y:6 [type=int] │ └── null [type=unknown] └── filters - └── eq [type=bool, outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + └── eq [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] ├── variable: t1.x:1 [type=int] - └── variable: t2.x:4 [type=int] + └── variable: t2.x:5 [type=int] # InnerJoin with an equality between one key column and one non-key column. # Neither input is guaranteed a match for every row. Rows from uv will not be @@ -1653,10 +1756,10 @@ norm SELECT * FROM xysd INNER JOIN uv ON x=v ---- inner-join (hash) - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:5(int) v:6(int!null) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:6(int) v:7(int!null) ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (1)==(6), (6)==(1) - ├── prune: (2-5) + ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (1)==(7), (7)==(1) + ├── prune: (2-4,6) ├── interesting orderings: (+1) (-3,+4,+1) ├── scan xysd │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) @@ -1664,15 +1767,15 @@ inner-join (hash) │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) │ ├── prune: (1-4) │ ├── interesting orderings: (+1) (-3,+4,+1) - │ └── unfiltered-cols: (1-4) + │ └── unfiltered-cols: (1-5) ├── scan uv - │ ├── columns: u:5(int) v:6(int!null) - │ ├── prune: (5,6) - │ └── unfiltered-cols: (5-7) + │ ├── columns: u:6(int) v:7(int!null) + │ ├── prune: (6,7) + │ └── unfiltered-cols: (6-9) └── filters - └── eq [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── eq [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] ├── variable: x:1 [type=int] - └── variable: v:6 [type=int] + └── variable: v:7 [type=int] # InnerJoin with a not-null foreign key equality. Since the foreign key is # not-null, rows from the fk table are guaranteed a match. Since x is a key @@ -1681,29 +1784,29 @@ norm SELECT * FROM fk INNER JOIN xysd ON x = r1 ---- inner-join (hash) - ├── columns: k:1(int!null) v:2(int) r1:3(int!null) r2:4(int) x:5(int!null) y:6(int) s:7(string) d:8(decimal!null) + ├── columns: k:1(int!null) v:2(int) r1:3(int!null) r2:4(int) x:6(int!null) y:7(int) s:8(string) d:9(decimal!null) ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) ├── key: (1) - ├── fd: (1)-->(2-4), (5)-->(6-8), (7,8)~~>(5,6), (3)==(5), (5)==(3) - ├── prune: (1,2,4,6-8) - ├── interesting orderings: (+1) (+3,+1) (+4,+1) (+5) (-7,+8,+5) + ├── fd: (1)-->(2-4), (6)-->(7-9), (8,9)~~>(6,7), (3)==(6), (6)==(3) + ├── prune: (1,2,4,7-9) + ├── interesting orderings: (+1) (+3,+1) (+4,+1) (+6) (-8,+9,+6) ├── scan fk │ ├── columns: k:1(int!null) v:2(int) r1:3(int!null) r2:4(int) │ ├── key: (1) │ ├── fd: (1)-->(2-4) │ ├── prune: (1-4) │ ├── interesting orderings: (+1) (+3,+1) (+4,+1) - │ └── unfiltered-cols: (1-4) + │ └── unfiltered-cols: (1-5) ├── scan xysd - │ ├── columns: x:5(int!null) y:6(int) s:7(string) d:8(decimal!null) - │ ├── key: (5) - │ ├── fd: (5)-->(6-8), (7,8)~~>(5,6) - │ ├── prune: (5-8) - │ ├── interesting orderings: (+5) (-7,+8,+5) - │ └── unfiltered-cols: (5-8) + │ ├── columns: x:6(int!null) y:7(int) s:8(string) d:9(decimal!null) + │ ├── key: (6) + │ ├── fd: (6)-->(7-9), (8,9)~~>(6,7) + │ ├── prune: (6-9) + │ ├── interesting orderings: (+6) (-8,+9,+6) + │ └── unfiltered-cols: (6-10) └── filters - └── eq [type=bool, outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ]), fd=(3)==(5), (5)==(3)] - ├── variable: x:5 [type=int] + └── eq [type=bool, outer=(3,6), constraints=(/3: (/NULL - ]; /6: (/NULL - ]), fd=(3)==(6), (6)==(3)] + ├── variable: x:6 [type=int] └── variable: r1:3 [type=int] # InnerJoin with a nullable foreign key equality condition. @@ -1711,29 +1814,29 @@ norm SELECT * FROM fk INNER JOIN xysd ON x = r2 ---- inner-join (hash) - ├── columns: k:1(int!null) v:2(int) r1:3(int!null) r2:4(int!null) x:5(int!null) y:6(int) s:7(string) d:8(decimal!null) + ├── columns: k:1(int!null) v:2(int) r1:3(int!null) r2:4(int!null) x:6(int!null) y:7(int) s:8(string) d:9(decimal!null) ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) ├── key: (1) - ├── fd: (1)-->(2-4), (5)-->(6-8), (7,8)~~>(5,6), (4)==(5), (5)==(4) - ├── prune: (1-3,6-8) - ├── interesting orderings: (+1) (+3,+1) (+4,+1) (+5) (-7,+8,+5) + ├── fd: (1)-->(2-4), (6)-->(7-9), (8,9)~~>(6,7), (4)==(6), (6)==(4) + ├── prune: (1-3,7-9) + ├── interesting orderings: (+1) (+3,+1) (+4,+1) (+6) (-8,+9,+6) ├── scan fk │ ├── columns: k:1(int!null) v:2(int) r1:3(int!null) r2:4(int) │ ├── key: (1) │ ├── fd: (1)-->(2-4) │ ├── prune: (1-4) │ ├── interesting orderings: (+1) (+3,+1) (+4,+1) - │ └── unfiltered-cols: (1-4) + │ └── unfiltered-cols: (1-5) ├── scan xysd - │ ├── columns: x:5(int!null) y:6(int) s:7(string) d:8(decimal!null) - │ ├── key: (5) - │ ├── fd: (5)-->(6-8), (7,8)~~>(5,6) - │ ├── prune: (5-8) - │ ├── interesting orderings: (+5) (-7,+8,+5) - │ └── unfiltered-cols: (5-8) + │ ├── columns: x:6(int!null) y:7(int) s:8(string) d:9(decimal!null) + │ ├── key: (6) + │ ├── fd: (6)-->(7-9), (8,9)~~>(6,7) + │ ├── prune: (6-9) + │ ├── interesting orderings: (+6) (-8,+9,+6) + │ └── unfiltered-cols: (6-10) └── filters - └── eq [type=bool, outer=(4,5), constraints=(/4: (/NULL - ]; /5: (/NULL - ]), fd=(4)==(5), (5)==(4)] - ├── variable: x:5 [type=int] + └── eq [type=bool, outer=(4,6), constraints=(/4: (/NULL - ]; /6: (/NULL - ]), fd=(4)==(6), (6)==(4)] + ├── variable: x:6 [type=int] └── variable: r2:4 [type=int] # Cross join. Rows from fk are guaranteed matches because the not-null foreign @@ -1742,26 +1845,26 @@ norm SELECT * FROM fk CROSS JOIN xysd ---- inner-join (cross) - ├── columns: k:1(int!null) v:2(int) r1:3(int!null) r2:4(int) x:5(int!null) y:6(int) s:7(string) d:8(decimal!null) + ├── columns: k:1(int!null) v:2(int) r1:3(int!null) r2:4(int) x:6(int!null) y:7(int) s:8(string) d:9(decimal!null) ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-more) - ├── key: (1,5) - ├── fd: (1)-->(2-4), (5)-->(6-8), (7,8)~~>(5,6) - ├── prune: (1-8) - ├── interesting orderings: (+1) (+3,+1) (+4,+1) (+5) (-7,+8,+5) + ├── key: (1,6) + ├── fd: (1)-->(2-4), (6)-->(7-9), (8,9)~~>(6,7) + ├── prune: (1-4,6-9) + ├── interesting orderings: (+1) (+3,+1) (+4,+1) (+6) (-8,+9,+6) ├── scan fk │ ├── columns: k:1(int!null) v:2(int) r1:3(int!null) r2:4(int) │ ├── key: (1) │ ├── fd: (1)-->(2-4) │ ├── prune: (1-4) │ ├── interesting orderings: (+1) (+3,+1) (+4,+1) - │ └── unfiltered-cols: (1-4) + │ └── unfiltered-cols: (1-5) ├── scan xysd - │ ├── columns: x:5(int!null) y:6(int) s:7(string) d:8(decimal!null) - │ ├── key: (5) - │ ├── fd: (5)-->(6-8), (7,8)~~>(5,6) - │ ├── prune: (5-8) - │ ├── interesting orderings: (+5) (-7,+8,+5) - │ └── unfiltered-cols: (5-8) + │ ├── columns: x:6(int!null) y:7(int) s:8(string) d:9(decimal!null) + │ ├── key: (6) + │ ├── fd: (6)-->(7-9), (8,9)~~>(6,7) + │ ├── prune: (6-9) + │ ├── interesting orderings: (+6) (-8,+9,+6) + │ └── unfiltered-cols: (6-10) └── filters (true) # LeftJoin case with a not-null foreign key. Since fk rows are all guaranteed @@ -1771,29 +1874,29 @@ norm SELECT * FROM fk LEFT JOIN xysd ON x = r1 ---- inner-join (hash) - ├── columns: k:1(int!null) v:2(int) r1:3(int!null) r2:4(int) x:5(int!null) y:6(int) s:7(string) d:8(decimal!null) + ├── columns: k:1(int!null) v:2(int) r1:3(int!null) r2:4(int) x:6(int!null) y:7(int) s:8(string) d:9(decimal!null) ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) ├── key: (1) - ├── fd: (1)-->(2-4), (5)-->(6-8), (7,8)~~>(5,6), (3)==(5), (5)==(3) - ├── prune: (1,2,4,6-8) - ├── interesting orderings: (+1) (+3,+1) (+4,+1) (+5) (-7,+8,+5) + ├── fd: (1)-->(2-4), (6)-->(7-9), (8,9)~~>(6,7), (3)==(6), (6)==(3) + ├── prune: (1,2,4,7-9) + ├── interesting orderings: (+1) (+3,+1) (+4,+1) (+6) (-8,+9,+6) ├── scan fk │ ├── columns: k:1(int!null) v:2(int) r1:3(int!null) r2:4(int) │ ├── key: (1) │ ├── fd: (1)-->(2-4) │ ├── prune: (1-4) │ ├── interesting orderings: (+1) (+3,+1) (+4,+1) - │ └── unfiltered-cols: (1-4) + │ └── unfiltered-cols: (1-5) ├── scan xysd - │ ├── columns: x:5(int!null) y:6(int) s:7(string) d:8(decimal!null) - │ ├── key: (5) - │ ├── fd: (5)-->(6-8), (7,8)~~>(5,6) - │ ├── prune: (5-8) - │ ├── interesting orderings: (+5) (-7,+8,+5) - │ └── unfiltered-cols: (5-8) + │ ├── columns: x:6(int!null) y:7(int) s:8(string) d:9(decimal!null) + │ ├── key: (6) + │ ├── fd: (6)-->(7-9), (8,9)~~>(6,7) + │ ├── prune: (6-9) + │ ├── interesting orderings: (+6) (-8,+9,+6) + │ └── unfiltered-cols: (6-10) └── filters - └── eq [type=bool, outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ]), fd=(3)==(5), (5)==(3)] - ├── variable: x:5 [type=int] + └── eq [type=bool, outer=(3,6), constraints=(/3: (/NULL - ]; /6: (/NULL - ]), fd=(3)==(6), (6)==(3)] + ├── variable: x:6 [type=int] └── variable: r1:3 [type=int] @@ -1803,30 +1906,30 @@ norm SELECT * FROM fk LEFT JOIN xysd ON x = r2 ---- left-join (hash) - ├── columns: k:1(int!null) v:2(int) r1:3(int!null) r2:4(int) x:5(int) y:6(int) s:7(string) d:8(decimal) + ├── columns: k:1(int!null) v:2(int) r1:3(int!null) r2:4(int) x:6(int) y:7(int) s:8(string) d:9(decimal) ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) ├── key: (1) - ├── fd: (1)-->(2-8), (5)-->(6-8), (7,8)~~>(5,6) - ├── prune: (1-3,6-8) - ├── reject-nulls: (5-8) - ├── interesting orderings: (+1) (+3,+1) (+4,+1) (+5) (-7,+8,+5) + ├── fd: (1)-->(2-4,6-9), (6)-->(7-9), (8,9)~~>(6,7) + ├── prune: (1-3,7-9) + ├── reject-nulls: (6-9) + ├── interesting orderings: (+1) (+3,+1) (+4,+1) (+6) (-8,+9,+6) ├── scan fk │ ├── columns: k:1(int!null) v:2(int) r1:3(int!null) r2:4(int) │ ├── key: (1) │ ├── fd: (1)-->(2-4) │ ├── prune: (1-4) │ ├── interesting orderings: (+1) (+3,+1) (+4,+1) - │ └── unfiltered-cols: (1-4) + │ └── unfiltered-cols: (1-5) ├── scan xysd - │ ├── columns: x:5(int!null) y:6(int) s:7(string) d:8(decimal!null) - │ ├── key: (5) - │ ├── fd: (5)-->(6-8), (7,8)~~>(5,6) - │ ├── prune: (5-8) - │ ├── interesting orderings: (+5) (-7,+8,+5) - │ └── unfiltered-cols: (5-8) + │ ├── columns: x:6(int!null) y:7(int) s:8(string) d:9(decimal!null) + │ ├── key: (6) + │ ├── fd: (6)-->(7-9), (8,9)~~>(6,7) + │ ├── prune: (6-9) + │ ├── interesting orderings: (+6) (-8,+9,+6) + │ └── unfiltered-cols: (6-10) └── filters - └── eq [type=bool, outer=(4,5), constraints=(/4: (/NULL - ]; /5: (/NULL - ]), fd=(4)==(5), (5)==(4)] - ├── variable: x:5 [type=int] + └── eq [type=bool, outer=(4,6), constraints=(/4: (/NULL - ]; /6: (/NULL - ]), fd=(4)==(6), (6)==(4)] + ├── variable: x:6 [type=int] └── variable: r2:4 [type=int] # FullJoin with equality between key columns. The FullJoin adds back any rows @@ -1836,31 +1939,31 @@ norm SELECT * FROM mn FULL JOIN xysd ON m = x ---- full-join (hash) - ├── columns: m:1(int) n:2(int) x:3(int) y:4(int) s:5(string) d:6(decimal) + ├── columns: m:1(int) n:2(int) x:4(int) y:5(int) s:6(string) d:7(decimal) ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) - ├── key: (1,3) - ├── fd: (1)-->(2), (2)~~>(1), (3)-->(4-6), (5,6)~~>(3,4) - ├── prune: (2,4-6) - ├── reject-nulls: (1-6) - ├── interesting orderings: (+1) (+2,+1) (+3) (-5,+6,+3) + ├── key: (1,4) + ├── fd: (1)-->(2), (2)~~>(1), (4)-->(5-7), (6,7)~~>(4,5) + ├── prune: (2,5-7) + ├── reject-nulls: (1,2,4-7) + ├── interesting orderings: (+1) (+2,+1) (+4) (-6,+7,+4) ├── scan mn │ ├── columns: m:1(int!null) n:2(int) │ ├── key: (1) │ ├── fd: (1)-->(2), (2)~~>(1) │ ├── prune: (1,2) │ ├── interesting orderings: (+1) (+2,+1) - │ └── unfiltered-cols: (1,2) + │ └── unfiltered-cols: (1-3) ├── scan xysd - │ ├── columns: x:3(int!null) y:4(int) s:5(string) d:6(decimal!null) - │ ├── key: (3) - │ ├── fd: (3)-->(4-6), (5,6)~~>(3,4) - │ ├── prune: (3-6) - │ ├── interesting orderings: (+3) (-5,+6,+3) - │ └── unfiltered-cols: (3-6) + │ ├── columns: x:4(int!null) y:5(int) s:6(string) d:7(decimal!null) + │ ├── key: (4) + │ ├── fd: (4)-->(5-7), (6,7)~~>(4,5) + │ ├── prune: (4-7) + │ ├── interesting orderings: (+4) (-6,+7,+4) + │ └── unfiltered-cols: (4-8) └── filters - └── eq [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + └── eq [type=bool, outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] ├── variable: m:1 [type=int] - └── variable: x:3 [type=int] + └── variable: x:4 [type=int] # Self-join case. Since the condition is equating a key column with itself, # every row from both inputs is guaranteed to be included in the join output @@ -1869,30 +1972,30 @@ norm SELECT * FROM xysd INNER JOIN xysd AS a ON xysd.x = a.x ---- inner-join (hash) - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) x:5(int!null) y:6(int) s:7(string) d:8(decimal!null) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) x:6(int!null) y:7(int) s:8(string) d:9(decimal!null) ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) - ├── key: (5) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (5)-->(6-8), (7,8)~~>(5,6), (1)==(5), (5)==(1) - ├── prune: (2-4,6-8) - ├── interesting orderings: (+1) (-3,+4,+1) (+5) (-7,+8,+5) + ├── key: (6) + ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (6)-->(7-9), (8,9)~~>(6,7), (1)==(6), (6)==(1) + ├── prune: (2-4,7-9) + ├── interesting orderings: (+1) (-3,+4,+1) (+6) (-8,+9,+6) ├── scan xysd │ ├── columns: xysd.x:1(int!null) xysd.y:2(int) xysd.s:3(string) xysd.d:4(decimal!null) │ ├── key: (1) │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) │ ├── prune: (1-4) │ ├── interesting orderings: (+1) (-3,+4,+1) - │ └── unfiltered-cols: (1-4) + │ └── unfiltered-cols: (1-5) ├── scan a - │ ├── columns: a.x:5(int!null) a.y:6(int) a.s:7(string) a.d:8(decimal!null) - │ ├── key: (5) - │ ├── fd: (5)-->(6-8), (7,8)~~>(5,6) - │ ├── prune: (5-8) - │ ├── interesting orderings: (+5) (-7,+8,+5) - │ └── unfiltered-cols: (5-8) + │ ├── columns: a.x:6(int!null) a.y:7(int) a.s:8(string) a.d:9(decimal!null) + │ ├── key: (6) + │ ├── fd: (6)-->(7-9), (8,9)~~>(6,7) + │ ├── prune: (6-9) + │ ├── interesting orderings: (+6) (-8,+9,+6) + │ └── unfiltered-cols: (6-10) └── filters - └── eq [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + └── eq [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] ├── variable: xysd.x:1 [type=int] - └── variable: a.x:5 [type=int] + └── variable: a.x:6 [type=int] # Case with duplicated referenced columns. norm @@ -1900,38 +2003,38 @@ SELECT * FROM fk INNER JOIN (SELECT * FROM xysd FULL JOIN (VALUES (1), (2)) ON False) ON r1 = x ---- inner-join (hash) - ├── columns: k:1(int!null) v:2(int) r1:3(int!null) r2:4(int) x:5(int!null) y:6(int) s:7(string) d:8(decimal) column1:9(int) + ├── columns: k:1(int!null) v:2(int) r1:3(int!null) r2:4(int) x:6(int!null) y:7(int) s:8(string) d:9(decimal) column1:11(int) ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-more) - ├── fd: (1)-->(2-4), (5)-->(6-8), (7,8)~~>(5,6), (3)==(5), (5)==(3) - ├── prune: (1,2,4,6-9) - ├── reject-nulls: (5-9) - ├── interesting orderings: (+1) (+3,+1) (+4,+1) (+5) (-7,+8,+5) + ├── fd: (1)-->(2-4), (6)-->(7-9), (8,9)~~>(6,7), (3)==(6), (6)==(3) + ├── prune: (1,2,4,7-9,11) + ├── reject-nulls: (6-9,11) + ├── interesting orderings: (+1) (+3,+1) (+4,+1) (+6) (-8,+9,+6) ├── scan fk │ ├── columns: k:1(int!null) v:2(int) r1:3(int!null) r2:4(int) │ ├── key: (1) │ ├── fd: (1)-->(2-4) │ ├── prune: (1-4) │ ├── interesting orderings: (+1) (+3,+1) (+4,+1) - │ └── unfiltered-cols: (1-4) + │ └── unfiltered-cols: (1-5) ├── full-join (cross) - │ ├── columns: x:5(int) y:6(int) s:7(string) d:8(decimal) column1:9(int) + │ ├── columns: x:6(int) y:7(int) s:8(string) d:9(decimal) column1:11(int) │ ├── cardinality: [2 - ] - │ ├── fd: (5)-->(6-8), (7,8)~~>(5,6) - │ ├── prune: (5-9) - │ ├── reject-nulls: (5-9) - │ ├── interesting orderings: (+5) (-7,+8,+5) - │ ├── unfiltered-cols: (5-8) + │ ├── fd: (6)-->(7-9), (8,9)~~>(6,7) + │ ├── prune: (6-9,11) + │ ├── reject-nulls: (6-9,11) + │ ├── interesting orderings: (+6) (-8,+9,+6) + │ ├── unfiltered-cols: (6-10) │ ├── scan xysd - │ │ ├── columns: x:5(int!null) y:6(int) s:7(string) d:8(decimal!null) - │ │ ├── key: (5) - │ │ ├── fd: (5)-->(6-8), (7,8)~~>(5,6) - │ │ ├── prune: (5-8) - │ │ ├── interesting orderings: (+5) (-7,+8,+5) - │ │ └── unfiltered-cols: (5-8) + │ │ ├── columns: x:6(int!null) y:7(int) s:8(string) d:9(decimal!null) + │ │ ├── key: (6) + │ │ ├── fd: (6)-->(7-9), (8,9)~~>(6,7) + │ │ ├── prune: (6-9) + │ │ ├── interesting orderings: (+6) (-8,+9,+6) + │ │ └── unfiltered-cols: (6-10) │ ├── values - │ │ ├── columns: column1:9(int!null) + │ │ ├── columns: column1:11(int!null) │ │ ├── cardinality: [2 - 2] - │ │ ├── prune: (9) + │ │ ├── prune: (11) │ │ ├── tuple [type=tuple{int}] │ │ │ └── const: 1 [type=int] │ │ └── tuple [type=tuple{int}] @@ -1939,9 +2042,9 @@ inner-join (hash) │ └── filters │ └── false [type=bool] └── filters - └── eq [type=bool, outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ]), fd=(3)==(5), (5)==(3)] + └── eq [type=bool, outer=(3,6), constraints=(/3: (/NULL - ]; /6: (/NULL - ]), fd=(3)==(6), (6)==(3)] ├── variable: r1:3 [type=int] - └── variable: x:5 [type=int] + └── variable: x:6 [type=int] # Case with a self-join in the input of an InnerJoin. norm @@ -1949,12 +2052,12 @@ SELECT * FROM fk INNER JOIN (SELECT * FROM xysd INNER JOIN xysd AS a ON xysd.x = a.x) f(x) ON r1 = f.x ---- inner-join (hash) - ├── columns: k:1(int!null) v:2(int) r1:3(int!null) r2:4(int) x:5(int!null) y:6(int) s:7(string) d:8(decimal!null) x:9(int!null) y:10(int) s:11(string) d:12(decimal!null) + ├── columns: k:1(int!null) v:2(int) r1:3(int!null) r2:4(int) x:6(int!null) y:7(int) s:8(string) d:9(decimal!null) x:11(int!null) y:12(int) s:13(string) d:14(decimal!null) ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) ├── key: (1) - ├── fd: (1)-->(2-4), (5)-->(6-8), (7,8)~~>(5,6), (9)-->(10-12), (11,12)~~>(9,10), (5)==(3,9), (9)==(3,5), (3)==(5,9) - ├── prune: (1,2,4,6-8,10-12) - ├── interesting orderings: (+1) (+3,+1) (+4,+1) (+5) (-7,+8,+5) (+9) (-11,+12,+9) + ├── fd: (1)-->(2-4), (6)-->(7-9), (8,9)~~>(6,7), (11)-->(12-14), (13,14)~~>(11,12), (6)==(3,11), (11)==(3,6), (3)==(6,11) + ├── prune: (1,2,4,7-9,12-14) + ├── interesting orderings: (+1) (+3,+1) (+4,+1) (+6) (-8,+9,+6) (+11) (-13,+14,+11) ├── join-size: 2 ├── scan fk │ ├── columns: k:1(int!null) v:2(int) r1:3(int!null) r2:4(int) @@ -1962,49 +2065,49 @@ inner-join (hash) │ ├── fd: (1)-->(2-4) │ ├── prune: (1-4) │ ├── interesting orderings: (+1) (+3,+1) (+4,+1) - │ └── unfiltered-cols: (1-4) + │ └── unfiltered-cols: (1-5) ├── inner-join (hash) - │ ├── columns: xysd.x:5(int!null) xysd.y:6(int) xysd.s:7(string) xysd.d:8(decimal!null) a.x:9(int!null) a.y:10(int) a.s:11(string) a.d:12(decimal!null) + │ ├── columns: xysd.x:6(int!null) xysd.y:7(int) xysd.s:8(string) xysd.d:9(decimal!null) a.x:11(int!null) a.y:12(int) a.s:13(string) a.d:14(decimal!null) │ ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) - │ ├── key: (9) - │ ├── fd: (5)-->(6-8), (7,8)~~>(5,6), (9)-->(10-12), (11,12)~~>(9,10), (5)==(9), (9)==(5) - │ ├── prune: (6-8,10-12) - │ ├── interesting orderings: (+5) (-7,+8,+5) (+9) (-11,+12,+9) - │ ├── unfiltered-cols: (5-12) + │ ├── key: (11) + │ ├── fd: (6)-->(7-9), (8,9)~~>(6,7), (11)-->(12-14), (13,14)~~>(11,12), (6)==(11), (11)==(6) + │ ├── prune: (7-9,12-14) + │ ├── interesting orderings: (+6) (-8,+9,+6) (+11) (-13,+14,+11) + │ ├── unfiltered-cols: (6-15) │ ├── scan xysd - │ │ ├── columns: xysd.x:5(int!null) xysd.y:6(int) xysd.s:7(string) xysd.d:8(decimal!null) - │ │ ├── key: (5) - │ │ ├── fd: (5)-->(6-8), (7,8)~~>(5,6) - │ │ ├── prune: (5-8) - │ │ ├── interesting orderings: (+5) (-7,+8,+5) - │ │ └── unfiltered-cols: (5-8) + │ │ ├── columns: xysd.x:6(int!null) xysd.y:7(int) xysd.s:8(string) xysd.d:9(decimal!null) + │ │ ├── key: (6) + │ │ ├── fd: (6)-->(7-9), (8,9)~~>(6,7) + │ │ ├── prune: (6-9) + │ │ ├── interesting orderings: (+6) (-8,+9,+6) + │ │ └── unfiltered-cols: (6-10) │ ├── scan a - │ │ ├── columns: a.x:9(int!null) a.y:10(int) a.s:11(string) a.d:12(decimal!null) - │ │ ├── key: (9) - │ │ ├── fd: (9)-->(10-12), (11,12)~~>(9,10) - │ │ ├── prune: (9-12) - │ │ ├── interesting orderings: (+9) (-11,+12,+9) - │ │ └── unfiltered-cols: (9-12) + │ │ ├── columns: a.x:11(int!null) a.y:12(int) a.s:13(string) a.d:14(decimal!null) + │ │ ├── key: (11) + │ │ ├── fd: (11)-->(12-14), (13,14)~~>(11,12) + │ │ ├── prune: (11-14) + │ │ ├── interesting orderings: (+11) (-13,+14,+11) + │ │ └── unfiltered-cols: (11-15) │ └── filters - │ └── eq [type=bool, outer=(5,9), constraints=(/5: (/NULL - ]; /9: (/NULL - ]), fd=(5)==(9), (9)==(5)] - │ ├── variable: xysd.x:5 [type=int] - │ └── variable: a.x:9 [type=int] + │ └── eq [type=bool, outer=(6,11), constraints=(/6: (/NULL - ]; /11: (/NULL - ]), fd=(6)==(11), (11)==(6)] + │ ├── variable: xysd.x:6 [type=int] + │ └── variable: a.x:11 [type=int] └── filters - └── eq [type=bool, outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ]), fd=(3)==(5), (5)==(3)] + └── eq [type=bool, outer=(3,6), constraints=(/3: (/NULL - ]; /6: (/NULL - ]), fd=(3)==(6), (6)==(3)] ├── variable: r1:3 [type=int] - └── variable: xysd.x:5 [type=int] + └── variable: xysd.x:6 [type=int] norm SELECT * FROM fk INNER JOIN (SELECT xysd.x, a.x AS t FROM xysd INNER JOIN xysd AS a ON xysd.x = a.x) ON r1 = t ---- inner-join (hash) - ├── columns: k:1(int!null) v:2(int) r1:3(int!null) r2:4(int) x:5(int!null) t:9(int!null) + ├── columns: k:1(int!null) v:2(int) r1:3(int!null) r2:4(int) x:6(int!null) t:11(int!null) ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) ├── key: (1) - ├── fd: (1)-->(2-4), (5)==(3,9), (9)==(3,5), (3)==(5,9) + ├── fd: (1)-->(2-4), (6)==(3,11), (11)==(3,6), (3)==(6,11) ├── prune: (1,2,4) - ├── interesting orderings: (+1) (+3,+1) (+4,+1) (+5) (+9) + ├── interesting orderings: (+1) (+3,+1) (+4,+1) (+6) (+11) ├── join-size: 2 ├── scan fk │ ├── columns: k:1(int!null) v:2(int) r1:3(int!null) r2:4(int) @@ -2012,149 +2115,149 @@ inner-join (hash) │ ├── fd: (1)-->(2-4) │ ├── prune: (1-4) │ ├── interesting orderings: (+1) (+3,+1) (+4,+1) - │ └── unfiltered-cols: (1-4) + │ └── unfiltered-cols: (1-5) ├── inner-join (hash) - │ ├── columns: xysd.x:5(int!null) a.x:9(int!null) + │ ├── columns: xysd.x:6(int!null) a.x:11(int!null) │ ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) - │ ├── key: (9) - │ ├── fd: (5)==(9), (9)==(5) - │ ├── interesting orderings: (+5) (+9) - │ ├── unfiltered-cols: (5-12) + │ ├── key: (11) + │ ├── fd: (6)==(11), (11)==(6) + │ ├── interesting orderings: (+6) (+11) + │ ├── unfiltered-cols: (6-15) │ ├── scan xysd - │ │ ├── columns: xysd.x:5(int!null) - │ │ ├── key: (5) - │ │ ├── prune: (5) - │ │ ├── interesting orderings: (+5) - │ │ └── unfiltered-cols: (5-8) + │ │ ├── columns: xysd.x:6(int!null) + │ │ ├── key: (6) + │ │ ├── prune: (6) + │ │ ├── interesting orderings: (+6) + │ │ └── unfiltered-cols: (6-10) │ ├── scan a - │ │ ├── columns: a.x:9(int!null) - │ │ ├── key: (9) - │ │ ├── prune: (9) - │ │ ├── interesting orderings: (+9) - │ │ └── unfiltered-cols: (9-12) + │ │ ├── columns: a.x:11(int!null) + │ │ ├── key: (11) + │ │ ├── prune: (11) + │ │ ├── interesting orderings: (+11) + │ │ └── unfiltered-cols: (11-15) │ └── filters - │ └── eq [type=bool, outer=(5,9), constraints=(/5: (/NULL - ]; /9: (/NULL - ]), fd=(5)==(9), (9)==(5)] - │ ├── variable: xysd.x:5 [type=int] - │ └── variable: a.x:9 [type=int] + │ └── eq [type=bool, outer=(6,11), constraints=(/6: (/NULL - ]; /11: (/NULL - ]), fd=(6)==(11), (11)==(6)] + │ ├── variable: xysd.x:6 [type=int] + │ └── variable: a.x:11 [type=int] └── filters - └── eq [type=bool, outer=(3,9), constraints=(/3: (/NULL - ]; /9: (/NULL - ]), fd=(3)==(9), (9)==(3)] + └── eq [type=bool, outer=(3,11), constraints=(/3: (/NULL - ]; /11: (/NULL - ]), fd=(3)==(11), (11)==(3)] ├── variable: r1:3 [type=int] - └── variable: a.x:9 [type=int] + └── variable: a.x:11 [type=int] # Case with an equality with a synthesized column. norm SELECT * FROM mn LEFT JOIN xysd ON y = (n * 2) ---- project - ├── columns: m:1(int!null) n:2(int) x:3(int) y:4(int) s:5(string) d:6(decimal) + ├── columns: m:1(int!null) n:2(int) x:4(int) y:5(int) s:6(string) d:7(decimal) ├── immutable - ├── key: (1,3) - ├── fd: (1)-->(2), (2)~~>(1), (3)-->(4-6), (5,6)~~>(3,4) - ├── prune: (1-6) - ├── reject-nulls: (3-6) - ├── interesting orderings: (+1) (+2,+1) (+3) (-5,+6,+3) + ├── key: (1,4) + ├── fd: (1)-->(2), (2)~~>(1), (4)-->(5-7), (6,7)~~>(4,5) + ├── prune: (1,2,4-7) + ├── reject-nulls: (4-7) + ├── interesting orderings: (+1) (+2,+1) (+4) (-6,+7,+4) └── left-join (hash) - ├── columns: m:1(int!null) n:2(int) x:3(int) y:4(int) s:5(string) d:6(decimal) column7:7(int) + ├── columns: m:1(int!null) n:2(int) x:4(int) y:5(int) s:6(string) d:7(decimal) column9:9(int) ├── immutable - ├── key: (1,3) - ├── fd: (1)-->(2), (2)~~>(1), (2)-->(7), (3)-->(4-6), (5,6)~~>(3,4) - ├── prune: (1-3,5,6) - ├── reject-nulls: (3-6) - ├── interesting orderings: (+1) (+2,+1) (+3) (-5,+6,+3) + ├── key: (1,4) + ├── fd: (1)-->(2), (2)~~>(1), (2)-->(9), (4)-->(5-7), (6,7)~~>(4,5) + ├── prune: (1,2,4,6,7) + ├── reject-nulls: (4-7) + ├── interesting orderings: (+1) (+2,+1) (+4) (-6,+7,+4) ├── project - │ ├── columns: column7:7(int) m:1(int!null) n:2(int) + │ ├── columns: column9:9(int) m:1(int!null) n:2(int) │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(2), (2)~~>(1), (2)-->(7) - │ ├── prune: (1,2,7) + │ ├── fd: (1)-->(2), (2)~~>(1), (2)-->(9) + │ ├── prune: (1,2,9) │ ├── interesting orderings: (+1) (+2,+1) - │ ├── unfiltered-cols: (1,2) + │ ├── unfiltered-cols: (1-3) │ ├── scan mn │ │ ├── columns: m:1(int!null) n:2(int) │ │ ├── key: (1) │ │ ├── fd: (1)-->(2), (2)~~>(1) │ │ ├── prune: (1,2) │ │ ├── interesting orderings: (+1) (+2,+1) - │ │ └── unfiltered-cols: (1,2) + │ │ └── unfiltered-cols: (1-3) │ └── projections - │ └── mult [as=column7:7, type=int, outer=(2), immutable] + │ └── mult [as=column9:9, type=int, outer=(2), immutable] │ ├── variable: n:2 [type=int] │ └── const: 2 [type=int] ├── scan xysd - │ ├── columns: x:3(int!null) y:4(int) s:5(string) d:6(decimal!null) - │ ├── key: (3) - │ ├── fd: (3)-->(4-6), (5,6)~~>(3,4) - │ ├── prune: (3-6) - │ ├── interesting orderings: (+3) (-5,+6,+3) - │ └── unfiltered-cols: (3-6) + │ ├── columns: x:4(int!null) y:5(int) s:6(string) d:7(decimal!null) + │ ├── key: (4) + │ ├── fd: (4)-->(5-7), (6,7)~~>(4,5) + │ ├── prune: (4-7) + │ ├── interesting orderings: (+4) (-6,+7,+4) + │ └── unfiltered-cols: (4-8) └── filters - └── eq [type=bool, outer=(4,7), constraints=(/4: (/NULL - ]; /7: (/NULL - ]), fd=(4)==(7), (7)==(4)] - ├── variable: column7:7 [type=int] - └── variable: y:4 [type=int] + └── eq [type=bool, outer=(5,9), constraints=(/5: (/NULL - ]; /9: (/NULL - ]), fd=(5)==(9), (9)==(5)] + ├── variable: column9:9 [type=int] + └── variable: y:5 [type=int] # Case with columns that don't come from base tables. norm SELECT * FROM (SELECT * FROM uv UNION (SELECT * FROM uv)) f(v1, v2) INNER JOIN xysd ON v2 = x ---- inner-join (hash) - ├── columns: v1:7(int) v2:8(int!null) x:9(int!null) y:10(int) s:11(string) d:12(decimal!null) + ├── columns: v1:9(int) v2:10(int!null) x:11(int!null) y:12(int) s:13(string) d:14(decimal!null) ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - ├── key: (7,9) - ├── fd: (9)-->(10-12), (11,12)~~>(9,10), (8)==(9), (9)==(8) - ├── prune: (10-12) - ├── interesting orderings: (+9) (-11,+12,+9) + ├── key: (9,11) + ├── fd: (11)-->(12-14), (13,14)~~>(11,12), (10)==(11), (11)==(10) + ├── prune: (12-14) + ├── interesting orderings: (+11) (-13,+14,+11) ├── union - │ ├── columns: u:7(int) v:8(int!null) + │ ├── columns: u:9(int) v:10(int!null) │ ├── left columns: uv.u:1(int) uv.v:2(int) - │ ├── right columns: uv.u:4(int) uv.v:5(int) - │ ├── key: (7,8) + │ ├── right columns: uv.u:5(int) uv.v:6(int) + │ ├── key: (9,10) │ ├── scan uv │ │ ├── columns: uv.u:1(int) uv.v:2(int!null) │ │ └── prune: (1,2) │ └── scan uv - │ ├── columns: uv.u:4(int) uv.v:5(int!null) - │ └── prune: (4,5) + │ ├── columns: uv.u:5(int) uv.v:6(int!null) + │ └── prune: (5,6) ├── scan xysd - │ ├── columns: x:9(int!null) y:10(int) s:11(string) d:12(decimal!null) - │ ├── key: (9) - │ ├── fd: (9)-->(10-12), (11,12)~~>(9,10) - │ ├── prune: (9-12) - │ ├── interesting orderings: (+9) (-11,+12,+9) - │ └── unfiltered-cols: (9-12) + │ ├── columns: x:11(int!null) y:12(int) s:13(string) d:14(decimal!null) + │ ├── key: (11) + │ ├── fd: (11)-->(12-14), (13,14)~~>(11,12) + │ ├── prune: (11-14) + │ ├── interesting orderings: (+11) (-13,+14,+11) + │ └── unfiltered-cols: (11-15) └── filters - └── eq [type=bool, outer=(8,9), constraints=(/8: (/NULL - ]; /9: (/NULL - ]), fd=(8)==(9), (9)==(8)] - ├── variable: v:8 [type=int] - └── variable: x:9 [type=int] + └── eq [type=bool, outer=(10,11), constraints=(/10: (/NULL - ]; /11: (/NULL - ]), fd=(10)==(11), (11)==(10)] + ├── variable: v:10 [type=int] + └── variable: x:11 [type=int] # Self-join case with different columns. norm SELECT * FROM xysd INNER JOIN xysd AS a ON xysd.x = a.y ---- inner-join (hash) - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) x:5(int!null) y:6(int!null) s:7(string) d:8(decimal!null) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) x:6(int!null) y:7(int!null) s:8(string) d:9(decimal!null) ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) - ├── key: (5) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (5)-->(6-8), (7,8)~~>(5,6), (1)==(6), (6)==(1) - ├── prune: (2-5,7,8) - ├── interesting orderings: (+1) (-3,+4,+1) (+5) (-7,+8,+5) + ├── key: (6) + ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (6)-->(7-9), (8,9)~~>(6,7), (1)==(7), (7)==(1) + ├── prune: (2-4,6,8,9) + ├── interesting orderings: (+1) (-3,+4,+1) (+6) (-8,+9,+6) ├── scan xysd │ ├── columns: xysd.x:1(int!null) xysd.y:2(int) xysd.s:3(string) xysd.d:4(decimal!null) │ ├── key: (1) │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) │ ├── prune: (1-4) │ ├── interesting orderings: (+1) (-3,+4,+1) - │ └── unfiltered-cols: (1-4) + │ └── unfiltered-cols: (1-5) ├── scan a - │ ├── columns: a.x:5(int!null) a.y:6(int) a.s:7(string) a.d:8(decimal!null) - │ ├── key: (5) - │ ├── fd: (5)-->(6-8), (7,8)~~>(5,6) - │ ├── prune: (5-8) - │ ├── interesting orderings: (+5) (-7,+8,+5) - │ └── unfiltered-cols: (5-8) + │ ├── columns: a.x:6(int!null) a.y:7(int) a.s:8(string) a.d:9(decimal!null) + │ ├── key: (6) + │ ├── fd: (6)-->(7-9), (8,9)~~>(6,7) + │ ├── prune: (6-9) + │ ├── interesting orderings: (+6) (-8,+9,+6) + │ └── unfiltered-cols: (6-10) └── filters - └── eq [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── eq [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] ├── variable: xysd.x:1 [type=int] - └── variable: a.y:6 [type=int] + └── variable: a.y:7 [type=int] # Case with an equality between a not-null foreign key and an unreferenced # column. @@ -2162,29 +2265,29 @@ norm SELECT * FROM fk INNER JOIN xysd ON r1 = y ---- inner-join (hash) - ├── columns: k:1(int!null) v:2(int) r1:3(int!null) r2:4(int) x:5(int!null) y:6(int!null) s:7(string) d:8(decimal!null) - ├── key: (1,5) - ├── fd: (1)-->(2-4), (5)-->(6-8), (7,8)~~>(5,6), (3)==(6), (6)==(3) - ├── prune: (1,2,4,5,7,8) - ├── interesting orderings: (+1) (+3,+1) (+4,+1) (+5) (-7,+8,+5) + ├── columns: k:1(int!null) v:2(int) r1:3(int!null) r2:4(int) x:6(int!null) y:7(int!null) s:8(string) d:9(decimal!null) + ├── key: (1,6) + ├── fd: (1)-->(2-4), (6)-->(7-9), (8,9)~~>(6,7), (3)==(7), (7)==(3) + ├── prune: (1,2,4,6,8,9) + ├── interesting orderings: (+1) (+3,+1) (+4,+1) (+6) (-8,+9,+6) ├── scan fk │ ├── columns: k:1(int!null) v:2(int) r1:3(int!null) r2:4(int) │ ├── key: (1) │ ├── fd: (1)-->(2-4) │ ├── prune: (1-4) │ ├── interesting orderings: (+1) (+3,+1) (+4,+1) - │ └── unfiltered-cols: (1-4) + │ └── unfiltered-cols: (1-5) ├── scan xysd - │ ├── columns: x:5(int!null) y:6(int) s:7(string) d:8(decimal!null) - │ ├── key: (5) - │ ├── fd: (5)-->(6-8), (7,8)~~>(5,6) - │ ├── prune: (5-8) - │ ├── interesting orderings: (+5) (-7,+8,+5) - │ └── unfiltered-cols: (5-8) + │ ├── columns: x:6(int!null) y:7(int) s:8(string) d:9(decimal!null) + │ ├── key: (6) + │ ├── fd: (6)-->(7-9), (8,9)~~>(6,7) + │ ├── prune: (6-9) + │ ├── interesting orderings: (+6) (-8,+9,+6) + │ └── unfiltered-cols: (6-10) └── filters - └── eq [type=bool, outer=(3,6), constraints=(/3: (/NULL - ]; /6: (/NULL - ]), fd=(3)==(6), (6)==(3)] + └── eq [type=bool, outer=(3,7), constraints=(/3: (/NULL - ]; /7: (/NULL - ]), fd=(3)==(7), (7)==(3)] ├── variable: r1:3 [type=int] - └── variable: y:6 [type=int] + └── variable: y:7 [type=int] # Case where left table has a foreign key that references a table that isn't # from the right input. @@ -2192,30 +2295,30 @@ norm SELECT * FROM fk INNER JOIN mn ON k = m ---- inner-join (hash) - ├── columns: k:1(int!null) v:2(int) r1:3(int!null) r2:4(int) m:5(int!null) n:6(int) + ├── columns: k:1(int!null) v:2(int) r1:3(int!null) r2:4(int) m:6(int!null) n:7(int) ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (5) - ├── fd: (1)-->(2-4), (5)-->(6), (6)~~>(5), (1)==(5), (5)==(1) - ├── prune: (2-4,6) - ├── interesting orderings: (+1) (+3,+1) (+4,+1) (+5) (+6,+5) + ├── key: (6) + ├── fd: (1)-->(2-4), (6)-->(7), (7)~~>(6), (1)==(6), (6)==(1) + ├── prune: (2-4,7) + ├── interesting orderings: (+1) (+3,+1) (+4,+1) (+6) (+7,+6) ├── scan fk │ ├── columns: k:1(int!null) v:2(int) r1:3(int!null) r2:4(int) │ ├── key: (1) │ ├── fd: (1)-->(2-4) │ ├── prune: (1-4) │ ├── interesting orderings: (+1) (+3,+1) (+4,+1) - │ └── unfiltered-cols: (1-4) + │ └── unfiltered-cols: (1-5) ├── scan mn - │ ├── columns: m:5(int!null) n:6(int) - │ ├── key: (5) - │ ├── fd: (5)-->(6), (6)~~>(5) - │ ├── prune: (5,6) - │ ├── interesting orderings: (+5) (+6,+5) - │ └── unfiltered-cols: (5,6) + │ ├── columns: m:6(int!null) n:7(int) + │ ├── key: (6) + │ ├── fd: (6)-->(7), (7)~~>(6) + │ ├── prune: (6,7) + │ ├── interesting orderings: (+6) (+7,+6) + │ └── unfiltered-cols: (6-8) └── filters - └── eq [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + └── eq [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] ├── variable: k:1 [type=int] - └── variable: m:5 [type=int] + └── variable: m:6 [type=int] # Case with a match-simple foreign key with one nullable column. norm @@ -2225,31 +2328,31 @@ INNER JOIN abc ON (r1, r2, r3) = (a, b, c) ---- inner-join (hash) - ├── columns: r1:1(int!null) r2:2(int!null) r3:3(int!null) a:5(int!null) b:6(int!null) c:7(int!null) + ├── columns: r1:1(int!null) r2:2(int!null) r3:3(int!null) a:6(int!null) b:7(int!null) c:8(int!null) ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2), (3)==(7), (7)==(3) - ├── interesting orderings: (+1,+2,+3) (+5,+6,+7) + ├── fd: (1)==(6), (6)==(1), (2)==(7), (7)==(2), (3)==(8), (8)==(3) + ├── interesting orderings: (+1,+2,+3) (+6,+7,+8) ├── scan ref │ ├── columns: r1:1(int!null) r2:2(int) r3:3(int!null) │ ├── prune: (1-3) │ ├── interesting orderings: (+1,+2,+3) - │ └── unfiltered-cols: (1-4) + │ └── unfiltered-cols: (1-5) ├── scan abc - │ ├── columns: a:5(int!null) b:6(int!null) c:7(int!null) - │ ├── key: (5-7) - │ ├── prune: (5-7) - │ ├── interesting orderings: (+5,+6,+7) - │ └── unfiltered-cols: (5-7) + │ ├── columns: a:6(int!null) b:7(int!null) c:8(int!null) + │ ├── key: (6-8) + │ ├── prune: (6-8) + │ ├── interesting orderings: (+6,+7,+8) + │ └── unfiltered-cols: (6-9) └── filters - ├── eq [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + ├── eq [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] │ ├── variable: r1:1 [type=int] - │ └── variable: a:5 [type=int] - ├── eq [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + │ └── variable: a:6 [type=int] + ├── eq [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] │ ├── variable: r2:2 [type=int] - │ └── variable: b:6 [type=int] - └── eq [type=bool, outer=(3,7), constraints=(/3: (/NULL - ]; /7: (/NULL - ]), fd=(3)==(7), (7)==(3)] + │ └── variable: b:7 [type=int] + └── eq [type=bool, outer=(3,8), constraints=(/3: (/NULL - ]; /8: (/NULL - ]), fd=(3)==(8), (8)==(3)] ├── variable: r3:3 [type=int] - └── variable: c:7 [type=int] + └── variable: c:8 [type=int] # Case with a not-null multi-column foreign key. norm @@ -2259,10 +2362,10 @@ INNER JOIN abc ON (r1, r2, r3) = (a, b, c) ---- inner-join (hash) - ├── columns: r1:1(int!null) r2:2(int!null) r3:3(int!null) a:5(int!null) b:6(int!null) c:7(int!null) + ├── columns: r1:1(int!null) r2:2(int!null) r3:3(int!null) a:6(int!null) b:7(int!null) c:8(int!null) ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2), (3)==(7), (7)==(3) - ├── interesting orderings: (+1,+2,+3) (+5,+6,+7) + ├── fd: (1)==(6), (6)==(1), (2)==(7), (7)==(2), (3)==(8), (8)==(3) + ├── interesting orderings: (+1,+2,+3) (+6,+7,+8) ├── select │ ├── columns: r1:1(int!null) r2:2(int!null) r3:3(int!null) │ ├── prune: (1,3) @@ -2276,21 +2379,21 @@ inner-join (hash) │ ├── variable: r2:2 [type=int] │ └── null [type=unknown] ├── scan abc - │ ├── columns: a:5(int!null) b:6(int!null) c:7(int!null) - │ ├── key: (5-7) - │ ├── prune: (5-7) - │ ├── interesting orderings: (+5,+6,+7) - │ └── unfiltered-cols: (5-7) + │ ├── columns: a:6(int!null) b:7(int!null) c:8(int!null) + │ ├── key: (6-8) + │ ├── prune: (6-8) + │ ├── interesting orderings: (+6,+7,+8) + │ └── unfiltered-cols: (6-9) └── filters - ├── eq [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + ├── eq [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] │ ├── variable: r1:1 [type=int] - │ └── variable: a:5 [type=int] - ├── eq [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + │ └── variable: a:6 [type=int] + ├── eq [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] │ ├── variable: r2:2 [type=int] - │ └── variable: b:6 [type=int] - └── eq [type=bool, outer=(3,7), constraints=(/3: (/NULL - ]; /7: (/NULL - ]), fd=(3)==(7), (7)==(3)] + │ └── variable: b:7 [type=int] + └── eq [type=bool, outer=(3,8), constraints=(/3: (/NULL - ]; /8: (/NULL - ]), fd=(3)==(8), (8)==(3)] ├── variable: r3:3 [type=int] - └── variable: c:7 [type=int] + └── variable: c:8 [type=int] # Case with a not-null multi-column foreign key and an equality on only one of # the foreign key columns. @@ -2301,11 +2404,11 @@ INNER JOIN abc ON r2 = b ---- inner-join (hash) - ├── columns: r2:2(int!null) a:5(int!null) b:6(int!null) c:7(int!null) + ├── columns: r2:2(int!null) a:6(int!null) b:7(int!null) c:8(int!null) ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-more) - ├── fd: (2)==(6), (6)==(2) - ├── prune: (5,7) - ├── interesting orderings: (+5,+6,+7) + ├── fd: (2)==(7), (7)==(2) + ├── prune: (6,8) + ├── interesting orderings: (+6,+7,+8) ├── select │ ├── columns: r2:2(int!null) │ ├── scan ref @@ -2316,15 +2419,15 @@ inner-join (hash) │ ├── variable: r2:2 [type=int] │ └── null [type=unknown] ├── scan abc - │ ├── columns: a:5(int!null) b:6(int!null) c:7(int!null) - │ ├── key: (5-7) - │ ├── prune: (5-7) - │ ├── interesting orderings: (+5,+6,+7) - │ └── unfiltered-cols: (5-7) + │ ├── columns: a:6(int!null) b:7(int!null) c:8(int!null) + │ ├── key: (6-8) + │ ├── prune: (6-8) + │ ├── interesting orderings: (+6,+7,+8) + │ └── unfiltered-cols: (6-9) └── filters - └── eq [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + └── eq [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] ├── variable: r2:2 [type=int] - └── variable: b:6 [type=int] + └── variable: b:7 [type=int] exec-ddl CREATE TABLE trade_type (tt_id INT PRIMARY KEY) @@ -2370,87 +2473,87 @@ WHERE st_id = t_st_id LIMIT 50; ---- limit - ├── columns: t_st_id:1(int!null) t_tt_id:2(int!null) t_s_symb:3(int!null) st_id:5(int!null) tt_id:6(int!null) s_symb:7(int!null) s_st_id:8(int!null) s_ex_id:9(int!null) ex_id:10(int!null) + ├── columns: t_st_id:1(int!null) t_tt_id:2(int!null) t_s_symb:3(int!null) st_id:6(int!null) tt_id:8(int!null) s_symb:10(int!null) s_st_id:11(int!null) s_ex_id:12(int!null) ex_id:14(int!null) ├── cardinality: [0 - 50] - ├── fd: (7)-->(8,9), (9)==(10), (10)==(9), (1)==(5), (5)==(1), (2)==(6), (6)==(2), (3)==(7), (7)==(3) - ├── prune: (8) - ├── interesting orderings: (+1) (+2) (+3) (+5) (+6) (+7) (+8,+7) (+9,+7) (+10) + ├── fd: (10)-->(11,12), (12)==(14), (14)==(12), (1)==(6), (6)==(1), (2)==(8), (8)==(2), (3)==(10), (10)==(3) + ├── prune: (11) + ├── interesting orderings: (+1) (+2) (+3) (+6) (+8) (+10) (+11,+10) (+12,+10) (+14) ├── inner-join (hash) - │ ├── columns: t_st_id:1(int!null) t_tt_id:2(int!null) t_s_symb:3(int!null) st_id:5(int!null) tt_id:6(int!null) s_symb:7(int!null) s_st_id:8(int!null) s_ex_id:9(int!null) ex_id:10(int!null) + │ ├── columns: t_st_id:1(int!null) t_tt_id:2(int!null) t_s_symb:3(int!null) st_id:6(int!null) tt_id:8(int!null) s_symb:10(int!null) s_st_id:11(int!null) s_ex_id:12(int!null) ex_id:14(int!null) │ ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) - │ ├── fd: (7)-->(8,9), (9)==(10), (10)==(9), (1)==(5), (5)==(1), (2)==(6), (6)==(2), (3)==(7), (7)==(3) + │ ├── fd: (10)-->(11,12), (12)==(14), (14)==(12), (1)==(6), (6)==(1), (2)==(8), (8)==(2), (3)==(10), (10)==(3) │ ├── limit hint: 50.00 - │ ├── prune: (8) - │ ├── interesting orderings: (+1) (+2) (+3) (+5) (+6) (+7) (+8,+7) (+9,+7) (+10) + │ ├── prune: (11) + │ ├── interesting orderings: (+1) (+2) (+3) (+6) (+8) (+10) (+11,+10) (+12,+10) (+14) │ ├── join-size: 4 │ ├── inner-join (cross) - │ │ ├── columns: st_id:5(int!null) tt_id:6(int!null) s_symb:7(int!null) s_st_id:8(int!null) s_ex_id:9(int!null) ex_id:10(int!null) + │ │ ├── columns: st_id:6(int!null) tt_id:8(int!null) s_symb:10(int!null) s_st_id:11(int!null) s_ex_id:12(int!null) ex_id:14(int!null) │ │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-more) - │ │ ├── key: (5-7) - │ │ ├── fd: (7)-->(8,9), (9)==(10), (10)==(9) - │ │ ├── prune: (5-8) - │ │ ├── interesting orderings: (+5) (+6) (+7) (+8,+7) (+9,+7) (+10) + │ │ ├── key: (6,8,10) + │ │ ├── fd: (10)-->(11,12), (12)==(14), (14)==(12) + │ │ ├── prune: (6,8,10,11) + │ │ ├── interesting orderings: (+6) (+8) (+10) (+11,+10) (+12,+10) (+14) │ │ ├── join-size: 3 │ │ ├── inner-join (cross) - │ │ │ ├── columns: tt_id:6(int!null) s_symb:7(int!null) s_st_id:8(int!null) s_ex_id:9(int!null) ex_id:10(int!null) - │ │ │ ├── key: (6,7) - │ │ │ ├── fd: (7)-->(8,9), (9)==(10), (10)==(9) - │ │ │ ├── prune: (6-8) - │ │ │ ├── interesting orderings: (+6) (+7) (+8,+7) (+9,+7) (+10) + │ │ │ ├── columns: tt_id:8(int!null) s_symb:10(int!null) s_st_id:11(int!null) s_ex_id:12(int!null) ex_id:14(int!null) + │ │ │ ├── key: (8,10) + │ │ │ ├── fd: (10)-->(11,12), (12)==(14), (14)==(12) + │ │ │ ├── prune: (8,10,11) + │ │ │ ├── interesting orderings: (+8) (+10) (+11,+10) (+12,+10) (+14) │ │ │ ├── join-size: 2 │ │ │ ├── scan trade_type - │ │ │ │ ├── columns: tt_id:6(int!null) - │ │ │ │ ├── key: (6) - │ │ │ │ ├── prune: (6) - │ │ │ │ ├── interesting orderings: (+6) - │ │ │ │ └── unfiltered-cols: (6) + │ │ │ │ ├── columns: tt_id:8(int!null) + │ │ │ │ ├── key: (8) + │ │ │ │ ├── prune: (8) + │ │ │ │ ├── interesting orderings: (+8) + │ │ │ │ └── unfiltered-cols: (8,9) │ │ │ ├── inner-join (hash) - │ │ │ │ ├── columns: s_symb:7(int!null) s_st_id:8(int!null) s_ex_id:9(int!null) ex_id:10(int!null) + │ │ │ │ ├── columns: s_symb:10(int!null) s_st_id:11(int!null) s_ex_id:12(int!null) ex_id:14(int!null) │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ │ │ │ ├── key: (7) - │ │ │ │ ├── fd: (7)-->(8,9), (9)==(10), (10)==(9) - │ │ │ │ ├── prune: (7,8) - │ │ │ │ ├── interesting orderings: (+7) (+8,+7) (+9,+7) (+10) - │ │ │ │ ├── unfiltered-cols: (7-9) + │ │ │ │ ├── key: (10) + │ │ │ │ ├── fd: (10)-->(11,12), (12)==(14), (14)==(12) + │ │ │ │ ├── prune: (10,11) + │ │ │ │ ├── interesting orderings: (+10) (+11,+10) (+12,+10) (+14) + │ │ │ │ ├── unfiltered-cols: (10-13) │ │ │ │ ├── scan security - │ │ │ │ │ ├── columns: s_symb:7(int!null) s_st_id:8(int!null) s_ex_id:9(int!null) - │ │ │ │ │ ├── key: (7) - │ │ │ │ │ ├── fd: (7)-->(8,9) - │ │ │ │ │ ├── prune: (7-9) - │ │ │ │ │ ├── interesting orderings: (+7) (+8,+7) (+9,+7) - │ │ │ │ │ └── unfiltered-cols: (7-9) - │ │ │ │ ├── scan exchange - │ │ │ │ │ ├── columns: ex_id:10(int!null) + │ │ │ │ │ ├── columns: s_symb:10(int!null) s_st_id:11(int!null) s_ex_id:12(int!null) │ │ │ │ │ ├── key: (10) - │ │ │ │ │ ├── prune: (10) - │ │ │ │ │ ├── interesting orderings: (+10) - │ │ │ │ │ └── unfiltered-cols: (10) + │ │ │ │ │ ├── fd: (10)-->(11,12) + │ │ │ │ │ ├── prune: (10-12) + │ │ │ │ │ ├── interesting orderings: (+10) (+11,+10) (+12,+10) + │ │ │ │ │ └── unfiltered-cols: (10-13) + │ │ │ │ ├── scan exchange + │ │ │ │ │ ├── columns: ex_id:14(int!null) + │ │ │ │ │ ├── key: (14) + │ │ │ │ │ ├── prune: (14) + │ │ │ │ │ ├── interesting orderings: (+14) + │ │ │ │ │ └── unfiltered-cols: (14,15) │ │ │ │ └── filters - │ │ │ │ └── eq [type=bool, outer=(9,10), constraints=(/9: (/NULL - ]; /10: (/NULL - ]), fd=(9)==(10), (10)==(9)] - │ │ │ │ ├── variable: ex_id:10 [type=int] - │ │ │ │ └── variable: s_ex_id:9 [type=int] + │ │ │ │ └── eq [type=bool, outer=(12,14), constraints=(/12: (/NULL - ]; /14: (/NULL - ]), fd=(12)==(14), (14)==(12)] + │ │ │ │ ├── variable: ex_id:14 [type=int] + │ │ │ │ └── variable: s_ex_id:12 [type=int] │ │ │ └── filters (true) │ │ ├── scan status_type - │ │ │ ├── columns: st_id:5(int!null) - │ │ │ ├── key: (5) - │ │ │ ├── prune: (5) - │ │ │ ├── interesting orderings: (+5) - │ │ │ └── unfiltered-cols: (5) + │ │ │ ├── columns: st_id:6(int!null) + │ │ │ ├── key: (6) + │ │ │ ├── prune: (6) + │ │ │ ├── interesting orderings: (+6) + │ │ │ └── unfiltered-cols: (6,7) │ │ └── filters (true) │ ├── scan trade │ │ ├── columns: t_st_id:1(int!null) t_tt_id:2(int!null) t_s_symb:3(int!null) │ │ ├── prune: (1-3) │ │ ├── interesting orderings: (+1) (+2) (+3) - │ │ └── unfiltered-cols: (1-4) + │ │ └── unfiltered-cols: (1-5) │ └── filters - │ ├── eq [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - │ │ ├── variable: st_id:5 [type=int] + │ ├── eq [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ │ ├── variable: st_id:6 [type=int] │ │ └── variable: t_st_id:1 [type=int] - │ ├── eq [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] - │ │ ├── variable: tt_id:6 [type=int] + │ ├── eq [type=bool, outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] + │ │ ├── variable: tt_id:8 [type=int] │ │ └── variable: t_tt_id:2 [type=int] - │ └── eq [type=bool, outer=(3,7), constraints=(/3: (/NULL - ]; /7: (/NULL - ]), fd=(3)==(7), (7)==(3)] - │ ├── variable: s_symb:7 [type=int] + │ └── eq [type=bool, outer=(3,10), constraints=(/3: (/NULL - ]; /10: (/NULL - ]), fd=(3)==(10), (10)==(3)] + │ ├── variable: s_symb:10 [type=int] │ └── variable: t_s_symb:3 [type=int] └── const: 50 [type=int] @@ -2471,27 +2574,27 @@ norm SELECT * FROM child LEFT JOIN parent ON r_a = a AND r_b = b ---- left-join (hash) - ├── columns: r_a:1(int!null) r_b:2(int!null) a:4(int) b:5(int) + ├── columns: r_a:1(int!null) r_b:2(int!null) a:5(int) b:6(int) ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - ├── fd: (4)-->(5), (5)-->(4) - ├── reject-nulls: (4,5) - ├── interesting orderings: (+1) (+2) (+4) (+5) + ├── fd: (5)-->(6), (6)-->(5) + ├── reject-nulls: (5,6) + ├── interesting orderings: (+1) (+2) (+5) (+6) ├── scan child │ ├── columns: r_a:1(int!null) r_b:2(int!null) │ ├── prune: (1,2) │ ├── interesting orderings: (+1) (+2) - │ └── unfiltered-cols: (1-3) + │ └── unfiltered-cols: (1-4) ├── scan parent - │ ├── columns: a:4(int!null) b:5(int!null) - │ ├── key: (5) - │ ├── fd: (4)-->(5), (5)-->(4) - │ ├── prune: (4,5) - │ ├── interesting orderings: (+4) (+5) - │ └── unfiltered-cols: (4-6) + │ ├── columns: a:5(int!null) b:6(int!null) + │ ├── key: (6) + │ ├── fd: (5)-->(6), (6)-->(5) + │ ├── prune: (5,6) + │ ├── interesting orderings: (+5) (+6) + │ └── unfiltered-cols: (5-8) └── filters - ├── eq [type=bool, outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + ├── eq [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] │ ├── variable: r_a:1 [type=int] - │ └── variable: a:4 [type=int] - └── eq [type=bool, outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ]), fd=(2)==(5), (5)==(2)] + │ └── variable: a:5 [type=int] + └── eq [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] ├── variable: r_b:2 [type=int] - └── variable: b:5 [type=int] + └── variable: b:6 [type=int] diff --git a/pkg/sql/opt/memo/testdata/logprops/limit b/pkg/sql/opt/memo/testdata/logprops/limit index ad1e3cb462dc..b61e5417cd81 100644 --- a/pkg/sql/opt/memo/testdata/logprops/limit +++ b/pkg/sql/opt/memo/testdata/logprops/limit @@ -16,40 +16,47 @@ limit ├── fd: ()-->(1-4) ├── prune: (1-4) ├── interesting orderings: (+1) (-4,+3,+1) - ├── scan xyzs + ├── project │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) │ ├── key: (1) │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) │ ├── limit hint: 1.00 │ ├── prune: (1-4) - │ └── interesting orderings: (+1) (-4,+3,+1) + │ ├── interesting orderings: (+1) (-4,+3,+1) + │ └── scan xyzs + │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) crdb_internal_mvcc_timestamp:5(decimal) + │ ├── key: (1) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── limit hint: 1.00 + │ ├── prune: (1-5) + │ └── interesting orderings: (+1) (-4,+3,+1) └── const: 1 [type=int] build SELECT count(*) FROM xyzs LIMIT 10 ---- limit - ├── columns: count:5(int!null) + ├── columns: count:6(int!null) ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(5) - ├── prune: (5) + ├── fd: ()-->(6) + ├── prune: (6) ├── scalar-group-by - │ ├── columns: count_rows:5(int!null) + │ ├── columns: count_rows:6(int!null) │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(5) + │ ├── fd: ()-->(6) │ ├── limit hint: 10.00 - │ ├── prune: (5) + │ ├── prune: (6) │ ├── project │ │ └── scan xyzs - │ │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) + │ │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) crdb_internal_mvcc_timestamp:5(decimal) │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ │ ├── prune: (1-4) + │ │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ │ ├── prune: (1-5) │ │ └── interesting orderings: (+1) (-4,+3,+1) │ └── aggregations - │ └── count-rows [as=count_rows:5, type=int] + │ └── count-rows [as=count_rows:6, type=int] └── const: 10 [type=int] build @@ -62,31 +69,37 @@ limit ├── fd: (1)-->(2-4), (3,4)~~>(1,2) ├── prune: (1-4) ├── interesting orderings: (+1) (-4,+3,+1) - ├── scan xyzs + ├── project │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) │ ├── key: (1) │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) │ ├── prune: (1-4) - │ └── interesting orderings: (+1) (-4,+3,+1) + │ ├── interesting orderings: (+1) (-4,+3,+1) + │ └── scan xyzs + │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) crdb_internal_mvcc_timestamp:5(decimal) + │ ├── key: (1) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── prune: (1-5) + │ └── interesting orderings: (+1) (-4,+3,+1) └── subquery [type=int] └── max1-row - ├── columns: "?column?":5(int!null) + ├── columns: "?column?":6(int!null) ├── error: "more than one row returned by a subquery used as an expression" ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(5) + ├── fd: ()-->(6) └── project - ├── columns: "?column?":5(int!null) + ├── columns: "?column?":6(int!null) ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(5) - ├── prune: (5) + ├── fd: ()-->(6) + ├── prune: (6) ├── values │ ├── cardinality: [1 - 1] │ ├── key: () │ └── tuple [type=tuple] └── projections - └── const: 1 [as="?column?":5, type=int] + └── const: 1 [as="?column?":6, type=int] build SELECT * FROM xyzs LIMIT 0 @@ -98,13 +111,20 @@ limit ├── fd: ()-->(1-4) ├── prune: (1-4) ├── interesting orderings: (+1) (-4,+3,+1) - ├── scan xyzs + ├── project │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) │ ├── key: (1) │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) │ ├── limit hint: 1.00 │ ├── prune: (1-4) - │ └── interesting orderings: (+1) (-4,+3,+1) + │ ├── interesting orderings: (+1) (-4,+3,+1) + │ └── scan xyzs + │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) crdb_internal_mvcc_timestamp:5(decimal) + │ ├── key: (1) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── limit hint: 1.00 + │ ├── prune: (1-5) + │ └── interesting orderings: (+1) (-4,+3,+1) └── const: 0 [type=int] # Propagate outer columns. @@ -112,44 +132,44 @@ build SELECT (SELECT x FROM kuv LIMIT y) FROM xyzs ---- project - ├── columns: x:9(int) + ├── columns: x:11(int) ├── immutable - ├── prune: (9) + ├── prune: (11) ├── scan xyzs - │ ├── columns: xyzs.x:1(int!null) y:2(int) z:3(float!null) s:4(string) + │ ├── columns: xyzs.x:1(int!null) y:2(int) z:3(float!null) s:4(string) xyzs.crdb_internal_mvcc_timestamp:5(decimal) │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ ├── prune: (1-4) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── prune: (1-5) │ └── interesting orderings: (+1) (-4,+3,+1) └── projections - └── subquery [as=x:9, type=int, outer=(1,2), immutable, correlated-subquery] + └── subquery [as=x:11, type=int, outer=(1,2), immutable, correlated-subquery] └── max1-row - ├── columns: x:8(int) + ├── columns: x:10(int) ├── error: "more than one row returned by a subquery used as an expression" ├── outer: (1,2) ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(8) + ├── fd: ()-->(10) └── limit - ├── columns: x:8(int) + ├── columns: x:10(int) ├── outer: (1,2) ├── immutable - ├── fd: ()-->(8) - ├── prune: (8) + ├── fd: ()-->(10) + ├── prune: (10) ├── project - │ ├── columns: x:8(int) + │ ├── columns: x:10(int) │ ├── outer: (1) - │ ├── fd: ()-->(8) - │ ├── prune: (8) + │ ├── fd: ()-->(10) + │ ├── prune: (10) │ ├── scan kuv - │ │ ├── columns: k:5(int!null) u:6(float) v:7(string) - │ │ ├── key: (5) - │ │ ├── fd: (5)-->(6,7) - │ │ ├── prune: (5-7) - │ │ └── interesting orderings: (+5) + │ │ ├── columns: k:6(int!null) u:7(float) v:8(string) kuv.crdb_internal_mvcc_timestamp:9(decimal) + │ │ ├── key: (6) + │ │ ├── fd: (6)-->(7-9) + │ │ ├── prune: (6-9) + │ │ └── interesting orderings: (+6) │ └── projections - │ └── variable: xyzs.x:1 [as=x:8, type=int, outer=(1)] + │ └── variable: xyzs.x:1 [as=x:10, type=int, outer=(1)] └── variable: y:2 [type=int] # Test very high limit (> max uint32). diff --git a/pkg/sql/opt/memo/testdata/logprops/lookup-join b/pkg/sql/opt/memo/testdata/logprops/lookup-join index 2511f2d409f9..afa3ec3d9207 100644 --- a/pkg/sql/opt/memo/testdata/logprops/lookup-join +++ b/pkg/sql/opt/memo/testdata/logprops/lookup-join @@ -25,20 +25,20 @@ opt SELECT * FROM small JOIN abcd ON a=m ---- inner-join (lookup abcd) - ├── columns: m:1(int!null) n:2(int) a:4(int!null) b:5(int) c:6(int) - ├── key columns: [7] = [7] + ├── columns: m:1(int!null) n:2(int) a:5(int!null) b:6(int) c:7(int) + ├── key columns: [8] = [8] ├── lookup columns are key - ├── fd: (1)==(4), (4)==(1) - ├── prune: (2,5,6) - ├── interesting orderings: (+4,+5) + ├── fd: (1)==(5), (5)==(1) + ├── prune: (2,6,7) + ├── interesting orderings: (+5,+6) ├── inner-join (lookup abcd@secondary) - │ ├── columns: m:1(int!null) n:2(int) a:4(int!null) b:5(int) abcd.rowid:7(int!null) - │ ├── key columns: [1] = [4] - │ ├── fd: (7)-->(4,5), (1)==(4), (4)==(1) + │ ├── columns: m:1(int!null) n:2(int) a:5(int!null) b:6(int) abcd.rowid:8(int!null) + │ ├── key columns: [1] = [5] + │ ├── fd: (8)-->(5,6), (1)==(5), (5)==(1) │ ├── scan small │ │ ├── columns: m:1(int) n:2(int) │ │ ├── prune: (1,2) - │ │ └── unfiltered-cols: (1-3) + │ │ └── unfiltered-cols: (1-4) │ └── filters (true) └── filters (true) @@ -48,16 +48,16 @@ opt SELECT * FROM small JOIN abcd ON a=m WHERE n > 2 ---- inner-join (lookup abcd) - ├── columns: m:1(int!null) n:2(int!null) a:4(int!null) b:5(int) c:6(int) - ├── key columns: [7] = [7] + ├── columns: m:1(int!null) n:2(int!null) a:5(int!null) b:6(int) c:7(int) + ├── key columns: [8] = [8] ├── lookup columns are key - ├── fd: (1)==(4), (4)==(1) - ├── prune: (5,6) - ├── interesting orderings: (+4,+5) + ├── fd: (1)==(5), (5)==(1) + ├── prune: (6,7) + ├── interesting orderings: (+5,+6) ├── inner-join (lookup abcd@secondary) - │ ├── columns: m:1(int!null) n:2(int!null) a:4(int!null) b:5(int) abcd.rowid:7(int!null) - │ ├── key columns: [1] = [4] - │ ├── fd: (7)-->(4,5), (1)==(4), (4)==(1) + │ ├── columns: m:1(int!null) n:2(int!null) a:5(int!null) b:6(int) abcd.rowid:8(int!null) + │ ├── key columns: [1] = [5] + │ ├── fd: (8)-->(5,6), (1)==(5), (5)==(1) │ ├── select │ │ ├── columns: m:1(int) n:2(int!null) │ │ ├── prune: (1) @@ -77,23 +77,23 @@ opt SELECT * FROM small JOIN abcd ON a=m WHERE b > 2 ---- inner-join (lookup abcd) - ├── columns: m:1(int!null) n:2(int) a:4(int!null) b:5(int!null) c:6(int) - ├── key columns: [7] = [7] + ├── columns: m:1(int!null) n:2(int) a:5(int!null) b:6(int!null) c:7(int) + ├── key columns: [8] = [8] ├── lookup columns are key - ├── fd: (1)==(4), (4)==(1) - ├── prune: (2,6) - ├── interesting orderings: (+4,+5) + ├── fd: (1)==(5), (5)==(1) + ├── prune: (2,7) + ├── interesting orderings: (+5,+6) ├── inner-join (lookup abcd@secondary) - │ ├── columns: m:1(int!null) n:2(int) a:4(int!null) b:5(int!null) abcd.rowid:7(int!null) - │ ├── key columns: [1] = [4] - │ ├── fd: (7)-->(4,5), (1)==(4), (4)==(1) + │ ├── columns: m:1(int!null) n:2(int) a:5(int!null) b:6(int!null) abcd.rowid:8(int!null) + │ ├── key columns: [1] = [5] + │ ├── fd: (8)-->(5,6), (1)==(5), (5)==(1) │ ├── scan small │ │ ├── columns: m:1(int) n:2(int) │ │ ├── prune: (1,2) - │ │ └── unfiltered-cols: (1-3) + │ │ └── unfiltered-cols: (1-4) │ └── filters - │ └── gt [type=bool, outer=(5), constraints=(/5: [/3 - ]; tight)] - │ ├── variable: b:5 [type=int] + │ └── gt [type=bool, outer=(6), constraints=(/6: [/3 - ]; tight)] + │ ├── variable: b:6 [type=int] │ └── const: 2 [type=int] └── filters (true) @@ -102,24 +102,24 @@ opt SELECT * FROM small JOIN abcd ON a=m WHERE c>2 ---- inner-join (lookup abcd) - ├── columns: m:1(int!null) n:2(int) a:4(int!null) b:5(int) c:6(int!null) - ├── key columns: [7] = [7] + ├── columns: m:1(int!null) n:2(int) a:5(int!null) b:6(int) c:7(int!null) + ├── key columns: [8] = [8] ├── lookup columns are key - ├── fd: (1)==(4), (4)==(1) - ├── prune: (2,5) - ├── interesting orderings: (+4,+5) + ├── fd: (1)==(5), (5)==(1) + ├── prune: (2,6) + ├── interesting orderings: (+5,+6) ├── inner-join (lookup abcd@secondary) - │ ├── columns: m:1(int!null) n:2(int) a:4(int!null) b:5(int) abcd.rowid:7(int!null) - │ ├── key columns: [1] = [4] - │ ├── fd: (7)-->(4,5), (1)==(4), (4)==(1) + │ ├── columns: m:1(int!null) n:2(int) a:5(int!null) b:6(int) abcd.rowid:8(int!null) + │ ├── key columns: [1] = [5] + │ ├── fd: (8)-->(5,6), (1)==(5), (5)==(1) │ ├── scan small │ │ ├── columns: m:1(int) n:2(int) │ │ ├── prune: (1,2) - │ │ └── unfiltered-cols: (1-3) + │ │ └── unfiltered-cols: (1-4) │ └── filters (true) └── filters - └── gt [type=bool, outer=(6), constraints=(/6: [/3 - ]; tight)] - ├── variable: c:6 [type=int] + └── gt [type=bool, outer=(7), constraints=(/7: [/3 - ]; tight)] + ├── variable: c:7 [type=int] └── const: 2 [type=int] # Multiple equalities. @@ -127,21 +127,21 @@ opt SELECT * FROM small JOIN abcd ON a=m AND b=n WHERE c>2 ---- inner-join (lookup abcd) - ├── columns: m:1(int!null) n:2(int!null) a:4(int!null) b:5(int!null) c:6(int!null) - ├── key columns: [7] = [7] + ├── columns: m:1(int!null) n:2(int!null) a:5(int!null) b:6(int!null) c:7(int!null) + ├── key columns: [8] = [8] ├── lookup columns are key - ├── fd: (1)==(4), (4)==(1), (2)==(5), (5)==(2) - ├── interesting orderings: (+4,+5) + ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2) + ├── interesting orderings: (+5,+6) ├── inner-join (lookup abcd@secondary) - │ ├── columns: m:1(int!null) n:2(int!null) a:4(int!null) b:5(int!null) abcd.rowid:7(int!null) - │ ├── key columns: [1 2] = [4 5] - │ ├── fd: (7)-->(4,5), (1)==(4), (4)==(1), (2)==(5), (5)==(2) + │ ├── columns: m:1(int!null) n:2(int!null) a:5(int!null) b:6(int!null) abcd.rowid:8(int!null) + │ ├── key columns: [1 2] = [5 6] + │ ├── fd: (8)-->(5,6), (1)==(5), (5)==(1), (2)==(6), (6)==(2) │ ├── scan small │ │ ├── columns: m:1(int) n:2(int) │ │ ├── prune: (1,2) - │ │ └── unfiltered-cols: (1-3) + │ │ └── unfiltered-cols: (1-4) │ └── filters (true) └── filters - └── gt [type=bool, outer=(6), constraints=(/6: [/3 - ]; tight)] - ├── variable: c:6 [type=int] + └── gt [type=bool, outer=(7), constraints=(/7: [/3 - ]; tight)] + ├── variable: c:7 [type=int] └── const: 2 [type=int] diff --git a/pkg/sql/opt/memo/testdata/logprops/max1row b/pkg/sql/opt/memo/testdata/logprops/max1row index 6a8b84fccdbb..2fe7e18fbb5d 100644 --- a/pkg/sql/opt/memo/testdata/logprops/max1row +++ b/pkg/sql/opt/memo/testdata/logprops/max1row @@ -9,34 +9,40 @@ CREATE TABLE kuv (k INT PRIMARY KEY, u FLOAT, v STRING) build SELECT * FROM xyzs WHERE (SELECT v FROM kuv) = 'foo' ---- -select +project ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) ├── key: (1) ├── fd: (1)-->(2-4), (3,4)~~>(1,2) ├── prune: (1-4) ├── interesting orderings: (+1) (-4,+3,+1) - ├── scan xyzs - │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) - │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ ├── prune: (1-4) - │ └── interesting orderings: (+1) (-4,+3,+1) - └── filters - └── eq [type=bool, subquery] - ├── subquery [type=string] - │ └── max1-row - │ ├── columns: v:7(string) - │ ├── error: "more than one row returned by a subquery used as an expression" - │ ├── cardinality: [0 - 1] - │ ├── key: () - │ ├── fd: ()-->(7) - │ └── project - │ ├── columns: v:7(string) - │ ├── prune: (7) - │ └── scan kuv - │ ├── columns: k:5(int!null) u:6(float) v:7(string) - │ ├── key: (5) - │ ├── fd: (5)-->(6,7) - │ ├── prune: (5-7) - │ └── interesting orderings: (+5) - └── const: 'foo' [type=string] + └── select + ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) xyzs.crdb_internal_mvcc_timestamp:5(decimal) + ├── key: (1) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + ├── prune: (1-5) + ├── interesting orderings: (+1) (-4,+3,+1) + ├── scan xyzs + │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) xyzs.crdb_internal_mvcc_timestamp:5(decimal) + │ ├── key: (1) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── prune: (1-5) + │ └── interesting orderings: (+1) (-4,+3,+1) + └── filters + └── eq [type=bool, subquery] + ├── subquery [type=string] + │ └── max1-row + │ ├── columns: v:8(string) + │ ├── error: "more than one row returned by a subquery used as an expression" + │ ├── cardinality: [0 - 1] + │ ├── key: () + │ ├── fd: ()-->(8) + │ └── project + │ ├── columns: v:8(string) + │ ├── prune: (8) + │ └── scan kuv + │ ├── columns: k:6(int!null) u:7(float) v:8(string) kuv.crdb_internal_mvcc_timestamp:9(decimal) + │ ├── key: (6) + │ ├── fd: (6)-->(7-9) + │ ├── prune: (6-9) + │ └── interesting orderings: (+6) + └── const: 'foo' [type=string] diff --git a/pkg/sql/opt/memo/testdata/logprops/offset b/pkg/sql/opt/memo/testdata/logprops/offset index d5330aa1b3d9..c42c31a68945 100644 --- a/pkg/sql/opt/memo/testdata/logprops/offset +++ b/pkg/sql/opt/memo/testdata/logprops/offset @@ -15,12 +15,18 @@ offset ├── fd: (1)-->(2-4), (3,4)~~>(1,2) ├── prune: (1-4) ├── interesting orderings: (+1) (-4,+3,+1) - ├── scan xyzs + ├── project │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) │ ├── key: (1) │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) │ ├── prune: (1-4) - │ └── interesting orderings: (+1) (-4,+3,+1) + │ ├── interesting orderings: (+1) (-4,+3,+1) + │ └── scan xyzs + │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) crdb_internal_mvcc_timestamp:5(decimal) + │ ├── key: (1) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── prune: (1-5) + │ └── interesting orderings: (+1) (-4,+3,+1) └── const: 1 [type=int] build @@ -32,31 +38,37 @@ offset ├── fd: (1)-->(2-4), (3,4)~~>(1,2) ├── prune: (1-4) ├── interesting orderings: (+1) (-4,+3,+1) - ├── scan xyzs + ├── project │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) │ ├── key: (1) │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) │ ├── prune: (1-4) - │ └── interesting orderings: (+1) (-4,+3,+1) + │ ├── interesting orderings: (+1) (-4,+3,+1) + │ └── scan xyzs + │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) crdb_internal_mvcc_timestamp:5(decimal) + │ ├── key: (1) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── prune: (1-5) + │ └── interesting orderings: (+1) (-4,+3,+1) └── subquery [type=int] └── max1-row - ├── columns: "?column?":5(int!null) + ├── columns: "?column?":6(int!null) ├── error: "more than one row returned by a subquery used as an expression" ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(5) + ├── fd: ()-->(6) └── project - ├── columns: "?column?":5(int!null) + ├── columns: "?column?":6(int!null) ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(5) - ├── prune: (5) + ├── fd: ()-->(6) + ├── prune: (6) ├── values │ ├── cardinality: [1 - 1] │ ├── key: () │ └── tuple [type=tuple] └── projections - └── const: 1 [as="?column?":5, type=int] + └── const: 1 [as="?column?":6, type=int] build SELECT * FROM xyzs OFFSET 0 @@ -67,12 +79,18 @@ offset ├── fd: (1)-->(2-4), (3,4)~~>(1,2) ├── prune: (1-4) ├── interesting orderings: (+1) (-4,+3,+1) - ├── scan xyzs + ├── project │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) │ ├── key: (1) │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) │ ├── prune: (1-4) - │ └── interesting orderings: (+1) (-4,+3,+1) + │ ├── interesting orderings: (+1) (-4,+3,+1) + │ └── scan xyzs + │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) crdb_internal_mvcc_timestamp:5(decimal) + │ ├── key: (1) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── prune: (1-5) + │ └── interesting orderings: (+1) (-4,+3,+1) └── const: 0 [type=int] # Propagate outer columns. @@ -80,41 +98,41 @@ build SELECT (SELECT x FROM kuv OFFSET y) FROM xyzs ---- project - ├── columns: x:9(int) - ├── prune: (9) + ├── columns: x:11(int) + ├── prune: (11) ├── scan xyzs - │ ├── columns: xyzs.x:1(int!null) y:2(int) z:3(float!null) s:4(string) + │ ├── columns: xyzs.x:1(int!null) y:2(int) z:3(float!null) s:4(string) xyzs.crdb_internal_mvcc_timestamp:5(decimal) │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ ├── prune: (1-4) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── prune: (1-5) │ └── interesting orderings: (+1) (-4,+3,+1) └── projections - └── subquery [as=x:9, type=int, outer=(1,2), correlated-subquery] + └── subquery [as=x:11, type=int, outer=(1,2), correlated-subquery] └── max1-row - ├── columns: x:8(int) + ├── columns: x:10(int) ├── error: "more than one row returned by a subquery used as an expression" ├── outer: (1,2) ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(8) + ├── fd: ()-->(10) └── offset - ├── columns: x:8(int) + ├── columns: x:10(int) ├── outer: (1,2) - ├── fd: ()-->(8) - ├── prune: (8) + ├── fd: ()-->(10) + ├── prune: (10) ├── project - │ ├── columns: x:8(int) + │ ├── columns: x:10(int) │ ├── outer: (1) - │ ├── fd: ()-->(8) - │ ├── prune: (8) + │ ├── fd: ()-->(10) + │ ├── prune: (10) │ ├── scan kuv - │ │ ├── columns: k:5(int!null) u:6(float) v:7(string) - │ │ ├── key: (5) - │ │ ├── fd: (5)-->(6,7) - │ │ ├── prune: (5-7) - │ │ └── interesting orderings: (+5) + │ │ ├── columns: k:6(int!null) u:7(float) v:8(string) kuv.crdb_internal_mvcc_timestamp:9(decimal) + │ │ ├── key: (6) + │ │ ├── fd: (6)-->(7-9) + │ │ ├── prune: (6-9) + │ │ └── interesting orderings: (+6) │ └── projections - │ └── variable: xyzs.x:1 [as=x:8, type=int, outer=(1)] + │ └── variable: xyzs.x:1 [as=x:10, type=int, outer=(1)] └── variable: y:2 [type=int] # Reduce cardinality of input set. @@ -124,15 +142,15 @@ FROM ((SELECT x FROM xyzs LIMIT 10) UNION ALL (SELECT * FROM (VALUES (1), (2), ( OFFSET 2 ---- offset - ├── columns: x:6(int!null) + ├── columns: x:7(int!null) ├── cardinality: [1 - 11] - ├── prune: (6) + ├── prune: (7) ├── union-all - │ ├── columns: x:6(int!null) + │ ├── columns: x:7(int!null) │ ├── left columns: xyzs.x:1(int) - │ ├── right columns: column1:5(int) + │ ├── right columns: column1:6(int) │ ├── cardinality: [3 - 13] - │ ├── prune: (6) + │ ├── prune: (7) │ ├── limit │ │ ├── columns: xyzs.x:1(int!null) │ │ ├── cardinality: [0 - 10] @@ -146,17 +164,17 @@ offset │ │ │ ├── prune: (1) │ │ │ ├── interesting orderings: (+1) │ │ │ └── scan xyzs - │ │ │ ├── columns: xyzs.x:1(int!null) y:2(int) z:3(float!null) s:4(string) + │ │ │ ├── columns: xyzs.x:1(int!null) y:2(int) z:3(float!null) s:4(string) crdb_internal_mvcc_timestamp:5(decimal) │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) + │ │ │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) │ │ │ ├── limit hint: 10.00 - │ │ │ ├── prune: (1-4) + │ │ │ ├── prune: (1-5) │ │ │ └── interesting orderings: (+1) (-4,+3,+1) │ │ └── const: 10 [type=int] │ └── values - │ ├── columns: column1:5(int!null) + │ ├── columns: column1:6(int!null) │ ├── cardinality: [3 - 3] - │ ├── prune: (5) + │ ├── prune: (6) │ ├── tuple [type=tuple{int}] │ │ └── const: 1 [type=int] │ ├── tuple [type=tuple{int}] diff --git a/pkg/sql/opt/memo/testdata/logprops/project b/pkg/sql/opt/memo/testdata/logprops/project index 2f2b2b08ff2a..faa5be5e027e 100644 --- a/pkg/sql/opt/memo/testdata/logprops/project +++ b/pkg/sql/opt/memo/testdata/logprops/project @@ -14,23 +14,23 @@ build SELECT y, x+1 AS a, 1 AS b, x FROM xysd ---- project - ├── columns: y:2(int) a:5(int!null) b:6(int!null) x:1(int!null) + ├── columns: y:2(int) a:6(int!null) b:7(int!null) x:1(int!null) ├── immutable ├── key: (1) - ├── fd: ()-->(6), (1)-->(2,5) - ├── prune: (1,2,5,6) + ├── fd: ()-->(7), (1)-->(2,6) + ├── prune: (1,2,6,7) ├── interesting orderings: (+1) ├── scan xysd - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ ├── prune: (1-4) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── prune: (1-5) │ └── interesting orderings: (+1) (-3,+4,+1) └── projections - ├── plus [as=a:5, type=int, outer=(1), immutable] + ├── plus [as=a:6, type=int, outer=(1), immutable] │ ├── variable: x:1 [type=int] │ └── const: 1 [type=int] - └── const: 1 [as=b:6, type=int] + └── const: 1 [as=b:7, type=int] build SELECT s FROM xysd @@ -40,86 +40,92 @@ project ├── prune: (3) ├── interesting orderings: (-3) └── scan xysd - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) ├── key: (1) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - ├── prune: (1-4) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + ├── prune: (1-5) └── interesting orderings: (+1) (-3,+4,+1) # Propagate outer columns. build SELECT * FROM xysd WHERE (SELECT (SELECT y) FROM kuv WHERE k=x) > 5 ---- -select +project ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) ├── key: (1) ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - ├── prune: (3,4) + ├── prune: (1-4) ├── interesting orderings: (+1) (-3,+4,+1) - ├── scan xysd - │ ├── columns: x:1(int!null) xysd.y:2(int) s:3(string) d:4(decimal!null) - │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ ├── prune: (1-4) - │ └── interesting orderings: (+1) (-3,+4,+1) - └── filters - └── gt [type=bool, outer=(1,2), correlated-subquery] - ├── subquery [type=int] - │ └── max1-row - │ ├── columns: y:9(int) - │ ├── error: "more than one row returned by a subquery used as an expression" - │ ├── outer: (1,2) - │ ├── cardinality: [0 - 1] - │ ├── key: () - │ ├── fd: ()-->(9) - │ └── project - │ ├── columns: y:9(int) - │ ├── outer: (1,2) - │ ├── cardinality: [0 - 1] - │ ├── key: () - │ ├── fd: ()-->(9) - │ ├── prune: (9) - │ ├── select - │ │ ├── columns: k:5(int!null) u:6(float) v:7(string) - │ │ ├── outer: (1) - │ │ ├── cardinality: [0 - 1] - │ │ ├── key: () - │ │ ├── fd: ()-->(5-7) - │ │ ├── prune: (6,7) - │ │ ├── interesting orderings: (+5) - │ │ ├── scan kuv - │ │ │ ├── columns: k:5(int!null) u:6(float) v:7(string) - │ │ │ ├── key: (5) - │ │ │ ├── fd: (5)-->(6,7) - │ │ │ ├── prune: (5-7) - │ │ │ └── interesting orderings: (+5) - │ │ └── filters - │ │ └── eq [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - │ │ ├── variable: k:5 [type=int] - │ │ └── variable: x:1 [type=int] - │ └── projections - │ └── subquery [as=y:9, type=int, outer=(2), correlated-subquery] - │ └── max1-row - │ ├── columns: y:8(int) - │ ├── error: "more than one row returned by a subquery used as an expression" - │ ├── outer: (2) - │ ├── cardinality: [1 - 1] - │ ├── key: () - │ ├── fd: ()-->(8) - │ └── project - │ ├── columns: y:8(int) - │ ├── outer: (2) - │ ├── cardinality: [1 - 1] - │ ├── key: () - │ ├── fd: ()-->(8) - │ ├── prune: (8) - │ ├── values - │ │ ├── cardinality: [1 - 1] - │ │ ├── key: () - │ │ └── tuple [type=tuple] - │ └── projections - │ └── variable: xysd.y:2 [as=y:8, type=int, outer=(2)] - └── const: 5 [type=int] + └── select + ├── columns: x:1(int!null) xysd.y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) + ├── key: (1) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + ├── prune: (3-5) + ├── interesting orderings: (+1) (-3,+4,+1) + ├── scan xysd + │ ├── columns: x:1(int!null) xysd.y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) + │ ├── key: (1) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── prune: (1-5) + │ └── interesting orderings: (+1) (-3,+4,+1) + └── filters + └── gt [type=bool, outer=(1,2), correlated-subquery] + ├── subquery [type=int] + │ └── max1-row + │ ├── columns: y:11(int) + │ ├── error: "more than one row returned by a subquery used as an expression" + │ ├── outer: (1,2) + │ ├── cardinality: [0 - 1] + │ ├── key: () + │ ├── fd: ()-->(11) + │ └── project + │ ├── columns: y:11(int) + │ ├── outer: (1,2) + │ ├── cardinality: [0 - 1] + │ ├── key: () + │ ├── fd: ()-->(11) + │ ├── prune: (11) + │ ├── select + │ │ ├── columns: k:6(int!null) u:7(float) v:8(string) kuv.crdb_internal_mvcc_timestamp:9(decimal) + │ │ ├── outer: (1) + │ │ ├── cardinality: [0 - 1] + │ │ ├── key: () + │ │ ├── fd: ()-->(6-9) + │ │ ├── prune: (7-9) + │ │ ├── interesting orderings: (+6) + │ │ ├── scan kuv + │ │ │ ├── columns: k:6(int!null) u:7(float) v:8(string) kuv.crdb_internal_mvcc_timestamp:9(decimal) + │ │ │ ├── key: (6) + │ │ │ ├── fd: (6)-->(7-9) + │ │ │ ├── prune: (6-9) + │ │ │ └── interesting orderings: (+6) + │ │ └── filters + │ │ └── eq [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ │ ├── variable: k:6 [type=int] + │ │ └── variable: x:1 [type=int] + │ └── projections + │ └── subquery [as=y:11, type=int, outer=(2), correlated-subquery] + │ └── max1-row + │ ├── columns: y:10(int) + │ ├── error: "more than one row returned by a subquery used as an expression" + │ ├── outer: (2) + │ ├── cardinality: [1 - 1] + │ ├── key: () + │ ├── fd: ()-->(10) + │ └── project + │ ├── columns: y:10(int) + │ ├── outer: (2) + │ ├── cardinality: [1 - 1] + │ ├── key: () + │ ├── fd: ()-->(10) + │ ├── prune: (10) + │ ├── values + │ │ ├── cardinality: [1 - 1] + │ │ ├── key: () + │ │ └── tuple [type=tuple] + │ └── projections + │ └── variable: xysd.y:2 [as=y:10, type=int, outer=(2)] + └── const: 5 [type=int] # Pass through cardinality. build @@ -139,13 +145,20 @@ project ├── fd: (1)-->(2-4), (3,4)~~>(1,2) ├── prune: (1-4) ├── interesting orderings: (+1) (-3,+4,+1) - ├── scan xysd + ├── project │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) │ ├── key: (1) │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) │ ├── limit hint: 10.00 │ ├── prune: (1-4) - │ └── interesting orderings: (+1) (-3,+4,+1) + │ ├── interesting orderings: (+1) (-3,+4,+1) + │ └── scan xysd + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) + │ ├── key: (1) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── limit hint: 10.00 + │ ├── prune: (1-5) + │ └── interesting orderings: (+1) (-3,+4,+1) └── const: 10 [type=int] # Constant null and not-null columns. @@ -153,21 +166,21 @@ build SELECT 1 AS a, 'foo' AS b, NULL AS c, 1::decimal + NULL AS d, NULL::STRING AS e FROM xysd ---- project - ├── columns: a:5(int!null) b:6(string!null) c:7(unknown) d:7(unknown) e:8(string) + ├── columns: a:6(int!null) b:7(string!null) c:8(unknown) d:8(unknown) e:9(string) ├── immutable - ├── fd: ()-->(5-8) - ├── prune: (5-8) + ├── fd: ()-->(6-9) + ├── prune: (6-9) ├── scan xysd - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ ├── prune: (1-4) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── prune: (1-5) │ └── interesting orderings: (+1) (-3,+4,+1) └── projections - ├── const: 1 [as=a:5, type=int] - ├── const: 'foo' [as=b:6, type=string] - ├── null [as=c:7, type=unknown] - └── cast: STRING [as=e:8, type=string, immutable] + ├── const: 1 [as=a:6, type=int] + ├── const: 'foo' [as=b:7, type=string] + ├── null [as=c:8, type=unknown] + └── cast: STRING [as=e:9, type=string, immutable] └── null [type=unknown] # Project constant over input with no needed columns and ensure that there is @@ -176,12 +189,12 @@ opt SELECT 1 FROM (SELECT x FROM xysd) ---- project - ├── columns: "?column?":5(int!null) - ├── fd: ()-->(5) - ├── prune: (5) + ├── columns: "?column?":6(int!null) + ├── fd: ()-->(6) + ├── prune: (6) ├── scan xysd@secondary └── projections - └── const: 1 [as="?column?":5, type=int] + └── const: 1 [as="?column?":6, type=int] # Project simple variable reference after constant folding; should be not-null # if the column it refers to is not-null. @@ -189,66 +202,66 @@ norm SELECT CASE WHEN true THEN x END FROM xysd ---- project - ├── columns: case:5(int!null) - ├── key: (5) - ├── prune: (5) + ├── columns: case:6(int!null) + ├── key: (6) + ├── prune: (6) ├── scan xysd │ ├── columns: x:1(int!null) │ ├── key: (1) │ ├── prune: (1) │ └── interesting orderings: (+1) └── projections - └── variable: x:1 [as=case:5, type=int, outer=(1)] + └── variable: x:1 [as=case:6, type=int, outer=(1)] # Project correlated subquery. build SELECT k, (SELECT y FROM xysd WHERE x=k) FROM kuv ---- project - ├── columns: k:1(int!null) y:8(int) + ├── columns: k:1(int!null) y:10(int) ├── key: (1) - ├── fd: (1)-->(8) - ├── prune: (1,8) + ├── fd: (1)-->(10) + ├── prune: (1,10) ├── interesting orderings: (+1) ├── scan kuv - │ ├── columns: k:1(int!null) u:2(float) v:3(string) + │ ├── columns: k:1(int!null) u:2(float) v:3(string) kuv.crdb_internal_mvcc_timestamp:4(decimal) │ ├── key: (1) - │ ├── fd: (1)-->(2,3) - │ ├── prune: (1-3) + │ ├── fd: (1)-->(2-4) + │ ├── prune: (1-4) │ └── interesting orderings: (+1) └── projections - └── subquery [as=y:8, type=int, outer=(1), correlated-subquery] + └── subquery [as=y:10, type=int, outer=(1), correlated-subquery] └── max1-row - ├── columns: xysd.y:5(int) + ├── columns: xysd.y:6(int) ├── error: "more than one row returned by a subquery used as an expression" ├── outer: (1) ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(5) + ├── fd: ()-->(6) └── project - ├── columns: xysd.y:5(int) + ├── columns: xysd.y:6(int) ├── outer: (1) ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(5) - ├── prune: (5) + ├── fd: ()-->(6) + ├── prune: (6) └── select - ├── columns: x:4(int!null) xysd.y:5(int) s:6(string) d:7(decimal!null) + ├── columns: x:5(int!null) xysd.y:6(int) s:7(string) d:8(decimal!null) xysd.crdb_internal_mvcc_timestamp:9(decimal) ├── outer: (1) ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(4-7) - ├── prune: (5-7) - ├── interesting orderings: (+4) (-6,+7,+4) + ├── fd: ()-->(5-9) + ├── prune: (6-9) + ├── interesting orderings: (+5) (-7,+8,+5) ├── scan xysd - │ ├── columns: x:4(int!null) xysd.y:5(int) s:6(string) d:7(decimal!null) - │ ├── key: (4) - │ ├── fd: (4)-->(5-7), (6,7)~~>(4,5) - │ ├── prune: (4-7) - │ └── interesting orderings: (+4) (-6,+7,+4) + │ ├── columns: x:5(int!null) xysd.y:6(int) s:7(string) d:8(decimal!null) xysd.crdb_internal_mvcc_timestamp:9(decimal) + │ ├── key: (5) + │ ├── fd: (5)-->(6-9), (7,8)~~>(5,6,9) + │ ├── prune: (5-9) + │ └── interesting orderings: (+5) (-7,+8,+5) └── filters - └── eq [type=bool, outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] - ├── variable: x:4 [type=int] + └── eq [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + ├── variable: x:5 [type=int] └── variable: k:1 [type=int] # Project nested correlated subquery. @@ -256,56 +269,56 @@ build SELECT k, EXISTS(SELECT EXISTS(SELECT y FROM xysd WHERE x=k) FROM xysd) FROM kuv ---- project - ├── columns: k:1(int!null) exists:13(bool) + ├── columns: k:1(int!null) exists:16(bool) ├── key: (1) - ├── fd: (1)-->(13) - ├── prune: (1,13) + ├── fd: (1)-->(16) + ├── prune: (1,16) ├── interesting orderings: (+1) ├── scan kuv - │ ├── columns: k:1(int!null) u:2(float) v:3(string) + │ ├── columns: k:1(int!null) u:2(float) v:3(string) kuv.crdb_internal_mvcc_timestamp:4(decimal) │ ├── key: (1) - │ ├── fd: (1)-->(2,3) - │ ├── prune: (1-3) + │ ├── fd: (1)-->(2-4) + │ ├── prune: (1-4) │ └── interesting orderings: (+1) └── projections - └── exists [as=exists:13, type=bool, outer=(1), correlated-subquery] + └── exists [as=exists:16, type=bool, outer=(1), correlated-subquery] └── project - ├── columns: exists:12(bool) + ├── columns: exists:15(bool) ├── outer: (1) - ├── fd: ()-->(12) - ├── prune: (12) + ├── fd: ()-->(15) + ├── prune: (15) ├── scan xysd - │ ├── columns: x:4(int!null) y:5(int) s:6(string) d:7(decimal!null) - │ ├── key: (4) - │ ├── fd: (4)-->(5-7), (6,7)~~>(4,5) - │ ├── prune: (4-7) - │ └── interesting orderings: (+4) (-6,+7,+4) + │ ├── columns: x:5(int!null) y:6(int) s:7(string) d:8(decimal!null) xysd.crdb_internal_mvcc_timestamp:9(decimal) + │ ├── key: (5) + │ ├── fd: (5)-->(6-9), (7,8)~~>(5,6,9) + │ ├── prune: (5-9) + │ └── interesting orderings: (+5) (-7,+8,+5) └── projections - └── exists [as=exists:12, type=bool, outer=(1), correlated-subquery] + └── exists [as=exists:15, type=bool, outer=(1), correlated-subquery] └── project - ├── columns: y:9(int) + ├── columns: y:11(int) ├── outer: (1) ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(9) - ├── prune: (9) + ├── fd: ()-->(11) + ├── prune: (11) └── select - ├── columns: x:8(int!null) y:9(int) s:10(string) d:11(decimal!null) + ├── columns: x:10(int!null) y:11(int) s:12(string) d:13(decimal!null) xysd.crdb_internal_mvcc_timestamp:14(decimal) ├── outer: (1) ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(8-11) - ├── prune: (9-11) - ├── interesting orderings: (+8) (-10,+11,+8) + ├── fd: ()-->(10-14) + ├── prune: (11-14) + ├── interesting orderings: (+10) (-12,+13,+10) ├── scan xysd - │ ├── columns: x:8(int!null) y:9(int) s:10(string) d:11(decimal!null) - │ ├── key: (8) - │ ├── fd: (8)-->(9-11), (10,11)~~>(8,9) - │ ├── prune: (8-11) - │ └── interesting orderings: (+8) (-10,+11,+8) + │ ├── columns: x:10(int!null) y:11(int) s:12(string) d:13(decimal!null) xysd.crdb_internal_mvcc_timestamp:14(decimal) + │ ├── key: (10) + │ ├── fd: (10)-->(11-14), (12,13)~~>(10,11,14) + │ ├── prune: (10-14) + │ └── interesting orderings: (+10) (-12,+13,+10) └── filters - └── eq [type=bool, outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] - ├── variable: x:8 [type=int] + └── eq [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] + ├── variable: x:10 [type=int] └── variable: k:1 [type=int] # We have the FD: y --> y::TEXT. @@ -313,18 +326,18 @@ build SELECT y, y::TEXT FROM xysd ---- project - ├── columns: y:2(int) y:5(string) + ├── columns: y:2(int) y:6(string) ├── immutable - ├── fd: (2)-->(5) - ├── prune: (2,5) + ├── fd: (2)-->(6) + ├── prune: (2,6) ├── scan xysd - │ ├── columns: x:1(int!null) xysd.y:2(int) s:3(string) d:4(decimal!null) + │ ├── columns: x:1(int!null) xysd.y:2(int) s:3(string) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ ├── prune: (1-4) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── prune: (1-5) │ └── interesting orderings: (+1) (-3,+4,+1) └── projections - └── cast: STRING [as=y:5, type=string, outer=(2), immutable] + └── cast: STRING [as=y:6, type=string, outer=(2), immutable] └── variable: xysd.y:2 [type=int] # We don't have the FD: d --> d::TEXT because d is a composite type. @@ -333,17 +346,17 @@ build SELECT d, d::TEXT FROM xysd ---- project - ├── columns: d:4(decimal!null) d:5(string!null) + ├── columns: d:4(decimal!null) d:6(string!null) ├── immutable - ├── prune: (4,5) + ├── prune: (4,6) ├── scan xysd - │ ├── columns: x:1(int!null) y:2(int) s:3(string) xysd.d:4(decimal!null) + │ ├── columns: x:1(int!null) y:2(int) s:3(string) xysd.d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ ├── prune: (1-4) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── prune: (1-5) │ └── interesting orderings: (+1) (-3,+4,+1) └── projections - └── cast: STRING [as=d:5, type=string, outer=(4), immutable] + └── cast: STRING [as=d:6, type=string, outer=(4), immutable] └── variable: xysd.d:4 [type=decimal] # We have the equality relation between the synthesized column and the column @@ -352,10 +365,10 @@ norm SELECT x, CASE WHEN true THEN x END FROM xysd ---- project - ├── columns: x:1(int!null) case:5(int!null) + ├── columns: x:1(int!null) case:6(int!null) ├── key: (1) - ├── fd: (1)==(5), (5)==(1) - ├── prune: (1,5) + ├── fd: (1)==(6), (6)==(1) + ├── prune: (1,6) ├── interesting orderings: (+1) ├── scan xysd │ ├── columns: x:1(int!null) @@ -363,7 +376,7 @@ project │ ├── prune: (1) │ └── interesting orderings: (+1) └── projections - └── variable: x:1 [as=case:5, type=int, outer=(1)] + └── variable: x:1 [as=case:6, type=int, outer=(1)] # Verify that a,b form a key. diff --git a/pkg/sql/opt/memo/testdata/logprops/rownumber b/pkg/sql/opt/memo/testdata/logprops/rownumber index 95cc8e3150e8..b8789f7a8b66 100644 --- a/pkg/sql/opt/memo/testdata/logprops/rownumber +++ b/pkg/sql/opt/memo/testdata/logprops/rownumber @@ -5,14 +5,19 @@ CREATE TABLE xy (x INT PRIMARY KEY, y INT) build SELECT * FROM xy WITH ORDINALITY ---- -ordinality - ├── columns: x:1(int!null) y:2(int) ordinality:3(int!null) +project + ├── columns: x:1(int!null) y:2(int) ordinality:4(int!null) ├── key: (1) - ├── fd: (1)-->(2,3), (3)-->(1,2) - ├── prune: (1,2) - └── scan xy - ├── columns: x:1(int!null) y:2(int) + ├── fd: (1)-->(2,4), (4)-->(1,2) + ├── prune: (1,2,4) + └── ordinality + ├── columns: x:1(int!null) y:2(int) crdb_internal_mvcc_timestamp:3(decimal) ordinality:4(int!null) ├── key: (1) - ├── fd: (1)-->(2) - ├── prune: (1,2) - └── interesting orderings: (+1) + ├── fd: (1)-->(2-4), (4)-->(1-3) + ├── prune: (1-3) + └── scan xy + ├── columns: x:1(int!null) y:2(int) crdb_internal_mvcc_timestamp:3(decimal) + ├── key: (1) + ├── fd: (1)-->(2,3) + ├── prune: (1-3) + └── interesting orderings: (+1) diff --git a/pkg/sql/opt/memo/testdata/logprops/scalar b/pkg/sql/opt/memo/testdata/logprops/scalar index f86d4bab143b..fb0de2d442e0 100644 --- a/pkg/sql/opt/memo/testdata/logprops/scalar +++ b/pkg/sql/opt/memo/testdata/logprops/scalar @@ -13,53 +13,59 @@ CREATE TABLE sf (s STRING, f FLOAT) build SELECT * FROM xy WHERE x < 5 ---- -select +project ├── columns: x:1(int!null) y:2(int) ├── key: (1) ├── fd: (1)-->(2) - ├── prune: (2) + ├── prune: (1,2) ├── interesting orderings: (+1) - ├── scan xy - │ ├── columns: x:1(int!null) y:2(int) - │ ├── key: (1) - │ ├── fd: (1)-->(2) - │ ├── prune: (1,2) - │ └── interesting orderings: (+1) - └── filters - └── lt [type=bool, outer=(1), constraints=(/1: (/NULL - /4]; tight)] - ├── variable: x:1 [type=int] - └── const: 5 [type=int] + └── select + ├── columns: x:1(int!null) y:2(int) crdb_internal_mvcc_timestamp:3(decimal) + ├── key: (1) + ├── fd: (1)-->(2,3) + ├── prune: (2,3) + ├── interesting orderings: (+1) + ├── scan xy + │ ├── columns: x:1(int!null) y:2(int) crdb_internal_mvcc_timestamp:3(decimal) + │ ├── key: (1) + │ ├── fd: (1)-->(2,3) + │ ├── prune: (1-3) + │ └── interesting orderings: (+1) + └── filters + └── lt [type=bool, outer=(1), constraints=(/1: (/NULL - /4]; tight)] + ├── variable: x:1 [type=int] + └── const: 5 [type=int] build SELECT xy.x + 1 = length('foo') + xy.y AS a, uv.rowid * xy.x AS b FROM xy, uv ---- project - ├── columns: a:6(bool) b:7(int!null) + ├── columns: a:8(bool) b:9(int!null) ├── immutable - ├── prune: (6,7) + ├── prune: (8,9) ├── inner-join (cross) - │ ├── columns: x:1(int!null) y:2(int) u:3(int) v:4(int!null) rowid:5(int!null) - │ ├── key: (1,5) - │ ├── fd: (1)-->(2), (5)-->(3,4) - │ ├── prune: (1-5) - │ ├── interesting orderings: (+1) (+5) + │ ├── columns: x:1(int!null) y:2(int) xy.crdb_internal_mvcc_timestamp:3(decimal) u:4(int) v:5(int!null) rowid:6(int!null) uv.crdb_internal_mvcc_timestamp:7(decimal) + │ ├── key: (1,6) + │ ├── fd: (1)-->(2,3), (6)-->(4,5,7) + │ ├── prune: (1-7) + │ ├── interesting orderings: (+1) (+6) │ ├── scan xy - │ │ ├── columns: x:1(int!null) y:2(int) + │ │ ├── columns: x:1(int!null) y:2(int) xy.crdb_internal_mvcc_timestamp:3(decimal) │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2) - │ │ ├── prune: (1,2) + │ │ ├── fd: (1)-->(2,3) + │ │ ├── prune: (1-3) │ │ ├── interesting orderings: (+1) - │ │ └── unfiltered-cols: (1,2) + │ │ └── unfiltered-cols: (1-3) │ ├── scan uv - │ │ ├── columns: u:3(int) v:4(int!null) rowid:5(int!null) - │ │ ├── key: (5) - │ │ ├── fd: (5)-->(3,4) - │ │ ├── prune: (3-5) - │ │ ├── interesting orderings: (+5) - │ │ └── unfiltered-cols: (3-5) + │ │ ├── columns: u:4(int) v:5(int!null) rowid:6(int!null) uv.crdb_internal_mvcc_timestamp:7(decimal) + │ │ ├── key: (6) + │ │ ├── fd: (6)-->(4,5,7) + │ │ ├── prune: (4-7) + │ │ ├── interesting orderings: (+6) + │ │ └── unfiltered-cols: (4-7) │ └── filters (true) └── projections - ├── eq [as=a:6, type=bool, outer=(1,2), immutable] + ├── eq [as=a:8, type=bool, outer=(1,2), immutable] │ ├── plus [type=int] │ │ ├── variable: x:1 [type=int] │ │ └── const: 1 [type=int] @@ -67,120 +73,140 @@ project │ ├── function: length [type=int] │ │ └── const: 'foo' [type=string] │ └── variable: y:2 [type=int] - └── mult [as=b:7, type=int, outer=(1,5), immutable] - ├── variable: rowid:5 [type=int] + └── mult [as=b:9, type=int, outer=(1,6), immutable] + ├── variable: rowid:6 [type=int] └── variable: x:1 [type=int] build SELECT * FROM xy WHERE EXISTS(SELECT * FROM uv WHERE u=x) ---- -select +project ├── columns: x:1(int!null) y:2(int) ├── key: (1) ├── fd: (1)-->(2) - ├── prune: (2) + ├── prune: (1,2) ├── interesting orderings: (+1) - ├── scan xy - │ ├── columns: x:1(int!null) y:2(int) - │ ├── key: (1) - │ ├── fd: (1)-->(2) - │ ├── prune: (1,2) - │ └── interesting orderings: (+1) - └── filters - └── exists [type=bool, outer=(1), correlated-subquery] - └── project - ├── columns: u:3(int!null) v:4(int!null) - ├── outer: (1) - ├── fd: ()-->(3) - ├── prune: (3,4) - └── select - ├── columns: u:3(int!null) v:4(int!null) rowid:5(int!null) + └── select + ├── columns: x:1(int!null) y:2(int) xy.crdb_internal_mvcc_timestamp:3(decimal) + ├── key: (1) + ├── fd: (1)-->(2,3) + ├── prune: (2,3) + ├── interesting orderings: (+1) + ├── scan xy + │ ├── columns: x:1(int!null) y:2(int) xy.crdb_internal_mvcc_timestamp:3(decimal) + │ ├── key: (1) + │ ├── fd: (1)-->(2,3) + │ ├── prune: (1-3) + │ └── interesting orderings: (+1) + └── filters + └── exists [type=bool, outer=(1), correlated-subquery] + └── project + ├── columns: u:4(int!null) v:5(int!null) ├── outer: (1) - ├── key: (5) - ├── fd: ()-->(3), (5)-->(4) + ├── fd: ()-->(4) ├── prune: (4,5) - ├── interesting orderings: (+5) - ├── scan uv - │ ├── columns: u:3(int) v:4(int!null) rowid:5(int!null) - │ ├── key: (5) - │ ├── fd: (5)-->(3,4) - │ ├── prune: (3-5) - │ └── interesting orderings: (+5) - └── filters - └── eq [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - ├── variable: u:3 [type=int] - └── variable: x:1 [type=int] + └── select + ├── columns: u:4(int!null) v:5(int!null) rowid:6(int!null) uv.crdb_internal_mvcc_timestamp:7(decimal) + ├── outer: (1) + ├── key: (6) + ├── fd: ()-->(4), (6)-->(5,7) + ├── prune: (5-7) + ├── interesting orderings: (+6) + ├── scan uv + │ ├── columns: u:4(int) v:5(int!null) rowid:6(int!null) uv.crdb_internal_mvcc_timestamp:7(decimal) + │ ├── key: (6) + │ ├── fd: (6)-->(4,5,7) + │ ├── prune: (4-7) + │ └── interesting orderings: (+6) + └── filters + └── eq [type=bool, outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + ├── variable: u:4 [type=int] + └── variable: x:1 [type=int] build SELECT * FROM xy WHERE y IN (SELECT v FROM uv WHERE u=x) ---- -select +project ├── columns: x:1(int!null) y:2(int) ├── key: (1) ├── fd: (1)-->(2) + ├── prune: (1,2) ├── interesting orderings: (+1) - ├── scan xy - │ ├── columns: x:1(int!null) y:2(int) - │ ├── key: (1) - │ ├── fd: (1)-->(2) - │ ├── prune: (1,2) - │ └── interesting orderings: (+1) - └── filters - └── any: eq [type=bool, outer=(1,2), correlated-subquery] - ├── project - │ ├── columns: v:4(int!null) - │ ├── outer: (1) - │ ├── prune: (4) - │ └── select - │ ├── columns: u:3(int!null) v:4(int!null) rowid:5(int!null) - │ ├── outer: (1) - │ ├── key: (5) - │ ├── fd: ()-->(3), (5)-->(4) - │ ├── prune: (4,5) - │ ├── interesting orderings: (+5) - │ ├── scan uv - │ │ ├── columns: u:3(int) v:4(int!null) rowid:5(int!null) - │ │ ├── key: (5) - │ │ ├── fd: (5)-->(3,4) - │ │ ├── prune: (3-5) - │ │ └── interesting orderings: (+5) - │ └── filters - │ └── eq [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - │ ├── variable: u:3 [type=int] - │ └── variable: x:1 [type=int] - └── variable: y:2 [type=int] + └── select + ├── columns: x:1(int!null) y:2(int) xy.crdb_internal_mvcc_timestamp:3(decimal) + ├── key: (1) + ├── fd: (1)-->(2,3) + ├── prune: (3) + ├── interesting orderings: (+1) + ├── scan xy + │ ├── columns: x:1(int!null) y:2(int) xy.crdb_internal_mvcc_timestamp:3(decimal) + │ ├── key: (1) + │ ├── fd: (1)-->(2,3) + │ ├── prune: (1-3) + │ └── interesting orderings: (+1) + └── filters + └── any: eq [type=bool, outer=(1,2), correlated-subquery] + ├── project + │ ├── columns: v:5(int!null) + │ ├── outer: (1) + │ ├── prune: (5) + │ └── select + │ ├── columns: u:4(int!null) v:5(int!null) rowid:6(int!null) uv.crdb_internal_mvcc_timestamp:7(decimal) + │ ├── outer: (1) + │ ├── key: (6) + │ ├── fd: ()-->(4), (6)-->(5,7) + │ ├── prune: (5-7) + │ ├── interesting orderings: (+6) + │ ├── scan uv + │ │ ├── columns: u:4(int) v:5(int!null) rowid:6(int!null) uv.crdb_internal_mvcc_timestamp:7(decimal) + │ │ ├── key: (6) + │ │ ├── fd: (6)-->(4,5,7) + │ │ ├── prune: (4-7) + │ │ └── interesting orderings: (+6) + │ └── filters + │ └── eq [type=bool, outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + │ ├── variable: u:4 [type=int] + │ └── variable: x:1 [type=int] + └── variable: y:2 [type=int] # Regression for 36137: need to detect correlation in 2nd Any operator argument. build SELECT * FROM xy WHERE x=1 OR y IN (SELECT v FROM uv) ---- -select +project ├── columns: x:1(int!null) y:2(int) ├── key: (1) ├── fd: (1)-->(2) + ├── prune: (1,2) ├── interesting orderings: (+1) - ├── scan xy - │ ├── columns: x:1(int!null) y:2(int) - │ ├── key: (1) - │ ├── fd: (1)-->(2) - │ ├── prune: (1,2) - │ └── interesting orderings: (+1) - └── filters - └── or [type=bool, outer=(1,2), correlated-subquery] - ├── eq [type=bool] - │ ├── variable: x:1 [type=int] - │ └── const: 1 [type=int] - └── any: eq [type=bool] - ├── project - │ ├── columns: v:4(int!null) - │ ├── prune: (4) - │ └── scan uv - │ ├── columns: u:3(int) v:4(int!null) rowid:5(int!null) - │ ├── key: (5) - │ ├── fd: (5)-->(3,4) - │ ├── prune: (3-5) - │ └── interesting orderings: (+5) - └── variable: y:2 [type=int] + └── select + ├── columns: x:1(int!null) y:2(int) xy.crdb_internal_mvcc_timestamp:3(decimal) + ├── key: (1) + ├── fd: (1)-->(2,3) + ├── prune: (3) + ├── interesting orderings: (+1) + ├── scan xy + │ ├── columns: x:1(int!null) y:2(int) xy.crdb_internal_mvcc_timestamp:3(decimal) + │ ├── key: (1) + │ ├── fd: (1)-->(2,3) + │ ├── prune: (1-3) + │ └── interesting orderings: (+1) + └── filters + └── or [type=bool, outer=(1,2), correlated-subquery] + ├── eq [type=bool] + │ ├── variable: x:1 [type=int] + │ └── const: 1 [type=int] + └── any: eq [type=bool] + ├── project + │ ├── columns: v:5(int!null) + │ ├── prune: (5) + │ └── scan uv + │ ├── columns: u:4(int) v:5(int!null) rowid:6(int!null) uv.crdb_internal_mvcc_timestamp:7(decimal) + │ ├── key: (6) + │ ├── fd: (6)-->(4,5,7) + │ ├── prune: (4-7) + │ └── interesting orderings: (+6) + └── variable: y:2 [type=int] # Side-effects: test DivOp and impure FuncOp. build @@ -191,72 +217,72 @@ ON x=u GROUP BY div ---- group-by - ├── columns: sum:7(decimal!null) div:3(decimal) - ├── grouping columns: div:3(decimal) + ├── columns: sum:9(decimal!null) div:4(decimal) + ├── grouping columns: div:4(decimal) ├── stable - ├── key: (3) - ├── fd: (3)-->(7) - ├── prune: (7) + ├── key: (4) + ├── fd: (4)-->(9) + ├── prune: (9) ├── project - │ ├── columns: x:1(int!null) div:3(decimal) + │ ├── columns: x:1(int!null) div:4(decimal) │ ├── stable - │ ├── fd: (1)-->(3) - │ ├── prune: (1,3) + │ ├── fd: (1)-->(4) + │ ├── prune: (1,4) │ ├── interesting orderings: (+1) │ └── inner-join (hash) - │ ├── columns: x:1(int!null) y:2(int) div:3(decimal) u:4(int!null) v:5(int!null) + │ ├── columns: x:1(int!null) y:2(int) div:4(decimal) u:5(int!null) v:6(int!null) │ ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) │ ├── stable - │ ├── fd: (1)-->(2,3), (1)==(4), (4)==(1) - │ ├── prune: (2,3,5) + │ ├── fd: (1)-->(2,4), (1)==(5), (5)==(1) + │ ├── prune: (2,4,6) │ ├── interesting orderings: (+1) │ ├── project - │ │ ├── columns: div:3(decimal) x:1(int!null) y:2(int) + │ │ ├── columns: div:4(decimal) x:1(int!null) y:2(int) │ │ ├── immutable │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2,3) - │ │ ├── prune: (1-3) + │ │ ├── fd: (1)-->(2,4) + │ │ ├── prune: (1,2,4) │ │ ├── interesting orderings: (+1) - │ │ ├── unfiltered-cols: (1,2) + │ │ ├── unfiltered-cols: (1-3) │ │ ├── scan xy - │ │ │ ├── columns: x:1(int!null) y:2(int) + │ │ │ ├── columns: x:1(int!null) y:2(int) xy.crdb_internal_mvcc_timestamp:3(decimal) │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(2) - │ │ │ ├── prune: (1,2) + │ │ │ ├── fd: (1)-->(2,3) + │ │ │ ├── prune: (1-3) │ │ │ ├── interesting orderings: (+1) - │ │ │ └── unfiltered-cols: (1,2) + │ │ │ └── unfiltered-cols: (1-3) │ │ └── projections - │ │ └── div [as=div:3, type=decimal, outer=(1,2), immutable] + │ │ └── div [as=div:4, type=decimal, outer=(1,2), immutable] │ │ ├── variable: x:1 [type=int] │ │ └── variable: y:2 [type=int] │ ├── project - │ │ ├── columns: u:4(int) v:5(int!null) + │ │ ├── columns: u:5(int) v:6(int!null) │ │ ├── stable - │ │ ├── prune: (4,5) + │ │ ├── prune: (5,6) │ │ └── select - │ │ ├── columns: u:4(int) v:5(int!null) rowid:6(int!null) + │ │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) uv.crdb_internal_mvcc_timestamp:8(decimal) │ │ ├── stable - │ │ ├── key: (6) - │ │ ├── fd: (6)-->(4,5) - │ │ ├── prune: (4-6) - │ │ ├── interesting orderings: (+6) + │ │ ├── key: (7) + │ │ ├── fd: (7)-->(5,6,8) + │ │ ├── prune: (5-8) + │ │ ├── interesting orderings: (+7) │ │ ├── scan uv - │ │ │ ├── columns: u:4(int) v:5(int!null) rowid:6(int!null) - │ │ │ ├── key: (6) - │ │ │ ├── fd: (6)-->(4,5) - │ │ │ ├── prune: (4-6) - │ │ │ └── interesting orderings: (+6) + │ │ │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) uv.crdb_internal_mvcc_timestamp:8(decimal) + │ │ │ ├── key: (7) + │ │ │ ├── fd: (7)-->(5,6,8) + │ │ │ ├── prune: (5-8) + │ │ │ └── interesting orderings: (+7) │ │ └── filters │ │ └── gt [type=bool, stable] │ │ ├── function: now [type=timestamptz] │ │ └── cast: TIMESTAMPTZ [type=timestamptz] │ │ └── const: '2018-01-01' [type=string] │ └── filters - │ └── eq [type=bool, outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + │ └── eq [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] │ ├── variable: x:1 [type=int] - │ └── variable: u:4 [type=int] + │ └── variable: u:5 [type=int] └── aggregations - └── sum [as=sum:7, type=decimal, outer=(1)] + └── sum [as=sum:9, type=decimal, outer=(1)] └── variable: x:1 [type=int] # Verify that we don't mark the division as side-effecting when the right-hand @@ -265,24 +291,24 @@ build SELECT x / 1, x::float / 2.0, x::decimal / 3.0 FROM xy ---- project - ├── columns: "?column?":3(decimal!null) "?column?":4(float!null) "?column?":5(decimal!null) + ├── columns: "?column?":4(decimal!null) "?column?":5(float!null) "?column?":6(decimal!null) ├── immutable - ├── prune: (3-5) + ├── prune: (4-6) ├── scan xy - │ ├── columns: x:1(int!null) y:2(int) + │ ├── columns: x:1(int!null) y:2(int) crdb_internal_mvcc_timestamp:3(decimal) │ ├── key: (1) - │ ├── fd: (1)-->(2) - │ ├── prune: (1,2) + │ ├── fd: (1)-->(2,3) + │ ├── prune: (1-3) │ └── interesting orderings: (+1) └── projections - ├── div [as="?column?":3, type=decimal, outer=(1)] + ├── div [as="?column?":4, type=decimal, outer=(1)] │ ├── variable: x:1 [type=int] │ └── const: 1 [type=int] - ├── div [as="?column?":4, type=float, outer=(1), immutable] + ├── div [as="?column?":5, type=float, outer=(1), immutable] │ ├── cast: FLOAT8 [type=float] │ │ └── variable: x:1 [type=int] │ └── const: 2.0 [type=float] - └── div [as="?column?":5, type=decimal, outer=(1), immutable] + └── div [as="?column?":6, type=decimal, outer=(1), immutable] ├── cast: DECIMAL [type=decimal] │ └── variable: x:1 [type=int] └── const: 3.0 [type=decimal] @@ -292,61 +318,61 @@ build SELECT s::TIMESTAMP FROM sf ---- project - ├── columns: s:4(timestamp) + ├── columns: s:5(timestamp) ├── stable - ├── prune: (4) + ├── prune: (5) ├── scan sf - │ ├── columns: sf.s:1(string) f:2(float) rowid:3(int!null) + │ ├── columns: sf.s:1(string) f:2(float) rowid:3(int!null) crdb_internal_mvcc_timestamp:4(decimal) │ ├── key: (3) - │ ├── fd: (3)-->(1,2) - │ ├── prune: (1-3) + │ ├── fd: (3)-->(1,2,4) + │ ├── prune: (1-4) │ └── interesting orderings: (+3) └── projections - └── cast: TIMESTAMP [as=s:4, type=timestamp, outer=(1), stable] + └── cast: TIMESTAMP [as=s:5, type=timestamp, outer=(1), stable] └── variable: sf.s:1 [type=string] build SELECT f::STRING FROM sf ---- project - ├── columns: f:4(string) + ├── columns: f:5(string) ├── stable - ├── prune: (4) + ├── prune: (5) ├── scan sf - │ ├── columns: s:1(string) sf.f:2(float) rowid:3(int!null) + │ ├── columns: s:1(string) sf.f:2(float) rowid:3(int!null) crdb_internal_mvcc_timestamp:4(decimal) │ ├── key: (3) - │ ├── fd: (3)-->(1,2) - │ ├── prune: (1-3) + │ ├── fd: (3)-->(1,2,4) + │ ├── prune: (1-4) │ └── interesting orderings: (+3) └── projections - └── cast: STRING [as=f:4, type=string, outer=(2), stable] + └── cast: STRING [as=f:5, type=string, outer=(2), stable] └── variable: sf.f:2 [type=float] build SELECT ARRAY(SELECT f FROM sf)::STRING[] ---- project - ├── columns: array:4(string[]) + ├── columns: array:5(string[]) ├── cardinality: [1 - 1] ├── stable ├── key: () - ├── fd: ()-->(4) - ├── prune: (4) + ├── fd: ()-->(5) + ├── prune: (5) ├── values │ ├── cardinality: [1 - 1] │ ├── key: () │ └── tuple [type=tuple] └── projections - └── cast: STRING[] [as=array:4, type=string[], stable, subquery] + └── cast: STRING[] [as=array:5, type=string[], stable, subquery] └── array-flatten [type=float[]] └── project ├── columns: f:2(float) ├── prune: (2) └── scan sf - ├── columns: s:1(string) f:2(float) rowid:3(int!null) + ├── columns: s:1(string) f:2(float) rowid:3(int!null) crdb_internal_mvcc_timestamp:4(decimal) ├── key: (3) - ├── fd: (3)-->(1,2) - ├── prune: (1-3) + ├── fd: (3)-->(1,2,4) + ├── prune: (1-4) └── interesting orderings: (+3) # Regression test for #50258: cast from unknown to tuple. diff --git a/pkg/sql/opt/memo/testdata/logprops/scan b/pkg/sql/opt/memo/testdata/logprops/scan index 544ac8eeb83f..aabb7b9d7fb7 100644 --- a/pkg/sql/opt/memo/testdata/logprops/scan +++ b/pkg/sql/opt/memo/testdata/logprops/scan @@ -9,12 +9,18 @@ CREATE TABLE b (x INT, z INT NOT NULL) build SELECT * FROM a ---- -scan a +project ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) ├── key: (1) ├── fd: (1)-->(2-4), (3,4)~~>(1,2) ├── prune: (1-4) - └── interesting orderings: (+1) (-3,+4,+1) + ├── interesting orderings: (+1) (-3,+4,+1) + └── scan a + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) + ├── key: (1) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + ├── prune: (1-5) + └── interesting orderings: (+1) (-3,+4,+1) build SELECT * FROM b @@ -23,10 +29,10 @@ project ├── columns: x:1(int) z:2(int!null) ├── prune: (1,2) └── scan b - ├── columns: x:1(int) z:2(int!null) rowid:3(int!null) + ├── columns: x:1(int) z:2(int!null) rowid:3(int!null) crdb_internal_mvcc_timestamp:4(decimal) ├── key: (3) - ├── fd: (3)-->(1,2) - ├── prune: (1-3) + ├── fd: (3)-->(1,2,4) + ├── prune: (1-4) └── interesting orderings: (+3) # Select subset of columns. @@ -107,11 +113,11 @@ opt SELECT 1 FROM t WHERE a > 1 AND a < 2 ---- values - ├── columns: "?column?":5(int!null) + ├── columns: "?column?":6(int!null) ├── cardinality: [0 - 0] ├── key: () - ├── fd: ()-->(5) - └── prune: (5) + ├── fd: ()-->(6) + └── prune: (6) opt SELECT * FROM t@bc WHERE b IN ('a', 'b') AND c IN (1, 2) AND a IN (2, 3) @@ -203,14 +209,21 @@ select build SELECT * FROM a FOR UPDATE ---- -scan a +project ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) - ├── locking: for-update ├── volatile ├── key: (1) ├── fd: (1)-->(2-4), (3,4)~~>(1,2) ├── prune: (1-4) - └── interesting orderings: (+1) (-3,+4,+1) + ├── interesting orderings: (+1) (-3,+4,+1) + └── scan a + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) + ├── locking: for-update + ├── volatile + ├── key: (1) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + ├── prune: (1-5) + └── interesting orderings: (+1) (-3,+4,+1) exec-ddl CREATE TABLE kab ( @@ -226,13 +239,19 @@ CREATE TABLE kab ( build SELECT * FROM kab ---- -scan kab +project ├── columns: k:1(int!null) a:2(int!null) b:3(int!null) - ├── check constraint expressions - │ └── eq [type=bool, outer=(3), constraints=(/3: [/0 - /0]; tight), fd=()-->(3)] - │ ├── variable: b:3 [type=int] - │ └── const: 0 [type=int] ├── key: (1) ├── fd: ()-->(3), (1)-->(2) ├── prune: (1-3) - └── interesting orderings: (+1) (+3,+2,+1) + ├── interesting orderings: (+1) (+3,+2,+1) + └── scan kab + ├── columns: k:1(int!null) a:2(int!null) b:3(int!null) crdb_internal_mvcc_timestamp:4(decimal) + ├── check constraint expressions + │ └── eq [type=bool, outer=(3), constraints=(/3: [/0 - /0]; tight), fd=()-->(3)] + │ ├── variable: b:3 [type=int] + │ └── const: 0 [type=int] + ├── key: (1) + ├── fd: ()-->(3), (1)-->(2,4) + ├── prune: (1-4) + └── interesting orderings: (+1) (+3,+2,+1) diff --git a/pkg/sql/opt/memo/testdata/logprops/select b/pkg/sql/opt/memo/testdata/logprops/select index 2fe9f952f01a..ce60e9395ea4 100644 --- a/pkg/sql/opt/memo/testdata/logprops/select +++ b/pkg/sql/opt/memo/testdata/logprops/select @@ -9,150 +9,183 @@ CREATE TABLE kuv (k INT PRIMARY KEY, u FLOAT, v STRING) build SELECT * FROM xy WHERE x=1 ---- -select +project ├── columns: x:1(int!null) y:2(int) ├── cardinality: [0 - 1] ├── key: () ├── fd: ()-->(1,2) - ├── prune: (2) + ├── prune: (1,2) ├── interesting orderings: (+1) - ├── scan xy - │ ├── columns: x:1(int!null) y:2(int) - │ ├── key: (1) - │ ├── fd: (1)-->(2) - │ ├── prune: (1,2) - │ └── interesting orderings: (+1) - └── filters - └── eq [type=bool, outer=(1), constraints=(/1: [/1 - /1]; tight), fd=()-->(1)] - ├── variable: x:1 [type=int] - └── const: 1 [type=int] + └── select + ├── columns: x:1(int!null) y:2(int) crdb_internal_mvcc_timestamp:3(decimal) + ├── cardinality: [0 - 1] + ├── key: () + ├── fd: ()-->(1-3) + ├── prune: (2,3) + ├── interesting orderings: (+1) + ├── scan xy + │ ├── columns: x:1(int!null) y:2(int) crdb_internal_mvcc_timestamp:3(decimal) + │ ├── key: (1) + │ ├── fd: (1)-->(2,3) + │ ├── prune: (1-3) + │ └── interesting orderings: (+1) + └── filters + └── eq [type=bool, outer=(1), constraints=(/1: [/1 - /1]; tight), fd=()-->(1)] + ├── variable: x:1 [type=int] + └── const: 1 [type=int] build SELECT * FROM xy,kuv WHERE xy.x=kuv.k ---- -select - ├── columns: x:1(int!null) y:2(int) k:3(int!null) u:4(float) v:5(string) - ├── key: (3) - ├── fd: (1)-->(2), (3)-->(4,5), (1)==(3), (3)==(1) - ├── prune: (2,4,5) - ├── interesting orderings: (+1) (+3) - ├── inner-join (cross) - │ ├── columns: x:1(int!null) y:2(int) k:3(int!null) u:4(float) v:5(string) - │ ├── key: (1,3) - │ ├── fd: (1)-->(2), (3)-->(4,5) - │ ├── prune: (1-5) - │ ├── interesting orderings: (+1) (+3) - │ ├── scan xy - │ │ ├── columns: x:1(int!null) y:2(int) - │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2) - │ │ ├── prune: (1,2) - │ │ ├── interesting orderings: (+1) - │ │ └── unfiltered-cols: (1,2) - │ ├── scan kuv - │ │ ├── columns: k:3(int!null) u:4(float) v:5(string) - │ │ ├── key: (3) - │ │ ├── fd: (3)-->(4,5) - │ │ ├── prune: (3-5) - │ │ ├── interesting orderings: (+3) - │ │ └── unfiltered-cols: (3-5) - │ └── filters (true) - └── filters - └── eq [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - ├── variable: x:1 [type=int] - └── variable: k:3 [type=int] +project + ├── columns: x:1(int!null) y:2(int) k:4(int!null) u:5(float) v:6(string) + ├── key: (4) + ├── fd: (1)-->(2), (4)-->(5,6), (1)==(4), (4)==(1) + ├── prune: (1,2,4-6) + ├── interesting orderings: (+1) (+4) + └── select + ├── columns: x:1(int!null) y:2(int) xy.crdb_internal_mvcc_timestamp:3(decimal) k:4(int!null) u:5(float) v:6(string) kuv.crdb_internal_mvcc_timestamp:7(decimal) + ├── key: (4) + ├── fd: (1)-->(2,3), (4)-->(5-7), (1)==(4), (4)==(1) + ├── prune: (2,3,5-7) + ├── interesting orderings: (+1) (+4) + ├── inner-join (cross) + │ ├── columns: x:1(int!null) y:2(int) xy.crdb_internal_mvcc_timestamp:3(decimal) k:4(int!null) u:5(float) v:6(string) kuv.crdb_internal_mvcc_timestamp:7(decimal) + │ ├── key: (1,4) + │ ├── fd: (1)-->(2,3), (4)-->(5-7) + │ ├── prune: (1-7) + │ ├── interesting orderings: (+1) (+4) + │ ├── scan xy + │ │ ├── columns: x:1(int!null) y:2(int) xy.crdb_internal_mvcc_timestamp:3(decimal) + │ │ ├── key: (1) + │ │ ├── fd: (1)-->(2,3) + │ │ ├── prune: (1-3) + │ │ ├── interesting orderings: (+1) + │ │ └── unfiltered-cols: (1-3) + │ ├── scan kuv + │ │ ├── columns: k:4(int!null) u:5(float) v:6(string) kuv.crdb_internal_mvcc_timestamp:7(decimal) + │ │ ├── key: (4) + │ │ ├── fd: (4)-->(5-7) + │ │ ├── prune: (4-7) + │ │ ├── interesting orderings: (+4) + │ │ └── unfiltered-cols: (4-7) + │ └── filters (true) + └── filters + └── eq [type=bool, outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + ├── variable: x:1 [type=int] + └── variable: k:4 [type=int] # Propagate outer columns. build SELECT * FROM xy WHERE EXISTS(SELECT * FROM (SELECT * FROM kuv WHERE k=y) WHERE k=x) ---- -select +project ├── columns: x:1(int!null) y:2(int) ├── key: (1) ├── fd: (1)-->(2) + ├── prune: (1,2) ├── interesting orderings: (+1) - ├── scan xy - │ ├── columns: x:1(int!null) y:2(int) - │ ├── key: (1) - │ ├── fd: (1)-->(2) - │ ├── prune: (1,2) - │ └── interesting orderings: (+1) - └── filters - └── exists [type=bool, outer=(1,2), correlated-subquery] - └── select - ├── columns: k:3(int!null) u:4(float) v:5(string) - ├── outer: (1,2) - ├── cardinality: [0 - 1] - ├── key: () - ├── fd: ()-->(3-5) - ├── prune: (4,5) - ├── interesting orderings: (+3) - ├── select - │ ├── columns: k:3(int!null) u:4(float) v:5(string) - │ ├── outer: (2) - │ ├── cardinality: [0 - 1] - │ ├── key: () - │ ├── fd: ()-->(3-5) - │ ├── prune: (4,5) - │ ├── interesting orderings: (+3) - │ ├── scan kuv - │ │ ├── columns: k:3(int!null) u:4(float) v:5(string) - │ │ ├── key: (3) - │ │ ├── fd: (3)-->(4,5) - │ │ ├── prune: (3-5) - │ │ └── interesting orderings: (+3) - │ └── filters - │ └── eq [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ]), fd=(2)==(3), (3)==(2)] - │ ├── variable: k:3 [type=int] - │ └── variable: y:2 [type=int] - └── filters - └── eq [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - ├── variable: k:3 [type=int] - └── variable: x:1 [type=int] + └── select + ├── columns: x:1(int!null) y:2(int) xy.crdb_internal_mvcc_timestamp:3(decimal) + ├── key: (1) + ├── fd: (1)-->(2,3) + ├── prune: (3) + ├── interesting orderings: (+1) + ├── scan xy + │ ├── columns: x:1(int!null) y:2(int) xy.crdb_internal_mvcc_timestamp:3(decimal) + │ ├── key: (1) + │ ├── fd: (1)-->(2,3) + │ ├── prune: (1-3) + │ └── interesting orderings: (+1) + └── filters + └── exists [type=bool, outer=(1,2), correlated-subquery] + └── select + ├── columns: k:4(int!null) u:5(float) v:6(string) + ├── outer: (1,2) + ├── cardinality: [0 - 1] + ├── key: () + ├── fd: ()-->(4-6) + ├── prune: (5,6) + ├── interesting orderings: (+4) + ├── project + │ ├── columns: k:4(int!null) u:5(float) v:6(string) + │ ├── outer: (2) + │ ├── cardinality: [0 - 1] + │ ├── key: () + │ ├── fd: ()-->(4-6) + │ ├── prune: (4-6) + │ ├── interesting orderings: (+4) + │ └── select + │ ├── columns: k:4(int!null) u:5(float) v:6(string) kuv.crdb_internal_mvcc_timestamp:7(decimal) + │ ├── outer: (2) + │ ├── cardinality: [0 - 1] + │ ├── key: () + │ ├── fd: ()-->(4-7) + │ ├── prune: (5-7) + │ ├── interesting orderings: (+4) + │ ├── scan kuv + │ │ ├── columns: k:4(int!null) u:5(float) v:6(string) kuv.crdb_internal_mvcc_timestamp:7(decimal) + │ │ ├── key: (4) + │ │ ├── fd: (4)-->(5-7) + │ │ ├── prune: (4-7) + │ │ └── interesting orderings: (+4) + │ └── filters + │ └── eq [type=bool, outer=(2,4), constraints=(/2: (/NULL - ]; /4: (/NULL - ]), fd=(2)==(4), (4)==(2)] + │ ├── variable: k:4 [type=int] + │ └── variable: y:2 [type=int] + └── filters + └── eq [type=bool, outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + ├── variable: k:4 [type=int] + └── variable: x:1 [type=int] # Reduce min cardinality. build SELECT count(*) FROM xy HAVING count(*) = 5 ---- select - ├── columns: count:3(int!null) + ├── columns: count:4(int!null) ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(3) + ├── fd: ()-->(4) ├── scalar-group-by - │ ├── columns: count_rows:3(int!null) + │ ├── columns: count_rows:4(int!null) │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(3) - │ ├── prune: (3) + │ ├── fd: ()-->(4) + │ ├── prune: (4) │ ├── project │ │ └── scan xy - │ │ ├── columns: x:1(int!null) y:2(int) + │ │ ├── columns: x:1(int!null) y:2(int) crdb_internal_mvcc_timestamp:3(decimal) │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2) - │ │ ├── prune: (1,2) + │ │ ├── fd: (1)-->(2,3) + │ │ ├── prune: (1-3) │ │ └── interesting orderings: (+1) │ └── aggregations - │ └── count-rows [as=count_rows:3, type=int] + │ └── count-rows [as=count_rows:4, type=int] └── filters - └── eq [type=bool, outer=(3), constraints=(/3: [/5 - /5]; tight), fd=()-->(3)] - ├── variable: count_rows:3 [type=int] + └── eq [type=bool, outer=(4), constraints=(/4: [/5 - /5]; tight), fd=()-->(4)] + ├── variable: count_rows:4 [type=int] └── const: 5 [type=int] build SELECT * FROM xy WITH ORDINALITY ---- -ordinality - ├── columns: x:1(int!null) y:2(int) ordinality:3(int!null) +project + ├── columns: x:1(int!null) y:2(int) ordinality:4(int!null) ├── key: (1) - ├── fd: (1)-->(2,3), (3)-->(1,2) - ├── prune: (1,2) - └── scan xy - ├── columns: x:1(int!null) y:2(int) + ├── fd: (1)-->(2,4), (4)-->(1,2) + ├── prune: (1,2,4) + └── ordinality + ├── columns: x:1(int!null) y:2(int) crdb_internal_mvcc_timestamp:3(decimal) ordinality:4(int!null) ├── key: (1) - ├── fd: (1)-->(2) - ├── prune: (1,2) - └── interesting orderings: (+1) + ├── fd: (1)-->(2-4), (4)-->(1-3) + ├── prune: (1-3) + └── scan xy + ├── columns: x:1(int!null) y:2(int) crdb_internal_mvcc_timestamp:3(decimal) + ├── key: (1) + ├── fd: (1)-->(2,3) + ├── prune: (1-3) + └── interesting orderings: (+1) # Verify not-null column deduction from constraints. exec-ddl @@ -166,16 +199,16 @@ project ├── columns: a:1(int!null) b:2(int!null) c:3(int) d:4(int) ├── prune: (1-4) └── select - ├── columns: a:1(int!null) b:2(int!null) c:3(int) d:4(int) rowid:5(int!null) + ├── columns: a:1(int!null) b:2(int!null) c:3(int) d:4(int) rowid:5(int!null) crdb_internal_mvcc_timestamp:6(decimal) ├── key: (5) - ├── fd: (5)-->(1-4) - ├── prune: (1-5) + ├── fd: (5)-->(1-4,6) + ├── prune: (1-6) ├── interesting orderings: (+5) ├── scan abcd - │ ├── columns: a:1(int!null) b:2(int!null) c:3(int) d:4(int) rowid:5(int!null) + │ ├── columns: a:1(int!null) b:2(int!null) c:3(int) d:4(int) rowid:5(int!null) crdb_internal_mvcc_timestamp:6(decimal) │ ├── key: (5) - │ ├── fd: (5)-->(1-4) - │ ├── prune: (1-5) + │ ├── fd: (5)-->(1-4,6) + │ ├── prune: (1-6) │ └── interesting orderings: (+5) └── filters └── true [type=bool] @@ -187,16 +220,16 @@ project ├── columns: a:1(int!null) b:2(int!null) c:3(int!null) d:4(int) ├── prune: (1-4) └── select - ├── columns: a:1(int!null) b:2(int!null) c:3(int!null) d:4(int) rowid:5(int!null) + ├── columns: a:1(int!null) b:2(int!null) c:3(int!null) d:4(int) rowid:5(int!null) crdb_internal_mvcc_timestamp:6(decimal) ├── key: (5) - ├── fd: (5)-->(1-4) - ├── prune: (1,2,4,5) + ├── fd: (5)-->(1-4,6) + ├── prune: (1,2,4-6) ├── interesting orderings: (+5) ├── scan abcd - │ ├── columns: a:1(int!null) b:2(int!null) c:3(int) d:4(int) rowid:5(int!null) + │ ├── columns: a:1(int!null) b:2(int!null) c:3(int) d:4(int) rowid:5(int!null) crdb_internal_mvcc_timestamp:6(decimal) │ ├── key: (5) - │ ├── fd: (5)-->(1-4) - │ ├── prune: (1-5) + │ ├── fd: (5)-->(1-4,6) + │ ├── prune: (1-6) │ └── interesting orderings: (+5) └── filters └── is-not [type=bool, outer=(3), constraints=(/3: (/NULL - ]; tight)] @@ -211,16 +244,16 @@ project ├── fd: (3)==(4), (4)==(3) ├── prune: (1-4) └── select - ├── columns: a:1(int!null) b:2(int!null) c:3(int!null) d:4(int!null) rowid:5(int!null) + ├── columns: a:1(int!null) b:2(int!null) c:3(int!null) d:4(int!null) rowid:5(int!null) crdb_internal_mvcc_timestamp:6(decimal) ├── key: (5) - ├── fd: (5)-->(1-4), (3)==(4), (4)==(3) - ├── prune: (1,2,5) + ├── fd: (5)-->(1-4,6), (3)==(4), (4)==(3) + ├── prune: (1,2,5,6) ├── interesting orderings: (+5) ├── scan abcd - │ ├── columns: a:1(int!null) b:2(int!null) c:3(int) d:4(int) rowid:5(int!null) + │ ├── columns: a:1(int!null) b:2(int!null) c:3(int) d:4(int) rowid:5(int!null) crdb_internal_mvcc_timestamp:6(decimal) │ ├── key: (5) - │ ├── fd: (5)-->(1-4) - │ ├── prune: (1-5) + │ ├── fd: (5)-->(1-4,6) + │ ├── prune: (1-6) │ └── interesting orderings: (+5) └── filters └── eq [type=bool, outer=(3,4), constraints=(/3: (/NULL - ]; /4: (/NULL - ]), fd=(3)==(4), (4)==(3)] @@ -234,16 +267,16 @@ project ├── columns: a:1(int!null) b:2(int!null) c:3(int!null) d:4(int) ├── prune: (1-4) └── select - ├── columns: a:1(int!null) b:2(int!null) c:3(int!null) d:4(int) rowid:5(int!null) + ├── columns: a:1(int!null) b:2(int!null) c:3(int!null) d:4(int) rowid:5(int!null) crdb_internal_mvcc_timestamp:6(decimal) ├── key: (5) - ├── fd: (5)-->(1-4) - ├── prune: (2,4,5) + ├── fd: (5)-->(1-4,6) + ├── prune: (2,4-6) ├── interesting orderings: (+5) ├── scan abcd - │ ├── columns: a:1(int!null) b:2(int!null) c:3(int) d:4(int) rowid:5(int!null) + │ ├── columns: a:1(int!null) b:2(int!null) c:3(int) d:4(int) rowid:5(int!null) crdb_internal_mvcc_timestamp:6(decimal) │ ├── key: (5) - │ ├── fd: (5)-->(1-4) - │ ├── prune: (1-5) + │ ├── fd: (5)-->(1-4,6) + │ ├── prune: (1-6) │ └── interesting orderings: (+5) └── filters └── gt [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] @@ -262,16 +295,16 @@ select │ ├── fd: (1)==(3), (3)==(1) │ ├── prune: (1-4) │ └── select - │ ├── columns: a:1(int!null) b:2(int!null) c:3(int!null) d:4(int) rowid:5(int!null) + │ ├── columns: a:1(int!null) b:2(int!null) c:3(int!null) d:4(int) rowid:5(int!null) crdb_internal_mvcc_timestamp:6(decimal) │ ├── key: (5) - │ ├── fd: (5)-->(1-4), (1)==(3), (3)==(1) - │ ├── prune: (2,4,5) + │ ├── fd: (5)-->(1-4,6), (1)==(3), (3)==(1) + │ ├── prune: (2,4-6) │ ├── interesting orderings: (+5) │ ├── scan abcd - │ │ ├── columns: a:1(int!null) b:2(int!null) c:3(int) d:4(int) rowid:5(int!null) + │ │ ├── columns: a:1(int!null) b:2(int!null) c:3(int) d:4(int) rowid:5(int!null) crdb_internal_mvcc_timestamp:6(decimal) │ │ ├── key: (5) - │ │ ├── fd: (5)-->(1-4) - │ │ ├── prune: (1-5) + │ │ ├── fd: (5)-->(1-4,6) + │ │ ├── prune: (1-6) │ │ └── interesting orderings: (+5) │ └── filters │ └── eq [type=bool, outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] @@ -290,55 +323,55 @@ project ├── columns: a:1(int!null) b:2(int!null) c:3(int) d:4(int) ├── prune: (1-4) └── select - ├── columns: a:1(int!null) b:2(int!null) c:3(int) d:4(int) rowid:5(int!null) + ├── columns: a:1(int!null) b:2(int!null) c:3(int) d:4(int) rowid:5(int!null) abcd.crdb_internal_mvcc_timestamp:6(decimal) ├── key: (5) - ├── fd: (5)-->(1-4) - ├── prune: (1,3-5) + ├── fd: (5)-->(1-4,6) + ├── prune: (1,3-6) ├── interesting orderings: (+5) ├── scan abcd - │ ├── columns: a:1(int!null) b:2(int!null) c:3(int) d:4(int) rowid:5(int!null) + │ ├── columns: a:1(int!null) b:2(int!null) c:3(int) d:4(int) rowid:5(int!null) abcd.crdb_internal_mvcc_timestamp:6(decimal) │ ├── key: (5) - │ ├── fd: (5)-->(1-4) - │ ├── prune: (1-5) + │ ├── fd: (5)-->(1-4,6) + │ ├── prune: (1-6) │ └── interesting orderings: (+5) └── filters └── gt [type=bool, outer=(2), correlated-subquery] ├── subquery [type=int] │ └── max1-row - │ ├── columns: count_rows:8(int!null) + │ ├── columns: count_rows:10(int!null) │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── outer: (2) │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(8) + │ ├── fd: ()-->(10) │ └── scalar-group-by - │ ├── columns: count_rows:8(int!null) + │ ├── columns: count_rows:10(int!null) │ ├── outer: (2) │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(8) - │ ├── prune: (8) + │ ├── fd: ()-->(10) + │ ├── prune: (10) │ ├── project │ │ ├── outer: (2) │ │ └── select - │ │ ├── columns: x:6(int!null) y:7(int!null) + │ │ ├── columns: x:7(int!null) y:8(int!null) xy.crdb_internal_mvcc_timestamp:9(decimal) │ │ ├── outer: (2) - │ │ ├── key: (6) - │ │ ├── fd: ()-->(7) - │ │ ├── prune: (6) - │ │ ├── interesting orderings: (+6) + │ │ ├── key: (7) + │ │ ├── fd: ()-->(8), (7)-->(9) + │ │ ├── prune: (7,9) + │ │ ├── interesting orderings: (+7) │ │ ├── scan xy - │ │ │ ├── columns: x:6(int!null) y:7(int) - │ │ │ ├── key: (6) - │ │ │ ├── fd: (6)-->(7) - │ │ │ ├── prune: (6,7) - │ │ │ └── interesting orderings: (+6) + │ │ │ ├── columns: x:7(int!null) y:8(int) xy.crdb_internal_mvcc_timestamp:9(decimal) + │ │ │ ├── key: (7) + │ │ │ ├── fd: (7)-->(8,9) + │ │ │ ├── prune: (7-9) + │ │ │ └── interesting orderings: (+7) │ │ └── filters - │ │ └── eq [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - │ │ ├── variable: y:7 [type=int] + │ │ └── eq [type=bool, outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] + │ │ ├── variable: y:8 [type=int] │ │ └── variable: b:2 [type=int] │ └── aggregations - │ └── count-rows [as=count_rows:8, type=int] + │ └── count-rows [as=count_rows:10, type=int] └── const: 0 [type=int] # Sequences always have a single row when selected from. diff --git a/pkg/sql/opt/memo/testdata/logprops/set b/pkg/sql/opt/memo/testdata/logprops/set index 18f8a6464a32..d3413de1f9ba 100644 --- a/pkg/sql/opt/memo/testdata/logprops/set +++ b/pkg/sql/opt/memo/testdata/logprops/set @@ -10,25 +10,31 @@ build SELECT * FROM xy UNION SELECT * FROM uv ---- union - ├── columns: x:6(int) y:7(int) + ├── columns: x:8(int) y:9(int) ├── left columns: xy.x:1(int) xy.y:2(int) - ├── right columns: u:3(int) v:4(int) - ├── key: (6,7) - ├── scan xy + ├── right columns: u:4(int) v:5(int) + ├── key: (8,9) + ├── project │ ├── columns: xy.x:1(int!null) xy.y:2(int) │ ├── key: (1) │ ├── fd: (1)-->(2) │ ├── prune: (1,2) - │ └── interesting orderings: (+1) + │ ├── interesting orderings: (+1) + │ └── scan xy + │ ├── columns: xy.x:1(int!null) xy.y:2(int) xy.crdb_internal_mvcc_timestamp:3(decimal) + │ ├── key: (1) + │ ├── fd: (1)-->(2,3) + │ ├── prune: (1-3) + │ └── interesting orderings: (+1) └── project - ├── columns: u:3(int) v:4(int!null) - ├── prune: (3,4) + ├── columns: u:4(int) v:5(int!null) + ├── prune: (4,5) └── scan uv - ├── columns: u:3(int) v:4(int!null) rowid:5(int!null) - ├── key: (5) - ├── fd: (5)-->(3,4) - ├── prune: (3-5) - └── interesting orderings: (+5) + ├── columns: u:4(int) v:5(int!null) rowid:6(int!null) uv.crdb_internal_mvcc_timestamp:7(decimal) + ├── key: (6) + ├── fd: (6)-->(4,5,7) + ├── prune: (4-7) + └── interesting orderings: (+6) build SELECT x, y, x FROM xy INTERSECT SELECT v, u, rowid FROM (SELECT *, rowid FROM uv WHERE u=1) uv @@ -36,30 +42,42 @@ SELECT x, y, x FROM xy INTERSECT SELECT v, u, rowid FROM (SELECT *, rowid FROM u intersect ├── columns: x:1(int!null) y:2(int) x:1(int!null) ├── left columns: x:1(int!null) y:2(int) x:1(int!null) - ├── right columns: v:4(int) u:3(int) rowid:5(int) + ├── right columns: v:5(int) u:4(int) rowid:6(int) ├── key: (1,2) - ├── scan xy + ├── project │ ├── columns: x:1(int!null) y:2(int) │ ├── key: (1) │ ├── fd: (1)-->(2) │ ├── prune: (1,2) - │ └── interesting orderings: (+1) - └── select - ├── columns: u:3(int!null) v:4(int!null) rowid:5(int!null) - ├── key: (5) - ├── fd: ()-->(3), (5)-->(4) - ├── prune: (4,5) - ├── interesting orderings: (+5) - ├── scan uv - │ ├── columns: u:3(int) v:4(int!null) rowid:5(int!null) - │ ├── key: (5) - │ ├── fd: (5)-->(3,4) - │ ├── prune: (3-5) - │ └── interesting orderings: (+5) - └── filters - └── eq [type=bool, outer=(3), constraints=(/3: [/1 - /1]; tight), fd=()-->(3)] - ├── variable: u:3 [type=int] - └── const: 1 [type=int] + │ ├── interesting orderings: (+1) + │ └── scan xy + │ ├── columns: x:1(int!null) y:2(int) xy.crdb_internal_mvcc_timestamp:3(decimal) + │ ├── key: (1) + │ ├── fd: (1)-->(2,3) + │ ├── prune: (1-3) + │ └── interesting orderings: (+1) + └── project + ├── columns: u:4(int!null) v:5(int!null) rowid:6(int!null) + ├── key: (6) + ├── fd: ()-->(4), (6)-->(5) + ├── prune: (4-6) + ├── interesting orderings: (+6) + └── select + ├── columns: u:4(int!null) v:5(int!null) rowid:6(int!null) uv.crdb_internal_mvcc_timestamp:7(decimal) + ├── key: (6) + ├── fd: ()-->(4), (6)-->(5,7) + ├── prune: (5-7) + ├── interesting orderings: (+6) + ├── scan uv + │ ├── columns: u:4(int) v:5(int!null) rowid:6(int!null) uv.crdb_internal_mvcc_timestamp:7(decimal) + │ ├── key: (6) + │ ├── fd: (6)-->(4,5,7) + │ ├── prune: (4-7) + │ └── interesting orderings: (+6) + └── filters + └── eq [type=bool, outer=(4), constraints=(/4: [/1 - /1]; tight), fd=()-->(4)] + ├── variable: u:4 [type=int] + └── const: 1 [type=int] build SELECT x, x, y FROM xy EXCEPT SELECT u, v, v FROM (SELECT * FROM uv WHERE u=1) uv @@ -67,104 +85,118 @@ SELECT x, x, y FROM xy EXCEPT SELECT u, v, v FROM (SELECT * FROM uv WHERE u=1) u except ├── columns: x:1(int!null) x:1(int!null) y:2(int) ├── left columns: x:1(int!null) x:1(int!null) y:2(int) - ├── right columns: u:3(int) v:4(int) v:4(int) + ├── right columns: u:4(int) v:5(int) v:5(int) ├── key: (1,2) - ├── scan xy + ├── project │ ├── columns: x:1(int!null) y:2(int) │ ├── key: (1) │ ├── fd: (1)-->(2) │ ├── prune: (1,2) - │ └── interesting orderings: (+1) + │ ├── interesting orderings: (+1) + │ └── scan xy + │ ├── columns: x:1(int!null) y:2(int) xy.crdb_internal_mvcc_timestamp:3(decimal) + │ ├── key: (1) + │ ├── fd: (1)-->(2,3) + │ ├── prune: (1-3) + │ └── interesting orderings: (+1) └── project - ├── columns: u:3(int!null) v:4(int!null) - ├── fd: ()-->(3) - ├── prune: (3,4) + ├── columns: u:4(int!null) v:5(int!null) + ├── fd: ()-->(4) + ├── prune: (4,5) └── select - ├── columns: u:3(int!null) v:4(int!null) rowid:5(int!null) - ├── key: (5) - ├── fd: ()-->(3), (5)-->(4) - ├── prune: (4,5) - ├── interesting orderings: (+5) + ├── columns: u:4(int!null) v:5(int!null) rowid:6(int!null) uv.crdb_internal_mvcc_timestamp:7(decimal) + ├── key: (6) + ├── fd: ()-->(4), (6)-->(5,7) + ├── prune: (5-7) + ├── interesting orderings: (+6) ├── scan uv - │ ├── columns: u:3(int) v:4(int!null) rowid:5(int!null) - │ ├── key: (5) - │ ├── fd: (5)-->(3,4) - │ ├── prune: (3-5) - │ └── interesting orderings: (+5) + │ ├── columns: u:4(int) v:5(int!null) rowid:6(int!null) uv.crdb_internal_mvcc_timestamp:7(decimal) + │ ├── key: (6) + │ ├── fd: (6)-->(4,5,7) + │ ├── prune: (4-7) + │ └── interesting orderings: (+6) └── filters - └── eq [type=bool, outer=(3), constraints=(/3: [/1 - /1]; tight), fd=()-->(3)] - ├── variable: u:3 [type=int] + └── eq [type=bool, outer=(4), constraints=(/4: [/1 - /1]; tight), fd=()-->(4)] + ├── variable: u:4 [type=int] └── const: 1 [type=int] # Propagate outer columns. build SELECT * FROM xy WHERE (SELECT x, u FROM uv UNION SELECT y, v FROM uv) = (1, 2) ---- -select +project ├── columns: x:1(int!null) y:2(int) ├── immutable ├── key: (1) ├── fd: (1)-->(2) + ├── prune: (1,2) ├── interesting orderings: (+1) - ├── scan xy - │ ├── columns: xy.x:1(int!null) xy.y:2(int) - │ ├── key: (1) - │ ├── fd: (1)-->(2) - │ ├── prune: (1,2) - │ └── interesting orderings: (+1) - └── filters - └── eq [type=bool, outer=(1,2), immutable, correlated-subquery] - ├── subquery [type=tuple{int, int}] - │ └── max1-row - │ ├── columns: column13:13(tuple{int, int}) - │ ├── error: "more than one row returned by a subquery used as an expression" - │ ├── outer: (1,2) - │ ├── cardinality: [0 - 1] - │ ├── key: () - │ ├── fd: ()-->(13) - │ └── project - │ ├── columns: column13:13(tuple{int, int}) - │ ├── outer: (1,2) - │ ├── prune: (13) - │ ├── union - │ │ ├── columns: x:11(int) u:12(int) - │ │ ├── left columns: x:6(int) uv.u:3(int) - │ │ ├── right columns: y:10(int) v:8(int) - │ │ ├── outer: (1,2) - │ │ ├── key: (11,12) - │ │ ├── project - │ │ │ ├── columns: x:6(int) uv.u:3(int) - │ │ │ ├── outer: (1) - │ │ │ ├── fd: ()-->(6) - │ │ │ ├── prune: (3,6) - │ │ │ ├── scan uv - │ │ │ │ ├── columns: uv.u:3(int) v:4(int!null) rowid:5(int!null) - │ │ │ │ ├── key: (5) - │ │ │ │ ├── fd: (5)-->(3,4) - │ │ │ │ ├── prune: (3-5) - │ │ │ │ └── interesting orderings: (+5) - │ │ │ └── projections - │ │ │ └── variable: xy.x:1 [as=x:6, type=int, outer=(1)] - │ │ └── project - │ │ ├── columns: y:10(int) v:8(int!null) - │ │ ├── outer: (2) - │ │ ├── fd: ()-->(10) - │ │ ├── prune: (8,10) - │ │ ├── scan uv - │ │ │ ├── columns: uv.u:7(int) v:8(int!null) rowid:9(int!null) - │ │ │ ├── key: (9) - │ │ │ ├── fd: (9)-->(7,8) - │ │ │ ├── prune: (7-9) - │ │ │ └── interesting orderings: (+9) - │ │ └── projections - │ │ └── variable: xy.y:2 [as=y:10, type=int, outer=(2)] - │ └── projections - │ └── tuple [as=column13:13, type=tuple{int, int}, outer=(11,12)] - │ ├── variable: x:11 [type=int] - │ └── variable: u:12 [type=int] - └── tuple [type=tuple{int, int}] - ├── const: 1 [type=int] - └── const: 2 [type=int] + └── select + ├── columns: xy.x:1(int!null) xy.y:2(int) xy.crdb_internal_mvcc_timestamp:3(decimal) + ├── immutable + ├── key: (1) + ├── fd: (1)-->(2,3) + ├── prune: (3) + ├── interesting orderings: (+1) + ├── scan xy + │ ├── columns: xy.x:1(int!null) xy.y:2(int) xy.crdb_internal_mvcc_timestamp:3(decimal) + │ ├── key: (1) + │ ├── fd: (1)-->(2,3) + │ ├── prune: (1-3) + │ └── interesting orderings: (+1) + └── filters + └── eq [type=bool, outer=(1,2), immutable, correlated-subquery] + ├── subquery [type=tuple{int, int}] + │ └── max1-row + │ ├── columns: column16:16(tuple{int, int}) + │ ├── error: "more than one row returned by a subquery used as an expression" + │ ├── outer: (1,2) + │ ├── cardinality: [0 - 1] + │ ├── key: () + │ ├── fd: ()-->(16) + │ └── project + │ ├── columns: column16:16(tuple{int, int}) + │ ├── outer: (1,2) + │ ├── prune: (16) + │ ├── union + │ │ ├── columns: x:14(int) u:15(int) + │ │ ├── left columns: x:8(int) uv.u:4(int) + │ │ ├── right columns: y:13(int) v:10(int) + │ │ ├── outer: (1,2) + │ │ ├── key: (14,15) + │ │ ├── project + │ │ │ ├── columns: x:8(int) uv.u:4(int) + │ │ │ ├── outer: (1) + │ │ │ ├── fd: ()-->(8) + │ │ │ ├── prune: (4,8) + │ │ │ ├── scan uv + │ │ │ │ ├── columns: uv.u:4(int) v:5(int!null) rowid:6(int!null) uv.crdb_internal_mvcc_timestamp:7(decimal) + │ │ │ │ ├── key: (6) + │ │ │ │ ├── fd: (6)-->(4,5,7) + │ │ │ │ ├── prune: (4-7) + │ │ │ │ └── interesting orderings: (+6) + │ │ │ └── projections + │ │ │ └── variable: xy.x:1 [as=x:8, type=int, outer=(1)] + │ │ └── project + │ │ ├── columns: y:13(int) v:10(int!null) + │ │ ├── outer: (2) + │ │ ├── fd: ()-->(13) + │ │ ├── prune: (10,13) + │ │ ├── scan uv + │ │ │ ├── columns: uv.u:9(int) v:10(int!null) rowid:11(int!null) uv.crdb_internal_mvcc_timestamp:12(decimal) + │ │ │ ├── key: (11) + │ │ │ ├── fd: (11)-->(9,10,12) + │ │ │ ├── prune: (9-12) + │ │ │ └── interesting orderings: (+11) + │ │ └── projections + │ │ └── variable: xy.y:2 [as=y:13, type=int, outer=(2)] + │ └── projections + │ └── tuple [as=column16:16, type=tuple{int, int}, outer=(14,15)] + │ ├── variable: x:14 [type=int] + │ └── variable: u:15 [type=int] + └── tuple [type=tuple{int, int}] + ├── const: 1 [type=int] + └── const: 2 [type=int] # Calculate union cardinality. build diff --git a/pkg/sql/opt/memo/testdata/logprops/srfs b/pkg/sql/opt/memo/testdata/logprops/srfs index e68146c4854e..86ced86fa732 100644 --- a/pkg/sql/opt/memo/testdata/logprops/srfs +++ b/pkg/sql/opt/memo/testdata/logprops/srfs @@ -10,7 +10,7 @@ opt SELECT generate_series(0,1) FROM (SELECT * FROM xy LIMIT 0) ---- project-set - ├── columns: generate_series:3(int) + ├── columns: generate_series:4(int) ├── cardinality: [0 - 0] ├── immutable ├── values @@ -25,70 +25,70 @@ opt SELECT (SELECT unnest(ARRAY[1,2,y,v]) FROM xy WHERE x = u) FROM uv ---- project - ├── columns: unnest:7(int) - ├── prune: (7) + ├── columns: unnest:9(int) + ├── prune: (9) ├── ensure-distinct-on - │ ├── columns: rowid:3(int!null) unnest:6(int) + │ ├── columns: rowid:3(int!null) unnest:8(int) │ ├── grouping columns: rowid:3(int!null) │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── key: (3) - │ ├── fd: (3)-->(6) - │ ├── prune: (6) + │ ├── fd: (3)-->(8) + │ ├── prune: (8) │ ├── left-join-apply - │ │ ├── columns: u:1(int) v:2(int!null) rowid:3(int!null) x:4(int) y:5(int) unnest:6(int) + │ │ ├── columns: u:1(int) v:2(int!null) rowid:3(int!null) x:5(int) y:6(int) unnest:8(int) │ │ ├── fd: (3)-->(1,2) - │ │ ├── prune: (3,6) - │ │ ├── reject-nulls: (4-6) - │ │ ├── interesting orderings: (+3) (+4) + │ │ ├── prune: (3,8) + │ │ ├── reject-nulls: (5,6,8) + │ │ ├── interesting orderings: (+3) (+5) │ │ ├── scan uv │ │ │ ├── columns: u:1(int) v:2(int!null) rowid:3(int!null) │ │ │ ├── key: (3) │ │ │ ├── fd: (3)-->(1,2) │ │ │ ├── prune: (1-3) │ │ │ ├── interesting orderings: (+3) - │ │ │ └── unfiltered-cols: (1-3) + │ │ │ └── unfiltered-cols: (1-4) │ │ ├── inner-join-apply - │ │ │ ├── columns: x:4(int!null) y:5(int) unnest:6(int) + │ │ │ ├── columns: x:5(int!null) y:6(int) unnest:8(int) │ │ │ ├── outer: (1,2) │ │ │ ├── cardinality: [0 - 4] - │ │ │ ├── fd: ()-->(4,5) - │ │ │ ├── prune: (6) - │ │ │ ├── interesting orderings: (+4) + │ │ │ ├── fd: ()-->(5,6) + │ │ │ ├── prune: (8) + │ │ │ ├── interesting orderings: (+5) │ │ │ ├── select - │ │ │ │ ├── columns: x:4(int!null) y:5(int) + │ │ │ │ ├── columns: x:5(int!null) y:6(int) │ │ │ │ ├── outer: (1) │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ ├── key: () - │ │ │ │ ├── fd: ()-->(4,5) - │ │ │ │ ├── prune: (5) - │ │ │ │ ├── interesting orderings: (+4) + │ │ │ │ ├── fd: ()-->(5,6) + │ │ │ │ ├── prune: (6) + │ │ │ │ ├── interesting orderings: (+5) │ │ │ │ ├── scan xy - │ │ │ │ │ ├── columns: x:4(int!null) y:5(int) - │ │ │ │ │ ├── key: (4) - │ │ │ │ │ ├── fd: (4)-->(5) - │ │ │ │ │ ├── prune: (4,5) - │ │ │ │ │ └── interesting orderings: (+4) + │ │ │ │ │ ├── columns: x:5(int!null) y:6(int) + │ │ │ │ │ ├── key: (5) + │ │ │ │ │ ├── fd: (5)-->(6) + │ │ │ │ │ ├── prune: (5,6) + │ │ │ │ │ └── interesting orderings: (+5) │ │ │ │ └── filters - │ │ │ │ └── eq [type=bool, outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] - │ │ │ │ ├── variable: x:4 [type=int] + │ │ │ │ └── eq [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + │ │ │ │ ├── variable: x:5 [type=int] │ │ │ │ └── variable: u:1 [type=int] │ │ │ ├── values - │ │ │ │ ├── columns: unnest:6(int) - │ │ │ │ ├── outer: (2,5) + │ │ │ │ ├── columns: unnest:8(int) + │ │ │ │ ├── outer: (2,6) │ │ │ │ ├── cardinality: [4 - 4] - │ │ │ │ ├── prune: (6) + │ │ │ │ ├── prune: (8) │ │ │ │ ├── tuple [type=tuple{int}] │ │ │ │ │ └── const: 1 [type=int] │ │ │ │ ├── tuple [type=tuple{int}] │ │ │ │ │ └── const: 2 [type=int] │ │ │ │ ├── tuple [type=tuple{int}] - │ │ │ │ │ └── variable: y:5 [type=int] + │ │ │ │ │ └── variable: y:6 [type=int] │ │ │ │ └── tuple [type=tuple{int}] │ │ │ │ └── variable: v:2 [type=int] │ │ │ └── filters (true) │ │ └── filters (true) │ └── aggregations - │ └── const-agg [as=unnest:6, type=int, outer=(6)] - │ └── variable: unnest:6 [type=int] + │ └── const-agg [as=unnest:8, type=int, outer=(8)] + │ └── variable: unnest:8 [type=int] └── projections - └── variable: unnest:6 [as=unnest:7, type=int, outer=(6)] + └── variable: unnest:8 [as=unnest:9, type=int, outer=(8)] diff --git a/pkg/sql/opt/memo/testdata/logprops/update b/pkg/sql/opt/memo/testdata/logprops/update index 36b0d1fca4c3..7c646dd55ce2 100644 --- a/pkg/sql/opt/memo/testdata/logprops/update +++ b/pkg/sql/opt/memo/testdata/logprops/update @@ -22,64 +22,64 @@ UPDATE abcde SET b=10 WHERE a=1 ---- update abcde ├── columns: - ├── fetch columns: a:7(int) b:8(int) c:9(int) d:10(int) rowid:11(int) e:12(int) + ├── fetch columns: a:8(int) b:9(int) c:10(int) d:11(int) rowid:12(int) e:13(int) ├── update-mapping: - │ ├── b_new:13 => b:2 - │ ├── column15:15 => d:4 - │ └── column14:14 => e:6 + │ ├── b_new:15 => b:2 + │ ├── column17:17 => d:4 + │ └── column16:16 => e:6 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: column15:15(int!null) a:7(int!null) b:8(int) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) b_new:13(int!null) column14:14(int!null) + ├── columns: column17:17(int!null) a:8(int!null) b:9(int) c:10(int!null) d:11(int) rowid:12(int!null) e:13(int) crdb_internal_mvcc_timestamp:14(decimal) b_new:15(int!null) column16:16(int!null) ├── immutable - ├── key: (11) - ├── fd: ()-->(7,13,14), (11)-->(8-10,12), (9)-->(15) - ├── prune: (7-15) - ├── interesting orderings: (+11) + ├── key: (12) + ├── fd: ()-->(8,15,16), (12)-->(9-11,13,14), (10)-->(17) + ├── prune: (8-17) + ├── interesting orderings: (+12) ├── project - │ ├── columns: column14:14(int!null) a:7(int!null) b:8(int) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) b_new:13(int!null) - │ ├── key: (11) - │ ├── fd: ()-->(7,13,14), (11)-->(8-10,12) - │ ├── prune: (7-14) - │ ├── interesting orderings: (+11) + │ ├── columns: column16:16(int!null) a:8(int!null) b:9(int) c:10(int!null) d:11(int) rowid:12(int!null) e:13(int) crdb_internal_mvcc_timestamp:14(decimal) b_new:15(int!null) + │ ├── key: (12) + │ ├── fd: ()-->(8,15,16), (12)-->(9-11,13,14) + │ ├── prune: (8-16) + │ ├── interesting orderings: (+12) │ ├── project - │ │ ├── columns: b_new:13(int!null) a:7(int!null) b:8(int) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) - │ │ ├── key: (11) - │ │ ├── fd: ()-->(7,13), (11)-->(8-10,12) - │ │ ├── prune: (7-13) - │ │ ├── interesting orderings: (+11) + │ │ ├── columns: b_new:15(int!null) a:8(int!null) b:9(int) c:10(int!null) d:11(int) rowid:12(int!null) e:13(int) crdb_internal_mvcc_timestamp:14(decimal) + │ │ ├── key: (12) + │ │ ├── fd: ()-->(8,15), (12)-->(9-11,13,14) + │ │ ├── prune: (8-15) + │ │ ├── interesting orderings: (+12) │ │ ├── select - │ │ │ ├── columns: a:7(int!null) b:8(int) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) - │ │ │ ├── key: (11) - │ │ │ ├── fd: ()-->(7), (11)-->(8-10,12) - │ │ │ ├── prune: (8-12) - │ │ │ ├── interesting orderings: (+11) + │ │ │ ├── columns: a:8(int!null) b:9(int) c:10(int!null) d:11(int) rowid:12(int!null) e:13(int) crdb_internal_mvcc_timestamp:14(decimal) + │ │ │ ├── key: (12) + │ │ │ ├── fd: ()-->(8), (12)-->(9-11,13,14) + │ │ │ ├── prune: (9-14) + │ │ │ ├── interesting orderings: (+12) │ │ │ ├── scan abcde - │ │ │ │ ├── columns: a:7(int!null) b:8(int) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) + │ │ │ │ ├── columns: a:8(int!null) b:9(int) c:10(int!null) d:11(int) rowid:12(int!null) e:13(int) crdb_internal_mvcc_timestamp:14(decimal) │ │ │ │ ├── computed column expressions - │ │ │ │ │ └── d:10 + │ │ │ │ │ └── d:11 │ │ │ │ │ └── plus [type=int] │ │ │ │ │ ├── plus [type=int] - │ │ │ │ │ │ ├── variable: b:8 [type=int] - │ │ │ │ │ │ └── variable: c:9 [type=int] + │ │ │ │ │ │ ├── variable: b:9 [type=int] + │ │ │ │ │ │ └── variable: c:10 [type=int] │ │ │ │ │ └── const: 1 [type=int] - │ │ │ │ ├── key: (11) - │ │ │ │ ├── fd: (11)-->(7-10,12) - │ │ │ │ ├── prune: (7-12) - │ │ │ │ └── interesting orderings: (+11) + │ │ │ │ ├── key: (12) + │ │ │ │ ├── fd: (12)-->(8-11,13,14) + │ │ │ │ ├── prune: (8-14) + │ │ │ │ └── interesting orderings: (+12) │ │ │ └── filters - │ │ │ └── eq [type=bool, outer=(7), constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] - │ │ │ ├── variable: a:7 [type=int] + │ │ │ └── eq [type=bool, outer=(8), constraints=(/8: [/1 - /1]; tight), fd=()-->(8)] + │ │ │ ├── variable: a:8 [type=int] │ │ │ └── const: 1 [type=int] │ │ └── projections - │ │ └── const: 10 [as=b_new:13, type=int] + │ │ └── const: 10 [as=b_new:15, type=int] │ └── projections - │ └── const: 0 [as=column14:14, type=int] + │ └── const: 0 [as=column16:16, type=int] └── projections - └── plus [as=column15:15, type=int, outer=(9,13), immutable] + └── plus [as=column17:17, type=int, outer=(10,15), immutable] ├── plus [type=int] - │ ├── variable: b_new:13 [type=int] - │ └── variable: c:9 [type=int] + │ ├── variable: b_new:15 [type=int] + │ └── variable: c:10 [type=int] └── const: 1 [type=int] # Properties with RETURNING clause. @@ -93,65 +93,65 @@ project ├── prune: (1-4) └── update abcde ├── columns: a:1(int!null) b:2(int!null) c:3(int!null) d:4(int!null) rowid:5(int!null) - ├── fetch columns: a:7(int) b:8(int) c:9(int) d:10(int) rowid:11(int) e:12(int) + ├── fetch columns: a:8(int) b:9(int) c:10(int) d:11(int) rowid:12(int) e:13(int) ├── update-mapping: - │ ├── b_new:13 => b:2 - │ ├── column15:15 => d:4 - │ └── column14:14 => e:6 + │ ├── b_new:15 => b:2 + │ ├── column17:17 => d:4 + │ └── column16:16 => e:6 ├── volatile, mutations ├── key: (5) ├── fd: ()-->(1,2), (5)-->(3,4), (3)-->(4) └── project - ├── columns: column15:15(int!null) a:7(int!null) b:8(int) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) b_new:13(int!null) column14:14(int!null) + ├── columns: column17:17(int!null) a:8(int!null) b:9(int) c:10(int!null) d:11(int) rowid:12(int!null) e:13(int) crdb_internal_mvcc_timestamp:14(decimal) b_new:15(int!null) column16:16(int!null) ├── immutable - ├── key: (11) - ├── fd: ()-->(7,13,14), (11)-->(8-10,12), (9)-->(15) - ├── prune: (7-15) - ├── interesting orderings: (+11) + ├── key: (12) + ├── fd: ()-->(8,15,16), (12)-->(9-11,13,14), (10)-->(17) + ├── prune: (8-17) + ├── interesting orderings: (+12) ├── project - │ ├── columns: column14:14(int!null) a:7(int!null) b:8(int) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) b_new:13(int!null) - │ ├── key: (11) - │ ├── fd: ()-->(7,13,14), (11)-->(8-10,12) - │ ├── prune: (7-14) - │ ├── interesting orderings: (+11) + │ ├── columns: column16:16(int!null) a:8(int!null) b:9(int) c:10(int!null) d:11(int) rowid:12(int!null) e:13(int) crdb_internal_mvcc_timestamp:14(decimal) b_new:15(int!null) + │ ├── key: (12) + │ ├── fd: ()-->(8,15,16), (12)-->(9-11,13,14) + │ ├── prune: (8-16) + │ ├── interesting orderings: (+12) │ ├── project - │ │ ├── columns: b_new:13(int!null) a:7(int!null) b:8(int) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) - │ │ ├── key: (11) - │ │ ├── fd: ()-->(7,13), (11)-->(8-10,12) - │ │ ├── prune: (7-13) - │ │ ├── interesting orderings: (+11) + │ │ ├── columns: b_new:15(int!null) a:8(int!null) b:9(int) c:10(int!null) d:11(int) rowid:12(int!null) e:13(int) crdb_internal_mvcc_timestamp:14(decimal) + │ │ ├── key: (12) + │ │ ├── fd: ()-->(8,15), (12)-->(9-11,13,14) + │ │ ├── prune: (8-15) + │ │ ├── interesting orderings: (+12) │ │ ├── select - │ │ │ ├── columns: a:7(int!null) b:8(int) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) - │ │ │ ├── key: (11) - │ │ │ ├── fd: ()-->(7), (11)-->(8-10,12) - │ │ │ ├── prune: (8-12) - │ │ │ ├── interesting orderings: (+11) + │ │ │ ├── columns: a:8(int!null) b:9(int) c:10(int!null) d:11(int) rowid:12(int!null) e:13(int) crdb_internal_mvcc_timestamp:14(decimal) + │ │ │ ├── key: (12) + │ │ │ ├── fd: ()-->(8), (12)-->(9-11,13,14) + │ │ │ ├── prune: (9-14) + │ │ │ ├── interesting orderings: (+12) │ │ │ ├── scan abcde - │ │ │ │ ├── columns: a:7(int!null) b:8(int) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) + │ │ │ │ ├── columns: a:8(int!null) b:9(int) c:10(int!null) d:11(int) rowid:12(int!null) e:13(int) crdb_internal_mvcc_timestamp:14(decimal) │ │ │ │ ├── computed column expressions - │ │ │ │ │ └── d:10 + │ │ │ │ │ └── d:11 │ │ │ │ │ └── plus [type=int] │ │ │ │ │ ├── plus [type=int] - │ │ │ │ │ │ ├── variable: b:8 [type=int] - │ │ │ │ │ │ └── variable: c:9 [type=int] + │ │ │ │ │ │ ├── variable: b:9 [type=int] + │ │ │ │ │ │ └── variable: c:10 [type=int] │ │ │ │ │ └── const: 1 [type=int] - │ │ │ │ ├── key: (11) - │ │ │ │ ├── fd: (11)-->(7-10,12) - │ │ │ │ ├── prune: (7-12) - │ │ │ │ └── interesting orderings: (+11) + │ │ │ │ ├── key: (12) + │ │ │ │ ├── fd: (12)-->(8-11,13,14) + │ │ │ │ ├── prune: (8-14) + │ │ │ │ └── interesting orderings: (+12) │ │ │ └── filters - │ │ │ └── eq [type=bool, outer=(7), constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] - │ │ │ ├── variable: a:7 [type=int] + │ │ │ └── eq [type=bool, outer=(8), constraints=(/8: [/1 - /1]; tight), fd=()-->(8)] + │ │ │ ├── variable: a:8 [type=int] │ │ │ └── const: 1 [type=int] │ │ └── projections - │ │ └── const: 10 [as=b_new:13, type=int] + │ │ └── const: 10 [as=b_new:15, type=int] │ └── projections - │ └── const: 0 [as=column14:14, type=int] + │ └── const: 0 [as=column16:16, type=int] └── projections - └── plus [as=column15:15, type=int, outer=(9,13), immutable] + └── plus [as=column17:17, type=int, outer=(10,15), immutable] ├── plus [type=int] - │ ├── variable: b_new:13 [type=int] - │ └── variable: c:9 [type=int] + │ ├── variable: b_new:15 [type=int] + │ └── variable: c:10 [type=int] └── const: 1 [type=int] # Input is cardinality 0-1 expression. @@ -167,70 +167,70 @@ project ├── prune: (1-4) └── update abcde ├── columns: a:1(int!null) b:2(int!null) c:3(int!null) d:4(int!null) rowid:5(int!null) - ├── fetch columns: a:7(int) b:8(int) c:9(int) d:10(int) rowid:11(int) e:12(int) + ├── fetch columns: a:8(int) b:9(int) c:10(int) d:11(int) rowid:12(int) e:13(int) ├── update-mapping: - │ ├── b_new:13 => b:2 - │ ├── column15:15 => d:4 - │ └── column14:14 => e:6 + │ ├── b_new:15 => b:2 + │ ├── column17:17 => d:4 + │ └── column16:16 => e:6 ├── cardinality: [0 - 1] ├── volatile, mutations ├── key: () ├── fd: ()-->(1-5) └── project - ├── columns: column15:15(int!null) a:7(int!null) b:8(int) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) b_new:13(int!null) column14:14(int!null) + ├── columns: column17:17(int!null) a:8(int!null) b:9(int) c:10(int!null) d:11(int) rowid:12(int!null) e:13(int) crdb_internal_mvcc_timestamp:14(decimal) b_new:15(int!null) column16:16(int!null) ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(7-15) - ├── prune: (7-15) - ├── interesting orderings: (+11) + ├── fd: ()-->(8-17) + ├── prune: (8-17) + ├── interesting orderings: (+12) ├── project - │ ├── columns: column14:14(int!null) a:7(int!null) b:8(int) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) b_new:13(int!null) + │ ├── columns: column16:16(int!null) a:8(int!null) b:9(int) c:10(int!null) d:11(int) rowid:12(int!null) e:13(int) crdb_internal_mvcc_timestamp:14(decimal) b_new:15(int!null) │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(7-14) - │ ├── prune: (7-14) - │ ├── interesting orderings: (+11) + │ ├── fd: ()-->(8-16) + │ ├── prune: (8-16) + │ ├── interesting orderings: (+12) │ ├── project - │ │ ├── columns: b_new:13(int!null) a:7(int!null) b:8(int) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) + │ │ ├── columns: b_new:15(int!null) a:8(int!null) b:9(int) c:10(int!null) d:11(int) rowid:12(int!null) e:13(int) crdb_internal_mvcc_timestamp:14(decimal) │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(7-13) - │ │ ├── prune: (7-13) - │ │ ├── interesting orderings: (+11) + │ │ ├── fd: ()-->(8-15) + │ │ ├── prune: (8-15) + │ │ ├── interesting orderings: (+12) │ │ ├── select - │ │ │ ├── columns: a:7(int!null) b:8(int) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) + │ │ │ ├── columns: a:8(int!null) b:9(int) c:10(int!null) d:11(int) rowid:12(int!null) e:13(int) crdb_internal_mvcc_timestamp:14(decimal) │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(7-12) - │ │ │ ├── prune: (7-10,12) - │ │ │ ├── interesting orderings: (+11) + │ │ │ ├── fd: ()-->(8-14) + │ │ │ ├── prune: (8-11,13,14) + │ │ │ ├── interesting orderings: (+12) │ │ │ ├── scan abcde - │ │ │ │ ├── columns: a:7(int!null) b:8(int) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) + │ │ │ │ ├── columns: a:8(int!null) b:9(int) c:10(int!null) d:11(int) rowid:12(int!null) e:13(int) crdb_internal_mvcc_timestamp:14(decimal) │ │ │ │ ├── computed column expressions - │ │ │ │ │ └── d:10 + │ │ │ │ │ └── d:11 │ │ │ │ │ └── plus [type=int] │ │ │ │ │ ├── plus [type=int] - │ │ │ │ │ │ ├── variable: b:8 [type=int] - │ │ │ │ │ │ └── variable: c:9 [type=int] + │ │ │ │ │ │ ├── variable: b:9 [type=int] + │ │ │ │ │ │ └── variable: c:10 [type=int] │ │ │ │ │ └── const: 1 [type=int] - │ │ │ │ ├── key: (11) - │ │ │ │ ├── fd: (11)-->(7-10,12) - │ │ │ │ ├── prune: (7-12) - │ │ │ │ └── interesting orderings: (+11) + │ │ │ │ ├── key: (12) + │ │ │ │ ├── fd: (12)-->(8-11,13,14) + │ │ │ │ ├── prune: (8-14) + │ │ │ │ └── interesting orderings: (+12) │ │ │ └── filters - │ │ │ └── eq [type=bool, outer=(11), constraints=(/11: [/1 - /1]; tight), fd=()-->(11)] - │ │ │ ├── variable: rowid:11 [type=int] + │ │ │ └── eq [type=bool, outer=(12), constraints=(/12: [/1 - /1]; tight), fd=()-->(12)] + │ │ │ ├── variable: rowid:12 [type=int] │ │ │ └── const: 1 [type=int] │ │ └── projections - │ │ └── const: 10 [as=b_new:13, type=int] + │ │ └── const: 10 [as=b_new:15, type=int] │ └── projections - │ └── const: 0 [as=column14:14, type=int] + │ └── const: 0 [as=column16:16, type=int] └── projections - └── plus [as=column15:15, type=int, outer=(9,13), immutable] + └── plus [as=column17:17, type=int, outer=(10,15), immutable] ├── plus [type=int] - │ ├── variable: b_new:13 [type=int] - │ └── variable: c:9 [type=int] + │ ├── variable: b_new:15 [type=int] + │ └── variable: c:10 [type=int] └── const: 1 [type=int] # Filter FD set. @@ -244,63 +244,63 @@ project ├── prune: (1-4) └── update abcde ├── columns: a:1(int!null) b:2(int!null) c:3(int!null) d:4(int!null) rowid:5(int!null) - ├── fetch columns: a:7(int) b:8(int) c:9(int) d:10(int) rowid:11(int) e:12(int) + ├── fetch columns: a:8(int) b:9(int) c:10(int) d:11(int) rowid:12(int) e:13(int) ├── update-mapping: - │ ├── a_new:13 => a:1 - │ ├── column15:15 => d:4 - │ └── column14:14 => e:6 + │ ├── a_new:15 => a:1 + │ ├── column17:17 => d:4 + │ └── column16:16 => e:6 ├── volatile, mutations ├── key: (5) ├── fd: ()-->(1), (2)==(3), (3)==(2), (5)-->(2-4), (2)-->(4) └── project - ├── columns: column15:15(int!null) a:7(int!null) b:8(int!null) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) a_new:13(int!null) column14:14(int!null) + ├── columns: column17:17(int!null) a:8(int!null) b:9(int!null) c:10(int!null) d:11(int) rowid:12(int!null) e:13(int) crdb_internal_mvcc_timestamp:14(decimal) a_new:15(int!null) column16:16(int!null) ├── immutable - ├── key: (11) - ├── fd: ()-->(13,14), (11)-->(7-10,12), (8)==(9), (9)==(8), (9)-->(15) - ├── prune: (7-15) - ├── interesting orderings: (+11) + ├── key: (12) + ├── fd: ()-->(15,16), (12)-->(8-11,13,14), (9)==(10), (10)==(9), (10)-->(17) + ├── prune: (8-17) + ├── interesting orderings: (+12) ├── project - │ ├── columns: column14:14(int!null) a:7(int!null) b:8(int!null) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) a_new:13(int!null) - │ ├── key: (11) - │ ├── fd: ()-->(13,14), (11)-->(7-10,12), (8)==(9), (9)==(8) - │ ├── prune: (7-14) - │ ├── interesting orderings: (+11) + │ ├── columns: column16:16(int!null) a:8(int!null) b:9(int!null) c:10(int!null) d:11(int) rowid:12(int!null) e:13(int) crdb_internal_mvcc_timestamp:14(decimal) a_new:15(int!null) + │ ├── key: (12) + │ ├── fd: ()-->(15,16), (12)-->(8-11,13,14), (9)==(10), (10)==(9) + │ ├── prune: (8-16) + │ ├── interesting orderings: (+12) │ ├── project - │ │ ├── columns: a_new:13(int!null) a:7(int!null) b:8(int!null) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) - │ │ ├── key: (11) - │ │ ├── fd: ()-->(13), (11)-->(7-10,12), (8)==(9), (9)==(8) - │ │ ├── prune: (7-13) - │ │ ├── interesting orderings: (+11) + │ │ ├── columns: a_new:15(int!null) a:8(int!null) b:9(int!null) c:10(int!null) d:11(int) rowid:12(int!null) e:13(int) crdb_internal_mvcc_timestamp:14(decimal) + │ │ ├── key: (12) + │ │ ├── fd: ()-->(15), (12)-->(8-11,13,14), (9)==(10), (10)==(9) + │ │ ├── prune: (8-15) + │ │ ├── interesting orderings: (+12) │ │ ├── select - │ │ │ ├── columns: a:7(int!null) b:8(int!null) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) - │ │ │ ├── key: (11) - │ │ │ ├── fd: (11)-->(7-10,12), (8)==(9), (9)==(8) - │ │ │ ├── prune: (7,10-12) - │ │ │ ├── interesting orderings: (+11) + │ │ │ ├── columns: a:8(int!null) b:9(int!null) c:10(int!null) d:11(int) rowid:12(int!null) e:13(int) crdb_internal_mvcc_timestamp:14(decimal) + │ │ │ ├── key: (12) + │ │ │ ├── fd: (12)-->(8-11,13,14), (9)==(10), (10)==(9) + │ │ │ ├── prune: (8,11-14) + │ │ │ ├── interesting orderings: (+12) │ │ │ ├── scan abcde - │ │ │ │ ├── columns: a:7(int!null) b:8(int) c:9(int!null) d:10(int) rowid:11(int!null) e:12(int) + │ │ │ │ ├── columns: a:8(int!null) b:9(int) c:10(int!null) d:11(int) rowid:12(int!null) e:13(int) crdb_internal_mvcc_timestamp:14(decimal) │ │ │ │ ├── computed column expressions - │ │ │ │ │ └── d:10 + │ │ │ │ │ └── d:11 │ │ │ │ │ └── plus [type=int] │ │ │ │ │ ├── plus [type=int] - │ │ │ │ │ │ ├── variable: b:8 [type=int] - │ │ │ │ │ │ └── variable: c:9 [type=int] + │ │ │ │ │ │ ├── variable: b:9 [type=int] + │ │ │ │ │ │ └── variable: c:10 [type=int] │ │ │ │ │ └── const: 1 [type=int] - │ │ │ │ ├── key: (11) - │ │ │ │ ├── fd: (11)-->(7-10,12) - │ │ │ │ ├── prune: (7-12) - │ │ │ │ └── interesting orderings: (+11) + │ │ │ │ ├── key: (12) + │ │ │ │ ├── fd: (12)-->(8-11,13,14) + │ │ │ │ ├── prune: (8-14) + │ │ │ │ └── interesting orderings: (+12) │ │ │ └── filters - │ │ │ └── eq [type=bool, outer=(8,9), constraints=(/8: (/NULL - ]; /9: (/NULL - ]), fd=(8)==(9), (9)==(8)] - │ │ │ ├── variable: b:8 [type=int] - │ │ │ └── variable: c:9 [type=int] + │ │ │ └── eq [type=bool, outer=(9,10), constraints=(/9: (/NULL - ]; /10: (/NULL - ]), fd=(9)==(10), (10)==(9)] + │ │ │ ├── variable: b:9 [type=int] + │ │ │ └── variable: c:10 [type=int] │ │ └── projections - │ │ └── const: 1 [as=a_new:13, type=int] + │ │ └── const: 1 [as=a_new:15, type=int] │ └── projections - │ └── const: 0 [as=column14:14, type=int] + │ └── const: 0 [as=column16:16, type=int] └── projections - └── plus [as=column15:15, type=int, outer=(8,9), immutable] + └── plus [as=column17:17, type=int, outer=(9,10), immutable] ├── plus [type=int] - │ ├── variable: b:8 [type=int] - │ └── variable: c:9 [type=int] + │ ├── variable: b:9 [type=int] + │ └── variable: c:10 [type=int] └── const: 1 [type=int] diff --git a/pkg/sql/opt/memo/testdata/logprops/upsert b/pkg/sql/opt/memo/testdata/logprops/upsert index c581105f4fd4..89cd5698d6c5 100644 --- a/pkg/sql/opt/memo/testdata/logprops/upsert +++ b/pkg/sql/opt/memo/testdata/logprops/upsert @@ -33,177 +33,177 @@ project ├── prune: (1-3) └── upsert abc ├── columns: a:1(int!null) b:2(int) c:3(int) rowid:4(int!null) - ├── canary column: 13 - ├── fetch columns: a:10(int) b:11(int) c:12(int) rowid:13(int) + ├── canary column: 15 + ├── fetch columns: a:12(int) b:13(int) c:14(int) rowid:15(int) ├── insert-mapping: - │ ├── x:5 => a:1 - │ ├── y:6 => b:2 - │ ├── column9:9 => c:3 - │ └── column8:8 => rowid:4 + │ ├── x:6 => a:1 + │ ├── y:7 => b:2 + │ ├── column11:11 => c:3 + │ └── column10:10 => rowid:4 ├── update-mapping: - │ ├── upsert_a:17 => a:1 - │ ├── upsert_b:18 => b:2 - │ └── upsert_c:19 => c:3 + │ ├── upsert_a:20 => a:1 + │ ├── upsert_b:21 => b:2 + │ └── upsert_c:22 => c:3 ├── return-mapping: - │ ├── upsert_a:17 => a:1 - │ ├── upsert_b:18 => b:2 - │ ├── upsert_c:19 => c:3 - │ └── upsert_rowid:20 => rowid:4 + │ ├── upsert_a:20 => a:1 + │ ├── upsert_b:21 => b:2 + │ ├── upsert_c:22 => c:3 + │ └── upsert_rowid:23 => rowid:4 ├── cardinality: [0 - 1] ├── volatile, mutations └── project - ├── columns: upsert_a:17(int!null) upsert_b:18(int) upsert_c:19(int) upsert_rowid:20(int) x:5(int!null) y:6(int!null) column8:8(int) column9:9(int!null) a:10(int) b:11(int) c:12(int) rowid:13(int) a_new:14(int!null) b_new:15(int) column16:16(int) + ├── columns: upsert_a:20(int!null) upsert_b:21(int) upsert_c:22(int) upsert_rowid:23(int) x:6(int!null) y:7(int!null) column10:10(int) column11:11(int!null) a:12(int) b:13(int) c:14(int) rowid:15(int) abc.crdb_internal_mvcc_timestamp:16(decimal) a_new:17(int!null) b_new:18(int) column19:19(int) ├── cardinality: [0 - 1] ├── volatile - ├── key: (13) - ├── fd: ()-->(5,6,8,9,14), (13)-->(10-12,17-20), (10)-->(11-13), (11,12)~~>(10,13), (12)-->(15), (15)-->(16) - ├── prune: (5,6,8-20) - ├── reject-nulls: (10-13) - ├── interesting orderings: (+13) (+10) (+11,+12,+13) + ├── key: (15) + ├── fd: ()-->(6,7,10,11,17), (15)-->(12-14,16,20-23), (12)-->(13-16), (13,14)~~>(12,15,16), (14)-->(18), (18)-->(19) + ├── prune: (6,7,10-23) + ├── reject-nulls: (12-16) + ├── interesting orderings: (+15) (+12) (+13,+14,+15) ├── project - │ ├── columns: column16:16(int) x:5(int!null) y:6(int!null) column8:8(int) column9:9(int!null) a:10(int) b:11(int) c:12(int) rowid:13(int) a_new:14(int!null) b_new:15(int) + │ ├── columns: column19:19(int) x:6(int!null) y:7(int!null) column10:10(int) column11:11(int!null) a:12(int) b:13(int) c:14(int) rowid:15(int) abc.crdb_internal_mvcc_timestamp:16(decimal) a_new:17(int!null) b_new:18(int) │ ├── cardinality: [0 - 1] │ ├── volatile - │ ├── key: (13) - │ ├── fd: ()-->(5,6,8,9,14), (13)-->(10-12), (10)-->(11-13), (11,12)~~>(10,13), (12)-->(15), (15)-->(16) - │ ├── prune: (5,6,8-16) - │ ├── reject-nulls: (10-13) - │ ├── interesting orderings: (+13) (+10) (+11,+12,+13) + │ ├── key: (15) + │ ├── fd: ()-->(6,7,10,11,17), (15)-->(12-14,16), (12)-->(13-16), (13,14)~~>(12,15,16), (14)-->(18), (18)-->(19) + │ ├── prune: (6,7,10-19) + │ ├── reject-nulls: (12-16) + │ ├── interesting orderings: (+15) (+12) (+13,+14,+15) │ ├── project - │ │ ├── columns: a_new:14(int!null) b_new:15(int) x:5(int!null) y:6(int!null) column8:8(int) column9:9(int!null) a:10(int) b:11(int) c:12(int) rowid:13(int) + │ │ ├── columns: a_new:17(int!null) b_new:18(int) x:6(int!null) y:7(int!null) column10:10(int) column11:11(int!null) a:12(int) b:13(int) c:14(int) rowid:15(int) abc.crdb_internal_mvcc_timestamp:16(decimal) │ │ ├── cardinality: [0 - 1] │ │ ├── volatile - │ │ ├── key: (13) - │ │ ├── fd: ()-->(5,6,8,9,14), (13)-->(10-12), (10)-->(11-13), (11,12)~~>(10,13), (12)-->(15) - │ │ ├── prune: (5,6,8-15) - │ │ ├── reject-nulls: (10-13) - │ │ ├── interesting orderings: (+13) (+10) (+11,+12,+13) + │ │ ├── key: (15) + │ │ ├── fd: ()-->(6,7,10,11,17), (15)-->(12-14,16), (12)-->(13-16), (13,14)~~>(12,15,16), (14)-->(18) + │ │ ├── prune: (6,7,10-18) + │ │ ├── reject-nulls: (12-16) + │ │ ├── interesting orderings: (+15) (+12) (+13,+14,+15) │ │ ├── left-join (hash) - │ │ │ ├── columns: x:5(int!null) y:6(int!null) column8:8(int) column9:9(int!null) a:10(int) b:11(int) c:12(int) rowid:13(int) + │ │ │ ├── columns: x:6(int!null) y:7(int!null) column10:10(int) column11:11(int!null) a:12(int) b:13(int) c:14(int) rowid:15(int) abc.crdb_internal_mvcc_timestamp:16(decimal) │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ │ ├── volatile - │ │ │ ├── key: (13) - │ │ │ ├── fd: ()-->(5,6,8,9), (13)-->(10-12), (10)-->(11-13), (11,12)~~>(10,13) - │ │ │ ├── prune: (10,13) - │ │ │ ├── reject-nulls: (10-13) - │ │ │ ├── interesting orderings: (+13) (+10) (+11,+12,+13) + │ │ │ ├── key: (15) + │ │ │ ├── fd: ()-->(6,7,10,11), (15)-->(12-14,16), (12)-->(13-16), (13,14)~~>(12,15,16) + │ │ │ ├── prune: (12,15,16) + │ │ │ ├── reject-nulls: (12-16) + │ │ │ ├── interesting orderings: (+15) (+12) (+13,+14,+15) │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ ├── columns: x:5(int!null) y:6(int!null) column8:8(int) column9:9(int!null) - │ │ │ │ ├── grouping columns: y:6(int!null) column9:9(int!null) + │ │ │ │ ├── columns: x:6(int!null) y:7(int!null) column10:10(int) column11:11(int!null) + │ │ │ │ ├── grouping columns: y:7(int!null) column11:11(int!null) │ │ │ │ ├── error: "UPSERT or INSERT...ON CONFLICT command cannot affect row a second time" │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ ├── volatile │ │ │ │ ├── key: () - │ │ │ │ ├── fd: ()-->(5,6,8,9) + │ │ │ │ ├── fd: ()-->(6,7,10,11) │ │ │ │ ├── project - │ │ │ │ │ ├── columns: column9:9(int!null) x:5(int!null) y:6(int!null) column8:8(int) + │ │ │ │ │ ├── columns: column11:11(int!null) x:6(int!null) y:7(int!null) column10:10(int) │ │ │ │ │ ├── volatile - │ │ │ │ │ ├── key: (5) - │ │ │ │ │ ├── fd: ()-->(6,9), (5)-->(8) - │ │ │ │ │ ├── prune: (5,6,8,9) - │ │ │ │ │ ├── interesting orderings: (+5) (+6) + │ │ │ │ │ ├── key: (6) + │ │ │ │ │ ├── fd: ()-->(7,11), (6)-->(10) + │ │ │ │ │ ├── prune: (6,7,10,11) + │ │ │ │ │ ├── interesting orderings: (+6) (+7) │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: column8:8(int) x:5(int!null) y:6(int!null) + │ │ │ │ │ │ ├── columns: column10:10(int) x:6(int!null) y:7(int!null) │ │ │ │ │ │ ├── volatile - │ │ │ │ │ │ ├── key: (5) - │ │ │ │ │ │ ├── fd: ()-->(6), (5)-->(8) - │ │ │ │ │ │ ├── prune: (5,6,8) - │ │ │ │ │ │ ├── interesting orderings: (+5) (+6) + │ │ │ │ │ │ ├── key: (6) + │ │ │ │ │ │ ├── fd: ()-->(7), (6)-->(10) + │ │ │ │ │ │ ├── prune: (6,7,10) + │ │ │ │ │ │ ├── interesting orderings: (+6) (+7) │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ ├── columns: x:5(int!null) y:6(int!null) - │ │ │ │ │ │ │ ├── key: (5) - │ │ │ │ │ │ │ ├── fd: ()-->(6) - │ │ │ │ │ │ │ ├── prune: (5,6) - │ │ │ │ │ │ │ ├── interesting orderings: (+5) (+6) + │ │ │ │ │ │ │ ├── columns: x:6(int!null) y:7(int!null) + │ │ │ │ │ │ │ ├── key: (6) + │ │ │ │ │ │ │ ├── fd: ()-->(7) + │ │ │ │ │ │ │ ├── prune: (6,7) + │ │ │ │ │ │ │ ├── interesting orderings: (+6) (+7) │ │ │ │ │ │ │ └── select - │ │ │ │ │ │ │ ├── columns: x:5(int!null) y:6(int!null) z:7(int) - │ │ │ │ │ │ │ ├── key: (5) - │ │ │ │ │ │ │ ├── fd: ()-->(6), (5)-->(7), (6,7)~~>(5) - │ │ │ │ │ │ │ ├── prune: (5,7) - │ │ │ │ │ │ │ ├── interesting orderings: (+5) (+6,+7,+5) (+7,+6,+5) + │ │ │ │ │ │ │ ├── columns: x:6(int!null) y:7(int!null) z:8(int) xyz.crdb_internal_mvcc_timestamp:9(decimal) + │ │ │ │ │ │ │ ├── key: (6) + │ │ │ │ │ │ │ ├── fd: ()-->(7), (6)-->(8,9), (7,8)~~>(6,9) + │ │ │ │ │ │ │ ├── prune: (6,8,9) + │ │ │ │ │ │ │ ├── interesting orderings: (+6) (+7,+8,+6) (+8,+7,+6) │ │ │ │ │ │ │ ├── scan xyz - │ │ │ │ │ │ │ │ ├── columns: x:5(int!null) y:6(int) z:7(int) - │ │ │ │ │ │ │ │ ├── key: (5) - │ │ │ │ │ │ │ │ ├── fd: (5)-->(6,7), (6,7)~~>(5) - │ │ │ │ │ │ │ │ ├── prune: (5-7) - │ │ │ │ │ │ │ │ └── interesting orderings: (+5) (+6,+7,+5) (+7,+6,+5) + │ │ │ │ │ │ │ │ ├── columns: x:6(int!null) y:7(int) z:8(int) xyz.crdb_internal_mvcc_timestamp:9(decimal) + │ │ │ │ │ │ │ │ ├── key: (6) + │ │ │ │ │ │ │ │ ├── fd: (6)-->(7-9), (7,8)~~>(6,9) + │ │ │ │ │ │ │ │ ├── prune: (6-9) + │ │ │ │ │ │ │ │ └── interesting orderings: (+6) (+7,+8,+6) (+8,+7,+6) │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── eq [type=bool, outer=(6), constraints=(/6: [/1 - /1]; tight), fd=()-->(6)] - │ │ │ │ │ │ │ ├── variable: y:6 [type=int] + │ │ │ │ │ │ │ └── eq [type=bool, outer=(7), constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] + │ │ │ │ │ │ │ ├── variable: y:7 [type=int] │ │ │ │ │ │ │ └── const: 1 [type=int] │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── function: unique_rowid [as=column8:8, type=int, volatile] + │ │ │ │ │ │ └── function: unique_rowid [as=column10:10, type=int, volatile] │ │ │ │ │ └── projections - │ │ │ │ │ └── plus [as=column9:9, type=int, outer=(6), immutable] - │ │ │ │ │ ├── variable: y:6 [type=int] + │ │ │ │ │ └── plus [as=column11:11, type=int, outer=(7), immutable] + │ │ │ │ │ ├── variable: y:7 [type=int] │ │ │ │ │ └── const: 1 [type=int] │ │ │ │ └── aggregations - │ │ │ │ ├── first-agg [as=x:5, type=int, outer=(5)] - │ │ │ │ │ └── variable: x:5 [type=int] - │ │ │ │ └── first-agg [as=column8:8, type=int, outer=(8)] - │ │ │ │ └── variable: column8:8 [type=int] + │ │ │ │ ├── first-agg [as=x:6, type=int, outer=(6)] + │ │ │ │ │ └── variable: x:6 [type=int] + │ │ │ │ └── first-agg [as=column10:10, type=int, outer=(10)] + │ │ │ │ └── variable: column10:10 [type=int] │ │ │ ├── scan abc - │ │ │ │ ├── columns: a:10(int!null) b:11(int) c:12(int) rowid:13(int!null) + │ │ │ │ ├── columns: a:12(int!null) b:13(int) c:14(int) rowid:15(int!null) abc.crdb_internal_mvcc_timestamp:16(decimal) │ │ │ │ ├── computed column expressions - │ │ │ │ │ └── c:12 + │ │ │ │ │ └── c:14 │ │ │ │ │ └── plus [type=int] - │ │ │ │ │ ├── variable: b:11 [type=int] + │ │ │ │ │ ├── variable: b:13 [type=int] │ │ │ │ │ └── const: 1 [type=int] - │ │ │ │ ├── key: (13) - │ │ │ │ ├── fd: (13)-->(10-12), (10)-->(11-13), (11,12)~~>(10,13) - │ │ │ │ ├── prune: (10-13) - │ │ │ │ ├── interesting orderings: (+13) (+10) (+11,+12,+13) - │ │ │ │ └── unfiltered-cols: (10-13) + │ │ │ │ ├── key: (15) + │ │ │ │ ├── fd: (15)-->(12-14,16), (12)-->(13-16), (13,14)~~>(12,15,16) + │ │ │ │ ├── prune: (12-16) + │ │ │ │ ├── interesting orderings: (+15) (+12) (+13,+14,+15) + │ │ │ │ └── unfiltered-cols: (12-16) │ │ │ └── filters - │ │ │ ├── eq [type=bool, outer=(6,11), constraints=(/6: (/NULL - ]; /11: (/NULL - ]), fd=(6)==(11), (11)==(6)] - │ │ │ │ ├── variable: y:6 [type=int] - │ │ │ │ └── variable: b:11 [type=int] - │ │ │ └── eq [type=bool, outer=(9,12), constraints=(/9: (/NULL - ]; /12: (/NULL - ]), fd=(9)==(12), (12)==(9)] - │ │ │ ├── variable: column9:9 [type=int] - │ │ │ └── variable: c:12 [type=int] + │ │ │ ├── eq [type=bool, outer=(7,13), constraints=(/7: (/NULL - ]; /13: (/NULL - ]), fd=(7)==(13), (13)==(7)] + │ │ │ │ ├── variable: y:7 [type=int] + │ │ │ │ └── variable: b:13 [type=int] + │ │ │ └── eq [type=bool, outer=(11,14), constraints=(/11: (/NULL - ]; /14: (/NULL - ]), fd=(11)==(14), (14)==(11)] + │ │ │ ├── variable: column11:11 [type=int] + │ │ │ └── variable: c:14 [type=int] │ │ └── projections - │ │ ├── const: 1 [as=a_new:14, type=int] - │ │ └── plus [as=b_new:15, type=int, outer=(6,12), immutable] - │ │ ├── variable: y:6 [type=int] - │ │ └── variable: c:12 [type=int] + │ │ ├── const: 1 [as=a_new:17, type=int] + │ │ └── plus [as=b_new:18, type=int, outer=(7,14), immutable] + │ │ ├── variable: y:7 [type=int] + │ │ └── variable: c:14 [type=int] │ └── projections - │ └── plus [as=column16:16, type=int, outer=(15), immutable] - │ ├── variable: b_new:15 [type=int] + │ └── plus [as=column19:19, type=int, outer=(18), immutable] + │ ├── variable: b_new:18 [type=int] │ └── const: 1 [type=int] └── projections - ├── case [as=upsert_a:17, type=int, outer=(5,13,14)] + ├── case [as=upsert_a:20, type=int, outer=(6,15,17)] │ ├── true [type=bool] │ ├── when [type=int] │ │ ├── is [type=bool] - │ │ │ ├── variable: rowid:13 [type=int] + │ │ │ ├── variable: rowid:15 [type=int] │ │ │ └── null [type=unknown] - │ │ └── variable: x:5 [type=int] - │ └── variable: a_new:14 [type=int] - ├── case [as=upsert_b:18, type=int, outer=(6,13,15)] + │ │ └── variable: x:6 [type=int] + │ └── variable: a_new:17 [type=int] + ├── case [as=upsert_b:21, type=int, outer=(7,15,18)] │ ├── true [type=bool] │ ├── when [type=int] │ │ ├── is [type=bool] - │ │ │ ├── variable: rowid:13 [type=int] + │ │ │ ├── variable: rowid:15 [type=int] │ │ │ └── null [type=unknown] - │ │ └── variable: y:6 [type=int] - │ └── variable: b_new:15 [type=int] - ├── case [as=upsert_c:19, type=int, outer=(9,13,16)] + │ │ └── variable: y:7 [type=int] + │ └── variable: b_new:18 [type=int] + ├── case [as=upsert_c:22, type=int, outer=(11,15,19)] │ ├── true [type=bool] │ ├── when [type=int] │ │ ├── is [type=bool] - │ │ │ ├── variable: rowid:13 [type=int] + │ │ │ ├── variable: rowid:15 [type=int] │ │ │ └── null [type=unknown] - │ │ └── variable: column9:9 [type=int] - │ └── variable: column16:16 [type=int] - └── case [as=upsert_rowid:20, type=int, outer=(8,13)] + │ │ └── variable: column11:11 [type=int] + │ └── variable: column19:19 [type=int] + └── case [as=upsert_rowid:23, type=int, outer=(10,15)] ├── true [type=bool] ├── when [type=int] │ ├── is [type=bool] - │ │ ├── variable: rowid:13 [type=int] + │ │ ├── variable: rowid:15 [type=int] │ │ └── null [type=unknown] - │ └── variable: column8:8 [type=int] - └── variable: rowid:13 [type=int] + │ └── variable: column10:10 [type=int] + └── variable: rowid:15 [type=int] # DO NOTHING case. build @@ -221,362 +221,362 @@ project └── insert abc ├── columns: a:1(int!null) b:2(int) c:3(int) rowid:4(int!null) ├── insert-mapping: - │ ├── x:5 => a:1 - │ ├── y:6 => b:2 - │ ├── column9:9 => c:3 - │ └── column8:8 => rowid:4 + │ ├── x:6 => a:1 + │ ├── y:7 => b:2 + │ ├── column11:11 => c:3 + │ └── column10:10 => rowid:4 ├── volatile, mutations ├── key: (1) ├── fd: (1)-->(2-4), (2)-->(3), (4)~~>(1-3), (2,3)~~>(1,4) └── upsert-distinct-on - ├── columns: x:5(int!null) y:6(int) column8:8(int) column9:9(int) - ├── grouping columns: y:6(int) column9:9(int) + ├── columns: x:6(int!null) y:7(int) column10:10(int) column11:11(int) + ├── grouping columns: y:7(int) column11:11(int) ├── volatile - ├── key: (5) - ├── fd: (5)-->(6,8,9), (6)-->(9), (8)~~>(5,6,9), (6,9)~~>(5,8) + ├── key: (6) + ├── fd: (6)-->(7,10,11), (7)-->(11), (10)~~>(6,7,11), (7,11)~~>(6,10) ├── project - │ ├── columns: x:5(int!null) y:6(int) column8:8(int) column9:9(int) + │ ├── columns: x:6(int!null) y:7(int) column10:10(int) column11:11(int) │ ├── volatile - │ ├── key: (5) - │ ├── fd: (5)-->(6,8,9), (6)-->(9), (8)~~>(5,6,9) - │ ├── prune: (5,6,8,9) + │ ├── key: (6) + │ ├── fd: (6)-->(7,10,11), (7)-->(11), (10)~~>(6,7,11) + │ ├── prune: (6,7,10,11) │ └── select - │ ├── columns: x:5(int!null) y:6(int) column8:8(int) column9:9(int) a:18(int) b:19(int) c:20(int) rowid:21(int) + │ ├── columns: x:6(int!null) y:7(int) column10:10(int) column11:11(int) a:22(int) b:23(int) c:24(int) rowid:25(int) abc.crdb_internal_mvcc_timestamp:26(decimal) │ ├── volatile - │ ├── key: (5) - │ ├── fd: ()-->(18-21), (5)-->(6,8,9), (6)-->(9), (8)~~>(5,6,9) - │ ├── prune: (18) - │ ├── interesting orderings: (+21) (+18) (+19,+20,+21) + │ ├── key: (6) + │ ├── fd: ()-->(22-26), (6)-->(7,10,11), (7)-->(11), (10)~~>(6,7,11) + │ ├── prune: (22,26) + │ ├── interesting orderings: (+25) (+22) (+23,+24,+25) │ ├── left-join (hash) - │ │ ├── columns: x:5(int!null) y:6(int) column8:8(int) column9:9(int) a:18(int) b:19(int) c:20(int) rowid:21(int) + │ │ ├── columns: x:6(int!null) y:7(int) column10:10(int) column11:11(int) a:22(int) b:23(int) c:24(int) rowid:25(int) abc.crdb_internal_mvcc_timestamp:26(decimal) │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) │ │ ├── volatile - │ │ ├── key: (5,21) - │ │ ├── fd: (5)-->(6,8,9), (6)-->(9), (8)~~>(5,6,9), (21)-->(18-20), (18)-->(19-21), (19,20)~~>(18,21) - │ │ ├── prune: (18,21) - │ │ ├── reject-nulls: (18-21) - │ │ ├── interesting orderings: (+21) (+18) (+19,+20,+21) + │ │ ├── key: (6,25) + │ │ ├── fd: (6)-->(7,10,11), (7)-->(11), (10)~~>(6,7,11), (25)-->(22-24,26), (22)-->(23-26), (23,24)~~>(22,25,26) + │ │ ├── prune: (22,25,26) + │ │ ├── reject-nulls: (22-26) + │ │ ├── interesting orderings: (+25) (+22) (+23,+24,+25) │ │ ├── upsert-distinct-on - │ │ │ ├── columns: x:5(int!null) y:6(int) column8:8(int) column9:9(int) - │ │ │ ├── grouping columns: x:5(int!null) + │ │ │ ├── columns: x:6(int!null) y:7(int) column10:10(int) column11:11(int) + │ │ │ ├── grouping columns: x:6(int!null) │ │ │ ├── volatile - │ │ │ ├── key: (5) - │ │ │ ├── fd: (5)-->(6,8,9), (6)-->(9), (8)~~>(5,6,9) + │ │ │ ├── key: (6) + │ │ │ ├── fd: (6)-->(7,10,11), (7)-->(11), (10)~~>(6,7,11) │ │ │ ├── project - │ │ │ │ ├── columns: x:5(int!null) y:6(int) column8:8(int) column9:9(int) + │ │ │ │ ├── columns: x:6(int!null) y:7(int) column10:10(int) column11:11(int) │ │ │ │ ├── volatile - │ │ │ │ ├── key: (5) - │ │ │ │ ├── fd: (5)-->(6,8), (6)-->(9), (8)~~>(5,6,9) - │ │ │ │ ├── prune: (5,6,8,9) + │ │ │ │ ├── key: (6) + │ │ │ │ ├── fd: (6)-->(7,10), (7)-->(11), (10)~~>(6,7,11) + │ │ │ │ ├── prune: (6,7,10,11) │ │ │ │ └── select - │ │ │ │ ├── columns: x:5(int!null) y:6(int) column8:8(int) column9:9(int) a:14(int) b:15(int) c:16(int) rowid:17(int) + │ │ │ │ ├── columns: x:6(int!null) y:7(int) column10:10(int) column11:11(int) a:17(int) b:18(int) c:19(int) rowid:20(int) abc.crdb_internal_mvcc_timestamp:21(decimal) │ │ │ │ ├── volatile - │ │ │ │ ├── key: (5) - │ │ │ │ ├── fd: ()-->(14-17), (5)-->(6,8), (6)-->(9), (8)~~>(5,6,9) - │ │ │ │ ├── prune: (15-17) - │ │ │ │ ├── interesting orderings: (+17) (+14) (+15,+16,+17) + │ │ │ │ ├── key: (6) + │ │ │ │ ├── fd: ()-->(17-21), (6)-->(7,10), (7)-->(11), (10)~~>(6,7,11) + │ │ │ │ ├── prune: (18-21) + │ │ │ │ ├── interesting orderings: (+20) (+17) (+18,+19,+20) │ │ │ │ ├── left-join (hash) - │ │ │ │ │ ├── columns: x:5(int!null) y:6(int) column8:8(int) column9:9(int) a:14(int) b:15(int) c:16(int) rowid:17(int) + │ │ │ │ │ ├── columns: x:6(int!null) y:7(int) column10:10(int) column11:11(int) a:17(int) b:18(int) c:19(int) rowid:20(int) abc.crdb_internal_mvcc_timestamp:21(decimal) │ │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ │ │ │ ├── volatile - │ │ │ │ │ ├── key: (5) - │ │ │ │ │ ├── fd: (5)-->(6,8,14-17), (6)-->(9), (8)~~>(5,6,9), (17)-->(14-16), (14)-->(15-17), (15,16)~~>(14,17) - │ │ │ │ │ ├── prune: (15-17) - │ │ │ │ │ ├── reject-nulls: (14-17) - │ │ │ │ │ ├── interesting orderings: (+17) (+14) (+15,+16,+17) + │ │ │ │ │ ├── key: (6) + │ │ │ │ │ ├── fd: (6)-->(7,10,17-21), (7)-->(11), (10)~~>(6,7,11), (20)-->(17-19,21), (17)-->(18-21), (18,19)~~>(17,20,21) + │ │ │ │ │ ├── prune: (18-21) + │ │ │ │ │ ├── reject-nulls: (17-21) + │ │ │ │ │ ├── interesting orderings: (+20) (+17) (+18,+19,+20) │ │ │ │ │ ├── upsert-distinct-on - │ │ │ │ │ │ ├── columns: x:5(int!null) y:6(int) column8:8(int) column9:9(int) - │ │ │ │ │ │ ├── grouping columns: column8:8(int) + │ │ │ │ │ │ ├── columns: x:6(int!null) y:7(int) column10:10(int) column11:11(int) + │ │ │ │ │ │ ├── grouping columns: column10:10(int) │ │ │ │ │ │ ├── volatile - │ │ │ │ │ │ ├── key: (5) - │ │ │ │ │ │ ├── fd: (5)-->(6,8), (6)-->(9), (8)~~>(5,6,9) + │ │ │ │ │ │ ├── key: (6) + │ │ │ │ │ │ ├── fd: (6)-->(7,10), (7)-->(11), (10)~~>(6,7,11) │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ ├── columns: x:5(int!null) y:6(int) column8:8(int) column9:9(int) + │ │ │ │ │ │ │ ├── columns: x:6(int!null) y:7(int) column10:10(int) column11:11(int) │ │ │ │ │ │ │ ├── volatile - │ │ │ │ │ │ │ ├── key: (5) - │ │ │ │ │ │ │ ├── fd: (5)-->(6,8), (6)-->(9) - │ │ │ │ │ │ │ ├── prune: (5,6,8,9) - │ │ │ │ │ │ │ ├── interesting orderings: (+5) (+6) + │ │ │ │ │ │ │ ├── key: (6) + │ │ │ │ │ │ │ ├── fd: (6)-->(7,10), (7)-->(11) + │ │ │ │ │ │ │ ├── prune: (6,7,10,11) + │ │ │ │ │ │ │ ├── interesting orderings: (+6) (+7) │ │ │ │ │ │ │ └── select - │ │ │ │ │ │ │ ├── columns: x:5(int!null) y:6(int) column8:8(int) column9:9(int) a:10(int) b:11(int) c:12(int) rowid:13(int) + │ │ │ │ │ │ │ ├── columns: x:6(int!null) y:7(int) column10:10(int) column11:11(int) a:12(int) b:13(int) c:14(int) rowid:15(int) abc.crdb_internal_mvcc_timestamp:16(decimal) │ │ │ │ │ │ │ ├── volatile - │ │ │ │ │ │ │ ├── key: (5) - │ │ │ │ │ │ │ ├── fd: ()-->(10-13), (5)-->(6,8), (6)-->(9) - │ │ │ │ │ │ │ ├── prune: (5,6,9-12) - │ │ │ │ │ │ │ ├── interesting orderings: (+5) (+6) (+13) (+10) (+11,+12,+13) + │ │ │ │ │ │ │ ├── key: (6) + │ │ │ │ │ │ │ ├── fd: ()-->(12-16), (6)-->(7,10), (7)-->(11) + │ │ │ │ │ │ │ ├── prune: (6,7,11-14,16) + │ │ │ │ │ │ │ ├── interesting orderings: (+6) (+7) (+15) (+12) (+13,+14,+15) │ │ │ │ │ │ │ ├── left-join (hash) - │ │ │ │ │ │ │ │ ├── columns: x:5(int!null) y:6(int) column8:8(int) column9:9(int) a:10(int) b:11(int) c:12(int) rowid:13(int) + │ │ │ │ │ │ │ │ ├── columns: x:6(int!null) y:7(int) column10:10(int) column11:11(int) a:12(int) b:13(int) c:14(int) rowid:15(int) abc.crdb_internal_mvcc_timestamp:16(decimal) │ │ │ │ │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) │ │ │ │ │ │ │ │ ├── volatile - │ │ │ │ │ │ │ │ ├── key: (5) - │ │ │ │ │ │ │ │ ├── fd: (5)-->(6,8,10-13), (6)-->(9), (13)-->(10-12), (10)-->(11-13), (11,12)~~>(10,13) - │ │ │ │ │ │ │ │ ├── prune: (5,6,9-12) - │ │ │ │ │ │ │ │ ├── reject-nulls: (10-13) - │ │ │ │ │ │ │ │ ├── interesting orderings: (+5) (+6) (+13) (+10) (+11,+12,+13) + │ │ │ │ │ │ │ │ ├── key: (6) + │ │ │ │ │ │ │ │ ├── fd: (6)-->(7,10,12-16), (7)-->(11), (15)-->(12-14,16), (12)-->(13-16), (13,14)~~>(12,15,16) + │ │ │ │ │ │ │ │ ├── prune: (6,7,11-14,16) + │ │ │ │ │ │ │ │ ├── reject-nulls: (12-16) + │ │ │ │ │ │ │ │ ├── interesting orderings: (+6) (+7) (+15) (+12) (+13,+14,+15) │ │ │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ │ │ ├── columns: column9:9(int) x:5(int!null) y:6(int) column8:8(int) + │ │ │ │ │ │ │ │ │ ├── columns: column11:11(int) x:6(int!null) y:7(int) column10:10(int) │ │ │ │ │ │ │ │ │ ├── volatile - │ │ │ │ │ │ │ │ │ ├── key: (5) - │ │ │ │ │ │ │ │ │ ├── fd: (5)-->(6,8), (6)-->(9) - │ │ │ │ │ │ │ │ │ ├── prune: (5,6,8,9) - │ │ │ │ │ │ │ │ │ ├── interesting orderings: (+5) (+6) - │ │ │ │ │ │ │ │ │ ├── unfiltered-cols: (5-7) + │ │ │ │ │ │ │ │ │ ├── key: (6) + │ │ │ │ │ │ │ │ │ ├── fd: (6)-->(7,10), (7)-->(11) + │ │ │ │ │ │ │ │ │ ├── prune: (6,7,10,11) + │ │ │ │ │ │ │ │ │ ├── interesting orderings: (+6) (+7) + │ │ │ │ │ │ │ │ │ ├── unfiltered-cols: (6-9) │ │ │ │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ │ │ │ ├── columns: column8:8(int) x:5(int!null) y:6(int) + │ │ │ │ │ │ │ │ │ │ ├── columns: column10:10(int) x:6(int!null) y:7(int) │ │ │ │ │ │ │ │ │ │ ├── volatile - │ │ │ │ │ │ │ │ │ │ ├── key: (5) - │ │ │ │ │ │ │ │ │ │ ├── fd: (5)-->(6,8) - │ │ │ │ │ │ │ │ │ │ ├── prune: (5,6,8) - │ │ │ │ │ │ │ │ │ │ ├── interesting orderings: (+5) (+6) - │ │ │ │ │ │ │ │ │ │ ├── unfiltered-cols: (5-7) + │ │ │ │ │ │ │ │ │ │ ├── key: (6) + │ │ │ │ │ │ │ │ │ │ ├── fd: (6)-->(7,10) + │ │ │ │ │ │ │ │ │ │ ├── prune: (6,7,10) + │ │ │ │ │ │ │ │ │ │ ├── interesting orderings: (+6) (+7) + │ │ │ │ │ │ │ │ │ │ ├── unfiltered-cols: (6-9) │ │ │ │ │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ │ │ │ │ ├── columns: x:5(int!null) y:6(int) - │ │ │ │ │ │ │ │ │ │ │ ├── key: (5) - │ │ │ │ │ │ │ │ │ │ │ ├── fd: (5)-->(6) - │ │ │ │ │ │ │ │ │ │ │ ├── prune: (5,6) - │ │ │ │ │ │ │ │ │ │ │ ├── interesting orderings: (+5) (+6) - │ │ │ │ │ │ │ │ │ │ │ ├── unfiltered-cols: (5-7) + │ │ │ │ │ │ │ │ │ │ │ ├── columns: x:6(int!null) y:7(int) + │ │ │ │ │ │ │ │ │ │ │ ├── key: (6) + │ │ │ │ │ │ │ │ │ │ │ ├── fd: (6)-->(7) + │ │ │ │ │ │ │ │ │ │ │ ├── prune: (6,7) + │ │ │ │ │ │ │ │ │ │ │ ├── interesting orderings: (+6) (+7) + │ │ │ │ │ │ │ │ │ │ │ ├── unfiltered-cols: (6-9) │ │ │ │ │ │ │ │ │ │ │ └── scan xyz - │ │ │ │ │ │ │ │ │ │ │ ├── columns: x:5(int!null) y:6(int) z:7(int) - │ │ │ │ │ │ │ │ │ │ │ ├── key: (5) - │ │ │ │ │ │ │ │ │ │ │ ├── fd: (5)-->(6,7), (6,7)~~>(5) - │ │ │ │ │ │ │ │ │ │ │ ├── prune: (5-7) - │ │ │ │ │ │ │ │ │ │ │ ├── interesting orderings: (+5) (+6,+7,+5) (+7,+6,+5) - │ │ │ │ │ │ │ │ │ │ │ └── unfiltered-cols: (5-7) + │ │ │ │ │ │ │ │ │ │ │ ├── columns: x:6(int!null) y:7(int) z:8(int) xyz.crdb_internal_mvcc_timestamp:9(decimal) + │ │ │ │ │ │ │ │ │ │ │ ├── key: (6) + │ │ │ │ │ │ │ │ │ │ │ ├── fd: (6)-->(7-9), (7,8)~~>(6,9) + │ │ │ │ │ │ │ │ │ │ │ ├── prune: (6-9) + │ │ │ │ │ │ │ │ │ │ │ ├── interesting orderings: (+6) (+7,+8,+6) (+8,+7,+6) + │ │ │ │ │ │ │ │ │ │ │ └── unfiltered-cols: (6-9) │ │ │ │ │ │ │ │ │ │ └── projections - │ │ │ │ │ │ │ │ │ │ └── function: unique_rowid [as=column8:8, type=int, volatile] + │ │ │ │ │ │ │ │ │ │ └── function: unique_rowid [as=column10:10, type=int, volatile] │ │ │ │ │ │ │ │ │ └── projections - │ │ │ │ │ │ │ │ │ └── plus [as=column9:9, type=int, outer=(6), immutable] - │ │ │ │ │ │ │ │ │ ├── variable: y:6 [type=int] + │ │ │ │ │ │ │ │ │ └── plus [as=column11:11, type=int, outer=(7), immutable] + │ │ │ │ │ │ │ │ │ ├── variable: y:7 [type=int] │ │ │ │ │ │ │ │ │ └── const: 1 [type=int] │ │ │ │ │ │ │ │ ├── scan abc - │ │ │ │ │ │ │ │ │ ├── columns: a:10(int!null) b:11(int) c:12(int) rowid:13(int!null) + │ │ │ │ │ │ │ │ │ ├── columns: a:12(int!null) b:13(int) c:14(int) rowid:15(int!null) abc.crdb_internal_mvcc_timestamp:16(decimal) │ │ │ │ │ │ │ │ │ ├── computed column expressions - │ │ │ │ │ │ │ │ │ │ └── c:12 + │ │ │ │ │ │ │ │ │ │ └── c:14 │ │ │ │ │ │ │ │ │ │ └── plus [type=int] - │ │ │ │ │ │ │ │ │ │ ├── variable: b:11 [type=int] + │ │ │ │ │ │ │ │ │ │ ├── variable: b:13 [type=int] │ │ │ │ │ │ │ │ │ │ └── const: 1 [type=int] - │ │ │ │ │ │ │ │ │ ├── key: (13) - │ │ │ │ │ │ │ │ │ ├── fd: (13)-->(10-12), (10)-->(11-13), (11,12)~~>(10,13) - │ │ │ │ │ │ │ │ │ ├── prune: (10-13) - │ │ │ │ │ │ │ │ │ ├── interesting orderings: (+13) (+10) (+11,+12,+13) - │ │ │ │ │ │ │ │ │ └── unfiltered-cols: (10-13) + │ │ │ │ │ │ │ │ │ ├── key: (15) + │ │ │ │ │ │ │ │ │ ├── fd: (15)-->(12-14,16), (12)-->(13-16), (13,14)~~>(12,15,16) + │ │ │ │ │ │ │ │ │ ├── prune: (12-16) + │ │ │ │ │ │ │ │ │ ├── interesting orderings: (+15) (+12) (+13,+14,+15) + │ │ │ │ │ │ │ │ │ └── unfiltered-cols: (12-16) │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ └── eq [type=bool, outer=(8,13), constraints=(/8: (/NULL - ]; /13: (/NULL - ]), fd=(8)==(13), (13)==(8)] - │ │ │ │ │ │ │ │ ├── variable: column8:8 [type=int] - │ │ │ │ │ │ │ │ └── variable: rowid:13 [type=int] + │ │ │ │ │ │ │ │ └── eq [type=bool, outer=(10,15), constraints=(/10: (/NULL - ]; /15: (/NULL - ]), fd=(10)==(15), (15)==(10)] + │ │ │ │ │ │ │ │ ├── variable: column10:10 [type=int] + │ │ │ │ │ │ │ │ └── variable: rowid:15 [type=int] │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── is [type=bool, outer=(13), constraints=(/13: [/NULL - /NULL]; tight), fd=()-->(13)] - │ │ │ │ │ │ │ ├── variable: rowid:13 [type=int] + │ │ │ │ │ │ │ └── is [type=bool, outer=(15), constraints=(/15: [/NULL - /NULL]; tight), fd=()-->(15)] + │ │ │ │ │ │ │ ├── variable: rowid:15 [type=int] │ │ │ │ │ │ │ └── null [type=unknown] │ │ │ │ │ │ └── aggregations - │ │ │ │ │ │ ├── first-agg [as=x:5, type=int, outer=(5)] - │ │ │ │ │ │ │ └── variable: x:5 [type=int] - │ │ │ │ │ │ ├── first-agg [as=y:6, type=int, outer=(6)] - │ │ │ │ │ │ │ └── variable: y:6 [type=int] - │ │ │ │ │ │ └── first-agg [as=column9:9, type=int, outer=(9)] - │ │ │ │ │ │ └── variable: column9:9 [type=int] + │ │ │ │ │ │ ├── first-agg [as=x:6, type=int, outer=(6)] + │ │ │ │ │ │ │ └── variable: x:6 [type=int] + │ │ │ │ │ │ ├── first-agg [as=y:7, type=int, outer=(7)] + │ │ │ │ │ │ │ └── variable: y:7 [type=int] + │ │ │ │ │ │ └── first-agg [as=column11:11, type=int, outer=(11)] + │ │ │ │ │ │ └── variable: column11:11 [type=int] │ │ │ │ │ ├── scan abc - │ │ │ │ │ │ ├── columns: a:14(int!null) b:15(int) c:16(int) rowid:17(int!null) + │ │ │ │ │ │ ├── columns: a:17(int!null) b:18(int) c:19(int) rowid:20(int!null) abc.crdb_internal_mvcc_timestamp:21(decimal) │ │ │ │ │ │ ├── computed column expressions - │ │ │ │ │ │ │ └── c:16 + │ │ │ │ │ │ │ └── c:19 │ │ │ │ │ │ │ └── plus [type=int] - │ │ │ │ │ │ │ ├── variable: b:15 [type=int] + │ │ │ │ │ │ │ ├── variable: b:18 [type=int] │ │ │ │ │ │ │ └── const: 1 [type=int] - │ │ │ │ │ │ ├── key: (17) - │ │ │ │ │ │ ├── fd: (17)-->(14-16), (14)-->(15-17), (15,16)~~>(14,17) - │ │ │ │ │ │ ├── prune: (14-17) - │ │ │ │ │ │ ├── interesting orderings: (+17) (+14) (+15,+16,+17) - │ │ │ │ │ │ └── unfiltered-cols: (14-17) + │ │ │ │ │ │ ├── key: (20) + │ │ │ │ │ │ ├── fd: (20)-->(17-19,21), (17)-->(18-21), (18,19)~~>(17,20,21) + │ │ │ │ │ │ ├── prune: (17-21) + │ │ │ │ │ │ ├── interesting orderings: (+20) (+17) (+18,+19,+20) + │ │ │ │ │ │ └── unfiltered-cols: (17-21) │ │ │ │ │ └── filters - │ │ │ │ │ └── eq [type=bool, outer=(5,14), constraints=(/5: (/NULL - ]; /14: (/NULL - ]), fd=(5)==(14), (14)==(5)] - │ │ │ │ │ ├── variable: x:5 [type=int] - │ │ │ │ │ └── variable: a:14 [type=int] + │ │ │ │ │ └── eq [type=bool, outer=(6,17), constraints=(/6: (/NULL - ]; /17: (/NULL - ]), fd=(6)==(17), (17)==(6)] + │ │ │ │ │ ├── variable: x:6 [type=int] + │ │ │ │ │ └── variable: a:17 [type=int] │ │ │ │ └── filters - │ │ │ │ └── is [type=bool, outer=(14), constraints=(/14: [/NULL - /NULL]; tight), fd=()-->(14)] - │ │ │ │ ├── variable: a:14 [type=int] + │ │ │ │ └── is [type=bool, outer=(17), constraints=(/17: [/NULL - /NULL]; tight), fd=()-->(17)] + │ │ │ │ ├── variable: a:17 [type=int] │ │ │ │ └── null [type=unknown] │ │ │ └── aggregations - │ │ │ ├── first-agg [as=y:6, type=int, outer=(6)] - │ │ │ │ └── variable: y:6 [type=int] - │ │ │ ├── first-agg [as=column8:8, type=int, outer=(8)] - │ │ │ │ └── variable: column8:8 [type=int] - │ │ │ └── first-agg [as=column9:9, type=int, outer=(9)] - │ │ │ └── variable: column9:9 [type=int] + │ │ │ ├── first-agg [as=y:7, type=int, outer=(7)] + │ │ │ │ └── variable: y:7 [type=int] + │ │ │ ├── first-agg [as=column10:10, type=int, outer=(10)] + │ │ │ │ └── variable: column10:10 [type=int] + │ │ │ └── first-agg [as=column11:11, type=int, outer=(11)] + │ │ │ └── variable: column11:11 [type=int] │ │ ├── scan abc - │ │ │ ├── columns: a:18(int!null) b:19(int) c:20(int) rowid:21(int!null) + │ │ │ ├── columns: a:22(int!null) b:23(int) c:24(int) rowid:25(int!null) abc.crdb_internal_mvcc_timestamp:26(decimal) │ │ │ ├── computed column expressions - │ │ │ │ └── c:20 + │ │ │ │ └── c:24 │ │ │ │ └── plus [type=int] - │ │ │ │ ├── variable: b:19 [type=int] + │ │ │ │ ├── variable: b:23 [type=int] │ │ │ │ └── const: 1 [type=int] - │ │ │ ├── key: (21) - │ │ │ ├── fd: (21)-->(18-20), (18)-->(19-21), (19,20)~~>(18,21) - │ │ │ ├── prune: (18-21) - │ │ │ ├── interesting orderings: (+21) (+18) (+19,+20,+21) - │ │ │ └── unfiltered-cols: (18-21) + │ │ │ ├── key: (25) + │ │ │ ├── fd: (25)-->(22-24,26), (22)-->(23-26), (23,24)~~>(22,25,26) + │ │ │ ├── prune: (22-26) + │ │ │ ├── interesting orderings: (+25) (+22) (+23,+24,+25) + │ │ │ └── unfiltered-cols: (22-26) │ │ └── filters - │ │ ├── eq [type=bool, outer=(6,19), constraints=(/6: (/NULL - ]; /19: (/NULL - ]), fd=(6)==(19), (19)==(6)] - │ │ │ ├── variable: y:6 [type=int] - │ │ │ └── variable: b:19 [type=int] - │ │ └── eq [type=bool, outer=(9,20), constraints=(/9: (/NULL - ]; /20: (/NULL - ]), fd=(9)==(20), (20)==(9)] - │ │ ├── variable: column9:9 [type=int] - │ │ └── variable: c:20 [type=int] + │ │ ├── eq [type=bool, outer=(7,23), constraints=(/7: (/NULL - ]; /23: (/NULL - ]), fd=(7)==(23), (23)==(7)] + │ │ │ ├── variable: y:7 [type=int] + │ │ │ └── variable: b:23 [type=int] + │ │ └── eq [type=bool, outer=(11,24), constraints=(/11: (/NULL - ]; /24: (/NULL - ]), fd=(11)==(24), (24)==(11)] + │ │ ├── variable: column11:11 [type=int] + │ │ └── variable: c:24 [type=int] │ └── filters - │ └── is [type=bool, outer=(21), constraints=(/21: [/NULL - /NULL]; tight), fd=()-->(21)] - │ ├── variable: rowid:21 [type=int] + │ └── is [type=bool, outer=(25), constraints=(/25: [/NULL - /NULL]; tight), fd=()-->(25)] + │ ├── variable: rowid:25 [type=int] │ └── null [type=unknown] └── aggregations - ├── first-agg [as=x:5, type=int, outer=(5)] - │ └── variable: x:5 [type=int] - └── first-agg [as=column8:8, type=int, outer=(8)] - └── variable: column8:8 [type=int] + ├── first-agg [as=x:6, type=int, outer=(6)] + │ └── variable: x:6 [type=int] + └── first-agg [as=column10:10, type=int, outer=(10)] + └── variable: column10:10 [type=int] # UPSERT case. build UPSERT INTO abc (a) VALUES (1), (2) RETURNING b+c ---- project - ├── columns: "?column?":17(int) + ├── columns: "?column?":19(int) ├── cardinality: [1 - 2] ├── volatile, mutations - ├── prune: (17) + ├── prune: (19) ├── upsert abc │ ├── columns: a:1(int!null) b:2(int) c:3(int) rowid:4(int!null) - │ ├── canary column: 12 - │ ├── fetch columns: a:9(int) b:10(int) c:11(int) rowid:12(int) + │ ├── canary column: 13 + │ ├── fetch columns: a:10(int) b:11(int) c:12(int) rowid:13(int) │ ├── insert-mapping: - │ │ ├── column1:5 => a:1 - │ │ ├── column6:6 => b:2 - │ │ ├── column8:8 => c:3 - │ │ └── column7:7 => rowid:4 + │ │ ├── column1:6 => a:1 + │ │ ├── column7:7 => b:2 + │ │ ├── column9:9 => c:3 + │ │ └── column8:8 => rowid:4 │ ├── update-mapping: - │ │ ├── column1:5 => a:1 - │ │ └── upsert_c:15 => c:3 + │ │ ├── column1:6 => a:1 + │ │ └── upsert_c:17 => c:3 │ ├── return-mapping: - │ │ ├── column1:5 => a:1 - │ │ ├── upsert_b:14 => b:2 - │ │ ├── upsert_c:15 => c:3 - │ │ └── upsert_rowid:16 => rowid:4 + │ │ ├── column1:6 => a:1 + │ │ ├── upsert_b:16 => b:2 + │ │ ├── upsert_c:17 => c:3 + │ │ └── upsert_rowid:18 => rowid:4 │ ├── cardinality: [1 - 2] │ ├── volatile, mutations │ └── project - │ ├── columns: upsert_b:14(int) upsert_c:15(int) upsert_rowid:16(int) column1:5(int!null) column6:6(int!null) column7:7(int) column8:8(int!null) a:9(int) b:10(int) c:11(int) rowid:12(int) column13:13(int) + │ ├── columns: upsert_b:16(int) upsert_c:17(int) upsert_rowid:18(int) column1:6(int!null) column7:7(int!null) column8:8(int) column9:9(int!null) a:10(int) b:11(int) c:12(int) rowid:13(int) crdb_internal_mvcc_timestamp:14(decimal) column15:15(int) │ ├── cardinality: [1 - 2] │ ├── volatile - │ ├── lax-key: (7,12) - │ ├── fd: ()-->(6,8), (7)~~>(5), (12)-->(9-11,14,15), (9)-->(10-12), (10,11)~~>(9,12), (10)-->(13), (7,12)-->(16) - │ ├── prune: (5-16) - │ ├── reject-nulls: (9-12) - │ ├── interesting orderings: (+12) (+9) (+10,+11,+12) + │ ├── lax-key: (8,13) + │ ├── fd: ()-->(7,9), (8)~~>(6), (13)-->(10-12,14,16,17), (10)-->(11-14), (11,12)~~>(10,13,14), (11)-->(15), (8,13)-->(18) + │ ├── prune: (6-18) + │ ├── reject-nulls: (10-14) + │ ├── interesting orderings: (+13) (+10) (+11,+12,+13) │ ├── project - │ │ ├── columns: column13:13(int) column1:5(int!null) column6:6(int!null) column7:7(int) column8:8(int!null) a:9(int) b:10(int) c:11(int) rowid:12(int) + │ │ ├── columns: column15:15(int) column1:6(int!null) column7:7(int!null) column8:8(int) column9:9(int!null) a:10(int) b:11(int) c:12(int) rowid:13(int) crdb_internal_mvcc_timestamp:14(decimal) │ │ ├── cardinality: [1 - 2] │ │ ├── volatile - │ │ ├── lax-key: (7,12) - │ │ ├── fd: ()-->(6,8), (7)~~>(5), (12)-->(9-11), (9)-->(10-12), (10,11)~~>(9,12), (10)-->(13) - │ │ ├── prune: (5-13) - │ │ ├── reject-nulls: (9-12) - │ │ ├── interesting orderings: (+12) (+9) (+10,+11,+12) + │ │ ├── lax-key: (8,13) + │ │ ├── fd: ()-->(7,9), (8)~~>(6), (13)-->(10-12,14), (10)-->(11-14), (11,12)~~>(10,13,14), (11)-->(15) + │ │ ├── prune: (6-15) + │ │ ├── reject-nulls: (10-14) + │ │ ├── interesting orderings: (+13) (+10) (+11,+12,+13) │ │ ├── left-join (hash) - │ │ │ ├── columns: column1:5(int!null) column6:6(int!null) column7:7(int) column8:8(int!null) a:9(int) b:10(int) c:11(int) rowid:12(int) + │ │ │ ├── columns: column1:6(int!null) column7:7(int!null) column8:8(int) column9:9(int!null) a:10(int) b:11(int) c:12(int) rowid:13(int) crdb_internal_mvcc_timestamp:14(decimal) │ │ │ ├── cardinality: [1 - 2] │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ │ ├── volatile - │ │ │ ├── lax-key: (7,12) - │ │ │ ├── fd: ()-->(6,8), (7)~~>(5), (12)-->(9-11), (9)-->(10-12), (10,11)~~>(9,12) - │ │ │ ├── prune: (9-11) - │ │ │ ├── reject-nulls: (9-12) - │ │ │ ├── interesting orderings: (+12) (+9) (+10,+11,+12) + │ │ │ ├── lax-key: (8,13) + │ │ │ ├── fd: ()-->(7,9), (8)~~>(6), (13)-->(10-12,14), (10)-->(11-14), (11,12)~~>(10,13,14) + │ │ │ ├── prune: (10-12,14) + │ │ │ ├── reject-nulls: (10-14) + │ │ │ ├── interesting orderings: (+13) (+10) (+11,+12,+13) │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ ├── columns: column1:5(int!null) column6:6(int!null) column7:7(int) column8:8(int!null) - │ │ │ │ ├── grouping columns: column7:7(int) + │ │ │ │ ├── columns: column1:6(int!null) column7:7(int!null) column8:8(int) column9:9(int!null) + │ │ │ │ ├── grouping columns: column8:8(int) │ │ │ │ ├── error: "UPSERT or INSERT...ON CONFLICT command cannot affect row a second time" │ │ │ │ ├── cardinality: [1 - 2] │ │ │ │ ├── volatile - │ │ │ │ ├── lax-key: (7) - │ │ │ │ ├── fd: ()-->(6,8), (7)~~>(5,6,8) + │ │ │ │ ├── lax-key: (8) + │ │ │ │ ├── fd: ()-->(7,9), (8)~~>(6,7,9) │ │ │ │ ├── project - │ │ │ │ │ ├── columns: column8:8(int!null) column1:5(int!null) column6:6(int!null) column7:7(int) + │ │ │ │ │ ├── columns: column9:9(int!null) column1:6(int!null) column7:7(int!null) column8:8(int) │ │ │ │ │ ├── cardinality: [2 - 2] │ │ │ │ │ ├── volatile - │ │ │ │ │ ├── fd: ()-->(6,8) - │ │ │ │ │ ├── prune: (5-8) + │ │ │ │ │ ├── fd: ()-->(7,9) + │ │ │ │ │ ├── prune: (6-9) │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: column6:6(int!null) column7:7(int) column1:5(int!null) + │ │ │ │ │ │ ├── columns: column7:7(int!null) column8:8(int) column1:6(int!null) │ │ │ │ │ │ ├── cardinality: [2 - 2] │ │ │ │ │ │ ├── volatile - │ │ │ │ │ │ ├── fd: ()-->(6) - │ │ │ │ │ │ ├── prune: (5-7) + │ │ │ │ │ │ ├── fd: ()-->(7) + │ │ │ │ │ │ ├── prune: (6-8) │ │ │ │ │ │ ├── values - │ │ │ │ │ │ │ ├── columns: column1:5(int!null) + │ │ │ │ │ │ │ ├── columns: column1:6(int!null) │ │ │ │ │ │ │ ├── cardinality: [2 - 2] - │ │ │ │ │ │ │ ├── prune: (5) + │ │ │ │ │ │ │ ├── prune: (6) │ │ │ │ │ │ │ ├── tuple [type=tuple{int}] │ │ │ │ │ │ │ │ └── const: 1 [type=int] │ │ │ │ │ │ │ └── tuple [type=tuple{int}] │ │ │ │ │ │ │ └── const: 2 [type=int] │ │ │ │ │ │ └── projections - │ │ │ │ │ │ ├── const: 10 [as=column6:6, type=int] - │ │ │ │ │ │ └── function: unique_rowid [as=column7:7, type=int, volatile] + │ │ │ │ │ │ ├── const: 10 [as=column7:7, type=int] + │ │ │ │ │ │ └── function: unique_rowid [as=column8:8, type=int, volatile] │ │ │ │ │ └── projections - │ │ │ │ │ └── plus [as=column8:8, type=int, outer=(6), immutable] - │ │ │ │ │ ├── variable: column6:6 [type=int] + │ │ │ │ │ └── plus [as=column9:9, type=int, outer=(7), immutable] + │ │ │ │ │ ├── variable: column7:7 [type=int] │ │ │ │ │ └── const: 1 [type=int] │ │ │ │ └── aggregations - │ │ │ │ ├── first-agg [as=column1:5, type=int, outer=(5)] - │ │ │ │ │ └── variable: column1:5 [type=int] - │ │ │ │ ├── first-agg [as=column6:6, type=int, outer=(6)] - │ │ │ │ │ └── variable: column6:6 [type=int] - │ │ │ │ └── first-agg [as=column8:8, type=int, outer=(8)] - │ │ │ │ └── variable: column8:8 [type=int] + │ │ │ │ ├── first-agg [as=column1:6, type=int, outer=(6)] + │ │ │ │ │ └── variable: column1:6 [type=int] + │ │ │ │ ├── first-agg [as=column7:7, type=int, outer=(7)] + │ │ │ │ │ └── variable: column7:7 [type=int] + │ │ │ │ └── first-agg [as=column9:9, type=int, outer=(9)] + │ │ │ │ └── variable: column9:9 [type=int] │ │ │ ├── scan abc - │ │ │ │ ├── columns: a:9(int!null) b:10(int) c:11(int) rowid:12(int!null) + │ │ │ │ ├── columns: a:10(int!null) b:11(int) c:12(int) rowid:13(int!null) crdb_internal_mvcc_timestamp:14(decimal) │ │ │ │ ├── computed column expressions - │ │ │ │ │ └── c:11 + │ │ │ │ │ └── c:12 │ │ │ │ │ └── plus [type=int] - │ │ │ │ │ ├── variable: b:10 [type=int] + │ │ │ │ │ ├── variable: b:11 [type=int] │ │ │ │ │ └── const: 1 [type=int] - │ │ │ │ ├── key: (12) - │ │ │ │ ├── fd: (12)-->(9-11), (9)-->(10-12), (10,11)~~>(9,12) - │ │ │ │ ├── prune: (9-12) - │ │ │ │ ├── interesting orderings: (+12) (+9) (+10,+11,+12) - │ │ │ │ └── unfiltered-cols: (9-12) + │ │ │ │ ├── key: (13) + │ │ │ │ ├── fd: (13)-->(10-12,14), (10)-->(11-14), (11,12)~~>(10,13,14) + │ │ │ │ ├── prune: (10-14) + │ │ │ │ ├── interesting orderings: (+13) (+10) (+11,+12,+13) + │ │ │ │ └── unfiltered-cols: (10-14) │ │ │ └── filters - │ │ │ └── eq [type=bool, outer=(7,12), constraints=(/7: (/NULL - ]; /12: (/NULL - ]), fd=(7)==(12), (12)==(7)] - │ │ │ ├── variable: column7:7 [type=int] - │ │ │ └── variable: rowid:12 [type=int] + │ │ │ └── eq [type=bool, outer=(8,13), constraints=(/8: (/NULL - ]; /13: (/NULL - ]), fd=(8)==(13), (13)==(8)] + │ │ │ ├── variable: column8:8 [type=int] + │ │ │ └── variable: rowid:13 [type=int] │ │ └── projections - │ │ └── plus [as=column13:13, type=int, outer=(10), immutable] - │ │ ├── variable: b:10 [type=int] + │ │ └── plus [as=column15:15, type=int, outer=(11), immutable] + │ │ ├── variable: b:11 [type=int] │ │ └── const: 1 [type=int] │ └── projections - │ ├── case [as=upsert_b:14, type=int, outer=(6,10,12)] + │ ├── case [as=upsert_b:16, type=int, outer=(7,11,13)] │ │ ├── true [type=bool] │ │ ├── when [type=int] │ │ │ ├── is [type=bool] - │ │ │ │ ├── variable: rowid:12 [type=int] + │ │ │ │ ├── variable: rowid:13 [type=int] │ │ │ │ └── null [type=unknown] - │ │ │ └── variable: column6:6 [type=int] - │ │ └── variable: b:10 [type=int] - │ ├── case [as=upsert_c:15, type=int, outer=(8,12,13)] + │ │ │ └── variable: column7:7 [type=int] + │ │ └── variable: b:11 [type=int] + │ ├── case [as=upsert_c:17, type=int, outer=(9,13,15)] │ │ ├── true [type=bool] │ │ ├── when [type=int] │ │ │ ├── is [type=bool] - │ │ │ │ ├── variable: rowid:12 [type=int] + │ │ │ │ ├── variable: rowid:13 [type=int] │ │ │ │ └── null [type=unknown] - │ │ │ └── variable: column8:8 [type=int] - │ │ └── variable: column13:13 [type=int] - │ └── case [as=upsert_rowid:16, type=int, outer=(7,12)] + │ │ │ └── variable: column9:9 [type=int] + │ │ └── variable: column15:15 [type=int] + │ └── case [as=upsert_rowid:18, type=int, outer=(8,13)] │ ├── true [type=bool] │ ├── when [type=int] │ │ ├── is [type=bool] - │ │ │ ├── variable: rowid:12 [type=int] + │ │ │ ├── variable: rowid:13 [type=int] │ │ │ └── null [type=unknown] - │ │ └── variable: column7:7 [type=int] - │ └── variable: rowid:12 [type=int] + │ │ └── variable: column8:8 [type=int] + │ └── variable: rowid:13 [type=int] └── projections - └── plus [as="?column?":17, type=int, outer=(2,3), immutable] + └── plus [as="?column?":19, type=int, outer=(2,3), immutable] ├── variable: b:2 [type=int] └── variable: c:3 [type=int] @@ -590,156 +590,156 @@ UPDATE SET b=2 ---- upsert abc ├── columns: - ├── canary column: 14 - ├── fetch columns: a:11(int) b:12(int) c:13(int) rowid:14(int) + ├── canary column: 16 + ├── fetch columns: a:13(int) b:14(int) c:15(int) rowid:16(int) ├── insert-mapping: - │ ├── y:6 => a:1 - │ ├── column8:8 => b:2 - │ ├── column10:10 => c:3 - │ └── column9:9 => rowid:4 + │ ├── y:7 => a:1 + │ ├── column10:10 => b:2 + │ ├── column12:12 => c:3 + │ └── column11:11 => rowid:4 ├── update-mapping: - │ ├── upsert_b:18 => b:2 - │ └── upsert_c:19 => c:3 + │ ├── upsert_b:21 => b:2 + │ └── upsert_c:22 => c:3 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: upsert_a:17(int) upsert_b:18(int!null) upsert_c:19(int!null) upsert_rowid:20(int) y:6(int!null) column8:8(int!null) column9:9(int) column10:10(int!null) a:11(int) b:12(int) c:13(int) rowid:14(int) b_new:15(int!null) column16:16(int!null) + ├── columns: upsert_a:20(int) upsert_b:21(int!null) upsert_c:22(int!null) upsert_rowid:23(int) y:7(int!null) column10:10(int!null) column11:11(int) column12:12(int!null) a:13(int) b:14(int) c:15(int) rowid:16(int) abc.crdb_internal_mvcc_timestamp:17(decimal) b_new:18(int!null) column19:19(int!null) ├── volatile - ├── key: (6) - ├── fd: ()-->(8,10,15,16), (6)-->(9,11-14,17), (14)-->(11-13,18,19), (11)-->(12-14), (12,13)~~>(11,14), (9,14)-->(20) - ├── prune: (6,8-20) - ├── reject-nulls: (11-14) - ├── interesting orderings: (+14) (+11) (+12,+13,+14) + ├── key: (7) + ├── fd: ()-->(10,12,18,19), (7)-->(11,13-17,20), (16)-->(13-15,17,21,22), (13)-->(14-17), (14,15)~~>(13,16,17), (11,16)-->(23) + ├── prune: (7,10-23) + ├── reject-nulls: (13-17) + ├── interesting orderings: (+16) (+13) (+14,+15,+16) ├── project - │ ├── columns: column16:16(int!null) y:6(int!null) column8:8(int!null) column9:9(int) column10:10(int!null) a:11(int) b:12(int) c:13(int) rowid:14(int) b_new:15(int!null) + │ ├── columns: column19:19(int!null) y:7(int!null) column10:10(int!null) column11:11(int) column12:12(int!null) a:13(int) b:14(int) c:15(int) rowid:16(int) abc.crdb_internal_mvcc_timestamp:17(decimal) b_new:18(int!null) │ ├── volatile - │ ├── key: (6) - │ ├── fd: ()-->(8,10,15,16), (6)-->(9,11-14), (14)-->(11-13), (11)-->(12-14), (12,13)~~>(11,14) - │ ├── prune: (6,8-16) - │ ├── reject-nulls: (11-14) - │ ├── interesting orderings: (+14) (+11) (+12,+13,+14) + │ ├── key: (7) + │ ├── fd: ()-->(10,12,18,19), (7)-->(11,13-17), (16)-->(13-15,17), (13)-->(14-17), (14,15)~~>(13,16,17) + │ ├── prune: (7,10-19) + │ ├── reject-nulls: (13-17) + │ ├── interesting orderings: (+16) (+13) (+14,+15,+16) │ ├── project - │ │ ├── columns: b_new:15(int!null) y:6(int!null) column8:8(int!null) column9:9(int) column10:10(int!null) a:11(int) b:12(int) c:13(int) rowid:14(int) + │ │ ├── columns: b_new:18(int!null) y:7(int!null) column10:10(int!null) column11:11(int) column12:12(int!null) a:13(int) b:14(int) c:15(int) rowid:16(int) abc.crdb_internal_mvcc_timestamp:17(decimal) │ │ ├── volatile - │ │ ├── key: (6) - │ │ ├── fd: ()-->(8,10,15), (6)-->(9,11-14), (14)-->(11-13), (11)-->(12-14), (12,13)~~>(11,14) - │ │ ├── prune: (6,8-15) - │ │ ├── reject-nulls: (11-14) - │ │ ├── interesting orderings: (+14) (+11) (+12,+13,+14) + │ │ ├── key: (7) + │ │ ├── fd: ()-->(10,12,18), (7)-->(11,13-17), (16)-->(13-15,17), (13)-->(14-17), (14,15)~~>(13,16,17) + │ │ ├── prune: (7,10-18) + │ │ ├── reject-nulls: (13-17) + │ │ ├── interesting orderings: (+16) (+13) (+14,+15,+16) │ │ ├── left-join (hash) - │ │ │ ├── columns: y:6(int!null) column8:8(int!null) column9:9(int) column10:10(int!null) a:11(int) b:12(int) c:13(int) rowid:14(int) + │ │ │ ├── columns: y:7(int!null) column10:10(int!null) column11:11(int) column12:12(int!null) a:13(int) b:14(int) c:15(int) rowid:16(int) abc.crdb_internal_mvcc_timestamp:17(decimal) │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ │ ├── volatile - │ │ │ ├── key: (6) - │ │ │ ├── fd: ()-->(8,10), (6)-->(9,11-14), (14)-->(11-13), (11)-->(12-14), (12,13)~~>(11,14) - │ │ │ ├── prune: (12-14) - │ │ │ ├── reject-nulls: (11-14) - │ │ │ ├── interesting orderings: (+14) (+11) (+12,+13,+14) + │ │ │ ├── key: (7) + │ │ │ ├── fd: ()-->(10,12), (7)-->(11,13-17), (16)-->(13-15,17), (13)-->(14-17), (14,15)~~>(13,16,17) + │ │ │ ├── prune: (14-17) + │ │ │ ├── reject-nulls: (13-17) + │ │ │ ├── interesting orderings: (+16) (+13) (+14,+15,+16) │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ ├── columns: y:6(int!null) column8:8(int!null) column9:9(int) column10:10(int!null) - │ │ │ │ ├── grouping columns: y:6(int!null) + │ │ │ │ ├── columns: y:7(int!null) column10:10(int!null) column11:11(int) column12:12(int!null) + │ │ │ │ ├── grouping columns: y:7(int!null) │ │ │ │ ├── error: "UPSERT or INSERT...ON CONFLICT command cannot affect row a second time" │ │ │ │ ├── volatile - │ │ │ │ ├── key: (6) - │ │ │ │ ├── fd: ()-->(8,10), (6)-->(8-10) + │ │ │ │ ├── key: (7) + │ │ │ │ ├── fd: ()-->(10,12), (7)-->(10-12) │ │ │ │ ├── project - │ │ │ │ │ ├── columns: column10:10(int!null) y:6(int!null) column8:8(int!null) column9:9(int) + │ │ │ │ │ ├── columns: column12:12(int!null) y:7(int!null) column10:10(int!null) column11:11(int) │ │ │ │ │ ├── volatile - │ │ │ │ │ ├── fd: ()-->(8,10) - │ │ │ │ │ ├── prune: (6,8-10) - │ │ │ │ │ ├── interesting orderings: (+6) + │ │ │ │ │ ├── fd: ()-->(10,12) + │ │ │ │ │ ├── prune: (7,10-12) + │ │ │ │ │ ├── interesting orderings: (+7) │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: column8:8(int!null) column9:9(int) y:6(int!null) + │ │ │ │ │ │ ├── columns: column10:10(int!null) column11:11(int) y:7(int!null) │ │ │ │ │ │ ├── volatile - │ │ │ │ │ │ ├── fd: ()-->(8) - │ │ │ │ │ │ ├── prune: (6,8,9) - │ │ │ │ │ │ ├── interesting orderings: (+6) + │ │ │ │ │ │ ├── fd: ()-->(10) + │ │ │ │ │ │ ├── prune: (7,10,11) + │ │ │ │ │ │ ├── interesting orderings: (+7) │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ ├── columns: y:6(int!null) - │ │ │ │ │ │ │ ├── prune: (6) - │ │ │ │ │ │ │ ├── interesting orderings: (+6) + │ │ │ │ │ │ │ ├── columns: y:7(int!null) + │ │ │ │ │ │ │ ├── prune: (7) + │ │ │ │ │ │ │ ├── interesting orderings: (+7) │ │ │ │ │ │ │ └── select - │ │ │ │ │ │ │ ├── columns: x:5(int!null) y:6(int!null) z:7(int) - │ │ │ │ │ │ │ ├── key: (5) - │ │ │ │ │ │ │ ├── fd: (5)-->(6,7), (6,7)~~>(5) - │ │ │ │ │ │ │ ├── prune: (5,7) - │ │ │ │ │ │ │ ├── interesting orderings: (+5) (+6,+7,+5) (+7,+6,+5) + │ │ │ │ │ │ │ ├── columns: x:6(int!null) y:7(int!null) z:8(int) xyz.crdb_internal_mvcc_timestamp:9(decimal) + │ │ │ │ │ │ │ ├── key: (6) + │ │ │ │ │ │ │ ├── fd: (6)-->(7-9), (7,8)~~>(6,9) + │ │ │ │ │ │ │ ├── prune: (6,8,9) + │ │ │ │ │ │ │ ├── interesting orderings: (+6) (+7,+8,+6) (+8,+7,+6) │ │ │ │ │ │ │ ├── scan xyz - │ │ │ │ │ │ │ │ ├── columns: x:5(int!null) y:6(int) z:7(int) - │ │ │ │ │ │ │ │ ├── key: (5) - │ │ │ │ │ │ │ │ ├── fd: (5)-->(6,7), (6,7)~~>(5) - │ │ │ │ │ │ │ │ ├── prune: (5-7) - │ │ │ │ │ │ │ │ └── interesting orderings: (+5) (+6,+7,+5) (+7,+6,+5) + │ │ │ │ │ │ │ │ ├── columns: x:6(int!null) y:7(int) z:8(int) xyz.crdb_internal_mvcc_timestamp:9(decimal) + │ │ │ │ │ │ │ │ ├── key: (6) + │ │ │ │ │ │ │ │ ├── fd: (6)-->(7-9), (7,8)~~>(6,9) + │ │ │ │ │ │ │ │ ├── prune: (6-9) + │ │ │ │ │ │ │ │ └── interesting orderings: (+6) (+7,+8,+6) (+8,+7,+6) │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── is-not [type=bool, outer=(6), constraints=(/6: (/NULL - ]; tight)] - │ │ │ │ │ │ │ ├── variable: y:6 [type=int] + │ │ │ │ │ │ │ └── is-not [type=bool, outer=(7), constraints=(/7: (/NULL - ]; tight)] + │ │ │ │ │ │ │ ├── variable: y:7 [type=int] │ │ │ │ │ │ │ └── null [type=unknown] │ │ │ │ │ │ └── projections - │ │ │ │ │ │ ├── const: 10 [as=column8:8, type=int] - │ │ │ │ │ │ └── function: unique_rowid [as=column9:9, type=int, volatile] + │ │ │ │ │ │ ├── const: 10 [as=column10:10, type=int] + │ │ │ │ │ │ └── function: unique_rowid [as=column11:11, type=int, volatile] │ │ │ │ │ └── projections - │ │ │ │ │ └── plus [as=column10:10, type=int, outer=(8), immutable] - │ │ │ │ │ ├── variable: column8:8 [type=int] + │ │ │ │ │ └── plus [as=column12:12, type=int, outer=(10), immutable] + │ │ │ │ │ ├── variable: column10:10 [type=int] │ │ │ │ │ └── const: 1 [type=int] │ │ │ │ └── aggregations - │ │ │ │ ├── first-agg [as=column8:8, type=int, outer=(8)] - │ │ │ │ │ └── variable: column8:8 [type=int] - │ │ │ │ ├── first-agg [as=column9:9, type=int, outer=(9)] - │ │ │ │ │ └── variable: column9:9 [type=int] - │ │ │ │ └── first-agg [as=column10:10, type=int, outer=(10)] - │ │ │ │ └── variable: column10:10 [type=int] + │ │ │ │ ├── first-agg [as=column10:10, type=int, outer=(10)] + │ │ │ │ │ └── variable: column10:10 [type=int] + │ │ │ │ ├── first-agg [as=column11:11, type=int, outer=(11)] + │ │ │ │ │ └── variable: column11:11 [type=int] + │ │ │ │ └── first-agg [as=column12:12, type=int, outer=(12)] + │ │ │ │ └── variable: column12:12 [type=int] │ │ │ ├── scan abc - │ │ │ │ ├── columns: a:11(int!null) b:12(int) c:13(int) rowid:14(int!null) + │ │ │ │ ├── columns: a:13(int!null) b:14(int) c:15(int) rowid:16(int!null) abc.crdb_internal_mvcc_timestamp:17(decimal) │ │ │ │ ├── computed column expressions - │ │ │ │ │ └── c:13 + │ │ │ │ │ └── c:15 │ │ │ │ │ └── plus [type=int] - │ │ │ │ │ ├── variable: b:12 [type=int] + │ │ │ │ │ ├── variable: b:14 [type=int] │ │ │ │ │ └── const: 1 [type=int] - │ │ │ │ ├── key: (14) - │ │ │ │ ├── fd: (14)-->(11-13), (11)-->(12-14), (12,13)~~>(11,14) - │ │ │ │ ├── prune: (11-14) - │ │ │ │ ├── interesting orderings: (+14) (+11) (+12,+13,+14) - │ │ │ │ └── unfiltered-cols: (11-14) + │ │ │ │ ├── key: (16) + │ │ │ │ ├── fd: (16)-->(13-15,17), (13)-->(14-17), (14,15)~~>(13,16,17) + │ │ │ │ ├── prune: (13-17) + │ │ │ │ ├── interesting orderings: (+16) (+13) (+14,+15,+16) + │ │ │ │ └── unfiltered-cols: (13-17) │ │ │ └── filters - │ │ │ └── eq [type=bool, outer=(6,11), constraints=(/6: (/NULL - ]; /11: (/NULL - ]), fd=(6)==(11), (11)==(6)] - │ │ │ ├── variable: y:6 [type=int] - │ │ │ └── variable: a:11 [type=int] + │ │ │ └── eq [type=bool, outer=(7,13), constraints=(/7: (/NULL - ]; /13: (/NULL - ]), fd=(7)==(13), (13)==(7)] + │ │ │ ├── variable: y:7 [type=int] + │ │ │ └── variable: a:13 [type=int] │ │ └── projections - │ │ └── const: 2 [as=b_new:15, type=int] + │ │ └── const: 2 [as=b_new:18, type=int] │ └── projections - │ └── plus [as=column16:16, type=int, outer=(15), immutable] - │ ├── variable: b_new:15 [type=int] + │ └── plus [as=column19:19, type=int, outer=(18), immutable] + │ ├── variable: b_new:18 [type=int] │ └── const: 1 [type=int] └── projections - ├── case [as=upsert_a:17, type=int, outer=(6,11,14)] + ├── case [as=upsert_a:20, type=int, outer=(7,13,16)] │ ├── true [type=bool] │ ├── when [type=int] │ │ ├── is [type=bool] - │ │ │ ├── variable: rowid:14 [type=int] + │ │ │ ├── variable: rowid:16 [type=int] │ │ │ └── null [type=unknown] - │ │ └── variable: y:6 [type=int] - │ └── variable: a:11 [type=int] - ├── case [as=upsert_b:18, type=int, outer=(8,14,15)] + │ │ └── variable: y:7 [type=int] + │ └── variable: a:13 [type=int] + ├── case [as=upsert_b:21, type=int, outer=(10,16,18)] │ ├── true [type=bool] │ ├── when [type=int] │ │ ├── is [type=bool] - │ │ │ ├── variable: rowid:14 [type=int] + │ │ │ ├── variable: rowid:16 [type=int] │ │ │ └── null [type=unknown] - │ │ └── variable: column8:8 [type=int] - │ └── variable: b_new:15 [type=int] - ├── case [as=upsert_c:19, type=int, outer=(10,14,16)] + │ │ └── variable: column10:10 [type=int] + │ └── variable: b_new:18 [type=int] + ├── case [as=upsert_c:22, type=int, outer=(12,16,19)] │ ├── true [type=bool] │ ├── when [type=int] │ │ ├── is [type=bool] - │ │ │ ├── variable: rowid:14 [type=int] + │ │ │ ├── variable: rowid:16 [type=int] │ │ │ └── null [type=unknown] - │ │ └── variable: column10:10 [type=int] - │ └── variable: column16:16 [type=int] - └── case [as=upsert_rowid:20, type=int, outer=(9,14)] + │ │ └── variable: column12:12 [type=int] + │ └── variable: column19:19 [type=int] + └── case [as=upsert_rowid:23, type=int, outer=(11,16)] ├── true [type=bool] ├── when [type=int] │ ├── is [type=bool] - │ │ ├── variable: rowid:14 [type=int] + │ │ ├── variable: rowid:16 [type=int] │ │ └── null [type=unknown] - │ └── variable: column9:9 [type=int] - └── variable: rowid:14 [type=int] + │ └── variable: column11:11 [type=int] + └── variable: rowid:16 [type=int] diff --git a/pkg/sql/opt/memo/testdata/logprops/values b/pkg/sql/opt/memo/testdata/logprops/values index ebf8a5475cc3..44dff7f06872 100644 --- a/pkg/sql/opt/memo/testdata/logprops/values +++ b/pkg/sql/opt/memo/testdata/logprops/values @@ -63,31 +63,31 @@ build SELECT (VALUES (x), (y+1)) FROM xy ---- project - ├── columns: column1:4(int) + ├── columns: column1:5(int) ├── immutable - ├── prune: (4) + ├── prune: (5) ├── scan xy - │ ├── columns: x:1(int!null) y:2(int) + │ ├── columns: x:1(int!null) y:2(int) crdb_internal_mvcc_timestamp:3(decimal) │ ├── key: (1) - │ ├── fd: (1)-->(2) - │ ├── prune: (1,2) + │ ├── fd: (1)-->(2,3) + │ ├── prune: (1-3) │ └── interesting orderings: (+1) └── projections - └── subquery [as=column1:4, type=int, outer=(1,2), immutable, correlated-subquery] + └── subquery [as=column1:5, type=int, outer=(1,2), immutable, correlated-subquery] └── max1-row - ├── columns: column1:3(int) + ├── columns: column1:4(int) ├── error: "more than one row returned by a subquery used as an expression" ├── outer: (1,2) ├── cardinality: [1 - 1] ├── immutable ├── key: () - ├── fd: ()-->(3) + ├── fd: ()-->(4) └── values - ├── columns: column1:3(int) + ├── columns: column1:4(int) ├── outer: (1,2) ├── cardinality: [2 - 2] ├── immutable - ├── prune: (3) + ├── prune: (4) ├── tuple [type=tuple{int}] │ └── variable: x:1 [type=int] └── tuple [type=tuple{int}] diff --git a/pkg/sql/opt/memo/testdata/logprops/window b/pkg/sql/opt/memo/testdata/logprops/window index 5cbe31bb381c..24f017061b37 100644 --- a/pkg/sql/opt/memo/testdata/logprops/window +++ b/pkg/sql/opt/memo/testdata/logprops/window @@ -21,17 +21,17 @@ build SELECT k, rank() OVER () FROM (SELECT * FROM kv LIMIT 10) ---- project - ├── columns: k:1(int!null) rank:8(int) + ├── columns: k:1(int!null) rank:9(int) ├── cardinality: [0 - 10] ├── key: (1) - ├── fd: (1)-->(8) - ├── prune: (1,8) + ├── fd: (1)-->(9) + ├── prune: (1,9) └── window partition=() - ├── columns: k:1(int!null) v:2(int) w:3(int) f:4(float) d:5(decimal) s:6(string) b:7(bool) rank:8(int) + ├── columns: k:1(int!null) v:2(int) w:3(int) f:4(float) d:5(decimal) s:6(string) b:7(bool) rank:9(int) ├── cardinality: [0 - 10] ├── key: (1) ├── fd: (1)-->(2-7) - ├── prune: (1-8) + ├── prune: (1-7,9) ├── limit │ ├── columns: k:1(int!null) v:2(int) w:3(int) f:4(float) d:5(decimal) s:6(string) b:7(bool) │ ├── cardinality: [0 - 10] @@ -39,32 +39,39 @@ project │ ├── fd: (1)-->(2-7) │ ├── prune: (1-7) │ ├── interesting orderings: (+1) - │ ├── scan kv + │ ├── project │ │ ├── columns: k:1(int!null) v:2(int) w:3(int) f:4(float) d:5(decimal) s:6(string) b:7(bool) │ │ ├── key: (1) │ │ ├── fd: (1)-->(2-7) │ │ ├── limit hint: 10.00 │ │ ├── prune: (1-7) - │ │ └── interesting orderings: (+1) + │ │ ├── interesting orderings: (+1) + │ │ └── scan kv + │ │ ├── columns: k:1(int!null) v:2(int) w:3(int) f:4(float) d:5(decimal) s:6(string) b:7(bool) crdb_internal_mvcc_timestamp:8(decimal) + │ │ ├── key: (1) + │ │ ├── fd: (1)-->(2-8) + │ │ ├── limit hint: 10.00 + │ │ ├── prune: (1-8) + │ │ └── interesting orderings: (+1) │ └── const: 10 [type=int] └── windows - └── rank [as=rank:8, type=int] + └── rank [as=rank:9, type=int] build SELECT k, rank() OVER (PARTITION BY v ORDER BY f) FROM (SELECT * FROM kv LIMIT 10) ---- project - ├── columns: k:1(int!null) rank:8(int) + ├── columns: k:1(int!null) rank:9(int) ├── cardinality: [0 - 10] ├── key: (1) - ├── fd: (1)-->(8) - ├── prune: (1,8) + ├── fd: (1)-->(9) + ├── prune: (1,9) └── window partition=(2) ordering=+4 - ├── columns: k:1(int!null) v:2(int) w:3(int) f:4(float) d:5(decimal) s:6(string) b:7(bool) rank:8(int) + ├── columns: k:1(int!null) v:2(int) w:3(int) f:4(float) d:5(decimal) s:6(string) b:7(bool) rank:9(int) ├── cardinality: [0 - 10] ├── key: (1) ├── fd: (1)-->(2-7) - ├── prune: (1,3,5-8) + ├── prune: (1,3,5-7,9) ├── limit │ ├── columns: k:1(int!null) v:2(int) w:3(int) f:4(float) d:5(decimal) s:6(string) b:7(bool) │ ├── cardinality: [0 - 10] @@ -72,16 +79,23 @@ project │ ├── fd: (1)-->(2-7) │ ├── prune: (1-7) │ ├── interesting orderings: (+1) - │ ├── scan kv + │ ├── project │ │ ├── columns: k:1(int!null) v:2(int) w:3(int) f:4(float) d:5(decimal) s:6(string) b:7(bool) │ │ ├── key: (1) │ │ ├── fd: (1)-->(2-7) │ │ ├── limit hint: 10.00 │ │ ├── prune: (1-7) - │ │ └── interesting orderings: (+1) + │ │ ├── interesting orderings: (+1) + │ │ └── scan kv + │ │ ├── columns: k:1(int!null) v:2(int) w:3(int) f:4(float) d:5(decimal) s:6(string) b:7(bool) crdb_internal_mvcc_timestamp:8(decimal) + │ │ ├── key: (1) + │ │ ├── fd: (1)-->(2-8) + │ │ ├── limit hint: 10.00 + │ │ ├── prune: (1-8) + │ │ └── interesting orderings: (+1) │ └── const: 10 [type=int] └── windows - └── rank [as=rank:8, type=int] + └── rank [as=rank:9, type=int] # Outer cols. @@ -89,102 +103,102 @@ build SELECT k, (SELECT rank() OVER () + x FROM (SELECT k AS x)) FROM kv ---- project - ├── columns: k:1(int!null) "?column?":11(int) + ├── columns: k:1(int!null) "?column?":12(int) ├── immutable ├── key: (1) - ├── fd: (1)-->(11) - ├── prune: (1,11) + ├── fd: (1)-->(12) + ├── prune: (1,12) ├── interesting orderings: (+1) ├── scan kv - │ ├── columns: k:1(int!null) v:2(int) w:3(int) f:4(float) d:5(decimal) s:6(string) b:7(bool) + │ ├── columns: k:1(int!null) v:2(int) w:3(int) f:4(float) d:5(decimal) s:6(string) b:7(bool) crdb_internal_mvcc_timestamp:8(decimal) │ ├── key: (1) - │ ├── fd: (1)-->(2-7) - │ ├── prune: (1-7) + │ ├── fd: (1)-->(2-8) + │ ├── prune: (1-8) │ └── interesting orderings: (+1) └── projections - └── subquery [as="?column?":11, type=int, outer=(1), immutable, correlated-subquery] + └── subquery [as="?column?":12, type=int, outer=(1), immutable, correlated-subquery] └── max1-row - ├── columns: "?column?":10(int) + ├── columns: "?column?":11(int) ├── error: "more than one row returned by a subquery used as an expression" ├── outer: (1) ├── cardinality: [1 - 1] ├── immutable ├── key: () - ├── fd: ()-->(10) + ├── fd: ()-->(11) └── project - ├── columns: "?column?":10(int) + ├── columns: "?column?":11(int) ├── outer: (1) ├── cardinality: [1 - 1] ├── immutable ├── key: () - ├── fd: ()-->(10) - ├── prune: (10) + ├── fd: ()-->(11) + ├── prune: (11) ├── window partition=() - │ ├── columns: x:8(int) rank:9(int) + │ ├── columns: x:9(int) rank:10(int) │ ├── outer: (1) │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(8) - │ ├── prune: (8,9) + │ ├── fd: ()-->(9) + │ ├── prune: (9,10) │ ├── project - │ │ ├── columns: x:8(int) + │ │ ├── columns: x:9(int) │ │ ├── outer: (1) │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(8) - │ │ ├── prune: (8) + │ │ ├── fd: ()-->(9) + │ │ ├── prune: (9) │ │ ├── values │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () │ │ │ └── tuple [type=tuple] │ │ └── projections - │ │ └── variable: k:1 [as=x:8, type=int, outer=(1)] + │ │ └── variable: k:1 [as=x:9, type=int, outer=(1)] │ └── windows - │ └── rank [as=rank:9, type=int] + │ └── rank [as=rank:10, type=int] └── projections - └── plus [as="?column?":10, type=int, outer=(8,9), immutable] - ├── variable: rank:9 [type=int] - └── variable: x:8 [type=int] + └── plus [as="?column?":11, type=int, outer=(9,10), immutable] + ├── variable: rank:10 [type=int] + └── variable: x:9 [type=int] build SELECT lag('foo'::string) OVER (), lag(1) OVER () FROM kv ---- project - ├── columns: lag:8(string) lag:9(int) + ├── columns: lag:9(string) lag:10(int) ├── immutable - ├── prune: (8,9) + ├── prune: (9,10) └── window partition=() - ├── columns: k:1(int!null) v:2(int) w:3(int) f:4(float) d:5(decimal) s:6(string) b:7(bool) lag:8(string) lag:9(int) lag_1_arg1:10(string!null) lag_1_arg2:11(int!null) lag_1_arg3:12(string) lag_2_arg3:13(int) + ├── columns: k:1(int!null) v:2(int) w:3(int) f:4(float) d:5(decimal) s:6(string) b:7(bool) crdb_internal_mvcc_timestamp:8(decimal) lag:9(string) lag:10(int) lag_1_arg1:11(string!null) lag_1_arg2:12(int!null) lag_1_arg3:13(string) lag_2_arg3:14(int) ├── immutable ├── key: (1) - ├── fd: ()-->(10-13), (1)-->(2-7) - ├── prune: (1-9) + ├── fd: ()-->(11-14), (1)-->(2-8) + ├── prune: (1-10) ├── project - │ ├── columns: lag_1_arg1:10(string!null) lag_1_arg2:11(int!null) lag_1_arg3:12(string) lag_2_arg3:13(int) k:1(int!null) v:2(int) w:3(int) f:4(float) d:5(decimal) s:6(string) b:7(bool) + │ ├── columns: lag_1_arg1:11(string!null) lag_1_arg2:12(int!null) lag_1_arg3:13(string) lag_2_arg3:14(int) k:1(int!null) v:2(int) w:3(int) f:4(float) d:5(decimal) s:6(string) b:7(bool) crdb_internal_mvcc_timestamp:8(decimal) │ ├── immutable │ ├── key: (1) - │ ├── fd: ()-->(10-13), (1)-->(2-7) - │ ├── prune: (1-7,10-13) + │ ├── fd: ()-->(11-14), (1)-->(2-8) + │ ├── prune: (1-8,11-14) │ ├── interesting orderings: (+1) │ ├── scan kv - │ │ ├── columns: k:1(int!null) v:2(int) w:3(int) f:4(float) d:5(decimal) s:6(string) b:7(bool) + │ │ ├── columns: k:1(int!null) v:2(int) w:3(int) f:4(float) d:5(decimal) s:6(string) b:7(bool) crdb_internal_mvcc_timestamp:8(decimal) │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2-7) - │ │ ├── prune: (1-7) + │ │ ├── fd: (1)-->(2-8) + │ │ ├── prune: (1-8) │ │ └── interesting orderings: (+1) │ └── projections - │ ├── const: 'foo' [as=lag_1_arg1:10, type=string] - │ ├── const: 1 [as=lag_1_arg2:11, type=int] - │ ├── cast: STRING [as=lag_1_arg3:12, type=string, immutable] + │ ├── const: 'foo' [as=lag_1_arg1:11, type=string] + │ ├── const: 1 [as=lag_1_arg2:12, type=int] + │ ├── cast: STRING [as=lag_1_arg3:13, type=string, immutable] │ │ └── null [type=unknown] - │ └── cast: INT8 [as=lag_2_arg3:13, type=int, immutable] + │ └── cast: INT8 [as=lag_2_arg3:14, type=int, immutable] │ └── null [type=unknown] └── windows - ├── lag [as=lag:8, type=string, outer=(10-12)] - │ ├── variable: lag_1_arg1:10 [type=string] - │ ├── variable: lag_1_arg2:11 [type=int] - │ └── variable: lag_1_arg3:12 [type=string] - └── lag [as=lag:9, type=int, outer=(11,13)] - ├── variable: lag_1_arg2:11 [type=int] - ├── variable: lag_1_arg2:11 [type=int] - └── variable: lag_2_arg3:13 [type=int] + ├── lag [as=lag:9, type=string, outer=(11-13)] + │ ├── variable: lag_1_arg1:11 [type=string] + │ ├── variable: lag_1_arg2:12 [type=int] + │ └── variable: lag_1_arg3:13 [type=string] + └── lag [as=lag:10, type=int, outer=(12,14)] + ├── variable: lag_1_arg2:12 [type=int] + ├── variable: lag_1_arg2:12 [type=int] + └── variable: lag_2_arg3:14 [type=int] diff --git a/pkg/sql/opt/memo/testdata/logprops/with b/pkg/sql/opt/memo/testdata/logprops/with index c4746cc03e51..40742f823d37 100644 --- a/pkg/sql/opt/memo/testdata/logprops/with +++ b/pkg/sql/opt/memo/testdata/logprops/with @@ -6,24 +6,30 @@ build WITH foo AS (SELECT * FROM xy) SELECT * FROM foo ---- with &1 (foo) - ├── columns: x:3(int!null) y:4(int) - ├── key: (3) - ├── fd: (3)-->(4) - ├── prune: (3,4) - ├── scan xy + ├── columns: x:4(int!null) y:5(int) + ├── key: (4) + ├── fd: (4)-->(5) + ├── prune: (4,5) + ├── project │ ├── columns: xy.x:1(int!null) xy.y:2(int) │ ├── key: (1) │ ├── fd: (1)-->(2) │ ├── prune: (1,2) - │ └── interesting orderings: (+1) + │ ├── interesting orderings: (+1) + │ └── scan xy + │ ├── columns: xy.x:1(int!null) xy.y:2(int) crdb_internal_mvcc_timestamp:3(decimal) + │ ├── key: (1) + │ ├── fd: (1)-->(2,3) + │ ├── prune: (1-3) + │ └── interesting orderings: (+1) └── with-scan &1 (foo) - ├── columns: x:3(int!null) y:4(int) + ├── columns: x:4(int!null) y:5(int) ├── mapping: - │ ├── xy.x:1(int) => x:3(int) - │ └── xy.y:2(int) => y:4(int) - ├── key: (3) - ├── fd: (3)-->(4) - ├── prune: (3,4) + │ ├── xy.x:1(int) => x:4(int) + │ └── xy.y:2(int) => y:5(int) + ├── key: (4) + ├── fd: (4)-->(5) + ├── prune: (4,5) └── cte-uses └── &1: count=1 used-columns=(1,2) @@ -191,17 +197,17 @@ FROM t40930, with_4189 ---- project - ├── columns: "?column?":7(unknown) - ├── fd: ()-->(7) - ├── prune: (7) + ├── columns: "?column?":9(unknown) + ├── fd: ()-->(9) + ├── prune: (9) ├── inner-join (cross) │ ├── scan t40930 - │ │ └── unfiltered-cols: (4,5) + │ │ └── unfiltered-cols: (5-7) │ ├── scan tab_10102 - │ │ └── unfiltered-cols: (1,2) + │ │ └── unfiltered-cols: (1-3) │ └── filters (true) └── projections - └── null [as="?column?":7, type=unknown] + └── null [as="?column?":9, type=unknown] build SELECT diff --git a/pkg/sql/opt/memo/testdata/memo b/pkg/sql/opt/memo/testdata/memo index 1eb62c1d5384..91eb3bac2b20 100644 --- a/pkg/sql/opt/memo/testdata/memo +++ b/pkg/sql/opt/memo/testdata/memo @@ -14,51 +14,51 @@ ORDER BY y LIMIT 10 ---- limit - ├── columns: y:2(int!null) x:3(string!null) c:5(int!null) + ├── columns: y:2(int!null) x:4(string!null) c:7(int!null) ├── internal-ordering: +2 ├── cardinality: [0 - 10] ├── immutable - ├── fd: (2)-->(5) + ├── fd: (2)-->(7) ├── ordering: +2 ├── sort - │ ├── columns: y:2(int!null) b.x:3(string!null) c:5(int!null) + │ ├── columns: y:2(int!null) b.x:4(string!null) c:7(int!null) │ ├── immutable - │ ├── fd: (2)-->(5) + │ ├── fd: (2)-->(7) │ ├── ordering: +2 │ ├── limit hint: 10.00 │ └── project - │ ├── columns: c:5(int!null) y:2(int!null) b.x:3(string!null) + │ ├── columns: c:7(int!null) y:2(int!null) b.x:4(string!null) │ ├── immutable - │ ├── fd: (2)-->(5) + │ ├── fd: (2)-->(7) │ ├── select - │ │ ├── columns: a.x:1(int!null) y:2(int!null) b.x:3(string!null) z:4(decimal!null) + │ │ ├── columns: a.x:1(int!null) y:2(int!null) a.crdb_internal_mvcc_timestamp:3(decimal) b.x:4(string!null) z:5(decimal!null) b.crdb_internal_mvcc_timestamp:6(decimal) │ │ ├── immutable - │ │ ├── key: (1,3) - │ │ ├── fd: (1)-->(2), (3)-->(4) + │ │ ├── key: (1,4) + │ │ ├── fd: (1)-->(2,3), (4)-->(5,6) │ │ ├── inner-join (cross) - │ │ │ ├── columns: a.x:1(int!null) y:2(int) b.x:3(string!null) z:4(decimal!null) - │ │ │ ├── key: (1,3) - │ │ │ ├── fd: (1)-->(2), (3)-->(4) + │ │ │ ├── columns: a.x:1(int!null) y:2(int) a.crdb_internal_mvcc_timestamp:3(decimal) b.x:4(string!null) z:5(decimal!null) b.crdb_internal_mvcc_timestamp:6(decimal) + │ │ │ ├── key: (1,4) + │ │ │ ├── fd: (1)-->(2,3), (4)-->(5,6) │ │ │ ├── scan a - │ │ │ │ ├── columns: a.x:1(int!null) y:2(int) + │ │ │ │ ├── columns: a.x:1(int!null) y:2(int) a.crdb_internal_mvcc_timestamp:3(decimal) │ │ │ │ ├── key: (1) - │ │ │ │ └── fd: (1)-->(2) + │ │ │ │ └── fd: (1)-->(2,3) │ │ │ ├── scan b - │ │ │ │ ├── columns: b.x:3(string!null) z:4(decimal!null) - │ │ │ │ ├── key: (3) - │ │ │ │ └── fd: (3)-->(4) + │ │ │ │ ├── columns: b.x:4(string!null) z:5(decimal!null) b.crdb_internal_mvcc_timestamp:6(decimal) + │ │ │ │ ├── key: (4) + │ │ │ │ └── fd: (4)-->(5,6) │ │ │ └── filters (true) │ │ └── filters - │ │ └── and [type=bool, outer=(1-3), immutable, constraints=(/2: [/2 - ])] + │ │ └── and [type=bool, outer=(1,2,4), immutable, constraints=(/2: [/2 - ])] │ │ ├── gt [type=bool] │ │ │ ├── variable: y:2 [type=int] │ │ │ └── const: 1 [type=int] │ │ └── eq [type=bool] │ │ ├── cast: STRING [type=string] │ │ │ └── variable: a.x:1 [type=int] - │ │ └── variable: b.x:3 [type=string] + │ │ └── variable: b.x:4 [type=string] │ └── projections - │ └── plus [as=c:5, type=int, outer=(2), immutable] + │ └── plus [as=c:7, type=int, outer=(2), immutable] │ ├── variable: y:2 [type=int] │ └── const: 1 [type=int] └── const: 10 [type=int] @@ -71,33 +71,33 @@ ORDER BY y LIMIT 10 ---- project - ├── columns: y:2(int!null) x:3(string!null) c:6(int!null) + ├── columns: y:2(int!null) x:4(string!null) c:8(int!null) ├── cardinality: [0 - 10] ├── immutable - ├── fd: (2)-->(6) + ├── fd: (2)-->(8) ├── ordering: +2 ├── limit - │ ├── columns: y:2(int!null) b.x:3(string!null) column5:5(string!null) + │ ├── columns: y:2(int!null) b.x:4(string!null) column7:7(string!null) │ ├── internal-ordering: +2 │ ├── cardinality: [0 - 10] │ ├── immutable - │ ├── fd: (3)==(5), (5)==(3) + │ ├── fd: (4)==(7), (7)==(4) │ ├── ordering: +2 │ ├── inner-join (lookup b) - │ │ ├── columns: y:2(int!null) b.x:3(string!null) column5:5(string!null) - │ │ ├── key columns: [5] = [3] + │ │ ├── columns: y:2(int!null) b.x:4(string!null) column7:7(string!null) + │ │ ├── key columns: [7] = [4] │ │ ├── lookup columns are key │ │ ├── immutable - │ │ ├── fd: (3)==(5), (5)==(3) + │ │ ├── fd: (4)==(7), (7)==(4) │ │ ├── ordering: +2 │ │ ├── limit hint: 10.00 │ │ ├── sort - │ │ │ ├── columns: y:2(int!null) column5:5(string!null) + │ │ │ ├── columns: y:2(int!null) column7:7(string!null) │ │ │ ├── immutable │ │ │ ├── ordering: +2 │ │ │ ├── limit hint: 100.00 │ │ │ └── project - │ │ │ ├── columns: column5:5(string!null) y:2(int!null) + │ │ │ ├── columns: column7:7(string!null) y:2(int!null) │ │ │ ├── immutable │ │ │ ├── select │ │ │ │ ├── columns: a.x:1(int!null) y:2(int!null) @@ -112,12 +112,12 @@ project │ │ │ │ ├── variable: y:2 [type=int] │ │ │ │ └── const: 1 [type=int] │ │ │ └── projections - │ │ │ └── cast: STRING [as=column5:5, type=string, outer=(1), immutable] + │ │ │ └── cast: STRING [as=column7:7, type=string, outer=(1), immutable] │ │ │ └── variable: a.x:1 [type=int] │ │ └── filters (true) │ └── const: 10 [type=int] └── projections - └── plus [as=c:6, type=int, outer=(2), immutable] + └── plus [as=c:8, type=int, outer=(2), immutable] ├── variable: y:2 [type=int] └── const: 1 [type=int] @@ -128,9 +128,9 @@ WHERE a.y>1 AND a.x::string=b.x ORDER BY y LIMIT 10 ---- -memo (optimized, ~18KB, required=[presentation: y:2,x:3,c:6] [ordering: +2]) +memo (optimized, ~20KB, required=[presentation: y:2,x:4,c:8] [ordering: +2]) ├── G1: (project G2 G3 y x) - │ ├── [presentation: y:2,x:3,c:6] [ordering: +2] + │ ├── [presentation: y:2,x:4,c:8] [ordering: +2] │ │ ├── best: (project G2="[ordering: +2]" G3 y x) │ │ └── cost: 1723.60 │ └── [] @@ -144,9 +144,9 @@ memo (optimized, ~18KB, required=[presentation: y:2,x:3,c:6] [ordering: +2]) │ ├── best: (limit G4="[ordering: +2] [limit hint: 10.00]" G5 ordering=+2) │ └── cost: 1723.39 ├── G3: (projections G6) - ├── G4: (inner-join G7 G8 G9) (inner-join G8 G7 G9) (lookup-join G7 G10 b,keyCols=[5],outCols=(2,3,5)) (merge-join G8 G7 G10 inner-join,+3,+5) + ├── G4: (inner-join G7 G8 G9) (inner-join G8 G7 G9) (lookup-join G7 G10 b,keyCols=[7],outCols=(2,4,7)) (merge-join G8 G7 G10 inner-join,+4,+7) │ ├── [ordering: +2] [limit hint: 10.00] - │ │ ├── best: (lookup-join G7="[ordering: +2] [limit hint: 100.00]" G10 b,keyCols=[5],outCols=(2,3,5)) + │ │ ├── best: (lookup-join G7="[ordering: +2] [limit hint: 100.00]" G10 b,keyCols=[7],outCols=(2,4,7)) │ │ └── cost: 1723.28 │ └── [] │ ├── best: (inner-join G8 G7 G9) @@ -157,18 +157,18 @@ memo (optimized, ~18KB, required=[presentation: y:2,x:3,c:6] [ordering: +2]) │ ├── [ordering: +2] [limit hint: 100.00] │ │ ├── best: (sort G7) │ │ └── cost: 1119.27 - │ ├── [ordering: +5] + │ ├── [ordering: +7] │ │ ├── best: (sort G7) │ │ └── cost: 1119.27 │ └── [] │ ├── best: (project G13 G14 y) │ └── cost: 1056.72 - ├── G8: (scan b,cols=(3)) - │ ├── [ordering: +3] - │ │ ├── best: (scan b,cols=(3)) + ├── G8: (scan b,cols=(4)) + │ ├── [ordering: +4] + │ │ ├── best: (scan b,cols=(4)) │ │ └── cost: 1030.02 │ └── [] - │ ├── best: (scan b,cols=(3)) + │ ├── best: (scan b,cols=(4)) │ └── cost: 1030.02 ├── G9: (filters G15) ├── G10: (filters) @@ -183,16 +183,16 @@ memo (optimized, ~18KB, required=[presentation: y:2,x:3,c:6] [ordering: +2]) │ └── cost: 1050.04 ├── G14: (projections G18) ├── G15: (eq G19 G20) - ├── G16: (scan a) + ├── G16: (scan a,cols=(1,2)) │ ├── [ordering: +2] [limit hint: 300.00] │ │ ├── best: (sort G16) │ │ └── cost: 1259.35 │ └── [] - │ ├── best: (scan a) + │ ├── best: (scan a,cols=(1,2)) │ └── cost: 1040.02 ├── G17: (filters G21) ├── G18: (cast G22 STRING) - ├── G19: (variable column5) + ├── G19: (variable column7) ├── G20: (variable b.x) ├── G21: (gt G11 G12) └── G22: (variable a.x) @@ -203,9 +203,9 @@ SELECT 1 AS a, 1+z AS b, left(x, 10)::TIMESTAMP AS c, left(x, 10)::TIMESTAMPTZ A FROM b WHERE z=1 AND concat(x, 'foo', x)=concat(x, 'foo', x) ---- -memo (optimized, ~4KB, required=[presentation: a:3,b:4,c:5,d:6]) +memo (optimized, ~6KB, required=[presentation: a:4,b:5,c:6,d:7]) ├── G1: (project G2 G3) - │ └── [presentation: a:3,b:4,c:5,d:6] + │ └── [presentation: a:4,b:5,c:6,d:7] │ ├── best: (project G2 G3) │ └── cost: 1050.23 ├── G2: (select G4 G5) @@ -213,9 +213,9 @@ memo (optimized, ~4KB, required=[presentation: a:3,b:4,c:5,d:6]) │ ├── best: (select G4 G5) │ └── cost: 1050.05 ├── G3: (projections G6 G7 G8 G9) - ├── G4: (scan b) + ├── G4: (scan b,cols=(1,2)) │ └── [] - │ ├── best: (scan b) + │ ├── best: (scan b,cols=(1,2)) │ └── cost: 1040.02 ├── G5: (filters G10 G11) ├── G6: (const 1) @@ -238,7 +238,7 @@ memo (optimized, ~4KB, required=[presentation: a:3,b:4,c:5,d:6]) memo SELECT x FROM a WHERE x = 1 AND x+y = 1 ---- -memo (optimized, ~5KB, required=[presentation: x:1]) +memo (optimized, ~7KB, required=[presentation: x:1]) ├── G1: (project G2 G3 x) │ └── [presentation: x:1] │ ├── best: (project G2 G3 x) @@ -248,14 +248,14 @@ memo (optimized, ~5KB, required=[presentation: x:1]) │ ├── best: (select G6 G7) │ └── cost: 1.08 ├── G3: (projections) - ├── G4: (scan a) + ├── G4: (scan a,cols=(1,2)) │ └── [] - │ ├── best: (scan a) + │ ├── best: (scan a,cols=(1,2)) │ └── cost: 1040.02 ├── G5: (filters G8 G9) - ├── G6: (scan a,constrained) + ├── G6: (scan a,cols=(1,2),constrained) │ └── [] - │ ├── best: (scan a,constrained) + │ ├── best: (scan a,cols=(1,2),constrained) │ └── cost: 1.05 ├── G7: (filters G9) ├── G8: (eq G10 G11) @@ -268,22 +268,22 @@ memo (optimized, ~5KB, required=[presentation: x:1]) memo SELECT x, y FROM a UNION SELECT x+1, y+1 FROM a ---- -memo (optimized, ~5KB, required=[presentation: x:7,y:8]) +memo (optimized, ~7KB, required=[presentation: x:9,y:10]) ├── G1: (union G2 G3) - │ └── [presentation: x:7,y:8] + │ └── [presentation: x:9,y:10] │ ├── best: (union G2 G3) │ └── cost: 2150.06 - ├── G2: (scan a) + ├── G2: (scan a,cols=(1,2)) │ └── [] - │ ├── best: (scan a) + │ ├── best: (scan a,cols=(1,2)) │ └── cost: 1040.02 ├── G3: (project G4 G5) │ └── [] │ ├── best: (project G4 G5) │ └── cost: 1070.03 - ├── G4: (scan a) + ├── G4: (scan a,cols=(4,5)) │ └── [] - │ ├── best: (scan a) + │ ├── best: (scan a,cols=(4,5)) │ └── cost: 1040.02 ├── G5: (projections G6 G7) ├── G6: (plus G8 G9) @@ -295,9 +295,9 @@ memo (optimized, ~5KB, required=[presentation: x:7,y:8]) memo SELECT array_agg(x) FROM (SELECT * FROM a) ---- -memo (optimized, ~3KB, required=[presentation: array_agg:3]) +memo (optimized, ~4KB, required=[presentation: array_agg:4]) ├── G1: (scalar-group-by G2 G3 cols=()) - │ └── [presentation: array_agg:3] + │ └── [presentation: array_agg:4] │ ├── best: (scalar-group-by G2 G3 cols=()) │ └── cost: 1040.05 ├── G2: (scan a,cols=(1)) @@ -311,9 +311,9 @@ memo (optimized, ~3KB, required=[presentation: array_agg:3]) memo SELECT array_agg(x) FROM (SELECT * FROM a) GROUP BY y ---- -memo (optimized, ~3KB, required=[presentation: array_agg:3]) +memo (optimized, ~4KB, required=[presentation: array_agg:4]) ├── G1: (project G2 G3 array_agg) - │ └── [presentation: array_agg:3] + │ └── [presentation: array_agg:4] │ ├── best: (project G2 G3 array_agg) │ └── cost: 1072.05 ├── G2: (group-by G4 G5 cols=(2)) @@ -321,9 +321,9 @@ memo (optimized, ~3KB, required=[presentation: array_agg:3]) │ ├── best: (group-by G4 G5 cols=(2)) │ └── cost: 1071.04 ├── G3: (projections) - ├── G4: (scan a) + ├── G4: (scan a,cols=(1,2)) │ └── [] - │ ├── best: (scan a) + │ ├── best: (scan a,cols=(1,2)) │ └── cost: 1040.02 ├── G5: (aggregations G6) ├── G6: (array-agg G7) @@ -332,17 +332,17 @@ memo (optimized, ~3KB, required=[presentation: array_agg:3]) memo SELECT array_agg(x) FROM (SELECT * FROM a ORDER BY y) ---- -memo (optimized, ~2KB, required=[presentation: array_agg:3]) +memo (optimized, ~3KB, required=[presentation: array_agg:4]) ├── G1: (scalar-group-by G2 G3 cols=(),ordering=+2) - │ └── [presentation: array_agg:3] + │ └── [presentation: array_agg:4] │ ├── best: (scalar-group-by G2="[ordering: +2]" G3 cols=(),ordering=+2) │ └── cost: 1269.38 - ├── G2: (scan a) + ├── G2: (scan a,cols=(1,2)) │ ├── [ordering: +2] │ │ ├── best: (sort G2) │ │ └── cost: 1259.35 │ └── [] - │ ├── best: (scan a) + │ ├── best: (scan a,cols=(1,2)) │ └── cost: 1040.02 ├── G3: (aggregations G4) ├── G4: (array-agg G5) diff --git a/pkg/sql/opt/memo/testdata/stats/delete b/pkg/sql/opt/memo/testdata/stats/delete index 39515acc24be..465fd17c61ca 100644 --- a/pkg/sql/opt/memo/testdata/stats/delete +++ b/pkg/sql/opt/memo/testdata/stats/delete @@ -39,46 +39,46 @@ FROM [DELETE FROM xyz WHERE z=5.5 RETURNING *] WHERE x > 'foo' ---- with &1 - ├── columns: x:7(string!null) y:8(int!null) z:9(float!null) + ├── columns: x:9(string!null) y:10(int!null) z:11(float!null) ├── volatile, mutations - ├── stats: [rows=3.33333333, distinct(7)=3.33333333, null(7)=0] - ├── key: (7) - ├── fd: ()-->(9), (7)-->(8) + ├── stats: [rows=3.33333333, distinct(9)=3.33333333, null(9)=0] + ├── key: (9) + ├── fd: ()-->(11), (9)-->(10) ├── delete xyz │ ├── columns: xyz.x:1(string!null) xyz.y:2(int!null) xyz.z:3(float!null) - │ ├── fetch columns: xyz.x:4(string) xyz.y:5(int) xyz.z:6(float) + │ ├── fetch columns: xyz.x:5(string) xyz.y:6(int) xyz.z:7(float) │ ├── volatile, mutations │ ├── stats: [rows=10] │ ├── key: (1) │ ├── fd: ()-->(3), (1)-->(2) │ └── select - │ ├── columns: xyz.x:4(string!null) xyz.y:5(int!null) xyz.z:6(float!null) - │ ├── stats: [rows=10, distinct(6)=1, null(6)=0] - │ ├── key: (4) - │ ├── fd: ()-->(6), (4)-->(5) + │ ├── columns: xyz.x:5(string!null) xyz.y:6(int!null) xyz.z:7(float!null) crdb_internal_mvcc_timestamp:8(decimal) + │ ├── stats: [rows=10, distinct(7)=1, null(7)=0] + │ ├── key: (5) + │ ├── fd: ()-->(7), (5)-->(6,8) │ ├── scan xyz - │ │ ├── columns: xyz.x:4(string!null) xyz.y:5(int!null) xyz.z:6(float) - │ │ ├── stats: [rows=1000, distinct(4)=1000, null(4)=0, distinct(5)=100, null(5)=0, distinct(6)=100, null(6)=10] - │ │ ├── key: (4) - │ │ └── fd: (4)-->(5,6) + │ │ ├── columns: xyz.x:5(string!null) xyz.y:6(int!null) xyz.z:7(float) crdb_internal_mvcc_timestamp:8(decimal) + │ │ ├── stats: [rows=1000, distinct(5)=1000, null(5)=0, distinct(6)=100, null(6)=0, distinct(7)=100, null(7)=10] + │ │ ├── key: (5) + │ │ └── fd: (5)-->(6-8) │ └── filters - │ └── xyz.z:6 = 5.5 [type=bool, outer=(6), constraints=(/6: [/5.5 - /5.5]; tight), fd=()-->(6)] + │ └── xyz.z:7 = 5.5 [type=bool, outer=(7), constraints=(/7: [/5.5 - /5.5]; tight), fd=()-->(7)] └── select - ├── columns: x:7(string!null) y:8(int!null) z:9(float!null) - ├── stats: [rows=3.33333333, distinct(7)=3.33333333, null(7)=0] - ├── key: (7) - ├── fd: ()-->(9), (7)-->(8) + ├── columns: x:9(string!null) y:10(int!null) z:11(float!null) + ├── stats: [rows=3.33333333, distinct(9)=3.33333333, null(9)=0] + ├── key: (9) + ├── fd: ()-->(11), (9)-->(10) ├── with-scan &1 - │ ├── columns: x:7(string!null) y:8(int!null) z:9(float!null) + │ ├── columns: x:9(string!null) y:10(int!null) z:11(float!null) │ ├── mapping: - │ │ ├── xyz.x:1(string) => x:7(string) - │ │ ├── xyz.y:2(int) => y:8(int) - │ │ └── xyz.z:3(float) => z:9(float) - │ ├── stats: [rows=10, distinct(7)=10, null(7)=0, distinct(8)=1, null(8)=0, distinct(9)=1, null(9)=0] - │ ├── key: (7) - │ └── fd: ()-->(9), (7)-->(8) + │ │ ├── xyz.x:1(string) => x:9(string) + │ │ ├── xyz.y:2(int) => y:10(int) + │ │ └── xyz.z:3(float) => z:11(float) + │ ├── stats: [rows=10, distinct(9)=10, null(9)=0, distinct(10)=1, null(10)=0, distinct(11)=1, null(11)=0] + │ ├── key: (9) + │ └── fd: ()-->(11), (9)-->(10) └── filters - └── x:7 > 'foo' [type=bool, outer=(7), constraints=(/7: [/e'foo\x00' - ]; tight)] + └── x:9 > 'foo' [type=bool, outer=(9), constraints=(/9: [/e'foo\x00' - ]; tight)] # Cardinality is zero. build @@ -86,22 +86,22 @@ DELETE FROM xyz WHERE False RETURNING * ---- delete xyz ├── columns: x:1(string!null) y:2(int!null) z:3(float) - ├── fetch columns: x:4(string) y:5(int) z:6(float) + ├── fetch columns: x:5(string) y:6(int) z:7(float) ├── cardinality: [0 - 0] ├── volatile, mutations ├── stats: [rows=0] ├── key: (1) ├── fd: (1)-->(2,3) └── select - ├── columns: x:4(string!null) y:5(int!null) z:6(float) + ├── columns: x:5(string!null) y:6(int!null) z:7(float) crdb_internal_mvcc_timestamp:8(decimal) ├── cardinality: [0 - 0] ├── stats: [rows=0] - ├── key: (4) - ├── fd: (4)-->(5,6) + ├── key: (5) + ├── fd: (5)-->(6-8) ├── scan xyz - │ ├── columns: x:4(string!null) y:5(int!null) z:6(float) + │ ├── columns: x:5(string!null) y:6(int!null) z:7(float) crdb_internal_mvcc_timestamp:8(decimal) │ ├── stats: [rows=1000] - │ ├── key: (4) - │ └── fd: (4)-->(5,6) + │ ├── key: (5) + │ └── fd: (5)-->(6-8) └── filters └── false [type=bool] diff --git a/pkg/sql/opt/memo/testdata/stats/groupby b/pkg/sql/opt/memo/testdata/stats/groupby index d65bcae3ec7d..08b6aae595e5 100644 --- a/pkg/sql/opt/memo/testdata/stats/groupby +++ b/pkg/sql/opt/memo/testdata/stats/groupby @@ -51,36 +51,36 @@ project ├── key: (1) ├── fd: (1)-->(2) └── scan a - ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) + ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) crdb_internal_mvcc_timestamp:5(decimal) ├── stats: [rows=2000, distinct(1,2)=2000, null(1,2)=0] ├── key: (1) - └── fd: (1)-->(2-4), (3,4)~~>(1,2) + └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) # Group by single column key. build SELECT max(y) FROM a GROUP BY x ---- project - ├── columns: max:5(int) + ├── columns: max:6(int) ├── stats: [rows=2000] └── group-by - ├── columns: x:1(int!null) max:5(int) + ├── columns: x:1(int!null) max:6(int) ├── grouping columns: x:1(int!null) ├── stats: [rows=2000, distinct(1)=2000, null(1)=0] ├── key: (1) - ├── fd: (1)-->(5) + ├── fd: (1)-->(6) ├── project │ ├── columns: x:1(int!null) y:2(int) │ ├── stats: [rows=2000, distinct(1)=2000, null(1)=0] │ ├── key: (1) │ ├── fd: (1)-->(2) │ └── scan a - │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) + │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) crdb_internal_mvcc_timestamp:5(decimal) │ ├── stats: [rows=2000, distinct(1)=2000, null(1)=0] │ ├── key: (1) - │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) + │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) └── aggregations - └── max [as=max:5, type=int, outer=(2)] + └── max [as=max:6, type=int, outer=(2)] └── y:2 [type=int] # Group by non-key. @@ -88,94 +88,104 @@ build SELECT y, sum(z) FROM a GROUP BY y ---- group-by - ├── columns: y:2(int) sum:5(float!null) + ├── columns: y:2(int) sum:6(float!null) ├── grouping columns: y:2(int) ├── stats: [rows=400, distinct(2)=400, null(2)=0] ├── key: (2) - ├── fd: (2)-->(5) + ├── fd: (2)-->(6) ├── project │ ├── columns: y:2(int) z:3(float!null) │ ├── stats: [rows=2000, distinct(2)=400, null(2)=0] │ └── scan a - │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) + │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) crdb_internal_mvcc_timestamp:5(decimal) │ ├── stats: [rows=2000, distinct(2)=400, null(2)=0] │ ├── key: (1) - │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) + │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) └── aggregations - └── sum [as=sum:5, type=float, outer=(3)] + └── sum [as=sum:6, type=float, outer=(3)] └── z:3 [type=float] build SELECT max(x) FROM a GROUP BY y, z, s ---- project - ├── columns: max:5(int!null) + ├── columns: max:6(int!null) ├── stats: [rows=600] └── group-by - ├── columns: y:2(int) z:3(float!null) s:4(string) max:5(int!null) + ├── columns: y:2(int) z:3(float!null) s:4(string) max:6(int!null) ├── grouping columns: y:2(int) z:3(float!null) s:4(string) ├── stats: [rows=600, distinct(2-4)=600, null(2-4)=0] ├── key: (2-4) - ├── fd: (3,4)~~>(2), (2-4)-->(5) - ├── scan a + ├── fd: (3,4)~~>(2), (2-4)-->(6) + ├── project │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) │ ├── stats: [rows=2000, distinct(2-4)=600, null(2-4)=0] │ ├── key: (1) - │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) + │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) + │ └── scan a + │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) crdb_internal_mvcc_timestamp:5(decimal) + │ ├── stats: [rows=2000, distinct(2-4)=600, null(2-4)=0] + │ ├── key: (1) + │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) └── aggregations - └── max [as=max:5, type=int, outer=(1)] + └── max [as=max:6, type=int, outer=(1)] └── x:1 [type=int] build SELECT min(x) FROM a GROUP BY y, z ---- project - ├── columns: min:5(int!null) + ├── columns: min:6(int!null) ├── stats: [rows=2000] └── group-by - ├── columns: y:2(int) z:3(float!null) min:5(int!null) + ├── columns: y:2(int) z:3(float!null) min:6(int!null) ├── grouping columns: y:2(int) z:3(float!null) ├── stats: [rows=2000, distinct(2,3)=2000, null(2,3)=0] ├── key: (2,3) - ├── fd: (2,3)-->(5) + ├── fd: (2,3)-->(6) ├── project │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) │ ├── stats: [rows=2000, distinct(2,3)=2000, null(2,3)=0] │ ├── key: (1) │ ├── fd: (1)-->(2,3) │ └── scan a - │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) + │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) crdb_internal_mvcc_timestamp:5(decimal) │ ├── stats: [rows=2000, distinct(2,3)=2000, null(2,3)=0] │ ├── key: (1) - │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) + │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) └── aggregations - └── min [as=min:5, type=int, outer=(1)] + └── min [as=min:6, type=int, outer=(1)] └── x:1 [type=int] build SELECT max(x) FROM a GROUP BY y, z, s HAVING s IN ('a', 'b') ---- project - ├── columns: max:5(int!null) + ├── columns: max:6(int!null) ├── stats: [rows=120] └── select - ├── columns: y:2(int) z:3(float!null) s:4(string!null) max:5(int!null) + ├── columns: y:2(int) z:3(float!null) s:4(string!null) max:6(int!null) ├── stats: [rows=120, distinct(4)=2, null(4)=0] ├── key: (3,4) - ├── fd: (3,4)-->(2), (2-4)-->(5) + ├── fd: (3,4)-->(2), (2-4)-->(6) ├── group-by - │ ├── columns: y:2(int) z:3(float!null) s:4(string) max:5(int!null) + │ ├── columns: y:2(int) z:3(float!null) s:4(string) max:6(int!null) │ ├── grouping columns: y:2(int) z:3(float!null) s:4(string) - │ ├── stats: [rows=600, distinct(3)=200, null(3)=0, distinct(4)=10, null(4)=0, distinct(5)=600, null(5)=0, distinct(2-4)=600, null(2-4)=0] + │ ├── stats: [rows=600, distinct(3)=200, null(3)=0, distinct(4)=10, null(4)=0, distinct(6)=600, null(6)=0, distinct(2-4)=600, null(2-4)=0] │ ├── key: (2-4) - │ ├── fd: (3,4)~~>(2), (2-4)-->(5) - │ ├── scan a + │ ├── fd: (3,4)~~>(2), (2-4)-->(6) + │ ├── project │ │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) │ │ ├── stats: [rows=2000, distinct(3)=200, null(3)=0, distinct(4)=10, null(4)=0, distinct(2-4)=600, null(2-4)=0] │ │ ├── key: (1) - │ │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) + │ │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) + │ │ └── scan a + │ │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) crdb_internal_mvcc_timestamp:5(decimal) + │ │ ├── stats: [rows=2000, distinct(3)=200, null(3)=0, distinct(4)=10, null(4)=0, distinct(2-4)=600, null(2-4)=0] + │ │ ├── key: (1) + │ │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) │ └── aggregations - │ └── max [as=max:5, type=int, outer=(1)] + │ └── max [as=max:6, type=int, outer=(1)] │ └── x:1 [type=int] └── filters └── s:4 IN ('a', 'b') [type=bool, outer=(4), constraints=(/4: [/'a' - /'a'] [/'b' - /'b']; tight)] @@ -185,64 +195,64 @@ build SELECT sum(x), s FROM a GROUP BY s HAVING sum(x) = 5 ---- select - ├── columns: sum:5(decimal!null) s:4(string) + ├── columns: sum:6(decimal!null) s:4(string) ├── immutable - ├── stats: [rows=1, distinct(5)=1, null(5)=0] + ├── stats: [rows=1, distinct(6)=1, null(6)=0] ├── key: (4) - ├── fd: ()-->(5) + ├── fd: ()-->(6) ├── group-by - │ ├── columns: s:4(string) sum:5(decimal!null) + │ ├── columns: s:4(string) sum:6(decimal!null) │ ├── grouping columns: s:4(string) - │ ├── stats: [rows=10, distinct(4)=10, null(4)=0, distinct(5)=10, null(5)=0] + │ ├── stats: [rows=10, distinct(4)=10, null(4)=0, distinct(6)=10, null(6)=0] │ ├── key: (4) - │ ├── fd: (4)-->(5) + │ ├── fd: (4)-->(6) │ ├── project │ │ ├── columns: x:1(int!null) s:4(string) │ │ ├── stats: [rows=2000, distinct(4)=10, null(4)=0] │ │ ├── key: (1) │ │ ├── fd: (1)-->(4) │ │ └── scan a - │ │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) + │ │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) crdb_internal_mvcc_timestamp:5(decimal) │ │ ├── stats: [rows=2000, distinct(4)=10, null(4)=0] │ │ ├── key: (1) - │ │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) + │ │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) │ └── aggregations - │ └── sum [as=sum:5, type=decimal, outer=(1)] + │ └── sum [as=sum:6, type=decimal, outer=(1)] │ └── x:1 [type=int] └── filters - └── sum:5 = 5 [type=bool, outer=(5), immutable, constraints=(/5: [/5 - /5]; tight), fd=()-->(5)] + └── sum:6 = 5 [type=bool, outer=(6), immutable, constraints=(/6: [/5 - /5]; tight), fd=()-->(6)] # Scalar GroupBy. build SELECT max(y), sum(z) FROM a HAVING sum(z) = 5.0 ---- select - ├── columns: max:5(int) sum:6(float!null) + ├── columns: max:6(int) sum:7(float!null) ├── cardinality: [0 - 1] - ├── stats: [rows=1, distinct(6)=1, null(6)=0] + ├── stats: [rows=1, distinct(7)=1, null(7)=0] ├── key: () - ├── fd: ()-->(5,6) + ├── fd: ()-->(6,7) ├── scalar-group-by - │ ├── columns: max:5(int) sum:6(float) + │ ├── columns: max:6(int) sum:7(float) │ ├── cardinality: [1 - 1] - │ ├── stats: [rows=1, distinct(6)=1, null(6)=0] + │ ├── stats: [rows=1, distinct(7)=1, null(7)=0] │ ├── key: () - │ ├── fd: ()-->(5,6) + │ ├── fd: ()-->(6,7) │ ├── project │ │ ├── columns: y:2(int) z:3(float!null) │ │ ├── stats: [rows=2000] │ │ └── scan a - │ │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) + │ │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) crdb_internal_mvcc_timestamp:5(decimal) │ │ ├── stats: [rows=2000] │ │ ├── key: (1) - │ │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) + │ │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) │ └── aggregations - │ ├── max [as=max:5, type=int, outer=(2)] + │ ├── max [as=max:6, type=int, outer=(2)] │ │ └── y:2 [type=int] - │ └── sum [as=sum:6, type=float, outer=(3)] + │ └── sum [as=sum:7, type=float, outer=(3)] │ └── z:3 [type=float] └── filters - └── sum:6 = 5.0 [type=bool, outer=(6), constraints=(/6: [/5.0 - /5.0]; tight), fd=()-->(6)] + └── sum:7 = 5.0 [type=bool, outer=(7), constraints=(/7: [/5.0 - /5.0]; tight), fd=()-->(7)] # Bump up null counts. exec-ddl @@ -297,36 +307,36 @@ project ├── key: (1) ├── fd: (1)-->(2) └── scan a - ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) + ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) crdb_internal_mvcc_timestamp:5(decimal) ├── stats: [rows=2000, distinct(1,2)=2000, null(1,2)=0] ├── key: (1) - └── fd: (1)-->(2-4), (3,4)~~>(1,2) + └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) # Group by single column key. build SELECT max(y) FROM a GROUP BY x ---- project - ├── columns: max:5(int) + ├── columns: max:6(int) ├── stats: [rows=2000] └── group-by - ├── columns: x:1(int!null) max:5(int) + ├── columns: x:1(int!null) max:6(int) ├── grouping columns: x:1(int!null) ├── stats: [rows=2000, distinct(1)=2000, null(1)=0] ├── key: (1) - ├── fd: (1)-->(5) + ├── fd: (1)-->(6) ├── project │ ├── columns: x:1(int!null) y:2(int) │ ├── stats: [rows=2000, distinct(1)=2000, null(1)=0] │ ├── key: (1) │ ├── fd: (1)-->(2) │ └── scan a - │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) + │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) crdb_internal_mvcc_timestamp:5(decimal) │ ├── stats: [rows=2000, distinct(1)=2000, null(1)=0] │ ├── key: (1) - │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) + │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) └── aggregations - └── max [as=max:5, type=int, outer=(2)] + └── max [as=max:6, type=int, outer=(2)] └── y:2 [type=int] # Group by non-key. @@ -334,94 +344,104 @@ build SELECT y, sum(z) FROM a GROUP BY y ---- group-by - ├── columns: y:2(int) sum:5(float!null) + ├── columns: y:2(int) sum:6(float!null) ├── grouping columns: y:2(int) ├── stats: [rows=400, distinct(2)=400, null(2)=1] ├── key: (2) - ├── fd: (2)-->(5) + ├── fd: (2)-->(6) ├── project │ ├── columns: y:2(int) z:3(float!null) │ ├── stats: [rows=2000, distinct(2)=400, null(2)=1000] │ └── scan a - │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) + │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) crdb_internal_mvcc_timestamp:5(decimal) │ ├── stats: [rows=2000, distinct(2)=400, null(2)=1000] │ ├── key: (1) - │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) + │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) └── aggregations - └── sum [as=sum:5, type=float, outer=(3)] + └── sum [as=sum:6, type=float, outer=(3)] └── z:3 [type=float] build SELECT max(x) FROM a GROUP BY y, z, s ---- project - ├── columns: max:5(int!null) + ├── columns: max:6(int!null) ├── stats: [rows=600] └── group-by - ├── columns: y:2(int) z:3(float!null) s:4(string) max:5(int!null) + ├── columns: y:2(int) z:3(float!null) s:4(string) max:6(int!null) ├── grouping columns: y:2(int) z:3(float!null) s:4(string) ├── stats: [rows=600, distinct(2-4)=600, null(2-4)=0] ├── key: (2-4) - ├── fd: (3,4)~~>(2), (2-4)-->(5) - ├── scan a + ├── fd: (3,4)~~>(2), (2-4)-->(6) + ├── project │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) │ ├── stats: [rows=2000, distinct(2-4)=600, null(2-4)=0] │ ├── key: (1) - │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) + │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) + │ └── scan a + │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) crdb_internal_mvcc_timestamp:5(decimal) + │ ├── stats: [rows=2000, distinct(2-4)=600, null(2-4)=0] + │ ├── key: (1) + │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) └── aggregations - └── max [as=max:5, type=int, outer=(1)] + └── max [as=max:6, type=int, outer=(1)] └── x:1 [type=int] build SELECT min(x) FROM a GROUP BY y, z ---- project - ├── columns: min:5(int!null) + ├── columns: min:6(int!null) ├── stats: [rows=2000] └── group-by - ├── columns: y:2(int) z:3(float!null) min:5(int!null) + ├── columns: y:2(int) z:3(float!null) min:6(int!null) ├── grouping columns: y:2(int) z:3(float!null) ├── stats: [rows=2000, distinct(2,3)=2000, null(2,3)=0] ├── key: (2,3) - ├── fd: (2,3)-->(5) + ├── fd: (2,3)-->(6) ├── project │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) │ ├── stats: [rows=2000, distinct(2,3)=2000, null(2,3)=0] │ ├── key: (1) │ ├── fd: (1)-->(2,3) │ └── scan a - │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) + │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) crdb_internal_mvcc_timestamp:5(decimal) │ ├── stats: [rows=2000, distinct(2,3)=2000, null(2,3)=0] │ ├── key: (1) - │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) + │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) └── aggregations - └── min [as=min:5, type=int, outer=(1)] + └── min [as=min:6, type=int, outer=(1)] └── x:1 [type=int] build SELECT max(x) FROM a GROUP BY y, z, s HAVING s IN ('a', 'b') ---- project - ├── columns: max:5(int!null) + ├── columns: max:6(int!null) ├── stats: [rows=133.111111] └── select - ├── columns: y:2(int) z:3(float!null) s:4(string!null) max:5(int!null) + ├── columns: y:2(int) z:3(float!null) s:4(string!null) max:6(int!null) ├── stats: [rows=133.111111, distinct(4)=2, null(4)=0] ├── key: (3,4) - ├── fd: (3,4)-->(2), (2-4)-->(5) + ├── fd: (3,4)-->(2), (2-4)-->(6) ├── group-by - │ ├── columns: y:2(int) z:3(float!null) s:4(string) max:5(int!null) + │ ├── columns: y:2(int) z:3(float!null) s:4(string) max:6(int!null) │ ├── grouping columns: y:2(int) z:3(float!null) s:4(string) - │ ├── stats: [rows=600, distinct(3)=200, null(3)=0, distinct(4)=10, null(4)=1, distinct(5)=600, null(5)=0, distinct(2-4)=600, null(2-4)=0] + │ ├── stats: [rows=600, distinct(3)=200, null(3)=0, distinct(4)=10, null(4)=1, distinct(6)=600, null(6)=0, distinct(2-4)=600, null(2-4)=0] │ ├── key: (2-4) - │ ├── fd: (3,4)~~>(2), (2-4)-->(5) - │ ├── scan a + │ ├── fd: (3,4)~~>(2), (2-4)-->(6) + │ ├── project │ │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) │ │ ├── stats: [rows=2000, distinct(3)=200, null(3)=0, distinct(4)=10, null(4)=1000, distinct(2-4)=600, null(2-4)=0] │ │ ├── key: (1) - │ │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) + │ │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) + │ │ └── scan a + │ │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) crdb_internal_mvcc_timestamp:5(decimal) + │ │ ├── stats: [rows=2000, distinct(3)=200, null(3)=0, distinct(4)=10, null(4)=1000, distinct(2-4)=600, null(2-4)=0] + │ │ ├── key: (1) + │ │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) │ └── aggregations - │ └── max [as=max:5, type=int, outer=(1)] + │ └── max [as=max:6, type=int, outer=(1)] │ └── x:1 [type=int] └── filters └── s:4 IN ('a', 'b') [type=bool, outer=(4), constraints=(/4: [/'a' - /'a'] [/'b' - /'b']; tight)] @@ -431,32 +451,32 @@ build SELECT sum(x), s FROM a GROUP BY s HAVING sum(x) = 5 ---- select - ├── columns: sum:5(decimal!null) s:4(string) + ├── columns: sum:6(decimal!null) s:4(string) ├── immutable - ├── stats: [rows=1, distinct(5)=1, null(5)=0] + ├── stats: [rows=1, distinct(6)=1, null(6)=0] ├── key: (4) - ├── fd: ()-->(5) + ├── fd: ()-->(6) ├── group-by - │ ├── columns: s:4(string) sum:5(decimal!null) + │ ├── columns: s:4(string) sum:6(decimal!null) │ ├── grouping columns: s:4(string) - │ ├── stats: [rows=10, distinct(4)=10, null(4)=1, distinct(5)=10, null(5)=0] + │ ├── stats: [rows=10, distinct(4)=10, null(4)=1, distinct(6)=10, null(6)=0] │ ├── key: (4) - │ ├── fd: (4)-->(5) + │ ├── fd: (4)-->(6) │ ├── project │ │ ├── columns: x:1(int!null) s:4(string) │ │ ├── stats: [rows=2000, distinct(4)=10, null(4)=1000] │ │ ├── key: (1) │ │ ├── fd: (1)-->(4) │ │ └── scan a - │ │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) + │ │ ├── columns: x:1(int!null) y:2(int) z:3(float!null) s:4(string) crdb_internal_mvcc_timestamp:5(decimal) │ │ ├── stats: [rows=2000, distinct(4)=10, null(4)=1000] │ │ ├── key: (1) - │ │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) + │ │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) │ └── aggregations - │ └── sum [as=sum:5, type=decimal, outer=(1)] + │ └── sum [as=sum:6, type=decimal, outer=(1)] │ └── x:1 [type=int] └── filters - └── sum:5 = 5 [type=bool, outer=(5), immutable, constraints=(/5: [/5 - /5]; tight), fd=()-->(5)] + └── sum:6 = 5 [type=bool, outer=(6), immutable, constraints=(/6: [/5 - /5]; tight), fd=()-->(6)] # Regression test for #36442. norm diff --git a/pkg/sql/opt/memo/testdata/stats/index-join b/pkg/sql/opt/memo/testdata/stats/index-join index 30baf15b0bfe..7130be22d782 100644 --- a/pkg/sql/opt/memo/testdata/stats/index-join +++ b/pkg/sql/opt/memo/testdata/stats/index-join @@ -38,16 +38,16 @@ opt SELECT count(*) FROM (SELECT * FROM a WHERE s = 'foo' AND x + y = 10) GROUP BY s, y ---- project - ├── columns: count:5(int!null) + ├── columns: count:6(int!null) ├── immutable ├── stats: [rows=49.2384513] └── group-by - ├── columns: y:2(int) count_rows:5(int!null) + ├── columns: y:2(int) count_rows:6(int!null) ├── grouping columns: y:2(int) ├── immutable ├── stats: [rows=49.2384513, distinct(2)=49.2384513, null(2)=0] ├── key: (2) - ├── fd: (2)-->(5) + ├── fd: (2)-->(6) ├── select │ ├── columns: x:1(int!null) y:2(int) s:3(string!null) │ ├── immutable @@ -68,7 +68,7 @@ project │ └── filters │ └── (x:1 + y:2) = 10 [type=bool, outer=(1,2), immutable] └── aggregations - └── count-rows [as=count_rows:5, type=int] + └── count-rows [as=count_rows:6, type=int] opt colstat=1 colstat=2 colstat=3 colstat=4 colstat=(1,2,3) SELECT * FROM a WHERE s = 'foo' AND x + y = 10 diff --git a/pkg/sql/opt/memo/testdata/stats/insert b/pkg/sql/opt/memo/testdata/stats/insert index bc8eef66e861..f44273f56a79 100644 --- a/pkg/sql/opt/memo/testdata/stats/insert +++ b/pkg/sql/opt/memo/testdata/stats/insert @@ -39,52 +39,52 @@ FROM [INSERT INTO xyz (x, y, z) SELECT b, a, c FROM abc WHERE b='foo' RETURNING WHERE z > 1.0 ---- with &1 - ├── columns: x:8(string!null) y:9(int!null) z:10(float!null) + ├── columns: x:10(string!null) y:11(int!null) z:12(float!null) ├── volatile, mutations - ├── stats: [rows=69.4736842, distinct(10)=6.66666667, null(10)=0] - ├── fd: ()-->(8) + ├── stats: [rows=69.4736842, distinct(12)=6.66666667, null(12)=0] + ├── fd: ()-->(10) ├── insert xyz │ ├── columns: xyz.x:1(string!null) xyz.y:2(int!null) xyz.z:3(float) │ ├── insert-mapping: - │ │ ├── b:5 => xyz.x:1 - │ │ ├── a:4 => xyz.y:2 - │ │ └── c:6 => xyz.z:3 + │ │ ├── b:6 => xyz.x:1 + │ │ ├── a:5 => xyz.y:2 + │ │ └── c:7 => xyz.z:3 │ ├── volatile, mutations │ ├── stats: [rows=200] │ ├── fd: ()-->(1) │ └── project - │ ├── columns: a:4(int!null) b:5(string!null) c:6(float) + │ ├── columns: a:5(int!null) b:6(string!null) c:7(float) │ ├── stats: [rows=200] - │ ├── fd: ()-->(5) + │ ├── fd: ()-->(6) │ └── select - │ ├── columns: a:4(int!null) b:5(string!null) c:6(float) rowid:7(int!null) - │ ├── stats: [rows=200, distinct(5)=1, null(5)=0] - │ ├── key: (7) - │ ├── fd: ()-->(5), (7)-->(4,6) + │ ├── columns: a:5(int!null) b:6(string!null) c:7(float) rowid:8(int!null) abc.crdb_internal_mvcc_timestamp:9(decimal) + │ ├── stats: [rows=200, distinct(6)=1, null(6)=0] + │ ├── key: (8) + │ ├── fd: ()-->(6), (8)-->(5,7,9) │ ├── scan abc - │ │ ├── columns: a:4(int!null) b:5(string) c:6(float) rowid:7(int!null) + │ │ ├── columns: a:5(int!null) b:6(string) c:7(float) rowid:8(int!null) abc.crdb_internal_mvcc_timestamp:9(decimal) │ │ ├── computed column expressions - │ │ │ └── c:6 - │ │ │ └── a:4::FLOAT8 [type=float] - │ │ ├── stats: [rows=2000, distinct(4)=2000, null(4)=0, distinct(5)=10, null(5)=0, distinct(7)=2000, null(7)=0] - │ │ ├── key: (7) - │ │ └── fd: (7)-->(4-6) + │ │ │ └── c:7 + │ │ │ └── a:5::FLOAT8 [type=float] + │ │ ├── stats: [rows=2000, distinct(5)=2000, null(5)=0, distinct(6)=10, null(6)=0, distinct(8)=2000, null(8)=0] + │ │ ├── key: (8) + │ │ └── fd: (8)-->(5-7,9) │ └── filters - │ └── b:5 = 'foo' [type=bool, outer=(5), constraints=(/5: [/'foo' - /'foo']; tight), fd=()-->(5)] + │ └── b:6 = 'foo' [type=bool, outer=(6), constraints=(/6: [/'foo' - /'foo']; tight), fd=()-->(6)] └── select - ├── columns: x:8(string!null) y:9(int!null) z:10(float!null) - ├── stats: [rows=69.4736842, distinct(10)=6.66666667, null(10)=0] - ├── fd: ()-->(8) + ├── columns: x:10(string!null) y:11(int!null) z:12(float!null) + ├── stats: [rows=69.4736842, distinct(12)=6.66666667, null(12)=0] + ├── fd: ()-->(10) ├── with-scan &1 - │ ├── columns: x:8(string!null) y:9(int!null) z:10(float) + │ ├── columns: x:10(string!null) y:11(int!null) z:12(float) │ ├── mapping: - │ │ ├── xyz.x:1(string) => x:8(string) - │ │ ├── xyz.y:2(int) => y:9(int) - │ │ └── xyz.z:3(float) => z:10(float) - │ ├── stats: [rows=200, distinct(8)=20, null(8)=0, distinct(9)=20, null(9)=0, distinct(10)=20, null(10)=2] - │ └── fd: ()-->(8) + │ │ ├── xyz.x:1(string) => x:10(string) + │ │ ├── xyz.y:2(int) => y:11(int) + │ │ └── xyz.z:3(float) => z:12(float) + │ ├── stats: [rows=200, distinct(10)=20, null(10)=0, distinct(11)=20, null(11)=0, distinct(12)=20, null(12)=2] + │ └── fd: ()-->(10) └── filters - └── z:10 > 1.0 [type=bool, outer=(10), constraints=(/10: [/1.0000000000000002 - ]; tight)] + └── z:12 > 1.0 [type=bool, outer=(12), constraints=(/12: [/1.0000000000000002 - ]; tight)] # Cardinality is zero. build @@ -93,29 +93,29 @@ INSERT INTO xyz (x, y, z) SELECT b, a, c FROM abc WHERE False RETURNING * insert xyz ├── columns: x:1(string!null) y:2(int!null) z:3(float) ├── insert-mapping: - │ ├── b:5 => x:1 - │ ├── a:4 => y:2 - │ └── c:6 => z:3 + │ ├── b:6 => x:1 + │ ├── a:5 => y:2 + │ └── c:7 => z:3 ├── cardinality: [0 - 0] ├── volatile, mutations ├── stats: [rows=0] └── project - ├── columns: a:4(int!null) b:5(string) c:6(float) + ├── columns: a:5(int!null) b:6(string) c:7(float) ├── cardinality: [0 - 0] ├── stats: [rows=0] └── select - ├── columns: a:4(int!null) b:5(string) c:6(float) rowid:7(int!null) + ├── columns: a:5(int!null) b:6(string) c:7(float) rowid:8(int!null) abc.crdb_internal_mvcc_timestamp:9(decimal) ├── cardinality: [0 - 0] ├── stats: [rows=0] - ├── key: (7) - ├── fd: (7)-->(4-6) + ├── key: (8) + ├── fd: (8)-->(5-7,9) ├── scan abc - │ ├── columns: a:4(int!null) b:5(string) c:6(float) rowid:7(int!null) + │ ├── columns: a:5(int!null) b:6(string) c:7(float) rowid:8(int!null) abc.crdb_internal_mvcc_timestamp:9(decimal) │ ├── computed column expressions - │ │ └── c:6 - │ │ └── a:4::FLOAT8 [type=float] + │ │ └── c:7 + │ │ └── a:5::FLOAT8 [type=float] │ ├── stats: [rows=2000] - │ ├── key: (7) - │ └── fd: (7)-->(4-6) + │ ├── key: (8) + │ └── fd: (8)-->(5-7,9) └── filters └── false [type=bool] diff --git a/pkg/sql/opt/memo/testdata/stats/inverted-join b/pkg/sql/opt/memo/testdata/stats/inverted-join index c661b1550197..62772764c66b 100644 --- a/pkg/sql/opt/memo/testdata/stats/inverted-join +++ b/pkg/sql/opt/memo/testdata/stats/inverted-join @@ -17,24 +17,24 @@ opt SELECT ltable.k, rtable.k FROM ltable JOIN rtable ON ST_Intersects(ltable.geom, rtable.geom) ---- project - ├── columns: k:1(int!null) k:3(int!null) + ├── columns: k:1(int!null) k:4(int!null) ├── immutable ├── stats: [rows=333333.333] - ├── key: (1,3) + ├── key: (1,4) └── inner-join (lookup rtable) - ├── columns: ltable.k:1(int!null) ltable.geom:2(geometry) rtable.k:3(int!null) rtable.geom:4(geometry) - ├── key columns: [3] = [3] + ├── columns: ltable.k:1(int!null) ltable.geom:2(geometry) rtable.k:4(int!null) rtable.geom:5(geometry) + ├── key columns: [4] = [4] ├── lookup columns are key ├── immutable ├── stats: [rows=333333.333] - ├── key: (1,3) - ├── fd: (1)-->(2), (3)-->(4) + ├── key: (1,4) + ├── fd: (1)-->(2), (4)-->(5) ├── inner-join (inverted-lookup rtable@geom_index) - │ ├── columns: ltable.k:1(int!null) ltable.geom:2(geometry) rtable.k:3(int!null) + │ ├── columns: ltable.k:1(int!null) ltable.geom:2(geometry) rtable.k:4(int!null) │ ├── inverted-expr - │ │ └── st_intersects(ltable.geom:2, rtable.geom:4) [type=bool] - │ ├── stats: [rows=10000, distinct(1)=999.956829, null(1)=0, distinct(3)=999.956829, null(3)=0] - │ ├── key: (1,3) + │ │ └── st_intersects(ltable.geom:2, rtable.geom:5) [type=bool] + │ ├── stats: [rows=10000, distinct(1)=999.956829, null(1)=0, distinct(4)=999.956829, null(4)=0] + │ ├── key: (1,4) │ ├── fd: (1)-->(2) │ ├── scan ltable │ │ ├── columns: ltable.k:1(int!null) ltable.geom:2(geometry) @@ -43,30 +43,30 @@ project │ │ └── fd: (1)-->(2) │ └── filters (true) └── filters - └── st_intersects(ltable.geom:2, rtable.geom:4) [type=bool, outer=(2,4), immutable] + └── st_intersects(ltable.geom:2, rtable.geom:5) [type=bool, outer=(2,5), immutable] opt SELECT ltable.k, rtable.k FROM ltable JOIN rtable@geom_index ON ST_Intersects(ltable.geom, rtable.geom) ---- project - ├── columns: k:1(int!null) k:3(int!null) + ├── columns: k:1(int!null) k:4(int!null) ├── immutable ├── stats: [rows=333333.333] - ├── key: (1,3) + ├── key: (1,4) └── inner-join (lookup rtable) - ├── columns: ltable.k:1(int!null) ltable.geom:2(geometry) rtable.k:3(int!null) rtable.geom:4(geometry) - ├── key columns: [3] = [3] + ├── columns: ltable.k:1(int!null) ltable.geom:2(geometry) rtable.k:4(int!null) rtable.geom:5(geometry) + ├── key columns: [4] = [4] ├── lookup columns are key ├── immutable ├── stats: [rows=333333.333] - ├── key: (1,3) - ├── fd: (1)-->(2), (3)-->(4) + ├── key: (1,4) + ├── fd: (1)-->(2), (4)-->(5) ├── inner-join (inverted-lookup rtable@geom_index) - │ ├── columns: ltable.k:1(int!null) ltable.geom:2(geometry) rtable.k:3(int!null) + │ ├── columns: ltable.k:1(int!null) ltable.geom:2(geometry) rtable.k:4(int!null) │ ├── inverted-expr - │ │ └── st_intersects(ltable.geom:2, rtable.geom:4) [type=bool] - │ ├── stats: [rows=10000, distinct(1)=999.956829, null(1)=0, distinct(3)=999.956829, null(3)=0] - │ ├── key: (1,3) + │ │ └── st_intersects(ltable.geom:2, rtable.geom:5) [type=bool] + │ ├── stats: [rows=10000, distinct(1)=999.956829, null(1)=0, distinct(4)=999.956829, null(4)=0] + │ ├── key: (1,4) │ ├── fd: (1)-->(2) │ ├── scan ltable │ │ ├── columns: ltable.k:1(int!null) ltable.geom:2(geometry) @@ -75,4 +75,4 @@ project │ │ └── fd: (1)-->(2) │ └── filters (true) └── filters - └── st_intersects(ltable.geom:2, rtable.geom:4) [type=bool, outer=(2,4), immutable] + └── st_intersects(ltable.geom:2, rtable.geom:5) [type=bool, outer=(2,5), immutable] diff --git a/pkg/sql/opt/memo/testdata/stats/join b/pkg/sql/opt/memo/testdata/stats/join index 685799d30c71..59f958aa084b 100644 --- a/pkg/sql/opt/memo/testdata/stats/join +++ b/pkg/sql/opt/memo/testdata/stats/join @@ -50,7 +50,7 @@ norm SELECT * FROM xysd JOIN uv ON true ---- inner-join (cross) - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:5(int) v:6(int!null) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:6(int) v:7(int!null) ├── stats: [rows=50000000] ├── fd: (1)-->(2-4), (3,4)~~>(1,2) ├── scan xysd @@ -59,16 +59,16 @@ inner-join (cross) │ ├── key: (1) │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) ├── scan uv - │ ├── columns: u:5(int) v:6(int!null) + │ ├── columns: u:6(int) v:7(int!null) │ └── stats: [rows=10000] └── filters (true) -norm colstat=1 colstat=2 colstat=3 colstat=4 colstat=5 colstat=6 colstat=(2,5,6) +norm colstat=1 colstat=2 colstat=3 colstat=4 colstat=6 colstat=7 colstat=(2,6,7) SELECT * FROM xysd JOIN uv ON true ---- inner-join (cross) - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:5(int) v:6(int!null) - ├── stats: [rows=50000000, distinct(1)=5000, null(1)=0, distinct(2)=400, null(2)=0, distinct(3)=500, null(3)=500000, distinct(4)=500, null(4)=0, distinct(5)=500, null(5)=0, distinct(6)=100, null(6)=0, distinct(2,5,6)=4000000, null(2,5,6)=0] + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:6(int) v:7(int!null) + ├── stats: [rows=50000000, distinct(1)=5000, null(1)=0, distinct(2)=400, null(2)=0, distinct(3)=500, null(3)=500000, distinct(4)=500, null(4)=0, distinct(6)=500, null(6)=0, distinct(7)=100, null(7)=0, distinct(2,6,7)=4000000, null(2,6,7)=0] ├── fd: (1)-->(2-4), (3,4)~~>(1,2) ├── scan xysd │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) @@ -76,181 +76,206 @@ inner-join (cross) │ ├── key: (1) │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) ├── scan uv - │ ├── columns: u:5(int) v:6(int!null) - │ └── stats: [rows=10000, distinct(5)=500, null(5)=0, distinct(6)=100, null(6)=0, distinct(5,6)=10000, null(5,6)=0] + │ ├── columns: u:6(int) v:7(int!null) + │ └── stats: [rows=10000, distinct(6)=500, null(6)=0, distinct(7)=100, null(7)=0, distinct(6,7)=10000, null(6,7)=0] └── filters (true) norm SELECT * FROM xysd JOIN uv ON false ---- values - ├── columns: x:1(int!null) y:2(int!null) s:3(string!null) d:4(decimal!null) u:5(int!null) v:6(int!null) + ├── columns: x:1(int!null) y:2(int!null) s:3(string!null) d:4(decimal!null) u:6(int!null) v:7(int!null) ├── cardinality: [0 - 0] ├── stats: [rows=0] ├── key: () - └── fd: ()-->(1-6) + └── fd: ()-->(1-4,6,7) build colstat=2 SELECT *, rowid FROM xysd INNER JOIN uv ON x=u ---- -inner-join (hash) - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:5(int!null) v:6(int!null) rowid:7(int!null) - ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) - ├── stats: [rows=10000, distinct(1)=500, null(1)=0, distinct(2)=400, null(2)=0, distinct(5)=500, null(5)=0] - ├── key: (7) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (7)-->(5,6), (1)==(5), (5)==(1) - ├── scan xysd - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) - │ ├── stats: [rows=5000, distinct(1)=5000, null(1)=0, distinct(2)=400, null(2)=0, distinct(4)=500, null(4)=0] - │ ├── key: (1) - │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) - ├── scan uv - │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) - │ ├── stats: [rows=10000, distinct(5)=500, null(5)=0, distinct(6)=100, null(6)=0, distinct(7)=10000, null(7)=0] - │ ├── key: (7) - │ └── fd: (7)-->(5,6) - └── filters - └── x:1 = u:5 [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] +project + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:6(int!null) v:7(int!null) rowid:8(int!null) + ├── stats: [rows=10000, distinct(2)=400, null(2)=0] + ├── key: (8) + ├── fd: (1)-->(2-4), (3,4)~~>(1,2,6), (8)-->(1-4,6,7), (1)==(6), (6)==(1) + └── inner-join (hash) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) u:6(int!null) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) + ├── stats: [rows=10000, distinct(1)=500, null(1)=0, distinct(2)=400, null(2)=0, distinct(6)=500, null(6)=0] + ├── key: (8) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (8)-->(6,7,9), (1)==(6), (6)==(1) + ├── scan xysd + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) + │ ├── stats: [rows=5000, distinct(1)=5000, null(1)=0, distinct(2)=400, null(2)=0, distinct(4)=500, null(4)=0] + │ ├── key: (1) + │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + ├── scan uv + │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + │ ├── stats: [rows=10000, distinct(6)=500, null(6)=0, distinct(7)=100, null(7)=0, distinct(8)=10000, null(8)=0] + │ ├── key: (8) + │ └── fd: (8)-->(6,7,9) + └── filters + └── x:1 = u:6 [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] build SELECT *, rowid FROM xysd LEFT JOIN uv ON x=u ---- -left-join (hash) - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:5(int) v:6(int) rowid:7(int) - ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) - ├── stats: [rows=10000, distinct(5)=500, null(5)=0] - ├── key: (1,7) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (7)-->(5,6) - ├── scan xysd - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) - │ ├── stats: [rows=5000, distinct(1)=5000, null(1)=0, distinct(4)=500, null(4)=0] - │ ├── key: (1) - │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) - ├── scan uv - │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) - │ ├── stats: [rows=10000, distinct(5)=500, null(5)=0] - │ ├── key: (7) - │ └── fd: (7)-->(5,6) - └── filters - └── x:1 = u:5 [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] +project + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:6(int) v:7(int) rowid:8(int) + ├── stats: [rows=10000] + ├── key: (1,8) + ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (8)-->(6,7) + └── left-join (hash) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) u:6(int) v:7(int) rowid:8(int) uv.crdb_internal_mvcc_timestamp:9(decimal) + ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) + ├── stats: [rows=10000, distinct(6)=500, null(6)=0] + ├── key: (1,8) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (8)-->(6,7,9) + ├── scan xysd + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) + │ ├── stats: [rows=5000, distinct(1)=5000, null(1)=0, distinct(4)=500, null(4)=0] + │ ├── key: (1) + │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + ├── scan uv + │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + │ ├── stats: [rows=10000, distinct(6)=500, null(6)=0] + │ ├── key: (8) + │ └── fd: (8)-->(6,7,9) + └── filters + └── x:1 = u:6 [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] build SELECT *, rowid FROM xysd RIGHT JOIN uv ON x=u ---- -right-join (hash) - ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) u:5(int) v:6(int!null) rowid:7(int!null) - ├── stats: [rows=10000, distinct(1)=500, null(1)=0] - ├── key: (7) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (7)-->(1-6) - ├── scan xysd - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) - │ ├── stats: [rows=5000, distinct(1)=5000, null(1)=0] - │ ├── key: (1) - │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) - ├── scan uv - │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) - │ ├── stats: [rows=10000, distinct(5)=500, null(5)=0, distinct(6)=100, null(6)=0, distinct(7)=10000, null(7)=0] - │ ├── key: (7) - │ └── fd: (7)-->(5,6) - └── filters - └── x:1 = u:5 [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] +project + ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) u:6(int) v:7(int!null) rowid:8(int!null) + ├── stats: [rows=10000] + ├── key: (8) + ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (8)-->(1-4,6,7) + └── right-join (hash) + ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) xysd.crdb_internal_mvcc_timestamp:5(decimal) u:6(int) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + ├── stats: [rows=10000, distinct(1)=500, null(1)=0] + ├── key: (8) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (8)-->(1-7,9) + ├── scan xysd + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) + │ ├── stats: [rows=5000, distinct(1)=5000, null(1)=0] + │ ├── key: (1) + │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + ├── scan uv + │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + │ ├── stats: [rows=10000, distinct(6)=500, null(6)=0, distinct(7)=100, null(7)=0, distinct(8)=10000, null(8)=0] + │ ├── key: (8) + │ └── fd: (8)-->(6,7,9) + └── filters + └── x:1 = u:6 [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] build SELECT *, rowid FROM xysd FULL JOIN uv ON x=u ---- -full-join (hash) - ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) u:5(int) v:6(int) rowid:7(int) - ├── multiplicity: left-rows(one-or-more), right-rows(exactly-one) +project + ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) u:6(int) v:7(int) rowid:8(int) ├── stats: [rows=10000] - ├── key: (1,7) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (7)-->(5,6) - ├── scan xysd - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) - │ ├── stats: [rows=5000, distinct(1)=5000, null(1)=0] - │ ├── key: (1) - │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) - ├── scan uv - │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) - │ ├── stats: [rows=10000, distinct(5)=500, null(5)=0] - │ ├── key: (7) - │ └── fd: (7)-->(5,6) - └── filters - └── x:1 = u:5 [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + ├── key: (1,8) + ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (8)-->(6,7) + └── full-join (hash) + ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) xysd.crdb_internal_mvcc_timestamp:5(decimal) u:6(int) v:7(int) rowid:8(int) uv.crdb_internal_mvcc_timestamp:9(decimal) + ├── multiplicity: left-rows(one-or-more), right-rows(exactly-one) + ├── stats: [rows=10000] + ├── key: (1,8) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (8)-->(6,7,9) + ├── scan xysd + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) + │ ├── stats: [rows=5000, distinct(1)=5000, null(1)=0] + │ ├── key: (1) + │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + ├── scan uv + │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + │ ├── stats: [rows=10000, distinct(6)=500, null(6)=0] + │ ├── key: (8) + │ └── fd: (8)-->(6,7,9) + └── filters + └── x:1 = u:6 [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] build SELECT * FROM xysd, uv ---- project - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:5(int) v:6(int!null) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:6(int) v:7(int!null) ├── stats: [rows=50000000] ├── fd: (1)-->(2-4), (3,4)~~>(1,2) └── inner-join (cross) - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:5(int) v:6(int!null) rowid:7(int!null) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) u:6(int) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) ├── stats: [rows=50000000] - ├── key: (1,7) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (7)-->(5,6) + ├── key: (1,8) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (8)-->(6,7,9) ├── scan xysd - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) │ ├── stats: [rows=5000] │ ├── key: (1) - │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) + │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) ├── scan uv - │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) + │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) │ ├── stats: [rows=10000] - │ ├── key: (7) - │ └── fd: (7)-->(5,6) + │ ├── key: (8) + │ └── fd: (8)-->(6,7,9) └── filters (true) build SELECT * FROM xysd, xysd AS xysd ---- -inner-join (cross) - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) x:5(int!null) y:6(int) s:7(string) d:8(decimal!null) +project + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) x:6(int!null) y:7(int) s:8(string) d:9(decimal!null) ├── stats: [rows=25000000] - ├── key: (1,5) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (5)-->(6-8), (7,8)~~>(5,6) - ├── scan xysd - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) - │ ├── stats: [rows=5000] - │ ├── key: (1) - │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) - ├── scan xysd - │ ├── columns: x:5(int!null) y:6(int) s:7(string) d:8(decimal!null) - │ ├── stats: [rows=5000] - │ ├── key: (5) - │ └── fd: (5)-->(6-8), (7,8)~~>(5,6) - └── filters (true) + ├── key: (1,6) + ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (6)-->(7-9), (8,9)~~>(6,7) + └── inner-join (cross) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) x:6(int!null) y:7(int) s:8(string) d:9(decimal!null) crdb_internal_mvcc_timestamp:10(decimal) + ├── stats: [rows=25000000] + ├── key: (1,6) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (6)-->(7-10), (8,9)~~>(6,7,10) + ├── scan xysd + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) + │ ├── stats: [rows=5000] + │ ├── key: (1) + │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + ├── scan xysd + │ ├── columns: x:6(int!null) y:7(int) s:8(string) d:9(decimal!null) crdb_internal_mvcc_timestamp:10(decimal) + │ ├── stats: [rows=5000] + │ ├── key: (6) + │ └── fd: (6)-->(7-10), (8,9)~~>(6,7,10) + └── filters (true) build SELECT * FROM xysd, uv WHERE v = 5 ---- project - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:5(int) v:6(int!null) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:6(int) v:7(int!null) ├── stats: [rows=500000] - ├── fd: ()-->(6), (1)-->(2-4), (3,4)~~>(1,2) + ├── fd: ()-->(7), (1)-->(2-4), (3,4)~~>(1,2) └── select - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:5(int) v:6(int!null) rowid:7(int!null) - ├── stats: [rows=500000, distinct(6)=1, null(6)=0] - ├── key: (1,7) - ├── fd: ()-->(6), (1)-->(2-4), (3,4)~~>(1,2), (7)-->(5) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) u:6(int) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + ├── stats: [rows=500000, distinct(7)=1, null(7)=0] + ├── key: (1,8) + ├── fd: ()-->(7), (1)-->(2-5), (3,4)~~>(1,2,5), (8)-->(6,9) ├── inner-join (cross) - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:5(int) v:6(int!null) rowid:7(int!null) - │ ├── stats: [rows=50000000, distinct(1)=5000, null(1)=0, distinct(4)=500, null(4)=0, distinct(6)=100, null(6)=0, distinct(7)=10000, null(7)=0] - │ ├── key: (1,7) - │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (7)-->(5,6) + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) u:6(int) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + │ ├── stats: [rows=50000000, distinct(1)=5000, null(1)=0, distinct(4)=500, null(4)=0, distinct(7)=100, null(7)=0, distinct(8)=10000, null(8)=0] + │ ├── key: (1,8) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (8)-->(6,7,9) │ ├── scan xysd - │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) + │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) │ │ ├── stats: [rows=5000, distinct(1)=5000, null(1)=0, distinct(4)=500, null(4)=0] │ │ ├── key: (1) - │ │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) + │ │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) │ ├── scan uv - │ │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) - │ │ ├── stats: [rows=10000, distinct(6)=100, null(6)=0, distinct(7)=10000, null(7)=0] - │ │ ├── key: (7) - │ │ └── fd: (7)-->(5,6) + │ │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + │ │ ├── stats: [rows=10000, distinct(7)=100, null(7)=0, distinct(8)=10000, null(8)=0] + │ │ ├── key: (8) + │ │ └── fd: (8)-->(6,7,9) │ └── filters (true) └── filters - └── v:6 = 5 [type=bool, outer=(6), constraints=(/6: [/5 - /5]; tight), fd=()-->(6)] + └── v:7 = 5 [type=bool, outer=(7), constraints=(/7: [/5 - /5]; tight), fd=()-->(7)] # Force calculation of the distinct count for the column set spanning both # tables in the join. @@ -258,61 +283,61 @@ build SELECT sum(v), x, v FROM xysd, uv GROUP BY x, v ---- group-by - ├── columns: sum:8(decimal!null) x:1(int!null) v:6(int!null) - ├── grouping columns: x:1(int!null) v:6(int!null) - ├── stats: [rows=500000, distinct(1,6)=500000, null(1,6)=0] - ├── key: (1,6) - ├── fd: (1,6)-->(8) + ├── columns: sum:10(decimal!null) x:1(int!null) v:7(int!null) + ├── grouping columns: x:1(int!null) v:7(int!null) + ├── stats: [rows=500000, distinct(1,7)=500000, null(1,7)=0] + ├── key: (1,7) + ├── fd: (1,7)-->(10) ├── project - │ ├── columns: x:1(int!null) v:6(int!null) - │ ├── stats: [rows=50000000, distinct(1,6)=500000, null(1,6)=0] + │ ├── columns: x:1(int!null) v:7(int!null) + │ ├── stats: [rows=50000000, distinct(1,7)=500000, null(1,7)=0] │ └── inner-join (cross) - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:5(int) v:6(int!null) rowid:7(int!null) - │ ├── stats: [rows=50000000, distinct(1,6)=500000, null(1,6)=0] - │ ├── key: (1,7) - │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (7)-->(5,6) + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) u:6(int) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + │ ├── stats: [rows=50000000, distinct(1,7)=500000, null(1,7)=0] + │ ├── key: (1,8) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (8)-->(6,7,9) │ ├── scan xysd - │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) + │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) │ │ ├── stats: [rows=5000, distinct(1)=5000, null(1)=0] │ │ ├── key: (1) - │ │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) + │ │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) │ ├── scan uv - │ │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) - │ │ ├── stats: [rows=10000, distinct(6)=100, null(6)=0] - │ │ ├── key: (7) - │ │ └── fd: (7)-->(5,6) + │ │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + │ │ ├── stats: [rows=10000, distinct(7)=100, null(7)=0] + │ │ ├── key: (8) + │ │ └── fd: (8)-->(6,7,9) │ └── filters (true) └── aggregations - └── sum [as=sum:8, type=decimal, outer=(6)] - └── v:6 [type=int] + └── sum [as=sum:10, type=decimal, outer=(7)] + └── v:7 [type=int] # Join selectivity: 1/max(distinct(x), distinct(u)) = 1/5000. norm SELECT sum(v), x, v FROM xysd, uv WHERE x=u GROUP BY x, v ---- group-by - ├── columns: sum:8(decimal!null) x:1(int!null) v:6(int!null) - ├── grouping columns: x:1(int!null) v:6(int!null) - ├── stats: [rows=10000, distinct(1,6)=10000, null(1,6)=0] - ├── key: (1,6) - ├── fd: (1,6)-->(8) + ├── columns: sum:10(decimal!null) x:1(int!null) v:7(int!null) + ├── grouping columns: x:1(int!null) v:7(int!null) + ├── stats: [rows=10000, distinct(1,7)=10000, null(1,7)=0] + ├── key: (1,7) + ├── fd: (1,7)-->(10) ├── inner-join (hash) - │ ├── columns: x:1(int!null) u:5(int!null) v:6(int!null) + │ ├── columns: x:1(int!null) u:6(int!null) v:7(int!null) │ ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) - │ ├── stats: [rows=10000, distinct(1)=500, null(1)=0, distinct(5)=500, null(5)=0, distinct(1,6)=10000, null(1,6)=0] - │ ├── fd: (1)==(5), (5)==(1) + │ ├── stats: [rows=10000, distinct(1)=500, null(1)=0, distinct(6)=500, null(6)=0, distinct(1,7)=10000, null(1,7)=0] + │ ├── fd: (1)==(6), (6)==(1) │ ├── scan xysd │ │ ├── columns: x:1(int!null) │ │ ├── stats: [rows=5000, distinct(1)=5000, null(1)=0] │ │ └── key: (1) │ ├── scan uv - │ │ ├── columns: u:5(int) v:6(int!null) - │ │ └── stats: [rows=10000, distinct(5)=500, null(5)=0, distinct(6)=100, null(6)=0] + │ │ ├── columns: u:6(int) v:7(int!null) + │ │ └── stats: [rows=10000, distinct(6)=500, null(6)=0, distinct(7)=100, null(7)=0] │ └── filters - │ └── x:1 = u:5 [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + │ └── x:1 = u:6 [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] └── aggregations - └── sum [as=sum:8, type=decimal, outer=(6)] - └── v:6 [type=int] + └── sum [as=sum:10, type=decimal, outer=(7)] + └── v:7 [type=int] # Semi-join. norm @@ -329,10 +354,10 @@ semi-join (hash) │ ├── key: (1) │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) ├── scan uv - │ ├── columns: u:5(int) - │ └── stats: [rows=10000, distinct(5)=500, null(5)=0] + │ ├── columns: u:6(int) + │ └── stats: [rows=10000, distinct(6)=500, null(6)=0] └── filters - └── x:1 = u:5 [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + └── x:1 = u:6 [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] # Anti-join. norm @@ -349,42 +374,42 @@ anti-join (hash) │ ├── key: (1) │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) ├── scan uv - │ ├── columns: u:5(int) - │ └── stats: [rows=10000, distinct(5)=500, null(5)=0] + │ ├── columns: u:6(int) + │ └── stats: [rows=10000, distinct(6)=500, null(6)=0] └── filters - └── x:1 = u:5 [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + └── x:1 = u:6 [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] # Multiple equality conditions. norm SELECT * FROM xysd JOIN uv ON x=u AND y=v ---- inner-join (hash) - ├── columns: x:1(int!null) y:2(int!null) s:3(string) d:4(decimal!null) u:5(int!null) v:6(int!null) + ├── columns: x:1(int!null) y:2(int!null) s:3(string) d:4(decimal!null) u:6(int!null) v:7(int!null) ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) - ├── stats: [rows=25, distinct(1)=25, null(1)=0, distinct(2)=25, null(2)=0, distinct(5)=25, null(5)=0, distinct(6)=25, null(6)=0] - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (1)==(5), (5)==(1), (2)==(6), (6)==(2) + ├── stats: [rows=25, distinct(1)=25, null(1)=0, distinct(2)=25, null(2)=0, distinct(6)=25, null(6)=0, distinct(7)=25, null(7)=0] + ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (1)==(6), (6)==(1), (2)==(7), (7)==(2) ├── scan xysd │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) │ ├── stats: [rows=5000, distinct(1)=5000, null(1)=0, distinct(2)=400, null(2)=0, distinct(4)=500, null(4)=0] │ ├── key: (1) │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) ├── scan uv - │ ├── columns: u:5(int) v:6(int!null) - │ └── stats: [rows=10000, distinct(5)=500, null(5)=0, distinct(6)=100, null(6)=0] + │ ├── columns: u:6(int) v:7(int!null) + │ └── stats: [rows=10000, distinct(6)=500, null(6)=0, distinct(7)=100, null(7)=0] └── filters - ├── x:1 = u:5 [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - └── y:2 = v:6 [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + ├── x:1 = u:6 [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── y:2 = v:7 [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] # Equality condition + extra filters. norm SELECT * FROM xysd JOIN uv ON x=u AND y+v=5 AND y > 0 AND y < 300 ---- inner-join (hash) - ├── columns: x:1(int!null) y:2(int!null) s:3(string) d:4(decimal!null) u:5(int!null) v:6(int!null) + ├── columns: x:1(int!null) y:2(int!null) s:3(string) d:4(decimal!null) u:6(int!null) v:7(int!null) ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) ├── immutable - ├── stats: [rows=3333.33333, distinct(1)=500, null(1)=0, distinct(5)=500, null(5)=0] - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (1)==(5), (5)==(1) + ├── stats: [rows=3333.33333, distinct(1)=500, null(1)=0, distinct(6)=500, null(6)=0] + ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (1)==(6), (6)==(1) ├── select │ ├── columns: x:1(int!null) y:2(int!null) s:3(string) d:4(decimal!null) │ ├── stats: [rows=3737.5, distinct(1)=3737.5, null(1)=0, distinct(2)=299, null(2)=0, distinct(4)=499.999473, null(4)=0] @@ -398,11 +423,11 @@ inner-join (hash) │ └── filters │ └── (y:2 > 0) AND (y:2 < 300) [type=bool, outer=(2), constraints=(/2: [/1 - /299]; tight)] ├── scan uv - │ ├── columns: u:5(int) v:6(int!null) - │ └── stats: [rows=10000, distinct(5)=500, null(5)=0, distinct(6)=100, null(6)=0] + │ ├── columns: u:6(int) v:7(int!null) + │ └── stats: [rows=10000, distinct(6)=500, null(6)=0, distinct(7)=100, null(7)=0] └── filters - ├── x:1 = u:5 [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - └── (y:2 + v:6) = 5 [type=bool, outer=(2,6), immutable] + ├── x:1 = u:6 [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── (y:2 + v:7) = 5 [type=bool, outer=(2,7), immutable] # Force column statistics calculation for semi-join. norm @@ -411,16 +436,16 @@ FROM (SELECT * FROM xysd WHERE EXISTS (SELECT * FROM uv WHERE x=u AND y+v=5)) AS GROUP BY y ---- project - ├── columns: count:8(int!null) + ├── columns: count:10(int!null) ├── immutable ├── stats: [rows=138.170075] └── group-by - ├── columns: y:2(int) count_rows:8(int!null) + ├── columns: y:2(int) count_rows:10(int!null) ├── grouping columns: y:2(int) ├── immutable ├── stats: [rows=138.170075, distinct(2)=138.170075, null(2)=0] ├── key: (2) - ├── fd: (2)-->(8) + ├── fd: (2)-->(10) ├── semi-join (hash) │ ├── columns: x:1(int!null) y:2(int) │ ├── immutable @@ -433,13 +458,13 @@ project │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ ├── scan uv - │ │ ├── columns: u:5(int) v:6(int!null) - │ │ └── stats: [rows=10000, distinct(5)=500, null(5)=0, distinct(6)=100, null(6)=0] + │ │ ├── columns: u:6(int) v:7(int!null) + │ │ └── stats: [rows=10000, distinct(6)=500, null(6)=0, distinct(7)=100, null(7)=0] │ └── filters - │ ├── x:1 = u:5 [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - │ └── (y:2 + v:6) = 5 [type=bool, outer=(2,6), immutable] + │ ├── x:1 = u:6 [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ └── (y:2 + v:7) = 5 [type=bool, outer=(2,7), immutable] └── aggregations - └── count-rows [as=count_rows:8, type=int] + └── count-rows [as=count_rows:10, type=int] # Force column statistics calculation for anti-join. norm @@ -448,16 +473,16 @@ FROM (SELECT * FROM xysd WHERE NOT EXISTS (SELECT * FROM uv WHERE x=u AND y+v=5) GROUP BY y ---- project - ├── columns: count:8(int!null) + ├── columns: count:10(int!null) ├── immutable ├── stats: [rows=400] └── group-by - ├── columns: y:2(int) count_rows:8(int!null) + ├── columns: y:2(int) count_rows:10(int!null) ├── grouping columns: y:2(int) ├── immutable ├── stats: [rows=400, distinct(2)=400, null(2)=0] ├── key: (2) - ├── fd: (2)-->(8) + ├── fd: (2)-->(10) ├── anti-join (hash) │ ├── columns: x:1(int!null) y:2(int) │ ├── immutable @@ -470,13 +495,13 @@ project │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ ├── scan uv - │ │ ├── columns: u:5(int) v:6(int!null) - │ │ └── stats: [rows=10000, distinct(5)=500, null(5)=0, distinct(6)=100, null(6)=0] + │ │ ├── columns: u:6(int) v:7(int!null) + │ │ └── stats: [rows=10000, distinct(6)=500, null(6)=0, distinct(7)=100, null(7)=0] │ └── filters - │ ├── x:1 = u:5 [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - │ └── (y:2 + v:6) = 5 [type=bool, outer=(2,6), immutable] + │ ├── x:1 = u:6 [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ └── (y:2 + v:7) = 5 [type=bool, outer=(2,7), immutable] └── aggregations - └── count-rows [as=count_rows:8, type=int] + └── count-rows [as=count_rows:10, type=int] # Force column statistics calculation for left join. norm @@ -485,21 +510,21 @@ FROM (SELECT * FROM xysd LEFT OUTER JOIN uv ON x=u AND y+v=5) AS a GROUP BY y ---- project - ├── columns: count:8(int!null) + ├── columns: count:10(int!null) ├── immutable ├── stats: [rows=400] └── group-by - ├── columns: y:2(int) count_rows:8(int!null) + ├── columns: y:2(int) count_rows:10(int!null) ├── grouping columns: y:2(int) ├── immutable ├── stats: [rows=400, distinct(2)=400, null(2)=0] ├── key: (2) - ├── fd: (2)-->(8) + ├── fd: (2)-->(10) ├── left-join (hash) - │ ├── columns: x:1(int!null) y:2(int) u:5(int) v:6(int) + │ ├── columns: x:1(int!null) y:2(int) u:6(int) v:7(int) │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) │ ├── immutable - │ ├── stats: [rows=5000, distinct(2)=400, null(2)=0, distinct(5)=500, null(5)=1666.66667] + │ ├── stats: [rows=5000, distinct(2)=400, null(2)=0, distinct(6)=500, null(6)=1666.66667] │ ├── fd: (1)-->(2) │ ├── scan xysd │ │ ├── columns: x:1(int!null) y:2(int) @@ -507,13 +532,13 @@ project │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ ├── scan uv - │ │ ├── columns: u:5(int) v:6(int!null) - │ │ └── stats: [rows=10000, distinct(5)=500, null(5)=0] + │ │ ├── columns: u:6(int) v:7(int!null) + │ │ └── stats: [rows=10000, distinct(6)=500, null(6)=0] │ └── filters - │ ├── x:1 = u:5 [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - │ └── (y:2 + v:6) = 5 [type=bool, outer=(2,6), immutable] + │ ├── x:1 = u:6 [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ └── (y:2 + v:7) = 5 [type=bool, outer=(2,7), immutable] └── aggregations - └── count-rows [as=count_rows:8, type=int] + └── count-rows [as=count_rows:10, type=int] # Force column statistics calculation for right join. norm @@ -522,35 +547,35 @@ FROM (SELECT * FROM xysd RIGHT OUTER JOIN uv ON x=u AND y+v=5) AS a GROUP BY y ---- project - ├── columns: count:8(int!null) + ├── columns: count:10(int!null) ├── immutable ├── stats: [rows=399.903879] └── group-by - ├── columns: y:2(int) count_rows:8(int!null) + ├── columns: y:2(int) count_rows:10(int!null) ├── grouping columns: y:2(int) ├── immutable ├── stats: [rows=399.903879, distinct(2)=399.903879, null(2)=1] ├── key: (2) - ├── fd: (2)-->(8) + ├── fd: (2)-->(10) ├── left-join (hash) - │ ├── columns: x:1(int) y:2(int) u:5(int) v:6(int!null) + │ ├── columns: x:1(int) y:2(int) u:6(int) v:7(int!null) │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) │ ├── immutable │ ├── stats: [rows=10000, distinct(1)=500, null(1)=6666.66667, distinct(2)=399.903879, null(2)=6666.66667] │ ├── fd: (1)-->(2) │ ├── scan uv - │ │ ├── columns: u:5(int) v:6(int!null) - │ │ └── stats: [rows=10000, distinct(5)=500, null(5)=0, distinct(6)=100, null(6)=0] + │ │ ├── columns: u:6(int) v:7(int!null) + │ │ └── stats: [rows=10000, distinct(6)=500, null(6)=0, distinct(7)=100, null(7)=0] │ ├── scan xysd │ │ ├── columns: x:1(int!null) y:2(int) │ │ ├── stats: [rows=5000, distinct(1)=5000, null(1)=0, distinct(2)=400, null(2)=0] │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ └── filters - │ ├── x:1 = u:5 [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - │ └── (y:2 + v:6) = 5 [type=bool, outer=(2,6), immutable] + │ ├── x:1 = u:6 [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ └── (y:2 + v:7) = 5 [type=bool, outer=(2,7), immutable] └── aggregations - └── count-rows [as=count_rows:8, type=int] + └── count-rows [as=count_rows:10, type=int] # Force column statistics calculation for outer join. norm @@ -559,18 +584,18 @@ FROM (SELECT * FROM xysd FULL OUTER JOIN uv ON x=u AND y+v=5) AS a GROUP BY y ---- project - ├── columns: count:8(int!null) + ├── columns: count:10(int!null) ├── immutable ├── stats: [rows=400] └── group-by - ├── columns: y:2(int) count_rows:8(int!null) + ├── columns: y:2(int) count_rows:10(int!null) ├── grouping columns: y:2(int) ├── immutable ├── stats: [rows=400, distinct(2)=400, null(2)=1] ├── key: (2) - ├── fd: (2)-->(8) + ├── fd: (2)-->(10) ├── full-join (hash) - │ ├── columns: x:1(int) y:2(int) u:5(int) v:6(int) + │ ├── columns: x:1(int) y:2(int) u:6(int) v:7(int) │ ├── multiplicity: left-rows(one-or-more), right-rows(exactly-one) │ ├── immutable │ ├── stats: [rows=11666.6667, distinct(2)=400, null(2)=6666.66667] @@ -581,13 +606,13 @@ project │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ ├── scan uv - │ │ ├── columns: u:5(int) v:6(int!null) - │ │ └── stats: [rows=10000, distinct(5)=500, null(5)=0] + │ │ ├── columns: u:6(int) v:7(int!null) + │ │ └── stats: [rows=10000, distinct(6)=500, null(6)=0] │ └── filters - │ ├── x:1 = u:5 [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - │ └── (y:2 + v:6) = 5 [type=bool, outer=(2,6), immutable] + │ ├── x:1 = u:6 [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ └── (y:2 + v:7) = 5 [type=bool, outer=(2,7), immutable] └── aggregations - └── count-rows [as=count_rows:8, type=int] + └── count-rows [as=count_rows:10, type=int] exec-ddl CREATE TABLE uvw (u INT, v INT, w INT) @@ -606,9 +631,9 @@ norm disable=(PushFilterIntoJoinLeftAndRight,PushFilterIntoJoinLeft,PushFilterIn SELECT * FROM (SELECT * FROM uvw WHERE w=1) JOIN (SELECT * FROM xyz WHERE x=10) ON u=x ---- inner-join (hash) - ├── columns: u:1(int!null) v:2(int) w:3(int!null) x:5(int!null) y:6(int) z:7(int) - ├── stats: [rows=10.3537072, distinct(1)=1, null(1)=0, distinct(5)=1, null(5)=0] - ├── fd: ()-->(1,3,5), (1)==(5), (5)==(1) + ├── columns: u:1(int!null) v:2(int) w:3(int!null) x:6(int!null) y:7(int) z:8(int) + ├── stats: [rows=10.3537072, distinct(1)=1, null(1)=0, distinct(6)=1, null(6)=0] + ├── fd: ()-->(1,3,6), (1)==(6), (6)==(1) ├── select │ ├── columns: u:1(int) v:2(int) w:3(int!null) │ ├── stats: [rows=10, distinct(1)=9.5617925, null(1)=0.1, distinct(3)=1, null(3)=0] @@ -619,24 +644,24 @@ inner-join (hash) │ └── filters │ └── w:3 = 1 [type=bool, outer=(3), constraints=(/3: [/1 - /1]; tight), fd=()-->(3)] ├── select - │ ├── columns: x:5(int!null) y:6(int) z:7(int) - │ ├── stats: [rows=10, distinct(5)=1, null(5)=0] - │ ├── fd: ()-->(5) + │ ├── columns: x:6(int!null) y:7(int) z:8(int) + │ ├── stats: [rows=10, distinct(6)=1, null(6)=0] + │ ├── fd: ()-->(6) │ ├── scan xyz - │ │ ├── columns: x:5(int) y:6(int) z:7(int) - │ │ └── stats: [rows=1000, distinct(5)=100, null(5)=10] + │ │ ├── columns: x:6(int) y:7(int) z:8(int) + │ │ └── stats: [rows=1000, distinct(6)=100, null(6)=10] │ └── filters - │ └── x:5 = 10 [type=bool, outer=(5), constraints=(/5: [/10 - /10]; tight), fd=()-->(5)] + │ └── x:6 = 10 [type=bool, outer=(6), constraints=(/6: [/10 - /10]; tight), fd=()-->(6)] └── filters - └── u:1 = x:5 [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + └── u:1 = x:6 [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] norm disable=(PushFilterIntoJoinLeftAndRight,PushFilterIntoJoinLeft,PushFilterIntoJoinRight,MapFilterIntoJoinLeft,MapFilterIntoJoinRight) SELECT * FROM (SELECT * FROM uvw WHERE w=1) JOIN xyz ON u=x AND x=10 ---- inner-join (hash) - ├── columns: u:1(int!null) v:2(int) w:3(int!null) x:5(int!null) y:6(int) z:7(int) - ├── stats: [rows=10.3537072, distinct(1)=1, null(1)=0, distinct(5)=1, null(5)=0] - ├── fd: ()-->(1,3,5), (1)==(5), (5)==(1) + ├── columns: u:1(int!null) v:2(int) w:3(int!null) x:6(int!null) y:7(int) z:8(int) + ├── stats: [rows=10.3537072, distinct(1)=1, null(1)=0, distinct(6)=1, null(6)=0] + ├── fd: ()-->(1,3,6), (1)==(6), (6)==(1) ├── select │ ├── columns: u:1(int) v:2(int) w:3(int!null) │ ├── stats: [rows=10, distinct(1)=9.5617925, null(1)=0.1, distinct(3)=1, null(3)=0] @@ -647,11 +672,11 @@ inner-join (hash) │ └── filters │ └── w:3 = 1 [type=bool, outer=(3), constraints=(/3: [/1 - /1]; tight), fd=()-->(3)] ├── scan xyz - │ ├── columns: x:5(int) y:6(int) z:7(int) - │ └── stats: [rows=1000, distinct(5)=100, null(5)=10] + │ ├── columns: x:6(int) y:7(int) z:8(int) + │ └── stats: [rows=1000, distinct(6)=100, null(6)=10] └── filters - ├── u:1 = x:5 [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - └── x:5 = 10 [type=bool, outer=(5), constraints=(/5: [/10 - /10]; tight), fd=()-->(5)] + ├── u:1 = x:6 [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── x:6 = 10 [type=bool, outer=(6), constraints=(/6: [/10 - /10]; tight), fd=()-->(6)] # Bump up null counts. exec-ddl @@ -696,168 +721,198 @@ ALTER TABLE uv INJECT STATISTICS '[ ]' ---- -build colstat=2 colstat=(1,2,7) colstat=(2,3) colstat=3 colstat=(3,5) colstat=5 +build colstat=2 colstat=(1,2,8) colstat=(2,3) colstat=3 colstat=(3,6) colstat=6 SELECT *, rowid FROM xysd INNER JOIN uv ON x=u ---- -inner-join (hash) - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:5(int!null) v:6(int!null) rowid:7(int!null) - ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) - ├── stats: [rows=5000, distinct(1)=499, null(1)=0, distinct(2)=399.99851, null(2)=2500, distinct(3)=499.977311, null(3)=50, distinct(5)=499, null(5)=0, distinct(2,3)=3160.69477, null(2,3)=25, distinct(3,5)=5000, null(3,5)=0, distinct(1,2,7)=5000, null(1,2,7)=0] - ├── key: (7) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (7)-->(5,6), (1)==(5), (5)==(1) - ├── scan xysd - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) - │ ├── stats: [rows=5000, distinct(1)=5000, null(1)=0, distinct(2)=400, null(2)=2500, distinct(3)=500, null(3)=50, distinct(4)=500, null(4)=0, distinct(1,2)=5000, null(1,2)=0, distinct(2,3)=5000, null(2,3)=25] - │ ├── key: (1) - │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) - ├── scan uv - │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) - │ ├── stats: [rows=10000, distinct(5)=500, null(5)=5000, distinct(6)=100, null(6)=0, distinct(7)=10000, null(7)=0] - │ ├── key: (7) - │ └── fd: (7)-->(5,6) - └── filters - └── x:1 = u:5 [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] +project + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:6(int!null) v:7(int!null) rowid:8(int!null) + ├── stats: [rows=5000, distinct(2)=399.99851, null(2)=2500, distinct(3)=499.977311, null(3)=50, distinct(6)=499, null(6)=0, distinct(2,3)=3160.69477, null(2,3)=25, distinct(3,6)=5000, null(3,6)=0, distinct(1,2,8)=5000, null(1,2,8)=0] + ├── key: (8) + ├── fd: (1)-->(2-4), (3,4)~~>(1,2,6), (8)-->(1-4,6,7), (1)==(6), (6)==(1) + └── inner-join (hash) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) u:6(int!null) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) + ├── stats: [rows=5000, distinct(1)=499, null(1)=0, distinct(2)=399.99851, null(2)=2500, distinct(3)=499.977311, null(3)=50, distinct(6)=499, null(6)=0, distinct(2,3)=3160.69477, null(2,3)=25, distinct(3,6)=5000, null(3,6)=0, distinct(1,2,8)=5000, null(1,2,8)=0] + ├── key: (8) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (8)-->(6,7,9), (1)==(6), (6)==(1) + ├── scan xysd + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) + │ ├── stats: [rows=5000, distinct(1)=5000, null(1)=0, distinct(2)=400, null(2)=2500, distinct(3)=500, null(3)=50, distinct(4)=500, null(4)=0, distinct(1,2)=5000, null(1,2)=0, distinct(2,3)=5000, null(2,3)=25] + │ ├── key: (1) + │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + ├── scan uv + │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + │ ├── stats: [rows=10000, distinct(6)=500, null(6)=5000, distinct(7)=100, null(7)=0, distinct(8)=10000, null(8)=0] + │ ├── key: (8) + │ └── fd: (8)-->(6,7,9) + └── filters + └── x:1 = u:6 [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] -build colstat=2 colstat=(1,2,7) colstat=(2,3) colstat=3 colstat=(3,5) colstat=5 +build colstat=2 colstat=(1,2,8) colstat=(2,3) colstat=3 colstat=(3,6) colstat=6 SELECT *, rowid FROM xysd LEFT JOIN uv ON x=u ---- -left-join (hash) - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:5(int) v:6(int) rowid:7(int) - ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) - ├── stats: [rows=10000, distinct(2)=400, null(2)=5000, distinct(3)=500, null(3)=100, distinct(5)=500, null(5)=0, distinct(2,3)=5000, null(2,3)=50, distinct(3,5)=10000, null(3,5)=50, distinct(1,2,7)=10000, null(1,2,7)=0] - ├── key: (1,7) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (7)-->(5,6) - ├── scan xysd - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) - │ ├── stats: [rows=5000, distinct(1)=5000, null(1)=0, distinct(2)=400, null(2)=2500, distinct(3)=500, null(3)=50, distinct(4)=500, null(4)=0, distinct(1,2)=5000, null(1,2)=0, distinct(2,3)=5000, null(2,3)=25] - │ ├── key: (1) - │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) - ├── scan uv - │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) - │ ├── stats: [rows=10000, distinct(5)=500, null(5)=5000, distinct(7)=10000, null(7)=0] - │ ├── key: (7) - │ └── fd: (7)-->(5,6) - └── filters - └── x:1 = u:5 [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] +project + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:6(int) v:7(int) rowid:8(int) + ├── stats: [rows=10000, distinct(2)=400, null(2)=5000, distinct(3)=500, null(3)=100, distinct(6)=500, null(6)=0, distinct(2,3)=5000, null(2,3)=50, distinct(3,6)=10000, null(3,6)=50, distinct(1,2,8)=10000, null(1,2,8)=0] + ├── key: (1,8) + ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (8)-->(6,7) + └── left-join (hash) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) u:6(int) v:7(int) rowid:8(int) uv.crdb_internal_mvcc_timestamp:9(decimal) + ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) + ├── stats: [rows=10000, distinct(2)=400, null(2)=5000, distinct(3)=500, null(3)=100, distinct(6)=500, null(6)=0, distinct(2,3)=5000, null(2,3)=50, distinct(3,6)=10000, null(3,6)=50, distinct(1,2,8)=10000, null(1,2,8)=0] + ├── key: (1,8) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (8)-->(6,7,9) + ├── scan xysd + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) + │ ├── stats: [rows=5000, distinct(1)=5000, null(1)=0, distinct(2)=400, null(2)=2500, distinct(3)=500, null(3)=50, distinct(4)=500, null(4)=0, distinct(1,2)=5000, null(1,2)=0, distinct(2,3)=5000, null(2,3)=25] + │ ├── key: (1) + │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + ├── scan uv + │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + │ ├── stats: [rows=10000, distinct(6)=500, null(6)=5000, distinct(8)=10000, null(8)=0] + │ ├── key: (8) + │ └── fd: (8)-->(6,7,9) + └── filters + └── x:1 = u:6 [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] -build colstat=2 colstat=(1,2,7) colstat=(2,3) colstat=3 colstat=(3,5) colstat=5 +build colstat=2 colstat=(1,2,8) colstat=(2,3) colstat=3 colstat=(3,6) colstat=6 SELECT *, rowid FROM xysd RIGHT JOIN uv ON x=u ---- -right-join (hash) - ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) u:5(int) v:6(int!null) rowid:7(int!null) - ├── stats: [rows=10000, distinct(1)=500, null(1)=0, distinct(2)=400, null(2)=5000, distinct(3)=499.999999, null(3)=100, distinct(5)=500, null(5)=5000, distinct(2,3)=4323.45892, null(2,3)=50, distinct(3,5)=10000, null(3,5)=50, distinct(1,2,7)=10000, null(1,2,7)=0] - ├── key: (7) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (7)-->(1-6) - ├── scan xysd - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) - │ ├── stats: [rows=5000, distinct(1)=5000, null(1)=0, distinct(2)=400, null(2)=2500, distinct(3)=500, null(3)=50, distinct(1,2)=5000, null(1,2)=0, distinct(2,3)=5000, null(2,3)=25] - │ ├── key: (1) - │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) - ├── scan uv - │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) - │ ├── stats: [rows=10000, distinct(5)=500, null(5)=5000, distinct(6)=100, null(6)=0, distinct(7)=10000, null(7)=0] - │ ├── key: (7) - │ └── fd: (7)-->(5,6) - └── filters - └── x:1 = u:5 [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] +project + ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) u:6(int) v:7(int!null) rowid:8(int!null) + ├── stats: [rows=10000, distinct(2)=400, null(2)=5000, distinct(3)=499.999999, null(3)=100, distinct(6)=500, null(6)=5000, distinct(2,3)=4323.45892, null(2,3)=50, distinct(3,6)=10000, null(3,6)=50, distinct(1,2,8)=10000, null(1,2,8)=0] + ├── key: (8) + ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (8)-->(1-4,6,7) + └── right-join (hash) + ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) xysd.crdb_internal_mvcc_timestamp:5(decimal) u:6(int) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + ├── stats: [rows=10000, distinct(1)=500, null(1)=0, distinct(2)=400, null(2)=5000, distinct(3)=499.999999, null(3)=100, distinct(6)=500, null(6)=5000, distinct(2,3)=4323.45892, null(2,3)=50, distinct(3,6)=10000, null(3,6)=50, distinct(1,2,8)=10000, null(1,2,8)=0] + ├── key: (8) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (8)-->(1-7,9) + ├── scan xysd + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) + │ ├── stats: [rows=5000, distinct(1)=5000, null(1)=0, distinct(2)=400, null(2)=2500, distinct(3)=500, null(3)=50, distinct(1,2)=5000, null(1,2)=0, distinct(2,3)=5000, null(2,3)=25] + │ ├── key: (1) + │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + ├── scan uv + │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + │ ├── stats: [rows=10000, distinct(6)=500, null(6)=5000, distinct(7)=100, null(7)=0, distinct(8)=10000, null(8)=0] + │ ├── key: (8) + │ └── fd: (8)-->(6,7,9) + └── filters + └── x:1 = u:6 [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] -build colstat=2 colstat=(1,2,7) colstat=(2,3) colstat=3 colstat=(3,5) colstat=5 +build colstat=2 colstat=(1,2,8) colstat=(2,3) colstat=3 colstat=(3,6) colstat=6 SELECT *, rowid FROM xysd FULL JOIN uv ON x=u ---- -full-join (hash) - ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) u:5(int) v:6(int) rowid:7(int) - ├── multiplicity: left-rows(one-or-more), right-rows(exactly-one) - ├── stats: [rows=10000, distinct(2)=400, null(2)=5000, distinct(3)=500, null(3)=100, distinct(5)=500, null(5)=5000, distinct(2,3)=5000, null(2,3)=50, distinct(3,5)=10000, null(3,5)=50, distinct(1,2,7)=10000, null(1,2,7)=0] - ├── key: (1,7) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (7)-->(5,6) - ├── scan xysd - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) - │ ├── stats: [rows=5000, distinct(1)=5000, null(1)=0, distinct(2)=400, null(2)=2500, distinct(3)=500, null(3)=50, distinct(1,2)=5000, null(1,2)=0, distinct(2,3)=5000, null(2,3)=25] - │ ├── key: (1) - │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) - ├── scan uv - │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) - │ ├── stats: [rows=10000, distinct(5)=500, null(5)=5000, distinct(7)=10000, null(7)=0] - │ ├── key: (7) - │ └── fd: (7)-->(5,6) - └── filters - └── x:1 = u:5 [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] +project + ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) u:6(int) v:7(int) rowid:8(int) + ├── stats: [rows=10000, distinct(2)=400, null(2)=5000, distinct(3)=500, null(3)=100, distinct(6)=500, null(6)=5000, distinct(2,3)=5000, null(2,3)=50, distinct(3,6)=10000, null(3,6)=50, distinct(1,2,8)=10000, null(1,2,8)=0] + ├── key: (1,8) + ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (8)-->(6,7) + └── full-join (hash) + ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) xysd.crdb_internal_mvcc_timestamp:5(decimal) u:6(int) v:7(int) rowid:8(int) uv.crdb_internal_mvcc_timestamp:9(decimal) + ├── multiplicity: left-rows(one-or-more), right-rows(exactly-one) + ├── stats: [rows=10000, distinct(2)=400, null(2)=5000, distinct(3)=500, null(3)=100, distinct(6)=500, null(6)=5000, distinct(2,3)=5000, null(2,3)=50, distinct(3,6)=10000, null(3,6)=50, distinct(1,2,8)=10000, null(1,2,8)=0] + ├── key: (1,8) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (8)-->(6,7,9) + ├── scan xysd + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) + │ ├── stats: [rows=5000, distinct(1)=5000, null(1)=0, distinct(2)=400, null(2)=2500, distinct(3)=500, null(3)=50, distinct(1,2)=5000, null(1,2)=0, distinct(2,3)=5000, null(2,3)=25] + │ ├── key: (1) + │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + ├── scan uv + │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + │ ├── stats: [rows=10000, distinct(6)=500, null(6)=5000, distinct(8)=10000, null(8)=0] + │ ├── key: (8) + │ └── fd: (8)-->(6,7,9) + └── filters + └── x:1 = u:6 [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] # Set one of the columns to non-nullable and see impact on multi-column null counts. -build colstat=2 colstat=(1,2,7) colstat=(2,3) colstat=3 colstat=(3,5) colstat=5 +build colstat=2 colstat=(1,2,8) colstat=(2,3) colstat=3 colstat=(3,6) colstat=6 SELECT *, rowid FROM xysd FULL JOIN uv ON x=u WHERE s IS NOT NULL ---- -select - ├── columns: x:1(int) y:2(int) s:3(string!null) d:4(decimal) u:5(int) v:6(int) rowid:7(int) - ├── stats: [rows=9900, distinct(2)=400, null(2)=4950, distinct(3)=500, null(3)=0, distinct(5)=500, null(5)=4950, distinct(2,3)=4999.5, null(2,3)=0, distinct(3,5)=9900, null(3,5)=0, distinct(1,2,7)=9900, null(1,2,7)=0] - ├── key: (1,7) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (7)-->(5,6) - ├── full-join (hash) - │ ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) u:5(int) v:6(int) rowid:7(int) - │ ├── multiplicity: left-rows(one-or-more), right-rows(exactly-one) - │ ├── stats: [rows=10000, distinct(2)=400, null(2)=5000, distinct(3)=500, null(3)=100, distinct(5)=500, null(5)=5000, distinct(2,3)=5000, null(2,3)=50, distinct(3,5)=10000, null(3,5)=50, distinct(1,2,7)=10000, null(1,2,7)=0] - │ ├── key: (1,7) - │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (7)-->(5,6) - │ ├── scan xysd - │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) - │ │ ├── stats: [rows=5000, distinct(1)=5000, null(1)=0, distinct(2)=400, null(2)=2500, distinct(3)=500, null(3)=50, distinct(1,2)=5000, null(1,2)=0, distinct(2,3)=5000, null(2,3)=25] - │ │ ├── key: (1) - │ │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ ├── scan uv - │ │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) - │ │ ├── stats: [rows=10000, distinct(5)=500, null(5)=5000, distinct(7)=10000, null(7)=0] - │ │ ├── key: (7) - │ │ └── fd: (7)-->(5,6) - │ └── filters - │ └── x:1 = u:5 [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - └── filters - └── s:3 IS NOT NULL [type=bool, outer=(3), constraints=(/3: (/NULL - ]; tight)] +project + ├── columns: x:1(int) y:2(int) s:3(string!null) d:4(decimal) u:6(int) v:7(int) rowid:8(int) + ├── stats: [rows=9900, distinct(2)=400, null(2)=4950, distinct(3)=500, null(3)=0, distinct(6)=500, null(6)=4950, distinct(2,3)=4999.5, null(2,3)=0, distinct(3,6)=9900, null(3,6)=0, distinct(1,2,8)=9900, null(1,2,8)=0] + ├── key: (1,8) + ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (8)-->(6,7) + └── select + ├── columns: x:1(int) y:2(int) s:3(string!null) d:4(decimal) xysd.crdb_internal_mvcc_timestamp:5(decimal) u:6(int) v:7(int) rowid:8(int) uv.crdb_internal_mvcc_timestamp:9(decimal) + ├── stats: [rows=9900, distinct(2)=400, null(2)=4950, distinct(3)=500, null(3)=0, distinct(6)=500, null(6)=4950, distinct(2,3)=4999.5, null(2,3)=0, distinct(3,6)=9900, null(3,6)=0, distinct(1,2,8)=9900, null(1,2,8)=0] + ├── key: (1,8) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (8)-->(6,7,9) + ├── full-join (hash) + │ ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) xysd.crdb_internal_mvcc_timestamp:5(decimal) u:6(int) v:7(int) rowid:8(int) uv.crdb_internal_mvcc_timestamp:9(decimal) + │ ├── multiplicity: left-rows(one-or-more), right-rows(exactly-one) + │ ├── stats: [rows=10000, distinct(2)=400, null(2)=5000, distinct(3)=500, null(3)=100, distinct(6)=500, null(6)=5000, distinct(2,3)=5000, null(2,3)=50, distinct(3,6)=10000, null(3,6)=50, distinct(1,2,8)=10000, null(1,2,8)=0] + │ ├── key: (1,8) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (8)-->(6,7,9) + │ ├── scan xysd + │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) + │ │ ├── stats: [rows=5000, distinct(1)=5000, null(1)=0, distinct(2)=400, null(2)=2500, distinct(3)=500, null(3)=50, distinct(1,2)=5000, null(1,2)=0, distinct(2,3)=5000, null(2,3)=25] + │ │ ├── key: (1) + │ │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ ├── scan uv + │ │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + │ │ ├── stats: [rows=10000, distinct(6)=500, null(6)=5000, distinct(8)=10000, null(8)=0] + │ │ ├── key: (8) + │ │ └── fd: (8)-->(6,7,9) + │ └── filters + │ └── x:1 = u:6 [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── filters + └── s:3 IS NOT NULL [type=bool, outer=(3), constraints=(/3: (/NULL - ]; tight)] # Do a full join on a condition that results in 0 rows on one side. All null counts # on the right side should be greater due to expected null-extension of columns. -build colstat=2 colstat=(1,2,7) colstat=(2,3) colstat=3 colstat=(3,5) colstat=5 +build colstat=2 colstat=(1,2,8) colstat=(2,3) colstat=3 colstat=(3,6) colstat=6 SELECT *, rowid FROM xysd FULL JOIN uv ON u > 4 AND u < 2 ---- -full-join (cross) - ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) u:5(int) v:6(int) rowid:7(int) - ├── stats: [rows=50000000, distinct(2)=400, null(2)=25000000, distinct(3)=500, null(3)=500000, distinct(5)=500, null(5)=25000000, distinct(2,3)=5000, null(2,3)=250000, distinct(3,5)=250000, null(3,5)=250000, distinct(1,2,7)=50000000, null(1,2,7)=0] - ├── key: (1,7) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (7)-->(5,6) - ├── scan xysd - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) - │ ├── stats: [rows=5000, distinct(2)=400, null(2)=2500, distinct(3)=500, null(3)=50, distinct(1,2)=5000, null(1,2)=0, distinct(2,3)=5000, null(2,3)=25] - │ ├── key: (1) - │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) - ├── scan uv - │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) - │ ├── stats: [rows=10000, distinct(5)=500, null(5)=5000, distinct(7)=10000, null(7)=0] - │ ├── key: (7) - │ └── fd: (7)-->(5,6) - └── filters - └── (u:5 > 4) AND (u:5 < 2) [type=bool, outer=(5), constraints=(contradiction; tight)] +project + ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) u:6(int) v:7(int) rowid:8(int) + ├── stats: [rows=50000000, distinct(2)=400, null(2)=25000000, distinct(3)=500, null(3)=500000, distinct(6)=500, null(6)=25000000, distinct(2,3)=5000, null(2,3)=250000, distinct(3,6)=250000, null(3,6)=250000, distinct(1,2,8)=50000000, null(1,2,8)=0] + ├── key: (1,8) + ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (8)-->(6,7) + └── full-join (cross) + ├── columns: x:1(int) y:2(int) s:3(string) d:4(decimal) xysd.crdb_internal_mvcc_timestamp:5(decimal) u:6(int) v:7(int) rowid:8(int) uv.crdb_internal_mvcc_timestamp:9(decimal) + ├── stats: [rows=50000000, distinct(2)=400, null(2)=25000000, distinct(3)=500, null(3)=500000, distinct(6)=500, null(6)=25000000, distinct(2,3)=5000, null(2,3)=250000, distinct(3,6)=250000, null(3,6)=250000, distinct(1,2,8)=50000000, null(1,2,8)=0] + ├── key: (1,8) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (8)-->(6,7,9) + ├── scan xysd + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) + │ ├── stats: [rows=5000, distinct(2)=400, null(2)=2500, distinct(3)=500, null(3)=50, distinct(1,2)=5000, null(1,2)=0, distinct(2,3)=5000, null(2,3)=25] + │ ├── key: (1) + │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + ├── scan uv + │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + │ ├── stats: [rows=10000, distinct(6)=500, null(6)=5000, distinct(8)=10000, null(8)=0] + │ ├── key: (8) + │ └── fd: (8)-->(6,7,9) + └── filters + └── (u:6 > 4) AND (u:6 < 2) [type=bool, outer=(6), constraints=(contradiction; tight)] -build colstat=2 colstat=(1,2,7) +build colstat=2 colstat=(1,2,8) SELECT * FROM xysd, uv ---- project - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:5(int) v:6(int!null) - ├── stats: [rows=50000000, distinct(2)=400, null(2)=25000000, distinct(1,2,7)=50000000, null(1,2,7)=0] + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:6(int) v:7(int!null) + ├── stats: [rows=50000000, distinct(2)=400, null(2)=25000000, distinct(1,2,8)=50000000, null(1,2,8)=0] ├── fd: (1)-->(2-4), (3,4)~~>(1,2) └── inner-join (cross) - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:5(int) v:6(int!null) rowid:7(int!null) - ├── stats: [rows=50000000, distinct(2)=400, null(2)=25000000, distinct(1,2,7)=50000000, null(1,2,7)=0] - ├── key: (1,7) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (7)-->(5,6) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) u:6(int) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + ├── stats: [rows=50000000, distinct(2)=400, null(2)=25000000, distinct(1,2,8)=50000000, null(1,2,8)=0] + ├── key: (1,8) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (8)-->(6,7,9) ├── scan xysd - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) xysd.crdb_internal_mvcc_timestamp:5(decimal) │ ├── stats: [rows=5000, distinct(2)=400, null(2)=2500, distinct(1,2)=5000, null(1,2)=0] │ ├── key: (1) - │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) + │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) ├── scan uv - │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) - │ ├── stats: [rows=10000, distinct(7)=10000, null(7)=0] - │ ├── key: (7) - │ └── fd: (7)-->(5,6) + │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) uv.crdb_internal_mvcc_timestamp:9(decimal) + │ ├── stats: [rows=10000, distinct(8)=10000, null(8)=0] + │ ├── key: (8) + │ └── fd: (8)-->(6,7,9) └── filters (true) norm @@ -874,10 +929,10 @@ semi-join (hash) │ ├── key: (1) │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) ├── scan uv - │ ├── columns: u:5(int) - │ └── stats: [rows=10000, distinct(5)=500, null(5)=5000] + │ ├── columns: u:6(int) + │ └── stats: [rows=10000, distinct(6)=500, null(6)=5000] └── filters - └── x:1 = u:5 [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + └── x:1 = u:6 [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] norm SELECT * FROM uv WHERE EXISTS(SELECT * FROM xysd WHERE x=u) @@ -889,14 +944,14 @@ semi-join (hash) │ ├── columns: u:1(int) v:2(int!null) │ └── stats: [rows=10000, distinct(1)=500, null(1)=5000, distinct(2)=100, null(2)=0] ├── scan xysd - │ ├── columns: x:4(int!null) - │ ├── stats: [rows=5000, distinct(4)=5000, null(4)=0] - │ └── key: (4) + │ ├── columns: x:5(int!null) + │ ├── stats: [rows=5000, distinct(5)=5000, null(5)=0] + │ └── key: (5) └── filters - └── x:4 = u:1 [type=bool, outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + └── x:5 = u:1 [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] # Merge join (inner). -expr colstat=2 colstat=(1,2,7) colstat=(2,3) colstat=3 colstat=(3,5) colstat=5 +expr colstat=2 colstat=(1,2,7) colstat=(2,3) colstat=3 colstat=(3,6) colstat=6 (MergeJoin (Scan [ (Table "xysd") (Cols "x,y,s,d") ]) (Sort (Scan [ (Table "uv") (Cols "u,v,rowid") ])) @@ -911,12 +966,12 @@ expr colstat=2 colstat=(1,2,7) colstat=(2,3) colstat=3 colstat=(3,5) colstat=5 ) ---- inner-join (merge) - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:5(int!null) v:6(int!null) rowid:7(int!null) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:6(int!null) v:7(int!null) rowid:8(int!null) ├── left ordering: +1 - ├── right ordering: +5 - ├── stats: [rows=5000, distinct(1)=499, null(1)=0, distinct(2)=400, null(2)=2500, distinct(3)=500, null(3)=50, distinct(5)=499, null(5)=0, distinct(2,3)=5000, null(2,3)=25, distinct(3,5)=5000, null(3,5)=0, distinct(1,2,7)=5000, null(1,2,7)=0] - ├── key: (7) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (7)-->(5,6), (1)==(5), (5)==(1) + ├── right ordering: +6 + ├── stats: [rows=5000, distinct(1)=499, null(1)=0, distinct(2)=400, null(2)=2500, distinct(3)=500, null(3)=50, distinct(6)=499, null(6)=0, distinct(2,3)=5000, null(2,3)=25, distinct(3,6)=5000, null(3,6)=0, distinct(1,2,7)=5000, null(1,2,7)=0] + ├── key: (8) + ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (8)-->(6,7), (1)==(6), (6)==(1) ├── scan xysd │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) │ ├── stats: [rows=5000, distinct(1)=5000, null(1)=0, distinct(2)=400, null(2)=2500, distinct(3)=500, null(3)=50, distinct(4)=500, null(4)=0, distinct(1,2)=5000, null(1,2)=0, distinct(2,3)=5000, null(2,3)=25] @@ -924,20 +979,20 @@ inner-join (merge) │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) │ └── ordering: +1 ├── sort - │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) - │ ├── stats: [rows=10000, distinct(5)=500, null(5)=5000, distinct(6)=100, null(6)=0, distinct(7)=10000, null(7)=0] - │ ├── key: (7) - │ ├── fd: (7)-->(5,6) - │ ├── ordering: +5 + │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) + │ ├── stats: [rows=10000, distinct(6)=500, null(6)=5000, distinct(7)=100, null(7)=0, distinct(8)=10000, null(8)=0] + │ ├── key: (8) + │ ├── fd: (8)-->(6,7) + │ ├── ordering: +6 │ └── scan uv - │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) - │ ├── stats: [rows=10000, distinct(5)=500, null(5)=5000, distinct(6)=100, null(6)=0, distinct(7)=10000, null(7)=0] - │ ├── key: (7) - │ └── fd: (7)-->(5,6) + │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) + │ ├── stats: [rows=10000, distinct(6)=500, null(6)=5000, distinct(7)=100, null(7)=0, distinct(8)=10000, null(8)=0] + │ ├── key: (8) + │ └── fd: (8)-->(6,7) └── filters (true) # Merge join (left) with extra ON condition. -expr colstat=2 colstat=(1,2,7) colstat=(2,3) colstat=3 colstat=(3,5) colstat=5 +expr colstat=2 colstat=(1,2,8) colstat=(2,3) colstat=3 colstat=(3,6) colstat=6 (MergeJoin (Scan [ (Table "xysd") (Cols "x,y,s,d") ]) (Sort (Scan [ (Table "uv") (Cols "u,v,rowid") ])) @@ -952,12 +1007,12 @@ expr colstat=2 colstat=(1,2,7) colstat=(2,3) colstat=3 colstat=(3,5) colstat=5 ) ---- left-join (merge) - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:5(int) v:6(int) rowid:7(int) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) u:6(int) v:7(int) rowid:8(int) ├── left ordering: +1 - ├── right ordering: +5 - ├── stats: [rows=5000, distinct(2)=400, null(2)=2500, distinct(3)=500, null(3)=50, distinct(5)=500, null(5)=1666.66667, distinct(6)=100, null(6)=1666.66667, distinct(2,3)=5000, null(2,3)=25, distinct(3,5)=5000, null(3,5)=25, distinct(1,2,7)=5000, null(1,2,7)=0] - ├── key: (1,7) - ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (7)-->(5,6) + ├── right ordering: +6 + ├── stats: [rows=5000, distinct(2)=400, null(2)=2500, distinct(3)=500, null(3)=50, distinct(6)=500, null(6)=1666.66667, distinct(7)=100, null(7)=1666.66667, distinct(2,3)=5000, null(2,3)=25, distinct(3,6)=5000, null(3,6)=25, distinct(1,2,8)=5000, null(1,2,8)=0] + ├── key: (1,8) + ├── fd: (1)-->(2-4), (3,4)~~>(1,2), (8)-->(6,7) ├── scan xysd │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) │ ├── stats: [rows=5000, distinct(1)=5000, null(1)=0, distinct(2)=400, null(2)=2500, distinct(3)=500, null(3)=50, distinct(4)=500, null(4)=0, distinct(1,2)=5000, null(1,2)=0, distinct(2,3)=5000, null(2,3)=25] @@ -965,18 +1020,18 @@ left-join (merge) │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) │ └── ordering: +1 ├── sort - │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) - │ ├── stats: [rows=10000, distinct(5)=500, null(5)=5000, distinct(6)=100, null(6)=0, distinct(7)=10000, null(7)=0] - │ ├── key: (7) - │ ├── fd: (7)-->(5,6) - │ ├── ordering: +5 + │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) + │ ├── stats: [rows=10000, distinct(6)=500, null(6)=5000, distinct(7)=100, null(7)=0, distinct(8)=10000, null(8)=0] + │ ├── key: (8) + │ ├── fd: (8)-->(6,7) + │ ├── ordering: +6 │ └── scan uv - │ ├── columns: u:5(int) v:6(int!null) rowid:7(int!null) - │ ├── stats: [rows=10000, distinct(5)=500, null(5)=5000, distinct(6)=100, null(6)=0, distinct(7)=10000, null(7)=0] - │ ├── key: (7) - │ └── fd: (7)-->(5,6) + │ ├── columns: u:6(int) v:7(int!null) rowid:8(int!null) + │ ├── stats: [rows=10000, distinct(6)=500, null(6)=5000, distinct(7)=100, null(7)=0, distinct(8)=10000, null(8)=0] + │ ├── key: (8) + │ └── fd: (8)-->(6,7) └── filters - └── y:2 > v:6 [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + └── y:2 > v:7 [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] # Check that true filters are handled correctly for all join types. norm @@ -1169,17 +1224,17 @@ SELECT ( FROM table0 AS t0 ---- project - ├── columns: "?column?":16(int) + ├── columns: "?column?":20(int) ├── stats: [rows=1000000] ├── ensure-distinct-on - │ ├── columns: t0.rowid:4(int!null) "?column?":15(int) + │ ├── columns: t0.rowid:4(int!null) "?column?":19(int) │ ├── grouping columns: t0.rowid:4(int!null) │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── stats: [rows=1000000] │ ├── key: (4) - │ ├── fd: (4)-->(15) + │ ├── fd: (4)-->(19) │ ├── left-join-apply - │ │ ├── columns: t0.col1:2(bool) t0.rowid:4(int!null) "?column?":15(int) + │ │ ├── columns: t0.col1:2(bool) t0.rowid:4(int!null) "?column?":19(int) │ │ ├── stats: [rows=1000000] │ │ ├── fd: (4)-->(2) │ │ ├── scan t0 @@ -1188,10 +1243,10 @@ project │ │ │ ├── key: (4) │ │ │ └── fd: (4)-->(2) │ │ ├── project - │ │ │ ├── columns: "?column?":15(int!null) + │ │ │ ├── columns: "?column?":19(int!null) │ │ │ ├── outer: (2) │ │ │ ├── stats: [rows=1000] - │ │ │ ├── fd: ()-->(15) + │ │ │ ├── fd: ()-->(19) │ │ │ ├── left-join (cross) │ │ │ │ ├── outer: (2) │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) @@ -1205,13 +1260,13 @@ project │ │ │ │ └── filters │ │ │ │ └── t0.col1:2 [type=bool, outer=(2), constraints=(/2: [/true - /true]; tight), fd=()-->(2)] │ │ │ └── projections - │ │ │ └── 1 [as="?column?":15, type=int] + │ │ │ └── 1 [as="?column?":19, type=int] │ │ └── filters (true) │ └── aggregations - │ └── const-agg [as="?column?":15, type=int, outer=(15)] - │ └── "?column?":15 [type=int] + │ └── const-agg [as="?column?":19, type=int, outer=(19)] + │ └── "?column?":19 [type=int] └── projections - └── "?column?":15 [as="?column?":16, type=int, outer=(15)] + └── "?column?":19 [as="?column?":20, type=int, outer=(19)] norm colstat=1 colstat=2 SELECT * FROM (SELECT 1) AS a(x) LEFT JOIN (SELECT 2) AS b(x) ON a.x = b.x @@ -1282,7 +1337,7 @@ ALTER TABLE def INJECT STATISTICS '[ # TODO(rytaft): The cardinality estimates here are unrealistically low. # Maybe revisit this? I doubt there's anything we can do there though. -expr format=show-all colstat=5 colstat=6 colstat=(5, 6) colstat=1 colstat=2 colstat=3 colstat=(1, 2, 3) +expr format=show-all colstat=6 colstat=7 colstat=(6, 7) colstat=1 colstat=2 colstat=3 colstat=(1, 2, 3) (MakeLookupJoin (Scan [ (Table "abc") (Cols "a,b,c") ]) [ (JoinType "inner-join") (Table "def") (Index "def@primary") (KeyCols "a,b") (Cols "a,b,c,d,f,e") ] @@ -1290,12 +1345,12 @@ expr format=show-all colstat=5 colstat=6 colstat=(5, 6) colstat=1 colstat=2 cols ) ---- inner-join (lookup def) - ├── columns: t.public.abc.a:1(int!null) t.public.abc.b:2(int!null) t.public.abc.c:3(int) t.public.def.d:4(int!null) t.public.def.e:5(int!null) t.public.def.f:6(int) - ├── key columns: [1 2] = [4 5] - ├── stats: [rows=0.01, distinct(1)=0.01, null(1)=0, distinct(2)=0.01, null(2)=0, distinct(3)=0.00999500175, null(3)=0.0001, distinct(4)=0.01, null(4)=0, distinct(5)=0.01, null(5)=0, distinct(6)=0.00999995009, null(6)=0.0001, distinct(5,6)=0.00999999509, null(5,6)=0, distinct(1-3)=0.0099995001, null(1-3)=0] + ├── columns: t.public.abc.a:1(int!null) t.public.abc.b:2(int!null) t.public.abc.c:3(int) t.public.def.d:5(int!null) t.public.def.e:6(int!null) t.public.def.f:7(int) + ├── key columns: [1 2] = [5 6] + ├── stats: [rows=0.01, distinct(1)=0.01, null(1)=0, distinct(2)=0.01, null(2)=0, distinct(3)=0.00999500175, null(3)=0.0001, distinct(5)=0.01, null(5)=0, distinct(6)=0.01, null(6)=0, distinct(7)=0.00999995009, null(7)=0.0001, distinct(6,7)=0.00999999509, null(6,7)=0, distinct(1-3)=0.0099995001, null(1-3)=0] ├── cost: 2106.0507 - ├── key: (4,5) - ├── fd: (1,2)-->(3), (4,5)-->(6), (1)==(4), (4)==(1), (2)==(5), (5)==(2) + ├── key: (5,6) + ├── fd: (1,2)-->(3), (5,6)-->(7), (1)==(5), (5)==(1), (2)==(6), (6)==(2) ├── interesting orderings: (+1,+2) ├── scan t.public.abc │ ├── columns: t.public.abc.a:1(int!null) t.public.abc.b:2(int!null) t.public.abc.c:3(int) @@ -1310,7 +1365,7 @@ inner-join (lookup def) # TODO(rytaft): The cardinality estimates for the semi-join are the same as the table. # The semi-join currently ignores the selectivities of the filters in the On condition. # We should fix this. -expr format=show-all colstat=5 colstat=6 colstat=(5, 6) colstat=1 colstat=2 colstat=3 colstat=(1, 2, 3) +expr format=show-all colstat=6 colstat=7 colstat=(6, 7) colstat=1 colstat=2 colstat=3 colstat=(1, 2, 3) (MakeLookupJoin (Scan [ (Table "abc") (Cols "a,b,c") ]) [ (JoinType "semi-join") (Table "def") (Index "def@primary") (KeyCols "a,b") (Cols "a,b,c,d,e,f") ] @@ -1319,8 +1374,8 @@ expr format=show-all colstat=5 colstat=6 colstat=(5, 6) colstat=1 colstat=2 cols ---- semi-join (lookup def) ├── columns: t.public.abc.a:1(int!null) t.public.abc.b:2(int!null) t.public.abc.c:3(int) - ├── key columns: [1 2] = [4 5] - ├── stats: [rows=100, distinct(1)=100, null(1)=0, distinct(2)=10, null(2)=0, distinct(3)=10, null(3)=1, distinct(5)=1, null(5)=0, distinct(6)=1, null(6)=0, distinct(5,6)=1, null(5,6)=0, distinct(1-3)=100, null(1-3)=0] + ├── key columns: [1 2] = [5 6] + ├── stats: [rows=100, distinct(1)=100, null(1)=0, distinct(2)=10, null(2)=0, distinct(3)=10, null(3)=1, distinct(6)=1, null(6)=0, distinct(7)=1, null(7)=0, distinct(6,7)=1, null(6,7)=0, distinct(1-3)=100, null(1-3)=0] ├── cost: 2106.0507 ├── key: (1,2) ├── fd: (1,2)-->(3) @@ -1335,7 +1390,7 @@ semi-join (lookup def) │ └── interesting orderings: (+1,+2) └── filters (true) -expr format=show-all colstat=5 colstat=6 colstat=(5, 6) colstat=1 colstat=2 colstat=3 colstat=(1, 2, 3) +expr format=show-all colstat=6 colstat=7 colstat=(6, 7) colstat=1 colstat=2 colstat=3 colstat=(1, 2, 3) (MakeLookupJoin (Scan [ (Table "abc") (Cols "a,b,c") ]) [ (JoinType "anti-join") (Table "def") (Index "def@primary") (KeyCols "a,b") (Cols "a,b,c,d,e,f") ] @@ -1344,8 +1399,8 @@ expr format=show-all colstat=5 colstat=6 colstat=(5, 6) colstat=1 colstat=2 cols ---- anti-join (lookup def) ├── columns: t.public.abc.a:1(int!null) t.public.abc.b:2(int!null) t.public.abc.c:3(int) - ├── key columns: [1 2] = [4 5] - ├── stats: [rows=1e-10, distinct(1)=1e-10, null(1)=0, distinct(2)=1e-10, null(2)=0, distinct(3)=1e-10, null(3)=1e-10, distinct(5)=1e-10, null(5)=0, distinct(6)=1e-10, null(6)=0, distinct(5,6)=1e-10, null(5,6)=0, distinct(1-3)=1e-10, null(1-3)=0] + ├── key columns: [1 2] = [5 6] + ├── stats: [rows=1e-10, distinct(1)=1e-10, null(1)=0, distinct(2)=1e-10, null(2)=0, distinct(3)=1e-10, null(3)=1e-10, distinct(6)=1e-10, null(6)=0, distinct(7)=1e-10, null(7)=0, distinct(6,7)=1e-10, null(6,7)=0, distinct(1-3)=1e-10, null(1-3)=0] ├── cost: 2106.0507 ├── key: (1,2) ├── fd: (1,2)-->(3) @@ -1360,7 +1415,7 @@ anti-join (lookup def) │ └── interesting orderings: (+1,+2) └── filters (true) -expr format=show-all colstat=5 colstat=6 colstat=(5, 6) colstat=1 colstat=2 colstat=3 colstat=(1, 2, 3) +expr format=show-all colstat=6 colstat=7 colstat=(6, 7) colstat=1 colstat=2 colstat=3 colstat=(1, 2, 3) (MakeLookupJoin (Scan [ (Table "abc") (Cols "a,b,c") ]) [ (JoinType "semi-join") (Table "def") (Index "def@primary") (KeyCols "a,b") (Cols "a,b,c,d,e,f") ] @@ -1369,8 +1424,8 @@ expr format=show-all colstat=5 colstat=6 colstat=(5, 6) colstat=1 colstat=2 cols ---- semi-join (lookup def) ├── columns: t.public.abc.a:1(int!null) t.public.abc.b:2(int!null) t.public.abc.c:3(int) - ├── key columns: [1 2] = [4 5] - ├── stats: [rows=0, distinct(1)=0, null(1)=0, distinct(2)=0, null(2)=0, distinct(3)=0, null(3)=0, distinct(5)=0, null(5)=0, distinct(6)=0, null(6)=0, distinct(5,6)=0, null(5,6)=0, distinct(1-3)=0, null(1-3)=0] + ├── key columns: [1 2] = [5 6] + ├── stats: [rows=0, distinct(1)=0, null(1)=0, distinct(2)=0, null(2)=0, distinct(3)=0, null(3)=0, distinct(6)=0, null(6)=0, distinct(7)=0, null(7)=0, distinct(6,7)=0, null(6,7)=0, distinct(1-3)=0, null(1-3)=0] ├── cost: 2106.0607 ├── key: (1,2) ├── fd: (1,2)-->(3) @@ -1386,7 +1441,7 @@ semi-join (lookup def) └── filters └── false [type=bool] -expr format=show-all colstat=5 colstat=6 colstat=(5, 6) colstat=1 colstat=2 colstat=3 colstat=(1, 2, 3) +expr format=show-all colstat=6 colstat=7 colstat=(6, 7) colstat=1 colstat=2 colstat=3 colstat=(1, 2, 3) (MakeLookupJoin (Scan [ (Table "abc") (Cols "a,b,c") ]) [ (JoinType "anti-join") (Table "def") (Index "def@primary") (KeyCols "a,b") (Cols "a,b,c,d,e,f") ] @@ -1395,8 +1450,8 @@ expr format=show-all colstat=5 colstat=6 colstat=(5, 6) colstat=1 colstat=2 cols ---- anti-join (lookup def) ├── columns: t.public.abc.a:1(int!null) t.public.abc.b:2(int!null) t.public.abc.c:3(int) - ├── key columns: [1 2] = [4 5] - ├── stats: [rows=100, distinct(1)=100, null(1)=0, distinct(2)=10, null(2)=0, distinct(3)=10, null(3)=1, distinct(5)=1, null(5)=0, distinct(6)=1, null(6)=0, distinct(5,6)=1, null(5,6)=0, distinct(1-3)=100, null(1-3)=0] + ├── key columns: [1 2] = [5 6] + ├── stats: [rows=100, distinct(1)=100, null(1)=0, distinct(2)=10, null(2)=0, distinct(3)=10, null(3)=1, distinct(6)=1, null(6)=0, distinct(7)=1, null(7)=0, distinct(6,7)=1, null(6,7)=0, distinct(1-3)=100, null(1-3)=0] ├── cost: 2106.0607 ├── key: (1,2) ├── fd: (1,2)-->(3) @@ -1427,23 +1482,23 @@ FROM ); ---- full-join (cross) - ├── columns: a:1(int) b:2(int) c:3(int) a:4(int) b:5(int) c:6(int) + ├── columns: a:1(int) b:2(int) c:3(int) a:5(int) b:6(int) c:7(int) ├── multiplicity: left-rows(exactly-one), right-rows(one-or-more) ├── immutable ├── stats: [rows=100] ├── key: (1,2) - ├── fd: (1,2)-->(3-6) + ├── fd: (1,2)-->(3,5-7) ├── scan abc │ ├── columns: a:1(int!null) b:2(int!null) c:3(int) │ ├── stats: [rows=100] │ ├── key: (1,2) │ └── fd: (1,2)-->(3) ├── values - │ ├── columns: a:4(int!null) b:5(int!null) c:6(int!null) + │ ├── columns: a:5(int!null) b:6(int!null) c:7(int!null) │ ├── cardinality: [0 - 0] │ ├── stats: [rows=0] │ ├── key: () - │ └── fd: ()-->(4-6) + │ └── fd: ()-->(5-7) └── filters └── is [type=bool, immutable, subquery] ├── function: not_like_escape [type=bool] @@ -1452,11 +1507,11 @@ full-join (cross) │ └── cast: STRING [type=string] │ └── subquery [type=unknown] │ └── values - │ ├── columns: "?column?":7(unknown) + │ ├── columns: "?column?":9(unknown) │ ├── cardinality: [1 - 1] │ ├── stats: [rows=1] │ ├── key: () - │ ├── fd: ()-->(7) + │ ├── fd: ()-->(9) │ └── (NULL,) [type=tuple{unknown}] └── false [type=bool] @@ -1580,16 +1635,16 @@ opt SELECT * FROM xysd, uv WHERE (s = 'foo' AND u = 3 AND v = 4) OR (s = 'bar' AND u = 5 AND v = 6) ---- inner-join (cross) - ├── columns: x:1(int!null) y:2(int) s:3(string!null) d:4(decimal!null) u:5(int!null) v:6(int!null) - ├── stats: [rows=11979.6897, distinct(3)=2, null(3)=0, distinct(5)=2, null(5)=0, distinct(6)=2, null(6)=0, distinct(5,6)=2.19138756, null(5,6)=0] + ├── columns: x:1(int!null) y:2(int) s:3(string!null) d:4(decimal!null) u:6(int!null) v:7(int!null) + ├── stats: [rows=11979.6897, distinct(3)=2, null(3)=0, distinct(6)=2, null(6)=0, distinct(7)=2, null(7)=0, distinct(6,7)=2.19138756, null(6,7)=0] ├── fd: (1)-->(2-4), (3,4)-->(1,2) ├── scan uv - │ ├── columns: u:5(int) v:6(int!null) - │ └── stats: [rows=10000, distinct(5)=500, null(5)=0, distinct(6)=100, null(6)=0, distinct(5,6)=550, null(5,6)=0] + │ ├── columns: u:6(int) v:7(int!null) + │ └── stats: [rows=10000, distinct(6)=500, null(6)=0, distinct(7)=100, null(7)=0, distinct(6,7)=550, null(6,7)=0] ├── scan xysd │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) │ ├── stats: [rows=5000, distinct(1)=5000, null(1)=0, distinct(3)=10, null(3)=0, distinct(4)=500, null(4)=0] │ ├── key: (1) │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) └── filters - └── (((s:3 = 'foo') AND (u:5 = 3)) AND (v:6 = 4)) OR (((s:3 = 'bar') AND (u:5 = 5)) AND (v:6 = 6)) [type=bool, outer=(3,5,6), constraints=(/3: [/'bar' - /'bar'] [/'foo' - /'foo']; /5: [/3 - /3] [/5 - /5]; /6: [/4 - /4] [/6 - /6])] + └── (((s:3 = 'foo') AND (u:6 = 3)) AND (v:7 = 4)) OR (((s:3 = 'bar') AND (u:6 = 5)) AND (v:7 = 6)) [type=bool, outer=(3,6,7), constraints=(/3: [/'bar' - /'bar'] [/'foo' - /'foo']; /6: [/3 - /3] [/5 - /5]; /7: [/4 - /4] [/6 - /6])] diff --git a/pkg/sql/opt/memo/testdata/stats/limit b/pkg/sql/opt/memo/testdata/stats/limit index c48fdc6c742d..ff0dc0ebbe1e 100644 --- a/pkg/sql/opt/memo/testdata/stats/limit +++ b/pkg/sql/opt/memo/testdata/stats/limit @@ -40,20 +40,26 @@ limit ├── stats: [rows=5] ├── key: (1) ├── fd: ()-->(3), (1)-->(2,4), (4)-->(1,2) - ├── select + ├── project │ ├── columns: x:1(int!null) y:2(int) s:3(string!null) d:4(decimal!null) - │ ├── stats: [rows=200, distinct(3)=1, null(3)=0] + │ ├── stats: [rows=200] │ ├── key: (1) │ ├── fd: ()-->(3), (1)-->(2,4), (4)-->(1,2) │ ├── limit hint: 5.00 - │ ├── scan a - │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) - │ │ ├── stats: [rows=2000, distinct(1)=2000, null(1)=0, distinct(3)=10, null(3)=0, distinct(4)=200, null(4)=0] - │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ │ └── limit hint: 50.00 - │ └── filters - │ └── s:3 = 'foo' [type=bool, outer=(3), constraints=(/3: [/'foo' - /'foo']; tight), fd=()-->(3)] + │ └── select + │ ├── columns: x:1(int!null) y:2(int) s:3(string!null) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) + │ ├── stats: [rows=200, distinct(3)=1, null(3)=0] + │ ├── key: (1) + │ ├── fd: ()-->(3), (1)-->(2,4,5), (4)-->(1,2,5) + │ ├── limit hint: 5.00 + │ ├── scan a + │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) + │ │ ├── stats: [rows=2000, distinct(1)=2000, null(1)=0, distinct(3)=10, null(3)=0, distinct(4)=200, null(4)=0] + │ │ ├── key: (1) + │ │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ │ └── limit hint: 50.00 + │ └── filters + │ └── s:3 = 'foo' [type=bool, outer=(3), constraints=(/3: [/'foo' - /'foo']; tight), fd=()-->(3)] └── 5 [type=int] build @@ -65,39 +71,44 @@ limit ├── stats: [rows=200] ├── key: (1) ├── fd: ()-->(3), (1)-->(2,4), (4)-->(1,2) - ├── select + ├── project │ ├── columns: x:1(int!null) y:2(int) s:3(string!null) d:4(decimal!null) - │ ├── stats: [rows=200, distinct(3)=1, null(3)=0] + │ ├── stats: [rows=200] │ ├── key: (1) │ ├── fd: ()-->(3), (1)-->(2,4), (4)-->(1,2) - │ ├── scan a - │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) - │ │ ├── stats: [rows=2000, distinct(1)=2000, null(1)=0, distinct(3)=10, null(3)=0, distinct(4)=200, null(4)=0] - │ │ ├── key: (1) - │ │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ └── filters - │ └── s:3 = 'foo' [type=bool, outer=(3), constraints=(/3: [/'foo' - /'foo']; tight), fd=()-->(3)] + │ └── select + │ ├── columns: x:1(int!null) y:2(int) s:3(string!null) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) + │ ├── stats: [rows=200, distinct(3)=1, null(3)=0] + │ ├── key: (1) + │ ├── fd: ()-->(3), (1)-->(2,4,5), (4)-->(1,2,5) + │ ├── scan a + │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) + │ │ ├── stats: [rows=2000, distinct(1)=2000, null(1)=0, distinct(3)=10, null(3)=0, distinct(4)=200, null(4)=0] + │ │ ├── key: (1) + │ │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ └── filters + │ └── s:3 = 'foo' [type=bool, outer=(3), constraints=(/3: [/'foo' - /'foo']; tight), fd=()-->(3)] └── subquery [type=int] └── max1-row - ├── columns: c:5(int!null) + ├── columns: c:6(int!null) ├── error: "more than one row returned by a subquery used as an expression" ├── cardinality: [1 - 1] ├── stats: [rows=1] ├── key: () - ├── fd: ()-->(5) + ├── fd: ()-->(6) └── project - ├── columns: c:5(int!null) + ├── columns: c:6(int!null) ├── cardinality: [1 - 1] ├── stats: [rows=1] ├── key: () - ├── fd: ()-->(5) + ├── fd: ()-->(6) ├── values │ ├── cardinality: [1 - 1] │ ├── stats: [rows=1] │ ├── key: () │ └── () [type=tuple] └── projections - └── 5 [as=c:5, type=int] + └── 5 [as=c:6, type=int] build SELECT * FROM (SELECT * FROM a ORDER BY s LIMIT 5) WHERE s = 'foo' @@ -122,11 +133,16 @@ select │ │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) │ │ ├── ordering: +3 │ │ ├── limit hint: 5.00 - │ │ └── scan a + │ │ └── project │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) │ │ ├── stats: [rows=2000, distinct(1)=2000, null(1)=0, distinct(3)=10, null(3)=0, distinct(4)=200, null(4)=0] │ │ ├── key: (1) - │ │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) + │ │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) + │ │ └── scan a + │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) + │ │ ├── stats: [rows=2000, distinct(1)=2000, null(1)=0, distinct(3)=10, null(3)=0, distinct(4)=200, null(4)=0] + │ │ ├── key: (1) + │ │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) │ └── 5 [type=int] └── filters └── s:3 = 'foo' [type=bool, outer=(3), constraints=(/3: [/'foo' - /'foo']; tight), fd=()-->(3)] @@ -173,12 +189,18 @@ limit ├── stats: [rows=5, distinct(2)=4.98133092, null(2)=2.5, distinct(3)=4.02187199, null(3)=2.5, distinct(2,3)=5, null(2,3)=1.25] ├── key: (1) ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - ├── scan a + ├── project │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) │ ├── stats: [rows=2000, distinct(2)=501, null(2)=1000, distinct(3)=11, null(3)=1000, distinct(2,3)=2000, null(2,3)=500] │ ├── key: (1) │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ └── limit hint: 5.00 + │ ├── limit hint: 5.00 + │ └── scan a + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) + │ ├── stats: [rows=2000, distinct(2)=501, null(2)=1000, distinct(3)=11, null(3)=1000, distinct(2,3)=2000, null(2,3)=500] + │ ├── key: (1) + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ └── limit hint: 5.00 └── 5 [type=int] build colstat=2 colstat=3 colstat=(2,3) @@ -190,20 +212,26 @@ limit ├── stats: [rows=5, distinct(2)=4.99009771, null(2)=2.5, distinct(3)=0.994079471, null(3)=0, distinct(2,3)=4.99009771, null(2,3)=0] ├── key: (1) ├── fd: ()-->(3), (1)-->(2,4), (4)-->(1,2) - ├── select + ├── project │ ├── columns: x:1(int!null) y:2(int) s:3(string!null) d:4(decimal!null) │ ├── stats: [rows=100, distinct(2)=92.7652197, null(2)=50, distinct(3)=1, null(3)=0, distinct(2,3)=92.7652197, null(2,3)=0] │ ├── key: (1) │ ├── fd: ()-->(3), (1)-->(2,4), (4)-->(1,2) │ ├── limit hint: 5.00 - │ ├── scan a - │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) - │ │ ├── stats: [rows=2000, distinct(1)=2000, null(1)=0, distinct(2)=501, null(2)=1000, distinct(3)=11, null(3)=1000, distinct(4)=200, null(4)=0, distinct(2,3)=2000, null(2,3)=500] - │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - │ │ └── limit hint: 100.00 - │ └── filters - │ └── s:3 = 'foo' [type=bool, outer=(3), constraints=(/3: [/'foo' - /'foo']; tight), fd=()-->(3)] + │ └── select + │ ├── columns: x:1(int!null) y:2(int) s:3(string!null) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) + │ ├── stats: [rows=100, distinct(2)=92.7652197, null(2)=50, distinct(3)=1, null(3)=0, distinct(2,3)=92.7652197, null(2,3)=0] + │ ├── key: (1) + │ ├── fd: ()-->(3), (1)-->(2,4,5), (4)-->(1,2,5) + │ ├── limit hint: 5.00 + │ ├── scan a + │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) + │ │ ├── stats: [rows=2000, distinct(1)=2000, null(1)=0, distinct(2)=501, null(2)=1000, distinct(3)=11, null(3)=1000, distinct(4)=200, null(4)=0, distinct(2,3)=2000, null(2,3)=500] + │ │ ├── key: (1) + │ │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ │ └── limit hint: 100.00 + │ └── filters + │ └── s:3 = 'foo' [type=bool, outer=(3), constraints=(/3: [/'foo' - /'foo']; tight), fd=()-->(3)] └── 5 [type=int] exec-ddl diff --git a/pkg/sql/opt/memo/testdata/stats/lookup-join b/pkg/sql/opt/memo/testdata/stats/lookup-join index 2290a92e750c..cf7eaa3718a6 100644 --- a/pkg/sql/opt/memo/testdata/stats/lookup-join +++ b/pkg/sql/opt/memo/testdata/stats/lookup-join @@ -25,16 +25,16 @@ opt SELECT * FROM small JOIN abcd ON a=m ---- inner-join (lookup abcd) - ├── columns: m:1(int!null) n:2(int) a:4(int!null) b:5(int) c:6(int) - ├── key columns: [7] = [7] + ├── columns: m:1(int!null) n:2(int) a:5(int!null) b:6(int) c:7(int) + ├── key columns: [8] = [8] ├── lookup columns are key - ├── stats: [rows=99, distinct(1)=10, null(1)=0, distinct(4)=10, null(4)=0] - ├── fd: (1)==(4), (4)==(1) + ├── stats: [rows=99, distinct(1)=10, null(1)=0, distinct(5)=10, null(5)=0] + ├── fd: (1)==(5), (5)==(1) ├── inner-join (lookup abcd@secondary) - │ ├── columns: m:1(int!null) n:2(int) a:4(int!null) b:5(int) abcd.rowid:7(int!null) - │ ├── key columns: [1] = [4] - │ ├── stats: [rows=99, distinct(1)=10, null(1)=0, distinct(4)=10, null(4)=0] - │ ├── fd: (7)-->(4,5), (1)==(4), (4)==(1) + │ ├── columns: m:1(int!null) n:2(int) a:5(int!null) b:6(int) abcd.rowid:8(int!null) + │ ├── key columns: [1] = [5] + │ ├── stats: [rows=99, distinct(1)=10, null(1)=0, distinct(5)=10, null(5)=0] + │ ├── fd: (8)-->(5,6), (1)==(5), (5)==(1) │ ├── scan small │ │ ├── columns: m:1(int) n:2(int) │ │ └── stats: [rows=10, distinct(1)=10, null(1)=0] @@ -47,16 +47,16 @@ opt SELECT * FROM small JOIN abcd ON a=m WHERE n > 2 ---- inner-join (lookup abcd) - ├── columns: m:1(int!null) n:2(int!null) a:4(int!null) b:5(int) c:6(int) - ├── key columns: [7] = [7] + ├── columns: m:1(int!null) n:2(int!null) a:5(int!null) b:6(int) c:7(int) + ├── key columns: [8] = [8] ├── lookup columns are key - ├── stats: [rows=98.01, distinct(1)=9.9, null(1)=0, distinct(4)=9.9, null(4)=0] - ├── fd: (1)==(4), (4)==(1) + ├── stats: [rows=98.01, distinct(1)=9.9, null(1)=0, distinct(5)=9.9, null(5)=0] + ├── fd: (1)==(5), (5)==(1) ├── inner-join (lookup abcd@secondary) - │ ├── columns: m:1(int!null) n:2(int!null) a:4(int!null) b:5(int) abcd.rowid:7(int!null) - │ ├── key columns: [1] = [4] - │ ├── stats: [rows=98.01, distinct(1)=9.9, null(1)=0, distinct(4)=9.9, null(4)=0] - │ ├── fd: (7)-->(4,5), (1)==(4), (4)==(1) + │ ├── columns: m:1(int!null) n:2(int!null) a:5(int!null) b:6(int) abcd.rowid:8(int!null) + │ ├── key columns: [1] = [5] + │ ├── stats: [rows=98.01, distinct(1)=9.9, null(1)=0, distinct(5)=9.9, null(5)=0] + │ ├── fd: (8)-->(5,6), (1)==(5), (5)==(1) │ ├── select │ │ ├── columns: m:1(int) n:2(int!null) │ │ ├── stats: [rows=9.9, distinct(1)=9.9, null(1)=0, distinct(2)=1, null(2)=0] @@ -74,21 +74,21 @@ opt SELECT * FROM small JOIN abcd ON a=m WHERE b > 2 ---- inner-join (lookup abcd) - ├── columns: m:1(int!null) n:2(int) a:4(int!null) b:5(int!null) c:6(int) - ├── key columns: [7] = [7] + ├── columns: m:1(int!null) n:2(int) a:5(int!null) b:6(int!null) c:7(int) + ├── key columns: [8] = [8] ├── lookup columns are key - ├── stats: [rows=33.5823697, distinct(1)=10, null(1)=0, distinct(4)=10, null(4)=0] - ├── fd: (1)==(4), (4)==(1) + ├── stats: [rows=33.5823697, distinct(1)=10, null(1)=0, distinct(5)=10, null(5)=0] + ├── fd: (1)==(5), (5)==(1) ├── inner-join (lookup abcd@secondary) - │ ├── columns: m:1(int!null) n:2(int) a:4(int!null) b:5(int!null) abcd.rowid:7(int!null) - │ ├── key columns: [1] = [4] - │ ├── stats: [rows=33, distinct(1)=10, null(1)=0, distinct(4)=10, null(4)=0, distinct(5)=33, null(5)=0] - │ ├── fd: (7)-->(4,5), (1)==(4), (4)==(1) + │ ├── columns: m:1(int!null) n:2(int) a:5(int!null) b:6(int!null) abcd.rowid:8(int!null) + │ ├── key columns: [1] = [5] + │ ├── stats: [rows=33, distinct(1)=10, null(1)=0, distinct(5)=10, null(5)=0, distinct(6)=33, null(6)=0] + │ ├── fd: (8)-->(5,6), (1)==(5), (5)==(1) │ ├── scan small │ │ ├── columns: m:1(int) n:2(int) │ │ └── stats: [rows=10, distinct(1)=10, null(1)=0] │ └── filters - │ └── b:5 > 2 [type=bool, outer=(5), constraints=(/5: [/3 - ]; tight)] + │ └── b:6 > 2 [type=bool, outer=(6), constraints=(/6: [/3 - ]; tight)] └── filters (true) # Filter that can only be applied after the primary index join. @@ -96,44 +96,44 @@ opt SELECT * FROM small JOIN abcd ON a=m WHERE c>2 ---- inner-join (lookup abcd) - ├── columns: m:1(int!null) n:2(int) a:4(int!null) b:5(int) c:6(int!null) - ├── key columns: [7] = [7] + ├── columns: m:1(int!null) n:2(int) a:5(int!null) b:6(int) c:7(int!null) + ├── key columns: [8] = [8] ├── lookup columns are key - ├── stats: [rows=33.5823697, distinct(1)=10, null(1)=0, distinct(4)=10, null(4)=0] - ├── fd: (1)==(4), (4)==(1) + ├── stats: [rows=33.5823697, distinct(1)=10, null(1)=0, distinct(5)=10, null(5)=0] + ├── fd: (1)==(5), (5)==(1) ├── inner-join (lookup abcd@secondary) - │ ├── columns: m:1(int!null) n:2(int) a:4(int!null) b:5(int) abcd.rowid:7(int!null) - │ ├── key columns: [1] = [4] - │ ├── stats: [rows=99, distinct(1)=10, null(1)=0, distinct(4)=10, null(4)=0] - │ ├── fd: (7)-->(4,5), (1)==(4), (4)==(1) + │ ├── columns: m:1(int!null) n:2(int) a:5(int!null) b:6(int) abcd.rowid:8(int!null) + │ ├── key columns: [1] = [5] + │ ├── stats: [rows=99, distinct(1)=10, null(1)=0, distinct(5)=10, null(5)=0] + │ ├── fd: (8)-->(5,6), (1)==(5), (5)==(1) │ ├── scan small │ │ ├── columns: m:1(int) n:2(int) │ │ └── stats: [rows=10, distinct(1)=10, null(1)=0] │ └── filters (true) └── filters - └── c:6 > 2 [type=bool, outer=(6), constraints=(/6: [/3 - ]; tight)] + └── c:7 > 2 [type=bool, outer=(7), constraints=(/7: [/3 - ]; tight)] # Multiple equalities. opt SELECT * FROM small JOIN abcd ON a=m AND b=n WHERE c>2 ---- inner-join (lookup abcd) - ├── columns: m:1(int!null) n:2(int!null) a:4(int!null) b:5(int!null) c:6(int!null) - ├── key columns: [7] = [7] + ├── columns: m:1(int!null) n:2(int!null) a:5(int!null) b:6(int!null) c:7(int!null) + ├── key columns: [8] = [8] ├── lookup columns are key - ├── stats: [rows=0.334949339, distinct(1)=0.334949339, null(1)=0, distinct(2)=1e-10, null(2)=0, distinct(4)=0.334949339, null(4)=0, distinct(5)=1e-10, null(5)=0] - ├── fd: (1)==(4), (4)==(1), (2)==(5), (5)==(2) + ├── stats: [rows=0.334949339, distinct(1)=0.334949339, null(1)=0, distinct(2)=1e-10, null(2)=0, distinct(5)=0.334949339, null(5)=0, distinct(6)=1e-10, null(6)=0] + ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2) ├── inner-join (lookup abcd@secondary) - │ ├── columns: m:1(int!null) n:2(int!null) a:4(int!null) b:5(int!null) abcd.rowid:7(int!null) - │ ├── key columns: [1 2] = [4 5] - │ ├── stats: [rows=0.970299, distinct(1)=0.970299, null(1)=0, distinct(2)=1e-10, null(2)=0, distinct(4)=0.970299, null(4)=0, distinct(5)=1e-10, null(5)=0] - │ ├── fd: (7)-->(4,5), (1)==(4), (4)==(1), (2)==(5), (5)==(2) + │ ├── columns: m:1(int!null) n:2(int!null) a:5(int!null) b:6(int!null) abcd.rowid:8(int!null) + │ ├── key columns: [1 2] = [5 6] + │ ├── stats: [rows=0.970299, distinct(1)=0.970299, null(1)=0, distinct(2)=1e-10, null(2)=0, distinct(5)=0.970299, null(5)=0, distinct(6)=1e-10, null(6)=0] + │ ├── fd: (8)-->(5,6), (1)==(5), (5)==(1), (2)==(6), (6)==(2) │ ├── scan small │ │ ├── columns: m:1(int) n:2(int) │ │ └── stats: [rows=10, distinct(1)=10, null(1)=0, distinct(2)=1, null(2)=0.1] │ └── filters (true) └── filters - └── c:6 > 2 [type=bool, outer=(6), constraints=(/6: [/3 - ]; tight)] + └── c:7 > 2 [type=bool, outer=(7), constraints=(/7: [/3 - ]; tight)] exec-ddl CREATE TABLE abc (a INT, b INT, c INT, PRIMARY KEY (a, c)) @@ -183,11 +183,11 @@ opt SELECT a, b, c, d, e, f FROM abc JOIN def ON a = f ---- inner-join (lookup def) - ├── columns: a:1(int!null) b:2(int) c:3(int!null) d:4(int) e:5(int!null) f:6(int!null) - ├── key columns: [1] = [6] - ├── stats: [rows=100, distinct(1)=100, null(1)=0, distinct(6)=100, null(6)=0] - ├── key: (3,5,6) - ├── fd: (1,3)-->(2), (5,6)-->(4), (1)==(6), (6)==(1) + ├── columns: a:1(int!null) b:2(int) c:3(int!null) d:5(int) e:6(int!null) f:7(int!null) + ├── key columns: [1] = [7] + ├── stats: [rows=100, distinct(1)=100, null(1)=0, distinct(7)=100, null(7)=0] + ├── key: (3,6,7) + ├── fd: (1,3)-->(2), (6,7)-->(5), (1)==(7), (7)==(1) ├── scan abc │ ├── columns: a:1(int!null) b:2(int) c:3(int!null) │ ├── stats: [rows=100, distinct(1)=100, null(1)=0, distinct(3)=10, null(3)=0] @@ -200,12 +200,12 @@ opt SELECT a, b, c, d, e, f FROM abc JOIN def ON a = e ---- inner-join (merge) - ├── columns: a:1(int!null) b:2(int) c:3(int!null) d:4(int) e:5(int!null) f:6(int!null) + ├── columns: a:1(int!null) b:2(int) c:3(int!null) d:5(int) e:6(int!null) f:7(int!null) ├── left ordering: +1 - ├── right ordering: +5 - ├── stats: [rows=10000, distinct(1)=100, null(1)=0, distinct(5)=100, null(5)=0] - ├── key: (3,5,6) - ├── fd: (1,3)-->(2), (5,6)-->(4), (1)==(5), (5)==(1) + ├── right ordering: +6 + ├── stats: [rows=10000, distinct(1)=100, null(1)=0, distinct(6)=100, null(6)=0] + ├── key: (3,6,7) + ├── fd: (1,3)-->(2), (6,7)-->(5), (1)==(6), (6)==(1) ├── scan abc │ ├── columns: a:1(int!null) b:2(int) c:3(int!null) │ ├── stats: [rows=100, distinct(1)=100, null(1)=0, distinct(3)=10, null(3)=0] @@ -213,23 +213,23 @@ inner-join (merge) │ ├── fd: (1,3)-->(2) │ └── ordering: +1 ├── scan def@e_idx - │ ├── columns: d:4(int) e:5(int!null) f:6(int!null) - │ ├── stats: [rows=10000, distinct(5)=100, null(5)=0, distinct(6)=10000, null(6)=0] - │ ├── key: (5,6) - │ ├── fd: (5,6)-->(4) - │ └── ordering: +5 + │ ├── columns: d:5(int) e:6(int!null) f:7(int!null) + │ ├── stats: [rows=10000, distinct(6)=100, null(6)=0, distinct(7)=10000, null(7)=0] + │ ├── key: (6,7) + │ ├── fd: (6,7)-->(5) + │ └── ordering: +6 └── filters (true) # Check column statistics for lookup join. -opt colstat=1 colstat=2 colstat=3 colstat=4 colstat=5 colstat=6 colstat=(2,5,6) +opt colstat=1 colstat=2 colstat=3 colstat=4 colstat=6 colstat=7 colstat=(2,6,7) SELECT a, b, c, d, e, f FROM abc JOIN DEF ON a = f ---- inner-join (lookup def) - ├── columns: a:1(int!null) b:2(int) c:3(int!null) d:4(int) e:5(int!null) f:6(int!null) - ├── key columns: [1] = [6] - ├── stats: [rows=100, distinct(1)=100, null(1)=0, distinct(2)=9.99954623, null(2)=1, distinct(3)=9.99954623, null(3)=0, distinct(4)=95.1671064, null(4)=0, distinct(5)=63.2138954, null(5)=0, distinct(6)=100, null(6)=0, distinct(2,5,6)=100, null(2,5,6)=0] - ├── key: (3,5,6) - ├── fd: (1,3)-->(2), (5,6)-->(4), (1)==(6), (6)==(1) + ├── columns: a:1(int!null) b:2(int) c:3(int!null) d:5(int) e:6(int!null) f:7(int!null) + ├── key columns: [1] = [7] + ├── stats: [rows=100, distinct(1)=100, null(1)=0, distinct(2)=9.99954623, null(2)=1, distinct(3)=9.99954623, null(3)=0, distinct(4)=1, null(4)=0, distinct(6)=63.2138954, null(6)=0, distinct(7)=100, null(7)=0, distinct(2,6,7)=100, null(2,6,7)=0] + ├── key: (3,6,7) + ├── fd: (1,3)-->(2), (6,7)-->(5), (1)==(7), (7)==(1) ├── scan abc │ ├── columns: a:1(int!null) b:2(int) c:3(int!null) │ ├── stats: [rows=100, distinct(1)=100, null(1)=0, distinct(2)=10, null(2)=1, distinct(3)=10, null(3)=0] @@ -238,22 +238,22 @@ inner-join (lookup def) └── filters (true) # Check column statistics for double lookup join. -opt colstat=7 +opt colstat=8 SELECT * FROM abc LEFT JOIN DEF ON a = d AND b = 3 ---- left-join (lookup def) - ├── columns: a:1(int!null) b:2(int) c:3(int!null) d:4(int) e:5(int) f:6(int) g:7(float) - ├── key columns: [6 5] = [6 5] + ├── columns: a:1(int!null) b:2(int) c:3(int!null) d:5(int) e:6(int) f:7(int) g:8(float) + ├── key columns: [7 6] = [7 6] ├── lookup columns are key - ├── stats: [rows=100, distinct(4)=10, null(4)=90, distinct(7)=9.95021575, null(7)=91] - ├── key: (1,3,5,6) - ├── fd: (1,3)-->(2), (5,6)-->(4,7) + ├── stats: [rows=100, distinct(5)=10, null(5)=90, distinct(8)=9.95021575, null(8)=91] + ├── key: (1,3,6,7) + ├── fd: (1,3)-->(2), (6,7)-->(5,8) ├── left-join (lookup def@d_idx) - │ ├── columns: a:1(int!null) b:2(int) c:3(int!null) d:4(int) e:5(int) f:6(int) - │ ├── key columns: [1] = [4] - │ ├── stats: [rows=100, distinct(4)=10, null(4)=90] - │ ├── key: (1,3,5,6) - │ ├── fd: (1,3)-->(2), (5,6)-->(4) + │ ├── columns: a:1(int!null) b:2(int) c:3(int!null) d:5(int) e:6(int) f:7(int) + │ ├── key columns: [1] = [5] + │ ├── stats: [rows=100, distinct(5)=10, null(5)=90] + │ ├── key: (1,3,6,7) + │ ├── fd: (1,3)-->(2), (6,7)-->(5) │ ├── scan abc │ │ ├── columns: a:1(int!null) b:2(int) c:3(int!null) │ │ ├── stats: [rows=100, distinct(1)=100, null(1)=0, distinct(2)=10, null(2)=1, distinct(3)=10, null(3)=0] @@ -265,26 +265,26 @@ left-join (lookup def) # The filter a=e is not very selective, so we do not expect a lookup join, even # though there is an additional filter. -opt colstat=7 +opt colstat=8 SELECT * FROM abc LEFT JOIN DEF ON a = e AND b = 3 ---- right-join (hash) - ├── columns: a:1(int!null) b:2(int) c:3(int!null) d:4(int) e:5(int) f:6(int) g:7(float) - ├── stats: [rows=100, distinct(5)=100, null(5)=0, distinct(7)=95.1671064, null(7)=1] - ├── key: (1,3,5,6) - ├── fd: (1,3)-->(2), (5,6)-->(4,7) + ├── columns: a:1(int!null) b:2(int) c:3(int!null) d:5(int) e:6(int) f:7(int) g:8(float) + ├── stats: [rows=100, distinct(6)=100, null(6)=0, distinct(8)=95.1671064, null(8)=1] + ├── key: (1,3,6,7) + ├── fd: (1,3)-->(2), (6,7)-->(5,8) ├── scan def - │ ├── columns: d:4(int) e:5(int!null) f:6(int!null) g:7(float) - │ ├── stats: [rows=10000, distinct(5)=100, null(5)=0, distinct(7)=1000, null(7)=100] - │ ├── key: (5,6) - │ └── fd: (5,6)-->(4,7) + │ ├── columns: d:5(int) e:6(int!null) f:7(int!null) g:8(float) + │ ├── stats: [rows=10000, distinct(6)=100, null(6)=0, distinct(8)=1000, null(8)=100] + │ ├── key: (6,7) + │ └── fd: (6,7)-->(5,8) ├── scan abc │ ├── columns: a:1(int!null) b:2(int) c:3(int!null) │ ├── stats: [rows=100, distinct(1)=100, null(1)=0, distinct(2)=10, null(2)=1, distinct(3)=10, null(3)=0] │ ├── key: (1,3) │ └── fd: (1,3)-->(2) └── filters - ├── a:1 = e:5 [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + ├── a:1 = e:6 [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] └── b:2 = 3 [type=bool, outer=(2), constraints=(/2: [/3 - /3]; tight), fd=()-->(2)] exec-ddl @@ -335,7 +335,7 @@ SELECT * FROM u WHERE EXISTS (SELECT * FROM t WHERE u.x=t.x AND u.y=t.y); ---- semi-join (lookup t@xy_idx) ├── columns: x:1(int) y:2(int) - ├── key columns: [1 2] = [4 5] + ├── key columns: [1 2] = [5 6] ├── stats: [rows=10, distinct(1)=2, null(1)=0, distinct(2)=2, null(2)=0] ├── scan u │ ├── columns: u.x:1(int) u.y:2(int) @@ -347,7 +347,7 @@ SELECT * FROM u WHERE NOT EXISTS (SELECT * FROM t WHERE u.x=t.x AND u.y=t.y); ---- anti-join (lookup t@xy_idx) ├── columns: x:1(int) y:2(int) - ├── key columns: [1 2] = [4 5] + ├── key columns: [1 2] = [5 6] ├── stats: [rows=1e-10] ├── scan u │ ├── columns: u.x:1(int) u.y:2(int) @@ -395,25 +395,25 @@ opt SELECT * FROM medium INNER JOIN wxyz ON m=x AND y=10 ---- inner-join (lookup wxyz) - ├── columns: m:1(int!null) n:2(int) w:4(int) x:5(int!null) y:6(int!null) z:7(int) - ├── key columns: [8] = [8] + ├── columns: m:1(int!null) n:2(int) w:5(int) x:6(int!null) y:7(int!null) z:8(int) + ├── key columns: [9] = [9] ├── lookup columns are key - ├── stats: [rows=49.3441882, distinct(1)=39.1263061, null(1)=0, distinct(5)=39.1263061, null(5)=0] - ├── fd: ()-->(6), (1)==(5), (5)==(1) + ├── stats: [rows=49.3441882, distinct(1)=39.1263061, null(1)=0, distinct(6)=39.1263061, null(6)=0] + ├── fd: ()-->(7), (1)==(6), (6)==(1) ├── inner-join (lookup wxyz@secondary) - │ ├── columns: m:1(int!null) n:2(int) x:5(int!null) y:6(int!null) z:7(int) wxyz.rowid:8(int!null) - │ ├── key columns: [1 9] = [5 6] - │ ├── stats: [rows=19.8, distinct(1)=19.8, null(1)=0, distinct(5)=19.8, null(5)=0, distinct(6)=1, null(6)=0, distinct(9)=1, null(9)=0] - │ ├── fd: ()-->(6), (8)-->(5,7), (1)==(5), (5)==(1) + │ ├── columns: m:1(int!null) n:2(int) x:6(int!null) y:7(int!null) z:8(int) wxyz.rowid:9(int!null) + │ ├── key columns: [1 11] = [6 7] + │ ├── stats: [rows=19.8, distinct(1)=19.8, null(1)=0, distinct(6)=19.8, null(6)=0, distinct(7)=1, null(7)=0, distinct(11)=1, null(11)=0] + │ ├── fd: ()-->(7), (9)-->(6,8), (1)==(6), (6)==(1) │ ├── project - │ │ ├── columns: "project_const_col_@6":9(int!null) m:1(int) n:2(int) - │ │ ├── stats: [rows=40, distinct(1)=40, null(1)=0, distinct(9)=1, null(9)=0] - │ │ ├── fd: ()-->(9) + │ │ ├── columns: "project_const_col_@7":11(int!null) m:1(int) n:2(int) + │ │ ├── stats: [rows=40, distinct(1)=40, null(1)=0, distinct(11)=1, null(11)=0] + │ │ ├── fd: ()-->(11) │ │ ├── scan medium │ │ │ ├── columns: m:1(int) n:2(int) │ │ │ └── stats: [rows=40, distinct(1)=40, null(1)=0] │ │ └── projections - │ │ └── 10 [as="project_const_col_@6":9, type=int] + │ │ └── 10 [as="project_const_col_@7":11, type=int] │ └── filters (true) └── filters (true) @@ -438,24 +438,24 @@ opt SELECT * FROM medium INNER JOIN wxyz ON m=x AND y=10 ---- inner-join (hash) - ├── columns: m:1(int!null) n:2(int) w:4(int) x:5(int!null) y:6(int!null) z:7(int) - ├── stats: [rows=356.4, distinct(1)=40, null(1)=0, distinct(5)=40, null(5)=0] - ├── fd: ()-->(6), (1)==(5), (5)==(1) + ├── columns: m:1(int!null) n:2(int) w:5(int) x:6(int!null) y:7(int!null) z:8(int) + ├── stats: [rows=356.4, distinct(1)=40, null(1)=0, distinct(6)=40, null(6)=0] + ├── fd: ()-->(7), (1)==(6), (6)==(1) ├── select - │ ├── columns: w:4(int) x:5(int) y:6(int!null) z:7(int) - │ ├── stats: [rows=900, distinct(5)=100, null(5)=9, distinct(6)=1, null(6)=0] - │ │ histogram(6)= 0 900 + │ ├── columns: w:5(int) x:6(int) y:7(int!null) z:8(int) + │ ├── stats: [rows=900, distinct(6)=100, null(6)=9, distinct(7)=1, null(7)=0] + │ │ histogram(7)= 0 900 │ │ <--- 10 - │ ├── fd: ()-->(6) + │ ├── fd: ()-->(7) │ ├── scan wxyz - │ │ ├── columns: w:4(int) x:5(int) y:6(int) z:7(int) - │ │ └── stats: [rows=1000, distinct(5)=100, null(5)=10, distinct(6)=11, null(6)=0] - │ │ histogram(6)= 0 10 90 900 + │ │ ├── columns: w:5(int) x:6(int) y:7(int) z:8(int) + │ │ └── stats: [rows=1000, distinct(6)=100, null(6)=10, distinct(7)=11, null(7)=0] + │ │ histogram(7)= 0 10 90 900 │ │ <--- 0 ---- 10 │ └── filters - │ └── y:6 = 10 [type=bool, outer=(6), constraints=(/6: [/10 - /10]; tight), fd=()-->(6)] + │ └── y:7 = 10 [type=bool, outer=(7), constraints=(/7: [/10 - /10]; tight), fd=()-->(7)] ├── scan medium │ ├── columns: m:1(int) n:2(int) │ └── stats: [rows=40, distinct(1)=40, null(1)=0] └── filters - └── m:1 = x:5 [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + └── m:1 = x:6 [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] diff --git a/pkg/sql/opt/memo/testdata/stats/ordinality b/pkg/sql/opt/memo/testdata/stats/ordinality index 9343c8e99752..d98462569de1 100644 --- a/pkg/sql/opt/memo/testdata/stats/ordinality +++ b/pkg/sql/opt/memo/testdata/stats/ordinality @@ -23,37 +23,37 @@ norm SELECT * FROM (SELECT * FROM a WITH ORDINALITY) WHERE ordinality > 0 AND ordinality <= 10 ---- select - ├── columns: x:1(int!null) y:2(int) ordinality:3(int!null) + ├── columns: x:1(int!null) y:2(int) ordinality:4(int!null) ├── cardinality: [0 - 10] - ├── stats: [rows=10, distinct(3)=10, null(3)=0] + ├── stats: [rows=10, distinct(4)=10, null(4)=0] ├── key: (1) - ├── fd: (1)-->(2,3), (3)-->(1,2) + ├── fd: (1)-->(2,4), (4)-->(1,2) ├── ordinality - │ ├── columns: x:1(int!null) y:2(int) ordinality:3(int!null) - │ ├── stats: [rows=4000, distinct(1)=4000, null(1)=0, distinct(3)=4000, null(3)=0] + │ ├── columns: x:1(int!null) y:2(int) ordinality:4(int!null) + │ ├── stats: [rows=4000, distinct(1)=4000, null(1)=0, distinct(4)=4000, null(4)=0] │ ├── key: (1) - │ ├── fd: (1)-->(2,3), (3)-->(1,2) + │ ├── fd: (1)-->(2,4), (4)-->(1,2) │ └── scan a │ ├── columns: x:1(int!null) y:2(int) │ ├── stats: [rows=4000, distinct(1)=4000, null(1)=0] │ ├── key: (1) │ └── fd: (1)-->(2) └── filters - └── (ordinality:3 > 0) AND (ordinality:3 <= 10) [type=bool, outer=(3), constraints=(/3: [/1 - /10]; tight)] + └── (ordinality:4 > 0) AND (ordinality:4 <= 10) [type=bool, outer=(4), constraints=(/4: [/1 - /10]; tight)] norm SELECT * FROM (SELECT * FROM a WITH ORDINALITY) WHERE y > 0 AND y <= 10 ---- select - ├── columns: x:1(int!null) y:2(int!null) ordinality:3(int!null) + ├── columns: x:1(int!null) y:2(int!null) ordinality:4(int!null) ├── stats: [rows=100, distinct(2)=10, null(2)=0] ├── key: (1) - ├── fd: (1)-->(2,3), (3)-->(1,2) + ├── fd: (1)-->(2,4), (4)-->(1,2) ├── ordinality - │ ├── columns: x:1(int!null) y:2(int) ordinality:3(int!null) - │ ├── stats: [rows=4000, distinct(1)=4000, null(1)=0, distinct(2)=400, null(2)=0, distinct(3)=4000, null(3)=0] + │ ├── columns: x:1(int!null) y:2(int) ordinality:4(int!null) + │ ├── stats: [rows=4000, distinct(1)=4000, null(1)=0, distinct(2)=400, null(2)=0, distinct(4)=4000, null(4)=0] │ ├── key: (1) - │ ├── fd: (1)-->(2,3), (3)-->(1,2) + │ ├── fd: (1)-->(2,4), (4)-->(1,2) │ └── scan a │ ├── columns: x:1(int!null) y:2(int) │ ├── stats: [rows=4000, distinct(1)=4000, null(1)=0, distinct(2)=400, null(2)=0] @@ -66,17 +66,17 @@ norm SELECT 1 x FROM a WITH ORDINALITY ---- project - ├── columns: x:4(int!null) + ├── columns: x:5(int!null) ├── stats: [rows=4000] - ├── fd: ()-->(4) + ├── fd: ()-->(5) ├── ordinality - │ ├── columns: ordinality:3(int!null) + │ ├── columns: ordinality:4(int!null) │ ├── stats: [rows=4000] - │ ├── key: (3) + │ ├── key: (4) │ └── scan a │ └── stats: [rows=4000] └── projections - └── 1 [as=x:4, type=int] + └── 1 [as=x:5, type=int] norm SELECT x FROM (SELECT * FROM a WITH ORDINALITY) WHERE ordinality > 0 AND ordinality <= 10 @@ -87,65 +87,70 @@ project ├── stats: [rows=10] ├── key: (1) └── select - ├── columns: x:1(int!null) ordinality:3(int!null) + ├── columns: x:1(int!null) ordinality:4(int!null) ├── cardinality: [0 - 10] - ├── stats: [rows=10, distinct(3)=10, null(3)=0] + ├── stats: [rows=10, distinct(4)=10, null(4)=0] ├── key: (1) - ├── fd: (1)-->(3), (3)-->(1) + ├── fd: (1)-->(4), (4)-->(1) ├── ordinality - │ ├── columns: x:1(int!null) ordinality:3(int!null) - │ ├── stats: [rows=4000, distinct(1)=4000, null(1)=0, distinct(3)=4000, null(3)=0] + │ ├── columns: x:1(int!null) ordinality:4(int!null) + │ ├── stats: [rows=4000, distinct(1)=4000, null(1)=0, distinct(4)=4000, null(4)=0] │ ├── key: (1) - │ ├── fd: (1)-->(3), (3)-->(1) + │ ├── fd: (1)-->(4), (4)-->(1) │ └── scan a │ ├── columns: x:1(int!null) │ ├── stats: [rows=4000, distinct(1)=4000, null(1)=0] │ └── key: (1) └── filters - └── (ordinality:3 > 0) AND (ordinality:3 <= 10) [type=bool, outer=(3), constraints=(/3: [/1 - /10]; tight)] + └── (ordinality:4 > 0) AND (ordinality:4 <= 10) [type=bool, outer=(4), constraints=(/4: [/1 - /10]; tight)] norm SELECT * FROM (SELECT * FROM a WITH ORDINALITY) WHERE ordinality = 2 ---- select - ├── columns: x:1(int!null) y:2(int) ordinality:3(int!null) + ├── columns: x:1(int!null) y:2(int) ordinality:4(int!null) ├── cardinality: [0 - 1] - ├── stats: [rows=1, distinct(3)=1, null(3)=0] + ├── stats: [rows=1, distinct(4)=1, null(4)=0] ├── key: () - ├── fd: ()-->(1-3) + ├── fd: ()-->(1,2,4) ├── ordinality - │ ├── columns: x:1(int!null) y:2(int) ordinality:3(int!null) - │ ├── stats: [rows=4000, distinct(1)=4000, null(1)=0, distinct(3)=4000, null(3)=0] + │ ├── columns: x:1(int!null) y:2(int) ordinality:4(int!null) + │ ├── stats: [rows=4000, distinct(1)=4000, null(1)=0, distinct(4)=4000, null(4)=0] │ ├── key: (1) - │ ├── fd: (1)-->(2,3), (3)-->(1,2) + │ ├── fd: (1)-->(2,4), (4)-->(1,2) │ └── scan a │ ├── columns: x:1(int!null) y:2(int) │ ├── stats: [rows=4000, distinct(1)=4000, null(1)=0] │ ├── key: (1) │ └── fd: (1)-->(2) └── filters - └── ordinality:3 = 2 [type=bool, outer=(3), constraints=(/3: [/2 - /2]; tight), fd=()-->(3)] + └── ordinality:4 = 2 [type=bool, outer=(4), constraints=(/4: [/2 - /2]; tight), fd=()-->(4)] build SELECT DISTINCT ordinality FROM (SELECT * FROM a WITH ORDINALITY) ---- distinct-on - ├── columns: ordinality:3(int!null) - ├── grouping columns: ordinality:3(int!null) - ├── stats: [rows=4000, distinct(3)=4000, null(3)=0] - ├── key: (3) + ├── columns: ordinality:4(int!null) + ├── grouping columns: ordinality:4(int!null) + ├── stats: [rows=4000, distinct(4)=4000, null(4)=0] + ├── key: (4) └── project - ├── columns: ordinality:3(int!null) - ├── stats: [rows=4000, distinct(3)=4000, null(3)=0] - ├── key: (3) - └── ordinality - ├── columns: x:1(int!null) y:2(int) ordinality:3(int!null) - ├── stats: [rows=4000, distinct(3)=4000, null(3)=0] + ├── columns: ordinality:4(int!null) + ├── stats: [rows=4000, distinct(4)=4000, null(4)=0] + ├── key: (4) + └── project + ├── columns: x:1(int!null) y:2(int) ordinality:4(int!null) + ├── stats: [rows=4000, distinct(4)=4000, null(4)=0] ├── key: (1) - ├── fd: (1)-->(2,3), (3)-->(1,2) - └── scan a - ├── columns: x:1(int!null) y:2(int) - ├── stats: [rows=4000] + ├── fd: (1)-->(2,4), (4)-->(1,2) + └── ordinality + ├── columns: x:1(int!null) y:2(int) crdb_internal_mvcc_timestamp:3(decimal) ordinality:4(int!null) + ├── stats: [rows=4000, distinct(4)=4000, null(4)=0] ├── key: (1) - └── fd: (1)-->(2) + ├── fd: (1)-->(2-4), (4)-->(1-3) + └── scan a + ├── columns: x:1(int!null) y:2(int) crdb_internal_mvcc_timestamp:3(decimal) + ├── stats: [rows=4000] + ├── key: (1) + └── fd: (1)-->(2,3) diff --git a/pkg/sql/opt/memo/testdata/stats/project b/pkg/sql/opt/memo/testdata/stats/project index af1a0bdf38e9..cac4eeafef6c 100644 --- a/pkg/sql/opt/memo/testdata/stats/project +++ b/pkg/sql/opt/memo/testdata/stats/project @@ -39,10 +39,10 @@ project ├── stats: [rows=2000] ├── key: (1) └── scan a - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) ├── stats: [rows=2000] ├── key: (1) - └── fd: (1)-->(2-4), (3,4)~~>(1,2) + └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) build SELECT y, s FROM a @@ -51,105 +51,105 @@ project ├── columns: y:2(int) s:3(string) ├── stats: [rows=2000] └── scan a - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) ├── stats: [rows=2000] ├── key: (1) - └── fd: (1)-->(2-4), (3,4)~~>(1,2) + └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) build SELECT count(*) FROM (SELECT x, y FROM a) GROUP BY x, y ---- project - ├── columns: count:5(int!null) + ├── columns: count:6(int!null) ├── stats: [rows=2000] └── group-by - ├── columns: x:1(int!null) y:2(int) count_rows:5(int!null) + ├── columns: x:1(int!null) y:2(int) count_rows:6(int!null) ├── grouping columns: x:1(int!null) y:2(int) ├── stats: [rows=2000, distinct(1,2)=2000, null(1,2)=0] ├── key: (1) - ├── fd: (1)-->(2,5) + ├── fd: (1)-->(2,6) ├── project │ ├── columns: x:1(int!null) y:2(int) │ ├── stats: [rows=2000, distinct(1,2)=2000, null(1,2)=0] │ ├── key: (1) │ ├── fd: (1)-->(2) │ └── scan a - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) │ ├── stats: [rows=2000, distinct(1,2)=2000, null(1,2)=0] │ ├── key: (1) - │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) + │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) └── aggregations - └── count-rows [as=count_rows:5, type=int] + └── count-rows [as=count_rows:6, type=int] # Test that the stats are calculated correctly for synthesized columns. build SELECT * FROM (SELECT concat(s, y::string) FROM a) AS q(v) WHERE v = 'foo' ---- select - ├── columns: v:5(string!null) + ├── columns: v:6(string!null) ├── immutable - ├── stats: [rows=20, distinct(5)=1, null(5)=0] - ├── fd: ()-->(5) + ├── stats: [rows=20, distinct(6)=1, null(6)=0] + ├── fd: ()-->(6) ├── project - │ ├── columns: concat:5(string) + │ ├── columns: concat:6(string) │ ├── immutable - │ ├── stats: [rows=2000, distinct(5)=100, null(5)=0] + │ ├── stats: [rows=2000, distinct(6)=100, null(6)=0] │ ├── scan a - │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) + │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) │ │ ├── stats: [rows=2000, distinct(2,3)=100, null(2,3)=0] │ │ ├── key: (1) - │ │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) + │ │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) │ └── projections - │ └── concat(s:3, y:2::STRING) [as=concat:5, type=string, outer=(2,3), immutable] + │ └── concat(s:3, y:2::STRING) [as=concat:6, type=string, outer=(2,3), immutable] └── filters - └── concat:5 = 'foo' [type=bool, outer=(5), constraints=(/5: [/'foo' - /'foo']; tight), fd=()-->(5)] + └── concat:6 = 'foo' [type=bool, outer=(6), constraints=(/6: [/'foo' - /'foo']; tight), fd=()-->(6)] # Test that stats for synthesized and non-synthesized columns are combined. build SELECT * FROM (SELECT concat(s, y::string), x FROM a) AS q(v, x) GROUP BY v, x ---- group-by - ├── columns: v:5(string) x:1(int!null) - ├── grouping columns: x:1(int!null) concat:5(string) + ├── columns: v:6(string) x:1(int!null) + ├── grouping columns: x:1(int!null) concat:6(string) ├── immutable - ├── stats: [rows=2000, distinct(1,5)=2000, null(1,5)=0] + ├── stats: [rows=2000, distinct(1,6)=2000, null(1,6)=0] ├── key: (1) - ├── fd: (1)-->(5) + ├── fd: (1)-->(6) └── project - ├── columns: concat:5(string) x:1(int!null) + ├── columns: concat:6(string) x:1(int!null) ├── immutable - ├── stats: [rows=2000, distinct(1,5)=2000, null(1,5)=0] + ├── stats: [rows=2000, distinct(1,6)=2000, null(1,6)=0] ├── key: (1) - ├── fd: (1)-->(5) + ├── fd: (1)-->(6) ├── scan a - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) │ ├── stats: [rows=2000, distinct(1-3)=2000, null(1-3)=0] │ ├── key: (1) - │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) + │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) └── projections - └── concat(s:3, y:2::STRING) [as=concat:5, type=string, outer=(2,3), immutable] + └── concat(s:3, y:2::STRING) [as=concat:6, type=string, outer=(2,3), immutable] # No available stats for column y. build SELECT * FROM (SELECT y + 3 AS v FROM a) WHERE v >= 1 AND v <= 100 ---- select - ├── columns: v:5(int!null) + ├── columns: v:6(int!null) ├── immutable - ├── stats: [rows=1000, distinct(5)=100, null(5)=0] + ├── stats: [rows=1000, distinct(6)=100, null(6)=0] ├── project - │ ├── columns: v:5(int) + │ ├── columns: v:6(int) │ ├── immutable - │ ├── stats: [rows=2000, distinct(5)=200, null(5)=0] + │ ├── stats: [rows=2000, distinct(6)=200, null(6)=0] │ ├── scan a - │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) + │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) │ │ ├── stats: [rows=2000, distinct(2)=200, null(2)=20] │ │ ├── key: (1) - │ │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) + │ │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) │ └── projections - │ └── y:2 + 3 [as=v:5, type=int, outer=(2), immutable] + │ └── y:2 + 3 [as=v:6, type=int, outer=(2), immutable] └── filters - └── (v:5 >= 1) AND (v:5 <= 100) [type=bool, outer=(5), constraints=(/5: [/1 - /100]; tight)] + └── (v:6 >= 1) AND (v:6 <= 100) [type=bool, outer=(6), constraints=(/6: [/1 - /100]; tight)] exec-ddl CREATE TABLE kuv (k INT PRIMARY KEY, u FLOAT, v STRING) @@ -176,35 +176,40 @@ ALTER TABLE kuv INJECT STATISTICS '[ build SELECT * FROM a WHERE EXISTS (SELECT s < v FROM kuv GROUP BY s < v) ---- -select +project ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) ├── stats: [rows=666.666667] ├── key: (1) ├── fd: (1)-->(2-4), (3,4)~~>(1,2) - ├── scan a - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) - │ ├── stats: [rows=2000, distinct(1)=2000, null(1)=0, distinct(4)=200, null(4)=0] - │ ├── key: (1) - │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) - └── filters - └── exists [type=bool, outer=(3), correlated-subquery] - └── group-by - ├── columns: column8:8(bool) - ├── grouping columns: column8:8(bool) - ├── outer: (3) - ├── stats: [rows=10, distinct(8)=10, null(8)=0] - ├── key: (8) - └── project - ├── columns: column8:8(bool) + └── select + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) a.crdb_internal_mvcc_timestamp:5(decimal) + ├── stats: [rows=666.666667] + ├── key: (1) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + ├── scan a + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) a.crdb_internal_mvcc_timestamp:5(decimal) + │ ├── stats: [rows=2000, distinct(1)=2000, null(1)=0, distinct(4)=200, null(4)=0] + │ ├── key: (1) + │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + └── filters + └── exists [type=bool, outer=(3), correlated-subquery] + └── group-by + ├── columns: column10:10(bool) + ├── grouping columns: column10:10(bool) ├── outer: (3) - ├── stats: [rows=2000, distinct(8)=10, null(8)=0] - ├── scan kuv - │ ├── columns: k:5(int!null) u:6(float) v:7(string) - │ ├── stats: [rows=2000, distinct(7)=10, null(7)=0] - │ ├── key: (5) - │ └── fd: (5)-->(6,7) - └── projections - └── s:3 < v:7 [as=column8:8, type=bool, outer=(3,7)] + ├── stats: [rows=10, distinct(10)=10, null(10)=0] + ├── key: (10) + └── project + ├── columns: column10:10(bool) + ├── outer: (3) + ├── stats: [rows=2000, distinct(10)=10, null(10)=0] + ├── scan kuv + │ ├── columns: k:6(int!null) u:7(float) v:8(string) kuv.crdb_internal_mvcc_timestamp:9(decimal) + │ ├── stats: [rows=2000, distinct(8)=10, null(8)=0] + │ ├── key: (6) + │ └── fd: (6)-->(7-9) + └── projections + └── s:3 < v:8 [as=column10:10, type=bool, outer=(3,8)] # Bump up null counts. exec-ddl @@ -239,46 +244,46 @@ project ├── columns: y:2(int) s:3(string) ├── stats: [rows=2000, distinct(2)=500, null(2)=1000, distinct(3)=500, null(3)=500, distinct(2,3)=2000, null(2,3)=250] └── scan a - ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) ├── stats: [rows=2000, distinct(2)=500, null(2)=1000, distinct(3)=500, null(3)=500, distinct(2,3)=2000, null(2,3)=250] ├── key: (1) - └── fd: (1)-->(2-4), (3,4)~~>(1,2) + └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) # Test that the stats are calculated correctly for synthesized columns. build SELECT * FROM (SELECT concat(s, y::string) FROM a) AS q(v) WHERE v = 'foo' ---- select - ├── columns: v:5(string!null) + ├── columns: v:6(string!null) ├── immutable - ├── stats: [rows=1, distinct(5)=1, null(5)=0] - ├── fd: ()-->(5) + ├── stats: [rows=1, distinct(6)=1, null(6)=0] + ├── fd: ()-->(6) ├── project - │ ├── columns: concat:5(string) + │ ├── columns: concat:6(string) │ ├── immutable - │ ├── stats: [rows=2000, distinct(5)=2000, null(5)=0] + │ ├── stats: [rows=2000, distinct(6)=2000, null(6)=0] │ ├── scan a - │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) + │ │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) │ │ ├── stats: [rows=2000, distinct(2,3)=2000, null(2,3)=250] │ │ ├── key: (1) - │ │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) + │ │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) │ └── projections - │ └── concat(s:3, y:2::STRING) [as=concat:5, type=string, outer=(2,3), immutable] + │ └── concat(s:3, y:2::STRING) [as=concat:6, type=string, outer=(2,3), immutable] └── filters - └── concat:5 = 'foo' [type=bool, outer=(5), constraints=(/5: [/'foo' - /'foo']; tight), fd=()-->(5)] + └── concat:6 = 'foo' [type=bool, outer=(6), constraints=(/6: [/'foo' - /'foo']; tight), fd=()-->(6)] -build colstat=5 colstat=6 colstat=(5,6) +build colstat=6 colstat=7 colstat=(6,7) SELECT NULL, NULLIF(x,y) FROM a ---- project - ├── columns: "?column?":5(unknown) nullif:6(int) - ├── stats: [rows=2000, distinct(5)=1, null(5)=2000, distinct(6)=2000, null(6)=0, distinct(5,6)=2000, null(5,6)=0] - ├── fd: ()-->(5) + ├── columns: "?column?":6(unknown) nullif:7(int) + ├── stats: [rows=2000, distinct(6)=1, null(6)=2000, distinct(7)=2000, null(7)=0, distinct(6,7)=2000, null(6,7)=0] + ├── fd: ()-->(6) ├── scan a - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) crdb_internal_mvcc_timestamp:5(decimal) │ ├── stats: [rows=2000, distinct(1,2)=2000, null(1,2)=0] │ ├── key: (1) - │ └── fd: (1)-->(2-4), (3,4)~~>(1,2) + │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) └── projections - ├── NULL [as="?column?":5, type=unknown] - └── CASE x:1 WHEN y:2 THEN CAST(NULL AS INT8) ELSE x:1 END [as=nullif:6, type=int, outer=(1,2)] + ├── NULL [as="?column?":6, type=unknown] + └── CASE x:1 WHEN y:2 THEN CAST(NULL AS INT8) ELSE x:1 END [as=nullif:7, type=int, outer=(1,2)] diff --git a/pkg/sql/opt/memo/testdata/stats/scan b/pkg/sql/opt/memo/testdata/stats/scan index 61f920407be5..bae2c05f3f43 100644 --- a/pkg/sql/opt/memo/testdata/stats/scan +++ b/pkg/sql/opt/memo/testdata/stats/scan @@ -16,29 +16,39 @@ ALTER TABLE a INJECT STATISTICS '[ build SELECT * FROM a ---- -scan a +project ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) b:5(bool) ├── stats: [rows=2000] ├── key: (1) - └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + └── scan a + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) b:5(bool) crdb_internal_mvcc_timestamp:6(decimal) + ├── stats: [rows=2000] + ├── key: (1) + └── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) # Check that boolean columns have distinct count 2 when there are no stats # available. build SELECT * FROM a WHERE b ---- -select +project ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) b:5(bool!null) - ├── stats: [rows=990, distinct(5)=1, null(5)=0] + ├── stats: [rows=990] ├── key: (1) ├── fd: ()-->(5), (1)-->(2-4), (3,4)~~>(1,2) - ├── scan a - │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) b:5(bool) - │ ├── stats: [rows=2000, distinct(1)=2000, null(1)=0, distinct(4)=200, null(4)=0, distinct(5)=3, null(5)=20] - │ ├── key: (1) - │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) - └── filters - └── b:5 [type=bool, outer=(5), constraints=(/5: [/true - /true]; tight), fd=()-->(5)] + └── select + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) b:5(bool!null) crdb_internal_mvcc_timestamp:6(decimal) + ├── stats: [rows=990, distinct(5)=1, null(5)=0] + ├── key: (1) + ├── fd: ()-->(5), (1)-->(2-4,6), (3,4)~~>(1,2,6) + ├── scan a + │ ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) b:5(bool) crdb_internal_mvcc_timestamp:6(decimal) + │ ├── stats: [rows=2000, distinct(1)=2000, null(1)=0, distinct(4)=200, null(4)=0, distinct(5)=3, null(5)=20] + │ ├── key: (1) + │ └── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) + └── filters + └── b:5 [type=bool, outer=(5), constraints=(/5: [/true - /true]; tight), fd=()-->(5)] exec-ddl ALTER TABLE a INJECT STATISTICS '[ @@ -84,11 +94,16 @@ ALTER TABLE a INJECT STATISTICS '[ build SELECT * FROM a ---- -scan a +project ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) b:5(bool) ├── stats: [rows=3000] ├── key: (1) - └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + └── scan a + ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) b:5(bool) crdb_internal_mvcc_timestamp:6(decimal) + ├── stats: [rows=3000] + ├── key: (1) + └── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) # Test constrained scan. opt @@ -118,13 +133,13 @@ opt SELECT count(*), y, x FROM a WHERE x > 0 AND x <= 100 GROUP BY x, y ---- group-by - ├── columns: count:6(int!null) y:2(int) x:1(int!null) + ├── columns: count:7(int!null) y:2(int) x:1(int!null) ├── grouping columns: x:1(int!null) ├── internal-ordering: +1 ├── cardinality: [0 - 100] ├── stats: [rows=100, distinct(1)=100, null(1)=0] ├── key: (1) - ├── fd: (1)-->(2,6) + ├── fd: (1)-->(2,7) ├── scan a │ ├── columns: x:1(int!null) y:2(int) │ ├── constraint: /1: [/1 - /100] @@ -134,7 +149,7 @@ group-by │ ├── fd: (1)-->(2) │ └── ordering: +1 └── aggregations - ├── count-rows [as=count_rows:6, type=int] + ├── count-rows [as=count_rows:7, type=int] └── const-agg [as=y:2, type=int, outer=(2)] └── y:2 [type=int] @@ -510,36 +525,36 @@ WHERE subq.col1; ---- project - ├── columns: "?column?":26(int!null) + ├── columns: "?column?":28(int!null) ├── immutable ├── stats: [rows=1] - ├── fd: ()-->(26) + ├── fd: ()-->(28) ├── select - │ ├── columns: col1:25(bool!null) + │ ├── columns: col1:27(bool!null) │ ├── immutable - │ ├── stats: [rows=1, distinct(25)=1, null(25)=0] - │ ├── fd: ()-->(25) + │ ├── stats: [rows=1, distinct(27)=1, null(27)=0] + │ ├── fd: ()-->(27) │ ├── project - │ │ ├── columns: col1:25(bool) + │ │ ├── columns: col1:27(bool) │ │ ├── immutable - │ │ ├── stats: [rows=333333.333, distinct(25)=333333.333, null(25)=0] + │ │ ├── stats: [rows=333333.333, distinct(27)=333333.333, null(27)=0] │ │ ├── inner-join (cross) - │ │ │ ├── columns: tab0.e:5(varchar) tab0.f:6("char") tab0.h:8(varchar) tab0.j:10(float!null) tab1.e:16(varchar) tab1.f:17("char") tab1.j:21(float!null) - │ │ │ ├── stats: [rows=333333.333, distinct(5,6,8,16,17)=333333.333, null(5,6,8,16,17)=3.33333333e-05] + │ │ │ ├── columns: tab0.e:5(varchar) tab0.f:6("char") tab0.h:8(varchar) tab0.j:10(float!null) tab1.e:17(varchar) tab1.f:18("char") tab1.j:22(float!null) + │ │ │ ├── stats: [rows=333333.333, distinct(5,6,8,17,18)=333333.333, null(5,6,8,17,18)=3.33333333e-05] │ │ │ ├── scan tab0 │ │ │ │ ├── columns: tab0.e:5(varchar) tab0.f:6("char") tab0.h:8(varchar) tab0.j:10(float!null) │ │ │ │ └── stats: [rows=1000, distinct(5,6,8)=1000, null(5,6,8)=0.001] │ │ │ ├── scan tab1 - │ │ │ │ ├── columns: tab1.e:16(varchar) tab1.f:17("char") tab1.j:21(float!null) - │ │ │ │ └── stats: [rows=1000, distinct(16,17)=1000, null(16,17)=0.1] + │ │ │ │ ├── columns: tab1.e:17(varchar) tab1.f:18("char") tab1.j:22(float!null) + │ │ │ │ └── stats: [rows=1000, distinct(17,18)=1000, null(17,18)=0.1] │ │ │ └── filters - │ │ │ └── tab0.j:10 IN (tab1.j:21,) [type=bool, outer=(10,21)] + │ │ │ └── tab0.j:10 IN (tab1.j:22,) [type=bool, outer=(10,22)] │ │ └── projections - │ │ └── CASE WHEN ilike_escape(regexp_replace(tab0.h:8, tab1.e:16, tab0.f:6, tab0.e:5::STRING), tab1.f:17, '') THEN true ELSE false END [as=col1:25, type=bool, outer=(5,6,8,16,17), immutable] + │ │ └── CASE WHEN ilike_escape(regexp_replace(tab0.h:8, tab1.e:17, tab0.f:6, tab0.e:5::STRING), tab1.f:18, '') THEN true ELSE false END [as=col1:27, type=bool, outer=(5,6,8,17,18), immutable] │ └── filters - │ └── col1:25 [type=bool, outer=(25), constraints=(/25: [/true - /true]; tight), fd=()-->(25)] + │ └── col1:27 [type=bool, outer=(27), constraints=(/27: [/true - /true]; tight), fd=()-->(27)] └── projections - └── 1 [as="?column?":26, type=int] + └── 1 [as="?column?":28, type=int] # --------------------- # Tests with Histograms @@ -1020,10 +1035,10 @@ opt SELECT a, b::string FROM t47742 WHERE b = true ---- project - ├── columns: a:1(int) b:4(string!null) + ├── columns: a:1(int) b:5(string!null) ├── immutable ├── stats: [rows=2640.64496] - ├── fd: ()-->(4) + ├── fd: ()-->(5) ├── index-join t47742 │ ├── columns: a:1(int) t47742.b:2(bool!null) │ ├── stats: [rows=2640.64496, distinct(2)=2.00246926, null(2)=0] @@ -1039,7 +1054,7 @@ project │ ├── key: (3) │ └── fd: ()-->(2) └── projections - └── t47742.b:2::STRING [as=b:4, type=string, outer=(2), immutable] + └── t47742.b:2::STRING [as=b:5, type=string, outer=(2), immutable] # Multi-column stats tests. exec-ddl diff --git a/pkg/sql/opt/memo/testdata/stats/select b/pkg/sql/opt/memo/testdata/stats/select index 8723cf2c1778..f1b97f05f7df 100644 --- a/pkg/sql/opt/memo/testdata/stats/select +++ b/pkg/sql/opt/memo/testdata/stats/select @@ -148,14 +148,14 @@ norm SELECT sum(x) FROM b WHERE x > 1000 AND x <= 2000 GROUP BY z ---- project - ├── columns: sum:4(decimal!null) + ├── columns: sum:5(decimal!null) ├── stats: [rows=100] └── group-by - ├── columns: z:2(int!null) sum:4(decimal!null) + ├── columns: z:2(int!null) sum:5(decimal!null) ├── grouping columns: z:2(int!null) ├── stats: [rows=100, distinct(2)=100, null(2)=0] ├── key: (2) - ├── fd: (2)-->(4) + ├── fd: (2)-->(5) ├── select │ ├── columns: x:1(int!null) z:2(int!null) │ ├── stats: [rows=2000, distinct(1)=1000, null(1)=0, distinct(2)=100, null(2)=0] @@ -165,7 +165,7 @@ project │ └── filters │ └── (x:1 > 1000) AND (x:1 <= 2000) [type=bool, outer=(1), constraints=(/1: [/1001 - /2000]; tight)] └── aggregations - └── sum [as=sum:4, type=decimal, outer=(1)] + └── sum [as=sum:5, type=decimal, outer=(1)] └── x:1 [type=int] # Regression: statistics builder panics when end key is NULL when it's trying @@ -234,14 +234,14 @@ opt SELECT count(*) FROM (SELECT * FROM tab0 WHERE col3 = 10) GROUP BY col0 ---- project - ├── columns: count:8(int!null) + ├── columns: count:9(int!null) ├── stats: [rows=0.999973439] └── group-by - ├── columns: col0:2(int) count_rows:8(int!null) + ├── columns: col0:2(int) count_rows:9(int!null) ├── grouping columns: col0:2(int) ├── stats: [rows=0.999973439, distinct(2)=0.999973439, null(2)=0.999973439] ├── key: (2) - ├── fd: (2)-->(8) + ├── fd: (2)-->(9) ├── select │ ├── columns: col0:2(int) col3:5(int!null) │ ├── stats: [rows=10, distinct(2)=0.999973439, null(2)=10, distinct(5)=1, null(5)=0] @@ -252,7 +252,7 @@ project │ └── filters │ └── col3:5 = 10 [type=bool, outer=(5), constraints=(/5: [/10 - /10]; tight), fd=()-->(5)] └── aggregations - └── count-rows [as=count_rows:8, type=int] + └── count-rows [as=count_rows:9, type=int] exec-ddl @@ -406,36 +406,36 @@ SELECT * FROM (SELECT * FROM customers, order_history WHERE id = customer_id) WHERE name='andy' ---- select - ├── columns: id:1(int!null) name:2(string!null) state:3(string) order_id:4(int) item_id:5(int) customer_id:6(int!null) year:7(int) + ├── columns: id:1(int!null) name:2(string!null) state:3(string) order_id:5(int) item_id:6(int) customer_id:7(int!null) year:8(int) ├── stats: [rows=2.29713221, distinct(2)=1, null(2)=0] - ├── fd: ()-->(2), (1)-->(3), (1)==(6), (6)==(1) + ├── fd: ()-->(2), (1)-->(3), (1)==(7), (7)==(1) ├── project - │ ├── columns: id:1(int!null) name:2(string) state:3(string) order_id:4(int) item_id:5(int) customer_id:6(int!null) year:7(int) - │ ├── stats: [rows=990, distinct(1)=99, null(1)=0, distinct(2)=430.972148, null(2)=0, distinct(6)=99, null(6)=0] - │ ├── fd: (1)-->(2,3), (1)==(6), (6)==(1) + │ ├── columns: id:1(int!null) name:2(string) state:3(string) order_id:5(int) item_id:6(int) customer_id:7(int!null) year:8(int) + │ ├── stats: [rows=990, distinct(1)=99, null(1)=0, distinct(2)=430.972148, null(2)=0, distinct(7)=99, null(7)=0] + │ ├── fd: (1)-->(2,3), (1)==(7), (7)==(1) │ └── select - │ ├── columns: id:1(int!null) name:2(string) state:3(string) order_id:4(int) item_id:5(int) customer_id:6(int!null) year:7(int) rowid:8(int!null) - │ ├── stats: [rows=990, distinct(1)=99, null(1)=0, distinct(2)=430.972148, null(2)=0, distinct(6)=99, null(6)=0] - │ ├── key: (8) - │ ├── fd: (1)-->(2,3), (8)-->(4-7), (1)==(6), (6)==(1) + │ ├── columns: id:1(int!null) name:2(string) state:3(string) customers.crdb_internal_mvcc_timestamp:4(decimal) order_id:5(int) item_id:6(int) customer_id:7(int!null) year:8(int) rowid:9(int!null) order_history.crdb_internal_mvcc_timestamp:10(decimal) + │ ├── stats: [rows=990, distinct(1)=99, null(1)=0, distinct(2)=430.972148, null(2)=0, distinct(7)=99, null(7)=0] + │ ├── key: (9) + │ ├── fd: (1)-->(2-4), (9)-->(5-8,10), (1)==(7), (7)==(1) │ ├── inner-join (cross) - │ │ ├── columns: id:1(int!null) name:2(string) state:3(string) order_id:4(int) item_id:5(int) customer_id:6(int) year:7(int) rowid:8(int!null) - │ │ ├── stats: [rows=10000000, distinct(1)=10000, null(1)=0, distinct(2)=500, null(2)=0, distinct(6)=100, null(6)=100000, distinct(8)=1000, null(8)=0] - │ │ ├── key: (1,8) - │ │ ├── fd: (1)-->(2,3), (8)-->(4-7) + │ │ ├── columns: id:1(int!null) name:2(string) state:3(string) customers.crdb_internal_mvcc_timestamp:4(decimal) order_id:5(int) item_id:6(int) customer_id:7(int) year:8(int) rowid:9(int!null) order_history.crdb_internal_mvcc_timestamp:10(decimal) + │ │ ├── stats: [rows=10000000, distinct(1)=10000, null(1)=0, distinct(2)=500, null(2)=0, distinct(7)=100, null(7)=100000, distinct(9)=1000, null(9)=0] + │ │ ├── key: (1,9) + │ │ ├── fd: (1)-->(2-4), (9)-->(5-8,10) │ │ ├── scan customers - │ │ │ ├── columns: id:1(int!null) name:2(string) state:3(string) + │ │ │ ├── columns: id:1(int!null) name:2(string) state:3(string) customers.crdb_internal_mvcc_timestamp:4(decimal) │ │ │ ├── stats: [rows=10000, distinct(1)=10000, null(1)=0, distinct(2)=500, null(2)=0] │ │ │ ├── key: (1) - │ │ │ └── fd: (1)-->(2,3) + │ │ │ └── fd: (1)-->(2-4) │ │ ├── scan order_history - │ │ │ ├── columns: order_id:4(int) item_id:5(int) customer_id:6(int) year:7(int) rowid:8(int!null) - │ │ │ ├── stats: [rows=1000, distinct(6)=100, null(6)=10, distinct(8)=1000, null(8)=0] - │ │ │ ├── key: (8) - │ │ │ └── fd: (8)-->(4-7) + │ │ │ ├── columns: order_id:5(int) item_id:6(int) customer_id:7(int) year:8(int) rowid:9(int!null) order_history.crdb_internal_mvcc_timestamp:10(decimal) + │ │ │ ├── stats: [rows=1000, distinct(7)=100, null(7)=10, distinct(9)=1000, null(9)=0] + │ │ │ ├── key: (9) + │ │ │ └── fd: (9)-->(5-8,10) │ │ └── filters (true) │ └── filters - │ └── id:1 = customer_id:6 [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ └── id:1 = customer_id:7 [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] └── filters └── name:2 = 'andy' [type=bool, outer=(2), constraints=(/2: [/'andy' - /'andy']; tight), fd=()-->(2)] @@ -454,36 +454,36 @@ SELECT * FROM (SELECT * FROM customers, order_history WHERE id = customer_id) WHERE id = 1 AND name='andy' ---- select - ├── columns: id:1(int!null) name:2(string!null) state:3(string) order_id:4(int) item_id:5(int) customer_id:6(int!null) year:7(int) + ├── columns: id:1(int!null) name:2(string!null) state:3(string) order_id:5(int) item_id:6(int) customer_id:7(int!null) year:8(int) ├── stats: [rows=10, distinct(1)=1, null(1)=0, distinct(2)=1, null(2)=0] - ├── fd: ()-->(1-3,6), (1)==(6), (6)==(1) + ├── fd: ()-->(1-3,7), (1)==(7), (7)==(1) ├── project - │ ├── columns: id:1(int!null) name:2(string) state:3(string) order_id:4(int) item_id:5(int) customer_id:6(int!null) year:7(int) - │ ├── stats: [rows=990, distinct(1)=99, null(1)=0, distinct(2)=430.972148, null(2)=0, distinct(6)=99, null(6)=0] - │ ├── fd: (1)-->(2,3), (1)==(6), (6)==(1) + │ ├── columns: id:1(int!null) name:2(string) state:3(string) order_id:5(int) item_id:6(int) customer_id:7(int!null) year:8(int) + │ ├── stats: [rows=990, distinct(1)=99, null(1)=0, distinct(2)=430.972148, null(2)=0, distinct(7)=99, null(7)=0] + │ ├── fd: (1)-->(2,3), (1)==(7), (7)==(1) │ └── select - │ ├── columns: id:1(int!null) name:2(string) state:3(string) order_id:4(int) item_id:5(int) customer_id:6(int!null) year:7(int) rowid:8(int!null) - │ ├── stats: [rows=990, distinct(1)=99, null(1)=0, distinct(2)=430.972148, null(2)=0, distinct(6)=99, null(6)=0] - │ ├── key: (8) - │ ├── fd: (1)-->(2,3), (8)-->(4-7), (1)==(6), (6)==(1) + │ ├── columns: id:1(int!null) name:2(string) state:3(string) customers.crdb_internal_mvcc_timestamp:4(decimal) order_id:5(int) item_id:6(int) customer_id:7(int!null) year:8(int) rowid:9(int!null) order_history.crdb_internal_mvcc_timestamp:10(decimal) + │ ├── stats: [rows=990, distinct(1)=99, null(1)=0, distinct(2)=430.972148, null(2)=0, distinct(7)=99, null(7)=0] + │ ├── key: (9) + │ ├── fd: (1)-->(2-4), (9)-->(5-8,10), (1)==(7), (7)==(1) │ ├── inner-join (cross) - │ │ ├── columns: id:1(int!null) name:2(string) state:3(string) order_id:4(int) item_id:5(int) customer_id:6(int) year:7(int) rowid:8(int!null) - │ │ ├── stats: [rows=10000000, distinct(1)=10000, null(1)=0, distinct(2)=500, null(2)=0, distinct(6)=100, null(6)=100000, distinct(8)=1000, null(8)=0] - │ │ ├── key: (1,8) - │ │ ├── fd: (1)-->(2,3), (8)-->(4-7) + │ │ ├── columns: id:1(int!null) name:2(string) state:3(string) customers.crdb_internal_mvcc_timestamp:4(decimal) order_id:5(int) item_id:6(int) customer_id:7(int) year:8(int) rowid:9(int!null) order_history.crdb_internal_mvcc_timestamp:10(decimal) + │ │ ├── stats: [rows=10000000, distinct(1)=10000, null(1)=0, distinct(2)=500, null(2)=0, distinct(7)=100, null(7)=100000, distinct(9)=1000, null(9)=0] + │ │ ├── key: (1,9) + │ │ ├── fd: (1)-->(2-4), (9)-->(5-8,10) │ │ ├── scan customers - │ │ │ ├── columns: id:1(int!null) name:2(string) state:3(string) + │ │ │ ├── columns: id:1(int!null) name:2(string) state:3(string) customers.crdb_internal_mvcc_timestamp:4(decimal) │ │ │ ├── stats: [rows=10000, distinct(1)=10000, null(1)=0, distinct(2)=500, null(2)=0] │ │ │ ├── key: (1) - │ │ │ └── fd: (1)-->(2,3) + │ │ │ └── fd: (1)-->(2-4) │ │ ├── scan order_history - │ │ │ ├── columns: order_id:4(int) item_id:5(int) customer_id:6(int) year:7(int) rowid:8(int!null) - │ │ │ ├── stats: [rows=1000, distinct(6)=100, null(6)=10, distinct(8)=1000, null(8)=0] - │ │ │ ├── key: (8) - │ │ │ └── fd: (8)-->(4-7) + │ │ │ ├── columns: order_id:5(int) item_id:6(int) customer_id:7(int) year:8(int) rowid:9(int!null) order_history.crdb_internal_mvcc_timestamp:10(decimal) + │ │ │ ├── stats: [rows=1000, distinct(7)=100, null(7)=10, distinct(9)=1000, null(9)=0] + │ │ │ ├── key: (9) + │ │ │ └── fd: (9)-->(5-8,10) │ │ └── filters (true) │ └── filters - │ └── id:1 = customer_id:6 [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ └── id:1 = customer_id:7 [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] └── filters └── (id:1 = 1) AND (name:2 = 'andy') [type=bool, outer=(1,2), constraints=(/1: [/1 - /1]; /2: [/'andy' - /'andy']; tight), fd=()-->(1,2)] @@ -1088,36 +1088,36 @@ SELECT * FROM (SELECT * FROM customers, order_history WHERE id = customer_id) WHERE name='andy' ---- select - ├── columns: id:1(int!null) name:2(string!null) state:3(string) order_id:4(int) item_id:5(int) customer_id:6(int!null) year:7(int) + ├── columns: id:1(int!null) name:2(string!null) state:3(string) order_id:5(int) item_id:6(int) customer_id:7(int!null) year:8(int) ├── stats: [rows=1.84197978, distinct(2)=1, null(2)=0] - ├── fd: ()-->(2), (1)-->(3), (1)==(6), (6)==(1) + ├── fd: ()-->(2), (1)-->(3), (1)==(7), (7)==(1) ├── project - │ ├── columns: id:1(int!null) name:2(string) state:3(string) order_id:4(int) item_id:5(int) customer_id:6(int!null) year:7(int) - │ ├── stats: [rows=990, distinct(1)=99, null(1)=0, distinct(2)=430.972148, null(2)=198, distinct(6)=99, null(6)=0] - │ ├── fd: (1)-->(2,3), (1)==(6), (6)==(1) + │ ├── columns: id:1(int!null) name:2(string) state:3(string) order_id:5(int) item_id:6(int) customer_id:7(int!null) year:8(int) + │ ├── stats: [rows=990, distinct(1)=99, null(1)=0, distinct(2)=430.972148, null(2)=198, distinct(7)=99, null(7)=0] + │ ├── fd: (1)-->(2,3), (1)==(7), (7)==(1) │ └── select - │ ├── columns: id:1(int!null) name:2(string) state:3(string) order_id:4(int) item_id:5(int) customer_id:6(int!null) year:7(int) rowid:8(int!null) - │ ├── stats: [rows=990, distinct(1)=99, null(1)=0, distinct(2)=430.972148, null(2)=198, distinct(6)=99, null(6)=0] - │ ├── key: (8) - │ ├── fd: (1)-->(2,3), (8)-->(4-7), (1)==(6), (6)==(1) + │ ├── columns: id:1(int!null) name:2(string) state:3(string) customers.crdb_internal_mvcc_timestamp:4(decimal) order_id:5(int) item_id:6(int) customer_id:7(int!null) year:8(int) rowid:9(int!null) order_history.crdb_internal_mvcc_timestamp:10(decimal) + │ ├── stats: [rows=990, distinct(1)=99, null(1)=0, distinct(2)=430.972148, null(2)=198, distinct(7)=99, null(7)=0] + │ ├── key: (9) + │ ├── fd: (1)-->(2-4), (9)-->(5-8,10), (1)==(7), (7)==(1) │ ├── inner-join (cross) - │ │ ├── columns: id:1(int!null) name:2(string) state:3(string) order_id:4(int) item_id:5(int) customer_id:6(int) year:7(int) rowid:8(int!null) - │ │ ├── stats: [rows=10000000, distinct(1)=10000, null(1)=0, distinct(2)=500, null(2)=2000000, distinct(6)=100, null(6)=100000, distinct(8)=1000, null(8)=0] - │ │ ├── key: (1,8) - │ │ ├── fd: (1)-->(2,3), (8)-->(4-7) + │ │ ├── columns: id:1(int!null) name:2(string) state:3(string) customers.crdb_internal_mvcc_timestamp:4(decimal) order_id:5(int) item_id:6(int) customer_id:7(int) year:8(int) rowid:9(int!null) order_history.crdb_internal_mvcc_timestamp:10(decimal) + │ │ ├── stats: [rows=10000000, distinct(1)=10000, null(1)=0, distinct(2)=500, null(2)=2000000, distinct(7)=100, null(7)=100000, distinct(9)=1000, null(9)=0] + │ │ ├── key: (1,9) + │ │ ├── fd: (1)-->(2-4), (9)-->(5-8,10) │ │ ├── scan customers - │ │ │ ├── columns: id:1(int!null) name:2(string) state:3(string) + │ │ │ ├── columns: id:1(int!null) name:2(string) state:3(string) customers.crdb_internal_mvcc_timestamp:4(decimal) │ │ │ ├── stats: [rows=10000, distinct(1)=10000, null(1)=0, distinct(2)=500, null(2)=2000] │ │ │ ├── key: (1) - │ │ │ └── fd: (1)-->(2,3) + │ │ │ └── fd: (1)-->(2-4) │ │ ├── scan order_history - │ │ │ ├── columns: order_id:4(int) item_id:5(int) customer_id:6(int) year:7(int) rowid:8(int!null) - │ │ │ ├── stats: [rows=1000, distinct(6)=100, null(6)=10, distinct(8)=1000, null(8)=0] - │ │ │ ├── key: (8) - │ │ │ └── fd: (8)-->(4-7) + │ │ │ ├── columns: order_id:5(int) item_id:6(int) customer_id:7(int) year:8(int) rowid:9(int!null) order_history.crdb_internal_mvcc_timestamp:10(decimal) + │ │ │ ├── stats: [rows=1000, distinct(7)=100, null(7)=10, distinct(9)=1000, null(9)=0] + │ │ │ ├── key: (9) + │ │ │ └── fd: (9)-->(5-8,10) │ │ └── filters (true) │ └── filters - │ └── id:1 = customer_id:6 [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ └── id:1 = customer_id:7 [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] └── filters └── name:2 = 'andy' [type=bool, outer=(2), constraints=(/2: [/'andy' - /'andy']; tight), fd=()-->(2)] @@ -1136,36 +1136,36 @@ SELECT * FROM (SELECT * FROM customers, order_history WHERE id = customer_id) WHERE id = 1 AND name='andy' ---- select - ├── columns: id:1(int!null) name:2(string!null) state:3(string) order_id:4(int) item_id:5(int) customer_id:6(int!null) year:7(int) + ├── columns: id:1(int!null) name:2(string!null) state:3(string) order_id:5(int) item_id:6(int) customer_id:7(int!null) year:8(int) ├── stats: [rows=8, distinct(1)=1, null(1)=0, distinct(2)=1, null(2)=0] - ├── fd: ()-->(1-3,6), (1)==(6), (6)==(1) + ├── fd: ()-->(1-3,7), (1)==(7), (7)==(1) ├── project - │ ├── columns: id:1(int!null) name:2(string) state:3(string) order_id:4(int) item_id:5(int) customer_id:6(int!null) year:7(int) - │ ├── stats: [rows=990, distinct(1)=99, null(1)=0, distinct(2)=430.972148, null(2)=198, distinct(6)=99, null(6)=0] - │ ├── fd: (1)-->(2,3), (1)==(6), (6)==(1) + │ ├── columns: id:1(int!null) name:2(string) state:3(string) order_id:5(int) item_id:6(int) customer_id:7(int!null) year:8(int) + │ ├── stats: [rows=990, distinct(1)=99, null(1)=0, distinct(2)=430.972148, null(2)=198, distinct(7)=99, null(7)=0] + │ ├── fd: (1)-->(2,3), (1)==(7), (7)==(1) │ └── select - │ ├── columns: id:1(int!null) name:2(string) state:3(string) order_id:4(int) item_id:5(int) customer_id:6(int!null) year:7(int) rowid:8(int!null) - │ ├── stats: [rows=990, distinct(1)=99, null(1)=0, distinct(2)=430.972148, null(2)=198, distinct(6)=99, null(6)=0] - │ ├── key: (8) - │ ├── fd: (1)-->(2,3), (8)-->(4-7), (1)==(6), (6)==(1) + │ ├── columns: id:1(int!null) name:2(string) state:3(string) customers.crdb_internal_mvcc_timestamp:4(decimal) order_id:5(int) item_id:6(int) customer_id:7(int!null) year:8(int) rowid:9(int!null) order_history.crdb_internal_mvcc_timestamp:10(decimal) + │ ├── stats: [rows=990, distinct(1)=99, null(1)=0, distinct(2)=430.972148, null(2)=198, distinct(7)=99, null(7)=0] + │ ├── key: (9) + │ ├── fd: (1)-->(2-4), (9)-->(5-8,10), (1)==(7), (7)==(1) │ ├── inner-join (cross) - │ │ ├── columns: id:1(int!null) name:2(string) state:3(string) order_id:4(int) item_id:5(int) customer_id:6(int) year:7(int) rowid:8(int!null) - │ │ ├── stats: [rows=10000000, distinct(1)=10000, null(1)=0, distinct(2)=500, null(2)=2000000, distinct(6)=100, null(6)=100000, distinct(8)=1000, null(8)=0] - │ │ ├── key: (1,8) - │ │ ├── fd: (1)-->(2,3), (8)-->(4-7) + │ │ ├── columns: id:1(int!null) name:2(string) state:3(string) customers.crdb_internal_mvcc_timestamp:4(decimal) order_id:5(int) item_id:6(int) customer_id:7(int) year:8(int) rowid:9(int!null) order_history.crdb_internal_mvcc_timestamp:10(decimal) + │ │ ├── stats: [rows=10000000, distinct(1)=10000, null(1)=0, distinct(2)=500, null(2)=2000000, distinct(7)=100, null(7)=100000, distinct(9)=1000, null(9)=0] + │ │ ├── key: (1,9) + │ │ ├── fd: (1)-->(2-4), (9)-->(5-8,10) │ │ ├── scan customers - │ │ │ ├── columns: id:1(int!null) name:2(string) state:3(string) + │ │ │ ├── columns: id:1(int!null) name:2(string) state:3(string) customers.crdb_internal_mvcc_timestamp:4(decimal) │ │ │ ├── stats: [rows=10000, distinct(1)=10000, null(1)=0, distinct(2)=500, null(2)=2000] │ │ │ ├── key: (1) - │ │ │ └── fd: (1)-->(2,3) + │ │ │ └── fd: (1)-->(2-4) │ │ ├── scan order_history - │ │ │ ├── columns: order_id:4(int) item_id:5(int) customer_id:6(int) year:7(int) rowid:8(int!null) - │ │ │ ├── stats: [rows=1000, distinct(6)=100, null(6)=10, distinct(8)=1000, null(8)=0] - │ │ │ ├── key: (8) - │ │ │ └── fd: (8)-->(4-7) + │ │ │ ├── columns: order_id:5(int) item_id:6(int) customer_id:7(int) year:8(int) rowid:9(int!null) order_history.crdb_internal_mvcc_timestamp:10(decimal) + │ │ │ ├── stats: [rows=1000, distinct(7)=100, null(7)=10, distinct(9)=1000, null(9)=0] + │ │ │ ├── key: (9) + │ │ │ └── fd: (9)-->(5-8,10) │ │ └── filters (true) │ └── filters - │ └── id:1 = customer_id:6 [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ └── id:1 = customer_id:7 [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] └── filters └── (id:1 = 1) AND (name:2 = 'andy') [type=bool, outer=(1,2), constraints=(/1: [/1 - /1]; /2: [/'andy' - /'andy']; tight), fd=()-->(1,2)] @@ -1193,15 +1193,15 @@ project ├── stats: [rows=9.9e-10] ├── fd: (1)==(2), (2)==(1) └── select - ├── columns: x:1(int!null) y:2(int!null) rowid:3(int!null) + ├── columns: x:1(int!null) y:2(int!null) rowid:3(int!null) crdb_internal_mvcc_timestamp:4(decimal) ├── stats: [rows=9.9e-10, distinct(1)=1e-10, null(1)=0, distinct(2)=1e-10, null(2)=0] ├── key: (3) - ├── fd: (3)-->(1,2), (1)==(2), (2)==(1) + ├── fd: (3)-->(1,2,4), (1)==(2), (2)==(1) ├── scan nulls - │ ├── columns: x:1(int) y:2(int) rowid:3(int!null) + │ ├── columns: x:1(int) y:2(int) rowid:3(int!null) crdb_internal_mvcc_timestamp:4(decimal) │ ├── stats: [rows=1000, distinct(1)=1, null(1)=1000, distinct(2)=100, null(2)=10, distinct(3)=1000, null(3)=0] │ ├── key: (3) - │ └── fd: (3)-->(1,2) + │ └── fd: (3)-->(1,2,4) └── filters └── x:1 = y:2 [type=bool, outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ]), fd=(1)==(2), (2)==(1)] @@ -1213,15 +1213,15 @@ project ├── stats: [rows=1000] ├── fd: ()-->(1) └── select - ├── columns: x:1(int) y:2(int) rowid:3(int!null) + ├── columns: x:1(int) y:2(int) rowid:3(int!null) crdb_internal_mvcc_timestamp:4(decimal) ├── stats: [rows=1000, distinct(1)=1, null(1)=1000] ├── key: (3) - ├── fd: ()-->(1), (3)-->(2) + ├── fd: ()-->(1), (3)-->(2,4) ├── scan nulls - │ ├── columns: x:1(int) y:2(int) rowid:3(int!null) + │ ├── columns: x:1(int) y:2(int) rowid:3(int!null) crdb_internal_mvcc_timestamp:4(decimal) │ ├── stats: [rows=1000, distinct(1)=1, null(1)=1000, distinct(3)=1000, null(3)=0] │ ├── key: (3) - │ └── fd: (3)-->(1,2) + │ └── fd: (3)-->(1,2,4) └── filters └── x:1 IS NULL [type=bool, outer=(1), constraints=(/1: [/NULL - /NULL]; tight), fd=()-->(1)] @@ -1236,15 +1236,15 @@ project ├── stats: [rows=10, distinct(1)=0.999956829, null(1)=10] ├── fd: ()-->(2) └── select - ├── columns: x:1(int) y:2(int!null) rowid:3(int!null) + ├── columns: x:1(int) y:2(int!null) rowid:3(int!null) crdb_internal_mvcc_timestamp:4(decimal) ├── stats: [rows=10, distinct(1)=0.999956829, null(1)=10, distinct(2)=1, null(2)=0] ├── key: (3) - ├── fd: ()-->(2), (3)-->(1) + ├── fd: ()-->(2), (3)-->(1,4) ├── scan nulls - │ ├── columns: x:1(int) y:2(int) rowid:3(int!null) + │ ├── columns: x:1(int) y:2(int) rowid:3(int!null) crdb_internal_mvcc_timestamp:4(decimal) │ ├── stats: [rows=1000, distinct(1)=1, null(1)=1000, distinct(2)=100, null(2)=10, distinct(3)=1000, null(3)=0] │ ├── key: (3) - │ └── fd: (3)-->(1,2) + │ └── fd: (3)-->(1,2,4) └── filters └── y:2 = 3 [type=bool, outer=(2), constraints=(/2: [/3 - /3]; tight), fd=()-->(2)] @@ -1427,37 +1427,37 @@ WITH t(x) AS ( SELECT x FROM t WHERE x ---- with &1 (t) - ├── columns: x:6(bool!null) + ├── columns: x:8(bool!null) ├── immutable - ├── stats: [rows=1.98e+20, distinct(6)=1, null(6)=0] - ├── fd: ()-->(6) + ├── stats: [rows=1.98e+20, distinct(8)=1, null(8)=0] + ├── fd: ()-->(8) ├── project - │ ├── columns: x:5(bool) + │ ├── columns: x:7(bool) │ ├── immutable │ ├── stats: [rows=4e+20] │ ├── left-join (cross) - │ │ ├── columns: t1.x:1(bool) t2.x:3(bool) + │ │ ├── columns: t1.x:1(bool) t2.x:4(bool) │ │ ├── stats: [rows=4e+20] │ │ ├── scan t1 │ │ │ ├── columns: t1.x:1(bool) │ │ │ └── stats: [rows=2e+10] │ │ ├── scan t2 - │ │ │ ├── columns: t2.x:3(bool) + │ │ │ ├── columns: t2.x:4(bool) │ │ │ └── stats: [rows=2e+10] │ │ └── filters (true) │ └── projections - │ └── (t1.x:1::INT8 << 5533)::BOOL OR t2.x:3 [as=x:5, type=bool, outer=(1,3), immutable] + │ └── (t1.x:1::INT8 << 5533)::BOOL OR t2.x:4 [as=x:7, type=bool, outer=(1,4), immutable] └── select - ├── columns: x:6(bool!null) - ├── stats: [rows=1.98e+20, distinct(6)=1, null(6)=0] - ├── fd: ()-->(6) + ├── columns: x:8(bool!null) + ├── stats: [rows=1.98e+20, distinct(8)=1, null(8)=0] + ├── fd: ()-->(8) ├── with-scan &1 (t) - │ ├── columns: x:6(bool) + │ ├── columns: x:8(bool) │ ├── mapping: - │ │ └── x:5(bool) => x:6(bool) - │ └── stats: [rows=4e+20, distinct(6)=3, null(6)=4e+18] + │ │ └── x:7(bool) => x:8(bool) + │ └── stats: [rows=4e+20, distinct(8)=3, null(8)=4e+18] └── filters - └── x:6 [type=bool, outer=(6), constraints=(/6: [/true - /true]; tight), fd=()-->(6)] + └── x:8 [type=bool, outer=(8), constraints=(/8: [/true - /true]; tight), fd=()-->(8)] # Regression test for #38375. Avoid floating point precision errors. exec-ddl @@ -1729,20 +1729,20 @@ norm SELECT * FROM v0 WHERE v0.c0 > 0 ---- select - ├── columns: c0:3(int!null) - ├── stats: [rows=1, distinct(3)=1, null(3)=0] + ├── columns: c0:4(int!null) + ├── stats: [rows=1, distinct(4)=1, null(4)=0] ├── project - │ ├── columns: rowid:3(int) - │ ├── stats: [rows=2, distinct(3)=2, null(3)=0] + │ ├── columns: rowid:4(int) + │ ├── stats: [rows=2, distinct(4)=2, null(4)=0] │ ├── scan t0 │ │ ├── columns: c0:1(int) t0.rowid:2(int!null) │ │ ├── stats: [rows=2, distinct(1,2)=2, null(1,2)=0] │ │ ├── key: (2) │ │ └── fd: (2)-->(1) │ └── projections - │ └── CASE WHEN c0:1 > 0 THEN 1 ELSE t0.rowid:2 END [as=rowid:3, type=int, outer=(1,2)] + │ └── CASE WHEN c0:1 > 0 THEN 1 ELSE t0.rowid:2 END [as=rowid:4, type=int, outer=(1,2)] └── filters - └── rowid:3 > 0 [type=bool, outer=(3), constraints=(/3: [/1 - ]; tight)] + └── rowid:4 > 0 [type=bool, outer=(4), constraints=(/4: [/1 - ]; tight)] exec-ddl ALTER TABLE a INJECT STATISTICS '[ @@ -1992,14 +1992,14 @@ project ├── stats: [rows=6.01] ├── fd: ()-->(1,2) └── select - ├── columns: x:1(int!null) z:2(int!null) rowid:3(int!null) + ├── columns: x:1(int!null) z:2(int!null) rowid:3(int!null) crdb_internal_mvcc_timestamp:4(decimal) ├── stats: [rows=6.01, distinct(1)=1, null(1)=0, distinct(2)=1, null(2)=0, distinct(1,2)=1, null(1,2)=0] ├── key: (3) - ├── fd: ()-->(1,2) + ├── fd: ()-->(1,2), (3)-->(4) ├── scan b - │ ├── columns: x:1(int) z:2(int!null) rowid:3(int!null) + │ ├── columns: x:1(int) z:2(int!null) rowid:3(int!null) crdb_internal_mvcc_timestamp:4(decimal) │ ├── stats: [rows=10000, distinct(1)=1000, null(1)=0, distinct(2)=100, null(2)=0, distinct(3)=10000, null(3)=0, distinct(1,2)=1500, null(1,2)=0] │ ├── key: (3) - │ └── fd: (3)-->(1,2) + │ └── fd: (3)-->(1,2,4) └── filters └── (x:1 = 1) AND (z:2 = 2) [type=bool, outer=(1,2), constraints=(/1: [/1 - /1]; /2: [/2 - /2]; tight), fd=()-->(1,2)] diff --git a/pkg/sql/opt/memo/testdata/stats/set b/pkg/sql/opt/memo/testdata/stats/set index a81f1bc8eb7c..c0bf1ef14e43 100644 --- a/pkg/sql/opt/memo/testdata/stats/set +++ b/pkg/sql/opt/memo/testdata/stats/set @@ -101,117 +101,137 @@ build SELECT *, x FROM a UNION SELECT *, rowid FROM b ---- union - ├── columns: x:8(int) y:9(int) s:10(string) x:11(int!null) + ├── columns: x:10(int) y:11(int) s:12(string) x:13(int!null) ├── left columns: a.x:1(int) a.y:2(int) a.s:3(string) a.x:1(int) - ├── right columns: b.x:4(int) z:5(int) b.s:6(string) rowid:7(int) - ├── stats: [rows=15000, distinct(8-11)=15000, null(8-11)=0] - ├── key: (8-11) - ├── scan a + ├── right columns: b.x:5(int) z:6(int) b.s:7(string) rowid:8(int) + ├── stats: [rows=15000, distinct(10-13)=15000, null(10-13)=0] + ├── key: (10-13) + ├── project │ ├── columns: a.x:1(int!null) a.y:2(int) a.s:3(string) │ ├── stats: [rows=5000, distinct(1-3)=5000, null(1-3)=0] │ ├── key: (1) - │ └── fd: (1)-->(2,3) - └── scan b - ├── columns: b.x:4(int) z:5(int!null) b.s:6(string) rowid:7(int!null) - ├── stats: [rows=10000, distinct(4-7)=10000, null(4-7)=0] - ├── key: (7) - └── fd: (7)-->(4-6) + │ ├── fd: (1)-->(2,3) + │ └── scan a + │ ├── columns: a.x:1(int!null) a.y:2(int) a.s:3(string) a.crdb_internal_mvcc_timestamp:4(decimal) + │ ├── stats: [rows=5000, distinct(1-3)=5000, null(1-3)=0] + │ ├── key: (1) + │ └── fd: (1)-->(2-4) + └── project + ├── columns: b.x:5(int) z:6(int!null) b.s:7(string) rowid:8(int!null) + ├── stats: [rows=10000, distinct(5-8)=10000, null(5-8)=0] + ├── key: (8) + ├── fd: (8)-->(5-7) + └── scan b + ├── columns: b.x:5(int) z:6(int!null) b.s:7(string) rowid:8(int!null) b.crdb_internal_mvcc_timestamp:9(decimal) + ├── stats: [rows=10000, distinct(5-8)=10000, null(5-8)=0] + ├── key: (8) + └── fd: (8)-->(5-7,9) build SELECT *, x FROM a UNION ALL SELECT *, rowid FROM b ---- union-all - ├── columns: x:8(int) y:9(int) s:10(string) x:11(int!null) + ├── columns: x:10(int) y:11(int) s:12(string) x:13(int!null) ├── left columns: a.x:1(int) a.y:2(int) a.s:3(string) a.x:1(int) - ├── right columns: b.x:4(int) z:5(int) b.s:6(string) rowid:7(int) + ├── right columns: b.x:5(int) z:6(int) b.s:7(string) rowid:8(int) ├── stats: [rows=15000] - ├── scan a + ├── project │ ├── columns: a.x:1(int!null) a.y:2(int) a.s:3(string) │ ├── stats: [rows=5000] │ ├── key: (1) - │ └── fd: (1)-->(2,3) - └── scan b - ├── columns: b.x:4(int) z:5(int!null) b.s:6(string) rowid:7(int!null) + │ ├── fd: (1)-->(2,3) + │ └── scan a + │ ├── columns: a.x:1(int!null) a.y:2(int) a.s:3(string) a.crdb_internal_mvcc_timestamp:4(decimal) + │ ├── stats: [rows=5000] + │ ├── key: (1) + │ └── fd: (1)-->(2-4) + └── project + ├── columns: b.x:5(int) z:6(int!null) b.s:7(string) rowid:8(int!null) ├── stats: [rows=10000] - ├── key: (7) - └── fd: (7)-->(4-6) + ├── key: (8) + ├── fd: (8)-->(5-7) + └── scan b + ├── columns: b.x:5(int) z:6(int!null) b.s:7(string) rowid:8(int!null) b.crdb_internal_mvcc_timestamp:9(decimal) + ├── stats: [rows=10000] + ├── key: (8) + └── fd: (8)-->(5-7,9) build SELECT y, s FROM a UNION SELECT z, s FROM c ---- union - ├── columns: y:8(int) s:9(string) + ├── columns: y:10(int) s:11(string) ├── left columns: a.y:2(int) a.s:3(string) - ├── right columns: z:5(int) c.s:6(string) - ├── stats: [rows=2000, distinct(8,9)=2000, null(8,9)=0] - ├── key: (8,9) + ├── right columns: z:6(int) c.s:7(string) + ├── stats: [rows=2000, distinct(10,11)=2000, null(10,11)=0] + ├── key: (10,11) ├── project │ ├── columns: a.y:2(int) a.s:3(string) │ ├── stats: [rows=5000, distinct(2,3)=1000, null(2,3)=0] │ └── scan a - │ ├── columns: a.x:1(int!null) a.y:2(int) a.s:3(string) + │ ├── columns: a.x:1(int!null) a.y:2(int) a.s:3(string) a.crdb_internal_mvcc_timestamp:4(decimal) │ ├── stats: [rows=5000, distinct(2,3)=1000, null(2,3)=0] │ ├── key: (1) - │ └── fd: (1)-->(2,3) + │ └── fd: (1)-->(2-4) └── project - ├── columns: z:5(int!null) c.s:6(string) - ├── stats: [rows=10000, distinct(5,6)=1000, null(5,6)=0] + ├── columns: z:6(int!null) c.s:7(string) + ├── stats: [rows=10000, distinct(6,7)=1000, null(6,7)=0] └── scan c - ├── columns: c.x:4(int) z:5(int!null) c.s:6(string) rowid:7(int!null) - ├── stats: [rows=10000, distinct(5,6)=1000, null(5,6)=0] - ├── key: (7) - └── fd: (7)-->(4-6) + ├── columns: c.x:5(int) z:6(int!null) c.s:7(string) rowid:8(int!null) c.crdb_internal_mvcc_timestamp:9(decimal) + ├── stats: [rows=10000, distinct(6,7)=1000, null(6,7)=0] + ├── key: (8) + └── fd: (8)-->(5-7,9) build SELECT y, s FROM a UNION SELECT z, s FROM b ---- union - ├── columns: y:8(int) s:9(string) + ├── columns: y:10(int) s:11(string) ├── left columns: a.y:2(int) a.s:3(string) - ├── right columns: z:5(int) b.s:6(string) - ├── stats: [rows=1200, distinct(8,9)=1200, null(8,9)=0] - ├── key: (8,9) + ├── right columns: z:6(int) b.s:7(string) + ├── stats: [rows=1200, distinct(10,11)=1200, null(10,11)=0] + ├── key: (10,11) ├── project │ ├── columns: a.y:2(int) a.s:3(string) │ ├── stats: [rows=5000, distinct(2,3)=1000, null(2,3)=0] │ └── scan a - │ ├── columns: a.x:1(int!null) a.y:2(int) a.s:3(string) + │ ├── columns: a.x:1(int!null) a.y:2(int) a.s:3(string) a.crdb_internal_mvcc_timestamp:4(decimal) │ ├── stats: [rows=5000, distinct(2,3)=1000, null(2,3)=0] │ ├── key: (1) - │ └── fd: (1)-->(2,3) + │ └── fd: (1)-->(2-4) └── project - ├── columns: z:5(int!null) b.s:6(string) - ├── stats: [rows=10000, distinct(5,6)=200, null(5,6)=0] + ├── columns: z:6(int!null) b.s:7(string) + ├── stats: [rows=10000, distinct(6,7)=200, null(6,7)=0] └── scan b - ├── columns: b.x:4(int) z:5(int!null) b.s:6(string) rowid:7(int!null) - ├── stats: [rows=10000, distinct(5,6)=200, null(5,6)=0] - ├── key: (7) - └── fd: (7)-->(4-6) + ├── columns: b.x:5(int) z:6(int!null) b.s:7(string) rowid:8(int!null) b.crdb_internal_mvcc_timestamp:9(decimal) + ├── stats: [rows=10000, distinct(6,7)=200, null(6,7)=0] + ├── key: (8) + └── fd: (8)-->(5-7,9) build SELECT y, s FROM a UNION ALL SELECT z, s FROM b ---- union-all - ├── columns: y:8(int) s:9(string) + ├── columns: y:10(int) s:11(string) ├── left columns: a.y:2(int) a.s:3(string) - ├── right columns: z:5(int) b.s:6(string) + ├── right columns: z:6(int) b.s:7(string) ├── stats: [rows=15000] ├── project │ ├── columns: a.y:2(int) a.s:3(string) │ ├── stats: [rows=5000] │ └── scan a - │ ├── columns: a.x:1(int!null) a.y:2(int) a.s:3(string) + │ ├── columns: a.x:1(int!null) a.y:2(int) a.s:3(string) a.crdb_internal_mvcc_timestamp:4(decimal) │ ├── stats: [rows=5000] │ ├── key: (1) - │ └── fd: (1)-->(2,3) + │ └── fd: (1)-->(2-4) └── project - ├── columns: z:5(int!null) b.s:6(string) + ├── columns: z:6(int!null) b.s:7(string) ├── stats: [rows=10000] └── scan b - ├── columns: b.x:4(int) z:5(int!null) b.s:6(string) rowid:7(int!null) + ├── columns: b.x:5(int) z:6(int!null) b.s:7(string) rowid:8(int!null) b.crdb_internal_mvcc_timestamp:9(decimal) ├── stats: [rows=10000] - ├── key: (7) - └── fd: (7)-->(4-6) + ├── key: (8) + └── fd: (8)-->(5-7,9) build SELECT x, y, x FROM a INTERSECT SELECT z, x, rowid FROM (SELECT *, rowid FROM b WHERE b.x=1) b @@ -219,7 +239,7 @@ SELECT x, y, x FROM a INTERSECT SELECT z, x, rowid FROM (SELECT *, rowid FROM b intersect ├── columns: x:1(int!null) y:2(int) x:1(int!null) ├── left columns: a.x:1(int!null) y:2(int) a.x:1(int!null) - ├── right columns: z:5(int) b.x:4(int) rowid:7(int) + ├── right columns: z:6(int) b.x:5(int) rowid:8(int) ├── stats: [rows=2, distinct(1,2)=2, null(1,2)=0] ├── key: (1,2) ├── project @@ -228,27 +248,32 @@ intersect │ ├── key: (1) │ ├── fd: (1)-->(2) │ └── scan a - │ ├── columns: a.x:1(int!null) y:2(int) a.s:3(string) + │ ├── columns: a.x:1(int!null) y:2(int) a.s:3(string) a.crdb_internal_mvcc_timestamp:4(decimal) │ ├── stats: [rows=5000, distinct(1,2)=5000, null(1,2)=0] │ ├── key: (1) - │ └── fd: (1)-->(2,3) + │ └── fd: (1)-->(2-4) └── project - ├── columns: b.x:4(int!null) z:5(int!null) rowid:7(int!null) - ├── stats: [rows=2, distinct(4,5,7)=2, null(4,5,7)=0] - ├── key: (7) - ├── fd: ()-->(4), (7)-->(5) - └── select - ├── columns: b.x:4(int!null) z:5(int!null) b.s:6(string) rowid:7(int!null) - ├── stats: [rows=2, distinct(4)=1, null(4)=0, distinct(4,5,7)=2, null(4,5,7)=0] - ├── key: (7) - ├── fd: ()-->(4), (7)-->(5,6) - ├── scan b - │ ├── columns: b.x:4(int) z:5(int!null) b.s:6(string) rowid:7(int!null) - │ ├── stats: [rows=10000, distinct(4)=5000, null(4)=0, distinct(5)=100, null(5)=0, distinct(7)=10000, null(7)=0, distinct(4,5,7)=10000, null(4,5,7)=0] - │ ├── key: (7) - │ └── fd: (7)-->(4-6) - └── filters - └── b.x:4 = 1 [type=bool, outer=(4), constraints=(/4: [/1 - /1]; tight), fd=()-->(4)] + ├── columns: b.x:5(int!null) z:6(int!null) rowid:8(int!null) + ├── stats: [rows=2, distinct(5,6,8)=2, null(5,6,8)=0] + ├── key: (8) + ├── fd: ()-->(5), (8)-->(6) + └── project + ├── columns: b.x:5(int!null) z:6(int!null) b.s:7(string) rowid:8(int!null) + ├── stats: [rows=2, distinct(5,6,8)=2, null(5,6,8)=0] + ├── key: (8) + ├── fd: ()-->(5), (8)-->(6,7) + └── select + ├── columns: b.x:5(int!null) z:6(int!null) b.s:7(string) rowid:8(int!null) b.crdb_internal_mvcc_timestamp:9(decimal) + ├── stats: [rows=2, distinct(5)=1, null(5)=0, distinct(5,6,8)=2, null(5,6,8)=0] + ├── key: (8) + ├── fd: ()-->(5), (8)-->(6,7,9) + ├── scan b + │ ├── columns: b.x:5(int) z:6(int!null) b.s:7(string) rowid:8(int!null) b.crdb_internal_mvcc_timestamp:9(decimal) + │ ├── stats: [rows=10000, distinct(5)=5000, null(5)=0, distinct(6)=100, null(6)=0, distinct(8)=10000, null(8)=0, distinct(5,6,8)=10000, null(5,6,8)=0] + │ ├── key: (8) + │ └── fd: (8)-->(5-7,9) + └── filters + └── b.x:5 = 1 [type=bool, outer=(5), constraints=(/5: [/1 - /1]; tight), fd=()-->(5)] build SELECT x, y, x FROM a INTERSECT ALL SELECT z, x, rowid FROM (SELECT *, rowid FROM b WHERE b.x=1) b @@ -256,7 +281,7 @@ SELECT x, y, x FROM a INTERSECT ALL SELECT z, x, rowid FROM (SELECT *, rowid FRO intersect-all ├── columns: x:1(int!null) y:2(int) x:1(int!null) ├── left columns: a.x:1(int!null) y:2(int) a.x:1(int!null) - ├── right columns: z:5(int) b.x:4(int) rowid:7(int) + ├── right columns: z:6(int) b.x:5(int) rowid:8(int) ├── stats: [rows=2] ├── project │ ├── columns: a.x:1(int!null) y:2(int) @@ -264,27 +289,32 @@ intersect-all │ ├── key: (1) │ ├── fd: (1)-->(2) │ └── scan a - │ ├── columns: a.x:1(int!null) y:2(int) a.s:3(string) + │ ├── columns: a.x:1(int!null) y:2(int) a.s:3(string) a.crdb_internal_mvcc_timestamp:4(decimal) │ ├── stats: [rows=5000] │ ├── key: (1) - │ └── fd: (1)-->(2,3) + │ └── fd: (1)-->(2-4) └── project - ├── columns: b.x:4(int!null) z:5(int!null) rowid:7(int!null) + ├── columns: b.x:5(int!null) z:6(int!null) rowid:8(int!null) ├── stats: [rows=2] - ├── key: (7) - ├── fd: ()-->(4), (7)-->(5) - └── select - ├── columns: b.x:4(int!null) z:5(int!null) b.s:6(string) rowid:7(int!null) - ├── stats: [rows=2, distinct(4)=1, null(4)=0] - ├── key: (7) - ├── fd: ()-->(4), (7)-->(5,6) - ├── scan b - │ ├── columns: b.x:4(int) z:5(int!null) b.s:6(string) rowid:7(int!null) - │ ├── stats: [rows=10000, distinct(4)=5000, null(4)=0, distinct(5)=100, null(5)=0, distinct(7)=10000, null(7)=0] - │ ├── key: (7) - │ └── fd: (7)-->(4-6) - └── filters - └── b.x:4 = 1 [type=bool, outer=(4), constraints=(/4: [/1 - /1]; tight), fd=()-->(4)] + ├── key: (8) + ├── fd: ()-->(5), (8)-->(6) + └── project + ├── columns: b.x:5(int!null) z:6(int!null) b.s:7(string) rowid:8(int!null) + ├── stats: [rows=2] + ├── key: (8) + ├── fd: ()-->(5), (8)-->(6,7) + └── select + ├── columns: b.x:5(int!null) z:6(int!null) b.s:7(string) rowid:8(int!null) b.crdb_internal_mvcc_timestamp:9(decimal) + ├── stats: [rows=2, distinct(5)=1, null(5)=0] + ├── key: (8) + ├── fd: ()-->(5), (8)-->(6,7,9) + ├── scan b + │ ├── columns: b.x:5(int) z:6(int!null) b.s:7(string) rowid:8(int!null) b.crdb_internal_mvcc_timestamp:9(decimal) + │ ├── stats: [rows=10000, distinct(5)=5000, null(5)=0, distinct(6)=100, null(6)=0, distinct(8)=10000, null(8)=0] + │ ├── key: (8) + │ └── fd: (8)-->(5-7,9) + └── filters + └── b.x:5 = 1 [type=bool, outer=(5), constraints=(/5: [/1 - /1]; tight), fd=()-->(5)] build SELECT y, s FROM a INTERSECT SELECT z, s FROM c @@ -292,25 +322,25 @@ SELECT y, s FROM a INTERSECT SELECT z, s FROM c intersect ├── columns: y:2(int) s:3(string) ├── left columns: y:2(int) a.s:3(string) - ├── right columns: z:5(int) c.s:6(string) + ├── right columns: z:6(int) c.s:7(string) ├── stats: [rows=1000, distinct(2,3)=1000, null(2,3)=0] ├── key: (2,3) ├── project │ ├── columns: y:2(int) a.s:3(string) │ ├── stats: [rows=5000, distinct(2,3)=1000, null(2,3)=0] │ └── scan a - │ ├── columns: a.x:1(int!null) y:2(int) a.s:3(string) + │ ├── columns: a.x:1(int!null) y:2(int) a.s:3(string) a.crdb_internal_mvcc_timestamp:4(decimal) │ ├── stats: [rows=5000, distinct(2,3)=1000, null(2,3)=0] │ ├── key: (1) - │ └── fd: (1)-->(2,3) + │ └── fd: (1)-->(2-4) └── project - ├── columns: z:5(int!null) c.s:6(string) - ├── stats: [rows=10000, distinct(5,6)=1000, null(5,6)=0] + ├── columns: z:6(int!null) c.s:7(string) + ├── stats: [rows=10000, distinct(6,7)=1000, null(6,7)=0] └── scan c - ├── columns: c.x:4(int) z:5(int!null) c.s:6(string) rowid:7(int!null) - ├── stats: [rows=10000, distinct(5,6)=1000, null(5,6)=0] - ├── key: (7) - └── fd: (7)-->(4-6) + ├── columns: c.x:5(int) z:6(int!null) c.s:7(string) rowid:8(int!null) c.crdb_internal_mvcc_timestamp:9(decimal) + ├── stats: [rows=10000, distinct(6,7)=1000, null(6,7)=0] + ├── key: (8) + └── fd: (8)-->(5-7,9) build SELECT y, s FROM a INTERSECT SELECT z, s FROM b @@ -318,25 +348,25 @@ SELECT y, s FROM a INTERSECT SELECT z, s FROM b intersect ├── columns: y:2(int) s:3(string) ├── left columns: y:2(int) a.s:3(string) - ├── right columns: z:5(int) b.s:6(string) + ├── right columns: z:6(int) b.s:7(string) ├── stats: [rows=200, distinct(2,3)=200, null(2,3)=0] ├── key: (2,3) ├── project │ ├── columns: y:2(int) a.s:3(string) │ ├── stats: [rows=5000, distinct(2,3)=1000, null(2,3)=0] │ └── scan a - │ ├── columns: a.x:1(int!null) y:2(int) a.s:3(string) + │ ├── columns: a.x:1(int!null) y:2(int) a.s:3(string) a.crdb_internal_mvcc_timestamp:4(decimal) │ ├── stats: [rows=5000, distinct(2,3)=1000, null(2,3)=0] │ ├── key: (1) - │ └── fd: (1)-->(2,3) + │ └── fd: (1)-->(2-4) └── project - ├── columns: z:5(int!null) b.s:6(string) - ├── stats: [rows=10000, distinct(5,6)=200, null(5,6)=0] + ├── columns: z:6(int!null) b.s:7(string) + ├── stats: [rows=10000, distinct(6,7)=200, null(6,7)=0] └── scan b - ├── columns: b.x:4(int) z:5(int!null) b.s:6(string) rowid:7(int!null) - ├── stats: [rows=10000, distinct(5,6)=200, null(5,6)=0] - ├── key: (7) - └── fd: (7)-->(4-6) + ├── columns: b.x:5(int) z:6(int!null) b.s:7(string) rowid:8(int!null) b.crdb_internal_mvcc_timestamp:9(decimal) + ├── stats: [rows=10000, distinct(6,7)=200, null(6,7)=0] + ├── key: (8) + └── fd: (8)-->(5-7,9) build SELECT y, s FROM a INTERSECT ALL SELECT z, s FROM b @@ -344,24 +374,24 @@ SELECT y, s FROM a INTERSECT ALL SELECT z, s FROM b intersect-all ├── columns: y:2(int) s:3(string) ├── left columns: y:2(int) a.s:3(string) - ├── right columns: z:5(int) b.s:6(string) + ├── right columns: z:6(int) b.s:7(string) ├── stats: [rows=5000] ├── project │ ├── columns: y:2(int) a.s:3(string) │ ├── stats: [rows=5000] │ └── scan a - │ ├── columns: a.x:1(int!null) y:2(int) a.s:3(string) + │ ├── columns: a.x:1(int!null) y:2(int) a.s:3(string) a.crdb_internal_mvcc_timestamp:4(decimal) │ ├── stats: [rows=5000] │ ├── key: (1) - │ └── fd: (1)-->(2,3) + │ └── fd: (1)-->(2-4) └── project - ├── columns: z:5(int!null) b.s:6(string) + ├── columns: z:6(int!null) b.s:7(string) ├── stats: [rows=10000] └── scan b - ├── columns: b.x:4(int) z:5(int!null) b.s:6(string) rowid:7(int!null) + ├── columns: b.x:5(int) z:6(int!null) b.s:7(string) rowid:8(int!null) b.crdb_internal_mvcc_timestamp:9(decimal) ├── stats: [rows=10000] - ├── key: (7) - └── fd: (7)-->(4-6) + ├── key: (8) + └── fd: (8)-->(5-7,9) build SELECT x, x, y FROM a EXCEPT SELECT x, z, z FROM (SELECT * FROM b WHERE b.x=1) b @@ -369,7 +399,7 @@ SELECT x, x, y FROM a EXCEPT SELECT x, z, z FROM (SELECT * FROM b WHERE b.x=1) b except ├── columns: x:1(int!null) x:1(int!null) y:2(int) ├── left columns: a.x:1(int!null) a.x:1(int!null) y:2(int) - ├── right columns: b.x:4(int) z:5(int) z:5(int) + ├── right columns: b.x:5(int) z:6(int) z:6(int) ├── stats: [rows=5000, distinct(1,2)=5000, null(1,2)=0] ├── key: (1,2) ├── project @@ -378,30 +408,30 @@ except │ ├── key: (1) │ ├── fd: (1)-->(2) │ └── scan a - │ ├── columns: a.x:1(int!null) y:2(int) a.s:3(string) + │ ├── columns: a.x:1(int!null) y:2(int) a.s:3(string) a.crdb_internal_mvcc_timestamp:4(decimal) │ ├── stats: [rows=5000, distinct(1,2)=5000, null(1,2)=0] │ ├── key: (1) - │ └── fd: (1)-->(2,3) + │ └── fd: (1)-->(2-4) └── project - ├── columns: b.x:4(int!null) z:5(int!null) - ├── stats: [rows=2, distinct(4,5)=2, null(4,5)=0] - ├── fd: ()-->(4) + ├── columns: b.x:5(int!null) z:6(int!null) + ├── stats: [rows=2, distinct(5,6)=2, null(5,6)=0] + ├── fd: ()-->(5) └── project - ├── columns: b.x:4(int!null) z:5(int!null) b.s:6(string) - ├── stats: [rows=2, distinct(4,5)=2, null(4,5)=0] - ├── fd: ()-->(4) + ├── columns: b.x:5(int!null) z:6(int!null) b.s:7(string) + ├── stats: [rows=2, distinct(5,6)=2, null(5,6)=0] + ├── fd: ()-->(5) └── select - ├── columns: b.x:4(int!null) z:5(int!null) b.s:6(string) rowid:7(int!null) - ├── stats: [rows=2, distinct(4)=1, null(4)=0, distinct(4,5)=2, null(4,5)=0] - ├── key: (7) - ├── fd: ()-->(4), (7)-->(5,6) + ├── columns: b.x:5(int!null) z:6(int!null) b.s:7(string) rowid:8(int!null) b.crdb_internal_mvcc_timestamp:9(decimal) + ├── stats: [rows=2, distinct(5)=1, null(5)=0, distinct(5,6)=2, null(5,6)=0] + ├── key: (8) + ├── fd: ()-->(5), (8)-->(6,7,9) ├── scan b - │ ├── columns: b.x:4(int) z:5(int!null) b.s:6(string) rowid:7(int!null) - │ ├── stats: [rows=10000, distinct(4)=5000, null(4)=0, distinct(5)=100, null(5)=0, distinct(7)=10000, null(7)=0, distinct(4,5)=10000, null(4,5)=0] - │ ├── key: (7) - │ └── fd: (7)-->(4-6) + │ ├── columns: b.x:5(int) z:6(int!null) b.s:7(string) rowid:8(int!null) b.crdb_internal_mvcc_timestamp:9(decimal) + │ ├── stats: [rows=10000, distinct(5)=5000, null(5)=0, distinct(6)=100, null(6)=0, distinct(8)=10000, null(8)=0, distinct(5,6)=10000, null(5,6)=0] + │ ├── key: (8) + │ └── fd: (8)-->(5-7,9) └── filters - └── b.x:4 = 1 [type=bool, outer=(4), constraints=(/4: [/1 - /1]; tight), fd=()-->(4)] + └── b.x:5 = 1 [type=bool, outer=(5), constraints=(/5: [/1 - /1]; tight), fd=()-->(5)] build SELECT x, x, y FROM a EXCEPT ALL SELECT x, z, z FROM (SELECT * FROM b WHERE b.x=1) b @@ -409,7 +439,7 @@ SELECT x, x, y FROM a EXCEPT ALL SELECT x, z, z FROM (SELECT * FROM b WHERE b.x= except-all ├── columns: x:1(int!null) x:1(int!null) y:2(int) ├── left columns: a.x:1(int!null) a.x:1(int!null) y:2(int) - ├── right columns: b.x:4(int) z:5(int) z:5(int) + ├── right columns: b.x:5(int) z:6(int) z:6(int) ├── stats: [rows=5000] ├── project │ ├── columns: a.x:1(int!null) y:2(int) @@ -417,30 +447,30 @@ except-all │ ├── key: (1) │ ├── fd: (1)-->(2) │ └── scan a - │ ├── columns: a.x:1(int!null) y:2(int) a.s:3(string) + │ ├── columns: a.x:1(int!null) y:2(int) a.s:3(string) a.crdb_internal_mvcc_timestamp:4(decimal) │ ├── stats: [rows=5000] │ ├── key: (1) - │ └── fd: (1)-->(2,3) + │ └── fd: (1)-->(2-4) └── project - ├── columns: b.x:4(int!null) z:5(int!null) + ├── columns: b.x:5(int!null) z:6(int!null) ├── stats: [rows=2] - ├── fd: ()-->(4) + ├── fd: ()-->(5) └── project - ├── columns: b.x:4(int!null) z:5(int!null) b.s:6(string) + ├── columns: b.x:5(int!null) z:6(int!null) b.s:7(string) ├── stats: [rows=2] - ├── fd: ()-->(4) + ├── fd: ()-->(5) └── select - ├── columns: b.x:4(int!null) z:5(int!null) b.s:6(string) rowid:7(int!null) - ├── stats: [rows=2, distinct(4)=1, null(4)=0] - ├── key: (7) - ├── fd: ()-->(4), (7)-->(5,6) + ├── columns: b.x:5(int!null) z:6(int!null) b.s:7(string) rowid:8(int!null) b.crdb_internal_mvcc_timestamp:9(decimal) + ├── stats: [rows=2, distinct(5)=1, null(5)=0] + ├── key: (8) + ├── fd: ()-->(5), (8)-->(6,7,9) ├── scan b - │ ├── columns: b.x:4(int) z:5(int!null) b.s:6(string) rowid:7(int!null) - │ ├── stats: [rows=10000, distinct(4)=5000, null(4)=0, distinct(5)=100, null(5)=0, distinct(7)=10000, null(7)=0] - │ ├── key: (7) - │ └── fd: (7)-->(4-6) + │ ├── columns: b.x:5(int) z:6(int!null) b.s:7(string) rowid:8(int!null) b.crdb_internal_mvcc_timestamp:9(decimal) + │ ├── stats: [rows=10000, distinct(5)=5000, null(5)=0, distinct(6)=100, null(6)=0, distinct(8)=10000, null(8)=0] + │ ├── key: (8) + │ └── fd: (8)-->(5-7,9) └── filters - └── b.x:4 = 1 [type=bool, outer=(4), constraints=(/4: [/1 - /1]; tight), fd=()-->(4)] + └── b.x:5 = 1 [type=bool, outer=(5), constraints=(/5: [/1 - /1]; tight), fd=()-->(5)] build SELECT y, s FROM a EXCEPT SELECT z, s FROM c @@ -448,25 +478,25 @@ SELECT y, s FROM a EXCEPT SELECT z, s FROM c except ├── columns: y:2(int) s:3(string) ├── left columns: y:2(int) a.s:3(string) - ├── right columns: z:5(int) c.s:6(string) + ├── right columns: z:6(int) c.s:7(string) ├── stats: [rows=1000, distinct(2,3)=1000, null(2,3)=0] ├── key: (2,3) ├── project │ ├── columns: y:2(int) a.s:3(string) │ ├── stats: [rows=5000, distinct(2,3)=1000, null(2,3)=0] │ └── scan a - │ ├── columns: a.x:1(int!null) y:2(int) a.s:3(string) + │ ├── columns: a.x:1(int!null) y:2(int) a.s:3(string) a.crdb_internal_mvcc_timestamp:4(decimal) │ ├── stats: [rows=5000, distinct(2,3)=1000, null(2,3)=0] │ ├── key: (1) - │ └── fd: (1)-->(2,3) + │ └── fd: (1)-->(2-4) └── project - ├── columns: z:5(int!null) c.s:6(string) - ├── stats: [rows=10000, distinct(5,6)=1000, null(5,6)=0] + ├── columns: z:6(int!null) c.s:7(string) + ├── stats: [rows=10000, distinct(6,7)=1000, null(6,7)=0] └── scan c - ├── columns: c.x:4(int) z:5(int!null) c.s:6(string) rowid:7(int!null) - ├── stats: [rows=10000, distinct(5,6)=1000, null(5,6)=0] - ├── key: (7) - └── fd: (7)-->(4-6) + ├── columns: c.x:5(int) z:6(int!null) c.s:7(string) rowid:8(int!null) c.crdb_internal_mvcc_timestamp:9(decimal) + ├── stats: [rows=10000, distinct(6,7)=1000, null(6,7)=0] + ├── key: (8) + └── fd: (8)-->(5-7,9) build SELECT y, s FROM a EXCEPT SELECT z, s FROM b @@ -474,25 +504,25 @@ SELECT y, s FROM a EXCEPT SELECT z, s FROM b except ├── columns: y:2(int) s:3(string) ├── left columns: y:2(int) a.s:3(string) - ├── right columns: z:5(int) b.s:6(string) + ├── right columns: z:6(int) b.s:7(string) ├── stats: [rows=1000, distinct(2,3)=1000, null(2,3)=0] ├── key: (2,3) ├── project │ ├── columns: y:2(int) a.s:3(string) │ ├── stats: [rows=5000, distinct(2,3)=1000, null(2,3)=0] │ └── scan a - │ ├── columns: a.x:1(int!null) y:2(int) a.s:3(string) + │ ├── columns: a.x:1(int!null) y:2(int) a.s:3(string) a.crdb_internal_mvcc_timestamp:4(decimal) │ ├── stats: [rows=5000, distinct(2,3)=1000, null(2,3)=0] │ ├── key: (1) - │ └── fd: (1)-->(2,3) + │ └── fd: (1)-->(2-4) └── project - ├── columns: z:5(int!null) b.s:6(string) - ├── stats: [rows=10000, distinct(5,6)=200, null(5,6)=0] + ├── columns: z:6(int!null) b.s:7(string) + ├── stats: [rows=10000, distinct(6,7)=200, null(6,7)=0] └── scan b - ├── columns: b.x:4(int) z:5(int!null) b.s:6(string) rowid:7(int!null) - ├── stats: [rows=10000, distinct(5,6)=200, null(5,6)=0] - ├── key: (7) - └── fd: (7)-->(4-6) + ├── columns: b.x:5(int) z:6(int!null) b.s:7(string) rowid:8(int!null) b.crdb_internal_mvcc_timestamp:9(decimal) + ├── stats: [rows=10000, distinct(6,7)=200, null(6,7)=0] + ├── key: (8) + └── fd: (8)-->(5-7,9) build SELECT y, s FROM a EXCEPT ALL SELECT z, s FROM b @@ -500,24 +530,24 @@ SELECT y, s FROM a EXCEPT ALL SELECT z, s FROM b except-all ├── columns: y:2(int) s:3(string) ├── left columns: y:2(int) a.s:3(string) - ├── right columns: z:5(int) b.s:6(string) + ├── right columns: z:6(int) b.s:7(string) ├── stats: [rows=5000] ├── project │ ├── columns: y:2(int) a.s:3(string) │ ├── stats: [rows=5000] │ └── scan a - │ ├── columns: a.x:1(int!null) y:2(int) a.s:3(string) + │ ├── columns: a.x:1(int!null) y:2(int) a.s:3(string) a.crdb_internal_mvcc_timestamp:4(decimal) │ ├── stats: [rows=5000] │ ├── key: (1) - │ └── fd: (1)-->(2,3) + │ └── fd: (1)-->(2-4) └── project - ├── columns: z:5(int!null) b.s:6(string) + ├── columns: z:6(int!null) b.s:7(string) ├── stats: [rows=10000] └── scan b - ├── columns: b.x:4(int) z:5(int!null) b.s:6(string) rowid:7(int!null) + ├── columns: b.x:5(int) z:6(int!null) b.s:7(string) rowid:8(int!null) b.crdb_internal_mvcc_timestamp:9(decimal) ├── stats: [rows=10000] - ├── key: (7) - └── fd: (7)-->(4-6) + ├── key: (8) + └── fd: (8)-->(5-7,9) build SELECT * FROM (SELECT y, s FROM a EXCEPT ALL SELECT z, s FROM b) WHERE y = 5 @@ -529,24 +559,24 @@ select ├── except-all │ ├── columns: y:2(int) a.s:3(string) │ ├── left columns: y:2(int) a.s:3(string) - │ ├── right columns: z:5(int) b.s:6(string) + │ ├── right columns: z:6(int) b.s:7(string) │ ├── stats: [rows=5000, distinct(2)=400, null(2)=0] │ ├── project │ │ ├── columns: y:2(int) a.s:3(string) │ │ ├── stats: [rows=5000, distinct(2)=400, null(2)=0] │ │ └── scan a - │ │ ├── columns: a.x:1(int!null) y:2(int) a.s:3(string) + │ │ ├── columns: a.x:1(int!null) y:2(int) a.s:3(string) a.crdb_internal_mvcc_timestamp:4(decimal) │ │ ├── stats: [rows=5000, distinct(2)=400, null(2)=0] │ │ ├── key: (1) - │ │ └── fd: (1)-->(2,3) + │ │ └── fd: (1)-->(2-4) │ └── project - │ ├── columns: z:5(int!null) b.s:6(string) - │ ├── stats: [rows=10000, distinct(5)=100, null(5)=0] + │ ├── columns: z:6(int!null) b.s:7(string) + │ ├── stats: [rows=10000, distinct(6)=100, null(6)=0] │ └── scan b - │ ├── columns: b.x:4(int) z:5(int!null) b.s:6(string) rowid:7(int!null) - │ ├── stats: [rows=10000, distinct(5)=100, null(5)=0] - │ ├── key: (7) - │ └── fd: (7)-->(4-6) + │ ├── columns: b.x:5(int) z:6(int!null) b.s:7(string) rowid:8(int!null) b.crdb_internal_mvcc_timestamp:9(decimal) + │ ├── stats: [rows=10000, distinct(6)=100, null(6)=0] + │ ├── key: (8) + │ └── fd: (8)-->(5-7,9) └── filters └── y:2 = 5 [type=bool, outer=(2), constraints=(/2: [/5 - /5]; tight), fd=()-->(2)] @@ -607,31 +637,31 @@ ALTER TABLE c INJECT STATISTICS '[ ]' ---- -build colstat=9 colstat=10 colstat=(9,10) +build colstat=11 colstat=12 colstat=(11,12) SELECT x,s FROM b UNION SELECT x,s FROM c ---- union - ├── columns: x:9(int) s:10(string) + ├── columns: x:11(int) s:12(string) ├── left columns: b.x:1(int) b.s:3(string) - ├── right columns: c.x:5(int) c.s:7(string) - ├── stats: [rows=20000, distinct(9)=10000, null(9)=1, distinct(10)=20, null(10)=1, distinct(9,10)=20000, null(9,10)=1] - ├── key: (9,10) + ├── right columns: c.x:6(int) c.s:8(string) + ├── stats: [rows=20000, distinct(11)=10000, null(11)=1, distinct(12)=20, null(12)=1, distinct(11,12)=20000, null(11,12)=1] + ├── key: (11,12) ├── project │ ├── columns: b.x:1(int) b.s:3(string) │ ├── stats: [rows=10000, distinct(1)=5000, null(1)=2500, distinct(3)=10, null(3)=5000, distinct(1,3)=10000, null(1,3)=1250] │ └── scan b - │ ├── columns: b.x:1(int) b.z:2(int!null) b.s:3(string) b.rowid:4(int!null) + │ ├── columns: b.x:1(int) b.z:2(int!null) b.s:3(string) b.rowid:4(int!null) b.crdb_internal_mvcc_timestamp:5(decimal) │ ├── stats: [rows=10000, distinct(1)=5000, null(1)=2500, distinct(3)=10, null(3)=5000, distinct(1,3)=10000, null(1,3)=1250] │ ├── key: (4) - │ └── fd: (4)-->(1-3) + │ └── fd: (4)-->(1-3,5) └── project - ├── columns: c.x:5(int) c.s:7(string) - ├── stats: [rows=10000, distinct(5)=5000, null(5)=1000, distinct(7)=10, null(7)=7500, distinct(5,7)=10000, null(5,7)=750] + ├── columns: c.x:6(int) c.s:8(string) + ├── stats: [rows=10000, distinct(6)=5000, null(6)=1000, distinct(8)=10, null(8)=7500, distinct(6,8)=10000, null(6,8)=750] └── scan c - ├── columns: c.x:5(int) c.z:6(int!null) c.s:7(string) c.rowid:8(int!null) - ├── stats: [rows=10000, distinct(5)=5000, null(5)=1000, distinct(7)=10, null(7)=7500, distinct(5,7)=10000, null(5,7)=750] - ├── key: (8) - └── fd: (8)-->(5-7) + ├── columns: c.x:6(int) c.z:7(int!null) c.s:8(string) c.rowid:9(int!null) c.crdb_internal_mvcc_timestamp:10(decimal) + ├── stats: [rows=10000, distinct(6)=5000, null(6)=1000, distinct(8)=10, null(8)=7500, distinct(6,8)=10000, null(6,8)=750] + ├── key: (9) + └── fd: (9)-->(6-8,10) build colstat=1 colstat=3 colstat=(1,3) SELECT x,s FROM b INTERSECT SELECT x,s FROM c @@ -639,25 +669,25 @@ SELECT x,s FROM b INTERSECT SELECT x,s FROM c intersect ├── columns: x:1(int) s:3(string) ├── left columns: b.x:1(int) b.s:3(string) - ├── right columns: c.x:5(int) c.s:7(string) + ├── right columns: c.x:6(int) c.s:8(string) ├── stats: [rows=10000, distinct(1)=5000, null(1)=1, distinct(3)=10, null(3)=1, distinct(1,3)=10000, null(1,3)=1] ├── key: (1,3) ├── project │ ├── columns: b.x:1(int) b.s:3(string) │ ├── stats: [rows=10000, distinct(1)=5000, null(1)=2500, distinct(3)=10, null(3)=5000, distinct(1,3)=10000, null(1,3)=1250] │ └── scan b - │ ├── columns: b.x:1(int) b.z:2(int!null) b.s:3(string) b.rowid:4(int!null) + │ ├── columns: b.x:1(int) b.z:2(int!null) b.s:3(string) b.rowid:4(int!null) b.crdb_internal_mvcc_timestamp:5(decimal) │ ├── stats: [rows=10000, distinct(1)=5000, null(1)=2500, distinct(3)=10, null(3)=5000, distinct(1,3)=10000, null(1,3)=1250] │ ├── key: (4) - │ └── fd: (4)-->(1-3) + │ └── fd: (4)-->(1-3,5) └── project - ├── columns: c.x:5(int) c.s:7(string) - ├── stats: [rows=10000, distinct(5)=5000, null(5)=1000, distinct(7)=10, null(7)=7500, distinct(5,7)=10000, null(5,7)=750] + ├── columns: c.x:6(int) c.s:8(string) + ├── stats: [rows=10000, distinct(6)=5000, null(6)=1000, distinct(8)=10, null(8)=7500, distinct(6,8)=10000, null(6,8)=750] └── scan c - ├── columns: c.x:5(int) c.z:6(int!null) c.s:7(string) c.rowid:8(int!null) - ├── stats: [rows=10000, distinct(5)=5000, null(5)=1000, distinct(7)=10, null(7)=7500, distinct(5,7)=10000, null(5,7)=750] - ├── key: (8) - └── fd: (8)-->(5-7) + ├── columns: c.x:6(int) c.z:7(int!null) c.s:8(string) c.rowid:9(int!null) c.crdb_internal_mvcc_timestamp:10(decimal) + ├── stats: [rows=10000, distinct(6)=5000, null(6)=1000, distinct(8)=10, null(8)=7500, distinct(6,8)=10000, null(6,8)=750] + ├── key: (9) + └── fd: (9)-->(6-8,10) build colstat=1 colstat=3 colstat=(1,3) SELECT x,s FROM b EXCEPT SELECT x,s FROM c @@ -665,51 +695,51 @@ SELECT x,s FROM b EXCEPT SELECT x,s FROM c except ├── columns: x:1(int) s:3(string) ├── left columns: b.x:1(int) b.s:3(string) - ├── right columns: c.x:5(int) c.s:7(string) + ├── right columns: c.x:6(int) c.s:8(string) ├── stats: [rows=10000, distinct(1)=5000, null(1)=0, distinct(3)=10, null(3)=0, distinct(1,3)=10000, null(1,3)=0] ├── key: (1,3) ├── project │ ├── columns: b.x:1(int) b.s:3(string) │ ├── stats: [rows=10000, distinct(1)=5000, null(1)=2500, distinct(3)=10, null(3)=5000, distinct(1,3)=10000, null(1,3)=1250] │ └── scan b - │ ├── columns: b.x:1(int) b.z:2(int!null) b.s:3(string) b.rowid:4(int!null) + │ ├── columns: b.x:1(int) b.z:2(int!null) b.s:3(string) b.rowid:4(int!null) b.crdb_internal_mvcc_timestamp:5(decimal) │ ├── stats: [rows=10000, distinct(1)=5000, null(1)=2500, distinct(3)=10, null(3)=5000, distinct(1,3)=10000, null(1,3)=1250] │ ├── key: (4) - │ └── fd: (4)-->(1-3) + │ └── fd: (4)-->(1-3,5) └── project - ├── columns: c.x:5(int) c.s:7(string) - ├── stats: [rows=10000, distinct(5)=5000, null(5)=1000, distinct(7)=10, null(7)=7500, distinct(5,7)=10000, null(5,7)=750] + ├── columns: c.x:6(int) c.s:8(string) + ├── stats: [rows=10000, distinct(6)=5000, null(6)=1000, distinct(8)=10, null(8)=7500, distinct(6,8)=10000, null(6,8)=750] └── scan c - ├── columns: c.x:5(int) c.z:6(int!null) c.s:7(string) c.rowid:8(int!null) - ├── stats: [rows=10000, distinct(5)=5000, null(5)=1000, distinct(7)=10, null(7)=7500, distinct(5,7)=10000, null(5,7)=750] - ├── key: (8) - └── fd: (8)-->(5-7) + ├── columns: c.x:6(int) c.z:7(int!null) c.s:8(string) c.rowid:9(int!null) c.crdb_internal_mvcc_timestamp:10(decimal) + ├── stats: [rows=10000, distinct(6)=5000, null(6)=1000, distinct(8)=10, null(8)=7500, distinct(6,8)=10000, null(6,8)=750] + ├── key: (9) + └── fd: (9)-->(6-8,10) build SELECT x FROM b UNION SELECT x FROM c ---- union - ├── columns: x:9(int) + ├── columns: x:11(int) ├── left columns: b.x:1(int) - ├── right columns: c.x:5(int) - ├── stats: [rows=10000, distinct(9)=10000, null(9)=1] - ├── key: (9) + ├── right columns: c.x:6(int) + ├── stats: [rows=10000, distinct(11)=10000, null(11)=1] + ├── key: (11) ├── project │ ├── columns: b.x:1(int) │ ├── stats: [rows=10000, distinct(1)=5000, null(1)=2500] │ └── scan b - │ ├── columns: b.x:1(int) b.z:2(int!null) b.s:3(string) b.rowid:4(int!null) + │ ├── columns: b.x:1(int) b.z:2(int!null) b.s:3(string) b.rowid:4(int!null) b.crdb_internal_mvcc_timestamp:5(decimal) │ ├── stats: [rows=10000, distinct(1)=5000, null(1)=2500] │ ├── key: (4) - │ └── fd: (4)-->(1-3) + │ └── fd: (4)-->(1-3,5) └── project - ├── columns: c.x:5(int) - ├── stats: [rows=10000, distinct(5)=5000, null(5)=1000] + ├── columns: c.x:6(int) + ├── stats: [rows=10000, distinct(6)=5000, null(6)=1000] └── scan c - ├── columns: c.x:5(int) c.z:6(int!null) c.s:7(string) c.rowid:8(int!null) - ├── stats: [rows=10000, distinct(5)=5000, null(5)=1000] - ├── key: (8) - └── fd: (8)-->(5-7) + ├── columns: c.x:6(int) c.z:7(int!null) c.s:8(string) c.rowid:9(int!null) c.crdb_internal_mvcc_timestamp:10(decimal) + ├── stats: [rows=10000, distinct(6)=5000, null(6)=1000] + ├── key: (9) + └── fd: (9)-->(6-8,10) build SELECT x FROM b INTERSECT SELECT x FROM c @@ -717,25 +747,25 @@ SELECT x FROM b INTERSECT SELECT x FROM c intersect ├── columns: x:1(int) ├── left columns: b.x:1(int) - ├── right columns: c.x:5(int) + ├── right columns: c.x:6(int) ├── stats: [rows=5000, distinct(1)=5000, null(1)=1] ├── key: (1) ├── project │ ├── columns: b.x:1(int) │ ├── stats: [rows=10000, distinct(1)=5000, null(1)=2500] │ └── scan b - │ ├── columns: b.x:1(int) b.z:2(int!null) b.s:3(string) b.rowid:4(int!null) + │ ├── columns: b.x:1(int) b.z:2(int!null) b.s:3(string) b.rowid:4(int!null) b.crdb_internal_mvcc_timestamp:5(decimal) │ ├── stats: [rows=10000, distinct(1)=5000, null(1)=2500] │ ├── key: (4) - │ └── fd: (4)-->(1-3) + │ └── fd: (4)-->(1-3,5) └── project - ├── columns: c.x:5(int) - ├── stats: [rows=10000, distinct(5)=5000, null(5)=1000] + ├── columns: c.x:6(int) + ├── stats: [rows=10000, distinct(6)=5000, null(6)=1000] └── scan c - ├── columns: c.x:5(int) c.z:6(int!null) c.s:7(string) c.rowid:8(int!null) - ├── stats: [rows=10000, distinct(5)=5000, null(5)=1000] - ├── key: (8) - └── fd: (8)-->(5-7) + ├── columns: c.x:6(int) c.z:7(int!null) c.s:8(string) c.rowid:9(int!null) c.crdb_internal_mvcc_timestamp:10(decimal) + ├── stats: [rows=10000, distinct(6)=5000, null(6)=1000] + ├── key: (9) + └── fd: (9)-->(6-8,10) build SELECT x FROM b EXCEPT SELECT x FROM c @@ -743,50 +773,50 @@ SELECT x FROM b EXCEPT SELECT x FROM c except ├── columns: x:1(int) ├── left columns: b.x:1(int) - ├── right columns: c.x:5(int) + ├── right columns: c.x:6(int) ├── stats: [rows=5000, distinct(1)=5000, null(1)=0] ├── key: (1) ├── project │ ├── columns: b.x:1(int) │ ├── stats: [rows=10000, distinct(1)=5000, null(1)=2500] │ └── scan b - │ ├── columns: b.x:1(int) b.z:2(int!null) b.s:3(string) b.rowid:4(int!null) + │ ├── columns: b.x:1(int) b.z:2(int!null) b.s:3(string) b.rowid:4(int!null) b.crdb_internal_mvcc_timestamp:5(decimal) │ ├── stats: [rows=10000, distinct(1)=5000, null(1)=2500] │ ├── key: (4) - │ └── fd: (4)-->(1-3) + │ └── fd: (4)-->(1-3,5) └── project - ├── columns: c.x:5(int) - ├── stats: [rows=10000, distinct(5)=5000, null(5)=1000] + ├── columns: c.x:6(int) + ├── stats: [rows=10000, distinct(6)=5000, null(6)=1000] └── scan c - ├── columns: c.x:5(int) c.z:6(int!null) c.s:7(string) c.rowid:8(int!null) - ├── stats: [rows=10000, distinct(5)=5000, null(5)=1000] - ├── key: (8) - └── fd: (8)-->(5-7) + ├── columns: c.x:6(int) c.z:7(int!null) c.s:8(string) c.rowid:9(int!null) c.crdb_internal_mvcc_timestamp:10(decimal) + ├── stats: [rows=10000, distinct(6)=5000, null(6)=1000] + ├── key: (9) + └── fd: (9)-->(6-8,10) -build colstat=9 colstat=10 colstat=(9,10) +build colstat=11 colstat=12 colstat=(11,12) SELECT x,s FROM b UNION ALL SELECT x,s FROM c ---- union-all - ├── columns: x:9(int) s:10(string) + ├── columns: x:11(int) s:12(string) ├── left columns: b.x:1(int) b.s:3(string) - ├── right columns: c.x:5(int) c.s:7(string) - ├── stats: [rows=20000, distinct(9)=10000, null(9)=3500, distinct(10)=20, null(10)=12500, distinct(9,10)=20000, null(9,10)=2000] + ├── right columns: c.x:6(int) c.s:8(string) + ├── stats: [rows=20000, distinct(11)=10000, null(11)=3500, distinct(12)=20, null(12)=12500, distinct(11,12)=20000, null(11,12)=2000] ├── project │ ├── columns: b.x:1(int) b.s:3(string) │ ├── stats: [rows=10000, distinct(1)=5000, null(1)=2500, distinct(3)=10, null(3)=5000, distinct(1,3)=10000, null(1,3)=1250] │ └── scan b - │ ├── columns: b.x:1(int) b.z:2(int!null) b.s:3(string) b.rowid:4(int!null) + │ ├── columns: b.x:1(int) b.z:2(int!null) b.s:3(string) b.rowid:4(int!null) b.crdb_internal_mvcc_timestamp:5(decimal) │ ├── stats: [rows=10000, distinct(1)=5000, null(1)=2500, distinct(3)=10, null(3)=5000, distinct(1,3)=10000, null(1,3)=1250] │ ├── key: (4) - │ └── fd: (4)-->(1-3) + │ └── fd: (4)-->(1-3,5) └── project - ├── columns: c.x:5(int) c.s:7(string) - ├── stats: [rows=10000, distinct(5)=5000, null(5)=1000, distinct(7)=10, null(7)=7500, distinct(5,7)=10000, null(5,7)=750] + ├── columns: c.x:6(int) c.s:8(string) + ├── stats: [rows=10000, distinct(6)=5000, null(6)=1000, distinct(8)=10, null(8)=7500, distinct(6,8)=10000, null(6,8)=750] └── scan c - ├── columns: c.x:5(int) c.z:6(int!null) c.s:7(string) c.rowid:8(int!null) - ├── stats: [rows=10000, distinct(5)=5000, null(5)=1000, distinct(7)=10, null(7)=7500, distinct(5,7)=10000, null(5,7)=750] - ├── key: (8) - └── fd: (8)-->(5-7) + ├── columns: c.x:6(int) c.z:7(int!null) c.s:8(string) c.rowid:9(int!null) c.crdb_internal_mvcc_timestamp:10(decimal) + ├── stats: [rows=10000, distinct(6)=5000, null(6)=1000, distinct(8)=10, null(8)=7500, distinct(6,8)=10000, null(6,8)=750] + ├── key: (9) + └── fd: (9)-->(6-8,10) build colstat=1 colstat=3 colstat=(1,3) SELECT x,s FROM b INTERSECT ALL SELECT x,s FROM c @@ -794,24 +824,24 @@ SELECT x,s FROM b INTERSECT ALL SELECT x,s FROM c intersect-all ├── columns: x:1(int) s:3(string) ├── left columns: b.x:1(int) b.s:3(string) - ├── right columns: c.x:5(int) c.s:7(string) + ├── right columns: c.x:6(int) c.s:8(string) ├── stats: [rows=10000, distinct(1)=5000, null(1)=1000, distinct(3)=10, null(3)=5000, distinct(1,3)=10000, null(1,3)=750] ├── project │ ├── columns: b.x:1(int) b.s:3(string) │ ├── stats: [rows=10000, distinct(1)=5000, null(1)=2500, distinct(3)=10, null(3)=5000, distinct(1,3)=10000, null(1,3)=1250] │ └── scan b - │ ├── columns: b.x:1(int) b.z:2(int!null) b.s:3(string) b.rowid:4(int!null) + │ ├── columns: b.x:1(int) b.z:2(int!null) b.s:3(string) b.rowid:4(int!null) b.crdb_internal_mvcc_timestamp:5(decimal) │ ├── stats: [rows=10000, distinct(1)=5000, null(1)=2500, distinct(3)=10, null(3)=5000, distinct(1,3)=10000, null(1,3)=1250] │ ├── key: (4) - │ └── fd: (4)-->(1-3) + │ └── fd: (4)-->(1-3,5) └── project - ├── columns: c.x:5(int) c.s:7(string) - ├── stats: [rows=10000, distinct(5)=5000, null(5)=1000, distinct(7)=10, null(7)=7500, distinct(5,7)=10000, null(5,7)=750] + ├── columns: c.x:6(int) c.s:8(string) + ├── stats: [rows=10000, distinct(6)=5000, null(6)=1000, distinct(8)=10, null(8)=7500, distinct(6,8)=10000, null(6,8)=750] └── scan c - ├── columns: c.x:5(int) c.z:6(int!null) c.s:7(string) c.rowid:8(int!null) - ├── stats: [rows=10000, distinct(5)=5000, null(5)=1000, distinct(7)=10, null(7)=7500, distinct(5,7)=10000, null(5,7)=750] - ├── key: (8) - └── fd: (8)-->(5-7) + ├── columns: c.x:6(int) c.z:7(int!null) c.s:8(string) c.rowid:9(int!null) c.crdb_internal_mvcc_timestamp:10(decimal) + ├── stats: [rows=10000, distinct(6)=5000, null(6)=1000, distinct(8)=10, null(8)=7500, distinct(6,8)=10000, null(6,8)=750] + ├── key: (9) + └── fd: (9)-->(6-8,10) build colstat=1 colstat=3 colstat=(1,3) SELECT x,s FROM b EXCEPT ALL SELECT x,s FROM c @@ -819,24 +849,24 @@ SELECT x,s FROM b EXCEPT ALL SELECT x,s FROM c except-all ├── columns: x:1(int) s:3(string) ├── left columns: b.x:1(int) b.s:3(string) - ├── right columns: c.x:5(int) c.s:7(string) + ├── right columns: c.x:6(int) c.s:8(string) ├── stats: [rows=10000, distinct(1)=5000, null(1)=1500, distinct(3)=10, null(3)=0, distinct(1,3)=10000, null(1,3)=500] ├── project │ ├── columns: b.x:1(int) b.s:3(string) │ ├── stats: [rows=10000, distinct(1)=5000, null(1)=2500, distinct(3)=10, null(3)=5000, distinct(1,3)=10000, null(1,3)=1250] │ └── scan b - │ ├── columns: b.x:1(int) b.z:2(int!null) b.s:3(string) b.rowid:4(int!null) + │ ├── columns: b.x:1(int) b.z:2(int!null) b.s:3(string) b.rowid:4(int!null) b.crdb_internal_mvcc_timestamp:5(decimal) │ ├── stats: [rows=10000, distinct(1)=5000, null(1)=2500, distinct(3)=10, null(3)=5000, distinct(1,3)=10000, null(1,3)=1250] │ ├── key: (4) - │ └── fd: (4)-->(1-3) + │ └── fd: (4)-->(1-3,5) └── project - ├── columns: c.x:5(int) c.s:7(string) - ├── stats: [rows=10000, distinct(5)=5000, null(5)=1000, distinct(7)=10, null(7)=7500, distinct(5,7)=10000, null(5,7)=750] + ├── columns: c.x:6(int) c.s:8(string) + ├── stats: [rows=10000, distinct(6)=5000, null(6)=1000, distinct(8)=10, null(8)=7500, distinct(6,8)=10000, null(6,8)=750] └── scan c - ├── columns: c.x:5(int) c.z:6(int!null) c.s:7(string) c.rowid:8(int!null) - ├── stats: [rows=10000, distinct(5)=5000, null(5)=1000, distinct(7)=10, null(7)=7500, distinct(5,7)=10000, null(5,7)=750] - ├── key: (8) - └── fd: (8)-->(5-7) + ├── columns: c.x:6(int) c.z:7(int!null) c.s:8(string) c.rowid:9(int!null) c.crdb_internal_mvcc_timestamp:10(decimal) + ├── stats: [rows=10000, distinct(6)=5000, null(6)=1000, distinct(8)=10, null(8)=7500, distinct(6,8)=10000, null(6,8)=750] + ├── key: (9) + └── fd: (9)-->(6-8,10) # Regression test for #35715. opt colstat=(5,2) diff --git a/pkg/sql/opt/memo/testdata/stats/srfs b/pkg/sql/opt/memo/testdata/stats/srfs index 2e54b6c909b5..c2b93bbc623f 100644 --- a/pkg/sql/opt/memo/testdata/stats/srfs +++ b/pkg/sql/opt/memo/testdata/stats/srfs @@ -92,11 +92,11 @@ opt colstat=3 colstat=(1,2,3) SELECT unnest(ARRAY[x,y]) FROM xy ---- project - ├── columns: unnest:3(int) - ├── stats: [rows=2000, distinct(3)=2, null(3)=0, distinct(1-3)=2000, null(1-3)=0] + ├── columns: unnest:4(int) + ├── stats: [rows=2000, distinct(3)=1, null(3)=2000, distinct(1-3)=1000, null(1-3)=0] └── inner-join-apply - ├── columns: x:1(int!null) y:2(int) unnest:3(int) - ├── stats: [rows=2000, distinct(3)=2, null(3)=0, distinct(1-3)=2000, null(1-3)=0] + ├── columns: x:1(int!null) y:2(int) unnest:4(int) + ├── stats: [rows=2000, distinct(1,2)=1000, null(1,2)=0] ├── fd: (1)-->(2) ├── scan xy │ ├── columns: x:1(int!null) y:2(int) @@ -104,21 +104,21 @@ project │ ├── key: (1) │ └── fd: (1)-->(2) ├── values - │ ├── columns: unnest:3(int) + │ ├── columns: unnest:4(int) │ ├── outer: (1,2) │ ├── cardinality: [2 - 2] - │ ├── stats: [rows=2, distinct(3)=2, null(3)=0] + │ ├── stats: [rows=2] │ ├── (x:1,) [type=tuple{int}] │ └── (y:2,) [type=tuple{int}] └── filters (true) -opt colstat=3 colstat=4 colstat=(3, 4) colstat=(1, 3) colstat=(2, 4) +opt colstat=4 colstat=5 colstat=(4, 5) colstat=(1, 4) colstat=(2, 5) SELECT xy.*, generate_series(x, y), generate_series(0, 1) FROM xy ---- project-set - ├── columns: x:1(int!null) y:2(int) generate_series:3(int) generate_series:4(int) + ├── columns: x:1(int!null) y:2(int) generate_series:4(int) generate_series:5(int) ├── immutable - ├── stats: [rows=10000, distinct(3)=700, null(3)=100, distinct(4)=7, null(4)=100, distinct(1,3)=10000, null(1,3)=0, distinct(2,4)=700, null(2,4)=1, distinct(3,4)=4900, null(3,4)=1] + ├── stats: [rows=10000, distinct(4)=700, null(4)=100, distinct(5)=7, null(5)=100, distinct(1,4)=10000, null(1,4)=0, distinct(2,5)=700, null(2,5)=1, distinct(4,5)=4900, null(4,5)=1] ├── fd: (1)-->(2) ├── scan xy │ ├── columns: x:1(int!null) y:2(int) @@ -165,14 +165,14 @@ distinct-on ├── stats: [rows=9.85601173, distinct(1)=9.85601173, null(1)=0] ├── key: (1) └── select - ├── columns: id:1(int!null) title:4(string!null) tag_list:6(string[]) upper:10(string!null) unnest:11(string) generate_series:12(int) lower:13(string) + ├── columns: id:1(int!null) title:4(string!null) tag_list:6(string[]) upper:11(string!null) unnest:12(string) generate_series:13(int) lower:14(string) ├── immutable - ├── stats: [rows=9.9, distinct(1)=9.85601173, null(1)=0, distinct(4)=9.9, null(4)=0, distinct(10)=9.9, null(10)=0] - ├── fd: (1)-->(4,6), (4)==(10), (10)==(4) + ├── stats: [rows=9.9, distinct(1)=9.85601173, null(1)=0, distinct(4)=9.9, null(4)=0, distinct(11)=9.9, null(11)=0] + ├── fd: (1)-->(4,6), (4)==(11), (11)==(4) ├── project-set - │ ├── columns: id:1(int!null) title:4(string) tag_list:6(string[]) upper:10(string) unnest:11(string) generate_series:12(int) lower:13(string) + │ ├── columns: id:1(int!null) title:4(string) tag_list:6(string[]) upper:11(string) unnest:12(string) generate_series:13(int) lower:14(string) │ ├── immutable - │ ├── stats: [rows=10000, distinct(1)=1000, null(1)=0, distinct(4)=100, null(4)=100, distinct(10)=100, null(10)=9000] + │ ├── stats: [rows=10000, distinct(1)=1000, null(1)=0, distinct(4)=100, null(4)=100, distinct(11)=100, null(11)=9000] │ ├── fd: (1)-->(4,6) │ ├── scan articles │ │ ├── columns: id:1(int!null) title:4(string) tag_list:6(string[]) @@ -185,7 +185,7 @@ distinct-on │ ├── generate_series(0, 1) [type=int, immutable] │ └── 'abc' [type=string] └── filters - └── title:4 = upper:10 [type=bool, outer=(4,10), constraints=(/4: (/NULL - ]; /10: (/NULL - ]), fd=(4)==(10), (10)==(4)] + └── title:4 = upper:11 [type=bool, outer=(4,11), constraints=(/4: (/NULL - ]; /11: (/NULL - ]), fd=(4)==(11), (11)==(4)] opt SELECT id FROM articles WHERE title = ANY( @@ -199,14 +199,14 @@ distinct-on ├── stats: [rows=13.9135391, distinct(1)=13.9135391, null(1)=0] ├── key: (1) └── select - ├── columns: id:1(int!null) title:4(string!null) tag_list:6(string[]) upper:10(string) unnest:11(string!null) generate_series:12(int) lower:13(string) + ├── columns: id:1(int!null) title:4(string!null) tag_list:6(string[]) upper:11(string) unnest:12(string!null) generate_series:13(int) lower:14(string) ├── immutable - ├── stats: [rows=14.0014286, distinct(1)=13.9135391, null(1)=0, distinct(4)=14.0014286, null(4)=0, distinct(11)=14.0014286, null(11)=0] - ├── fd: (1)-->(4,6), (4)==(11), (11)==(4) + ├── stats: [rows=14.0014286, distinct(1)=13.9135391, null(1)=0, distinct(4)=14.0014286, null(4)=0, distinct(12)=14.0014286, null(12)=0] + ├── fd: (1)-->(4,6), (4)==(12), (12)==(4) ├── project-set - │ ├── columns: id:1(int!null) title:4(string) tag_list:6(string[]) upper:10(string) unnest:11(string) generate_series:12(int) lower:13(string) + │ ├── columns: id:1(int!null) title:4(string) tag_list:6(string[]) upper:11(string) unnest:12(string) generate_series:13(int) lower:14(string) │ ├── immutable - │ ├── stats: [rows=10000, distinct(1)=1000, null(1)=0, distinct(4)=100, null(4)=100, distinct(11)=700, null(11)=100] + │ ├── stats: [rows=10000, distinct(1)=1000, null(1)=0, distinct(4)=100, null(4)=100, distinct(12)=700, null(12)=100] │ ├── fd: (1)-->(4,6) │ ├── scan articles │ │ ├── columns: id:1(int!null) title:4(string) tag_list:6(string[]) @@ -219,7 +219,7 @@ distinct-on │ ├── generate_series(0, 1) [type=int, immutable] │ └── 'abc' [type=string] └── filters - └── title:4 = unnest:11 [type=bool, outer=(4,11), constraints=(/4: (/NULL - ]; /11: (/NULL - ]), fd=(4)==(11), (11)==(4)] + └── title:4 = unnest:12 [type=bool, outer=(4,12), constraints=(/4: (/NULL - ]; /12: (/NULL - ]), fd=(4)==(12), (12)==(4)] opt SELECT id FROM articles WHERE id = ANY( @@ -233,14 +233,14 @@ distinct-on ├── stats: [rows=6, distinct(1)=6, null(1)=0] ├── key: (1) └── select - ├── columns: id:1(int!null) title:4(string) tag_list:6(string[]) upper:10(string) unnest:11(string) generate_series:12(int!null) lower:13(string) + ├── columns: id:1(int!null) title:4(string) tag_list:6(string[]) upper:11(string) unnest:12(string) generate_series:13(int!null) lower:14(string) ├── immutable - ├── stats: [rows=9.9, distinct(1)=6, null(1)=0, distinct(12)=6, null(12)=0] - ├── fd: (1)-->(4,6), (1)==(12), (12)==(1) + ├── stats: [rows=9.9, distinct(1)=6, null(1)=0, distinct(13)=6, null(13)=0] + ├── fd: (1)-->(4,6), (1)==(13), (13)==(1) ├── project-set - │ ├── columns: id:1(int!null) title:4(string) tag_list:6(string[]) upper:10(string) unnest:11(string) generate_series:12(int) lower:13(string) + │ ├── columns: id:1(int!null) title:4(string) tag_list:6(string[]) upper:11(string) unnest:12(string) generate_series:13(int) lower:14(string) │ ├── immutable - │ ├── stats: [rows=10000, distinct(1)=1000, null(1)=0, distinct(12)=7, null(12)=100] + │ ├── stats: [rows=10000, distinct(1)=1000, null(1)=0, distinct(13)=7, null(13)=100] │ ├── fd: (1)-->(4,6) │ ├── scan articles │ │ ├── columns: id:1(int!null) title:4(string) tag_list:6(string[]) @@ -253,7 +253,7 @@ distinct-on │ ├── generate_series(0, 1) [type=int, immutable] │ └── 'abc' [type=string] └── filters - └── id:1 = generate_series:12 [type=bool, outer=(1,12), constraints=(/1: (/NULL - ]; /12: (/NULL - ]), fd=(1)==(12), (12)==(1)] + └── id:1 = generate_series:13 [type=bool, outer=(1,13), constraints=(/1: (/NULL - ]; /13: (/NULL - ]), fd=(1)==(13), (13)==(1)] opt SELECT id FROM articles WHERE title = ANY( @@ -267,14 +267,14 @@ distinct-on ├── stats: [rows=9.85601173, distinct(1)=9.85601173, null(1)=0] ├── key: (1) └── select - ├── columns: id:1(int!null) title:4(string!null) tag_list:6(string[]) upper:10(string) unnest:11(string) generate_series:12(int) lower:13(string!null) + ├── columns: id:1(int!null) title:4(string!null) tag_list:6(string[]) upper:11(string) unnest:12(string) generate_series:13(int) lower:14(string!null) ├── immutable - ├── stats: [rows=9.9, distinct(1)=9.85601173, null(1)=0, distinct(4)=1e-10, null(4)=0, distinct(13)=1e-10, null(13)=0] - ├── fd: (1)-->(4,6), (4)==(13), (13)==(4) + ├── stats: [rows=9.9, distinct(1)=9.85601173, null(1)=0, distinct(4)=1e-10, null(4)=0, distinct(14)=1e-10, null(14)=0] + ├── fd: (1)-->(4,6), (4)==(14), (14)==(4) ├── project-set - │ ├── columns: id:1(int!null) title:4(string) tag_list:6(string[]) upper:10(string) unnest:11(string) generate_series:12(int) lower:13(string) + │ ├── columns: id:1(int!null) title:4(string) tag_list:6(string[]) upper:11(string) unnest:12(string) generate_series:13(int) lower:14(string) │ ├── immutable - │ ├── stats: [rows=10000, distinct(1)=1000, null(1)=0, distinct(4)=100, null(4)=100, distinct(13)=1, null(13)=9000] + │ ├── stats: [rows=10000, distinct(1)=1000, null(1)=0, distinct(4)=100, null(4)=100, distinct(14)=1, null(14)=9000] │ ├── fd: (1)-->(4,6) │ ├── scan articles │ │ ├── columns: id:1(int!null) title:4(string) tag_list:6(string[]) @@ -287,4 +287,4 @@ distinct-on │ ├── generate_series(0, 1) [type=int, immutable] │ └── 'abc' [type=string] └── filters - └── title:4 = lower:13 [type=bool, outer=(4,13), constraints=(/4: (/NULL - ]; /13: (/NULL - ]), fd=(4)==(13), (13)==(4)] + └── title:4 = lower:14 [type=bool, outer=(4,14), constraints=(/4: (/NULL - ]; /14: (/NULL - ]), fd=(4)==(14), (14)==(4)] diff --git a/pkg/sql/opt/memo/testdata/stats/update b/pkg/sql/opt/memo/testdata/stats/update index 917e941b4e29..dca69b76cb3d 100644 --- a/pkg/sql/opt/memo/testdata/stats/update +++ b/pkg/sql/opt/memo/testdata/stats/update @@ -39,55 +39,55 @@ FROM [UPDATE xyz SET y=5 WHERE z=5.5 RETURNING *] WHERE x > 'foo' ---- with &1 - ├── columns: x:8(string!null) y:9(int!null) z:10(float!null) + ├── columns: x:10(string!null) y:11(int!null) z:12(float!null) ├── volatile, mutations - ├── stats: [rows=3.33333333, distinct(8)=3.33333333, null(8)=0] - ├── key: (8) - ├── fd: ()-->(9,10) + ├── stats: [rows=3.33333333, distinct(10)=3.33333333, null(10)=0] + ├── key: (10) + ├── fd: ()-->(11,12) ├── update xyz │ ├── columns: xyz.x:1(string!null) xyz.y:2(int!null) xyz.z:3(float!null) - │ ├── fetch columns: xyz.x:4(string) xyz.y:5(int) xyz.z:6(float) + │ ├── fetch columns: xyz.x:5(string) xyz.y:6(int) xyz.z:7(float) │ ├── update-mapping: - │ │ └── y_new:7 => xyz.y:2 + │ │ └── y_new:9 => xyz.y:2 │ ├── volatile, mutations │ ├── stats: [rows=10] │ ├── key: (1) │ ├── fd: ()-->(2,3) │ └── project - │ ├── columns: y_new:7(int!null) xyz.x:4(string!null) xyz.y:5(int!null) xyz.z:6(float!null) + │ ├── columns: y_new:9(int!null) xyz.x:5(string!null) xyz.y:6(int!null) xyz.z:7(float!null) crdb_internal_mvcc_timestamp:8(decimal) │ ├── stats: [rows=10] - │ ├── key: (4) - │ ├── fd: ()-->(6,7), (4)-->(5) + │ ├── key: (5) + │ ├── fd: ()-->(7,9), (5)-->(6,8) │ ├── select - │ │ ├── columns: xyz.x:4(string!null) xyz.y:5(int!null) xyz.z:6(float!null) - │ │ ├── stats: [rows=10, distinct(6)=1, null(6)=0] - │ │ ├── key: (4) - │ │ ├── fd: ()-->(6), (4)-->(5) + │ │ ├── columns: xyz.x:5(string!null) xyz.y:6(int!null) xyz.z:7(float!null) crdb_internal_mvcc_timestamp:8(decimal) + │ │ ├── stats: [rows=10, distinct(7)=1, null(7)=0] + │ │ ├── key: (5) + │ │ ├── fd: ()-->(7), (5)-->(6,8) │ │ ├── scan xyz - │ │ │ ├── columns: xyz.x:4(string!null) xyz.y:5(int!null) xyz.z:6(float) - │ │ │ ├── stats: [rows=1000, distinct(4)=1000, null(4)=0, distinct(5)=100, null(5)=0, distinct(6)=100, null(6)=10] - │ │ │ ├── key: (4) - │ │ │ └── fd: (4)-->(5,6) + │ │ │ ├── columns: xyz.x:5(string!null) xyz.y:6(int!null) xyz.z:7(float) crdb_internal_mvcc_timestamp:8(decimal) + │ │ │ ├── stats: [rows=1000, distinct(5)=1000, null(5)=0, distinct(6)=100, null(6)=0, distinct(7)=100, null(7)=10] + │ │ │ ├── key: (5) + │ │ │ └── fd: (5)-->(6-8) │ │ └── filters - │ │ └── xyz.z:6 = 5.5 [type=bool, outer=(6), constraints=(/6: [/5.5 - /5.5]; tight), fd=()-->(6)] + │ │ └── xyz.z:7 = 5.5 [type=bool, outer=(7), constraints=(/7: [/5.5 - /5.5]; tight), fd=()-->(7)] │ └── projections - │ └── 5 [as=y_new:7, type=int] + │ └── 5 [as=y_new:9, type=int] └── select - ├── columns: x:8(string!null) y:9(int!null) z:10(float!null) - ├── stats: [rows=3.33333333, distinct(8)=3.33333333, null(8)=0] - ├── key: (8) - ├── fd: ()-->(9,10) + ├── columns: x:10(string!null) y:11(int!null) z:12(float!null) + ├── stats: [rows=3.33333333, distinct(10)=3.33333333, null(10)=0] + ├── key: (10) + ├── fd: ()-->(11,12) ├── with-scan &1 - │ ├── columns: x:8(string!null) y:9(int!null) z:10(float!null) + │ ├── columns: x:10(string!null) y:11(int!null) z:12(float!null) │ ├── mapping: - │ │ ├── xyz.x:1(string) => x:8(string) - │ │ ├── xyz.y:2(int) => y:9(int) - │ │ └── xyz.z:3(float) => z:10(float) - │ ├── stats: [rows=10, distinct(8)=10, null(8)=0, distinct(9)=1, null(9)=0, distinct(10)=1, null(10)=0] - │ ├── key: (8) - │ └── fd: ()-->(9,10) + │ │ ├── xyz.x:1(string) => x:10(string) + │ │ ├── xyz.y:2(int) => y:11(int) + │ │ └── xyz.z:3(float) => z:12(float) + │ ├── stats: [rows=10, distinct(10)=10, null(10)=0, distinct(11)=1, null(11)=0, distinct(12)=1, null(12)=0] + │ ├── key: (10) + │ └── fd: ()-->(11,12) └── filters - └── x:8 > 'foo' [type=bool, outer=(8), constraints=(/8: [/e'foo\x00' - ]; tight)] + └── x:10 > 'foo' [type=bool, outer=(10), constraints=(/10: [/e'foo\x00' - ]; tight)] # Cardinality is zero. build @@ -95,31 +95,31 @@ UPDATE xyz SET x='foo' WHERE False RETURNING * ---- update xyz ├── columns: x:1(string!null) y:2(int!null) z:3(float) - ├── fetch columns: x:4(string) y:5(int) z:6(float) + ├── fetch columns: x:5(string) y:6(int) z:7(float) ├── update-mapping: - │ └── x_new:7 => x:1 + │ └── x_new:9 => x:1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── stats: [rows=0] ├── fd: ()-->(1) └── project - ├── columns: x_new:7(string!null) x:4(string!null) y:5(int!null) z:6(float) + ├── columns: x_new:9(string!null) x:5(string!null) y:6(int!null) z:7(float) crdb_internal_mvcc_timestamp:8(decimal) ├── cardinality: [0 - 0] ├── stats: [rows=0] - ├── key: (4) - ├── fd: ()-->(7), (4)-->(5,6) + ├── key: (5) + ├── fd: ()-->(9), (5)-->(6-8) ├── select - │ ├── columns: x:4(string!null) y:5(int!null) z:6(float) + │ ├── columns: x:5(string!null) y:6(int!null) z:7(float) crdb_internal_mvcc_timestamp:8(decimal) │ ├── cardinality: [0 - 0] │ ├── stats: [rows=0] - │ ├── key: (4) - │ ├── fd: (4)-->(5,6) + │ ├── key: (5) + │ ├── fd: (5)-->(6-8) │ ├── scan xyz - │ │ ├── columns: x:4(string!null) y:5(int!null) z:6(float) + │ │ ├── columns: x:5(string!null) y:6(int!null) z:7(float) crdb_internal_mvcc_timestamp:8(decimal) │ │ ├── stats: [rows=1000] - │ │ ├── key: (4) - │ │ └── fd: (4)-->(5,6) + │ │ ├── key: (5) + │ │ └── fd: (5)-->(6-8) │ └── filters │ └── false [type=bool] └── projections - └── 'foo' [as=x_new:7, type=string] + └── 'foo' [as=x_new:9, type=string] diff --git a/pkg/sql/opt/memo/testdata/stats/upsert b/pkg/sql/opt/memo/testdata/stats/upsert index 6bd0f194f731..56e50b7d361f 100644 --- a/pkg/sql/opt/memo/testdata/stats/upsert +++ b/pkg/sql/opt/memo/testdata/stats/upsert @@ -89,109 +89,109 @@ FROM WHERE y=10 ---- with &1 - ├── columns: x:16(string!null) y:17(int!null) z:18(float) + ├── columns: x:19(string!null) y:20(int!null) z:21(float) ├── volatile, mutations - ├── stats: [rows=9.94974874, distinct(17)=0.994974874, null(17)=0] - ├── fd: ()-->(17) + ├── stats: [rows=9.94974874, distinct(20)=0.994974874, null(20)=0] + ├── fd: ()-->(20) ├── upsert xyz │ ├── columns: xyz.x:1(string!null) xyz.y:2(int!null) xyz.z:3(float) - │ ├── canary column: 9 - │ ├── fetch columns: xyz.x:9(string) xyz.y:10(int) xyz.z:11(float) + │ ├── canary column: 11 + │ ├── fetch columns: xyz.x:11(string) xyz.y:12(int) xyz.z:13(float) │ ├── insert-mapping: - │ │ ├── b:5 => xyz.x:1 - │ │ ├── a:4 => xyz.y:2 - │ │ └── column8:8 => xyz.z:3 + │ │ ├── b:6 => xyz.x:1 + │ │ ├── a:5 => xyz.y:2 + │ │ └── column10:10 => xyz.z:3 │ ├── update-mapping: - │ │ └── upsert_y:14 => xyz.y:2 + │ │ └── upsert_y:17 => xyz.y:2 │ ├── return-mapping: - │ │ ├── upsert_x:13 => xyz.x:1 - │ │ ├── upsert_y:14 => xyz.y:2 - │ │ └── upsert_z:15 => xyz.z:3 + │ │ ├── upsert_x:16 => xyz.x:1 + │ │ ├── upsert_y:17 => xyz.y:2 + │ │ └── upsert_z:18 => xyz.z:3 │ ├── volatile, mutations │ ├── stats: [rows=9.94974874] │ └── project - │ ├── columns: upsert_x:13(string) upsert_y:14(int!null) upsert_z:15(float) a:4(int!null) b:5(string) column8:8(float) xyz.x:9(string) xyz.y:10(int) xyz.z:11(float) y_new:12(int!null) + │ ├── columns: upsert_x:16(string) upsert_y:17(int!null) upsert_z:18(float) a:5(int!null) b:6(string) column10:10(float) xyz.x:11(string) xyz.y:12(int) xyz.z:13(float) xyz.crdb_internal_mvcc_timestamp:14(decimal) y_new:15(int!null) │ ├── immutable │ ├── stats: [rows=9.94974874] - │ ├── lax-key: (5,9) - │ ├── fd: ()-->(8,12), (5)~~>(4), (9)-->(10,11), (5,9)-->(13), (4,9)-->(14), (5,9)~~>(4,14,15) + │ ├── lax-key: (6,11) + │ ├── fd: ()-->(10,15), (6)~~>(5), (11)-->(12-14), (6,11)-->(16), (5,11)-->(17), (6,11)~~>(5,17,18) │ ├── project - │ │ ├── columns: y_new:12(int!null) a:4(int!null) b:5(string) column8:8(float) xyz.x:9(string) xyz.y:10(int) xyz.z:11(float) + │ │ ├── columns: y_new:15(int!null) a:5(int!null) b:6(string) column10:10(float) xyz.x:11(string) xyz.y:12(int) xyz.z:13(float) xyz.crdb_internal_mvcc_timestamp:14(decimal) │ │ ├── immutable │ │ ├── stats: [rows=9.94974874] - │ │ ├── lax-key: (5,9) - │ │ ├── fd: ()-->(8,12), (5)~~>(4), (9)-->(10,11) + │ │ ├── lax-key: (6,11) + │ │ ├── fd: ()-->(10,15), (6)~~>(5), (11)-->(12-14) │ │ ├── left-join (hash) - │ │ │ ├── columns: a:4(int!null) b:5(string) column8:8(float) xyz.x:9(string) xyz.y:10(int) xyz.z:11(float) + │ │ │ ├── columns: a:5(int!null) b:6(string) column10:10(float) xyz.x:11(string) xyz.y:12(int) xyz.z:13(float) xyz.crdb_internal_mvcc_timestamp:14(decimal) │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ │ ├── immutable - │ │ │ ├── stats: [rows=9.94974874, distinct(9)=9.94974874, null(9)=0] - │ │ │ ├── lax-key: (5,9) - │ │ │ ├── fd: ()-->(8), (5)~~>(4), (9)-->(10,11) + │ │ │ ├── stats: [rows=9.94974874, distinct(11)=9.94974874, null(11)=0] + │ │ │ ├── lax-key: (6,11) + │ │ │ ├── fd: ()-->(10), (6)~~>(5), (11)-->(12-14) │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ ├── columns: a:4(int!null) b:5(string) column8:8(float) - │ │ │ │ ├── grouping columns: b:5(string) + │ │ │ │ ├── columns: a:5(int!null) b:6(string) column10:10(float) + │ │ │ │ ├── grouping columns: b:6(string) │ │ │ │ ├── error: "UPSERT or INSERT...ON CONFLICT command cannot affect row a second time" │ │ │ │ ├── immutable - │ │ │ │ ├── stats: [rows=9.94974874, distinct(4)=6.31184239, null(4)=0, distinct(5)=9.94974874, null(5)=0] - │ │ │ │ ├── lax-key: (5) - │ │ │ │ ├── fd: ()-->(8), (5)~~>(4,8) + │ │ │ │ ├── stats: [rows=9.94974874, distinct(5)=6.31184239, null(5)=0, distinct(6)=9.94974874, null(6)=0] + │ │ │ │ ├── lax-key: (6) + │ │ │ │ ├── fd: ()-->(10), (6)~~>(5,10) │ │ │ │ ├── project - │ │ │ │ │ ├── columns: column8:8(float) a:4(int!null) b:5(string) + │ │ │ │ │ ├── columns: column10:10(float) a:5(int!null) b:6(string) │ │ │ │ │ ├── immutable - │ │ │ │ │ ├── stats: [rows=9.94974874, distinct(5)=6.31184239, null(5)=0] - │ │ │ │ │ ├── fd: ()-->(8) + │ │ │ │ │ ├── stats: [rows=9.94974874, distinct(6)=6.31184239, null(6)=0] + │ │ │ │ │ ├── fd: ()-->(10) │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: a:4(int!null) b:5(string) - │ │ │ │ │ │ ├── stats: [rows=9.94974874, distinct(5)=6.31184239, null(5)=0] + │ │ │ │ │ │ ├── columns: a:5(int!null) b:6(string) + │ │ │ │ │ │ ├── stats: [rows=9.94974874, distinct(6)=6.31184239, null(6)=0] │ │ │ │ │ │ └── select - │ │ │ │ │ │ ├── columns: a:4(int!null) b:5(string) c:6(float!null) rowid:7(int!null) - │ │ │ │ │ │ ├── stats: [rows=9.94974874, distinct(5)=6.31184239, null(5)=0, distinct(6)=1, null(6)=0] - │ │ │ │ │ │ ├── key: (7) - │ │ │ │ │ │ ├── fd: ()-->(6), (7)-->(4,5) + │ │ │ │ │ │ ├── columns: a:5(int!null) b:6(string) c:7(float!null) rowid:8(int!null) abc.crdb_internal_mvcc_timestamp:9(decimal) + │ │ │ │ │ │ ├── stats: [rows=9.94974874, distinct(6)=6.31184239, null(6)=0, distinct(7)=1, null(7)=0] + │ │ │ │ │ │ ├── key: (8) + │ │ │ │ │ │ ├── fd: ()-->(7), (8)-->(5,6,9) │ │ │ │ │ │ ├── scan abc - │ │ │ │ │ │ │ ├── columns: a:4(int!null) b:5(string) c:6(float) rowid:7(int!null) + │ │ │ │ │ │ │ ├── columns: a:5(int!null) b:6(string) c:7(float) rowid:8(int!null) abc.crdb_internal_mvcc_timestamp:9(decimal) │ │ │ │ │ │ │ ├── computed column expressions - │ │ │ │ │ │ │ │ └── c:6 - │ │ │ │ │ │ │ │ └── a:4::FLOAT8 [type=float] - │ │ │ │ │ │ │ ├── stats: [rows=2000, distinct(4)=2000, null(4)=0, distinct(5)=10, null(5)=0, distinct(6)=200, null(6)=20, distinct(7)=2000, null(7)=0] - │ │ │ │ │ │ │ ├── key: (7) - │ │ │ │ │ │ │ └── fd: (7)-->(4-6) + │ │ │ │ │ │ │ │ └── c:7 + │ │ │ │ │ │ │ │ └── a:5::FLOAT8 [type=float] + │ │ │ │ │ │ │ ├── stats: [rows=2000, distinct(5)=2000, null(5)=0, distinct(6)=10, null(6)=0, distinct(7)=200, null(7)=20, distinct(8)=2000, null(8)=0] + │ │ │ │ │ │ │ ├── key: (8) + │ │ │ │ │ │ │ └── fd: (8)-->(5-7,9) │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── c:6 = 1.0 [type=bool, outer=(6), constraints=(/6: [/1.0 - /1.0]; tight), fd=()-->(6)] + │ │ │ │ │ │ └── c:7 = 1.0 [type=bool, outer=(7), constraints=(/7: [/1.0 - /1.0]; tight), fd=()-->(7)] │ │ │ │ │ └── projections - │ │ │ │ │ └── NULL::FLOAT8 [as=column8:8, type=float, immutable] + │ │ │ │ │ └── NULL::FLOAT8 [as=column10:10, type=float, immutable] │ │ │ │ └── aggregations - │ │ │ │ ├── first-agg [as=a:4, type=int, outer=(4)] - │ │ │ │ │ └── a:4 [type=int] - │ │ │ │ └── first-agg [as=column8:8, type=float, outer=(8)] - │ │ │ │ └── column8:8 [type=float] + │ │ │ │ ├── first-agg [as=a:5, type=int, outer=(5)] + │ │ │ │ │ └── a:5 [type=int] + │ │ │ │ └── first-agg [as=column10:10, type=float, outer=(10)] + │ │ │ │ └── column10:10 [type=float] │ │ │ ├── scan xyz - │ │ │ │ ├── columns: xyz.x:9(string!null) xyz.y:10(int!null) xyz.z:11(float) - │ │ │ │ ├── stats: [rows=1000, distinct(9)=1000, null(9)=0] - │ │ │ │ ├── key: (9) - │ │ │ │ └── fd: (9)-->(10,11) + │ │ │ │ ├── columns: xyz.x:11(string!null) xyz.y:12(int!null) xyz.z:13(float) xyz.crdb_internal_mvcc_timestamp:14(decimal) + │ │ │ │ ├── stats: [rows=1000, distinct(11)=1000, null(11)=0] + │ │ │ │ ├── key: (11) + │ │ │ │ └── fd: (11)-->(12-14) │ │ │ └── filters - │ │ │ └── b:5 = xyz.x:9 [type=bool, outer=(5,9), constraints=(/5: (/NULL - ]; /9: (/NULL - ]), fd=(5)==(9), (9)==(5)] + │ │ │ └── b:6 = xyz.x:11 [type=bool, outer=(6,11), constraints=(/6: (/NULL - ]; /11: (/NULL - ]), fd=(6)==(11), (11)==(6)] │ │ └── projections - │ │ └── 5 [as=y_new:12, type=int] + │ │ └── 5 [as=y_new:15, type=int] │ └── projections - │ ├── CASE WHEN xyz.x:9 IS NULL THEN b:5 ELSE xyz.x:9 END [as=upsert_x:13, type=string, outer=(5,9)] - │ ├── CASE WHEN xyz.x:9 IS NULL THEN a:4 ELSE y_new:12 END [as=upsert_y:14, type=int, outer=(4,9,12)] - │ └── CASE WHEN xyz.x:9 IS NULL THEN column8:8 ELSE xyz.z:11 END [as=upsert_z:15, type=float, outer=(8,9,11)] + │ ├── CASE WHEN xyz.x:11 IS NULL THEN b:6 ELSE xyz.x:11 END [as=upsert_x:16, type=string, outer=(6,11)] + │ ├── CASE WHEN xyz.x:11 IS NULL THEN a:5 ELSE y_new:15 END [as=upsert_y:17, type=int, outer=(5,11,15)] + │ └── CASE WHEN xyz.x:11 IS NULL THEN column10:10 ELSE xyz.z:13 END [as=upsert_z:18, type=float, outer=(10,11,13)] └── select - ├── columns: x:16(string!null) y:17(int!null) z:18(float) - ├── stats: [rows=9.94974874, distinct(17)=0.994974874, null(17)=0] - ├── fd: ()-->(17) + ├── columns: x:19(string!null) y:20(int!null) z:21(float) + ├── stats: [rows=9.94974874, distinct(20)=0.994974874, null(20)=0] + ├── fd: ()-->(20) ├── with-scan &1 - │ ├── columns: x:16(string!null) y:17(int!null) z:18(float) + │ ├── columns: x:19(string!null) y:20(int!null) z:21(float) │ ├── mapping: - │ │ ├── xyz.x:1(string) => x:16(string) - │ │ ├── xyz.y:2(int) => y:17(int) - │ │ └── xyz.z:3(float) => z:18(float) - │ └── stats: [rows=9.94974874, distinct(16)=0.994974874, null(16)=0, distinct(17)=0.994974874, null(17)=0] + │ │ ├── xyz.x:1(string) => x:19(string) + │ │ ├── xyz.y:2(int) => y:20(int) + │ │ └── xyz.z:3(float) => z:21(float) + │ └── stats: [rows=9.94974874, distinct(19)=0.994974874, null(19)=0, distinct(20)=0.994974874, null(20)=0] └── filters - └── y:17 = 10 [type=bool, outer=(17), constraints=(/17: [/10 - /10]; tight), fd=()-->(17)] + └── y:20 = 10 [type=bool, outer=(20), constraints=(/20: [/10 - /10]; tight), fd=()-->(20)] # Cardinality is zero. build @@ -200,41 +200,41 @@ UPSERT INTO xyz SELECT b, a FROM abc WHERE False RETURNING * upsert xyz ├── columns: x:1(string!null) y:2(int!null) z:3(float) ├── upsert-mapping: - │ ├── b:5 => x:1 - │ ├── a:4 => y:2 - │ └── column8:8 => z:3 + │ ├── b:6 => x:1 + │ ├── a:5 => y:2 + │ └── column10:10 => z:3 ├── cardinality: [0 - 0] ├── volatile, mutations ├── stats: [rows=0] ├── fd: ()-->(3) └── project - ├── columns: column8:8(float) a:4(int!null) b:5(string) + ├── columns: column10:10(float) a:5(int!null) b:6(string) ├── cardinality: [0 - 0] ├── immutable ├── stats: [rows=0] - ├── fd: ()-->(8) + ├── fd: ()-->(10) ├── project - │ ├── columns: a:4(int!null) b:5(string) + │ ├── columns: a:5(int!null) b:6(string) │ ├── cardinality: [0 - 0] │ ├── stats: [rows=0] │ └── select - │ ├── columns: a:4(int!null) b:5(string) c:6(float) rowid:7(int!null) + │ ├── columns: a:5(int!null) b:6(string) c:7(float) rowid:8(int!null) abc.crdb_internal_mvcc_timestamp:9(decimal) │ ├── cardinality: [0 - 0] │ ├── stats: [rows=0] - │ ├── key: (7) - │ ├── fd: (7)-->(4-6) + │ ├── key: (8) + │ ├── fd: (8)-->(5-7,9) │ ├── scan abc - │ │ ├── columns: a:4(int!null) b:5(string) c:6(float) rowid:7(int!null) + │ │ ├── columns: a:5(int!null) b:6(string) c:7(float) rowid:8(int!null) abc.crdb_internal_mvcc_timestamp:9(decimal) │ │ ├── computed column expressions - │ │ │ └── c:6 - │ │ │ └── a:4::FLOAT8 [type=float] + │ │ │ └── c:7 + │ │ │ └── a:5::FLOAT8 [type=float] │ │ ├── stats: [rows=2000] - │ │ ├── key: (7) - │ │ └── fd: (7)-->(4-6) + │ │ ├── key: (8) + │ │ └── fd: (8)-->(5-7,9) │ └── filters │ └── false [type=bool] └── projections - └── NULL::FLOAT8 [as=column8:8, type=float, immutable] + └── NULL::FLOAT8 [as=column10:10, type=float, immutable] # Nullable conflict column. Ensure that ensure-upsert-distinct-on passes through # the input's null count. @@ -245,75 +245,75 @@ ON CONFLICT (v) DO UPDATE SET v=1 ---- upsert uv ├── columns: - ├── canary column: 8 - ├── fetch columns: u:8(int) v:9(int) + ├── canary column: 10 + ├── fetch columns: u:10(int) v:11(int) ├── insert-mapping: - │ ├── column7:7 => u:1 - │ └── z:6 => v:2 + │ ├── column9:9 => u:1 + │ └── z:8 => v:2 ├── update-mapping: - │ └── upsert_v:12 => v:2 + │ └── upsert_v:15 => v:2 ├── cardinality: [0 - 0] ├── volatile, mutations ├── stats: [rows=0] └── project - ├── columns: upsert_u:11(int) upsert_v:12(int) z:6(int) column7:7(int) u:8(int) v:9(int) v_new:10(int!null) + ├── columns: upsert_u:14(int) upsert_v:15(int) z:8(int) column9:9(int) u:10(int) v:11(int) uv.crdb_internal_mvcc_timestamp:12(decimal) v_new:13(int!null) ├── volatile ├── stats: [rows=1000] - ├── lax-key: (6,8) - ├── fd: ()-->(10), (6)~~>(7), (8)-->(9), (9)~~>(8), (7,8)-->(11), (6,8)-->(12), (6,8)~~>(7,11) + ├── lax-key: (8,10) + ├── fd: ()-->(13), (8)~~>(9), (10)-->(11,12), (11)~~>(10,12), (9,10)-->(14), (8,10)-->(15), (8,10)~~>(9,14) ├── project - │ ├── columns: v_new:10(int!null) z:6(int) column7:7(int) u:8(int) v:9(int) + │ ├── columns: v_new:13(int!null) z:8(int) column9:9(int) u:10(int) v:11(int) uv.crdb_internal_mvcc_timestamp:12(decimal) │ ├── volatile │ ├── stats: [rows=1000] - │ ├── lax-key: (6,8) - │ ├── fd: ()-->(10), (6)~~>(7), (8)-->(9), (9)~~>(8) + │ ├── lax-key: (8,10) + │ ├── fd: ()-->(13), (8)~~>(9), (10)-->(11,12), (11)~~>(10,12) │ ├── left-join (hash) - │ │ ├── columns: z:6(int) column7:7(int) u:8(int) v:9(int) + │ │ ├── columns: z:8(int) column9:9(int) u:10(int) v:11(int) uv.crdb_internal_mvcc_timestamp:12(decimal) │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ ├── volatile - │ │ ├── stats: [rows=1000, distinct(9)=991, null(9)=0] - │ │ ├── lax-key: (6,8) - │ │ ├── fd: (6)~~>(7), (8)-->(9), (9)~~>(8) + │ │ ├── stats: [rows=1000, distinct(11)=991, null(11)=0] + │ │ ├── lax-key: (8,10) + │ │ ├── fd: (8)~~>(9), (10)-->(11,12), (11)~~>(10,12) │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: z:6(int) column7:7(int) - │ │ │ ├── grouping columns: z:6(int) + │ │ │ ├── columns: z:8(int) column9:9(int) + │ │ │ ├── grouping columns: z:8(int) │ │ │ ├── error: "UPSERT or INSERT...ON CONFLICT command cannot affect row a second time" │ │ │ ├── volatile - │ │ │ ├── stats: [rows=1000, distinct(6)=1000, null(6)=0] - │ │ │ ├── lax-key: (6) - │ │ │ ├── fd: (6)~~>(7) + │ │ │ ├── stats: [rows=1000, distinct(8)=1000, null(8)=0] + │ │ │ ├── lax-key: (8) + │ │ │ ├── fd: (8)~~>(9) │ │ │ ├── project - │ │ │ │ ├── columns: column7:7(int) z:6(int) + │ │ │ │ ├── columns: column9:9(int) z:8(int) │ │ │ │ ├── volatile - │ │ │ │ ├── stats: [rows=1000, distinct(6)=100, null(6)=0] + │ │ │ │ ├── stats: [rows=1000, distinct(8)=100, null(8)=0] │ │ │ │ ├── project - │ │ │ │ │ ├── columns: z:6(int) + │ │ │ │ │ ├── columns: z:8(int) │ │ │ │ │ ├── immutable - │ │ │ │ │ ├── stats: [rows=1000, distinct(6)=100, null(6)=0] + │ │ │ │ │ ├── stats: [rows=1000, distinct(8)=100, null(8)=0] │ │ │ │ │ ├── scan xyz - │ │ │ │ │ │ ├── columns: x:3(string!null) y:4(int!null) xyz.z:5(float) - │ │ │ │ │ │ ├── stats: [rows=1000, distinct(5)=100, null(5)=10] - │ │ │ │ │ │ ├── key: (3) - │ │ │ │ │ │ └── fd: (3)-->(4,5) + │ │ │ │ │ │ ├── columns: x:4(string!null) y:5(int!null) xyz.z:6(float) xyz.crdb_internal_mvcc_timestamp:7(decimal) + │ │ │ │ │ │ ├── stats: [rows=1000, distinct(6)=100, null(6)=10] + │ │ │ │ │ │ ├── key: (4) + │ │ │ │ │ │ └── fd: (4)-->(5-7) │ │ │ │ │ └── projections - │ │ │ │ │ └── xyz.z:5::INT8 [as=z:6, type=int, outer=(5), immutable] + │ │ │ │ │ └── xyz.z:6::INT8 [as=z:8, type=int, outer=(6), immutable] │ │ │ │ └── projections - │ │ │ │ └── unique_rowid() [as=column7:7, type=int, volatile] + │ │ │ │ └── unique_rowid() [as=column9:9, type=int, volatile] │ │ │ └── aggregations - │ │ │ └── first-agg [as=column7:7, type=int, outer=(7)] - │ │ │ └── column7:7 [type=int] + │ │ │ └── first-agg [as=column9:9, type=int, outer=(9)] + │ │ │ └── column9:9 [type=int] │ │ ├── scan uv - │ │ │ ├── columns: u:8(int!null) v:9(int) - │ │ │ ├── stats: [rows=1000, distinct(9)=991, null(9)=10] - │ │ │ ├── key: (8) - │ │ │ └── fd: (8)-->(9), (9)~~>(8) + │ │ │ ├── columns: u:10(int!null) v:11(int) uv.crdb_internal_mvcc_timestamp:12(decimal) + │ │ │ ├── stats: [rows=1000, distinct(11)=991, null(11)=10] + │ │ │ ├── key: (10) + │ │ │ └── fd: (10)-->(11,12), (11)~~>(10,12) │ │ └── filters - │ │ └── z:6 = v:9 [type=bool, outer=(6,9), constraints=(/6: (/NULL - ]; /9: (/NULL - ]), fd=(6)==(9), (9)==(6)] + │ │ └── z:8 = v:11 [type=bool, outer=(8,11), constraints=(/8: (/NULL - ]; /11: (/NULL - ]), fd=(8)==(11), (11)==(8)] │ └── projections - │ └── 1 [as=v_new:10, type=int] + │ └── 1 [as=v_new:13, type=int] └── projections - ├── CASE WHEN u:8 IS NULL THEN column7:7 ELSE u:8 END [as=upsert_u:11, type=int, outer=(7,8)] - └── CASE WHEN u:8 IS NULL THEN z:6 ELSE v_new:10 END [as=upsert_v:12, type=int, outer=(6,8,10)] + ├── CASE WHEN u:10 IS NULL THEN column9:9 ELSE u:10 END [as=upsert_u:14, type=int, outer=(9,10)] + └── CASE WHEN u:10 IS NULL THEN z:8 ELSE v_new:13 END [as=upsert_v:15, type=int, outer=(8,10,13)] # Multiple conflict columns. # TODO(andyk): The null counts for the left join are surprisingly high. It's due @@ -328,59 +328,64 @@ ON CONFLICT (n, o) DO UPDATE SET o = 5 ---- upsert mno ├── columns: - ├── canary column: 7 - ├── fetch columns: m:7(int) n:8(int) o:9(int) + ├── canary column: 9 + ├── fetch columns: m:9(int) n:10(int) o:11(int) ├── insert-mapping: - │ ├── m:4 => m:1 - │ ├── n:5 => n:2 - │ └── o:6 => o:3 + │ ├── m:5 => m:1 + │ ├── n:6 => n:2 + │ └── o:7 => o:3 ├── update-mapping: - │ └── upsert_o:13 => o:3 + │ └── upsert_o:16 => o:3 ├── cardinality: [0 - 0] ├── volatile, mutations ├── stats: [rows=0] └── project - ├── columns: upsert_m:11(int) upsert_n:12(int) upsert_o:13(int) m:4(int!null) n:5(int) o:6(int) m:7(int) n:8(int) o:9(int) o_new:10(int!null) + ├── columns: upsert_m:14(int) upsert_n:15(int) upsert_o:16(int) m:5(int!null) n:6(int) o:7(int) m:9(int) n:10(int) o:11(int) crdb_internal_mvcc_timestamp:12(decimal) o_new:13(int!null) ├── stats: [rows=2000] - ├── key: (4,7) - ├── fd: ()-->(10), (4)-->(5,6), (5,6)~~>(4), (7)-->(8,9), (8,9)~~>(7), (4,7)-->(11), (5,7)-->(12), (6,7)-->(13) + ├── key: (5,9) + ├── fd: ()-->(13), (5)-->(6,7), (6,7)~~>(5), (9)-->(10-12), (10,11)~~>(9,12), (5,9)-->(14), (6,9)-->(15), (7,9)-->(16) ├── project - │ ├── columns: o_new:10(int!null) m:4(int!null) n:5(int) o:6(int) m:7(int) n:8(int) o:9(int) + │ ├── columns: o_new:13(int!null) m:5(int!null) n:6(int) o:7(int) m:9(int) n:10(int) o:11(int) crdb_internal_mvcc_timestamp:12(decimal) │ ├── stats: [rows=2000] - │ ├── key: (4,7) - │ ├── fd: ()-->(10), (4)-->(5,6), (5,6)~~>(4), (7)-->(8,9), (8,9)~~>(7) + │ ├── key: (5,9) + │ ├── fd: ()-->(13), (5)-->(6,7), (6,7)~~>(5), (9)-->(10-12), (10,11)~~>(9,12) │ ├── left-join (hash) - │ │ ├── columns: m:4(int!null) n:5(int) o:6(int) m:7(int) n:8(int) o:9(int) + │ │ ├── columns: m:5(int!null) n:6(int) o:7(int) m:9(int) n:10(int) o:11(int) crdb_internal_mvcc_timestamp:12(decimal) │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) - │ │ ├── stats: [rows=2000, distinct(8)=21.0526316, null(8)=1988.94737, distinct(9)=21.0526316, null(9)=2000] - │ │ ├── key: (4,7) - │ │ ├── fd: (4)-->(5,6), (5,6)~~>(4), (7)-->(8,9), (8,9)~~>(7) + │ │ ├── stats: [rows=2000, distinct(10)=21.0526316, null(10)=1988.94737, distinct(11)=21.0526316, null(11)=2000] + │ │ ├── key: (5,9) + │ │ ├── fd: (5)-->(6,7), (6,7)~~>(5), (9)-->(10-12), (10,11)~~>(9,12) │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: m:4(int!null) n:5(int) o:6(int) - │ │ │ ├── grouping columns: n:5(int) o:6(int) + │ │ │ ├── columns: m:5(int!null) n:6(int) o:7(int) + │ │ │ ├── grouping columns: n:6(int) o:7(int) │ │ │ ├── error: "UPSERT or INSERT...ON CONFLICT command cannot affect row a second time" - │ │ │ ├── stats: [rows=2000, distinct(4)=1981, null(4)=0, distinct(5)=100, null(5)=10, distinct(6)=1900, null(6)=100] - │ │ │ ├── key: (4) - │ │ │ ├── fd: (4)-->(5,6), (5,6)~~>(4) - │ │ │ ├── scan mno - │ │ │ │ ├── columns: m:4(int!null) n:5(int) o:6(int) - │ │ │ │ ├── stats: [rows=2000, distinct(5)=100, null(5)=10, distinct(6)=1900, null(6)=100, distinct(5,6)=1981, null(5,6)=20] - │ │ │ │ ├── key: (4) - │ │ │ │ └── fd: (4)-->(5,6), (5,6)~~>(4) + │ │ │ ├── stats: [rows=2000, distinct(5)=1981, null(5)=0, distinct(6)=100, null(6)=10, distinct(7)=1900, null(7)=100] + │ │ │ ├── key: (5) + │ │ │ ├── fd: (5)-->(6,7), (6,7)~~>(5) + │ │ │ ├── project + │ │ │ │ ├── columns: m:5(int!null) n:6(int) o:7(int) + │ │ │ │ ├── stats: [rows=2000, distinct(6)=100, null(6)=10, distinct(7)=1900, null(7)=100, distinct(6,7)=1981, null(6,7)=20] + │ │ │ │ ├── key: (5) + │ │ │ │ ├── fd: (5)-->(6,7), (6,7)~~>(5) + │ │ │ │ └── scan mno + │ │ │ │ ├── columns: m:5(int!null) n:6(int) o:7(int) crdb_internal_mvcc_timestamp:8(decimal) + │ │ │ │ ├── stats: [rows=2000, distinct(6)=100, null(6)=10, distinct(7)=1900, null(7)=100, distinct(6,7)=1981, null(6,7)=20] + │ │ │ │ ├── key: (5) + │ │ │ │ └── fd: (5)-->(6-8), (6,7)~~>(5,8) │ │ │ └── aggregations - │ │ │ └── first-agg [as=m:4, type=int, outer=(4)] - │ │ │ └── m:4 [type=int] + │ │ │ └── first-agg [as=m:5, type=int, outer=(5)] + │ │ │ └── m:5 [type=int] │ │ ├── scan mno - │ │ │ ├── columns: m:7(int!null) n:8(int) o:9(int) - │ │ │ ├── stats: [rows=2000, distinct(8)=100, null(8)=10, distinct(9)=1900, null(9)=100] - │ │ │ ├── key: (7) - │ │ │ └── fd: (7)-->(8,9), (8,9)~~>(7) + │ │ │ ├── columns: m:9(int!null) n:10(int) o:11(int) crdb_internal_mvcc_timestamp:12(decimal) + │ │ │ ├── stats: [rows=2000, distinct(10)=100, null(10)=10, distinct(11)=1900, null(11)=100] + │ │ │ ├── key: (9) + │ │ │ └── fd: (9)-->(10-12), (10,11)~~>(9,12) │ │ └── filters - │ │ ├── n:5 = n:8 [type=bool, outer=(5,8), constraints=(/5: (/NULL - ]; /8: (/NULL - ]), fd=(5)==(8), (8)==(5)] - │ │ └── o:6 = o:9 [type=bool, outer=(6,9), constraints=(/6: (/NULL - ]; /9: (/NULL - ]), fd=(6)==(9), (9)==(6)] + │ │ ├── n:6 = n:10 [type=bool, outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ]), fd=(6)==(10), (10)==(6)] + │ │ └── o:7 = o:11 [type=bool, outer=(7,11), constraints=(/7: (/NULL - ]; /11: (/NULL - ]), fd=(7)==(11), (11)==(7)] │ └── projections - │ └── 5 [as=o_new:10, type=int] + │ └── 5 [as=o_new:13, type=int] └── projections - ├── CASE WHEN m:7 IS NULL THEN m:4 ELSE m:7 END [as=upsert_m:11, type=int, outer=(4,7)] - ├── CASE WHEN m:7 IS NULL THEN n:5 ELSE n:8 END [as=upsert_n:12, type=int, outer=(5,7,8)] - └── CASE WHEN m:7 IS NULL THEN o:6 ELSE o_new:10 END [as=upsert_o:13, type=int, outer=(6,7,10)] + ├── CASE WHEN m:9 IS NULL THEN m:5 ELSE m:9 END [as=upsert_m:14, type=int, outer=(5,9)] + ├── CASE WHEN m:9 IS NULL THEN n:6 ELSE n:10 END [as=upsert_n:15, type=int, outer=(6,9,10)] + └── CASE WHEN m:9 IS NULL THEN o:7 ELSE o_new:13 END [as=upsert_o:16, type=int, outer=(7,9,13)] diff --git a/pkg/sql/opt/memo/testdata/stats/window b/pkg/sql/opt/memo/testdata/stats/window index 402f1b599365..39ab6781d2df 100644 --- a/pkg/sql/opt/memo/testdata/stats/window +++ b/pkg/sql/opt/memo/testdata/stats/window @@ -13,19 +13,19 @@ CREATE TABLE kv ( ) ---- -build colstat=8 +build colstat=9 SELECT k, rank() OVER () FROM (SELECT * FROM kv LIMIT 10) ---- project - ├── columns: k:1(int!null) rank:8(int) + ├── columns: k:1(int!null) rank:9(int) ├── cardinality: [0 - 10] - ├── stats: [rows=10, distinct(8)=10, null(8)=0] + ├── stats: [rows=10, distinct(9)=10, null(9)=0] ├── key: (1) - ├── fd: (1)-->(8) + ├── fd: (1)-->(9) └── window partition=() - ├── columns: k:1(int!null) v:2(int) w:3(int) f:4(float) d:5(decimal) s:6(string) b:7(bool) rank:8(int) + ├── columns: k:1(int!null) v:2(int) w:3(int) f:4(float) d:5(decimal) s:6(string) b:7(bool) rank:9(int) ├── cardinality: [0 - 10] - ├── stats: [rows=10, distinct(8)=10, null(8)=0] + ├── stats: [rows=10, distinct(9)=10, null(9)=0] ├── key: (1) ├── fd: (1)-->(2-7) ├── limit @@ -34,12 +34,18 @@ project │ ├── stats: [rows=10] │ ├── key: (1) │ ├── fd: (1)-->(2-7) - │ ├── scan kv + │ ├── project │ │ ├── columns: k:1(int!null) v:2(int) w:3(int) f:4(float) d:5(decimal) s:6(string) b:7(bool) │ │ ├── stats: [rows=1000] │ │ ├── key: (1) │ │ ├── fd: (1)-->(2-7) - │ │ └── limit hint: 10.00 + │ │ ├── limit hint: 10.00 + │ │ └── scan kv + │ │ ├── columns: k:1(int!null) v:2(int) w:3(int) f:4(float) d:5(decimal) s:6(string) b:7(bool) crdb_internal_mvcc_timestamp:8(decimal) + │ │ ├── stats: [rows=1000] + │ │ ├── key: (1) + │ │ ├── fd: (1)-->(2-8) + │ │ └── limit hint: 10.00 │ └── 10 [type=int] └── windows - └── rank [as=rank:8, type=int] + └── rank [as=rank:9, type=int] diff --git a/pkg/sql/opt/memo/testdata/stats/with b/pkg/sql/opt/memo/testdata/stats/with index ee8ee3fa5ef1..8a4ad6d1a3e3 100644 --- a/pkg/sql/opt/memo/testdata/stats/with +++ b/pkg/sql/opt/memo/testdata/stats/with @@ -31,28 +31,33 @@ ALTER TABLE a INJECT STATISTICS '[ ]' ---- -build colstat=4 colstat=5 colstat=6 disable=InlineWith +build colstat=5 colstat=6 colstat=7 disable=InlineWith WITH foo AS (SELECT * FROM a) SELECT * FROM foo ---- with &1 (foo) - ├── columns: x:4(int!null) y:5(int) s:6(string) + ├── columns: x:5(int!null) y:6(int) s:7(string) ├── stats: [rows=5000] - ├── key: (4) - ├── fd: (4)-->(5,6) - ├── scan a + ├── key: (5) + ├── fd: (5)-->(6,7) + ├── project │ ├── columns: a.x:1(int!null) a.y:2(int) a.s:3(string) │ ├── stats: [rows=5000] │ ├── key: (1) - │ └── fd: (1)-->(2,3) + │ ├── fd: (1)-->(2,3) + │ └── scan a + │ ├── columns: a.x:1(int!null) a.y:2(int) a.s:3(string) crdb_internal_mvcc_timestamp:4(decimal) + │ ├── stats: [rows=5000] + │ ├── key: (1) + │ └── fd: (1)-->(2-4) └── with-scan &1 (foo) - ├── columns: x:4(int!null) y:5(int) s:6(string) + ├── columns: x:5(int!null) y:6(int) s:7(string) ├── mapping: - │ ├── a.x:1(int) => x:4(int) - │ ├── a.y:2(int) => y:5(int) - │ └── a.s:3(string) => s:6(string) - ├── stats: [rows=5000, distinct(4)=5000, null(4)=0, distinct(5)=500, null(5)=50, distinct(6)=500, null(6)=50] - ├── key: (4) - └── fd: (4)-->(5,6) + │ ├── a.x:1(int) => x:5(int) + │ ├── a.y:2(int) => y:6(int) + │ └── a.s:3(string) => s:7(string) + ├── stats: [rows=5000, distinct(5)=5000, null(5)=0, distinct(6)=500, null(6)=50, distinct(7)=500, null(7)=50] + ├── key: (5) + └── fd: (5)-->(6,7) # Regression test for #40296. opt disable=InlineWith @@ -71,9 +76,9 @@ WHERE ); ---- with &1 (t0) - ├── columns: "?column?":27(unknown) + ├── columns: "?column?":28(unknown) ├── stats: [rows=10000] - ├── fd: ()-->(27) + ├── fd: ()-->(28) ├── union │ ├── columns: column1:10(int) column2:11(oid!null) column3:12(date) column4:13(string) │ ├── left columns: column1:1(int) column2:2(oid) column3:9(date) column4:4(string) @@ -96,64 +101,64 @@ with &1 (t0) │ ├── fd: ()-->(5-8) │ └── (NULL, 0, '1970-09-08', NULL) [type=tuple{int, oid, date, string}] └── with &2 (t1) - ├── columns: "?column?":27(unknown) + ├── columns: "?column?":28(unknown) ├── stats: [rows=10000] - ├── fd: ()-->(27) + ├── fd: ()-->(28) ├── values - │ ├── columns: "?column?":21(unknown) + │ ├── columns: "?column?":22(unknown) │ ├── cardinality: [1 - 1] │ ├── stats: [rows=1] │ ├── key: () - │ ├── fd: ()-->(21) + │ ├── fd: ()-->(22) │ └── (NULL,) [type=tuple{unknown}] └── project - ├── columns: "?column?":27(unknown) + ├── columns: "?column?":28(unknown) ├── stats: [rows=10000] - ├── fd: ()-->(27) + ├── fd: ()-->(28) ├── inner-join (cross) - │ ├── columns: true_agg:25(bool!null) + │ ├── columns: true_agg:26(bool!null) │ ├── stats: [rows=10000] - │ ├── fd: ()-->(25) + │ ├── fd: ()-->(26) │ ├── scan a │ │ └── stats: [rows=5000] │ ├── inner-join (cross) - │ │ ├── columns: true_agg:25(bool!null) + │ │ ├── columns: true_agg:26(bool!null) │ │ ├── cardinality: [0 - 2] │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(one-or-more) │ │ ├── stats: [rows=2] - │ │ ├── fd: ()-->(25) + │ │ ├── fd: ()-->(26) │ │ ├── with-scan &1 (t0) │ │ │ ├── mapping: │ │ │ ├── cardinality: [1 - 2] │ │ │ └── stats: [rows=2] │ │ ├── select - │ │ │ ├── columns: true_agg:25(bool!null) + │ │ │ ├── columns: true_agg:26(bool!null) │ │ │ ├── cardinality: [0 - 1] - │ │ │ ├── stats: [rows=1, distinct(25)=1, null(25)=0] + │ │ │ ├── stats: [rows=1, distinct(26)=1, null(26)=0] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(25) + │ │ │ ├── fd: ()-->(26) │ │ │ ├── scalar-group-by - │ │ │ │ ├── columns: true_agg:25(bool) + │ │ │ │ ├── columns: true_agg:26(bool) │ │ │ │ ├── cardinality: [1 - 1] - │ │ │ │ ├── stats: [rows=1, distinct(25)=1, null(25)=0] + │ │ │ │ ├── stats: [rows=1, distinct(26)=1, null(26)=0] │ │ │ │ ├── key: () - │ │ │ │ ├── fd: ()-->(25) + │ │ │ │ ├── fd: ()-->(26) │ │ │ │ ├── values - │ │ │ │ │ ├── columns: true:24(bool!null) + │ │ │ │ │ ├── columns: true:25(bool!null) │ │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ │ ├── stats: [rows=1] │ │ │ │ │ ├── key: () - │ │ │ │ │ ├── fd: ()-->(24) + │ │ │ │ │ ├── fd: ()-->(25) │ │ │ │ │ └── (true,) [type=tuple{bool}] │ │ │ │ └── aggregations - │ │ │ │ └── const-agg [as=true_agg:25, type=bool, outer=(24)] - │ │ │ │ └── true:24 [type=bool] + │ │ │ │ └── const-agg [as=true_agg:26, type=bool, outer=(25)] + │ │ │ │ └── true:25 [type=bool] │ │ │ └── filters - │ │ │ └── true_agg:25 IS NOT NULL [type=bool, outer=(25), constraints=(/25: (/NULL - ]; tight)] + │ │ │ └── true_agg:26 IS NOT NULL [type=bool, outer=(26), constraints=(/26: (/NULL - ]; tight)] │ │ └── filters (true) │ └── filters (true) └── projections - └── NULL [as="?column?":27, type=unknown] + └── NULL [as="?column?":28, type=unknown] exec-ddl CREATE TABLE test ( @@ -169,13 +174,13 @@ WITH RECURSIVE hierarchy(id) as SELECT * FROM hierarchy ---- project - ├── columns: id:7(string) + ├── columns: id:9(string) ├── stats: [rows=10] ├── recursive-c-t-e - │ ├── columns: id:3(string) + │ ├── columns: id:4(string) │ ├── working table binding: &1 │ ├── initial columns: test.id:1(string) - │ ├── recursive columns: c.id:4(string) + │ ├── recursive columns: c.id:5(string) │ ├── stats: [rows=10] │ ├── values │ │ ├── columns: test.id:1(string!null) @@ -184,23 +189,23 @@ project │ │ ├── key: () │ │ └── fd: ()-->(1) │ └── inner-join (cross) - │ ├── columns: c.id:4(string!null) + │ ├── columns: c.id:5(string!null) │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-more) │ ├── stats: [rows=10] - │ ├── fd: ()-->(4) + │ ├── fd: ()-->(5) │ ├── select - │ │ ├── columns: c.id:4(string!null) - │ │ ├── stats: [rows=10, distinct(4)=1, null(4)=0] - │ │ ├── fd: ()-->(4) + │ │ ├── columns: c.id:5(string!null) + │ │ ├── stats: [rows=10, distinct(5)=1, null(5)=0] + │ │ ├── fd: ()-->(5) │ │ ├── scan c - │ │ │ ├── columns: c.id:4(string) - │ │ │ └── stats: [rows=1000, distinct(4)=100, null(4)=10] + │ │ │ ├── columns: c.id:5(string) + │ │ │ └── stats: [rows=1000, distinct(5)=100, null(5)=10] │ │ └── filters - │ │ └── c.id:4 = 'bar' [type=bool, outer=(4), constraints=(/4: [/'bar' - /'bar']; tight), fd=()-->(4)] + │ │ └── c.id:5 = 'bar' [type=bool, outer=(5), constraints=(/5: [/'bar' - /'bar']; tight), fd=()-->(5)] │ ├── with-scan &1 (hierarchy) │ │ ├── mapping: │ │ ├── cardinality: [1 - ] │ │ └── stats: [rows=1] │ └── filters (true) └── projections - └── id:3 [as=id:7, type=string, outer=(3)] + └── id:4 [as=id:9, type=string, outer=(4)] diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpcc b/pkg/sql/opt/memo/testdata/stats_quality/tpcc index b80bffa0dc8d..e77599877a33 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpcc +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpcc @@ -449,11 +449,11 @@ WHERE ol_w_id = 8 AND ol_d_id = 6 AND ol_o_id = 1000 ---- scalar-group-by ├── save-table-name: delivery_02_scalar_group_by_1 - ├── columns: sum:11(decimal) + ├── columns: sum:12(decimal) ├── cardinality: [1 - 1] - ├── stats: [rows=1, distinct(11)=1, null(11)=0] + ├── stats: [rows=1, distinct(12)=1, null(12)=0] ├── key: () - ├── fd: ()-->(11) + ├── fd: ()-->(12) ├── scan order_line │ ├── save-table-name: delivery_02_scan_2 │ ├── columns: ol_o_id:1(int!null) ol_d_id:2(int!null) ol_w_id:3(int!null) ol_amount:9(decimal) @@ -463,7 +463,7 @@ scalar-group-by │ │ <---- 8 -- │ └── fd: ()-->(1-3) └── aggregations - └── sum [as=sum:11, type=decimal, outer=(9)] + └── sum [as=sum:12, type=decimal, outer=(9)] └── ol_amount:9 [type=decimal] stats table=delivery_02_scan_2 @@ -541,24 +541,24 @@ WHERE ol_w_id = 1 ---- scalar-group-by ├── save-table-name: stock_level_02_scalar_group_by_1 - ├── columns: count:28(int!null) + ├── columns: count:30(int!null) ├── cardinality: [1 - 1] - ├── stats: [rows=1, distinct(28)=1, null(28)=0] + ├── stats: [rows=1, distinct(30)=1, null(30)=0] ├── key: () - ├── fd: ()-->(28) + ├── fd: ()-->(30) ├── distinct-on │ ├── save-table-name: stock_level_02_distinct_on_2 - │ ├── columns: s_i_id:11(int!null) - │ ├── grouping columns: s_i_id:11(int!null) - │ ├── stats: [rows=187.914589, distinct(11)=187.914589, null(11)=0] - │ ├── key: (11) + │ ├── columns: s_i_id:12(int!null) + │ ├── grouping columns: s_i_id:12(int!null) + │ ├── stats: [rows=187.914589, distinct(12)=187.914589, null(12)=0] + │ ├── key: (12) │ └── inner-join (lookup stock) │ ├── save-table-name: stock_level_02_lookup_join_3 - │ ├── columns: ol_o_id:1(int!null) ol_d_id:2(int!null) ol_w_id:3(int!null) ol_i_id:5(int!null) s_i_id:11(int!null) s_w_id:12(int!null) s_quantity:13(int!null) - │ ├── key columns: [3 5] = [12 11] + │ ├── columns: ol_o_id:1(int!null) ol_d_id:2(int!null) ol_w_id:3(int!null) ol_i_id:5(int!null) s_i_id:12(int!null) s_w_id:13(int!null) s_quantity:14(int!null) + │ ├── key columns: [3 5] = [13 12] │ ├── lookup columns are key - │ ├── stats: [rows=218.87081, distinct(1)=19.9996466, null(1)=0, distinct(2)=1, null(2)=0, distinct(3)=1, null(3)=0, distinct(5)=187.914589, null(5)=0, distinct(11)=187.914589, null(11)=0, distinct(12)=1, null(12)=0, distinct(13)=30.3110384, null(13)=0] - │ ├── fd: ()-->(2,3,12), (11)-->(13), (5)==(11), (11)==(5), (3)==(12), (12)==(3) + │ ├── stats: [rows=218.87081, distinct(1)=19.9996466, null(1)=0, distinct(2)=1, null(2)=0, distinct(3)=1, null(3)=0, distinct(5)=187.914589, null(5)=0, distinct(12)=187.914589, null(12)=0, distinct(13)=1, null(13)=0, distinct(14)=30.3110384, null(14)=0] + │ ├── fd: ()-->(2,3,13), (12)-->(14), (5)==(12), (12)==(5), (3)==(13), (13)==(3) │ ├── scan order_line │ │ ├── save-table-name: stock_level_02_scan_4 │ │ ├── columns: ol_o_id:1(int!null) ol_d_id:2(int!null) ol_w_id:3(int!null) ol_i_id:5(int!null) @@ -568,10 +568,10 @@ scalar-group-by │ │ │ <---- 1 -- │ │ └── fd: ()-->(2,3) │ └── filters - │ ├── s_w_id:12 = 1 [type=bool, outer=(12), constraints=(/12: [/1 - /1]; tight), fd=()-->(12)] - │ └── s_quantity:13 < 15 [type=bool, outer=(13), constraints=(/13: (/NULL - /14]; tight)] + │ ├── s_w_id:13 = 1 [type=bool, outer=(13), constraints=(/13: [/1 - /1]; tight), fd=()-->(13)] + │ └── s_quantity:14 < 15 [type=bool, outer=(14), constraints=(/14: (/NULL - /14]; tight)] └── aggregations - └── count-rows [as=count:28, type=int] + └── count-rows [as=count:30, type=int] stats table=stock_level_02_scan_4 ---- @@ -637,21 +637,21 @@ WHERE w_ytd != sum_d_ytd ---- scalar-group-by ├── save-table-name: consistency_01_scalar_group_by_1 - ├── columns: count:22(int!null) + ├── columns: count:24(int!null) ├── cardinality: [1 - 1] ├── immutable - ├── stats: [rows=1, distinct(22)=1, null(22)=0] + ├── stats: [rows=1, distinct(24)=1, null(24)=0] ├── key: () - ├── fd: ()-->(22) + ├── fd: ()-->(24) ├── inner-join (merge) │ ├── save-table-name: consistency_01_merge_join_2 - │ ├── columns: w_id:1(int!null) w_ytd:9(decimal!null) d_w_id:11(int!null) sum:21(decimal!null) + │ ├── columns: w_id:1(int!null) w_ytd:9(decimal!null) d_w_id:12(int!null) sum:23(decimal!null) │ ├── left ordering: +1 - │ ├── right ordering: +11 + │ ├── right ordering: +12 │ ├── immutable - │ ├── stats: [rows=3.33333333, distinct(1)=3.33333333, null(1)=0, distinct(9)=1, null(9)=0, distinct(11)=3.33333333, null(11)=0, distinct(21)=3.33333333, null(21)=0] - │ ├── key: (11) - │ ├── fd: (1)-->(9), (11)-->(21), (1)==(11), (11)==(1) + │ ├── stats: [rows=3.33333333, distinct(1)=3.33333333, null(1)=0, distinct(9)=1, null(9)=0, distinct(12)=3.33333333, null(12)=0, distinct(23)=3.33333333, null(23)=0] + │ ├── key: (12) + │ ├── fd: (1)-->(9), (12)-->(23), (1)==(12), (12)==(1) │ ├── scan warehouse │ │ ├── save-table-name: consistency_01_scan_3 │ │ ├── columns: w_id:1(int!null) w_ytd:9(decimal) @@ -663,26 +663,26 @@ scalar-group-by │ │ └── ordering: +1 │ ├── group-by │ │ ├── save-table-name: consistency_01_group_by_4 - │ │ ├── columns: d_w_id:11(int!null) sum:21(decimal) - │ │ ├── grouping columns: d_w_id:11(int!null) - │ │ ├── stats: [rows=10, distinct(11)=10, null(11)=0, distinct(21)=10, null(21)=0] - │ │ ├── key: (11) - │ │ ├── fd: (11)-->(21) - │ │ ├── ordering: +11 + │ │ ├── columns: d_w_id:12(int!null) sum:23(decimal) + │ │ ├── grouping columns: d_w_id:12(int!null) + │ │ ├── stats: [rows=10, distinct(12)=10, null(12)=0, distinct(23)=10, null(23)=0] + │ │ ├── key: (12) + │ │ ├── fd: (12)-->(23) + │ │ ├── ordering: +12 │ │ ├── scan district │ │ │ ├── save-table-name: consistency_01_scan_5 - │ │ │ ├── columns: d_w_id:11(int!null) d_ytd:19(decimal) - │ │ │ ├── stats: [rows=100, distinct(11)=10, null(11)=0, distinct(19)=1, null(19)=0] - │ │ │ │ histogram(11)= 0 10 0 10 0 10 0 10 0 10 0 10 0 10 0 10 0 10 0 10 + │ │ │ ├── columns: d_w_id:12(int!null) d_ytd:20(decimal) + │ │ │ ├── stats: [rows=100, distinct(12)=10, null(12)=0, distinct(20)=1, null(20)=0] + │ │ │ │ histogram(12)= 0 10 0 10 0 10 0 10 0 10 0 10 0 10 0 10 0 10 0 10 │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 --- 5 --- 6 --- 7 --- 8 --- 9 - │ │ │ └── ordering: +11 + │ │ │ └── ordering: +12 │ │ └── aggregations - │ │ └── sum [as=sum:21, type=decimal, outer=(19)] - │ │ └── d_ytd:19 [type=decimal] + │ │ └── sum [as=sum:23, type=decimal, outer=(20)] + │ │ └── d_ytd:20 [type=decimal] │ └── filters - │ └── w_ytd:9 != sum:21 [type=bool, outer=(9,21), immutable, constraints=(/9: (/NULL - ]; /21: (/NULL - ])] + │ └── w_ytd:9 != sum:23 [type=bool, outer=(9,23), immutable, constraints=(/9: (/NULL - ]; /23: (/NULL - ])] └── aggregations - └── count-rows [as=count_rows:22, type=int] + └── count-rows [as=count_rows:24, type=int] stats table=consistency_01_group_by_4 ---- @@ -739,11 +739,11 @@ ORDER BY no_w_id, no_d_id ---- group-by ├── save-table-name: consistency_03_group_by_1 - ├── columns: max:4(int!null) [hidden: no_d_id:2(int!null) no_w_id:3(int!null)] + ├── columns: max:5(int!null) [hidden: no_d_id:2(int!null) no_w_id:3(int!null)] ├── grouping columns: no_d_id:2(int!null) no_w_id:3(int!null) - ├── stats: [rows=100, distinct(2)=10, null(2)=0, distinct(3)=10, null(3)=0, distinct(4)=100, null(4)=0, distinct(2,3)=100, null(2,3)=0] + ├── stats: [rows=100, distinct(2)=10, null(2)=0, distinct(3)=10, null(3)=0, distinct(5)=100, null(5)=0, distinct(2,3)=100, null(2,3)=0] ├── key: (2,3) - ├── fd: (2,3)-->(4) + ├── fd: (2,3)-->(5) ├── ordering: +3,+2 ├── scan new_order │ ├── save-table-name: consistency_03_scan_2 @@ -754,7 +754,7 @@ group-by │ ├── key: (1-3) │ └── ordering: +3,+2 └── aggregations - └── max [as=max:4, type=int, outer=(1)] + └── max [as=max:5, type=int, outer=(1)] └── no_o_id:1 [type=int] stats table=consistency_03_group_by_1 @@ -777,11 +777,11 @@ ORDER BY o_w_id, o_d_id ---- group-by ├── save-table-name: consistency_04_group_by_1 - ├── columns: max:9(int!null) [hidden: o_d_id:2(int!null) o_w_id:3(int!null)] + ├── columns: max:10(int!null) [hidden: o_d_id:2(int!null) o_w_id:3(int!null)] ├── grouping columns: o_d_id:2(int!null) o_w_id:3(int!null) - ├── stats: [rows=100, distinct(2)=10, null(2)=0, distinct(3)=10, null(3)=0, distinct(9)=100, null(9)=0, distinct(2,3)=100, null(2,3)=0] + ├── stats: [rows=100, distinct(2)=10, null(2)=0, distinct(3)=10, null(3)=0, distinct(10)=100, null(10)=0, distinct(2,3)=100, null(2,3)=0] ├── key: (2,3) - ├── fd: (2,3)-->(9) + ├── fd: (2,3)-->(10) ├── ordering: +3,+2 ├── scan "order"@order_idx │ ├── save-table-name: consistency_04_scan_2 @@ -792,7 +792,7 @@ group-by │ ├── key: (1-3) │ └── ordering: +3,+2 └── aggregations - └── max [as=max:9, type=int, outer=(1)] + └── max [as=max:10, type=int, outer=(1)] └── o_id:1 [type=int] stats table=consistency_04_group_by_1 @@ -819,27 +819,27 @@ WHERE nod != -1 ---- scalar-group-by ├── save-table-name: consistency_05_scalar_group_by_1 - ├── columns: count:8(int!null) + ├── columns: count:9(int!null) ├── cardinality: [1 - 1] ├── immutable - ├── stats: [rows=1, distinct(8)=1, null(8)=0] + ├── stats: [rows=1, distinct(9)=1, null(9)=0] ├── key: () - ├── fd: ()-->(8) + ├── fd: ()-->(9) ├── select │ ├── save-table-name: consistency_05_select_2 - │ ├── columns: no_d_id:2(int!null) no_w_id:3(int!null) max:4(int!null) min:5(int!null) count_rows:6(int!null) + │ ├── columns: no_d_id:2(int!null) no_w_id:3(int!null) max:5(int!null) min:6(int!null) count_rows:7(int!null) │ ├── immutable - │ ├── stats: [rows=33.3333333, distinct(2)=9.8265847, null(2)=0, distinct(3)=9.8265847, null(3)=0, distinct(4)=33.3333333, null(4)=0, distinct(5)=33.3333333, null(5)=0, distinct(6)=33.3333333, null(6)=0] + │ ├── stats: [rows=33.3333333, distinct(2)=9.8265847, null(2)=0, distinct(3)=9.8265847, null(3)=0, distinct(5)=33.3333333, null(5)=0, distinct(6)=33.3333333, null(6)=0, distinct(7)=33.3333333, null(7)=0] │ ├── key: (2,3) - │ ├── fd: (2,3)-->(4-6) + │ ├── fd: (2,3)-->(5-7) │ ├── group-by │ │ ├── save-table-name: consistency_05_group_by_3 - │ │ ├── columns: no_d_id:2(int!null) no_w_id:3(int!null) max:4(int!null) min:5(int!null) count_rows:6(int!null) + │ │ ├── columns: no_d_id:2(int!null) no_w_id:3(int!null) max:5(int!null) min:6(int!null) count_rows:7(int!null) │ │ ├── grouping columns: no_d_id:2(int!null) no_w_id:3(int!null) │ │ ├── internal-ordering: +3,+2 - │ │ ├── stats: [rows=100, distinct(2)=10, null(2)=0, distinct(3)=10, null(3)=0, distinct(4)=100, null(4)=0, distinct(5)=100, null(5)=0, distinct(6)=100, null(6)=0, distinct(2,3)=100, null(2,3)=0] + │ │ ├── stats: [rows=100, distinct(2)=10, null(2)=0, distinct(3)=10, null(3)=0, distinct(5)=100, null(5)=0, distinct(6)=100, null(6)=0, distinct(7)=100, null(7)=0, distinct(2,3)=100, null(2,3)=0] │ │ ├── key: (2,3) - │ │ ├── fd: (2,3)-->(4-6) + │ │ ├── fd: (2,3)-->(5-7) │ │ ├── scan new_order │ │ │ ├── save-table-name: consistency_05_scan_4 │ │ │ ├── columns: no_o_id:1(int!null) no_d_id:2(int!null) no_w_id:3(int!null) @@ -849,15 +849,15 @@ scalar-group-by │ │ │ ├── key: (1-3) │ │ │ └── ordering: +3,+2 │ │ └── aggregations - │ │ ├── max [as=max:4, type=int, outer=(1)] + │ │ ├── max [as=max:5, type=int, outer=(1)] │ │ │ └── no_o_id:1 [type=int] - │ │ ├── min [as=min:5, type=int, outer=(1)] + │ │ ├── min [as=min:6, type=int, outer=(1)] │ │ │ └── no_o_id:1 [type=int] - │ │ └── count-rows [as=count_rows:6, type=int] + │ │ └── count-rows [as=count_rows:7, type=int] │ └── filters - │ └── ((max:4 - min:5) - count_rows:6) != -1 [type=bool, outer=(4-6), immutable] + │ └── ((max:5 - min:6) - count_rows:7) != -1 [type=bool, outer=(5-7), immutable] └── aggregations - └── count-rows [as=count_rows:8, type=int] + └── count-rows [as=count_rows:9, type=int] stats table=consistency_05_group_by_3 ---- @@ -907,11 +907,11 @@ ORDER BY o_w_id, o_d_id ---- group-by ├── save-table-name: consistency_06_group_by_1 - ├── columns: sum:9(decimal) [hidden: o_d_id:2(int!null) o_w_id:3(int!null)] + ├── columns: sum:10(decimal) [hidden: o_d_id:2(int!null) o_w_id:3(int!null)] ├── grouping columns: o_d_id:2(int!null) o_w_id:3(int!null) - ├── stats: [rows=100, distinct(2)=10, null(2)=0, distinct(3)=10, null(3)=0, distinct(9)=100, null(9)=0, distinct(2,3)=100, null(2,3)=0] + ├── stats: [rows=100, distinct(2)=10, null(2)=0, distinct(3)=10, null(3)=0, distinct(10)=100, null(10)=0, distinct(2,3)=100, null(2,3)=0] ├── key: (2,3) - ├── fd: (2,3)-->(9) + ├── fd: (2,3)-->(10) ├── ordering: +3,+2 ├── scan "order" │ ├── save-table-name: consistency_06_scan_2 @@ -921,7 +921,7 @@ group-by │ │ <---- 0 ----- 1 ----- 2 ----- 3 ----- 4 ----- 5 ----- 6 ----- 7 ----- 8 ----- 9 - │ └── ordering: +3,+2 └── aggregations - └── sum [as=sum:9, type=decimal, outer=(7)] + └── sum [as=sum:10, type=decimal, outer=(7)] └── o_ol_cnt:7 [type=int] stats table=consistency_06_group_by_1 @@ -944,18 +944,18 @@ ORDER BY ol_w_id, ol_d_id ---- sort ├── save-table-name: consistency_07_sort_1 - ├── columns: count:11(int!null) [hidden: ol_d_id:2(int!null) ol_w_id:3(int!null)] - ├── stats: [rows=100, distinct(2)=10, null(2)=0, distinct(3)=10, null(3)=0, distinct(11)=100, null(11)=0, distinct(2,3)=100, null(2,3)=0] + ├── columns: count:12(int!null) [hidden: ol_d_id:2(int!null) ol_w_id:3(int!null)] + ├── stats: [rows=100, distinct(2)=10, null(2)=0, distinct(3)=10, null(3)=0, distinct(12)=100, null(12)=0, distinct(2,3)=100, null(2,3)=0] ├── key: (2,3) - ├── fd: (2,3)-->(11) + ├── fd: (2,3)-->(12) ├── ordering: +3,+2 └── group-by ├── save-table-name: consistency_07_group_by_2 - ├── columns: ol_d_id:2(int!null) ol_w_id:3(int!null) count_rows:11(int!null) + ├── columns: ol_d_id:2(int!null) ol_w_id:3(int!null) count_rows:12(int!null) ├── grouping columns: ol_d_id:2(int!null) ol_w_id:3(int!null) - ├── stats: [rows=100, distinct(2)=10, null(2)=0, distinct(3)=10, null(3)=0, distinct(11)=100, null(11)=0, distinct(2,3)=100, null(2,3)=0] + ├── stats: [rows=100, distinct(2)=10, null(2)=0, distinct(3)=10, null(3)=0, distinct(12)=100, null(12)=0, distinct(2,3)=100, null(2,3)=0] ├── key: (2,3) - ├── fd: (2,3)-->(11) + ├── fd: (2,3)-->(12) ├── scan order_line@order_line_stock_fk_idx │ ├── save-table-name: consistency_07_scan_3 │ ├── columns: ol_d_id:2(int!null) ol_w_id:3(int!null) @@ -963,7 +963,7 @@ sort │ histogram(3)= 0 3.1213e+05 0 2.7851e+05 0 2.9892e+05 0 3.0732e+05 0 2.9892e+05 0 2.9622e+05 0 3.1363e+05 0 2.8392e+05 0 2.9892e+05 0 3.1273e+05 │ <------ 0 ---------- 1 ---------- 2 ---------- 3 ---------- 4 ---------- 5 ---------- 6 ---------- 7 ---------- 8 ---------- 9 ---- └── aggregations - └── count-rows [as=count_rows:11, type=int] + └── count-rows [as=count_rows:12, type=int] stats table=consistency_07_group_by_2 ---- @@ -986,7 +986,7 @@ except-all ├── save-table-name: consistency_08_except_all_1 ├── columns: no_w_id:3(int!null) no_d_id:2(int!null) no_o_id:1(int!null) ├── left columns: no_w_id:3(int!null) no_d_id:2(int!null) no_o_id:1(int!null) - ├── right columns: o_w_id:6(int) o_d_id:5(int) o_id:4(int) + ├── right columns: o_w_id:7(int) o_d_id:6(int) o_id:5(int) ├── stats: [rows=90000, distinct(1)=900, null(1)=0, distinct(2)=10, null(2)=0, distinct(3)=10, null(3)=0] ├── scan new_order │ ├── save-table-name: consistency_08_scan_2 @@ -997,25 +997,25 @@ except-all │ └── key: (1-3) └── project ├── save-table-name: consistency_08_project_3 - ├── columns: o_id:4(int!null) o_d_id:5(int!null) o_w_id:6(int!null) - ├── stats: [rows=90000, distinct(4)=2999, null(4)=0, distinct(5)=10, null(5)=0, distinct(6)=10, null(6)=0] - ├── key: (4-6) + ├── columns: o_id:5(int!null) o_d_id:6(int!null) o_w_id:7(int!null) + ├── stats: [rows=90000, distinct(5)=2999, null(5)=0, distinct(6)=10, null(6)=0, distinct(7)=10, null(7)=0] + ├── key: (5-7) └── select ├── save-table-name: consistency_08_select_4 - ├── columns: o_id:4(int!null) o_d_id:5(int!null) o_w_id:6(int!null) o_carrier_id:9(int) - ├── stats: [rows=90000, distinct(4)=2999, null(4)=0, distinct(5)=10, null(5)=0, distinct(6)=10, null(6)=0, distinct(9)=1, null(9)=90000] - ├── key: (4-6) - ├── fd: ()-->(9) + ├── columns: o_id:5(int!null) o_d_id:6(int!null) o_w_id:7(int!null) o_carrier_id:10(int) + ├── stats: [rows=90000, distinct(5)=2999, null(5)=0, distinct(6)=10, null(6)=0, distinct(7)=10, null(7)=0, distinct(10)=1, null(10)=90000] + ├── key: (5-7) + ├── fd: ()-->(10) ├── scan "order"@order_idx │ ├── save-table-name: consistency_08_scan_5 - │ ├── columns: o_id:4(int!null) o_d_id:5(int!null) o_w_id:6(int!null) o_carrier_id:9(int) - │ ├── stats: [rows=300000, distinct(4)=2999, null(4)=0, distinct(5)=10, null(5)=0, distinct(6)=10, null(6)=0, distinct(9)=11, null(9)=90000] - │ │ histogram(6)= 0 29280 0 29310 0 31560 0 30000 0 29220 0 30720 0 31110 0 30510 0 29070 0 29220 + │ ├── columns: o_id:5(int!null) o_d_id:6(int!null) o_w_id:7(int!null) o_carrier_id:10(int) + │ ├── stats: [rows=300000, distinct(5)=2999, null(5)=0, distinct(6)=10, null(6)=0, distinct(7)=10, null(7)=0, distinct(10)=11, null(10)=90000] + │ │ histogram(7)= 0 29280 0 29310 0 31560 0 30000 0 29220 0 30720 0 31110 0 30510 0 29070 0 29220 │ │ <---- 0 ----- 1 ----- 2 ----- 3 ----- 4 ----- 5 ----- 6 ----- 7 ----- 8 ----- 9 - - │ ├── key: (4-6) - │ └── fd: (4-6)-->(9) + │ ├── key: (5-7) + │ └── fd: (5-7)-->(10) └── filters - └── o_carrier_id:9 IS NULL [type=bool, outer=(9), constraints=(/9: [/NULL - /NULL]; tight), fd=()-->(9)] + └── o_carrier_id:10 IS NULL [type=bool, outer=(10), constraints=(/10: [/NULL - /NULL]; tight), fd=()-->(10)] stats table=consistency_08_select_4 ---- @@ -1052,7 +1052,7 @@ except-all ├── save-table-name: consistency_09_except_all_1 ├── columns: o_w_id:3(int!null) o_d_id:2(int!null) o_id:1(int!null) ├── left columns: o_w_id:3(int!null) o_d_id:2(int!null) o_id:1(int!null) - ├── right columns: no_w_id:11(int) no_d_id:10(int) no_o_id:9(int) + ├── right columns: no_w_id:12(int) no_d_id:11(int) no_o_id:10(int) ├── stats: [rows=90000, distinct(1)=2999, null(1)=0, distinct(2)=10, null(2)=0, distinct(3)=10, null(3)=0] ├── project │ ├── save-table-name: consistency_09_project_2 @@ -1077,11 +1077,11 @@ except-all │ └── o_carrier_id:6 IS NULL [type=bool, outer=(6), constraints=(/6: [/NULL - /NULL]; tight), fd=()-->(6)] └── scan new_order ├── save-table-name: consistency_09_scan_5 - ├── columns: no_o_id:9(int!null) no_d_id:10(int!null) no_w_id:11(int!null) - ├── stats: [rows=90000, distinct(9)=900, null(9)=0, distinct(10)=10, null(10)=0, distinct(11)=10, null(11)=0] - │ histogram(11)= 0 9333 0 9189 0 9261 0 9162 0 9306 0 8442 0 8946 0 8532 0 9135 0 8694 + ├── columns: no_o_id:10(int!null) no_d_id:11(int!null) no_w_id:12(int!null) + ├── stats: [rows=90000, distinct(10)=900, null(10)=0, distinct(11)=10, null(11)=0, distinct(12)=10, null(12)=0] + │ histogram(12)= 0 9333 0 9189 0 9261 0 9162 0 9306 0 8442 0 8946 0 8532 0 9135 0 8694 │ <--- 0 ---- 1 ---- 2 ---- 3 ---- 4 ---- 5 ---- 6 ---- 7 ---- 8 ---- 9 - - └── key: (9-11) + └── key: (10-12) stats table=consistency_09_except_all_1 ---- @@ -1113,7 +1113,7 @@ except-all ├── save-table-name: consistency_10_except_all_1 ├── columns: o_w_id:3(int!null) o_d_id:2(int!null) o_id:1(int!null) o_ol_cnt:7(int) ├── left columns: o_w_id:3(int!null) o_d_id:2(int!null) o_id:1(int!null) o_ol_cnt:7(int) - ├── right columns: ol_w_id:11(int) ol_d_id:10(int) ol_o_id:9(int) count_rows:19(int) + ├── right columns: ol_w_id:12(int) ol_d_id:11(int) ol_o_id:10(int) count_rows:21(int) ├── stats: [rows=300000, distinct(1)=2999, null(1)=0, distinct(2)=10, null(2)=0, distinct(3)=10, null(3)=0, distinct(7)=11, null(7)=0] ├── scan "order" │ ├── save-table-name: consistency_10_scan_2 @@ -1125,19 +1125,19 @@ except-all │ └── fd: (1-3)-->(7) └── group-by ├── save-table-name: consistency_10_group_by_3 - ├── columns: ol_o_id:9(int!null) ol_d_id:10(int!null) ol_w_id:11(int!null) count_rows:19(int!null) - ├── grouping columns: ol_o_id:9(int!null) ol_d_id:10(int!null) ol_w_id:11(int!null) - ├── stats: [rows=295745, distinct(9)=2999, null(9)=0, distinct(10)=10, null(10)=0, distinct(11)=10, null(11)=0, distinct(19)=295745, null(19)=0, distinct(9-11)=295745, null(9-11)=0] - ├── key: (9-11) - ├── fd: (9-11)-->(19) + ├── columns: ol_o_id:10(int!null) ol_d_id:11(int!null) ol_w_id:12(int!null) count_rows:21(int!null) + ├── grouping columns: ol_o_id:10(int!null) ol_d_id:11(int!null) ol_w_id:12(int!null) + ├── stats: [rows=295745, distinct(10)=2999, null(10)=0, distinct(11)=10, null(11)=0, distinct(12)=10, null(12)=0, distinct(21)=295745, null(21)=0, distinct(10-12)=295745, null(10-12)=0] + ├── key: (10-12) + ├── fd: (10-12)-->(21) ├── scan order_line@order_line_stock_fk_idx │ ├── save-table-name: consistency_10_scan_4 - │ ├── columns: ol_o_id:9(int!null) ol_d_id:10(int!null) ol_w_id:11(int!null) - │ └── stats: [rows=3001222, distinct(9)=2999, null(9)=0, distinct(10)=10, null(10)=0, distinct(11)=10, null(11)=0, distinct(9-11)=295745, null(9-11)=0] - │ histogram(11)= 0 3.1213e+05 0 2.7851e+05 0 2.9892e+05 0 3.0732e+05 0 2.9892e+05 0 2.9622e+05 0 3.1363e+05 0 2.8392e+05 0 2.9892e+05 0 3.1273e+05 + │ ├── columns: ol_o_id:10(int!null) ol_d_id:11(int!null) ol_w_id:12(int!null) + │ └── stats: [rows=3001222, distinct(10)=2999, null(10)=0, distinct(11)=10, null(11)=0, distinct(12)=10, null(12)=0, distinct(10-12)=295745, null(10-12)=0] + │ histogram(12)= 0 3.1213e+05 0 2.7851e+05 0 2.9892e+05 0 3.0732e+05 0 2.9892e+05 0 2.9622e+05 0 3.1363e+05 0 2.8392e+05 0 2.9892e+05 0 3.1273e+05 │ <------ 0 ---------- 1 ---------- 2 ---------- 3 ---------- 4 ---------- 5 ---------- 6 ---------- 7 ---------- 8 ---------- 9 ---- └── aggregations - └── count-rows [as=count_rows:19, type=int] + └── count-rows [as=count_rows:21, type=int] stats table=consistency_10_group_by_3 ---- @@ -1183,17 +1183,17 @@ EXCEPT ALL ---- except-all ├── save-table-name: consistency_11_except_all_1 - ├── columns: ol_w_id:3(int!null) ol_d_id:2(int!null) ol_o_id:1(int!null) count:11(int) - ├── left columns: ol_w_id:3(int!null) ol_d_id:2(int!null) ol_o_id:1(int!null) count_rows:11(int) - ├── right columns: o_w_id:14(int) o_d_id:13(int) o_id:12(int) o_ol_cnt:18(int) - ├── stats: [rows=295745, distinct(1)=2999, null(1)=0, distinct(2)=10, null(2)=0, distinct(3)=10, null(3)=0, distinct(11)=295745, null(11)=0] + ├── columns: ol_w_id:3(int!null) ol_d_id:2(int!null) ol_o_id:1(int!null) count:12(int) + ├── left columns: ol_w_id:3(int!null) ol_d_id:2(int!null) ol_o_id:1(int!null) count_rows:12(int) + ├── right columns: o_w_id:15(int) o_d_id:14(int) o_id:13(int) o_ol_cnt:19(int) + ├── stats: [rows=295745, distinct(1)=2999, null(1)=0, distinct(2)=10, null(2)=0, distinct(3)=10, null(3)=0, distinct(12)=295745, null(12)=0] ├── group-by │ ├── save-table-name: consistency_11_group_by_2 - │ ├── columns: ol_o_id:1(int!null) ol_d_id:2(int!null) ol_w_id:3(int!null) count_rows:11(int!null) + │ ├── columns: ol_o_id:1(int!null) ol_d_id:2(int!null) ol_w_id:3(int!null) count_rows:12(int!null) │ ├── grouping columns: ol_o_id:1(int!null) ol_d_id:2(int!null) ol_w_id:3(int!null) - │ ├── stats: [rows=295745, distinct(1)=2999, null(1)=0, distinct(2)=10, null(2)=0, distinct(3)=10, null(3)=0, distinct(11)=295745, null(11)=0, distinct(1-3)=295745, null(1-3)=0] + │ ├── stats: [rows=295745, distinct(1)=2999, null(1)=0, distinct(2)=10, null(2)=0, distinct(3)=10, null(3)=0, distinct(12)=295745, null(12)=0, distinct(1-3)=295745, null(1-3)=0] │ ├── key: (1-3) - │ ├── fd: (1-3)-->(11) + │ ├── fd: (1-3)-->(12) │ ├── scan order_line@order_line_stock_fk_idx │ │ ├── save-table-name: consistency_11_scan_3 │ │ ├── columns: ol_o_id:1(int!null) ol_d_id:2(int!null) ol_w_id:3(int!null) @@ -1201,15 +1201,15 @@ except-all │ │ histogram(3)= 0 3.1213e+05 0 2.7851e+05 0 2.9892e+05 0 3.0732e+05 0 2.9892e+05 0 2.9622e+05 0 3.1363e+05 0 2.8392e+05 0 2.9892e+05 0 3.1273e+05 │ │ <------ 0 ---------- 1 ---------- 2 ---------- 3 ---------- 4 ---------- 5 ---------- 6 ---------- 7 ---------- 8 ---------- 9 ---- │ └── aggregations - │ └── count-rows [as=count_rows:11, type=int] + │ └── count-rows [as=count_rows:12, type=int] └── scan "order" ├── save-table-name: consistency_11_scan_4 - ├── columns: o_id:12(int!null) o_d_id:13(int!null) o_w_id:14(int!null) o_ol_cnt:18(int) - ├── stats: [rows=300000, distinct(12)=2999, null(12)=0, distinct(13)=10, null(13)=0, distinct(14)=10, null(14)=0, distinct(18)=11, null(18)=0] - │ histogram(14)= 0 29280 0 29310 0 31560 0 30000 0 29220 0 30720 0 31110 0 30510 0 29070 0 29220 + ├── columns: o_id:13(int!null) o_d_id:14(int!null) o_w_id:15(int!null) o_ol_cnt:19(int) + ├── stats: [rows=300000, distinct(13)=2999, null(13)=0, distinct(14)=10, null(14)=0, distinct(15)=10, null(15)=0, distinct(19)=11, null(19)=0] + │ histogram(15)= 0 29280 0 29310 0 31560 0 30000 0 29220 0 30720 0 31110 0 30510 0 29070 0 29220 │ <---- 0 ----- 1 ----- 2 ----- 3 ----- 4 ----- 5 ----- 6 ----- 7 ----- 8 ----- 9 - - ├── key: (12-14) - └── fd: (12-14)-->(18) + ├── key: (13-15) + └── fd: (13-15)-->(19) stats table=consistency_11_except_all_1 ---- @@ -1244,37 +1244,37 @@ WHERE ol_o_id IS NULL OR o_id IS NULL ---- scalar-group-by ├── save-table-name: consistency_12_scalar_group_by_1 - ├── columns: count:19(int!null) + ├── columns: count:21(int!null) ├── cardinality: [1 - 1] - ├── stats: [rows=1, distinct(19)=1, null(19)=0] + ├── stats: [rows=1, distinct(21)=1, null(21)=0] ├── key: () - ├── fd: ()-->(19) + ├── fd: ()-->(21) ├── select │ ├── save-table-name: consistency_12_select_2 - │ ├── columns: o_id:1(int) o_d_id:2(int) o_w_id:3(int) ol_o_id:9(int) ol_d_id:10(int) ol_w_id:11(int) - │ ├── stats: [rows=299711.333, distinct(1)=2999, null(1)=209767.952, distinct(2)=10, null(2)=209767.952, distinct(3)=10, null(3)=209767.952, distinct(9)=2999, null(9)=0, distinct(10)=10, null(10)=0, distinct(11)=10, null(11)=0] + │ ├── columns: o_id:1(int) o_d_id:2(int) o_w_id:3(int) ol_o_id:10(int) ol_d_id:11(int) ol_w_id:12(int) + │ ├── stats: [rows=299711.333, distinct(1)=2999, null(1)=209767.952, distinct(2)=10, null(2)=209767.952, distinct(3)=10, null(3)=209767.952, distinct(10)=2999, null(10)=0, distinct(11)=10, null(11)=0, distinct(12)=10, null(12)=0] │ ├── full-join (hash) │ │ ├── save-table-name: consistency_12_full_join_3 - │ │ ├── columns: o_id:1(int) o_d_id:2(int) o_w_id:3(int) ol_o_id:9(int) ol_d_id:10(int) ol_w_id:11(int) + │ │ ├── columns: o_id:1(int) o_d_id:2(int) o_w_id:3(int) ol_o_id:10(int) ol_d_id:11(int) ol_w_id:12(int) │ │ ├── multiplicity: left-rows(exactly-one), right-rows(one-or-more) - │ │ ├── stats: [rows=899134, distinct(1)=2999, null(1)=629303.857, distinct(2)=10, null(2)=629303.857, distinct(3)=10, null(3)=629303.857, distinct(9)=2999, null(9)=0, distinct(10)=10, null(10)=0, distinct(11)=10, null(11)=0] + │ │ ├── stats: [rows=899134, distinct(1)=2999, null(1)=629303.857, distinct(2)=10, null(2)=629303.857, distinct(3)=10, null(3)=629303.857, distinct(10)=2999, null(10)=0, distinct(11)=10, null(11)=0, distinct(12)=10, null(12)=0] │ │ ├── project │ │ │ ├── save-table-name: consistency_12_project_4 - │ │ │ ├── columns: ol_o_id:9(int!null) ol_d_id:10(int!null) ol_w_id:11(int!null) - │ │ │ ├── stats: [rows=899134, distinct(9)=2999, null(9)=0, distinct(10)=10, null(10)=0, distinct(11)=10, null(11)=0] + │ │ │ ├── columns: ol_o_id:10(int!null) ol_d_id:11(int!null) ol_w_id:12(int!null) + │ │ │ ├── stats: [rows=899134, distinct(10)=2999, null(10)=0, distinct(11)=10, null(11)=0, distinct(12)=10, null(12)=0] │ │ │ └── select │ │ │ ├── save-table-name: consistency_12_select_5 - │ │ │ ├── columns: ol_o_id:9(int!null) ol_d_id:10(int!null) ol_w_id:11(int!null) ol_delivery_d:15(timestamp) - │ │ │ ├── stats: [rows=899134, distinct(9)=2999, null(9)=0, distinct(10)=10, null(10)=0, distinct(11)=10, null(11)=0, distinct(15)=1, null(15)=899134] - │ │ │ ├── fd: ()-->(15) + │ │ │ ├── columns: ol_o_id:10(int!null) ol_d_id:11(int!null) ol_w_id:12(int!null) ol_delivery_d:16(timestamp) + │ │ │ ├── stats: [rows=899134, distinct(10)=2999, null(10)=0, distinct(11)=10, null(11)=0, distinct(12)=10, null(12)=0, distinct(16)=1, null(16)=899134] + │ │ │ ├── fd: ()-->(16) │ │ │ ├── scan order_line │ │ │ │ ├── save-table-name: consistency_12_scan_6 - │ │ │ │ ├── columns: ol_o_id:9(int!null) ol_d_id:10(int!null) ol_w_id:11(int!null) ol_delivery_d:15(timestamp) - │ │ │ │ └── stats: [rows=3001222, distinct(9)=2999, null(9)=0, distinct(10)=10, null(10)=0, distinct(11)=10, null(11)=0, distinct(15)=2, null(15)=899134] - │ │ │ │ histogram(11)= 0 3.1213e+05 0 2.7851e+05 0 2.9892e+05 0 3.0732e+05 0 2.9892e+05 0 2.9622e+05 0 3.1363e+05 0 2.8392e+05 0 2.9892e+05 0 3.1273e+05 + │ │ │ │ ├── columns: ol_o_id:10(int!null) ol_d_id:11(int!null) ol_w_id:12(int!null) ol_delivery_d:16(timestamp) + │ │ │ │ └── stats: [rows=3001222, distinct(10)=2999, null(10)=0, distinct(11)=10, null(11)=0, distinct(12)=10, null(12)=0, distinct(16)=2, null(16)=899134] + │ │ │ │ histogram(12)= 0 3.1213e+05 0 2.7851e+05 0 2.9892e+05 0 3.0732e+05 0 2.9892e+05 0 2.9622e+05 0 3.1363e+05 0 2.8392e+05 0 2.9892e+05 0 3.1273e+05 │ │ │ │ <------ 0 ---------- 1 ---------- 2 ---------- 3 ---------- 4 ---------- 5 ---------- 6 ---------- 7 ---------- 8 ---------- 9 ---- │ │ │ └── filters - │ │ │ └── ol_delivery_d:15 IS NULL [type=bool, outer=(15), constraints=(/15: [/NULL - /NULL]; tight), fd=()-->(15)] + │ │ │ └── ol_delivery_d:16 IS NULL [type=bool, outer=(16), constraints=(/16: [/NULL - /NULL]; tight), fd=()-->(16)] │ │ ├── project │ │ │ ├── save-table-name: consistency_12_project_7 │ │ │ ├── columns: o_id:1(int!null) o_d_id:2(int!null) o_w_id:3(int!null) @@ -1297,13 +1297,13 @@ scalar-group-by │ │ │ └── filters │ │ │ └── o_carrier_id:6 IS NULL [type=bool, outer=(6), constraints=(/6: [/NULL - /NULL]; tight), fd=()-->(6)] │ │ └── filters - │ │ ├── ol_w_id:11 = o_w_id:3 [type=bool, outer=(3,11), constraints=(/3: (/NULL - ]; /11: (/NULL - ]), fd=(3)==(11), (11)==(3)] - │ │ ├── ol_d_id:10 = o_d_id:2 [type=bool, outer=(2,10), constraints=(/2: (/NULL - ]; /10: (/NULL - ]), fd=(2)==(10), (10)==(2)] - │ │ └── ol_o_id:9 = o_id:1 [type=bool, outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ]), fd=(1)==(9), (9)==(1)] + │ │ ├── ol_w_id:12 = o_w_id:3 [type=bool, outer=(3,12), constraints=(/3: (/NULL - ]; /12: (/NULL - ]), fd=(3)==(12), (12)==(3)] + │ │ ├── ol_d_id:11 = o_d_id:2 [type=bool, outer=(2,11), constraints=(/2: (/NULL - ]; /11: (/NULL - ]), fd=(2)==(11), (11)==(2)] + │ │ └── ol_o_id:10 = o_id:1 [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] │ └── filters - │ └── (ol_o_id:9 IS NULL) OR (o_id:1 IS NULL) [type=bool, outer=(1,9)] + │ └── (ol_o_id:10 IS NULL) OR (o_id:1 IS NULL) [type=bool, outer=(1,10)] └── aggregations - └── count-rows [as=count_rows:19, type=int] + └── count-rows [as=count_rows:21, type=int] stats table=consistency_12_select_5 ---- diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q01 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q01 index e0173c7581e7..b5c1e173a1ec 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q01 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q01 @@ -42,25 +42,25 @@ ORDER BY ---- sort ├── save-table-name: q1_sort_1 - ├── columns: l_returnflag:9(char!null) l_linestatus:10(char!null) sum_qty:17(float!null) sum_base_price:18(float!null) sum_disc_price:20(float!null) sum_charge:22(float!null) avg_qty:23(float!null) avg_price:24(float!null) avg_disc:25(float!null) count_order:26(int!null) + ├── columns: l_returnflag:9(char!null) l_linestatus:10(char!null) sum_qty:18(float!null) sum_base_price:19(float!null) sum_disc_price:21(float!null) sum_charge:23(float!null) avg_qty:24(float!null) avg_price:25(float!null) avg_disc:26(float!null) count_order:27(int!null) ├── immutable - ├── stats: [rows=6, distinct(9)=3, null(9)=0, distinct(10)=2, null(10)=0, distinct(17)=6, null(17)=0, distinct(18)=6, null(18)=0, distinct(20)=6, null(20)=0, distinct(22)=6, null(22)=0, distinct(23)=6, null(23)=0, distinct(24)=6, null(24)=0, distinct(25)=6, null(25)=0, distinct(26)=6, null(26)=0, distinct(9,10)=6, null(9,10)=0] + ├── stats: [rows=6, distinct(9)=3, null(9)=0, distinct(10)=2, null(10)=0, distinct(18)=6, null(18)=0, distinct(19)=6, null(19)=0, distinct(21)=6, null(21)=0, distinct(23)=6, null(23)=0, distinct(24)=6, null(24)=0, distinct(25)=6, null(25)=0, distinct(26)=6, null(26)=0, distinct(27)=6, null(27)=0, distinct(9,10)=6, null(9,10)=0] ├── key: (9,10) - ├── fd: (9,10)-->(17,18,20,22-26) + ├── fd: (9,10)-->(18,19,21,23-27) ├── ordering: +9,+10 └── group-by ├── save-table-name: q1_group_by_2 - ├── columns: l_returnflag:9(char!null) l_linestatus:10(char!null) sum:17(float!null) sum:18(float!null) sum:20(float!null) sum:22(float!null) avg:23(float!null) avg:24(float!null) avg:25(float!null) count_rows:26(int!null) + ├── columns: l_returnflag:9(char!null) l_linestatus:10(char!null) sum:18(float!null) sum:19(float!null) sum:21(float!null) sum:23(float!null) avg:24(float!null) avg:25(float!null) avg:26(float!null) count_rows:27(int!null) ├── grouping columns: l_returnflag:9(char!null) l_linestatus:10(char!null) ├── immutable - ├── stats: [rows=6, distinct(9)=3, null(9)=0, distinct(10)=2, null(10)=0, distinct(17)=6, null(17)=0, distinct(18)=6, null(18)=0, distinct(20)=6, null(20)=0, distinct(22)=6, null(22)=0, distinct(23)=6, null(23)=0, distinct(24)=6, null(24)=0, distinct(25)=6, null(25)=0, distinct(26)=6, null(26)=0, distinct(9,10)=6, null(9,10)=0] + ├── stats: [rows=6, distinct(9)=3, null(9)=0, distinct(10)=2, null(10)=0, distinct(18)=6, null(18)=0, distinct(19)=6, null(19)=0, distinct(21)=6, null(21)=0, distinct(23)=6, null(23)=0, distinct(24)=6, null(24)=0, distinct(25)=6, null(25)=0, distinct(26)=6, null(26)=0, distinct(27)=6, null(27)=0, distinct(9,10)=6, null(9,10)=0] ├── key: (9,10) - ├── fd: (9,10)-->(17,18,20,22-26) + ├── fd: (9,10)-->(18,19,21,23-27) ├── project │ ├── save-table-name: q1_project_3 - │ ├── columns: column19:19(float!null) column21:21(float!null) l_quantity:5(float!null) l_extendedprice:6(float!null) l_discount:7(float!null) l_returnflag:9(char!null) l_linestatus:10(char!null) + │ ├── columns: column20:20(float!null) column22:22(float!null) l_quantity:5(float!null) l_extendedprice:6(float!null) l_discount:7(float!null) l_returnflag:9(char!null) l_linestatus:10(char!null) │ ├── immutable - │ ├── stats: [rows=5925056.21, distinct(5)=50, null(5)=0, distinct(6)=925955, null(6)=0, distinct(7)=11, null(7)=0, distinct(9)=3, null(9)=0, distinct(10)=2, null(10)=0, distinct(19)=5925056.21, null(19)=0, distinct(21)=5925056.21, null(21)=0, distinct(9,10)=6, null(9,10)=0] + │ ├── stats: [rows=5925056.21, distinct(5)=50, null(5)=0, distinct(6)=925955, null(6)=0, distinct(7)=11, null(7)=0, distinct(9)=3, null(9)=0, distinct(10)=2, null(10)=0, distinct(20)=5925056.21, null(20)=0, distinct(22)=5925056.21, null(22)=0, distinct(9,10)=6, null(9,10)=0] │ ├── select │ │ ├── save-table-name: q1_select_4 │ │ ├── columns: l_quantity:5(float!null) l_extendedprice:6(float!null) l_discount:7(float!null) l_tax:8(float!null) l_returnflag:9(char!null) l_linestatus:10(char!null) l_shipdate:11(date!null) @@ -76,24 +76,24 @@ sort │ │ └── filters │ │ └── l_shipdate:11 <= '1998-09-02' [type=bool, outer=(11), constraints=(/11: (/NULL - /'1998-09-02']; tight)] │ └── projections - │ ├── l_extendedprice:6 * (1.0 - l_discount:7) [as=column19:19, type=float, outer=(6,7), immutable] - │ └── (l_extendedprice:6 * (1.0 - l_discount:7)) * (l_tax:8 + 1.0) [as=column21:21, type=float, outer=(6-8), immutable] + │ ├── l_extendedprice:6 * (1.0 - l_discount:7) [as=column20:20, type=float, outer=(6,7), immutable] + │ └── (l_extendedprice:6 * (1.0 - l_discount:7)) * (l_tax:8 + 1.0) [as=column22:22, type=float, outer=(6-8), immutable] └── aggregations - ├── sum [as=sum:17, type=float, outer=(5)] + ├── sum [as=sum:18, type=float, outer=(5)] │ └── l_quantity:5 [type=float] - ├── sum [as=sum:18, type=float, outer=(6)] + ├── sum [as=sum:19, type=float, outer=(6)] │ └── l_extendedprice:6 [type=float] - ├── sum [as=sum:20, type=float, outer=(19)] - │ └── column19:19 [type=float] - ├── sum [as=sum:22, type=float, outer=(21)] - │ └── column21:21 [type=float] - ├── avg [as=avg:23, type=float, outer=(5)] + ├── sum [as=sum:21, type=float, outer=(20)] + │ └── column20:20 [type=float] + ├── sum [as=sum:23, type=float, outer=(22)] + │ └── column22:22 [type=float] + ├── avg [as=avg:24, type=float, outer=(5)] │ └── l_quantity:5 [type=float] - ├── avg [as=avg:24, type=float, outer=(6)] + ├── avg [as=avg:25, type=float, outer=(6)] │ └── l_extendedprice:6 [type=float] - ├── avg [as=avg:25, type=float, outer=(7)] + ├── avg [as=avg:26, type=float, outer=(7)] │ └── l_discount:7 [type=float] - └── count-rows [as=count_rows:26, type=int] + └── count-rows [as=count_rows:27, type=int] stats table=q1_sort_1 ---- @@ -150,8 +150,8 @@ column_names row_count_est row_count_err distinct_count_est distinct_count stats table=q1_project_3 ---- column_names row_count distinct_count null_count -{column19} 5916591 4150364 0 -{column21} 5916591 5661182 0 +{column20} 5916591 4150364 0 +{column22} 5916591 5661182 0 {l_discount} 5916591 11 0 {l_extendedprice} 5916591 925955 0 {l_linestatus} 5916591 2 0 @@ -159,8 +159,8 @@ column_names row_count distinct_count null_count {l_returnflag} 5916591 3 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{column19} 5925056.00 1.00 5925056.00 1.43 0.00 1.00 -{column21} 5925056.00 1.00 5925056.00 1.05 0.00 1.00 +{column20} 5925056.00 1.00 5925056.00 1.43 0.00 1.00 +{column22} 5925056.00 1.00 5925056.00 1.05 0.00 1.00 {l_discount} 5925056.00 1.00 11.00 1.00 0.00 1.00 {l_extendedprice} 5925056.00 1.00 925955.00 1.00 0.00 1.00 {l_linestatus} 5925056.00 1.00 2.00 1.00 0.00 1.00 diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q02 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q02 index ce4a05cd0609..963927c60ba9 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q02 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q02 @@ -70,69 +70,69 @@ LIMIT 100; ---- project ├── save-table-name: q2_project_1 - ├── columns: s_acctbal:15(float!null) s_name:11(char!null) n_name:23(char!null) p_partkey:1(int!null) p_mfgr:3(char!null) s_address:12(varchar!null) s_phone:14(char!null) s_comment:16(varchar!null) + ├── columns: s_acctbal:16(float!null) s_name:12(char!null) n_name:26(char!null) p_partkey:1(int!null) p_mfgr:3(char!null) s_address:13(varchar!null) s_phone:15(char!null) s_comment:17(varchar!null) ├── cardinality: [0 - 100] - ├── stats: [rows=1, distinct(1)=1, null(1)=0, distinct(3)=1, null(3)=0, distinct(11)=1, null(11)=0, distinct(12)=1, null(12)=0, distinct(14)=1, null(14)=0, distinct(15)=1, null(15)=0, distinct(16)=1, null(16)=0, distinct(23)=1, null(23)=0] + ├── stats: [rows=1, distinct(1)=1, null(1)=0, distinct(3)=1, null(3)=0, distinct(12)=1, null(12)=0, distinct(13)=1, null(13)=0, distinct(15)=1, null(15)=0, distinct(16)=1, null(16)=0, distinct(17)=1, null(17)=0, distinct(26)=1, null(26)=0] ├── fd: (1)-->(3) - ├── ordering: -15,+23,+11,+1 + ├── ordering: -16,+26,+12,+1 └── limit ├── save-table-name: q2_limit_2 - ├── columns: p_partkey:1(int!null) p_mfgr:3(char!null) s_name:11(char!null) s_address:12(varchar!null) s_phone:14(char!null) s_acctbal:15(float!null) s_comment:16(varchar!null) ps_partkey:17(int!null) ps_suppkey:18(int!null) ps_supplycost:20(float!null) n_name:23(char!null) min:48(float!null) - ├── internal-ordering: -15,+23,+11,+(1|17) + ├── columns: p_partkey:1(int!null) p_mfgr:3(char!null) s_name:12(char!null) s_address:13(varchar!null) s_phone:15(char!null) s_acctbal:16(float!null) s_comment:17(varchar!null) ps_partkey:19(int!null) ps_suppkey:20(int!null) ps_supplycost:22(float!null) n_name:26(char!null) min:57(float!null) + ├── internal-ordering: -16,+26,+12,+(1|19) ├── cardinality: [0 - 100] - ├── stats: [rows=1, distinct(1)=1, null(1)=0, distinct(3)=1, null(3)=0, distinct(11)=1, null(11)=0, distinct(12)=1, null(12)=0, distinct(14)=1, null(14)=0, distinct(15)=1, null(15)=0, distinct(16)=1, null(16)=0, distinct(17)=0.999911987, null(17)=0, distinct(18)=0.99998212, null(18)=0, distinct(20)=1, null(20)=0, distinct(23)=1, null(23)=0, distinct(48)=1, null(48)=0] - ├── key: (17,18) - ├── fd: (1)-->(3), (17,18)-->(1,3,11,12,14-16,20,23,48), (1)==(17), (17)==(1), (18)-->(11,12,14-16,23), (20)==(48), (48)==(20) - ├── ordering: -15,+23,+11,+(1|17) [actual: -15,+23,+11,+1] + ├── stats: [rows=1, distinct(1)=1, null(1)=0, distinct(3)=1, null(3)=0, distinct(12)=1, null(12)=0, distinct(13)=1, null(13)=0, distinct(15)=1, null(15)=0, distinct(16)=1, null(16)=0, distinct(17)=1, null(17)=0, distinct(19)=0.999911987, null(19)=0, distinct(20)=0.99998212, null(20)=0, distinct(22)=1, null(22)=0, distinct(26)=1, null(26)=0, distinct(57)=1, null(57)=0] + ├── key: (19,20) + ├── fd: (1)-->(3), (19,20)-->(1,3,12,13,15-17,22,26,57), (1)==(19), (19)==(1), (20)-->(12,13,15-17,26), (22)==(57), (57)==(22) + ├── ordering: -16,+26,+12,+(1|19) [actual: -16,+26,+12,+1] ├── sort │ ├── save-table-name: q2_sort_3 - │ ├── columns: p_partkey:1(int!null) p_mfgr:3(char!null) s_name:11(char!null) s_address:12(varchar!null) s_phone:14(char!null) s_acctbal:15(float!null) s_comment:16(varchar!null) ps_partkey:17(int!null) ps_suppkey:18(int!null) ps_supplycost:20(float!null) n_name:23(char!null) min:48(float!null) - │ ├── stats: [rows=1, distinct(1)=1, null(1)=0, distinct(3)=1, null(3)=0, distinct(11)=1, null(11)=0, distinct(12)=1, null(12)=0, distinct(14)=1, null(14)=0, distinct(15)=1, null(15)=0, distinct(16)=1, null(16)=0, distinct(17)=0.999911987, null(17)=0, distinct(18)=0.99998212, null(18)=0, distinct(20)=1, null(20)=0, distinct(23)=1, null(23)=0, distinct(48)=1, null(48)=0] - │ ├── key: (17,18) - │ ├── fd: (1)-->(3), (17,18)-->(1,3,11,12,14-16,20,23,48), (1)==(17), (17)==(1), (18)-->(11,12,14-16,23), (20)==(48), (48)==(20) - │ ├── ordering: -15,+23,+11,+(1|17) [actual: -15,+23,+11,+1] + │ ├── columns: p_partkey:1(int!null) p_mfgr:3(char!null) s_name:12(char!null) s_address:13(varchar!null) s_phone:15(char!null) s_acctbal:16(float!null) s_comment:17(varchar!null) ps_partkey:19(int!null) ps_suppkey:20(int!null) ps_supplycost:22(float!null) n_name:26(char!null) min:57(float!null) + │ ├── stats: [rows=1, distinct(1)=1, null(1)=0, distinct(3)=1, null(3)=0, distinct(12)=1, null(12)=0, distinct(13)=1, null(13)=0, distinct(15)=1, null(15)=0, distinct(16)=1, null(16)=0, distinct(17)=1, null(17)=0, distinct(19)=0.999911987, null(19)=0, distinct(20)=0.99998212, null(20)=0, distinct(22)=1, null(22)=0, distinct(26)=1, null(26)=0, distinct(57)=1, null(57)=0] + │ ├── key: (19,20) + │ ├── fd: (1)-->(3), (19,20)-->(1,3,12,13,15-17,22,26,57), (1)==(19), (19)==(1), (20)-->(12,13,15-17,26), (22)==(57), (57)==(22) + │ ├── ordering: -16,+26,+12,+(1|19) [actual: -16,+26,+12,+1] │ ├── limit hint: 100.00 │ └── select │ ├── save-table-name: q2_select_4 - │ ├── columns: p_partkey:1(int!null) p_mfgr:3(char!null) s_name:11(char!null) s_address:12(varchar!null) s_phone:14(char!null) s_acctbal:15(float!null) s_comment:16(varchar!null) ps_partkey:17(int!null) ps_suppkey:18(int!null) ps_supplycost:20(float!null) n_name:23(char!null) min:48(float!null) - │ ├── stats: [rows=1, distinct(1)=1, null(1)=0, distinct(3)=1, null(3)=0, distinct(11)=1, null(11)=0, distinct(12)=1, null(12)=0, distinct(14)=1, null(14)=0, distinct(15)=1, null(15)=0, distinct(16)=1, null(16)=0, distinct(17)=0.999911987, null(17)=0, distinct(18)=0.99998212, null(18)=0, distinct(20)=1, null(20)=0, distinct(23)=1, null(23)=0, distinct(48)=1, null(48)=0] - │ ├── key: (17,18) - │ ├── fd: (1)-->(3), (17,18)-->(1,3,11,12,14-16,20,23,48), (1)==(17), (17)==(1), (18)-->(11,12,14-16,23), (20)==(48), (48)==(20) + │ ├── columns: p_partkey:1(int!null) p_mfgr:3(char!null) s_name:12(char!null) s_address:13(varchar!null) s_phone:15(char!null) s_acctbal:16(float!null) s_comment:17(varchar!null) ps_partkey:19(int!null) ps_suppkey:20(int!null) ps_supplycost:22(float!null) n_name:26(char!null) min:57(float!null) + │ ├── stats: [rows=1, distinct(1)=1, null(1)=0, distinct(3)=1, null(3)=0, distinct(12)=1, null(12)=0, distinct(13)=1, null(13)=0, distinct(15)=1, null(15)=0, distinct(16)=1, null(16)=0, distinct(17)=1, null(17)=0, distinct(19)=0.999911987, null(19)=0, distinct(20)=0.99998212, null(20)=0, distinct(22)=1, null(22)=0, distinct(26)=1, null(26)=0, distinct(57)=1, null(57)=0] + │ ├── key: (19,20) + │ ├── fd: (1)-->(3), (19,20)-->(1,3,12,13,15-17,22,26,57), (1)==(19), (19)==(1), (20)-->(12,13,15-17,26), (22)==(57), (57)==(22) │ ├── group-by │ │ ├── save-table-name: q2_group_by_5 - │ │ ├── columns: p_partkey:1(int!null) p_mfgr:3(char!null) s_name:11(char!null) s_address:12(varchar!null) s_phone:14(char!null) s_acctbal:15(float!null) s_comment:16(varchar!null) ps_partkey:17(int!null) ps_suppkey:18(int!null) ps_supplycost:20(float!null) n_name:23(char!null) min:48(float!null) - │ │ ├── grouping columns: ps_partkey:17(int!null) ps_suppkey:18(int!null) - │ │ ├── stats: [rows=1476.95652, distinct(1)=1476.95652, null(1)=0, distinct(3)=1476.95652, null(3)=0, distinct(11)=1476.95652, null(11)=0, distinct(12)=1476.95652, null(12)=0, distinct(14)=1476.95652, null(14)=0, distinct(15)=1476.95652, null(15)=0, distinct(16)=1476.95652, null(16)=0, distinct(17)=1172.24444, null(17)=0, distinct(18)=1402.87938, null(18)=0, distinct(20)=1476.95652, null(20)=0, distinct(23)=1476.95652, null(23)=0, distinct(48)=1476.95652, null(48)=0, distinct(17,18)=1476.95652, null(17,18)=0] - │ │ ├── key: (17,18) - │ │ ├── fd: (1)-->(3), (17,18)-->(1,3,11,12,14-16,20,23,48), (1)==(17), (17)==(1), (18)-->(11,12,14-16,23) + │ │ ├── columns: p_partkey:1(int!null) p_mfgr:3(char!null) s_name:12(char!null) s_address:13(varchar!null) s_phone:15(char!null) s_acctbal:16(float!null) s_comment:17(varchar!null) ps_partkey:19(int!null) ps_suppkey:20(int!null) ps_supplycost:22(float!null) n_name:26(char!null) min:57(float!null) + │ │ ├── grouping columns: ps_partkey:19(int!null) ps_suppkey:20(int!null) + │ │ ├── stats: [rows=1476.95652, distinct(1)=1476.95652, null(1)=0, distinct(3)=1476.95652, null(3)=0, distinct(12)=1476.95652, null(12)=0, distinct(13)=1476.95652, null(13)=0, distinct(15)=1476.95652, null(15)=0, distinct(16)=1476.95652, null(16)=0, distinct(17)=1476.95652, null(17)=0, distinct(19)=1172.24444, null(19)=0, distinct(20)=1402.87938, null(20)=0, distinct(22)=1476.95652, null(22)=0, distinct(26)=1476.95652, null(26)=0, distinct(57)=1476.95652, null(57)=0, distinct(19,20)=1476.95652, null(19,20)=0] + │ │ ├── key: (19,20) + │ │ ├── fd: (1)-->(3), (19,20)-->(1,3,12,13,15-17,22,26,57), (1)==(19), (19)==(1), (20)-->(12,13,15-17,26) │ │ ├── inner-join (hash) │ │ │ ├── save-table-name: q2_inner_join_6 - │ │ │ ├── columns: p_partkey:1(int!null) p_mfgr:3(char!null) p_type:5(varchar!null) p_size:6(int!null) s_suppkey:10(int!null) s_name:11(char!null) s_address:12(varchar!null) s_nationkey:13(int!null) s_phone:14(char!null) s_acctbal:15(float!null) s_comment:16(varchar!null) ps_partkey:17(int!null) ps_suppkey:18(int!null) ps_supplycost:20(float!null) n_nationkey:22(int!null) n_name:23(char!null) n_regionkey:24(int!null) r_regionkey:26(int!null) r_name:27(char!null) ps_partkey:29(int!null) ps_suppkey:30(int!null) ps_supplycost:32(float!null) s_suppkey:34(int!null) s_nationkey:37(int!null) n_nationkey:41(int!null) n_regionkey:43(int!null) r_regionkey:45(int!null) r_name:46(char!null) + │ │ │ ├── columns: p_partkey:1(int!null) p_mfgr:3(char!null) p_type:5(varchar!null) p_size:6(int!null) s_suppkey:11(int!null) s_name:12(char!null) s_address:13(varchar!null) s_nationkey:14(int!null) s_phone:15(char!null) s_acctbal:16(float!null) s_comment:17(varchar!null) ps_partkey:19(int!null) ps_suppkey:20(int!null) ps_supplycost:22(float!null) n_nationkey:25(int!null) n_name:26(char!null) n_regionkey:27(int!null) r_regionkey:30(int!null) r_name:31(char!null) ps_partkey:34(int!null) ps_suppkey:35(int!null) ps_supplycost:37(float!null) s_suppkey:40(int!null) s_nationkey:43(int!null) n_nationkey:48(int!null) n_regionkey:50(int!null) r_regionkey:53(int!null) r_name:54(char!null) │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ ├── stats: [rows=2818.04959, distinct(1)=1333.31636, null(1)=0, distinct(3)=5, null(3)=0, distinct(5)=149.999616, null(5)=0, distinct(6)=1, null(6)=0, distinct(10)=1402.87938, null(10)=0, distinct(11)=1403.43197, null(11)=0, distinct(12)=1403.51028, null(12)=0, distinct(13)=5, null(13)=0, distinct(14)=1403.51028, null(14)=0, distinct(15)=1403.25125, null(15)=0, distinct(16)=1402.99052, null(16)=0, distinct(17)=1172.24444, null(17)=0, distinct(18)=1402.87938, null(18)=0, distinct(20)=1472.76855, null(20)=0, distinct(22)=5, null(22)=0, distinct(23)=5, null(23)=0, distinct(24)=1, null(24)=0, distinct(26)=1, null(26)=0, distinct(27)=0.996222107, null(27)=0, distinct(29)=1333.31636, null(29)=0, distinct(30)=1444.36606, null(30)=0, distinct(32)=2769.15811, null(32)=0, distinct(34)=1444.36606, null(34)=0, distinct(37)=5, null(37)=0, distinct(41)=5, null(41)=0, distinct(43)=1, null(43)=0, distinct(45)=1, null(45)=0, distinct(46)=0.996222107, null(46)=0, distinct(17,18)=1476.95652, null(17,18)=0] - │ │ │ ├── key: (18,29,34) - │ │ │ ├── fd: ()-->(6,27,46), (1)-->(3,5), (10)-->(11-16), (17,18)-->(20), (10)==(18), (18)==(10), (22)-->(23,24), (24)==(26), (26)==(24), (13)==(22), (22)==(13), (1)==(17,29), (17)==(1,29), (29,30)-->(32), (34)-->(37), (41)-->(43), (43)==(45), (45)==(43), (37)==(41), (41)==(37), (30)==(34), (34)==(30), (29)==(1,17) + │ │ │ ├── stats: [rows=2818.04959, distinct(1)=1333.31636, null(1)=0, distinct(3)=5, null(3)=0, distinct(5)=149.999616, null(5)=0, distinct(6)=1, null(6)=0, distinct(11)=1402.87938, null(11)=0, distinct(12)=1403.43197, null(12)=0, distinct(13)=1403.51028, null(13)=0, distinct(14)=5, null(14)=0, distinct(15)=1403.51028, null(15)=0, distinct(16)=1403.25125, null(16)=0, distinct(17)=1402.99052, null(17)=0, distinct(19)=1172.24444, null(19)=0, distinct(20)=1402.87938, null(20)=0, distinct(22)=1472.76855, null(22)=0, distinct(25)=5, null(25)=0, distinct(26)=5, null(26)=0, distinct(27)=1, null(27)=0, distinct(30)=1, null(30)=0, distinct(31)=0.996222107, null(31)=0, distinct(34)=1333.31636, null(34)=0, distinct(35)=1444.36606, null(35)=0, distinct(37)=2769.15811, null(37)=0, distinct(40)=1444.36606, null(40)=0, distinct(43)=5, null(43)=0, distinct(48)=5, null(48)=0, distinct(50)=1, null(50)=0, distinct(53)=1, null(53)=0, distinct(54)=0.996222107, null(54)=0, distinct(19,20)=1476.95652, null(19,20)=0] + │ │ │ ├── key: (20,34,40) + │ │ │ ├── fd: ()-->(6,31,54), (1)-->(3,5), (11)-->(12-17), (19,20)-->(22), (11)==(20), (20)==(11), (25)-->(26,27), (27)==(30), (30)==(27), (14)==(25), (25)==(14), (1)==(19,34), (19)==(1,34), (34,35)-->(37), (40)-->(43), (48)-->(50), (50)==(53), (53)==(50), (43)==(48), (48)==(43), (35)==(40), (40)==(35), (34)==(1,19) │ │ │ ├── inner-join (lookup partsupp) │ │ │ │ ├── save-table-name: q2_lookup_join_7 - │ │ │ │ ├── columns: p_partkey:1(int!null) p_mfgr:3(char!null) p_type:5(varchar!null) p_size:6(int!null) s_suppkey:10(int!null) s_name:11(char!null) s_address:12(varchar!null) s_nationkey:13(int!null) s_phone:14(char!null) s_acctbal:15(float!null) s_comment:16(varchar!null) ps_partkey:17(int!null) ps_suppkey:18(int!null) ps_supplycost:20(float!null) n_nationkey:22(int!null) n_name:23(char!null) n_regionkey:24(int!null) r_regionkey:26(int!null) r_name:27(char!null) ps_partkey:29(int!null) ps_suppkey:30(int!null) ps_supplycost:32(float!null) - │ │ │ │ ├── key columns: [1] = [29] - │ │ │ │ ├── stats: [rows=4452.37425, distinct(1)=1333.31636, null(1)=0, distinct(3)=5, null(3)=0, distinct(5)=150, null(5)=0, distinct(6)=1, null(6)=0, distinct(10)=3587.31881, null(10)=0, distinct(11)=3592.54448, null(11)=0, distinct(12)=3593.28589, null(12)=0, distinct(13)=25, null(13)=0, distinct(14)=3593.28589, null(14)=0, distinct(15)=3590.83442, null(15)=0, distinct(16)=3588.36899, null(16)=0, distinct(17)=4401.6666, null(17)=0, distinct(18)=3587.31881, null(18)=0, distinct(20)=4355.01344, null(20)=0, distinct(22)=25, null(22)=0, distinct(23)=25, null(23)=0, distinct(24)=1, null(24)=0, distinct(26)=1, null(26)=0, distinct(27)=1, null(27)=0, distinct(29)=1333.31636, null(29)=0, distinct(30)=3587.31881, null(30)=0, distinct(32)=4355.01344, null(32)=0] - │ │ │ │ ├── key: (18,29,30) - │ │ │ │ ├── fd: ()-->(6,27), (1)-->(3,5), (22)-->(23,24), (10)-->(11-16), (17,18)-->(20), (29,30)-->(32), (17)==(1,29), (29)==(1,17), (10)==(18), (18)==(10), (13)==(22), (22)==(13), (24)==(26), (26)==(24), (1)==(17,29) + │ │ │ │ ├── columns: p_partkey:1(int!null) p_mfgr:3(char!null) p_type:5(varchar!null) p_size:6(int!null) s_suppkey:11(int!null) s_name:12(char!null) s_address:13(varchar!null) s_nationkey:14(int!null) s_phone:15(char!null) s_acctbal:16(float!null) s_comment:17(varchar!null) ps_partkey:19(int!null) ps_suppkey:20(int!null) ps_supplycost:22(float!null) n_nationkey:25(int!null) n_name:26(char!null) n_regionkey:27(int!null) r_regionkey:30(int!null) r_name:31(char!null) ps_partkey:34(int!null) ps_suppkey:35(int!null) ps_supplycost:37(float!null) + │ │ │ │ ├── key columns: [1] = [34] + │ │ │ │ ├── stats: [rows=4452.37425, distinct(1)=1333.31636, null(1)=0, distinct(3)=5, null(3)=0, distinct(5)=150, null(5)=0, distinct(6)=1, null(6)=0, distinct(11)=3587.31881, null(11)=0, distinct(12)=3592.54448, null(12)=0, distinct(13)=3593.28589, null(13)=0, distinct(14)=25, null(14)=0, distinct(15)=3593.28589, null(15)=0, distinct(16)=3590.83442, null(16)=0, distinct(17)=3588.36899, null(17)=0, distinct(19)=4401.6666, null(19)=0, distinct(20)=3587.31881, null(20)=0, distinct(22)=4355.01344, null(22)=0, distinct(25)=25, null(25)=0, distinct(26)=25, null(26)=0, distinct(27)=1, null(27)=0, distinct(30)=1, null(30)=0, distinct(31)=1, null(31)=0, distinct(34)=1333.31636, null(34)=0, distinct(35)=3587.31881, null(35)=0, distinct(37)=4355.01344, null(37)=0] + │ │ │ │ ├── key: (20,34,35) + │ │ │ │ ├── fd: ()-->(6,31), (1)-->(3,5), (25)-->(26,27), (11)-->(12-17), (19,20)-->(22), (34,35)-->(37), (19)==(1,34), (34)==(1,19), (11)==(20), (20)==(11), (14)==(25), (25)==(14), (27)==(30), (30)==(27), (1)==(19,34) │ │ │ │ ├── inner-join (hash) │ │ │ │ │ ├── save-table-name: q2_inner_join_8 - │ │ │ │ │ ├── columns: p_partkey:1(int!null) p_mfgr:3(char!null) p_type:5(varchar!null) p_size:6(int!null) s_suppkey:10(int!null) s_name:11(char!null) s_address:12(varchar!null) s_nationkey:13(int!null) s_phone:14(char!null) s_acctbal:15(float!null) s_comment:16(varchar!null) ps_partkey:17(int!null) ps_suppkey:18(int!null) ps_supplycost:20(float!null) n_nationkey:22(int!null) n_name:23(char!null) n_regionkey:24(int!null) r_regionkey:26(int!null) r_name:27(char!null) + │ │ │ │ │ ├── columns: p_partkey:1(int!null) p_mfgr:3(char!null) p_type:5(varchar!null) p_size:6(int!null) s_suppkey:11(int!null) s_name:12(char!null) s_address:13(varchar!null) s_nationkey:14(int!null) s_phone:15(char!null) s_acctbal:16(float!null) s_comment:17(varchar!null) ps_partkey:19(int!null) ps_suppkey:20(int!null) ps_supplycost:22(float!null) n_nationkey:25(int!null) n_name:26(char!null) n_regionkey:27(int!null) r_regionkey:30(int!null) r_name:31(char!null) │ │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ │ │ ├── stats: [rows=1931.84139, distinct(1)=1333.31636, null(1)=0, distinct(3)=5, null(3)=0, distinct(5)=149.999617, null(5)=0, distinct(6)=1, null(6)=0, distinct(10)=1755.38174, null(10)=0, distinct(11)=1756.54107, null(11)=0, distinct(12)=1756.70545, null(12)=0, distinct(13)=5, null(13)=0, distinct(14)=1756.70545, null(14)=0, distinct(15)=1756.16183, null(15)=0, distinct(16)=1755.61483, null(16)=0, distinct(17)=1333.31636, null(17)=0, distinct(18)=1755.38174, null(18)=0, distinct(20)=1908.96352, null(20)=0, distinct(22)=5, null(22)=0, distinct(23)=5, null(23)=0, distinct(24)=1, null(24)=0, distinct(26)=1, null(26)=0, distinct(27)=0.996222107, null(27)=0, distinct(17,18)=1918.63711, null(17,18)=0] - │ │ │ │ │ ├── key: (17,18) - │ │ │ │ │ ├── fd: ()-->(6,27), (1)-->(3,5), (10)-->(11-16), (17,18)-->(20), (10)==(18), (18)==(10), (22)-->(23,24), (24)==(26), (26)==(24), (13)==(22), (22)==(13), (1)==(17), (17)==(1) + │ │ │ │ │ ├── stats: [rows=1931.84139, distinct(1)=1333.31636, null(1)=0, distinct(3)=5, null(3)=0, distinct(5)=149.999617, null(5)=0, distinct(6)=1, null(6)=0, distinct(11)=1755.38174, null(11)=0, distinct(12)=1756.54107, null(12)=0, distinct(13)=1756.70545, null(13)=0, distinct(14)=5, null(14)=0, distinct(15)=1756.70545, null(15)=0, distinct(16)=1756.16183, null(16)=0, distinct(17)=1755.61483, null(17)=0, distinct(19)=1333.31636, null(19)=0, distinct(20)=1755.38174, null(20)=0, distinct(22)=1908.96352, null(22)=0, distinct(25)=5, null(25)=0, distinct(26)=5, null(26)=0, distinct(27)=1, null(27)=0, distinct(30)=1, null(30)=0, distinct(31)=0.996222107, null(31)=0, distinct(19,20)=1918.63711, null(19,20)=0] + │ │ │ │ │ ├── key: (19,20) + │ │ │ │ │ ├── fd: ()-->(6,31), (1)-->(3,5), (11)-->(12-17), (19,20)-->(22), (11)==(20), (20)==(11), (25)-->(26,27), (27)==(30), (30)==(27), (14)==(25), (25)==(14), (1)==(19), (19)==(1) │ │ │ │ │ ├── inner-join (lookup partsupp) │ │ │ │ │ │ ├── save-table-name: q2_lookup_join_9 - │ │ │ │ │ │ ├── columns: p_partkey:1(int!null) p_mfgr:3(char!null) p_type:5(varchar!null) p_size:6(int!null) ps_partkey:17(int!null) ps_suppkey:18(int!null) ps_supplycost:20(float!null) - │ │ │ │ │ │ ├── key columns: [1] = [17] - │ │ │ │ │ │ ├── stats: [rows=5353.65044, distinct(1)=1333.31636, null(1)=0, distinct(3)=5, null(3)=0, distinct(5)=150, null(5)=0, distinct(6)=1, null(6)=0, distinct(17)=1333.31636, null(17)=0, distinct(18)=4137.30947, null(18)=0, distinct(20)=5213.40104, null(20)=0] - │ │ │ │ │ │ ├── key: (17,18) - │ │ │ │ │ │ ├── fd: ()-->(6), (1)-->(3,5), (17,18)-->(20), (1)==(17), (17)==(1) + │ │ │ │ │ │ ├── columns: p_partkey:1(int!null) p_mfgr:3(char!null) p_type:5(varchar!null) p_size:6(int!null) ps_partkey:19(int!null) ps_suppkey:20(int!null) ps_supplycost:22(float!null) + │ │ │ │ │ │ ├── key columns: [1] = [19] + │ │ │ │ │ │ ├── stats: [rows=5353.65044, distinct(1)=1333.31636, null(1)=0, distinct(3)=5, null(3)=0, distinct(5)=150, null(5)=0, distinct(6)=1, null(6)=0, distinct(19)=1333.31636, null(19)=0, distinct(20)=4137.30947, null(20)=0, distinct(22)=5213.40104, null(22)=0] + │ │ │ │ │ │ ├── key: (19,20) + │ │ │ │ │ │ ├── fd: ()-->(6), (1)-->(3,5), (19,20)-->(22), (1)==(19), (19)==(1) │ │ │ │ │ │ ├── select │ │ │ │ │ │ │ ├── save-table-name: q2_select_10 │ │ │ │ │ │ │ ├── columns: p_partkey:1(int!null) p_mfgr:3(char!null) p_type:5(varchar!null) p_size:6(int!null) @@ -153,132 +153,132 @@ project │ │ │ │ │ │ └── filters (true) │ │ │ │ │ ├── inner-join (hash) │ │ │ │ │ │ ├── save-table-name: q2_inner_join_12 - │ │ │ │ │ │ ├── columns: s_suppkey:10(int!null) s_name:11(char!null) s_address:12(varchar!null) s_nationkey:13(int!null) s_phone:14(char!null) s_acctbal:15(float!null) s_comment:16(varchar!null) n_nationkey:22(int!null) n_name:23(char!null) n_regionkey:24(int!null) r_regionkey:26(int!null) r_name:27(char!null) + │ │ │ │ │ │ ├── columns: s_suppkey:11(int!null) s_name:12(char!null) s_address:13(varchar!null) s_nationkey:14(int!null) s_phone:15(char!null) s_acctbal:16(float!null) s_comment:17(varchar!null) n_nationkey:25(int!null) n_name:26(char!null) n_regionkey:27(int!null) r_regionkey:30(int!null) r_name:31(char!null) │ │ │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ │ │ │ ├── stats: [rows=2000, distinct(10)=1844.80594, null(10)=0, distinct(11)=1846.09084, null(11)=0, distinct(12)=1846.27302, null(12)=0, distinct(13)=5, null(13)=0, distinct(14)=1846.27302, null(14)=0, distinct(15)=1845.67052, null(15)=0, distinct(16)=1845.06427, null(16)=0, distinct(22)=5, null(22)=0, distinct(23)=5, null(23)=0, distinct(24)=1, null(24)=0, distinct(26)=1, null(26)=0, distinct(27)=0.996222107, null(27)=0] - │ │ │ │ │ │ ├── key: (10) - │ │ │ │ │ │ ├── fd: ()-->(27), (10)-->(11-16), (22)-->(23,24), (24)==(26), (26)==(24), (13)==(22), (22)==(13) + │ │ │ │ │ │ ├── stats: [rows=2000, distinct(11)=1844.80594, null(11)=0, distinct(12)=1846.09084, null(12)=0, distinct(13)=1846.27302, null(13)=0, distinct(14)=5, null(14)=0, distinct(15)=1846.27302, null(15)=0, distinct(16)=1845.67052, null(16)=0, distinct(17)=1845.06427, null(17)=0, distinct(25)=5, null(25)=0, distinct(26)=5, null(26)=0, distinct(27)=1, null(27)=0, distinct(30)=1, null(30)=0, distinct(31)=0.996222107, null(31)=0] + │ │ │ │ │ │ ├── key: (11) + │ │ │ │ │ │ ├── fd: ()-->(31), (11)-->(12-17), (25)-->(26,27), (27)==(30), (30)==(27), (14)==(25), (25)==(14) │ │ │ │ │ │ ├── scan supplier │ │ │ │ │ │ │ ├── save-table-name: q2_scan_13 - │ │ │ │ │ │ │ ├── columns: s_suppkey:10(int!null) s_name:11(char!null) s_address:12(varchar!null) s_nationkey:13(int!null) s_phone:14(char!null) s_acctbal:15(float!null) s_comment:16(varchar!null) - │ │ │ │ │ │ │ ├── stats: [rows=10000, distinct(10)=9920, null(10)=0, distinct(11)=9990, null(11)=0, distinct(12)=10000, null(12)=0, distinct(13)=25, null(13)=0, distinct(14)=10000, null(14)=0, distinct(15)=9967, null(15)=0, distinct(16)=9934, null(16)=0] - │ │ │ │ │ │ │ │ histogram(10)= 0 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 + │ │ │ │ │ │ │ ├── columns: s_suppkey:11(int!null) s_name:12(char!null) s_address:13(varchar!null) s_nationkey:14(int!null) s_phone:15(char!null) s_acctbal:16(float!null) s_comment:17(varchar!null) + │ │ │ │ │ │ │ ├── stats: [rows=10000, distinct(11)=9920, null(11)=0, distinct(12)=9990, null(12)=0, distinct(13)=10000, null(13)=0, distinct(14)=25, null(14)=0, distinct(15)=10000, null(15)=0, distinct(16)=9967, null(16)=0, distinct(17)=9934, null(17)=0] + │ │ │ │ │ │ │ │ histogram(11)= 0 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 │ │ │ │ │ │ │ │ <--- 1 ---- 51 ---- 101 ---- 151 ---- 201 ---- 251 ---- 301 ---- 351 ---- 401 ---- 451 ---- 501 ---- 551 ---- 601 ---- 651 ---- 701 ---- 751 ---- 801 ---- 851 ---- 901 ---- 951 ---- 1001 ---- 1051 ---- 1101 ---- 1151 ---- 1201 ---- 1251 ---- 1301 ---- 1351 ---- 1401 ---- 1451 ---- 1501 ---- 1551 ---- 1601 ---- 1651 ---- 1701 ---- 1751 ---- 1801 ---- 1851 ---- 1901 ---- 1951 ---- 2001 ---- 2051 ---- 2101 ---- 2151 ---- 2201 ---- 2251 ---- 2301 ---- 2351 ---- 2401 ---- 2451 ---- 2501 ---- 2551 ---- 2601 ---- 2651 ---- 2701 ---- 2751 ---- 2801 ---- 2851 ---- 2901 ---- 2951 ---- 3001 ---- 3051 ---- 3101 ---- 3151 ---- 3201 ---- 3251 ---- 3301 ---- 3351 ---- 3401 ---- 3451 ---- 3501 ---- 3551 ---- 3601 ---- 3651 ---- 3701 ---- 3751 ---- 3801 ---- 3851 ---- 3901 ---- 3951 ---- 4001 ---- 4051 ---- 4101 ---- 4151 ---- 4201 ---- 4251 ---- 4301 ---- 4351 ---- 4401 ---- 4451 ---- 4501 ---- 4551 ---- 4601 ---- 4651 ---- 4701 ---- 4751 ---- 4801 ---- 4851 ---- 4901 ---- 4951 ---- 5001 ---- 5051 ---- 5101 ---- 5151 ---- 5201 ---- 5251 ---- 5301 ---- 5351 ---- 5401 ---- 5451 ---- 5501 ---- 5551 ---- 5601 ---- 5651 ---- 5701 ---- 5751 ---- 5801 ---- 5851 ---- 5901 ---- 5951 ---- 6001 ---- 6051 ---- 6101 ---- 6151 ---- 6201 ---- 6251 ---- 6301 ---- 6351 ---- 6401 ---- 6451 ---- 6501 ---- 6551 ---- 6601 ---- 6651 ---- 6701 ---- 6751 ---- 6801 ---- 6851 ---- 6901 ---- 6951 ---- 7001 ---- 7051 ---- 7101 ---- 7151 ---- 7201 ---- 7251 ---- 7301 ---- 7351 ---- 7401 ---- 7451 ---- 7501 ---- 7552 ---- 7603 ---- 7654 ---- 7705 ---- 7756 ---- 7807 ---- 7858 ---- 7909 ---- 7960 ---- 8011 ---- 8062 ---- 8113 ---- 8164 ---- 8215 ---- 8266 ---- 8317 ---- 8368 ---- 8419 ---- 8470 ---- 8521 ---- 8572 ---- 8623 ---- 8674 ---- 8725 ---- 8776 ---- 8827 ---- 8878 ---- 8929 ---- 8980 ---- 9031 ---- 9082 ---- 9133 ---- 9184 ---- 9235 ---- 9286 ---- 9337 ---- 9388 ---- 9439 ---- 9490 ---- 9541 ---- 9592 ---- 9643 ---- 9694 ---- 9745 ---- 9796 ---- 9847 ---- 9898 ---- 9949 ---- 10000 - │ │ │ │ │ │ │ │ histogram(13)= 0 420 0 413 0 397 0 412 0 415 0 380 0 402 0 396 0 415 0 405 0 393 0 438 0 377 0 362 0 376 0 373 0 406 0 421 0 407 0 398 0 411 0 399 0 401 0 390 0 393 + │ │ │ │ │ │ │ │ histogram(14)= 0 420 0 413 0 397 0 412 0 415 0 380 0 402 0 396 0 415 0 405 0 393 0 438 0 377 0 362 0 376 0 373 0 406 0 421 0 407 0 398 0 411 0 399 0 401 0 390 0 393 │ │ │ │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 --- 5 --- 6 --- 7 --- 8 --- 9 --- 10 --- 11 --- 12 --- 13 --- 14 --- 15 --- 16 --- 17 --- 18 --- 19 --- 20 --- 21 --- 22 --- 23 --- 24 - │ │ │ │ │ │ │ ├── key: (10) - │ │ │ │ │ │ │ └── fd: (10)-->(11-16) + │ │ │ │ │ │ │ ├── key: (11) + │ │ │ │ │ │ │ └── fd: (11)-->(12-17) │ │ │ │ │ │ ├── inner-join (hash) │ │ │ │ │ │ │ ├── save-table-name: q2_inner_join_14 - │ │ │ │ │ │ │ ├── columns: n_nationkey:22(int!null) n_name:23(char!null) n_regionkey:24(int!null) r_regionkey:26(int!null) r_name:27(char!null) + │ │ │ │ │ │ │ ├── columns: n_nationkey:25(int!null) n_name:26(char!null) n_regionkey:27(int!null) r_regionkey:30(int!null) r_name:31(char!null) │ │ │ │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ │ │ │ │ ├── stats: [rows=5, distinct(22)=5, null(22)=0, distinct(23)=5, null(23)=0, distinct(24)=1, null(24)=0, distinct(26)=1, null(26)=0, distinct(27)=0.996222107, null(27)=0] - │ │ │ │ │ │ │ ├── key: (22) - │ │ │ │ │ │ │ ├── fd: ()-->(27), (22)-->(23,24), (24)==(26), (26)==(24) + │ │ │ │ │ │ │ ├── stats: [rows=5, distinct(25)=5, null(25)=0, distinct(26)=5, null(26)=0, distinct(27)=1, null(27)=0, distinct(30)=1, null(30)=0, distinct(31)=0.996222107, null(31)=0] + │ │ │ │ │ │ │ ├── key: (25) + │ │ │ │ │ │ │ ├── fd: ()-->(31), (25)-->(26,27), (27)==(30), (30)==(27) │ │ │ │ │ │ │ ├── scan nation │ │ │ │ │ │ │ │ ├── save-table-name: q2_scan_15 - │ │ │ │ │ │ │ │ ├── columns: n_nationkey:22(int!null) n_name:23(char!null) n_regionkey:24(int!null) - │ │ │ │ │ │ │ │ ├── stats: [rows=25, distinct(22)=25, null(22)=0, distinct(23)=25, null(23)=0, distinct(24)=5, null(24)=0] - │ │ │ │ │ │ │ │ │ histogram(22)= 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 + │ │ │ │ │ │ │ │ ├── columns: n_nationkey:25(int!null) n_name:26(char!null) n_regionkey:27(int!null) + │ │ │ │ │ │ │ │ ├── stats: [rows=25, distinct(25)=25, null(25)=0, distinct(26)=25, null(26)=0, distinct(27)=5, null(27)=0] + │ │ │ │ │ │ │ │ │ histogram(25)= 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 │ │ │ │ │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 --- 5 --- 6 --- 7 --- 8 --- 9 --- 10 --- 11 --- 12 --- 13 --- 14 --- 15 --- 16 --- 17 --- 18 --- 19 --- 20 --- 21 --- 22 --- 23 --- 24 - │ │ │ │ │ │ │ │ │ histogram(24)= 0 5 0 5 0 5 0 5 0 5 + │ │ │ │ │ │ │ │ │ histogram(27)= 0 5 0 5 0 5 0 5 0 5 │ │ │ │ │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 - │ │ │ │ │ │ │ │ ├── key: (22) - │ │ │ │ │ │ │ │ └── fd: (22)-->(23,24) + │ │ │ │ │ │ │ │ ├── key: (25) + │ │ │ │ │ │ │ │ └── fd: (25)-->(26,27) │ │ │ │ │ │ │ ├── select │ │ │ │ │ │ │ │ ├── save-table-name: q2_select_16 - │ │ │ │ │ │ │ │ ├── columns: r_regionkey:26(int!null) r_name:27(char!null) - │ │ │ │ │ │ │ │ ├── stats: [rows=1, distinct(26)=1, null(26)=0, distinct(27)=1, null(27)=0] - │ │ │ │ │ │ │ │ ├── key: (26) - │ │ │ │ │ │ │ │ ├── fd: ()-->(27) + │ │ │ │ │ │ │ │ ├── columns: r_regionkey:30(int!null) r_name:31(char!null) + │ │ │ │ │ │ │ │ ├── stats: [rows=1, distinct(30)=1, null(30)=0, distinct(31)=1, null(31)=0] + │ │ │ │ │ │ │ │ ├── key: (30) + │ │ │ │ │ │ │ │ ├── fd: ()-->(31) │ │ │ │ │ │ │ │ ├── scan region │ │ │ │ │ │ │ │ │ ├── save-table-name: q2_scan_17 - │ │ │ │ │ │ │ │ │ ├── columns: r_regionkey:26(int!null) r_name:27(char!null) - │ │ │ │ │ │ │ │ │ ├── stats: [rows=5, distinct(26)=5, null(26)=0, distinct(27)=5, null(27)=0] - │ │ │ │ │ │ │ │ │ │ histogram(26)= 0 1 0 1 0 1 0 1 0 1 + │ │ │ │ │ │ │ │ │ ├── columns: r_regionkey:30(int!null) r_name:31(char!null) + │ │ │ │ │ │ │ │ │ ├── stats: [rows=5, distinct(30)=5, null(30)=0, distinct(31)=5, null(31)=0] + │ │ │ │ │ │ │ │ │ │ histogram(30)= 0 1 0 1 0 1 0 1 0 1 │ │ │ │ │ │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 - │ │ │ │ │ │ │ │ │ ├── key: (26) - │ │ │ │ │ │ │ │ │ └── fd: (26)-->(27) + │ │ │ │ │ │ │ │ │ ├── key: (30) + │ │ │ │ │ │ │ │ │ └── fd: (30)-->(31) │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ └── r_name:27 = 'EUROPE' [type=bool, outer=(27), constraints=(/27: [/'EUROPE' - /'EUROPE']; tight), fd=()-->(27)] + │ │ │ │ │ │ │ │ └── r_name:31 = 'EUROPE' [type=bool, outer=(31), constraints=(/31: [/'EUROPE' - /'EUROPE']; tight), fd=()-->(31)] │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── n_regionkey:24 = r_regionkey:26 [type=bool, outer=(24,26), constraints=(/24: (/NULL - ]; /26: (/NULL - ]), fd=(24)==(26), (26)==(24)] + │ │ │ │ │ │ │ └── n_regionkey:27 = r_regionkey:30 [type=bool, outer=(27,30), constraints=(/27: (/NULL - ]; /30: (/NULL - ]), fd=(27)==(30), (30)==(27)] │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── s_nationkey:13 = n_nationkey:22 [type=bool, outer=(13,22), constraints=(/13: (/NULL - ]; /22: (/NULL - ]), fd=(13)==(22), (22)==(13)] + │ │ │ │ │ │ └── s_nationkey:14 = n_nationkey:25 [type=bool, outer=(14,25), constraints=(/14: (/NULL - ]; /25: (/NULL - ]), fd=(14)==(25), (25)==(14)] │ │ │ │ │ └── filters - │ │ │ │ │ └── s_suppkey:10 = ps_suppkey:18 [type=bool, outer=(10,18), constraints=(/10: (/NULL - ]; /18: (/NULL - ]), fd=(10)==(18), (18)==(10)] + │ │ │ │ │ └── s_suppkey:11 = ps_suppkey:20 [type=bool, outer=(11,20), constraints=(/11: (/NULL - ]; /20: (/NULL - ]), fd=(11)==(20), (20)==(11)] │ │ │ │ └── filters (true) │ │ │ ├── inner-join (lookup supplier@s_nk) │ │ │ │ ├── save-table-name: q2_lookup_join_18 - │ │ │ │ ├── columns: s_suppkey:34(int!null) s_nationkey:37(int!null) n_nationkey:41(int!null) n_regionkey:43(int!null) r_regionkey:45(int!null) r_name:46(char!null) - │ │ │ │ ├── key columns: [41] = [37] - │ │ │ │ ├── stats: [rows=2000, distinct(34)=1844.80594, null(34)=0, distinct(37)=5, null(37)=0, distinct(41)=5, null(41)=0, distinct(43)=1, null(43)=0, distinct(45)=1, null(45)=0, distinct(46)=0.996222107, null(46)=0] - │ │ │ │ ├── key: (34) - │ │ │ │ ├── fd: ()-->(46), (34)-->(37), (41)-->(43), (43)==(45), (45)==(43), (37)==(41), (41)==(37) + │ │ │ │ ├── columns: s_suppkey:40(int!null) s_nationkey:43(int!null) n_nationkey:48(int!null) n_regionkey:50(int!null) r_regionkey:53(int!null) r_name:54(char!null) + │ │ │ │ ├── key columns: [48] = [43] + │ │ │ │ ├── stats: [rows=2000, distinct(40)=1844.80594, null(40)=0, distinct(43)=5, null(43)=0, distinct(48)=5, null(48)=0, distinct(50)=1, null(50)=0, distinct(53)=1, null(53)=0, distinct(54)=0.996222107, null(54)=0] + │ │ │ │ ├── key: (40) + │ │ │ │ ├── fd: ()-->(54), (40)-->(43), (48)-->(50), (50)==(53), (53)==(50), (43)==(48), (48)==(43) │ │ │ │ ├── inner-join (merge) │ │ │ │ │ ├── save-table-name: q2_merge_join_19 - │ │ │ │ │ ├── columns: n_nationkey:41(int!null) n_regionkey:43(int!null) r_regionkey:45(int!null) r_name:46(char!null) - │ │ │ │ │ ├── left ordering: +43 - │ │ │ │ │ ├── right ordering: +45 - │ │ │ │ │ ├── stats: [rows=5, distinct(41)=5, null(41)=0, distinct(43)=1, null(43)=0, distinct(45)=1, null(45)=0, distinct(46)=0.996222107, null(46)=0] - │ │ │ │ │ ├── key: (41) - │ │ │ │ │ ├── fd: ()-->(46), (41)-->(43), (43)==(45), (45)==(43) + │ │ │ │ │ ├── columns: n_nationkey:48(int!null) n_regionkey:50(int!null) r_regionkey:53(int!null) r_name:54(char!null) + │ │ │ │ │ ├── left ordering: +50 + │ │ │ │ │ ├── right ordering: +53 + │ │ │ │ │ ├── stats: [rows=5, distinct(48)=5, null(48)=0, distinct(50)=1, null(50)=0, distinct(53)=1, null(53)=0, distinct(54)=0.996222107, null(54)=0] + │ │ │ │ │ ├── key: (48) + │ │ │ │ │ ├── fd: ()-->(54), (48)-->(50), (50)==(53), (53)==(50) │ │ │ │ │ ├── scan nation@n_rk │ │ │ │ │ │ ├── save-table-name: q2_scan_20 - │ │ │ │ │ │ ├── columns: n_nationkey:41(int!null) n_regionkey:43(int!null) - │ │ │ │ │ │ ├── stats: [rows=25, distinct(41)=25, null(41)=0, distinct(43)=5, null(43)=0] - │ │ │ │ │ │ │ histogram(41)= 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 + │ │ │ │ │ │ ├── columns: n_nationkey:48(int!null) n_regionkey:50(int!null) + │ │ │ │ │ │ ├── stats: [rows=25, distinct(48)=25, null(48)=0, distinct(50)=5, null(50)=0] + │ │ │ │ │ │ │ histogram(48)= 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 │ │ │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 --- 5 --- 6 --- 7 --- 8 --- 9 --- 10 --- 11 --- 12 --- 13 --- 14 --- 15 --- 16 --- 17 --- 18 --- 19 --- 20 --- 21 --- 22 --- 23 --- 24 - │ │ │ │ │ │ │ histogram(43)= 0 5 0 5 0 5 0 5 0 5 + │ │ │ │ │ │ │ histogram(50)= 0 5 0 5 0 5 0 5 0 5 │ │ │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 - │ │ │ │ │ │ ├── key: (41) - │ │ │ │ │ │ ├── fd: (41)-->(43) - │ │ │ │ │ │ └── ordering: +43 + │ │ │ │ │ │ ├── key: (48) + │ │ │ │ │ │ ├── fd: (48)-->(50) + │ │ │ │ │ │ └── ordering: +50 │ │ │ │ │ ├── select │ │ │ │ │ │ ├── save-table-name: q2_select_21 - │ │ │ │ │ │ ├── columns: r_regionkey:45(int!null) r_name:46(char!null) - │ │ │ │ │ │ ├── stats: [rows=1, distinct(45)=1, null(45)=0, distinct(46)=1, null(46)=0] - │ │ │ │ │ │ ├── key: (45) - │ │ │ │ │ │ ├── fd: ()-->(46) - │ │ │ │ │ │ ├── ordering: +45 opt(46) [actual: +45] + │ │ │ │ │ │ ├── columns: r_regionkey:53(int!null) r_name:54(char!null) + │ │ │ │ │ │ ├── stats: [rows=1, distinct(53)=1, null(53)=0, distinct(54)=1, null(54)=0] + │ │ │ │ │ │ ├── key: (53) + │ │ │ │ │ │ ├── fd: ()-->(54) + │ │ │ │ │ │ ├── ordering: +53 opt(54) [actual: +53] │ │ │ │ │ │ ├── scan region │ │ │ │ │ │ │ ├── save-table-name: q2_scan_22 - │ │ │ │ │ │ │ ├── columns: r_regionkey:45(int!null) r_name:46(char!null) - │ │ │ │ │ │ │ ├── stats: [rows=5, distinct(45)=5, null(45)=0, distinct(46)=5, null(46)=0] - │ │ │ │ │ │ │ │ histogram(45)= 0 1 0 1 0 1 0 1 0 1 + │ │ │ │ │ │ │ ├── columns: r_regionkey:53(int!null) r_name:54(char!null) + │ │ │ │ │ │ │ ├── stats: [rows=5, distinct(53)=5, null(53)=0, distinct(54)=5, null(54)=0] + │ │ │ │ │ │ │ │ histogram(53)= 0 1 0 1 0 1 0 1 0 1 │ │ │ │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 - │ │ │ │ │ │ │ ├── key: (45) - │ │ │ │ │ │ │ ├── fd: (45)-->(46) - │ │ │ │ │ │ │ └── ordering: +45 opt(46) [actual: +45] + │ │ │ │ │ │ │ ├── key: (53) + │ │ │ │ │ │ │ ├── fd: (53)-->(54) + │ │ │ │ │ │ │ └── ordering: +53 opt(54) [actual: +53] │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── r_name:46 = 'EUROPE' [type=bool, outer=(46), constraints=(/46: [/'EUROPE' - /'EUROPE']; tight), fd=()-->(46)] + │ │ │ │ │ │ └── r_name:54 = 'EUROPE' [type=bool, outer=(54), constraints=(/54: [/'EUROPE' - /'EUROPE']; tight), fd=()-->(54)] │ │ │ │ │ └── filters (true) │ │ │ │ └── filters (true) │ │ │ └── filters - │ │ │ └── s_suppkey:34 = ps_suppkey:30 [type=bool, outer=(30,34), constraints=(/30: (/NULL - ]; /34: (/NULL - ]), fd=(30)==(34), (34)==(30)] + │ │ │ └── s_suppkey:40 = ps_suppkey:35 [type=bool, outer=(35,40), constraints=(/35: (/NULL - ]; /40: (/NULL - ]), fd=(35)==(40), (40)==(35)] │ │ └── aggregations - │ │ ├── min [as=min:48, type=float, outer=(32)] - │ │ │ └── ps_supplycost:32 [type=float] - │ │ ├── const-agg [as=s_name:11, type=char, outer=(11)] - │ │ │ └── s_name:11 [type=char] - │ │ ├── const-agg [as=s_address:12, type=varchar, outer=(12)] - │ │ │ └── s_address:12 [type=varchar] - │ │ ├── const-agg [as=s_phone:14, type=char, outer=(14)] - │ │ │ └── s_phone:14 [type=char] - │ │ ├── const-agg [as=s_acctbal:15, type=float, outer=(15)] - │ │ │ └── s_acctbal:15 [type=float] - │ │ ├── const-agg [as=s_comment:16, type=varchar, outer=(16)] - │ │ │ └── s_comment:16 [type=varchar] - │ │ ├── const-agg [as=ps_supplycost:20, type=float, outer=(20)] - │ │ │ └── ps_supplycost:20 [type=float] - │ │ ├── const-agg [as=n_name:23, type=char, outer=(23)] - │ │ │ └── n_name:23 [type=char] + │ │ ├── min [as=min:57, type=float, outer=(37)] + │ │ │ └── ps_supplycost:37 [type=float] + │ │ ├── const-agg [as=s_name:12, type=char, outer=(12)] + │ │ │ └── s_name:12 [type=char] + │ │ ├── const-agg [as=s_address:13, type=varchar, outer=(13)] + │ │ │ └── s_address:13 [type=varchar] + │ │ ├── const-agg [as=s_phone:15, type=char, outer=(15)] + │ │ │ └── s_phone:15 [type=char] + │ │ ├── const-agg [as=s_acctbal:16, type=float, outer=(16)] + │ │ │ └── s_acctbal:16 [type=float] + │ │ ├── const-agg [as=s_comment:17, type=varchar, outer=(17)] + │ │ │ └── s_comment:17 [type=varchar] + │ │ ├── const-agg [as=ps_supplycost:22, type=float, outer=(22)] + │ │ │ └── ps_supplycost:22 [type=float] + │ │ ├── const-agg [as=n_name:26, type=char, outer=(26)] + │ │ │ └── n_name:26 [type=char] │ │ ├── const-agg [as=p_mfgr:3, type=char, outer=(3)] │ │ │ └── p_mfgr:3 [type=char] │ │ └── const-agg [as=p_partkey:1, type=int, outer=(1)] │ │ └── p_partkey:1 [type=int] │ └── filters - │ └── ps_supplycost:20 = min:48 [type=bool, outer=(20,48), constraints=(/20: (/NULL - ]; /48: (/NULL - ]), fd=(20)==(48), (48)==(20)] + │ └── ps_supplycost:22 = min:57 [type=bool, outer=(22,57), constraints=(/22: (/NULL - ]; /57: (/NULL - ]), fd=(22)==(57), (57)==(22)] └── 100 [type=int] stats table=q2_project_1 diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q03 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q03 index a47e3af03dd5..dacb6f3dbc9a 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q03 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q03 @@ -42,72 +42,72 @@ LIMIT 10; ---- limit ├── save-table-name: q3_limit_1 - ├── columns: l_orderkey:18(int!null) revenue:35(float!null) o_orderdate:13(date!null) o_shippriority:16(int!null) - ├── internal-ordering: -35,+13 + ├── columns: l_orderkey:20(int!null) revenue:38(float!null) o_orderdate:14(date!null) o_shippriority:17(int!null) + ├── internal-ordering: -38,+14 ├── cardinality: [0 - 10] ├── immutable - ├── stats: [rows=10, distinct(13)=10, null(13)=0, distinct(16)=10, null(16)=0, distinct(18)=10, null(18)=0, distinct(35)=10, null(35)=0] - ├── key: (18) - ├── fd: (18)-->(13,16,35) - ├── ordering: -35,+13 + ├── stats: [rows=10, distinct(14)=10, null(14)=0, distinct(17)=10, null(17)=0, distinct(20)=10, null(20)=0, distinct(38)=10, null(38)=0] + ├── key: (20) + ├── fd: (20)-->(14,17,38) + ├── ordering: -38,+14 ├── sort │ ├── save-table-name: q3_sort_2 - │ ├── columns: o_orderdate:13(date!null) o_shippriority:16(int!null) l_orderkey:18(int!null) sum:35(float!null) + │ ├── columns: o_orderdate:14(date!null) o_shippriority:17(int!null) l_orderkey:20(int!null) sum:38(float!null) │ ├── immutable - │ ├── stats: [rows=359560.406, distinct(13)=359560.406, null(13)=0, distinct(16)=359560.406, null(16)=0, distinct(18)=359560.406, null(18)=0, distinct(35)=359560.406, null(35)=0] - │ ├── key: (18) - │ ├── fd: (18)-->(13,16,35) - │ ├── ordering: -35,+13 + │ ├── stats: [rows=359560.406, distinct(14)=359560.406, null(14)=0, distinct(17)=359560.406, null(17)=0, distinct(20)=359560.406, null(20)=0, distinct(38)=359560.406, null(38)=0] + │ ├── key: (20) + │ ├── fd: (20)-->(14,17,38) + │ ├── ordering: -38,+14 │ ├── limit hint: 10.00 │ └── group-by │ ├── save-table-name: q3_group_by_3 - │ ├── columns: o_orderdate:13(date!null) o_shippriority:16(int!null) l_orderkey:18(int!null) sum:35(float!null) - │ ├── grouping columns: l_orderkey:18(int!null) + │ ├── columns: o_orderdate:14(date!null) o_shippriority:17(int!null) l_orderkey:20(int!null) sum:38(float!null) + │ ├── grouping columns: l_orderkey:20(int!null) │ ├── immutable - │ ├── stats: [rows=359560.406, distinct(13)=359560.406, null(13)=0, distinct(16)=359560.406, null(16)=0, distinct(18)=359560.406, null(18)=0, distinct(35)=359560.406, null(35)=0] - │ ├── key: (18) - │ ├── fd: (18)-->(13,16,35) + │ ├── stats: [rows=359560.406, distinct(14)=359560.406, null(14)=0, distinct(17)=359560.406, null(17)=0, distinct(20)=359560.406, null(20)=0, distinct(38)=359560.406, null(38)=0] + │ ├── key: (20) + │ ├── fd: (20)-->(14,17,38) │ ├── project │ │ ├── save-table-name: q3_project_4 - │ │ ├── columns: column34:34(float!null) o_orderdate:13(date!null) o_shippriority:16(int!null) l_orderkey:18(int!null) + │ │ ├── columns: column37:37(float!null) o_orderdate:14(date!null) o_shippriority:17(int!null) l_orderkey:20(int!null) │ │ ├── immutable - │ │ ├── stats: [rows=493779.215, distinct(13)=1169, null(13)=0, distinct(16)=1, null(16)=0, distinct(18)=359560.406, null(18)=0, distinct(34)=410295.908, null(34)=0] - │ │ ├── fd: (18)-->(13,16) + │ │ ├── stats: [rows=493779.215, distinct(14)=1169, null(14)=0, distinct(17)=1, null(17)=0, distinct(20)=359560.406, null(20)=0, distinct(37)=410295.908, null(37)=0] + │ │ ├── fd: (20)-->(14,17) │ │ ├── inner-join (lookup lineitem) │ │ │ ├── save-table-name: q3_lookup_join_5 - │ │ │ ├── columns: c_custkey:1(int!null) c_mktsegment:7(char!null) o_orderkey:9(int!null) o_custkey:10(int!null) o_orderdate:13(date!null) o_shippriority:16(int!null) l_orderkey:18(int!null) l_extendedprice:23(float!null) l_discount:24(float!null) l_shipdate:28(date!null) - │ │ │ ├── key columns: [9] = [18] - │ │ │ ├── stats: [rows=493779.215, distinct(1)=29974.3087, null(1)=0, distinct(7)=1, null(7)=0, distinct(9)=359560.406, null(9)=0, distinct(10)=29974.3087, null(10)=0, distinct(13)=1169, null(13)=0, distinct(16)=1, null(16)=0, distinct(18)=359560.406, null(18)=0, distinct(23)=363943.091, null(23)=0, distinct(24)=11, null(24)=0, distinct(28)=1352, null(28)=0, distinct(23,24)=410295.908, null(23,24)=0] - │ │ │ ├── fd: ()-->(7), (9)-->(10,13,16), (9)==(18), (18)==(9), (1)==(10), (10)==(1) + │ │ │ ├── columns: c_custkey:1(int!null) c_mktsegment:7(char!null) o_orderkey:10(int!null) o_custkey:11(int!null) o_orderdate:14(date!null) o_shippriority:17(int!null) l_orderkey:20(int!null) l_extendedprice:25(float!null) l_discount:26(float!null) l_shipdate:30(date!null) + │ │ │ ├── key columns: [10] = [20] + │ │ │ ├── stats: [rows=493779.215, distinct(1)=29974.3087, null(1)=0, distinct(7)=1, null(7)=0, distinct(10)=359560.406, null(10)=0, distinct(11)=29974.3087, null(11)=0, distinct(14)=1169, null(14)=0, distinct(17)=1, null(17)=0, distinct(20)=359560.406, null(20)=0, distinct(25)=363943.091, null(25)=0, distinct(26)=11, null(26)=0, distinct(30)=1352, null(30)=0, distinct(25,26)=410295.908, null(25,26)=0] + │ │ │ ├── fd: ()-->(7), (10)-->(11,14,17), (10)==(20), (20)==(10), (1)==(11), (11)==(1) │ │ │ ├── inner-join (hash) │ │ │ │ ├── save-table-name: q3_inner_join_6 - │ │ │ │ ├── columns: c_custkey:1(int!null) c_mktsegment:7(char!null) o_orderkey:9(int!null) o_custkey:10(int!null) o_orderdate:13(date!null) o_shippriority:16(int!null) + │ │ │ │ ├── columns: c_custkey:1(int!null) c_mktsegment:7(char!null) o_orderkey:10(int!null) o_custkey:11(int!null) o_orderdate:14(date!null) o_shippriority:17(int!null) │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ │ ├── stats: [rows=220818.994, distinct(1)=29974.3087, null(1)=0, distinct(7)=1, null(7)=0, distinct(9)=190731.989, null(9)=0, distinct(10)=29974.3087, null(10)=0, distinct(13)=1169, null(13)=0, distinct(16)=1, null(16)=0] - │ │ │ │ ├── key: (9) - │ │ │ │ ├── fd: ()-->(7), (9)-->(10,13,16), (1)==(10), (10)==(1) + │ │ │ │ ├── stats: [rows=220818.994, distinct(1)=29974.3087, null(1)=0, distinct(7)=1, null(7)=0, distinct(10)=190731.989, null(10)=0, distinct(11)=29974.3087, null(11)=0, distinct(14)=1169, null(14)=0, distinct(17)=1, null(17)=0] + │ │ │ │ ├── key: (10) + │ │ │ │ ├── fd: ()-->(7), (10)-->(11,14,17), (1)==(11), (11)==(1) │ │ │ │ ├── select │ │ │ │ │ ├── save-table-name: q3_select_7 - │ │ │ │ │ ├── columns: o_orderkey:9(int!null) o_custkey:10(int!null) o_orderdate:13(date!null) o_shippriority:16(int!null) - │ │ │ │ │ ├── stats: [rows=734900, distinct(9)=734900, null(9)=0, distinct(10)=99841.9545, null(10)=0, distinct(13)=1169, null(13)=0, distinct(16)=1, null(16)=0] - │ │ │ │ │ │ histogram(13)= 0 450 6600 900 6300 1350 7200 300 6750 1050 7350 300 7350 150 7200 300 7350 1200 6900 750 6600 900 6750 900 6300 1350 7350 750 7050 450 6900 900 7350 150 7200 450 7050 750 6750 750 6900 750 7350 600 6750 900 6750 750 7050 450 7050 900 6900 1050 6600 900 7350 750 6900 1050 6750 750 6750 900 7050 450 7050 750 7200 450 7050 750 6300 1800 6600 750 7200 1200 7050 600 7050 450 6600 1050 7050 450 6900 450 7200 450 6750 600 6600 900 7050 1200 6600 1200 7050 750 6600 900 6750 600 7200 450 6900 600 6600 1350 6900 1050 6300 1050 6750 1050 6900 450 7050 600 6900 450 7200 1500 7200 300 6450 900 7200 1050 6600 900 7050 600 6900 900 6600 900 6600 750 6900 1050 6450 900 7050 1350 6750 1050 6150 1500 7200 600 6900 600 6900 450 7050 300 6600 750 7200 600 6750 600 7200 300 7200 1350 7050 1200 6600 1050 7050 900 7050 450 6450 1050 6900 450 7050 1050 6750 900 6750 1200 6750 600 7200 1050 6150 1500 2400 800 + │ │ │ │ │ ├── columns: o_orderkey:10(int!null) o_custkey:11(int!null) o_orderdate:14(date!null) o_shippriority:17(int!null) + │ │ │ │ │ ├── stats: [rows=734900, distinct(10)=734900, null(10)=0, distinct(11)=99841.9545, null(11)=0, distinct(14)=1169, null(14)=0, distinct(17)=1, null(17)=0] + │ │ │ │ │ │ histogram(14)= 0 450 6600 900 6300 1350 7200 300 6750 1050 7350 300 7350 150 7200 300 7350 1200 6900 750 6600 900 6750 900 6300 1350 7350 750 7050 450 6900 900 7350 150 7200 450 7050 750 6750 750 6900 750 7350 600 6750 900 6750 750 7050 450 7050 900 6900 1050 6600 900 7350 750 6900 1050 6750 750 6750 900 7050 450 7050 750 7200 450 7050 750 6300 1800 6600 750 7200 1200 7050 600 7050 450 6600 1050 7050 450 6900 450 7200 450 6750 600 6600 900 7050 1200 6600 1200 7050 750 6600 900 6750 600 7200 450 6900 600 6600 1350 6900 1050 6300 1050 6750 1050 6900 450 7050 600 6900 450 7200 1500 7200 300 6450 900 7200 1050 6600 900 7050 600 6900 900 6600 900 6600 750 6900 1050 6450 900 7050 1350 6750 1050 6150 1500 7200 600 6900 600 6900 450 7050 300 6600 750 7200 600 6750 600 7200 300 7200 1350 7050 1200 6600 1050 7050 900 7050 450 6450 1050 6900 450 7050 1050 6750 900 6750 1200 6750 600 7200 1050 6150 1500 2400 800 │ │ │ │ │ │ <--- '1992-01-01' ------ '1992-01-11' ------ '1992-01-23' ------ '1992-02-04' ------ '1992-02-19' ------ '1992-03-03' ------ '1992-03-18' ------ '1992-03-29' ------ '1992-04-11' ------ '1992-04-23' ------ '1992-05-08' ------ '1992-05-20' ------ '1992-05-30' ------ '1992-06-11' ------ '1992-06-24' ------ '1992-07-05' ------ '1992-07-18' ------ '1992-07-30' ------ '1992-08-11' ------ '1992-08-23' ------ '1992-09-07' ------ '1992-09-21' ------ '1992-10-04' ------ '1992-10-18' ------ '1992-10-28' ------ '1992-11-07' ------ '1992-11-18' ------ '1992-11-30' ------ '1992-12-10' ------ '1992-12-25' ------ '1993-01-06' ------ '1993-01-19' ------ '1993-02-01' ------ '1993-02-14' ------ '1993-02-24' ------ '1993-03-08' ------ '1993-03-19' ------ '1993-04-03' ------ '1993-04-14' ------ '1993-04-25' ------ '1993-05-12' ------ '1993-05-25' ------ '1993-06-10' ------ '1993-06-18' ------ '1993-07-01' ------ '1993-07-12' ------ '1993-07-24' ------ '1993-08-05' ------ '1993-08-16' ------ '1993-08-29' ------ '1993-09-08' ------ '1993-09-19' ------ '1993-10-01' ------ '1993-10-13' ------ '1993-10-22' ------ '1993-11-04' ------ '1993-11-16' ------ '1993-11-30' ------ '1993-12-16' ------ '1993-12-29' ------ '1994-01-12' ------ '1994-01-23' ------ '1994-02-04' ------ '1994-02-16' ------ '1994-03-01' ------ '1994-03-14' ------ '1994-03-24' ------ '1994-04-06' ------ '1994-04-20' ------ '1994-05-01' ------ '1994-05-12' ------ '1994-05-23' ------ '1994-06-03' ------ '1994-06-16' ------ '1994-06-25' ------ '1994-07-08' ------ '1994-07-19' ------ '1994-08-01' ------ '1994-08-14' ------ '1994-08-25' ------ '1994-09-06' ------ '1994-09-18' ------ '1994-10-01' ------ '1994-10-17' ------ '1994-10-30' ------ '1994-11-10' ------ '1994-11-22' ------ '1994-12-02' ------ '1994-12-16' ------ '1994-12-30' ------ '1995-01-13' ------ '1995-01-25' ------ '1995-02-04' ------ '1995-02-16' ------ '1995-02-26' ------ '1995-03-10' ------ '1995-03-14' - │ │ │ │ │ ├── key: (9) - │ │ │ │ │ ├── fd: (9)-->(10,13,16) + │ │ │ │ │ ├── key: (10) + │ │ │ │ │ ├── fd: (10)-->(11,14,17) │ │ │ │ │ ├── scan orders │ │ │ │ │ │ ├── save-table-name: q3_scan_8 - │ │ │ │ │ │ ├── columns: o_orderkey:9(int!null) o_custkey:10(int!null) o_orderdate:13(date!null) o_shippriority:16(int!null) - │ │ │ │ │ │ ├── stats: [rows=1500000, distinct(9)=1500000, null(9)=0, distinct(10)=99846, null(10)=0, distinct(13)=2406, null(13)=0, distinct(16)=1, null(16)=0] - │ │ │ │ │ │ │ histogram(9)= 0 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 - │ │ │ │ │ │ │ <--- 998 ------ 26593 ------ 56613 ------ 85827 ------ 115587 ------ 140512 ------ 167076 ------ 198882 ------ 228580 ------ 262243 ------ 292064 ------ 316161 ------ 344194 ------ 368550 ------ 402722 ------ 434529 ------ 468807 ------ 495107 ------ 526338 ------ 563239 ------ 597733 ------ 641894 ------ 665730 ------ 688742 ------ 721767 ------ 750050 ------ 787879 ------ 814565 ------ 837381 ------ 866657 ------ 902855 ------ 927172 ------ 952772 ------ 975840 ------ 1008646 ------ 1032066 ------ 1074656 ------ 1104896 ------ 1138369 ------ 1166689 ------ 1194912 ------ 1225222 ------ 1253284 ------ 1284803 ------ 1318499 ------ 1348609 ------ 1376609 ------ 1404933 ------ 1435745 ------ 1460325 ------ 1491648 ------ 1518852 ------ 1542567 ------ 1574277 ------ 1597958 ------ 1624964 ------ 1647555 ------ 1683651 ------ 1717189 ------ 1750563 ------ 1784775 ------ 1812449 ------ 1838850 ------ 1867751 ------ 1891783 ------ 1918790 ------ 1949827 ------ 1986021 ------ 2020134 ------ 2045829 ------ 2077894 ------ 2118912 ------ 2143264 ------ 2176486 ------ 2204838 ------ 2234146 ------ 2260484 ------ 2288512 ------ 2317121 ------ 2344321 ------ 2362567 ------ 2396133 ------ 2423623 ------ 2454884 ------ 2483879 ------ 2520162 ------ 2545574 ------ 2571874 ------ 2608417 ------ 2637477 ------ 2665607 ------ 2692231 ------ 2724261 ------ 2764451 ------ 2798656 ------ 2824775 ------ 2858690 ------ 2881990 ------ 2911814 ------ 2945056 ------ 2981315 ------ 3010306 ------ 3032320 ------ 3064389 ------ 3084166 ------ 3108832 ------ 3136032 ------ 3168484 ------ 3204039 ------ 3229922 ------ 3256549 ------ 3283169 ------ 3315236 ------ 3346756 ------ 3375879 ------ 3408007 ------ 3441827 ------ 3471204 ------ 3505414 ------ 3527938 ------ 3562561 ------ 3598630 ------ 3627270 ------ 3659266 ------ 3686468 ------ 3713505 ------ 3750817 ------ 3777696 ------ 3808129 ------ 3834533 ------ 3871968 ------ 3904931 ------ 3934215 ------ 3960643 ------ 3990336 ------ 4023203 ------ 4057826 ------ 4090691 ------ 4121216 ------ 4159681 ------ 4190726 ------ 4222624 ------ 4246567 ------ 4283653 ------ 4314339 ------ 4342592 ------ 4373829 ------ 4402372 ------ 4431332 ------ 4464452 ------ 4496327 ------ 4526789 ------ 4551109 ------ 4582401 ------ 4615974 ------ 4648482 ------ 4669601 ------ 4705891 ------ 4732869 ------ 4766145 ------ 4805862 ------ 4840677 ------ 4866787 ------ 4895332 ------ 4928100 ------ 4956323 ------ 4992161 ------ 5027008 ------ 5065797 ------ 5099011 ------ 5130592 ------ 5163463 ------ 5197314 ------ 5224994 ------ 5253892 ------ 5277638 ------ 5307105 ------ 5335750 ------ 5369828 ------ 5402528 ------ 5434183 ------ 5464227 ------ 5491072 ------ 5520679 ------ 5543047 ------ 5576708 ------ 5614951 ------ 5646055 ------ 5674721 ------ 5700295 ------ 5732066 ------ 5761255 ------ 5791233 ------ 5819651 ------ 5852291 ------ 5880258 ------ 5909062 ------ 5943111 ------ 5973926 ------ 5998752 - │ │ │ │ │ │ │ histogram(10)= 0 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7200 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7200 300 7350 150 7350 150 7350 150 7350 150 7200 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7200 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 300 7350 150 7500 150 7500 150 7500 150 7500 150 7350 300 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7350 300 7500 150 7500 150 7500 150 7500 150 7500 150 7350 300 7500 150 + │ │ │ │ │ │ ├── columns: o_orderkey:10(int!null) o_custkey:11(int!null) o_orderdate:14(date!null) o_shippriority:17(int!null) + │ │ │ │ │ │ ├── stats: [rows=1500000, distinct(10)=1500000, null(10)=0, distinct(11)=99846, null(11)=0, distinct(14)=2406, null(14)=0, distinct(17)=1, null(17)=0] + │ │ │ │ │ │ │ histogram(10)= 0 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 + │ │ │ │ │ │ │ <--- 998 ------ 26593 ------ 56613 ------ 85827 ------ 115587 ------ 140512 ------ 167076 ------ 198882 ------ 228580 ------ 262243 ------ 292064 ------ 316161 ------ 344194 ------ 368550 ------ 402722 ------ 434529 ------ 468807 ------ 495107 ------ 526338 ------ 563239 ------ 597733 ------ 641894 ------ 665730 ------ 688742 ------ 721767 ------ 750050 ------ 787879 ------ 814565 ------ 837381 ------ 866657 ------ 902855 ------ 927172 ------ 952772 ------ 975840 ------ 1008646 ------ 1032066 ------ 1074656 ------ 1104896 ------ 1138369 ------ 1166689 ------ 1194912 ------ 1225222 ------ 1253284 ------ 1284803 ------ 1318499 ------ 1348609 ------ 1376609 ------ 1404933 ------ 1435745 ------ 1460325 ------ 1491648 ------ 1518852 ------ 1542567 ------ 1574277 ------ 1597958 ------ 1624964 ------ 1647555 ------ 1683651 ------ 1717189 ------ 1750563 ------ 1784775 ------ 1812449 ------ 1838850 ------ 1867751 ------ 1891783 ------ 1918790 ------ 1949827 ------ 1986021 ------ 2020134 ------ 2045829 ------ 2077894 ------ 2118912 ------ 2143264 ------ 2176486 ------ 2204838 ------ 2234146 ------ 2260484 ------ 2288512 ------ 2317121 ------ 2344321 ------ 2362567 ------ 2396133 ------ 2423623 ------ 2454884 ------ 2483879 ------ 2520162 ------ 2545574 ------ 2571874 ------ 2608417 ------ 2637477 ------ 2665607 ------ 2692231 ------ 2724261 ------ 2764451 ------ 2798656 ------ 2824775 ------ 2858690 ------ 2881990 ------ 2911814 ------ 2945056 ------ 2981315 ------ 3010306 ------ 3032320 ------ 3064389 ------ 3084166 ------ 3108832 ------ 3136032 ------ 3168484 ------ 3204039 ------ 3229922 ------ 3256549 ------ 3283169 ------ 3315236 ------ 3346756 ------ 3375879 ------ 3408007 ------ 3441827 ------ 3471204 ------ 3505414 ------ 3527938 ------ 3562561 ------ 3598630 ------ 3627270 ------ 3659266 ------ 3686468 ------ 3713505 ------ 3750817 ------ 3777696 ------ 3808129 ------ 3834533 ------ 3871968 ------ 3904931 ------ 3934215 ------ 3960643 ------ 3990336 ------ 4023203 ------ 4057826 ------ 4090691 ------ 4121216 ------ 4159681 ------ 4190726 ------ 4222624 ------ 4246567 ------ 4283653 ------ 4314339 ------ 4342592 ------ 4373829 ------ 4402372 ------ 4431332 ------ 4464452 ------ 4496327 ------ 4526789 ------ 4551109 ------ 4582401 ------ 4615974 ------ 4648482 ------ 4669601 ------ 4705891 ------ 4732869 ------ 4766145 ------ 4805862 ------ 4840677 ------ 4866787 ------ 4895332 ------ 4928100 ------ 4956323 ------ 4992161 ------ 5027008 ------ 5065797 ------ 5099011 ------ 5130592 ------ 5163463 ------ 5197314 ------ 5224994 ------ 5253892 ------ 5277638 ------ 5307105 ------ 5335750 ------ 5369828 ------ 5402528 ------ 5434183 ------ 5464227 ------ 5491072 ------ 5520679 ------ 5543047 ------ 5576708 ------ 5614951 ------ 5646055 ------ 5674721 ------ 5700295 ------ 5732066 ------ 5761255 ------ 5791233 ------ 5819651 ------ 5852291 ------ 5880258 ------ 5909062 ------ 5943111 ------ 5973926 ------ 5998752 + │ │ │ │ │ │ │ histogram(11)= 0 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7200 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7200 300 7350 150 7350 150 7350 150 7350 150 7200 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7200 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 300 7350 150 7500 150 7500 150 7500 150 7500 150 7350 300 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7350 300 7500 150 7500 150 7500 150 7500 150 7500 150 7350 300 7500 150 │ │ │ │ │ │ │ <--- 8 ------ 763 ------ 1477 ------ 2353 ------ 3229 ------ 4261 ------ 5191 ------ 5924 ------ 6406 ------ 7126 ------ 7870 ------ 8545 ------ 9301 ------ 9911 ------ 10591 ------ 11236 ------ 11854 ------ 12622 ------ 13426 ------ 14036 ------ 14708 ------ 15391 ------ 16138 ------ 16925 ------ 17681 ------ 18388 ------ 19156 ------ 20015 ------ 20660 ------ 21632 ------ 22472 ------ 23260 ------ 24055 ------ 24649 ------ 25352 ------ 25952 ------ 26774 ------ 27358 ------ 28070 ------ 28586 ------ 29296 ------ 30028 ------ 30629 ------ 31400 ------ 32102 ------ 32884 ------ 33796 ------ 34646 ------ 35479 ------ 36161 ------ 36811 ------ 37612 ------ 38314 ------ 39145 ------ 39844 ------ 40591 ------ 41254 ------ 41902 ------ 42496 ------ 43237 ------ 43948 ------ 44690 ------ 45481 ------ 46448 ------ 47242 ------ 47884 ------ 48676 ------ 49441 ------ 50155 ------ 50857 ------ 51463 ------ 52321 ------ 53014 ------ 53587 ------ 54413 ------ 55019 ------ 55771 ------ 56482 ------ 57314 ------ 58163 ------ 58894 ------ 59638 ------ 60478 ------ 61450 ------ 62417 ------ 63176 ------ 63944 ------ 64577 ------ 65344 ------ 66218 ------ 67021 ------ 67703 ------ 68477 ------ 69271 ------ 70073 ------ 70696 ------ 71512 ------ 72430 ------ 73345 ------ 74149 ------ 74780 ------ 75508 ------ 76273 ------ 76865 ------ 77666 ------ 78427 ------ 79042 ------ 79801 ------ 80557 ------ 81163 ------ 81833 ------ 82534 ------ 83368 ------ 84157 ------ 84938 ------ 85495 ------ 86257 ------ 87143 ------ 87998 ------ 88777 ------ 89566 ------ 90292 ------ 91099 ------ 91757 ------ 92401 ------ 93362 ------ 94252 ------ 94771 ------ 95527 ------ 96233 ------ 96952 ------ 97844 ------ 98411 ------ 99067 ------ 99691 ------ 100357 ------ 101009 ------ 101734 ------ 102569 ------ 103381 ------ 104114 ------ 105070 ------ 105884 ------ 106651 ------ 107393 ------ 107995 ------ 108772 ------ 109628 ------ 110383 ------ 111040 ------ 111658 ------ 112556 ------ 113338 ------ 114062 ------ 114895 ------ 115744 ------ 116554 ------ 117250 ------ 117967 ------ 118741 ------ 119540 ------ 120514 ------ 121333 ------ 122111 ------ 122989 ------ 123763 ------ 124735 ------ 125401 ------ 126022 ------ 126779 ------ 127618 ------ 128332 ------ 129167 ------ 129917 ------ 130669 ------ 131330 ------ 131911 ------ 132704 ------ 133553 ------ 134317 ------ 134960 ------ 135688 ------ 136519 ------ 137353 ------ 138061 ------ 138697 ------ 139468 ------ 140218 ------ 140942 ------ 141644 ------ 142415 ------ 143263 ------ 144352 ------ 145099 ------ 145811 ------ 146761 ------ 147643 ------ 148591 ------ 149164 ------ 149995 - │ │ │ │ │ │ │ histogram(13)= 0 450 6600 900 6300 1350 7200 300 6750 1050 7350 300 7350 150 7200 300 7350 1200 6900 750 6600 900 6750 900 6300 1350 7350 750 7050 450 6900 900 7350 150 7200 450 7050 750 6750 750 6900 750 7350 600 6750 900 6750 750 7050 450 7050 900 6900 1050 6600 900 7350 750 6900 1050 6750 750 6750 900 7050 450 7050 750 7200 450 7050 750 6300 1800 6600 750 7200 1200 7050 600 7050 450 6600 1050 7050 450 6900 450 7200 450 6750 600 6600 900 7050 1200 6600 1200 7050 750 6600 900 6750 600 7200 450 6900 600 6600 1350 6900 1050 6300 1050 6750 1050 6900 450 7050 600 6900 450 7200 1500 7200 300 6450 900 7200 1050 6600 900 7050 600 6900 900 6600 900 6600 750 6900 1050 6450 900 7050 1350 6750 1050 6150 1500 7200 600 6900 600 6900 450 7050 300 6600 750 7200 600 6750 600 7200 300 7200 1350 7050 1200 6600 1050 7050 900 7050 450 6450 1050 6900 450 7050 1050 6750 900 6750 1200 6750 600 7200 1050 6150 1500 7200 300 7050 300 6750 900 6900 450 7200 1050 7200 450 7050 600 6750 750 6150 1200 7200 1200 6750 750 7050 450 7200 600 6450 1050 6600 600 7050 750 7050 150 6600 1200 6150 1050 6450 1050 6150 1050 6750 450 7050 300 5700 1500 7050 600 6300 1050 6900 750 6600 750 7050 1050 6600 750 6750 1050 7050 450 6900 900 7050 600 6600 750 6750 900 6900 750 6750 600 6300 1050 6750 600 6900 750 7050 900 6750 600 7050 450 6900 750 6600 600 7050 1050 6900 900 6900 750 6450 900 6750 600 6750 450 7050 450 7050 150 6600 1050 6900 600 6750 600 6300 900 6450 750 6600 600 6900 450 7050 750 6450 750 6300 900 6600 750 6450 1650 6450 750 6450 750 6900 450 5850 1350 6450 600 6300 1350 6750 750 6900 600 6900 1350 6000 1650 6600 750 6600 900 6300 1050 6300 750 6600 900 6000 900 6000 900 6600 450 6750 450 6450 750 6750 300 6450 450 6300 900 6450 450 6300 600 6300 750 6300 900 6750 1050 5850 1200 6150 1050 6300 750 6300 600 6600 750 6000 900 5550 1200 6300 450 5700 600 6000 450 + │ │ │ │ │ │ │ histogram(14)= 0 450 6600 900 6300 1350 7200 300 6750 1050 7350 300 7350 150 7200 300 7350 1200 6900 750 6600 900 6750 900 6300 1350 7350 750 7050 450 6900 900 7350 150 7200 450 7050 750 6750 750 6900 750 7350 600 6750 900 6750 750 7050 450 7050 900 6900 1050 6600 900 7350 750 6900 1050 6750 750 6750 900 7050 450 7050 750 7200 450 7050 750 6300 1800 6600 750 7200 1200 7050 600 7050 450 6600 1050 7050 450 6900 450 7200 450 6750 600 6600 900 7050 1200 6600 1200 7050 750 6600 900 6750 600 7200 450 6900 600 6600 1350 6900 1050 6300 1050 6750 1050 6900 450 7050 600 6900 450 7200 1500 7200 300 6450 900 7200 1050 6600 900 7050 600 6900 900 6600 900 6600 750 6900 1050 6450 900 7050 1350 6750 1050 6150 1500 7200 600 6900 600 6900 450 7050 300 6600 750 7200 600 6750 600 7200 300 7200 1350 7050 1200 6600 1050 7050 900 7050 450 6450 1050 6900 450 7050 1050 6750 900 6750 1200 6750 600 7200 1050 6150 1500 7200 300 7050 300 6750 900 6900 450 7200 1050 7200 450 7050 600 6750 750 6150 1200 7200 1200 6750 750 7050 450 7200 600 6450 1050 6600 600 7050 750 7050 150 6600 1200 6150 1050 6450 1050 6150 1050 6750 450 7050 300 5700 1500 7050 600 6300 1050 6900 750 6600 750 7050 1050 6600 750 6750 1050 7050 450 6900 900 7050 600 6600 750 6750 900 6900 750 6750 600 6300 1050 6750 600 6900 750 7050 900 6750 600 7050 450 6900 750 6600 600 7050 1050 6900 900 6900 750 6450 900 6750 600 6750 450 7050 450 7050 150 6600 1050 6900 600 6750 600 6300 900 6450 750 6600 600 6900 450 7050 750 6450 750 6300 900 6600 750 6450 1650 6450 750 6450 750 6900 450 5850 1350 6450 600 6300 1350 6750 750 6900 600 6900 1350 6000 1650 6600 750 6600 900 6300 1050 6300 750 6600 900 6000 900 6000 900 6600 450 6750 450 6450 750 6750 300 6450 450 6300 900 6450 450 6300 600 6300 750 6300 900 6750 1050 5850 1200 6150 1050 6300 750 6300 600 6600 750 6000 900 5550 1200 6300 450 5700 600 6000 450 │ │ │ │ │ │ │ <--- '1992-01-01' ------ '1992-01-11' ------ '1992-01-23' ------ '1992-02-04' ------ '1992-02-19' ------ '1992-03-03' ------ '1992-03-18' ------ '1992-03-29' ------ '1992-04-11' ------ '1992-04-23' ------ '1992-05-08' ------ '1992-05-20' ------ '1992-05-30' ------ '1992-06-11' ------ '1992-06-24' ------ '1992-07-05' ------ '1992-07-18' ------ '1992-07-30' ------ '1992-08-11' ------ '1992-08-23' ------ '1992-09-07' ------ '1992-09-21' ------ '1992-10-04' ------ '1992-10-18' ------ '1992-10-28' ------ '1992-11-07' ------ '1992-11-18' ------ '1992-11-30' ------ '1992-12-10' ------ '1992-12-25' ------ '1993-01-06' ------ '1993-01-19' ------ '1993-02-01' ------ '1993-02-14' ------ '1993-02-24' ------ '1993-03-08' ------ '1993-03-19' ------ '1993-04-03' ------ '1993-04-14' ------ '1993-04-25' ------ '1993-05-12' ------ '1993-05-25' ------ '1993-06-10' ------ '1993-06-18' ------ '1993-07-01' ------ '1993-07-12' ------ '1993-07-24' ------ '1993-08-05' ------ '1993-08-16' ------ '1993-08-29' ------ '1993-09-08' ------ '1993-09-19' ------ '1993-10-01' ------ '1993-10-13' ------ '1993-10-22' ------ '1993-11-04' ------ '1993-11-16' ------ '1993-11-30' ------ '1993-12-16' ------ '1993-12-29' ------ '1994-01-12' ------ '1994-01-23' ------ '1994-02-04' ------ '1994-02-16' ------ '1994-03-01' ------ '1994-03-14' ------ '1994-03-24' ------ '1994-04-06' ------ '1994-04-20' ------ '1994-05-01' ------ '1994-05-12' ------ '1994-05-23' ------ '1994-06-03' ------ '1994-06-16' ------ '1994-06-25' ------ '1994-07-08' ------ '1994-07-19' ------ '1994-08-01' ------ '1994-08-14' ------ '1994-08-25' ------ '1994-09-06' ------ '1994-09-18' ------ '1994-10-01' ------ '1994-10-17' ------ '1994-10-30' ------ '1994-11-10' ------ '1994-11-22' ------ '1994-12-02' ------ '1994-12-16' ------ '1994-12-30' ------ '1995-01-13' ------ '1995-01-25' ------ '1995-02-04' ------ '1995-02-16' ------ '1995-02-26' ------ '1995-03-10' ------ '1995-03-20' ------ '1995-04-02' ------ '1995-04-15' ------ '1995-04-28' ------ '1995-05-10' ------ '1995-05-25' ------ '1995-06-09' ------ '1995-06-20' ------ '1995-07-01' ------ '1995-07-13' ------ '1995-07-21' ------ '1995-08-01' ------ '1995-08-13' ------ '1995-08-25' ------ '1995-09-06' ------ '1995-09-18' ------ '1995-10-01' ------ '1995-10-11' ------ '1995-10-23' ------ '1995-11-01' ------ '1995-11-13' ------ '1995-11-26' ------ '1995-12-08' ------ '1995-12-22' ------ '1996-01-06' ------ '1996-01-18' ------ '1996-01-30' ------ '1996-02-13' ------ '1996-02-25' ------ '1996-03-09' ------ '1996-03-21' ------ '1996-04-04' ------ '1996-04-14' ------ '1996-04-27' ------ '1996-05-10' ------ '1996-05-22' ------ '1996-06-02' ------ '1996-06-12' ------ '1996-06-23' ------ '1996-07-07' ------ '1996-07-19' ------ '1996-08-01' ------ '1996-08-14' ------ '1996-08-25' ------ '1996-09-06' ------ '1996-09-20' ------ '1996-10-04' ------ '1996-10-15' ------ '1996-10-28' ------ '1996-11-11' ------ '1996-11-22' ------ '1996-12-04' ------ '1996-12-18' ------ '1996-12-29' ------ '1997-01-10' ------ '1997-01-22' ------ '1997-02-01' ------ '1997-02-13' ------ '1997-02-24' ------ '1997-03-09' ------ '1997-03-24' ------ '1997-04-04' ------ '1997-04-16' ------ '1997-04-26' ------ '1997-05-07' ------ '1997-05-19' ------ '1997-05-29' ------ '1997-06-08' ------ '1997-06-22' ------ '1997-07-05' ------ '1997-07-16' ------ '1997-07-28' ------ '1997-08-12' ------ '1997-08-23' ------ '1997-09-04' ------ '1997-09-17' ------ '1997-09-30' ------ '1997-10-09' ------ '1997-10-19' ------ '1997-10-31' ------ '1997-11-14' ------ '1997-11-24' ------ '1997-12-09' ------ '1997-12-24' ------ '1998-01-04' ------ '1998-01-16' ------ '1998-01-28' ------ '1998-02-07' ------ '1998-02-17' ------ '1998-03-01' ------ '1998-03-12' ------ '1998-03-24' ------ '1998-04-03' ------ '1998-04-14' ------ '1998-04-24' ------ '1998-05-03' ------ '1998-05-14' ------ '1998-05-26' ------ '1998-06-08' ------ '1998-06-20' ------ '1998-06-30' ------ '1998-07-11' ------ '1998-07-22' ------ '1998-08-02' - │ │ │ │ │ │ ├── key: (9) - │ │ │ │ │ │ └── fd: (9)-->(10,13,16) + │ │ │ │ │ │ ├── key: (10) + │ │ │ │ │ │ └── fd: (10)-->(11,14,17) │ │ │ │ │ └── filters - │ │ │ │ │ └── o_orderdate:13 < '1995-03-15' [type=bool, outer=(13), constraints=(/13: (/NULL - /'1995-03-14']; tight)] + │ │ │ │ │ └── o_orderdate:14 < '1995-03-15' [type=bool, outer=(14), constraints=(/14: (/NULL - /'1995-03-14']; tight)] │ │ │ │ ├── select │ │ │ │ │ ├── save-table-name: q3_select_9 │ │ │ │ │ ├── columns: c_custkey:1(int!null) c_mktsegment:7(char!null) @@ -125,18 +125,18 @@ limit │ │ │ │ │ └── filters │ │ │ │ │ └── c_mktsegment:7 = 'BUILDING' [type=bool, outer=(7), constraints=(/7: [/'BUILDING' - /'BUILDING']; tight), fd=()-->(7)] │ │ │ │ └── filters - │ │ │ │ └── c_custkey:1 = o_custkey:10 [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] + │ │ │ │ └── c_custkey:1 = o_custkey:11 [type=bool, outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ]), fd=(1)==(11), (11)==(1)] │ │ │ └── filters - │ │ │ └── l_shipdate:28 > '1995-03-15' [type=bool, outer=(28), constraints=(/28: [/'1995-03-16' - ]; tight)] + │ │ │ └── l_shipdate:30 > '1995-03-15' [type=bool, outer=(30), constraints=(/30: [/'1995-03-16' - ]; tight)] │ │ └── projections - │ │ └── l_extendedprice:23 * (1.0 - l_discount:24) [as=column34:34, type=float, outer=(23,24), immutable] + │ │ └── l_extendedprice:25 * (1.0 - l_discount:26) [as=column37:37, type=float, outer=(25,26), immutable] │ └── aggregations - │ ├── sum [as=sum:35, type=float, outer=(34)] - │ │ └── column34:34 [type=float] - │ ├── const-agg [as=o_orderdate:13, type=date, outer=(13)] - │ │ └── o_orderdate:13 [type=date] - │ └── const-agg [as=o_shippriority:16, type=int, outer=(16)] - │ └── o_shippriority:16 [type=int] + │ ├── sum [as=sum:38, type=float, outer=(37)] + │ │ └── column37:37 [type=float] + │ ├── const-agg [as=o_orderdate:14, type=date, outer=(14)] + │ │ └── o_orderdate:14 [type=date] + │ └── const-agg [as=o_shippriority:17, type=int, outer=(17)] + │ └── o_shippriority:17 [type=int] └── 10 [type=int] stats table=q3_limit_1 @@ -184,13 +184,13 @@ column_names row_count_est row_count_err distinct_count_est distinct_cou stats table=q3_project_4 ---- column_names row_count distinct_count null_count -{column34} 30519 30424 0 +{column37} 30519 30424 0 {l_orderkey} 30519 11611 0 {o_orderdate} 30519 120 0 {o_shippriority} 30519 1 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{column34} 493779.00 16.18 <== 410296.00 13.49 <== 0.00 1.00 +{column37} 493779.00 16.18 <== 410296.00 13.49 <== 0.00 1.00 {l_orderkey} 493779.00 16.18 <== 359560.00 30.97 <== 0.00 1.00 {o_orderdate} 493779.00 16.18 <== 1169.00 9.74 <== 0.00 1.00 {o_shippriority} 493779.00 16.18 <== 1.00 1.00 0.00 1.00 diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q04 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q04 index e618be63738f..1b7949fac8a2 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q04 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q04 @@ -40,22 +40,22 @@ ORDER BY ---- sort ├── save-table-name: q4_sort_1 - ├── columns: o_orderpriority:6(char!null) order_count:26(int!null) - ├── stats: [rows=5, distinct(6)=5, null(6)=0, distinct(26)=5, null(26)=0] + ├── columns: o_orderpriority:6(char!null) order_count:28(int!null) + ├── stats: [rows=5, distinct(6)=5, null(6)=0, distinct(28)=5, null(28)=0] ├── key: (6) - ├── fd: (6)-->(26) + ├── fd: (6)-->(28) ├── ordering: +6 └── group-by ├── save-table-name: q4_group_by_2 - ├── columns: o_orderpriority:6(char!null) count_rows:26(int!null) + ├── columns: o_orderpriority:6(char!null) count_rows:28(int!null) ├── grouping columns: o_orderpriority:6(char!null) - ├── stats: [rows=5, distinct(6)=5, null(6)=0, distinct(26)=5, null(26)=0] + ├── stats: [rows=5, distinct(6)=5, null(6)=0, distinct(28)=5, null(28)=0] ├── key: (6) - ├── fd: (6)-->(26) + ├── fd: (6)-->(28) ├── semi-join (lookup lineitem) │ ├── save-table-name: q4_lookup_join_3 │ ├── columns: o_orderkey:1(int!null) o_orderdate:5(date!null) o_orderpriority:6(char!null) - │ ├── key columns: [1] = [10] + │ ├── key columns: [1] = [11] │ ├── stats: [rows=61200, distinct(1)=61200, null(1)=0, distinct(5)=92, null(5)=0, distinct(6)=5, null(6)=0] │ ├── key: (1) │ ├── fd: (1)-->(5,6) @@ -79,9 +79,9 @@ sort │ │ ├── key: (1) │ │ └── fd: (1)-->(5) │ └── filters - │ └── l_commitdate:21 < l_receiptdate:22 [type=bool, outer=(21,22), constraints=(/21: (/NULL - ]; /22: (/NULL - ])] + │ └── l_commitdate:22 < l_receiptdate:23 [type=bool, outer=(22,23), constraints=(/22: (/NULL - ]; /23: (/NULL - ])] └── aggregations - └── count-rows [as=count_rows:26, type=int] + └── count-rows [as=count_rows:28, type=int] stats table=q4_sort_1 ---- diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q05 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q05 index b41d3d081e74..a237568a4034 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q05 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q05 @@ -46,127 +46,127 @@ ORDER BY ---- sort ├── save-table-name: q5_sort_1 - ├── columns: n_name:42(char!null) revenue:49(float!null) + ├── columns: n_name:46(char!null) revenue:55(float!null) ├── immutable - ├── stats: [rows=5, distinct(42)=5, null(42)=0, distinct(49)=5, null(49)=0] - ├── key: (42) - ├── fd: (42)-->(49) - ├── ordering: -49 + ├── stats: [rows=5, distinct(46)=5, null(46)=0, distinct(55)=5, null(55)=0] + ├── key: (46) + ├── fd: (46)-->(55) + ├── ordering: -55 └── group-by ├── save-table-name: q5_group_by_2 - ├── columns: n_name:42(char!null) sum:49(float!null) - ├── grouping columns: n_name:42(char!null) + ├── columns: n_name:46(char!null) sum:55(float!null) + ├── grouping columns: n_name:46(char!null) ├── immutable - ├── stats: [rows=5, distinct(42)=5, null(42)=0, distinct(49)=5, null(49)=0] - ├── key: (42) - ├── fd: (42)-->(49) + ├── stats: [rows=5, distinct(46)=5, null(46)=0, distinct(55)=5, null(55)=0] + ├── key: (46) + ├── fd: (46)-->(55) ├── project │ ├── save-table-name: q5_project_3 - │ ├── columns: column48:48(float!null) n_name:42(char!null) + │ ├── columns: column54:54(float!null) n_name:46(char!null) │ ├── immutable - │ ├── stats: [rows=13445.4933, distinct(42)=5, null(42)=0, distinct(48)=13135.9517, null(48)=0] + │ ├── stats: [rows=13445.4933, distinct(46)=5, null(46)=0, distinct(54)=13135.9517, null(54)=0] │ ├── inner-join (hash) │ │ ├── save-table-name: q5_inner_join_4 - │ │ ├── columns: c_custkey:1(int!null) c_nationkey:4(int!null) o_orderkey:9(int!null) o_custkey:10(int!null) o_orderdate:13(date!null) l_orderkey:18(int!null) l_suppkey:20(int!null) l_extendedprice:23(float!null) l_discount:24(float!null) s_suppkey:34(int!null) s_nationkey:37(int!null) n_nationkey:41(int!null) n_name:42(char!null) n_regionkey:43(int!null) r_regionkey:45(int!null) r_name:46(char!null) + │ │ ├── columns: c_custkey:1(int!null) c_nationkey:4(int!null) o_orderkey:10(int!null) o_custkey:11(int!null) o_orderdate:14(date!null) l_orderkey:20(int!null) l_suppkey:22(int!null) l_extendedprice:25(float!null) l_discount:26(float!null) s_suppkey:37(int!null) s_nationkey:40(int!null) n_nationkey:45(int!null) n_name:46(char!null) n_regionkey:47(int!null) r_regionkey:50(int!null) r_name:51(char!null) │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ ├── stats: [rows=13445.4933, distinct(1)=13445.4933, null(1)=0, distinct(4)=5, null(4)=0, distinct(9)=13060.5826, null(9)=0, distinct(10)=13445.4933, null(10)=0, distinct(13)=365, null(13)=0, distinct(18)=13060.5826, null(18)=0, distinct(20)=1843.54504, null(20)=0, distinct(23)=13107.8032, null(23)=0, distinct(24)=11, null(24)=0, distinct(34)=1843.54504, null(34)=0, distinct(37)=5, null(37)=0, distinct(41)=5, null(41)=0, distinct(42)=5, null(42)=0, distinct(43)=1, null(43)=0, distinct(45)=1, null(45)=0, distinct(46)=0.996222107, null(46)=0, distinct(23,24)=13135.9517, null(23,24)=0] - │ │ ├── fd: ()-->(46), (1)-->(4), (9)-->(10,13), (34)-->(37), (41)-->(42,43), (43)==(45), (45)==(43), (37)==(4,41), (41)==(4,37), (20)==(34), (34)==(20), (9)==(18), (18)==(9), (1)==(10), (10)==(1), (4)==(37,41) + │ │ ├── stats: [rows=13445.4933, distinct(1)=13445.4933, null(1)=0, distinct(4)=5, null(4)=0, distinct(10)=13060.5826, null(10)=0, distinct(11)=13445.4933, null(11)=0, distinct(14)=365, null(14)=0, distinct(20)=13060.5826, null(20)=0, distinct(22)=1843.54504, null(22)=0, distinct(25)=13107.8032, null(25)=0, distinct(26)=11, null(26)=0, distinct(37)=1843.54504, null(37)=0, distinct(40)=5, null(40)=0, distinct(45)=5, null(45)=0, distinct(46)=5, null(46)=0, distinct(47)=1, null(47)=0, distinct(50)=1, null(50)=0, distinct(51)=0.996222107, null(51)=0, distinct(25,26)=13135.9517, null(25,26)=0] + │ │ ├── fd: ()-->(51), (1)-->(4), (10)-->(11,14), (37)-->(40), (45)-->(46,47), (47)==(50), (50)==(47), (40)==(4,45), (45)==(4,40), (22)==(37), (37)==(22), (10)==(20), (20)==(10), (1)==(11), (11)==(1), (4)==(40,45) │ │ ├── inner-join (lookup lineitem) │ │ │ ├── save-table-name: q5_lookup_join_5 - │ │ │ ├── columns: c_custkey:1(int!null) c_nationkey:4(int!null) o_orderkey:9(int!null) o_custkey:10(int!null) o_orderdate:13(date!null) l_orderkey:18(int!null) l_suppkey:20(int!null) l_extendedprice:23(float!null) l_discount:24(float!null) n_nationkey:41(int!null) n_name:42(char!null) n_regionkey:43(int!null) r_regionkey:45(int!null) r_name:46(char!null) - │ │ │ ├── key columns: [9] = [18] - │ │ │ ├── stats: [rows=296134.9, distinct(1)=27671.7064, null(1)=0, distinct(4)=5, null(4)=0, distinct(9)=64276.6285, null(9)=0, distinct(10)=27671.7064, null(10)=0, distinct(13)=365, null(13)=0, distinct(18)=64276.6285, null(18)=0, distinct(20)=9920, null(20)=0, distinct(23)=253449.774, null(23)=0, distinct(24)=11, null(24)=0, distinct(41)=5, null(41)=0, distinct(42)=5, null(42)=0, distinct(43)=1, null(43)=0, distinct(45)=1, null(45)=0, distinct(46)=0.996222107, null(46)=0] - │ │ │ ├── fd: ()-->(46), (9)-->(10,13), (1)-->(4), (41)-->(42,43), (43)==(45), (45)==(43), (4)==(41), (41)==(4), (1)==(10), (10)==(1), (9)==(18), (18)==(9) + │ │ │ ├── columns: c_custkey:1(int!null) c_nationkey:4(int!null) o_orderkey:10(int!null) o_custkey:11(int!null) o_orderdate:14(date!null) l_orderkey:20(int!null) l_suppkey:22(int!null) l_extendedprice:25(float!null) l_discount:26(float!null) n_nationkey:45(int!null) n_name:46(char!null) n_regionkey:47(int!null) r_regionkey:50(int!null) r_name:51(char!null) + │ │ │ ├── key columns: [10] = [20] + │ │ │ ├── stats: [rows=296134.9, distinct(1)=27671.7064, null(1)=0, distinct(4)=5, null(4)=0, distinct(10)=64276.6285, null(10)=0, distinct(11)=27671.7064, null(11)=0, distinct(14)=365, null(14)=0, distinct(20)=64276.6285, null(20)=0, distinct(22)=9920, null(22)=0, distinct(25)=253449.774, null(25)=0, distinct(26)=11, null(26)=0, distinct(45)=5, null(45)=0, distinct(46)=5, null(46)=0, distinct(47)=1, null(47)=0, distinct(50)=1, null(50)=0, distinct(51)=0.996222107, null(51)=0] + │ │ │ ├── fd: ()-->(51), (10)-->(11,14), (1)-->(4), (45)-->(46,47), (47)==(50), (50)==(47), (4)==(45), (45)==(4), (1)==(11), (11)==(1), (10)==(20), (20)==(10) │ │ │ ├── inner-join (hash) │ │ │ │ ├── save-table-name: q5_inner_join_6 - │ │ │ │ ├── columns: c_custkey:1(int!null) c_nationkey:4(int!null) o_orderkey:9(int!null) o_custkey:10(int!null) o_orderdate:13(date!null) n_nationkey:41(int!null) n_name:42(char!null) n_regionkey:43(int!null) r_regionkey:45(int!null) r_name:46(char!null) + │ │ │ │ ├── columns: c_custkey:1(int!null) c_nationkey:4(int!null) o_orderkey:10(int!null) o_custkey:11(int!null) o_orderdate:14(date!null) n_nationkey:45(int!null) n_name:46(char!null) n_regionkey:47(int!null) r_regionkey:50(int!null) r_name:51(char!null) │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ │ ├── stats: [rows=75364.3969, distinct(1)=27672.3293, null(1)=0, distinct(4)=5, null(4)=0, distinct(9)=64276.6285, null(9)=0, distinct(10)=27672.3293, null(10)=0, distinct(13)=365, null(13)=0, distinct(41)=5, null(41)=0, distinct(42)=5, null(42)=0, distinct(43)=1, null(43)=0, distinct(45)=1, null(45)=0, distinct(46)=0.996222107, null(46)=0] - │ │ │ │ ├── key: (9) - │ │ │ │ ├── fd: ()-->(46), (9)-->(10,13), (1)-->(4), (41)-->(42,43), (43)==(45), (45)==(43), (4)==(41), (41)==(4), (1)==(10), (10)==(1) + │ │ │ │ ├── stats: [rows=75364.3969, distinct(1)=27672.3293, null(1)=0, distinct(4)=5, null(4)=0, distinct(10)=64276.6285, null(10)=0, distinct(11)=27672.3293, null(11)=0, distinct(14)=365, null(14)=0, distinct(45)=5, null(45)=0, distinct(46)=5, null(46)=0, distinct(47)=1, null(47)=0, distinct(50)=1, null(50)=0, distinct(51)=0.996222107, null(51)=0] + │ │ │ │ ├── key: (10) + │ │ │ │ ├── fd: ()-->(51), (10)-->(11,14), (1)-->(4), (45)-->(46,47), (47)==(50), (50)==(47), (4)==(45), (45)==(4), (1)==(11), (11)==(1) │ │ │ │ ├── index-join orders │ │ │ │ │ ├── save-table-name: q5_index_join_7 - │ │ │ │ │ ├── columns: o_orderkey:9(int!null) o_custkey:10(int!null) o_orderdate:13(date!null) - │ │ │ │ │ ├── stats: [rows=230330.769, distinct(9)=230330.769, null(9)=0, distinct(10)=91686.8358, null(10)=0, distinct(13)=365, null(13)=0] - │ │ │ │ │ │ histogram(13)= 0 0 5838.5 450 7200 1500 7200 300 6450 900 7200 1050 6600 900 7050 600 6900 900 6600 900 6600 750 6900 1050 6450 900 7050 1350 6750 1050 6150 1500 7200 600 6900 600 6900 450 7050 300 6600 750 7200 600 6750 600 7200 300 7200 1350 7050 1200 6600 1050 7050 900 7050 450 6450 1050 6900 450 0 542.31 + │ │ │ │ │ ├── columns: o_orderkey:10(int!null) o_custkey:11(int!null) o_orderdate:14(date!null) + │ │ │ │ │ ├── stats: [rows=230330.769, distinct(10)=230330.769, null(10)=0, distinct(11)=91686.8358, null(11)=0, distinct(14)=365, null(14)=0] + │ │ │ │ │ │ histogram(14)= 0 0 5838.5 450 7200 1500 7200 300 6450 900 7200 1050 6600 900 7050 600 6900 900 6600 900 6600 750 6900 1050 6450 900 7050 1350 6750 1050 6150 1500 7200 600 6900 600 6900 450 7050 300 6600 750 7200 600 6750 600 7200 300 7200 1350 7050 1200 6600 1050 7050 900 7050 450 6450 1050 6900 450 0 542.31 │ │ │ │ │ │ <--- '1993-12-31' -------- '1994-01-12' ------ '1994-01-23' ------ '1994-02-04' ------ '1994-02-16' ------ '1994-03-01' ------ '1994-03-14' ------ '1994-03-24' ------ '1994-04-06' ------ '1994-04-20' ------ '1994-05-01' ------ '1994-05-12' ------ '1994-05-23' ------ '1994-06-03' ------ '1994-06-16' ------ '1994-06-25' ------ '1994-07-08' ------ '1994-07-19' ------ '1994-08-01' ------ '1994-08-14' ------ '1994-08-25' ------ '1994-09-06' ------ '1994-09-18' ------ '1994-10-01' ------ '1994-10-17' ------ '1994-10-30' ------ '1994-11-10' ------ '1994-11-22' ------ '1994-12-02' ------ '1994-12-16' ------ '1994-12-30' --- '1994-12-31' - │ │ │ │ │ ├── key: (9) - │ │ │ │ │ ├── fd: (9)-->(10,13) + │ │ │ │ │ ├── key: (10) + │ │ │ │ │ ├── fd: (10)-->(11,14) │ │ │ │ │ └── scan orders@o_od │ │ │ │ │ ├── save-table-name: q5_scan_8 - │ │ │ │ │ ├── columns: o_orderkey:9(int!null) o_orderdate:13(date!null) - │ │ │ │ │ ├── constraint: /13/9: [/'1994-01-01' - /'1994-12-31'] - │ │ │ │ │ ├── stats: [rows=230330.769, distinct(9)=230330.769, null(9)=0, distinct(13)=365, null(13)=0] - │ │ │ │ │ │ histogram(9)= 0 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 - │ │ │ │ │ │ <--- 998 --------- 26593 -------- 56613 -------- 85827 -------- 115587 -------- 140512 -------- 167076 -------- 198882 -------- 228580 -------- 262243 -------- 292064 -------- 316161 -------- 344194 -------- 368550 -------- 402722 -------- 434529 -------- 468807 -------- 495107 -------- 526338 -------- 563239 -------- 597733 -------- 641894 -------- 665730 -------- 688742 -------- 721767 -------- 750050 -------- 787879 -------- 814565 -------- 837381 -------- 866657 -------- 902855 -------- 927172 -------- 952772 -------- 975840 -------- 1008646 -------- 1032066 -------- 1074656 -------- 1104896 -------- 1138369 -------- 1166689 -------- 1194912 -------- 1225222 -------- 1253284 -------- 1284803 -------- 1318499 -------- 1348609 -------- 1376609 -------- 1404933 -------- 1435745 -------- 1460325 -------- 1491648 -------- 1518852 -------- 1542567 -------- 1574277 -------- 1597958 -------- 1624964 -------- 1647555 -------- 1683651 -------- 1717189 -------- 1750563 -------- 1784775 -------- 1812449 -------- 1838850 -------- 1867751 -------- 1891783 -------- 1918790 -------- 1949827 -------- 1986021 -------- 2020134 -------- 2045829 -------- 2077894 -------- 2118912 -------- 2143264 -------- 2176486 -------- 2204838 -------- 2234146 -------- 2260484 -------- 2288512 -------- 2317121 -------- 2344321 -------- 2362567 -------- 2396133 -------- 2423623 -------- 2454884 -------- 2483879 -------- 2520162 -------- 2545574 -------- 2571874 -------- 2608417 -------- 2637477 -------- 2665607 -------- 2692231 -------- 2724261 -------- 2764451 -------- 2798656 -------- 2824775 -------- 2858690 -------- 2881990 -------- 2911814 -------- 2945056 -------- 2981315 -------- 3010306 -------- 3032320 -------- 3064389 -------- 3084166 -------- 3108832 -------- 3136032 -------- 3168484 -------- 3204039 -------- 3229922 -------- 3256549 -------- 3283169 -------- 3315236 -------- 3346756 -------- 3375879 -------- 3408007 -------- 3441827 -------- 3471204 -------- 3505414 -------- 3527938 -------- 3562561 -------- 3598630 -------- 3627270 -------- 3659266 -------- 3686468 -------- 3713505 -------- 3750817 -------- 3777696 -------- 3808129 -------- 3834533 -------- 3871968 -------- 3904931 -------- 3934215 -------- 3960643 -------- 3990336 -------- 4023203 -------- 4057826 -------- 4090691 -------- 4121216 -------- 4159681 -------- 4190726 -------- 4222624 -------- 4246567 -------- 4283653 -------- 4314339 -------- 4342592 -------- 4373829 -------- 4402372 -------- 4431332 -------- 4464452 -------- 4496327 -------- 4526789 -------- 4551109 -------- 4582401 -------- 4615974 -------- 4648482 -------- 4669601 -------- 4705891 -------- 4732869 -------- 4766145 -------- 4805862 -------- 4840677 -------- 4866787 -------- 4895332 -------- 4928100 -------- 4956323 -------- 4992161 -------- 5027008 -------- 5065797 -------- 5099011 -------- 5130592 -------- 5163463 -------- 5197314 -------- 5224994 -------- 5253892 -------- 5277638 -------- 5307105 -------- 5335750 -------- 5369828 -------- 5402528 -------- 5434183 -------- 5464227 -------- 5491072 -------- 5520679 -------- 5543047 -------- 5576708 -------- 5614951 -------- 5646055 -------- 5674721 -------- 5700295 -------- 5732066 -------- 5761255 -------- 5791233 -------- 5819651 -------- 5852291 -------- 5880258 -------- 5909062 -------- 5943111 -------- 5973926 -------- 5998752 - │ │ │ │ │ │ histogram(13)= 0 0 5838.5 450 7200 1500 7200 300 6450 900 7200 1050 6600 900 7050 600 6900 900 6600 900 6600 750 6900 1050 6450 900 7050 1350 6750 1050 6150 1500 7200 600 6900 600 6900 450 7050 300 6600 750 7200 600 6750 600 7200 300 7200 1350 7050 1200 6600 1050 7050 900 7050 450 6450 1050 6900 450 0 542.31 + │ │ │ │ │ ├── columns: o_orderkey:10(int!null) o_orderdate:14(date!null) + │ │ │ │ │ ├── constraint: /14/10: [/'1994-01-01' - /'1994-12-31'] + │ │ │ │ │ ├── stats: [rows=230330.769, distinct(10)=230330.769, null(10)=0, distinct(14)=365, null(14)=0] + │ │ │ │ │ │ histogram(10)= 0 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1128.6 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 1151.7 23.033 + │ │ │ │ │ │ <--- 998 --------- 26593 -------- 56613 -------- 85827 -------- 115587 -------- 140512 -------- 167076 -------- 198882 -------- 228580 -------- 262243 -------- 292064 -------- 316161 -------- 344194 -------- 368550 -------- 402722 -------- 434529 -------- 468807 -------- 495107 -------- 526338 -------- 563239 -------- 597733 -------- 641894 -------- 665730 -------- 688742 -------- 721767 -------- 750050 -------- 787879 -------- 814565 -------- 837381 -------- 866657 -------- 902855 -------- 927172 -------- 952772 -------- 975840 -------- 1008646 -------- 1032066 -------- 1074656 -------- 1104896 -------- 1138369 -------- 1166689 -------- 1194912 -------- 1225222 -------- 1253284 -------- 1284803 -------- 1318499 -------- 1348609 -------- 1376609 -------- 1404933 -------- 1435745 -------- 1460325 -------- 1491648 -------- 1518852 -------- 1542567 -------- 1574277 -------- 1597958 -------- 1624964 -------- 1647555 -------- 1683651 -------- 1717189 -------- 1750563 -------- 1784775 -------- 1812449 -------- 1838850 -------- 1867751 -------- 1891783 -------- 1918790 -------- 1949827 -------- 1986021 -------- 2020134 -------- 2045829 -------- 2077894 -------- 2118912 -------- 2143264 -------- 2176486 -------- 2204838 -------- 2234146 -------- 2260484 -------- 2288512 -------- 2317121 -------- 2344321 -------- 2362567 -------- 2396133 -------- 2423623 -------- 2454884 -------- 2483879 -------- 2520162 -------- 2545574 -------- 2571874 -------- 2608417 -------- 2637477 -------- 2665607 -------- 2692231 -------- 2724261 -------- 2764451 -------- 2798656 -------- 2824775 -------- 2858690 -------- 2881990 -------- 2911814 -------- 2945056 -------- 2981315 -------- 3010306 -------- 3032320 -------- 3064389 -------- 3084166 -------- 3108832 -------- 3136032 -------- 3168484 -------- 3204039 -------- 3229922 -------- 3256549 -------- 3283169 -------- 3315236 -------- 3346756 -------- 3375879 -------- 3408007 -------- 3441827 -------- 3471204 -------- 3505414 -------- 3527938 -------- 3562561 -------- 3598630 -------- 3627270 -------- 3659266 -------- 3686468 -------- 3713505 -------- 3750817 -------- 3777696 -------- 3808129 -------- 3834533 -------- 3871968 -------- 3904931 -------- 3934215 -------- 3960643 -------- 3990336 -------- 4023203 -------- 4057826 -------- 4090691 -------- 4121216 -------- 4159681 -------- 4190726 -------- 4222624 -------- 4246567 -------- 4283653 -------- 4314339 -------- 4342592 -------- 4373829 -------- 4402372 -------- 4431332 -------- 4464452 -------- 4496327 -------- 4526789 -------- 4551109 -------- 4582401 -------- 4615974 -------- 4648482 -------- 4669601 -------- 4705891 -------- 4732869 -------- 4766145 -------- 4805862 -------- 4840677 -------- 4866787 -------- 4895332 -------- 4928100 -------- 4956323 -------- 4992161 -------- 5027008 -------- 5065797 -------- 5099011 -------- 5130592 -------- 5163463 -------- 5197314 -------- 5224994 -------- 5253892 -------- 5277638 -------- 5307105 -------- 5335750 -------- 5369828 -------- 5402528 -------- 5434183 -------- 5464227 -------- 5491072 -------- 5520679 -------- 5543047 -------- 5576708 -------- 5614951 -------- 5646055 -------- 5674721 -------- 5700295 -------- 5732066 -------- 5761255 -------- 5791233 -------- 5819651 -------- 5852291 -------- 5880258 -------- 5909062 -------- 5943111 -------- 5973926 -------- 5998752 + │ │ │ │ │ │ histogram(14)= 0 0 5838.5 450 7200 1500 7200 300 6450 900 7200 1050 6600 900 7050 600 6900 900 6600 900 6600 750 6900 1050 6450 900 7050 1350 6750 1050 6150 1500 7200 600 6900 600 6900 450 7050 300 6600 750 7200 600 6750 600 7200 300 7200 1350 7050 1200 6600 1050 7050 900 7050 450 6450 1050 6900 450 0 542.31 │ │ │ │ │ │ <--- '1993-12-31' -------- '1994-01-12' ------ '1994-01-23' ------ '1994-02-04' ------ '1994-02-16' ------ '1994-03-01' ------ '1994-03-14' ------ '1994-03-24' ------ '1994-04-06' ------ '1994-04-20' ------ '1994-05-01' ------ '1994-05-12' ------ '1994-05-23' ------ '1994-06-03' ------ '1994-06-16' ------ '1994-06-25' ------ '1994-07-08' ------ '1994-07-19' ------ '1994-08-01' ------ '1994-08-14' ------ '1994-08-25' ------ '1994-09-06' ------ '1994-09-18' ------ '1994-10-01' ------ '1994-10-17' ------ '1994-10-30' ------ '1994-11-10' ------ '1994-11-22' ------ '1994-12-02' ------ '1994-12-16' ------ '1994-12-30' --- '1994-12-31' - │ │ │ │ │ ├── key: (9) - │ │ │ │ │ └── fd: (9)-->(13) + │ │ │ │ │ ├── key: (10) + │ │ │ │ │ └── fd: (10)-->(14) │ │ │ │ ├── inner-join (lookup customer@c_nk) │ │ │ │ │ ├── save-table-name: q5_lookup_join_9 - │ │ │ │ │ ├── columns: c_custkey:1(int!null) c_nationkey:4(int!null) n_nationkey:41(int!null) n_name:42(char!null) n_regionkey:43(int!null) r_regionkey:45(int!null) r_name:46(char!null) - │ │ │ │ │ ├── key columns: [41] = [4] - │ │ │ │ │ ├── stats: [rows=30000, distinct(1)=27672.3293, null(1)=0, distinct(4)=5, null(4)=0, distinct(41)=5, null(41)=0, distinct(42)=5, null(42)=0, distinct(43)=1, null(43)=0, distinct(45)=1, null(45)=0, distinct(46)=0.996222107, null(46)=0] + │ │ │ │ │ ├── columns: c_custkey:1(int!null) c_nationkey:4(int!null) n_nationkey:45(int!null) n_name:46(char!null) n_regionkey:47(int!null) r_regionkey:50(int!null) r_name:51(char!null) + │ │ │ │ │ ├── key columns: [45] = [4] + │ │ │ │ │ ├── stats: [rows=30000, distinct(1)=27672.3293, null(1)=0, distinct(4)=5, null(4)=0, distinct(45)=5, null(45)=0, distinct(46)=5, null(46)=0, distinct(47)=1, null(47)=0, distinct(50)=1, null(50)=0, distinct(51)=0.996222107, null(51)=0] │ │ │ │ │ ├── key: (1) - │ │ │ │ │ ├── fd: ()-->(46), (1)-->(4), (41)-->(42,43), (43)==(45), (45)==(43), (4)==(41), (41)==(4) + │ │ │ │ │ ├── fd: ()-->(51), (1)-->(4), (45)-->(46,47), (47)==(50), (50)==(47), (4)==(45), (45)==(4) │ │ │ │ │ ├── inner-join (hash) │ │ │ │ │ │ ├── save-table-name: q5_inner_join_10 - │ │ │ │ │ │ ├── columns: n_nationkey:41(int!null) n_name:42(char!null) n_regionkey:43(int!null) r_regionkey:45(int!null) r_name:46(char!null) + │ │ │ │ │ │ ├── columns: n_nationkey:45(int!null) n_name:46(char!null) n_regionkey:47(int!null) r_regionkey:50(int!null) r_name:51(char!null) │ │ │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ │ │ │ ├── stats: [rows=5, distinct(41)=5, null(41)=0, distinct(42)=5, null(42)=0, distinct(43)=1, null(43)=0, distinct(45)=1, null(45)=0, distinct(46)=0.996222107, null(46)=0] - │ │ │ │ │ │ ├── key: (41) - │ │ │ │ │ │ ├── fd: ()-->(46), (41)-->(42,43), (43)==(45), (45)==(43) + │ │ │ │ │ │ ├── stats: [rows=5, distinct(45)=5, null(45)=0, distinct(46)=5, null(46)=0, distinct(47)=1, null(47)=0, distinct(50)=1, null(50)=0, distinct(51)=0.996222107, null(51)=0] + │ │ │ │ │ │ ├── key: (45) + │ │ │ │ │ │ ├── fd: ()-->(51), (45)-->(46,47), (47)==(50), (50)==(47) │ │ │ │ │ │ ├── scan nation │ │ │ │ │ │ │ ├── save-table-name: q5_scan_11 - │ │ │ │ │ │ │ ├── columns: n_nationkey:41(int!null) n_name:42(char!null) n_regionkey:43(int!null) - │ │ │ │ │ │ │ ├── stats: [rows=25, distinct(41)=25, null(41)=0, distinct(42)=25, null(42)=0, distinct(43)=5, null(43)=0] - │ │ │ │ │ │ │ │ histogram(41)= 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 + │ │ │ │ │ │ │ ├── columns: n_nationkey:45(int!null) n_name:46(char!null) n_regionkey:47(int!null) + │ │ │ │ │ │ │ ├── stats: [rows=25, distinct(45)=25, null(45)=0, distinct(46)=25, null(46)=0, distinct(47)=5, null(47)=0] + │ │ │ │ │ │ │ │ histogram(45)= 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 │ │ │ │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 --- 5 --- 6 --- 7 --- 8 --- 9 --- 10 --- 11 --- 12 --- 13 --- 14 --- 15 --- 16 --- 17 --- 18 --- 19 --- 20 --- 21 --- 22 --- 23 --- 24 - │ │ │ │ │ │ │ │ histogram(43)= 0 5 0 5 0 5 0 5 0 5 + │ │ │ │ │ │ │ │ histogram(47)= 0 5 0 5 0 5 0 5 0 5 │ │ │ │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 - │ │ │ │ │ │ │ ├── key: (41) - │ │ │ │ │ │ │ └── fd: (41)-->(42,43) + │ │ │ │ │ │ │ ├── key: (45) + │ │ │ │ │ │ │ └── fd: (45)-->(46,47) │ │ │ │ │ │ ├── select │ │ │ │ │ │ │ ├── save-table-name: q5_select_12 - │ │ │ │ │ │ │ ├── columns: r_regionkey:45(int!null) r_name:46(char!null) - │ │ │ │ │ │ │ ├── stats: [rows=1, distinct(45)=1, null(45)=0, distinct(46)=1, null(46)=0] - │ │ │ │ │ │ │ ├── key: (45) - │ │ │ │ │ │ │ ├── fd: ()-->(46) + │ │ │ │ │ │ │ ├── columns: r_regionkey:50(int!null) r_name:51(char!null) + │ │ │ │ │ │ │ ├── stats: [rows=1, distinct(50)=1, null(50)=0, distinct(51)=1, null(51)=0] + │ │ │ │ │ │ │ ├── key: (50) + │ │ │ │ │ │ │ ├── fd: ()-->(51) │ │ │ │ │ │ │ ├── scan region │ │ │ │ │ │ │ │ ├── save-table-name: q5_scan_13 - │ │ │ │ │ │ │ │ ├── columns: r_regionkey:45(int!null) r_name:46(char!null) - │ │ │ │ │ │ │ │ ├── stats: [rows=5, distinct(45)=5, null(45)=0, distinct(46)=5, null(46)=0] - │ │ │ │ │ │ │ │ │ histogram(45)= 0 1 0 1 0 1 0 1 0 1 + │ │ │ │ │ │ │ │ ├── columns: r_regionkey:50(int!null) r_name:51(char!null) + │ │ │ │ │ │ │ │ ├── stats: [rows=5, distinct(50)=5, null(50)=0, distinct(51)=5, null(51)=0] + │ │ │ │ │ │ │ │ │ histogram(50)= 0 1 0 1 0 1 0 1 0 1 │ │ │ │ │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 - │ │ │ │ │ │ │ │ ├── key: (45) - │ │ │ │ │ │ │ │ └── fd: (45)-->(46) + │ │ │ │ │ │ │ │ ├── key: (50) + │ │ │ │ │ │ │ │ └── fd: (50)-->(51) │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── r_name:46 = 'ASIA' [type=bool, outer=(46), constraints=(/46: [/'ASIA' - /'ASIA']; tight), fd=()-->(46)] + │ │ │ │ │ │ │ └── r_name:51 = 'ASIA' [type=bool, outer=(51), constraints=(/51: [/'ASIA' - /'ASIA']; tight), fd=()-->(51)] │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── n_regionkey:43 = r_regionkey:45 [type=bool, outer=(43,45), constraints=(/43: (/NULL - ]; /45: (/NULL - ]), fd=(43)==(45), (45)==(43)] + │ │ │ │ │ │ └── n_regionkey:47 = r_regionkey:50 [type=bool, outer=(47,50), constraints=(/47: (/NULL - ]; /50: (/NULL - ]), fd=(47)==(50), (50)==(47)] │ │ │ │ │ └── filters (true) │ │ │ │ └── filters - │ │ │ │ └── c_custkey:1 = o_custkey:10 [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] + │ │ │ │ └── c_custkey:1 = o_custkey:11 [type=bool, outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ]), fd=(1)==(11), (11)==(1)] │ │ │ └── filters (true) │ │ ├── scan supplier@s_nk │ │ │ ├── save-table-name: q5_scan_14 - │ │ │ ├── columns: s_suppkey:34(int!null) s_nationkey:37(int!null) - │ │ │ ├── stats: [rows=10000, distinct(34)=9920, null(34)=0, distinct(37)=25, null(37)=0] - │ │ │ │ histogram(34)= 0 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 + │ │ │ ├── columns: s_suppkey:37(int!null) s_nationkey:40(int!null) + │ │ │ ├── stats: [rows=10000, distinct(37)=9920, null(37)=0, distinct(40)=25, null(40)=0] + │ │ │ │ histogram(37)= 0 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 │ │ │ │ <--- 1 ---- 51 ---- 101 ---- 151 ---- 201 ---- 251 ---- 301 ---- 351 ---- 401 ---- 451 ---- 501 ---- 551 ---- 601 ---- 651 ---- 701 ---- 751 ---- 801 ---- 851 ---- 901 ---- 951 ---- 1001 ---- 1051 ---- 1101 ---- 1151 ---- 1201 ---- 1251 ---- 1301 ---- 1351 ---- 1401 ---- 1451 ---- 1501 ---- 1551 ---- 1601 ---- 1651 ---- 1701 ---- 1751 ---- 1801 ---- 1851 ---- 1901 ---- 1951 ---- 2001 ---- 2051 ---- 2101 ---- 2151 ---- 2201 ---- 2251 ---- 2301 ---- 2351 ---- 2401 ---- 2451 ---- 2501 ---- 2551 ---- 2601 ---- 2651 ---- 2701 ---- 2751 ---- 2801 ---- 2851 ---- 2901 ---- 2951 ---- 3001 ---- 3051 ---- 3101 ---- 3151 ---- 3201 ---- 3251 ---- 3301 ---- 3351 ---- 3401 ---- 3451 ---- 3501 ---- 3551 ---- 3601 ---- 3651 ---- 3701 ---- 3751 ---- 3801 ---- 3851 ---- 3901 ---- 3951 ---- 4001 ---- 4051 ---- 4101 ---- 4151 ---- 4201 ---- 4251 ---- 4301 ---- 4351 ---- 4401 ---- 4451 ---- 4501 ---- 4551 ---- 4601 ---- 4651 ---- 4701 ---- 4751 ---- 4801 ---- 4851 ---- 4901 ---- 4951 ---- 5001 ---- 5051 ---- 5101 ---- 5151 ---- 5201 ---- 5251 ---- 5301 ---- 5351 ---- 5401 ---- 5451 ---- 5501 ---- 5551 ---- 5601 ---- 5651 ---- 5701 ---- 5751 ---- 5801 ---- 5851 ---- 5901 ---- 5951 ---- 6001 ---- 6051 ---- 6101 ---- 6151 ---- 6201 ---- 6251 ---- 6301 ---- 6351 ---- 6401 ---- 6451 ---- 6501 ---- 6551 ---- 6601 ---- 6651 ---- 6701 ---- 6751 ---- 6801 ---- 6851 ---- 6901 ---- 6951 ---- 7001 ---- 7051 ---- 7101 ---- 7151 ---- 7201 ---- 7251 ---- 7301 ---- 7351 ---- 7401 ---- 7451 ---- 7501 ---- 7552 ---- 7603 ---- 7654 ---- 7705 ---- 7756 ---- 7807 ---- 7858 ---- 7909 ---- 7960 ---- 8011 ---- 8062 ---- 8113 ---- 8164 ---- 8215 ---- 8266 ---- 8317 ---- 8368 ---- 8419 ---- 8470 ---- 8521 ---- 8572 ---- 8623 ---- 8674 ---- 8725 ---- 8776 ---- 8827 ---- 8878 ---- 8929 ---- 8980 ---- 9031 ---- 9082 ---- 9133 ---- 9184 ---- 9235 ---- 9286 ---- 9337 ---- 9388 ---- 9439 ---- 9490 ---- 9541 ---- 9592 ---- 9643 ---- 9694 ---- 9745 ---- 9796 ---- 9847 ---- 9898 ---- 9949 ---- 10000 - │ │ │ │ histogram(37)= 0 420 0 413 0 397 0 412 0 415 0 380 0 402 0 396 0 415 0 405 0 393 0 438 0 377 0 362 0 376 0 373 0 406 0 421 0 407 0 398 0 411 0 399 0 401 0 390 0 393 + │ │ │ │ histogram(40)= 0 420 0 413 0 397 0 412 0 415 0 380 0 402 0 396 0 415 0 405 0 393 0 438 0 377 0 362 0 376 0 373 0 406 0 421 0 407 0 398 0 411 0 399 0 401 0 390 0 393 │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 --- 5 --- 6 --- 7 --- 8 --- 9 --- 10 --- 11 --- 12 --- 13 --- 14 --- 15 --- 16 --- 17 --- 18 --- 19 --- 20 --- 21 --- 22 --- 23 --- 24 - │ │ │ ├── key: (34) - │ │ │ └── fd: (34)-->(37) + │ │ │ ├── key: (37) + │ │ │ └── fd: (37)-->(40) │ │ └── filters - │ │ ├── l_suppkey:20 = s_suppkey:34 [type=bool, outer=(20,34), constraints=(/20: (/NULL - ]; /34: (/NULL - ]), fd=(20)==(34), (34)==(20)] - │ │ └── c_nationkey:4 = s_nationkey:37 [type=bool, outer=(4,37), constraints=(/4: (/NULL - ]; /37: (/NULL - ]), fd=(4)==(37), (37)==(4)] + │ │ ├── l_suppkey:22 = s_suppkey:37 [type=bool, outer=(22,37), constraints=(/22: (/NULL - ]; /37: (/NULL - ]), fd=(22)==(37), (37)==(22)] + │ │ └── c_nationkey:4 = s_nationkey:40 [type=bool, outer=(4,40), constraints=(/4: (/NULL - ]; /40: (/NULL - ]), fd=(4)==(40), (40)==(4)] │ └── projections - │ └── l_extendedprice:23 * (1.0 - l_discount:24) [as=column48:48, type=float, outer=(23,24), immutable] + │ └── l_extendedprice:25 * (1.0 - l_discount:26) [as=column54:54, type=float, outer=(25,26), immutable] └── aggregations - └── sum [as=sum:49, type=float, outer=(48)] - └── column48:48 [type=float] + └── sum [as=sum:55, type=float, outer=(54)] + └── column54:54 [type=float] stats table=q5_sort_1 ---- @@ -191,11 +191,11 @@ column_names row_count_est row_count_err distinct_count_est distinct_count_e stats table=q5_project_3 ---- column_names row_count distinct_count null_count -{column48} 7243 7245 0 +{column54} 7243 7245 0 {n_name} 7243 5 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{column48} 13445.00 1.86 13136.00 1.81 0.00 1.00 +{column54} 13445.00 1.86 13136.00 1.81 0.00 1.00 {n_name} 13445.00 1.86 5.00 1.00 0.00 1.00 stats table=q5_inner_join_4 diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q06 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q06 index e64711c89f24..44ec63b7eec8 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q06 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q06 @@ -32,17 +32,17 @@ WHERE ---- scalar-group-by ├── save-table-name: q6_scalar_group_by_1 - ├── columns: revenue:18(float) + ├── columns: revenue:19(float) ├── cardinality: [1 - 1] ├── immutable - ├── stats: [rows=1, distinct(18)=1, null(18)=0] + ├── stats: [rows=1, distinct(19)=1, null(19)=0] ├── key: () - ├── fd: ()-->(18) + ├── fd: ()-->(19) ├── project │ ├── save-table-name: q6_project_2 - │ ├── columns: column17:17(float!null) + │ ├── columns: column18:18(float!null) │ ├── immutable - │ ├── stats: [rows=34745.8339, distinct(17)=34745.8339, null(17)=0] + │ ├── stats: [rows=34745.8339, distinct(18)=34745.8339, null(18)=0] │ ├── select │ │ ├── save-table-name: q6_select_3 │ │ ├── columns: l_quantity:5(float!null) l_extendedprice:6(float!null) l_discount:7(float!null) l_shipdate:11(date!null) @@ -68,10 +68,10 @@ scalar-group-by │ │ ├── (l_discount:7 >= 0.05) AND (l_discount:7 <= 0.07) [type=bool, outer=(7), constraints=(/7: [/0.05 - /0.07]; tight)] │ │ └── l_quantity:5 < 24.0 [type=bool, outer=(5), constraints=(/5: (/NULL - /23.999999999999996]; tight)] │ └── projections - │ └── l_extendedprice:6 * l_discount:7 [as=column17:17, type=float, outer=(6,7), immutable] + │ └── l_extendedprice:6 * l_discount:7 [as=column18:18, type=float, outer=(6,7), immutable] └── aggregations - └── sum [as=sum:18, type=float, outer=(17)] - └── column17:17 [type=float] + └── sum [as=sum:19, type=float, outer=(18)] + └── column18:18 [type=float] stats table=q6_scalar_group_by_1 ---- @@ -84,10 +84,10 @@ column_names row_count_est row_count_err distinct_count_est distinct_count_e stats table=q6_project_2 ---- column_names row_count distinct_count null_count -{column17} 114160 108866 0 +{column18} 114160 108866 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{column17} 34746.00 3.29 <== 34746.00 3.13 <== 0.00 1.00 +{column18} 34746.00 3.29 <== 34746.00 3.13 <== 0.00 1.00 stats table=q6_select_3 ---- diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q07 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q07 index 3a276d924d17..3dcf76f174d9 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q07 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q07 @@ -58,107 +58,107 @@ ORDER BY ---- sort ├── save-table-name: q7_sort_1 - ├── columns: supp_nation:42(char!null) cust_nation:46(char!null) l_year:49(float) revenue:51(float!null) + ├── columns: supp_nation:46(char!null) cust_nation:51(char!null) l_year:55(float) revenue:57(float!null) ├── immutable - ├── stats: [rows=974.320532, distinct(42)=1.33333333, null(42)=0, distinct(46)=1.33333333, null(46)=0, distinct(49)=730.981616, null(49)=0, distinct(51)=974.320532, null(51)=0, distinct(42,46,49)=974.320532, null(42,46,49)=0] - ├── key: (42,46,49) - ├── fd: (42,46,49)-->(51) - ├── ordering: +42,+46,+49 + ├── stats: [rows=974.320532, distinct(46)=1.33333333, null(46)=0, distinct(51)=1.33333333, null(51)=0, distinct(55)=730.981616, null(55)=0, distinct(57)=974.320532, null(57)=0, distinct(46,51,55)=974.320532, null(46,51,55)=0] + ├── key: (46,51,55) + ├── fd: (46,51,55)-->(57) + ├── ordering: +46,+51,+55 └── group-by ├── save-table-name: q7_group_by_2 - ├── columns: n1.n_name:42(char!null) n2.n_name:46(char!null) l_year:49(float) sum:51(float!null) - ├── grouping columns: n1.n_name:42(char!null) n2.n_name:46(char!null) l_year:49(float) + ├── columns: n1.n_name:46(char!null) n2.n_name:51(char!null) l_year:55(float) sum:57(float!null) + ├── grouping columns: n1.n_name:46(char!null) n2.n_name:51(char!null) l_year:55(float) ├── immutable - ├── stats: [rows=974.320532, distinct(42)=1.33333333, null(42)=0, distinct(46)=1.33333333, null(46)=0, distinct(49)=730.981616, null(49)=0, distinct(51)=974.320532, null(51)=0, distinct(42,46,49)=974.320532, null(42,46,49)=0] - ├── key: (42,46,49) - ├── fd: (42,46,49)-->(51) + ├── stats: [rows=974.320532, distinct(46)=1.33333333, null(46)=0, distinct(51)=1.33333333, null(51)=0, distinct(55)=730.981616, null(55)=0, distinct(57)=974.320532, null(57)=0, distinct(46,51,55)=974.320532, null(46,51,55)=0] + ├── key: (46,51,55) + ├── fd: (46,51,55)-->(57) ├── project │ ├── save-table-name: q7_project_3 - │ ├── columns: l_year:49(float) volume:50(float!null) n1.n_name:42(char!null) n2.n_name:46(char!null) + │ ├── columns: l_year:55(float) volume:56(float!null) n1.n_name:46(char!null) n2.n_name:51(char!null) │ ├── immutable - │ ├── stats: [rows=7741.78379, distinct(42)=1.33333333, null(42)=0, distinct(46)=1.33333333, null(46)=0, distinct(49)=730.981616, null(49)=0, distinct(50)=7579.92926, null(50)=0, distinct(42,46,49)=974.320532, null(42,46,49)=0] + │ ├── stats: [rows=7741.78379, distinct(46)=1.33333333, null(46)=0, distinct(51)=1.33333333, null(51)=0, distinct(55)=730.981616, null(55)=0, distinct(56)=7579.92926, null(56)=0, distinct(46,51,55)=974.320532, null(46,51,55)=0] │ ├── inner-join (hash) │ │ ├── save-table-name: q7_inner_join_4 - │ │ ├── columns: s_suppkey:1(int!null) s_nationkey:4(int!null) l_orderkey:8(int!null) l_suppkey:10(int!null) l_extendedprice:13(float!null) l_discount:14(float!null) l_shipdate:18(date!null) o_orderkey:24(int!null) o_custkey:25(int!null) c_custkey:33(int!null) c_nationkey:36(int!null) n1.n_nationkey:41(int!null) n1.n_name:42(char!null) n2.n_nationkey:45(int!null) n2.n_name:46(char!null) + │ │ ├── columns: s_suppkey:1(int!null) s_nationkey:4(int!null) l_orderkey:9(int!null) l_suppkey:11(int!null) l_extendedprice:14(float!null) l_discount:15(float!null) l_shipdate:19(date!null) o_orderkey:26(int!null) o_custkey:27(int!null) c_custkey:36(int!null) c_nationkey:39(int!null) n1.n_nationkey:45(int!null) n1.n_name:46(char!null) n2.n_nationkey:50(int!null) n2.n_name:51(char!null) │ │ ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) - │ │ ├── stats: [rows=7741.78379, distinct(1)=7741.78379, null(1)=0, distinct(4)=1.29975178, null(4)=0, distinct(8)=7488.03308, null(8)=0, distinct(10)=7741.78379, null(10)=0, distinct(13)=7569.91685, null(13)=0, distinct(14)=11, null(14)=0, distinct(18)=730.981616, null(18)=0, distinct(24)=7488.03308, null(24)=0, distinct(25)=4946.3467, null(25)=0, distinct(33)=4946.3467, null(33)=0, distinct(36)=1.29975178, null(36)=0, distinct(41)=1.29975178, null(41)=0, distinct(42)=1.33333333, null(42)=0, distinct(45)=1.29975178, null(45)=0, distinct(46)=1.33333333, null(46)=0, distinct(13,14)=7579.92926, null(13,14)=0, distinct(18,42,46)=974.320532, null(18,42,46)=0] - │ │ ├── fd: (1)-->(4), (24)-->(25), (33)-->(36), (41)-->(42), (45)-->(46), (36)==(45), (45)==(36), (25)==(33), (33)==(25), (8)==(24), (24)==(8), (1)==(10), (10)==(1), (4)==(41), (41)==(4) + │ │ ├── stats: [rows=7741.78379, distinct(1)=7741.78379, null(1)=0, distinct(4)=1.29975178, null(4)=0, distinct(9)=7488.03308, null(9)=0, distinct(11)=7741.78379, null(11)=0, distinct(14)=7569.91685, null(14)=0, distinct(15)=11, null(15)=0, distinct(19)=730.981616, null(19)=0, distinct(26)=7488.03308, null(26)=0, distinct(27)=4946.3467, null(27)=0, distinct(36)=4946.3467, null(36)=0, distinct(39)=1.29975178, null(39)=0, distinct(45)=1.29975178, null(45)=0, distinct(46)=1.33333333, null(46)=0, distinct(50)=1.29975178, null(50)=0, distinct(51)=1.33333333, null(51)=0, distinct(14,15)=7579.92926, null(14,15)=0, distinct(19,46,51)=974.320532, null(19,46,51)=0] + │ │ ├── fd: (1)-->(4), (26)-->(27), (36)-->(39), (45)-->(46), (50)-->(51), (39)==(50), (50)==(39), (27)==(36), (36)==(27), (9)==(26), (26)==(9), (1)==(11), (11)==(1), (4)==(45), (45)==(4) │ │ ├── scan customer@c_nk │ │ │ ├── save-table-name: q7_scan_5 - │ │ │ ├── columns: c_custkey:33(int!null) c_nationkey:36(int!null) - │ │ │ ├── stats: [rows=150000, distinct(33)=148813, null(33)=0, distinct(36)=25, null(36)=0] - │ │ │ │ histogram(33)= 0 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 + │ │ │ ├── columns: c_custkey:36(int!null) c_nationkey:39(int!null) + │ │ │ ├── stats: [rows=150000, distinct(36)=148813, null(36)=0, distinct(39)=25, null(39)=0] + │ │ │ │ histogram(36)= 0 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 735 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 750 15 │ │ │ │ <--- 2 ----- 776 ----- 1422 ----- 2189 ----- 2973 ----- 3583 ----- 4390 ----- 5154 ----- 5962 ----- 6965 ----- 7596 ----- 8303 ----- 9167 ----- 9833 ----- 10695 ----- 11397 ----- 11979 ----- 12651 ----- 13397 ----- 14144 ----- 14951 ----- 15698 ----- 16460 ----- 17203 ----- 17846 ----- 18462 ----- 19390 ----- 20189 ----- 20852 ----- 21642 ----- 22379 ----- 23009 ----- 23856 ----- 24734 ----- 25473 ----- 26231 ----- 26978 ----- 27654 ----- 28276 ----- 29054 ----- 29727 ----- 30527 ----- 31177 ----- 32126 ----- 32984 ----- 33684 ----- 34316 ----- 35070 ----- 35703 ----- 36397 ----- 37156 ----- 37709 ----- 38488 ----- 39131 ----- 39740 ----- 40736 ----- 41459 ----- 42388 ----- 42999 ----- 43844 ----- 44571 ----- 45428 ----- 46283 ----- 46979 ----- 47712 ----- 48708 ----- 49487 ----- 50275 ----- 51131 ----- 51836 ----- 52652 ----- 53389 ----- 54179 ----- 54861 ----- 55609 ----- 56492 ----- 57284 ----- 57917 ----- 58793 ----- 59665 ----- 60285 ----- 60840 ----- 61523 ----- 62354 ----- 63178 ----- 63933 ----- 64642 ----- 65282 ----- 65864 ----- 66755 ----- 67407 ----- 68099 ----- 68875 ----- 69638 ----- 70304 ----- 71016 ----- 71830 ----- 72742 ----- 73590 ----- 74434 ----- 75274 ----- 75861 ----- 76547 ----- 77252 ----- 77978 ----- 78650 ----- 79313 ----- 79925 ----- 80677 ----- 81497 ----- 82205 ----- 82962 ----- 83879 ----- 84815 ----- 85521 ----- 86272 ----- 87140 ----- 87759 ----- 88634 ----- 89452 ----- 90192 ----- 90920 ----- 91756 ----- 92690 ----- 93299 ----- 93950 ----- 94812 ----- 95569 ----- 96295 ----- 96904 ----- 97499 ----- 98144 ----- 98764 ----- 99582 ----- 100453 ----- 101098 ----- 101892 ----- 102700 ----- 103419 ----- 104297 ----- 105040 ----- 105864 ----- 106498 ----- 107196 ----- 108022 ----- 108731 ----- 109398 ----- 110145 ----- 110849 ----- 111758 ----- 112501 ----- 113222 ----- 114019 ----- 114904 ----- 115693 ----- 116350 ----- 116955 ----- 117581 ----- 118366 ----- 119159 ----- 119902 ----- 120535 ----- 121321 ----- 121993 ----- 122769 ----- 123504 ----- 124225 ----- 124992 ----- 125632 ----- 126685 ----- 127641 ----- 128303 ----- 129042 ----- 129589 ----- 130548 ----- 131374 ----- 132325 ----- 133042 ----- 133883 ----- 134716 ----- 135520 ----- 136173 ----- 136858 ----- 137584 ----- 138381 ----- 139162 ----- 139923 ----- 140738 ----- 141557 ----- 142287 ----- 143002 ----- 143794 ----- 144420 ----- 145276 ----- 146100 ----- 146977 ----- 147821 ----- 148440 ----- 149247 ----- 149978 - │ │ │ │ histogram(36)= 0 6255 0 5190 0 6345 0 5505 0 5520 0 6585 0 6405 0 5895 0 5640 0 5775 0 5745 0 6630 0 6300 0 6255 0 6105 0 5760 0 6270 0 5850 0 6195 0 6330 0 6150 0 5820 0 5790 0 5985 0 5700 + │ │ │ │ histogram(39)= 0 6255 0 5190 0 6345 0 5505 0 5520 0 6585 0 6405 0 5895 0 5640 0 5775 0 5745 0 6630 0 6300 0 6255 0 6105 0 5760 0 6270 0 5850 0 6195 0 6330 0 6150 0 5820 0 5790 0 5985 0 5700 │ │ │ │ <--- 0 ---- 1 ---- 2 ---- 3 ---- 4 ---- 5 ---- 6 ---- 7 ---- 8 ---- 9 ---- 10 --- 11 --- 12 --- 13 --- 14 --- 15 --- 16 --- 17 --- 18 --- 19 --- 20 --- 21 --- 22 --- 23 --- 24 - │ │ │ ├── key: (33) - │ │ │ └── fd: (33)-->(36) + │ │ │ ├── key: (36) + │ │ │ └── fd: (36)-->(39) │ │ ├── inner-join (lookup orders) │ │ │ ├── save-table-name: q7_lookup_join_6 - │ │ │ ├── columns: s_suppkey:1(int!null) s_nationkey:4(int!null) l_orderkey:8(int!null) l_suppkey:10(int!null) l_extendedprice:13(float!null) l_discount:14(float!null) l_shipdate:18(date!null) o_orderkey:24(int!null) o_custkey:25(int!null) n1.n_nationkey:41(int!null) n1.n_name:42(char!null) n2.n_nationkey:45(int!null) n2.n_name:46(char!null) - │ │ │ ├── key columns: [8] = [24] + │ │ │ ├── columns: s_suppkey:1(int!null) s_nationkey:4(int!null) l_orderkey:9(int!null) l_suppkey:11(int!null) l_extendedprice:14(float!null) l_discount:15(float!null) l_shipdate:19(date!null) o_orderkey:26(int!null) o_custkey:27(int!null) n1.n_nationkey:45(int!null) n1.n_name:46(char!null) n2.n_nationkey:50(int!null) n2.n_name:51(char!null) + │ │ │ ├── key columns: [9] = [26] │ │ │ ├── lookup columns are key - │ │ │ ├── stats: [rows=101372.546, distinct(1)=529.630441, null(1)=0, distinct(4)=1.29975178, null(4)=0, distinct(8)=97145.926, null(8)=0, distinct(10)=529.630441, null(10)=0, distinct(13)=62473.272, null(13)=0, distinct(14)=11, null(14)=0, distinct(18)=731, null(18)=0, distinct(24)=97145.926, null(24)=0, distinct(25)=63672.0351, null(25)=0, distinct(41)=1.29975178, null(41)=0, distinct(42)=1.33333333, null(42)=0, distinct(45)=1.29975178, null(45)=0, distinct(46)=1.33333333, null(46)=0] - │ │ │ ├── fd: (24)-->(25), (1)-->(4), (41)-->(42), (45)-->(46), (4)==(41), (41)==(4), (1)==(10), (10)==(1), (8)==(24), (24)==(8) + │ │ │ ├── stats: [rows=101372.546, distinct(1)=529.630441, null(1)=0, distinct(4)=1.29975178, null(4)=0, distinct(9)=97145.926, null(9)=0, distinct(11)=529.630441, null(11)=0, distinct(14)=62473.272, null(14)=0, distinct(15)=11, null(15)=0, distinct(19)=731, null(19)=0, distinct(26)=97145.926, null(26)=0, distinct(27)=63672.0351, null(27)=0, distinct(45)=1.29975178, null(45)=0, distinct(46)=1.33333333, null(46)=0, distinct(50)=1.29975178, null(50)=0, distinct(51)=1.33333333, null(51)=0] + │ │ │ ├── fd: (26)-->(27), (1)-->(4), (45)-->(46), (50)-->(51), (4)==(45), (45)==(4), (1)==(11), (11)==(1), (9)==(26), (26)==(9) │ │ │ ├── inner-join (lookup lineitem) │ │ │ │ ├── save-table-name: q7_lookup_join_7 - │ │ │ │ ├── columns: s_suppkey:1(int!null) s_nationkey:4(int!null) l_orderkey:8(int!null) l_suppkey:10(int!null) l_extendedprice:13(float!null) l_discount:14(float!null) l_shipdate:18(date!null) n1.n_nationkey:41(int!null) n1.n_name:42(char!null) n2.n_nationkey:45(int!null) n2.n_name:46(char!null) - │ │ │ │ ├── key columns: [8 11] = [8 11] + │ │ │ │ ├── columns: s_suppkey:1(int!null) s_nationkey:4(int!null) l_orderkey:9(int!null) l_suppkey:11(int!null) l_extendedprice:14(float!null) l_discount:15(float!null) l_shipdate:19(date!null) n1.n_nationkey:45(int!null) n1.n_name:46(char!null) n2.n_nationkey:50(int!null) n2.n_name:51(char!null) + │ │ │ │ ├── key columns: [9 12] = [9 12] │ │ │ │ ├── lookup columns are key - │ │ │ │ ├── stats: [rows=101372.546, distinct(1)=529.630441, null(1)=0, distinct(4)=1.29975178, null(4)=0, distinct(8)=97145.926, null(8)=0, distinct(10)=529.630441, null(10)=0, distinct(13)=95536.4092, null(13)=0, distinct(14)=11, null(14)=0, distinct(18)=731, null(18)=0, distinct(41)=1.29975178, null(41)=0, distinct(42)=1.33333333, null(42)=0, distinct(45)=1.29975178, null(45)=0, distinct(46)=1.33333333, null(46)=0] - │ │ │ │ ├── fd: (1)-->(4), (41)-->(42), (45)-->(46), (4)==(41), (41)==(4), (1)==(10), (10)==(1) + │ │ │ │ ├── stats: [rows=101372.546, distinct(1)=529.630441, null(1)=0, distinct(4)=1.29975178, null(4)=0, distinct(9)=97145.926, null(9)=0, distinct(11)=529.630441, null(11)=0, distinct(14)=95536.4092, null(14)=0, distinct(15)=11, null(15)=0, distinct(19)=731, null(19)=0, distinct(45)=1.29975178, null(45)=0, distinct(46)=1.33333333, null(46)=0, distinct(50)=1.29975178, null(50)=0, distinct(51)=1.33333333, null(51)=0] + │ │ │ │ ├── fd: (1)-->(4), (45)-->(46), (50)-->(51), (4)==(45), (45)==(4), (1)==(11), (11)==(1) │ │ │ │ ├── inner-join (lookup lineitem@l_sk) │ │ │ │ │ ├── save-table-name: q7_lookup_join_8 - │ │ │ │ │ ├── columns: s_suppkey:1(int!null) s_nationkey:4(int!null) l_orderkey:8(int!null) l_suppkey:10(int!null) l_linenumber:11(int!null) n1.n_nationkey:41(int!null) n1.n_name:42(char!null) n2.n_nationkey:45(int!null) n2.n_name:46(char!null) - │ │ │ │ │ ├── key columns: [1] = [10] - │ │ │ │ │ ├── stats: [rows=322645.968, distinct(1)=529.630441, null(1)=0, distinct(4)=1.29975178, null(4)=0, distinct(8)=290856.916, null(8)=0, distinct(10)=529.630441, null(10)=0, distinct(11)=7, null(11)=0, distinct(41)=1.29975178, null(41)=0, distinct(42)=1.33333333, null(42)=0, distinct(45)=1.29975178, null(45)=0, distinct(46)=1.33333333, null(46)=0] - │ │ │ │ │ ├── key: (8,11,45) - │ │ │ │ │ ├── fd: (1)-->(4), (41)-->(42), (45)-->(46), (4)==(41), (41)==(4), (8,11)-->(10), (1)==(10), (10)==(1) + │ │ │ │ │ ├── columns: s_suppkey:1(int!null) s_nationkey:4(int!null) l_orderkey:9(int!null) l_suppkey:11(int!null) l_linenumber:12(int!null) n1.n_nationkey:45(int!null) n1.n_name:46(char!null) n2.n_nationkey:50(int!null) n2.n_name:51(char!null) + │ │ │ │ │ ├── key columns: [1] = [11] + │ │ │ │ │ ├── stats: [rows=322645.968, distinct(1)=529.630441, null(1)=0, distinct(4)=1.29975178, null(4)=0, distinct(9)=290856.916, null(9)=0, distinct(11)=529.630441, null(11)=0, distinct(12)=7, null(12)=0, distinct(45)=1.29975178, null(45)=0, distinct(46)=1.33333333, null(46)=0, distinct(50)=1.29975178, null(50)=0, distinct(51)=1.33333333, null(51)=0] + │ │ │ │ │ ├── key: (9,12,50) + │ │ │ │ │ ├── fd: (1)-->(4), (45)-->(46), (50)-->(51), (4)==(45), (45)==(4), (9,12)-->(11), (1)==(11), (11)==(1) │ │ │ │ │ ├── inner-join (lookup supplier@s_nk) │ │ │ │ │ │ ├── save-table-name: q7_lookup_join_9 - │ │ │ │ │ │ ├── columns: s_suppkey:1(int!null) s_nationkey:4(int!null) n1.n_nationkey:41(int!null) n1.n_name:42(char!null) n2.n_nationkey:45(int!null) n2.n_name:46(char!null) - │ │ │ │ │ │ ├── key columns: [41] = [4] - │ │ │ │ │ │ ├── stats: [rows=533.333333, distinct(1)=529.630441, null(1)=0, distinct(4)=1.29975178, null(4)=0, distinct(41)=1.29975178, null(41)=0, distinct(42)=1.33333333, null(42)=0, distinct(45)=1.29975178, null(45)=0, distinct(46)=1.33333333, null(46)=0] - │ │ │ │ │ │ ├── key: (1,45) - │ │ │ │ │ │ ├── fd: (1)-->(4), (41)-->(42), (45)-->(46), (4)==(41), (41)==(4) + │ │ │ │ │ │ ├── columns: s_suppkey:1(int!null) s_nationkey:4(int!null) n1.n_nationkey:45(int!null) n1.n_name:46(char!null) n2.n_nationkey:50(int!null) n2.n_name:51(char!null) + │ │ │ │ │ │ ├── key columns: [45] = [4] + │ │ │ │ │ │ ├── stats: [rows=533.333333, distinct(1)=529.630441, null(1)=0, distinct(4)=1.29975178, null(4)=0, distinct(45)=1.29975178, null(45)=0, distinct(46)=1.33333333, null(46)=0, distinct(50)=1.29975178, null(50)=0, distinct(51)=1.33333333, null(51)=0] + │ │ │ │ │ │ ├── key: (1,50) + │ │ │ │ │ │ ├── fd: (1)-->(4), (45)-->(46), (50)-->(51), (4)==(45), (45)==(4) │ │ │ │ │ │ ├── inner-join (cross) │ │ │ │ │ │ │ ├── save-table-name: q7_inner_join_10 - │ │ │ │ │ │ │ ├── columns: n1.n_nationkey:41(int!null) n1.n_name:42(char!null) n2.n_nationkey:45(int!null) n2.n_name:46(char!null) - │ │ │ │ │ │ │ ├── stats: [rows=1.33333333, distinct(41)=1.29975178, null(41)=0, distinct(42)=1.33333333, null(42)=0, distinct(45)=1.29975178, null(45)=0, distinct(46)=1.33333333, null(46)=0, distinct(42,46)=1.33333333, null(42,46)=0] - │ │ │ │ │ │ │ ├── key: (41,45) - │ │ │ │ │ │ │ ├── fd: (41)-->(42), (45)-->(46) + │ │ │ │ │ │ │ ├── columns: n1.n_nationkey:45(int!null) n1.n_name:46(char!null) n2.n_nationkey:50(int!null) n2.n_name:51(char!null) + │ │ │ │ │ │ │ ├── stats: [rows=1.33333333, distinct(45)=1.29975178, null(45)=0, distinct(46)=1.33333333, null(46)=0, distinct(50)=1.29975178, null(50)=0, distinct(51)=1.33333333, null(51)=0, distinct(46,51)=1.33333333, null(46,51)=0] + │ │ │ │ │ │ │ ├── key: (45,50) + │ │ │ │ │ │ │ ├── fd: (45)-->(46), (50)-->(51) │ │ │ │ │ │ │ ├── scan n1 │ │ │ │ │ │ │ │ ├── save-table-name: q7_scan_11 - │ │ │ │ │ │ │ │ ├── columns: n1.n_nationkey:41(int!null) n1.n_name:42(char!null) - │ │ │ │ │ │ │ │ ├── stats: [rows=25, distinct(41)=25, null(41)=0, distinct(42)=25, null(42)=0] - │ │ │ │ │ │ │ │ │ histogram(41)= 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 - │ │ │ │ │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 --- 5 --- 6 --- 7 --- 8 --- 9 --- 10 --- 11 --- 12 --- 13 --- 14 --- 15 --- 16 --- 17 --- 18 --- 19 --- 20 --- 21 --- 22 --- 23 --- 24 - │ │ │ │ │ │ │ │ ├── key: (41) - │ │ │ │ │ │ │ │ └── fd: (41)-->(42) - │ │ │ │ │ │ │ ├── scan n2 - │ │ │ │ │ │ │ │ ├── save-table-name: q7_scan_12 - │ │ │ │ │ │ │ │ ├── columns: n2.n_nationkey:45(int!null) n2.n_name:46(char!null) + │ │ │ │ │ │ │ │ ├── columns: n1.n_nationkey:45(int!null) n1.n_name:46(char!null) │ │ │ │ │ │ │ │ ├── stats: [rows=25, distinct(45)=25, null(45)=0, distinct(46)=25, null(46)=0] │ │ │ │ │ │ │ │ │ histogram(45)= 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 │ │ │ │ │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 --- 5 --- 6 --- 7 --- 8 --- 9 --- 10 --- 11 --- 12 --- 13 --- 14 --- 15 --- 16 --- 17 --- 18 --- 19 --- 20 --- 21 --- 22 --- 23 --- 24 │ │ │ │ │ │ │ │ ├── key: (45) │ │ │ │ │ │ │ │ └── fd: (45)-->(46) + │ │ │ │ │ │ │ ├── scan n2 + │ │ │ │ │ │ │ │ ├── save-table-name: q7_scan_12 + │ │ │ │ │ │ │ │ ├── columns: n2.n_nationkey:50(int!null) n2.n_name:51(char!null) + │ │ │ │ │ │ │ │ ├── stats: [rows=25, distinct(50)=25, null(50)=0, distinct(51)=25, null(51)=0] + │ │ │ │ │ │ │ │ │ histogram(50)= 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 + │ │ │ │ │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 --- 5 --- 6 --- 7 --- 8 --- 9 --- 10 --- 11 --- 12 --- 13 --- 14 --- 15 --- 16 --- 17 --- 18 --- 19 --- 20 --- 21 --- 22 --- 23 --- 24 + │ │ │ │ │ │ │ │ ├── key: (50) + │ │ │ │ │ │ │ │ └── fd: (50)-->(51) │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── ((n1.n_name:42 = 'FRANCE') AND (n2.n_name:46 = 'GERMANY')) OR ((n1.n_name:42 = 'GERMANY') AND (n2.n_name:46 = 'FRANCE')) [type=bool, outer=(42,46), constraints=(/42: [/'FRANCE' - /'FRANCE'] [/'GERMANY' - /'GERMANY']; /46: [/'FRANCE' - /'FRANCE'] [/'GERMANY' - /'GERMANY'])] + │ │ │ │ │ │ │ └── ((n1.n_name:46 = 'FRANCE') AND (n2.n_name:51 = 'GERMANY')) OR ((n1.n_name:46 = 'GERMANY') AND (n2.n_name:51 = 'FRANCE')) [type=bool, outer=(46,51), constraints=(/46: [/'FRANCE' - /'FRANCE'] [/'GERMANY' - /'GERMANY']; /51: [/'FRANCE' - /'FRANCE'] [/'GERMANY' - /'GERMANY'])] │ │ │ │ │ │ └── filters (true) │ │ │ │ │ └── filters (true) │ │ │ │ └── filters - │ │ │ │ └── (l_shipdate:18 >= '1995-01-01') AND (l_shipdate:18 <= '1996-12-31') [type=bool, outer=(18), constraints=(/18: [/'1995-01-01' - /'1996-12-31']; tight)] + │ │ │ │ └── (l_shipdate:19 >= '1995-01-01') AND (l_shipdate:19 <= '1996-12-31') [type=bool, outer=(19), constraints=(/19: [/'1995-01-01' - /'1996-12-31']; tight)] │ │ │ └── filters (true) │ │ └── filters - │ │ ├── c_custkey:33 = o_custkey:25 [type=bool, outer=(25,33), constraints=(/25: (/NULL - ]; /33: (/NULL - ]), fd=(25)==(33), (33)==(25)] - │ │ └── c_nationkey:36 = n2.n_nationkey:45 [type=bool, outer=(36,45), constraints=(/36: (/NULL - ]; /45: (/NULL - ]), fd=(36)==(45), (45)==(36)] + │ │ ├── c_custkey:36 = o_custkey:27 [type=bool, outer=(27,36), constraints=(/27: (/NULL - ]; /36: (/NULL - ]), fd=(27)==(36), (36)==(27)] + │ │ └── c_nationkey:39 = n2.n_nationkey:50 [type=bool, outer=(39,50), constraints=(/39: (/NULL - ]; /50: (/NULL - ]), fd=(39)==(50), (50)==(39)] │ └── projections - │ ├── extract('year', l_shipdate:18) [as=l_year:49, type=float, outer=(18), immutable] - │ └── l_extendedprice:13 * (1.0 - l_discount:14) [as=volume:50, type=float, outer=(13,14), immutable] + │ ├── extract('year', l_shipdate:19) [as=l_year:55, type=float, outer=(19), immutable] + │ └── l_extendedprice:14 * (1.0 - l_discount:15) [as=volume:56, type=float, outer=(14,15), immutable] └── aggregations - └── sum [as=sum:51, type=float, outer=(50)] - └── volume:50 [type=float] + └── sum [as=sum:57, type=float, outer=(56)] + └── volume:56 [type=float] stats table=q7_sort_1 ---- diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q08 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q08 index 3a85b57a6a8f..c8bf84a0f6f4 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q08 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q08 @@ -57,132 +57,132 @@ ORDER BY ---- sort ├── save-table-name: q8_sort_1 - ├── columns: o_year:61(float) mkt_share:66(float!null) + ├── columns: o_year:69(float) mkt_share:74(float!null) ├── immutable - ├── stats: [rows=728.968264, distinct(61)=728.968264, null(61)=0, distinct(66)=728.968264, null(66)=0] - ├── key: (61) - ├── fd: (61)-->(66) - ├── ordering: +61 + ├── stats: [rows=728.968264, distinct(69)=728.968264, null(69)=0, distinct(74)=728.968264, null(74)=0] + ├── key: (69) + ├── fd: (69)-->(74) + ├── ordering: +69 └── project ├── save-table-name: q8_project_2 - ├── columns: mkt_share:66(float!null) o_year:61(float) + ├── columns: mkt_share:74(float!null) o_year:69(float) ├── immutable - ├── stats: [rows=728.968264, distinct(61)=728.968264, null(61)=0, distinct(66)=728.968264, null(66)=0] - ├── key: (61) - ├── fd: (61)-->(66) + ├── stats: [rows=728.968264, distinct(69)=728.968264, null(69)=0, distinct(74)=728.968264, null(74)=0] + ├── key: (69) + ├── fd: (69)-->(74) ├── group-by │ ├── save-table-name: q8_group_by_3 - │ ├── columns: o_year:61(float) sum:64(float!null) sum:65(float!null) - │ ├── grouping columns: o_year:61(float) + │ ├── columns: o_year:69(float) sum:72(float!null) sum:73(float!null) + │ ├── grouping columns: o_year:69(float) │ ├── immutable - │ ├── stats: [rows=728.968264, distinct(61)=728.968264, null(61)=0, distinct(64)=728.968264, null(64)=0, distinct(65)=728.968264, null(65)=0, distinct(64,65)=728.968264, null(64,65)=0] - │ ├── key: (61) - │ ├── fd: (61)-->(64,65) + │ ├── stats: [rows=728.968264, distinct(69)=728.968264, null(69)=0, distinct(72)=728.968264, null(72)=0, distinct(73)=728.968264, null(73)=0, distinct(72,73)=728.968264, null(72,73)=0] + │ ├── key: (69) + │ ├── fd: (69)-->(72,73) │ ├── project │ │ ├── save-table-name: q8_project_4 - │ │ ├── columns: column63:63(float!null) o_year:61(float) volume:62(float!null) + │ │ ├── columns: column71:71(float!null) o_year:69(float) volume:70(float!null) │ │ ├── immutable - │ │ ├── stats: [rows=4302.3045, distinct(61)=728.968264, null(61)=0, distinct(62)=4268.40037, null(62)=0, distinct(63)=4285.33021, null(63)=0] + │ │ ├── stats: [rows=4302.3045, distinct(69)=728.968264, null(69)=0, distinct(70)=4268.40037, null(70)=0, distinct(71)=4285.33021, null(71)=0] │ │ ├── project │ │ │ ├── save-table-name: q8_project_5 - │ │ │ ├── columns: o_year:61(float) volume:62(float!null) n2.n_name:55(char!null) + │ │ │ ├── columns: o_year:69(float) volume:70(float!null) n2.n_name:61(char!null) │ │ │ ├── immutable - │ │ │ ├── stats: [rows=4302.3045, distinct(55)=25, null(55)=0, distinct(61)=728.968264, null(61)=0, distinct(62)=4268.40037, null(62)=0, distinct(55,62)=4285.33021, null(55,62)=0] + │ │ │ ├── stats: [rows=4302.3045, distinct(61)=25, null(61)=0, distinct(69)=728.968264, null(69)=0, distinct(70)=4268.40037, null(70)=0, distinct(61,70)=4285.33021, null(61,70)=0] │ │ │ ├── inner-join (hash) │ │ │ │ ├── save-table-name: q8_inner_join_6 - │ │ │ │ ├── columns: p_partkey:1(int!null) p_type:5(varchar!null) s_suppkey:10(int!null) s_nationkey:13(int!null) l_orderkey:17(int!null) l_partkey:18(int!null) l_suppkey:19(int!null) l_extendedprice:22(float!null) l_discount:23(float!null) o_orderkey:33(int!null) o_custkey:34(int!null) o_orderdate:37(date!null) c_custkey:42(int!null) c_nationkey:45(int!null) n1.n_nationkey:50(int!null) n1.n_regionkey:52(int!null) n2.n_nationkey:54(int!null) n2.n_name:55(char!null) r_regionkey:58(int!null) r_name:59(char!null) + │ │ │ │ ├── columns: p_partkey:1(int!null) p_type:5(varchar!null) s_suppkey:11(int!null) s_nationkey:14(int!null) l_orderkey:19(int!null) l_partkey:20(int!null) l_suppkey:21(int!null) l_extendedprice:24(float!null) l_discount:25(float!null) o_orderkey:36(int!null) o_custkey:37(int!null) o_orderdate:40(date!null) c_custkey:46(int!null) c_nationkey:49(int!null) n1.n_nationkey:55(int!null) n1.n_regionkey:57(int!null) n2.n_nationkey:60(int!null) n2.n_name:61(char!null) r_regionkey:65(int!null) r_name:66(char!null) │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ │ │ │ ├── stats: [rows=4302.3045, distinct(1)=1333.31636, null(1)=0, distinct(5)=1, null(5)=0, distinct(10)=3490.79055, null(10)=0, distinct(13)=25, null(13)=0, distinct(17)=4223.63113, null(17)=0, distinct(18)=1333.31636, null(18)=0, distinct(19)=3490.79055, null(19)=0, distinct(22)=4265.95085, null(22)=0, distinct(23)=11, null(23)=0, distinct(33)=4223.63113, null(33)=0, distinct(34)=3984.54821, null(34)=0, distinct(37)=728.968264, null(37)=0, distinct(42)=3984.54821, null(42)=0, distinct(45)=5, null(45)=0, distinct(50)=5, null(50)=0, distinct(52)=1, null(52)=0, distinct(54)=25, null(54)=0, distinct(55)=25, null(55)=0, distinct(58)=1, null(58)=0, distinct(59)=0.996222107, null(59)=0, distinct(22,23)=4268.40037, null(22,23)=0, distinct(22,23,55)=4285.33021, null(22,23,55)=0] - │ │ │ │ ├── fd: ()-->(5,59), (10)-->(13), (33)-->(34,37), (42)-->(45), (50)-->(52), (52)==(58), (58)==(52), (45)==(50), (50)==(45), (34)==(42), (42)==(34), (17)==(33), (33)==(17), (10)==(19), (19)==(10), (54)-->(55), (13)==(54), (54)==(13), (1)==(18), (18)==(1) + │ │ │ │ ├── stats: [rows=4302.3045, distinct(1)=1333.31636, null(1)=0, distinct(5)=1, null(5)=0, distinct(11)=3490.79055, null(11)=0, distinct(14)=25, null(14)=0, distinct(19)=4223.63113, null(19)=0, distinct(20)=1333.31636, null(20)=0, distinct(21)=3490.79055, null(21)=0, distinct(24)=4265.95085, null(24)=0, distinct(25)=11, null(25)=0, distinct(36)=4223.63113, null(36)=0, distinct(37)=3984.54821, null(37)=0, distinct(40)=728.968264, null(40)=0, distinct(46)=3984.54821, null(46)=0, distinct(49)=5, null(49)=0, distinct(55)=5, null(55)=0, distinct(57)=1, null(57)=0, distinct(60)=25, null(60)=0, distinct(61)=25, null(61)=0, distinct(65)=1, null(65)=0, distinct(66)=0.996222107, null(66)=0, distinct(24,25)=4268.40037, null(24,25)=0, distinct(24,25,61)=4285.33021, null(24,25,61)=0] + │ │ │ │ ├── fd: ()-->(5,66), (11)-->(14), (36)-->(37,40), (46)-->(49), (55)-->(57), (57)==(65), (65)==(57), (49)==(55), (55)==(49), (37)==(46), (46)==(37), (19)==(36), (36)==(19), (11)==(21), (21)==(11), (60)-->(61), (14)==(60), (60)==(14), (1)==(20), (20)==(1) │ │ │ │ ├── inner-join (hash) │ │ │ │ │ ├── save-table-name: q8_inner_join_7 - │ │ │ │ │ ├── columns: p_partkey:1(int!null) p_type:5(varchar!null) s_suppkey:10(int!null) s_nationkey:13(int!null) l_orderkey:17(int!null) l_partkey:18(int!null) l_suppkey:19(int!null) l_extendedprice:22(float!null) l_discount:23(float!null) o_orderkey:33(int!null) o_custkey:34(int!null) o_orderdate:37(date!null) c_custkey:42(int!null) c_nationkey:45(int!null) n1.n_nationkey:50(int!null) n1.n_regionkey:52(int!null) r_regionkey:58(int!null) r_name:59(char!null) + │ │ │ │ │ ├── columns: p_partkey:1(int!null) p_type:5(varchar!null) s_suppkey:11(int!null) s_nationkey:14(int!null) l_orderkey:19(int!null) l_partkey:20(int!null) l_suppkey:21(int!null) l_extendedprice:24(float!null) l_discount:25(float!null) o_orderkey:36(int!null) o_custkey:37(int!null) o_orderdate:40(date!null) c_custkey:46(int!null) c_nationkey:49(int!null) n1.n_nationkey:55(int!null) n1.n_regionkey:57(int!null) r_regionkey:65(int!null) r_name:66(char!null) │ │ │ │ │ ├── multiplicity: left-rows(zero-or-more), right-rows(exactly-one) - │ │ │ │ │ ├── stats: [rows=4118.35396, distinct(1)=1333.31636, null(1)=0, distinct(5)=1, null(5)=0, distinct(10)=3370.45821, null(10)=0, distinct(13)=25, null(13)=0, distinct(17)=4046.85511, null(17)=0, distinct(18)=1333.31636, null(18)=0, distinct(19)=3370.45821, null(19)=0, distinct(22)=4090.27659, null(22)=0, distinct(23)=11, null(23)=0, distinct(33)=4046.85511, null(33)=0, distinct(34)=3826.55932, null(34)=0, distinct(37)=728.386908, null(37)=0, distinct(42)=3826.55932, null(42)=0, distinct(45)=5, null(45)=0, distinct(50)=5, null(50)=0, distinct(52)=1, null(52)=0, distinct(58)=1, null(58)=0, distinct(59)=0.996222107, null(59)=0] - │ │ │ │ │ ├── fd: ()-->(5,59), (10)-->(13), (33)-->(34,37), (42)-->(45), (50)-->(52), (52)==(58), (58)==(52), (45)==(50), (50)==(45), (34)==(42), (42)==(34), (17)==(33), (33)==(17), (10)==(19), (19)==(10), (1)==(18), (18)==(1) + │ │ │ │ │ ├── stats: [rows=4118.35396, distinct(1)=1333.31636, null(1)=0, distinct(5)=1, null(5)=0, distinct(11)=3370.45821, null(11)=0, distinct(14)=25, null(14)=0, distinct(19)=4046.85511, null(19)=0, distinct(20)=1333.31636, null(20)=0, distinct(21)=3370.45821, null(21)=0, distinct(24)=4090.27659, null(24)=0, distinct(25)=11, null(25)=0, distinct(36)=4046.85511, null(36)=0, distinct(37)=3826.55932, null(37)=0, distinct(40)=728.386908, null(40)=0, distinct(46)=3826.55932, null(46)=0, distinct(49)=5, null(49)=0, distinct(55)=5, null(55)=0, distinct(57)=1, null(57)=0, distinct(65)=1, null(65)=0, distinct(66)=0.996222107, null(66)=0] + │ │ │ │ │ ├── fd: ()-->(5,66), (11)-->(14), (36)-->(37,40), (46)-->(49), (55)-->(57), (57)==(65), (65)==(57), (49)==(55), (55)==(49), (37)==(46), (46)==(37), (19)==(36), (36)==(19), (11)==(21), (21)==(11), (1)==(20), (20)==(1) │ │ │ │ │ ├── scan supplier@s_nk │ │ │ │ │ │ ├── save-table-name: q8_scan_8 - │ │ │ │ │ │ ├── columns: s_suppkey:10(int!null) s_nationkey:13(int!null) - │ │ │ │ │ │ ├── stats: [rows=10000, distinct(10)=9920, null(10)=0, distinct(13)=25, null(13)=0] - │ │ │ │ │ │ │ histogram(10)= 0 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 + │ │ │ │ │ │ ├── columns: s_suppkey:11(int!null) s_nationkey:14(int!null) + │ │ │ │ │ │ ├── stats: [rows=10000, distinct(11)=9920, null(11)=0, distinct(14)=25, null(14)=0] + │ │ │ │ │ │ │ histogram(11)= 0 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 │ │ │ │ │ │ │ <--- 1 ---- 51 ---- 101 ---- 151 ---- 201 ---- 251 ---- 301 ---- 351 ---- 401 ---- 451 ---- 501 ---- 551 ---- 601 ---- 651 ---- 701 ---- 751 ---- 801 ---- 851 ---- 901 ---- 951 ---- 1001 ---- 1051 ---- 1101 ---- 1151 ---- 1201 ---- 1251 ---- 1301 ---- 1351 ---- 1401 ---- 1451 ---- 1501 ---- 1551 ---- 1601 ---- 1651 ---- 1701 ---- 1751 ---- 1801 ---- 1851 ---- 1901 ---- 1951 ---- 2001 ---- 2051 ---- 2101 ---- 2151 ---- 2201 ---- 2251 ---- 2301 ---- 2351 ---- 2401 ---- 2451 ---- 2501 ---- 2551 ---- 2601 ---- 2651 ---- 2701 ---- 2751 ---- 2801 ---- 2851 ---- 2901 ---- 2951 ---- 3001 ---- 3051 ---- 3101 ---- 3151 ---- 3201 ---- 3251 ---- 3301 ---- 3351 ---- 3401 ---- 3451 ---- 3501 ---- 3551 ---- 3601 ---- 3651 ---- 3701 ---- 3751 ---- 3801 ---- 3851 ---- 3901 ---- 3951 ---- 4001 ---- 4051 ---- 4101 ---- 4151 ---- 4201 ---- 4251 ---- 4301 ---- 4351 ---- 4401 ---- 4451 ---- 4501 ---- 4551 ---- 4601 ---- 4651 ---- 4701 ---- 4751 ---- 4801 ---- 4851 ---- 4901 ---- 4951 ---- 5001 ---- 5051 ---- 5101 ---- 5151 ---- 5201 ---- 5251 ---- 5301 ---- 5351 ---- 5401 ---- 5451 ---- 5501 ---- 5551 ---- 5601 ---- 5651 ---- 5701 ---- 5751 ---- 5801 ---- 5851 ---- 5901 ---- 5951 ---- 6001 ---- 6051 ---- 6101 ---- 6151 ---- 6201 ---- 6251 ---- 6301 ---- 6351 ---- 6401 ---- 6451 ---- 6501 ---- 6551 ---- 6601 ---- 6651 ---- 6701 ---- 6751 ---- 6801 ---- 6851 ---- 6901 ---- 6951 ---- 7001 ---- 7051 ---- 7101 ---- 7151 ---- 7201 ---- 7251 ---- 7301 ---- 7351 ---- 7401 ---- 7451 ---- 7501 ---- 7552 ---- 7603 ---- 7654 ---- 7705 ---- 7756 ---- 7807 ---- 7858 ---- 7909 ---- 7960 ---- 8011 ---- 8062 ---- 8113 ---- 8164 ---- 8215 ---- 8266 ---- 8317 ---- 8368 ---- 8419 ---- 8470 ---- 8521 ---- 8572 ---- 8623 ---- 8674 ---- 8725 ---- 8776 ---- 8827 ---- 8878 ---- 8929 ---- 8980 ---- 9031 ---- 9082 ---- 9133 ---- 9184 ---- 9235 ---- 9286 ---- 9337 ---- 9388 ---- 9439 ---- 9490 ---- 9541 ---- 9592 ---- 9643 ---- 9694 ---- 9745 ---- 9796 ---- 9847 ---- 9898 ---- 9949 ---- 10000 - │ │ │ │ │ │ │ histogram(13)= 0 420 0 413 0 397 0 412 0 415 0 380 0 402 0 396 0 415 0 405 0 393 0 438 0 377 0 362 0 376 0 373 0 406 0 421 0 407 0 398 0 411 0 399 0 401 0 390 0 393 + │ │ │ │ │ │ │ histogram(14)= 0 420 0 413 0 397 0 412 0 415 0 380 0 402 0 396 0 415 0 405 0 393 0 438 0 377 0 362 0 376 0 373 0 406 0 421 0 407 0 398 0 411 0 399 0 401 0 390 0 393 │ │ │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 --- 5 --- 6 --- 7 --- 8 --- 9 --- 10 --- 11 --- 12 --- 13 --- 14 --- 15 --- 16 --- 17 --- 18 --- 19 --- 20 --- 21 --- 22 --- 23 --- 24 - │ │ │ │ │ │ ├── key: (10) - │ │ │ │ │ │ └── fd: (10)-->(13) + │ │ │ │ │ │ ├── key: (11) + │ │ │ │ │ │ └── fd: (11)-->(14) │ │ │ │ │ ├── inner-join (hash) │ │ │ │ │ │ ├── save-table-name: q8_inner_join_9 - │ │ │ │ │ │ ├── columns: p_partkey:1(int!null) p_type:5(varchar!null) l_orderkey:17(int!null) l_partkey:18(int!null) l_suppkey:19(int!null) l_extendedprice:22(float!null) l_discount:23(float!null) o_orderkey:33(int!null) o_custkey:34(int!null) o_orderdate:37(date!null) c_custkey:42(int!null) c_nationkey:45(int!null) n1.n_nationkey:50(int!null) n1.n_regionkey:52(int!null) r_regionkey:58(int!null) r_name:59(char!null) + │ │ │ │ │ │ ├── columns: p_partkey:1(int!null) p_type:5(varchar!null) l_orderkey:19(int!null) l_partkey:20(int!null) l_suppkey:21(int!null) l_extendedprice:24(float!null) l_discount:25(float!null) o_orderkey:36(int!null) o_custkey:37(int!null) o_orderdate:40(date!null) c_custkey:46(int!null) c_nationkey:49(int!null) n1.n_nationkey:55(int!null) n1.n_regionkey:57(int!null) r_regionkey:65(int!null) r_name:66(char!null) │ │ │ │ │ │ ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) - │ │ │ │ │ │ ├── stats: [rows=3865.67953, distinct(1)=1333.31636, null(1)=0, distinct(5)=1, null(5)=0, distinct(17)=3803.27383, null(17)=0, distinct(18)=1333.31636, null(18)=0, distinct(19)=3201.48978, null(19)=0, distinct(22)=3847.46234, null(22)=0, distinct(23)=11, null(23)=0, distinct(33)=3803.27383, null(33)=0, distinct(34)=3607.82676, null(34)=0, distinct(37)=727.307913, null(37)=0, distinct(42)=3607.82676, null(42)=0, distinct(45)=5, null(45)=0, distinct(50)=5, null(50)=0, distinct(52)=1, null(52)=0, distinct(58)=1, null(58)=0, distinct(59)=0.996222107, null(59)=0] - │ │ │ │ │ │ ├── fd: ()-->(5,59), (33)-->(34,37), (42)-->(45), (50)-->(52), (52)==(58), (58)==(52), (45)==(50), (50)==(45), (34)==(42), (42)==(34), (17)==(33), (33)==(17), (1)==(18), (18)==(1) + │ │ │ │ │ │ ├── stats: [rows=3865.67953, distinct(1)=1333.31636, null(1)=0, distinct(5)=1, null(5)=0, distinct(19)=3803.27383, null(19)=0, distinct(20)=1333.31636, null(20)=0, distinct(21)=3201.48978, null(21)=0, distinct(24)=3847.46234, null(24)=0, distinct(25)=11, null(25)=0, distinct(36)=3803.27383, null(36)=0, distinct(37)=3607.82676, null(37)=0, distinct(40)=727.307913, null(40)=0, distinct(46)=3607.82676, null(46)=0, distinct(49)=5, null(49)=0, distinct(55)=5, null(55)=0, distinct(57)=1, null(57)=0, distinct(65)=1, null(65)=0, distinct(66)=0.996222107, null(66)=0] + │ │ │ │ │ │ ├── fd: ()-->(5,66), (36)-->(37,40), (46)-->(49), (55)-->(57), (57)==(65), (65)==(57), (49)==(55), (55)==(49), (37)==(46), (46)==(37), (19)==(36), (36)==(19), (1)==(20), (20)==(1) │ │ │ │ │ │ ├── inner-join (lookup customer@c_nk) │ │ │ │ │ │ │ ├── save-table-name: q8_lookup_join_10 - │ │ │ │ │ │ │ ├── columns: c_custkey:42(int!null) c_nationkey:45(int!null) n1.n_nationkey:50(int!null) n1.n_regionkey:52(int!null) r_regionkey:58(int!null) r_name:59(char!null) - │ │ │ │ │ │ │ ├── key columns: [50] = [45] - │ │ │ │ │ │ │ ├── stats: [rows=30000, distinct(42)=27672.3293, null(42)=0, distinct(45)=5, null(45)=0, distinct(50)=5, null(50)=0, distinct(52)=1, null(52)=0, distinct(58)=1, null(58)=0, distinct(59)=0.996222107, null(59)=0] - │ │ │ │ │ │ │ ├── key: (42) - │ │ │ │ │ │ │ ├── fd: ()-->(59), (42)-->(45), (50)-->(52), (52)==(58), (58)==(52), (45)==(50), (50)==(45) + │ │ │ │ │ │ │ ├── columns: c_custkey:46(int!null) c_nationkey:49(int!null) n1.n_nationkey:55(int!null) n1.n_regionkey:57(int!null) r_regionkey:65(int!null) r_name:66(char!null) + │ │ │ │ │ │ │ ├── key columns: [55] = [49] + │ │ │ │ │ │ │ ├── stats: [rows=30000, distinct(46)=27672.3293, null(46)=0, distinct(49)=5, null(49)=0, distinct(55)=5, null(55)=0, distinct(57)=1, null(57)=0, distinct(65)=1, null(65)=0, distinct(66)=0.996222107, null(66)=0] + │ │ │ │ │ │ │ ├── key: (46) + │ │ │ │ │ │ │ ├── fd: ()-->(66), (46)-->(49), (55)-->(57), (57)==(65), (65)==(57), (49)==(55), (55)==(49) │ │ │ │ │ │ │ ├── inner-join (merge) │ │ │ │ │ │ │ │ ├── save-table-name: q8_merge_join_11 - │ │ │ │ │ │ │ │ ├── columns: n1.n_nationkey:50(int!null) n1.n_regionkey:52(int!null) r_regionkey:58(int!null) r_name:59(char!null) - │ │ │ │ │ │ │ │ ├── left ordering: +52 - │ │ │ │ │ │ │ │ ├── right ordering: +58 - │ │ │ │ │ │ │ │ ├── stats: [rows=5, distinct(50)=5, null(50)=0, distinct(52)=1, null(52)=0, distinct(58)=1, null(58)=0, distinct(59)=0.996222107, null(59)=0] - │ │ │ │ │ │ │ │ ├── key: (50) - │ │ │ │ │ │ │ │ ├── fd: ()-->(59), (50)-->(52), (52)==(58), (58)==(52) + │ │ │ │ │ │ │ │ ├── columns: n1.n_nationkey:55(int!null) n1.n_regionkey:57(int!null) r_regionkey:65(int!null) r_name:66(char!null) + │ │ │ │ │ │ │ │ ├── left ordering: +57 + │ │ │ │ │ │ │ │ ├── right ordering: +65 + │ │ │ │ │ │ │ │ ├── stats: [rows=5, distinct(55)=5, null(55)=0, distinct(57)=1, null(57)=0, distinct(65)=1, null(65)=0, distinct(66)=0.996222107, null(66)=0] + │ │ │ │ │ │ │ │ ├── key: (55) + │ │ │ │ │ │ │ │ ├── fd: ()-->(66), (55)-->(57), (57)==(65), (65)==(57) │ │ │ │ │ │ │ │ ├── scan n1@n_rk │ │ │ │ │ │ │ │ │ ├── save-table-name: q8_scan_12 - │ │ │ │ │ │ │ │ │ ├── columns: n1.n_nationkey:50(int!null) n1.n_regionkey:52(int!null) - │ │ │ │ │ │ │ │ │ ├── stats: [rows=25, distinct(50)=25, null(50)=0, distinct(52)=5, null(52)=0] - │ │ │ │ │ │ │ │ │ │ histogram(50)= 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 + │ │ │ │ │ │ │ │ │ ├── columns: n1.n_nationkey:55(int!null) n1.n_regionkey:57(int!null) + │ │ │ │ │ │ │ │ │ ├── stats: [rows=25, distinct(55)=25, null(55)=0, distinct(57)=5, null(57)=0] + │ │ │ │ │ │ │ │ │ │ histogram(55)= 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 │ │ │ │ │ │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 --- 5 --- 6 --- 7 --- 8 --- 9 --- 10 --- 11 --- 12 --- 13 --- 14 --- 15 --- 16 --- 17 --- 18 --- 19 --- 20 --- 21 --- 22 --- 23 --- 24 - │ │ │ │ │ │ │ │ │ │ histogram(52)= 0 5 0 5 0 5 0 5 0 5 + │ │ │ │ │ │ │ │ │ │ histogram(57)= 0 5 0 5 0 5 0 5 0 5 │ │ │ │ │ │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 - │ │ │ │ │ │ │ │ │ ├── key: (50) - │ │ │ │ │ │ │ │ │ ├── fd: (50)-->(52) - │ │ │ │ │ │ │ │ │ └── ordering: +52 + │ │ │ │ │ │ │ │ │ ├── key: (55) + │ │ │ │ │ │ │ │ │ ├── fd: (55)-->(57) + │ │ │ │ │ │ │ │ │ └── ordering: +57 │ │ │ │ │ │ │ │ ├── select │ │ │ │ │ │ │ │ │ ├── save-table-name: q8_select_13 - │ │ │ │ │ │ │ │ │ ├── columns: r_regionkey:58(int!null) r_name:59(char!null) - │ │ │ │ │ │ │ │ │ ├── stats: [rows=1, distinct(58)=1, null(58)=0, distinct(59)=1, null(59)=0] - │ │ │ │ │ │ │ │ │ ├── key: (58) - │ │ │ │ │ │ │ │ │ ├── fd: ()-->(59) - │ │ │ │ │ │ │ │ │ ├── ordering: +58 opt(59) [actual: +58] + │ │ │ │ │ │ │ │ │ ├── columns: r_regionkey:65(int!null) r_name:66(char!null) + │ │ │ │ │ │ │ │ │ ├── stats: [rows=1, distinct(65)=1, null(65)=0, distinct(66)=1, null(66)=0] + │ │ │ │ │ │ │ │ │ ├── key: (65) + │ │ │ │ │ │ │ │ │ ├── fd: ()-->(66) + │ │ │ │ │ │ │ │ │ ├── ordering: +65 opt(66) [actual: +65] │ │ │ │ │ │ │ │ │ ├── scan region │ │ │ │ │ │ │ │ │ │ ├── save-table-name: q8_scan_14 - │ │ │ │ │ │ │ │ │ │ ├── columns: r_regionkey:58(int!null) r_name:59(char!null) - │ │ │ │ │ │ │ │ │ │ ├── stats: [rows=5, distinct(58)=5, null(58)=0, distinct(59)=5, null(59)=0] - │ │ │ │ │ │ │ │ │ │ │ histogram(58)= 0 1 0 1 0 1 0 1 0 1 + │ │ │ │ │ │ │ │ │ │ ├── columns: r_regionkey:65(int!null) r_name:66(char!null) + │ │ │ │ │ │ │ │ │ │ ├── stats: [rows=5, distinct(65)=5, null(65)=0, distinct(66)=5, null(66)=0] + │ │ │ │ │ │ │ │ │ │ │ histogram(65)= 0 1 0 1 0 1 0 1 0 1 │ │ │ │ │ │ │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 - │ │ │ │ │ │ │ │ │ │ ├── key: (58) - │ │ │ │ │ │ │ │ │ │ ├── fd: (58)-->(59) - │ │ │ │ │ │ │ │ │ │ └── ordering: +58 opt(59) [actual: +58] + │ │ │ │ │ │ │ │ │ │ ├── key: (65) + │ │ │ │ │ │ │ │ │ │ ├── fd: (65)-->(66) + │ │ │ │ │ │ │ │ │ │ └── ordering: +65 opt(66) [actual: +65] │ │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ │ └── r_name:59 = 'AMERICA' [type=bool, outer=(59), constraints=(/59: [/'AMERICA' - /'AMERICA']; tight), fd=()-->(59)] + │ │ │ │ │ │ │ │ │ └── r_name:66 = 'AMERICA' [type=bool, outer=(66), constraints=(/66: [/'AMERICA' - /'AMERICA']; tight), fd=()-->(66)] │ │ │ │ │ │ │ │ └── filters (true) │ │ │ │ │ │ │ └── filters (true) │ │ │ │ │ │ ├── inner-join (lookup orders) │ │ │ │ │ │ │ ├── save-table-name: q8_lookup_join_15 - │ │ │ │ │ │ │ ├── columns: p_partkey:1(int!null) p_type:5(varchar!null) l_orderkey:17(int!null) l_partkey:18(int!null) l_suppkey:19(int!null) l_extendedprice:22(float!null) l_discount:23(float!null) o_orderkey:33(int!null) o_custkey:34(int!null) o_orderdate:37(date!null) - │ │ │ │ │ │ │ ├── key columns: [17] = [33] + │ │ │ │ │ │ │ ├── columns: p_partkey:1(int!null) p_type:5(varchar!null) l_orderkey:19(int!null) l_partkey:20(int!null) l_suppkey:21(int!null) l_extendedprice:24(float!null) l_discount:25(float!null) o_orderkey:36(int!null) o_custkey:37(int!null) o_orderdate:40(date!null) + │ │ │ │ │ │ │ ├── key columns: [19] = [36] │ │ │ │ │ │ │ ├── lookup columns are key - │ │ │ │ │ │ │ ├── stats: [rows=11954.404, distinct(1)=1333.31636, null(1)=0, distinct(5)=1, null(5)=0, distinct(17)=11798.6079, null(17)=0, distinct(18)=1333.31636, null(18)=0, distinct(19)=6947.30529, null(19)=0, distinct(22)=11864.6001, null(22)=0, distinct(23)=11, null(23)=0, distinct(33)=11798.6079, null(33)=0, distinct(34)=11263.5878, null(34)=0, distinct(37)=730.999942, null(37)=0] - │ │ │ │ │ │ │ ├── fd: ()-->(5), (33)-->(34,37), (17)==(33), (33)==(17), (1)==(18), (18)==(1) + │ │ │ │ │ │ │ ├── stats: [rows=11954.404, distinct(1)=1333.31636, null(1)=0, distinct(5)=1, null(5)=0, distinct(19)=11798.6079, null(19)=0, distinct(20)=1333.31636, null(20)=0, distinct(21)=6947.30529, null(21)=0, distinct(24)=11864.6001, null(24)=0, distinct(25)=11, null(25)=0, distinct(36)=11798.6079, null(36)=0, distinct(37)=11263.5878, null(37)=0, distinct(40)=730.999942, null(40)=0] + │ │ │ │ │ │ │ ├── fd: ()-->(5), (36)-->(37,40), (19)==(36), (36)==(19), (1)==(20), (20)==(1) │ │ │ │ │ │ │ ├── inner-join (lookup lineitem) │ │ │ │ │ │ │ │ ├── save-table-name: q8_lookup_join_16 - │ │ │ │ │ │ │ │ ├── columns: p_partkey:1(int!null) p_type:5(varchar!null) l_orderkey:17(int!null) l_partkey:18(int!null) l_suppkey:19(int!null) l_extendedprice:22(float!null) l_discount:23(float!null) - │ │ │ │ │ │ │ │ ├── key columns: [17 20] = [17 20] + │ │ │ │ │ │ │ │ ├── columns: p_partkey:1(int!null) p_type:5(varchar!null) l_orderkey:19(int!null) l_partkey:20(int!null) l_suppkey:21(int!null) l_extendedprice:24(float!null) l_discount:25(float!null) + │ │ │ │ │ │ │ │ ├── key columns: [19 22] = [19 22] │ │ │ │ │ │ │ │ ├── lookup columns are key - │ │ │ │ │ │ │ │ ├── stats: [rows=40160.5091, distinct(1)=1333.31636, null(1)=0, distinct(5)=1, null(5)=0, distinct(17)=39637.1826, null(17)=0, distinct(18)=1333.31636, null(18)=0, distinct(19)=9746.9017, null(19)=0, distinct(22)=39302.1409, null(22)=0, distinct(23)=11, null(23)=0] - │ │ │ │ │ │ │ │ ├── fd: ()-->(5), (1)==(18), (18)==(1) + │ │ │ │ │ │ │ │ ├── stats: [rows=40160.5091, distinct(1)=1333.31636, null(1)=0, distinct(5)=1, null(5)=0, distinct(19)=39637.1826, null(19)=0, distinct(20)=1333.31636, null(20)=0, distinct(21)=9746.9017, null(21)=0, distinct(24)=39302.1409, null(24)=0, distinct(25)=11, null(25)=0] + │ │ │ │ │ │ │ │ ├── fd: ()-->(5), (1)==(20), (20)==(1) │ │ │ │ │ │ │ │ ├── inner-join (lookup lineitem@l_pk) │ │ │ │ │ │ │ │ │ ├── save-table-name: q8_lookup_join_17 - │ │ │ │ │ │ │ │ │ ├── columns: p_partkey:1(int!null) p_type:5(varchar!null) l_orderkey:17(int!null) l_partkey:18(int!null) l_linenumber:20(int!null) - │ │ │ │ │ │ │ │ │ ├── key columns: [1] = [18] - │ │ │ │ │ │ │ │ │ ├── stats: [rows=40160.5091, distinct(1)=1333.31636, null(1)=0, distinct(5)=1, null(5)=0, distinct(17)=39637.1826, null(17)=0, distinct(18)=1333.31636, null(18)=0, distinct(20)=7, null(20)=0] - │ │ │ │ │ │ │ │ │ ├── key: (17,20) - │ │ │ │ │ │ │ │ │ ├── fd: ()-->(5), (17,20)-->(18), (1)==(18), (18)==(1) + │ │ │ │ │ │ │ │ │ ├── columns: p_partkey:1(int!null) p_type:5(varchar!null) l_orderkey:19(int!null) l_partkey:20(int!null) l_linenumber:22(int!null) + │ │ │ │ │ │ │ │ │ ├── key columns: [1] = [20] + │ │ │ │ │ │ │ │ │ ├── stats: [rows=40160.5091, distinct(1)=1333.31636, null(1)=0, distinct(5)=1, null(5)=0, distinct(19)=39637.1826, null(19)=0, distinct(20)=1333.31636, null(20)=0, distinct(22)=7, null(22)=0] + │ │ │ │ │ │ │ │ │ ├── key: (19,22) + │ │ │ │ │ │ │ │ │ ├── fd: ()-->(5), (19,22)-->(20), (1)==(20), (20)==(1) │ │ │ │ │ │ │ │ │ ├── select │ │ │ │ │ │ │ │ │ │ ├── save-table-name: q8_select_18 │ │ │ │ │ │ │ │ │ │ ├── columns: p_partkey:1(int!null) p_type:5(varchar!null) @@ -202,33 +202,33 @@ sort │ │ │ │ │ │ │ │ │ └── filters (true) │ │ │ │ │ │ │ │ └── filters (true) │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── (o_orderdate:37 >= '1995-01-01') AND (o_orderdate:37 <= '1996-12-31') [type=bool, outer=(37), constraints=(/37: [/'1995-01-01' - /'1996-12-31']; tight)] + │ │ │ │ │ │ │ └── (o_orderdate:40 >= '1995-01-01') AND (o_orderdate:40 <= '1996-12-31') [type=bool, outer=(40), constraints=(/40: [/'1995-01-01' - /'1996-12-31']; tight)] │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── o_custkey:34 = c_custkey:42 [type=bool, outer=(34,42), constraints=(/34: (/NULL - ]; /42: (/NULL - ]), fd=(34)==(42), (42)==(34)] + │ │ │ │ │ │ └── o_custkey:37 = c_custkey:46 [type=bool, outer=(37,46), constraints=(/37: (/NULL - ]; /46: (/NULL - ]), fd=(37)==(46), (46)==(37)] │ │ │ │ │ └── filters - │ │ │ │ │ └── s_suppkey:10 = l_suppkey:19 [type=bool, outer=(10,19), constraints=(/10: (/NULL - ]; /19: (/NULL - ]), fd=(10)==(19), (19)==(10)] + │ │ │ │ │ └── s_suppkey:11 = l_suppkey:21 [type=bool, outer=(11,21), constraints=(/11: (/NULL - ]; /21: (/NULL - ]), fd=(11)==(21), (21)==(11)] │ │ │ │ ├── scan n2 │ │ │ │ │ ├── save-table-name: q8_scan_20 - │ │ │ │ │ ├── columns: n2.n_nationkey:54(int!null) n2.n_name:55(char!null) - │ │ │ │ │ ├── stats: [rows=25, distinct(54)=25, null(54)=0, distinct(55)=25, null(55)=0] - │ │ │ │ │ │ histogram(54)= 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 + │ │ │ │ │ ├── columns: n2.n_nationkey:60(int!null) n2.n_name:61(char!null) + │ │ │ │ │ ├── stats: [rows=25, distinct(60)=25, null(60)=0, distinct(61)=25, null(61)=0] + │ │ │ │ │ │ histogram(60)= 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 │ │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 --- 5 --- 6 --- 7 --- 8 --- 9 --- 10 --- 11 --- 12 --- 13 --- 14 --- 15 --- 16 --- 17 --- 18 --- 19 --- 20 --- 21 --- 22 --- 23 --- 24 - │ │ │ │ │ ├── key: (54) - │ │ │ │ │ └── fd: (54)-->(55) + │ │ │ │ │ ├── key: (60) + │ │ │ │ │ └── fd: (60)-->(61) │ │ │ │ └── filters - │ │ │ │ └── s_nationkey:13 = n2.n_nationkey:54 [type=bool, outer=(13,54), constraints=(/13: (/NULL - ]; /54: (/NULL - ]), fd=(13)==(54), (54)==(13)] + │ │ │ │ └── s_nationkey:14 = n2.n_nationkey:60 [type=bool, outer=(14,60), constraints=(/14: (/NULL - ]; /60: (/NULL - ]), fd=(14)==(60), (60)==(14)] │ │ │ └── projections - │ │ │ ├── extract('year', o_orderdate:37) [as=o_year:61, type=float, outer=(37), immutable] - │ │ │ └── l_extendedprice:22 * (1.0 - l_discount:23) [as=volume:62, type=float, outer=(22,23), immutable] + │ │ │ ├── extract('year', o_orderdate:40) [as=o_year:69, type=float, outer=(40), immutable] + │ │ │ └── l_extendedprice:24 * (1.0 - l_discount:25) [as=volume:70, type=float, outer=(24,25), immutable] │ │ └── projections - │ │ └── CASE WHEN n2.n_name:55 = 'BRAZIL' THEN volume:62 ELSE 0.0 END [as=column63:63, type=float, outer=(55,62)] + │ │ └── CASE WHEN n2.n_name:61 = 'BRAZIL' THEN volume:70 ELSE 0.0 END [as=column71:71, type=float, outer=(61,70)] │ └── aggregations - │ ├── sum [as=sum:64, type=float, outer=(63)] - │ │ └── column63:63 [type=float] - │ └── sum [as=sum:65, type=float, outer=(62)] - │ └── volume:62 [type=float] + │ ├── sum [as=sum:72, type=float, outer=(71)] + │ │ └── column71:71 [type=float] + │ └── sum [as=sum:73, type=float, outer=(70)] + │ └── volume:70 [type=float] └── projections - └── sum:64 / sum:65 [as=mkt_share:66, type=float, outer=(64,65), immutable] + └── sum:72 / sum:73 [as=mkt_share:74, type=float, outer=(72,73), immutable] stats table=q8_sort_1 ---- @@ -265,12 +265,12 @@ column_names row_count_est row_count_err distinct_count_est distinct_count_e stats table=q8_project_4 ---- column_names row_count distinct_count null_count -{column63} 2603 108 0 +{column71} 2603 108 0 {o_year} 2603 2 0 {volume} 2603 2599 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{column63} 4302.00 1.65 4285.00 39.68 <== 0.00 1.00 +{column71} 4302.00 1.65 4285.00 39.68 <== 0.00 1.00 {o_year} 4302.00 1.65 729.00 364.50 <== 0.00 1.00 {volume} 4302.00 1.65 4268.00 1.64 0.00 1.00 diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q09 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q09 index 6ee8848b292b..3794ee9de16f 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q09 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q09 @@ -55,76 +55,76 @@ ORDER BY ---- sort ├── save-table-name: q9_sort_1 - ├── columns: nation:48(char!null) o_year:51(float) sum_profit:53(float!null) + ├── columns: nation:53(char!null) o_year:57(float) sum_profit:59(float!null) ├── immutable - ├── stats: [rows=1548.49892, distinct(48)=25, null(48)=0, distinct(51)=957.145308, null(51)=0, distinct(53)=1548.49892, null(53)=0, distinct(48,51)=1548.49892, null(48,51)=0] - ├── key: (48,51) - ├── fd: (48,51)-->(53) - ├── ordering: +48,-51 + ├── stats: [rows=1548.49892, distinct(53)=25, null(53)=0, distinct(57)=957.145308, null(57)=0, distinct(59)=1548.49892, null(59)=0, distinct(53,57)=1548.49892, null(53,57)=0] + ├── key: (53,57) + ├── fd: (53,57)-->(59) + ├── ordering: +53,-57 └── group-by ├── save-table-name: q9_group_by_2 - ├── columns: n_name:48(char!null) o_year:51(float) sum:53(float!null) - ├── grouping columns: n_name:48(char!null) o_year:51(float) + ├── columns: n_name:53(char!null) o_year:57(float) sum:59(float!null) + ├── grouping columns: n_name:53(char!null) o_year:57(float) ├── immutable - ├── stats: [rows=1548.49892, distinct(48)=25, null(48)=0, distinct(51)=957.145308, null(51)=0, distinct(53)=1548.49892, null(53)=0, distinct(48,51)=1548.49892, null(48,51)=0] - ├── key: (48,51) - ├── fd: (48,51)-->(53) + ├── stats: [rows=1548.49892, distinct(53)=25, null(53)=0, distinct(57)=957.145308, null(57)=0, distinct(59)=1548.49892, null(59)=0, distinct(53,57)=1548.49892, null(53,57)=0] + ├── key: (53,57) + ├── fd: (53,57)-->(59) ├── project │ ├── save-table-name: q9_project_3 - │ ├── columns: o_year:51(float) amount:52(float!null) n_name:48(char!null) + │ ├── columns: o_year:57(float) amount:58(float!null) n_name:53(char!null) │ ├── immutable - │ ├── stats: [rows=2450.41631, distinct(48)=25, null(48)=0, distinct(51)=957.145308, null(51)=0, distinct(52)=958.326364, null(52)=0, distinct(48,51)=1548.49892, null(48,51)=0] + │ ├── stats: [rows=2450.41631, distinct(53)=25, null(53)=0, distinct(57)=957.145308, null(57)=0, distinct(58)=958.326364, null(58)=0, distinct(53,57)=1548.49892, null(53,57)=0] │ ├── inner-join (hash) │ │ ├── save-table-name: q9_inner_join_4 - │ │ ├── columns: p_partkey:1(int!null) p_name:2(varchar!null) s_suppkey:10(int!null) s_nationkey:13(int!null) l_orderkey:17(int!null) l_partkey:18(int!null) l_suppkey:19(int!null) l_quantity:21(float!null) l_extendedprice:22(float!null) l_discount:23(float!null) ps_partkey:33(int!null) ps_suppkey:34(int!null) ps_supplycost:36(float!null) o_orderkey:38(int!null) o_orderdate:42(date!null) n_nationkey:47(int!null) n_name:48(char!null) + │ │ ├── columns: p_partkey:1(int!null) p_name:2(varchar!null) s_suppkey:11(int!null) s_nationkey:14(int!null) l_orderkey:19(int!null) l_partkey:20(int!null) l_suppkey:21(int!null) l_quantity:23(float!null) l_extendedprice:24(float!null) l_discount:25(float!null) ps_partkey:36(int!null) ps_suppkey:37(int!null) ps_supplycost:39(float!null) o_orderkey:42(int!null) o_orderdate:46(date!null) n_nationkey:52(int!null) n_name:53(char!null) │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ │ ├── stats: [rows=2450.41631, distinct(1)=1065.01249, null(1)=0, distinct(2)=2405.94672, null(2)=0, distinct(10)=1064.52386, null(10)=0, distinct(13)=25, null(13)=0, distinct(17)=1066.4323, null(17)=0, distinct(18)=1065.01249, null(18)=0, distinct(19)=1064.52386, null(19)=0, distinct(21)=50, null(21)=0, distinct(22)=958.164832, null(22)=0, distinct(23)=11, null(23)=0, distinct(33)=958.326364, null(33)=0, distinct(34)=958.326364, null(34)=0, distinct(36)=956.829183, null(36)=0, distinct(38)=1066.4323, null(38)=0, distinct(42)=957.145308, null(42)=0, distinct(47)=25, null(47)=0, distinct(48)=25, null(48)=0, distinct(42,48)=1548.49892, null(42,48)=0, distinct(21-23,36)=958.326364, null(21-23,36)=0] - │ │ ├── fd: (1)-->(2), (10)-->(13), (33,34)-->(36), (19)==(10,34), (34)==(10,19), (18)==(1,33), (33)==(1,18), (38)-->(42), (17)==(38), (38)==(17), (10)==(19,34), (47)-->(48), (13)==(47), (47)==(13), (1)==(18,33) + │ │ ├── stats: [rows=2450.41631, distinct(1)=1065.01249, null(1)=0, distinct(2)=2405.94672, null(2)=0, distinct(11)=1064.52386, null(11)=0, distinct(14)=25, null(14)=0, distinct(19)=1066.4323, null(19)=0, distinct(20)=1065.01249, null(20)=0, distinct(21)=1064.52386, null(21)=0, distinct(23)=50, null(23)=0, distinct(24)=958.164832, null(24)=0, distinct(25)=11, null(25)=0, distinct(36)=958.326364, null(36)=0, distinct(37)=958.326364, null(37)=0, distinct(39)=956.829183, null(39)=0, distinct(42)=1066.4323, null(42)=0, distinct(46)=957.145308, null(46)=0, distinct(52)=25, null(52)=0, distinct(53)=25, null(53)=0, distinct(46,53)=1548.49892, null(46,53)=0, distinct(23-25,39)=958.326364, null(23-25,39)=0] + │ │ ├── fd: (1)-->(2), (11)-->(14), (36,37)-->(39), (21)==(11,37), (37)==(11,21), (20)==(1,36), (36)==(1,20), (42)-->(46), (19)==(42), (42)==(19), (11)==(21,37), (52)-->(53), (14)==(52), (52)==(14), (1)==(20,36) │ │ ├── inner-join (hash) │ │ │ ├── save-table-name: q9_inner_join_5 - │ │ │ ├── columns: p_partkey:1(int!null) p_name:2(varchar!null) s_suppkey:10(int!null) s_nationkey:13(int!null) l_orderkey:17(int!null) l_partkey:18(int!null) l_suppkey:19(int!null) l_quantity:21(float!null) l_extendedprice:22(float!null) l_discount:23(float!null) ps_partkey:33(int!null) ps_suppkey:34(int!null) ps_supplycost:36(float!null) o_orderkey:38(int!null) o_orderdate:42(date!null) + │ │ │ ├── columns: p_partkey:1(int!null) p_name:2(varchar!null) s_suppkey:11(int!null) s_nationkey:14(int!null) l_orderkey:19(int!null) l_partkey:20(int!null) l_suppkey:21(int!null) l_quantity:23(float!null) l_extendedprice:24(float!null) l_discount:25(float!null) ps_partkey:36(int!null) ps_suppkey:37(int!null) ps_supplycost:39(float!null) o_orderkey:42(int!null) o_orderdate:46(date!null) │ │ │ ├── multiplicity: left-rows(zero-or-more), right-rows(exactly-one) - │ │ │ ├── stats: [rows=2450.41631, distinct(1)=1223.84511, null(1)=0, distinct(2)=2405.94672, null(2)=0, distinct(10)=1224.18162, null(10)=0, distinct(13)=25, null(13)=0, distinct(17)=1227.35103, null(17)=0, distinct(18)=1223.84511, null(18)=0, distinct(19)=1224.18162, null(19)=0, distinct(21)=50, null(21)=0, distinct(22)=1058.34962, null(22)=0, distinct(23)=11, null(23)=0, distinct(33)=1058.5867, null(33)=0, distinct(34)=1058.5867, null(34)=0, distinct(36)=1056.39091, null(36)=0, distinct(38)=1227.35103, null(38)=0, distinct(42)=1056.85425, null(42)=0] - │ │ │ ├── fd: (1)-->(2), (10)-->(13), (33,34)-->(36), (19)==(10,34), (34)==(10,19), (18)==(1,33), (33)==(1,18), (38)-->(42), (17)==(38), (38)==(17), (10)==(19,34), (1)==(18,33) + │ │ │ ├── stats: [rows=2450.41631, distinct(1)=1223.84511, null(1)=0, distinct(2)=2405.94672, null(2)=0, distinct(11)=1224.18162, null(11)=0, distinct(14)=25, null(14)=0, distinct(19)=1227.35103, null(19)=0, distinct(20)=1223.84511, null(20)=0, distinct(21)=1224.18162, null(21)=0, distinct(23)=50, null(23)=0, distinct(24)=1058.34962, null(24)=0, distinct(25)=11, null(25)=0, distinct(36)=1058.5867, null(36)=0, distinct(37)=1058.5867, null(37)=0, distinct(39)=1056.39091, null(39)=0, distinct(42)=1227.35103, null(42)=0, distinct(46)=1056.85425, null(46)=0] + │ │ │ ├── fd: (1)-->(2), (11)-->(14), (36,37)-->(39), (21)==(11,37), (37)==(11,21), (20)==(1,36), (36)==(1,20), (42)-->(46), (19)==(42), (42)==(19), (11)==(21,37), (1)==(20,36) │ │ │ ├── scan supplier@s_nk │ │ │ │ ├── save-table-name: q9_scan_6 - │ │ │ │ ├── columns: s_suppkey:10(int!null) s_nationkey:13(int!null) - │ │ │ │ ├── stats: [rows=10000, distinct(10)=9920, null(10)=0, distinct(13)=25, null(13)=0] - │ │ │ │ │ histogram(10)= 0 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 + │ │ │ │ ├── columns: s_suppkey:11(int!null) s_nationkey:14(int!null) + │ │ │ │ ├── stats: [rows=10000, distinct(11)=9920, null(11)=0, distinct(14)=25, null(14)=0] + │ │ │ │ │ histogram(11)= 0 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 │ │ │ │ │ <--- 1 ---- 51 ---- 101 ---- 151 ---- 201 ---- 251 ---- 301 ---- 351 ---- 401 ---- 451 ---- 501 ---- 551 ---- 601 ---- 651 ---- 701 ---- 751 ---- 801 ---- 851 ---- 901 ---- 951 ---- 1001 ---- 1051 ---- 1101 ---- 1151 ---- 1201 ---- 1251 ---- 1301 ---- 1351 ---- 1401 ---- 1451 ---- 1501 ---- 1551 ---- 1601 ---- 1651 ---- 1701 ---- 1751 ---- 1801 ---- 1851 ---- 1901 ---- 1951 ---- 2001 ---- 2051 ---- 2101 ---- 2151 ---- 2201 ---- 2251 ---- 2301 ---- 2351 ---- 2401 ---- 2451 ---- 2501 ---- 2551 ---- 2601 ---- 2651 ---- 2701 ---- 2751 ---- 2801 ---- 2851 ---- 2901 ---- 2951 ---- 3001 ---- 3051 ---- 3101 ---- 3151 ---- 3201 ---- 3251 ---- 3301 ---- 3351 ---- 3401 ---- 3451 ---- 3501 ---- 3551 ---- 3601 ---- 3651 ---- 3701 ---- 3751 ---- 3801 ---- 3851 ---- 3901 ---- 3951 ---- 4001 ---- 4051 ---- 4101 ---- 4151 ---- 4201 ---- 4251 ---- 4301 ---- 4351 ---- 4401 ---- 4451 ---- 4501 ---- 4551 ---- 4601 ---- 4651 ---- 4701 ---- 4751 ---- 4801 ---- 4851 ---- 4901 ---- 4951 ---- 5001 ---- 5051 ---- 5101 ---- 5151 ---- 5201 ---- 5251 ---- 5301 ---- 5351 ---- 5401 ---- 5451 ---- 5501 ---- 5551 ---- 5601 ---- 5651 ---- 5701 ---- 5751 ---- 5801 ---- 5851 ---- 5901 ---- 5951 ---- 6001 ---- 6051 ---- 6101 ---- 6151 ---- 6201 ---- 6251 ---- 6301 ---- 6351 ---- 6401 ---- 6451 ---- 6501 ---- 6551 ---- 6601 ---- 6651 ---- 6701 ---- 6751 ---- 6801 ---- 6851 ---- 6901 ---- 6951 ---- 7001 ---- 7051 ---- 7101 ---- 7151 ---- 7201 ---- 7251 ---- 7301 ---- 7351 ---- 7401 ---- 7451 ---- 7501 ---- 7552 ---- 7603 ---- 7654 ---- 7705 ---- 7756 ---- 7807 ---- 7858 ---- 7909 ---- 7960 ---- 8011 ---- 8062 ---- 8113 ---- 8164 ---- 8215 ---- 8266 ---- 8317 ---- 8368 ---- 8419 ---- 8470 ---- 8521 ---- 8572 ---- 8623 ---- 8674 ---- 8725 ---- 8776 ---- 8827 ---- 8878 ---- 8929 ---- 8980 ---- 9031 ---- 9082 ---- 9133 ---- 9184 ---- 9235 ---- 9286 ---- 9337 ---- 9388 ---- 9439 ---- 9490 ---- 9541 ---- 9592 ---- 9643 ---- 9694 ---- 9745 ---- 9796 ---- 9847 ---- 9898 ---- 9949 ---- 10000 - │ │ │ │ │ histogram(13)= 0 420 0 413 0 397 0 412 0 415 0 380 0 402 0 396 0 415 0 405 0 393 0 438 0 377 0 362 0 376 0 373 0 406 0 421 0 407 0 398 0 411 0 399 0 401 0 390 0 393 + │ │ │ │ │ histogram(14)= 0 420 0 413 0 397 0 412 0 415 0 380 0 402 0 396 0 415 0 405 0 393 0 438 0 377 0 362 0 376 0 373 0 406 0 421 0 407 0 398 0 411 0 399 0 401 0 390 0 393 │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 --- 5 --- 6 --- 7 --- 8 --- 9 --- 10 --- 11 --- 12 --- 13 --- 14 --- 15 --- 16 --- 17 --- 18 --- 19 --- 20 --- 21 --- 22 --- 23 --- 24 - │ │ │ │ ├── key: (10) - │ │ │ │ └── fd: (10)-->(13) + │ │ │ │ ├── key: (11) + │ │ │ │ └── fd: (11)-->(14) │ │ │ ├── inner-join (lookup orders) │ │ │ │ ├── save-table-name: q9_lookup_join_7 - │ │ │ │ ├── columns: p_partkey:1(int!null) p_name:2(varchar!null) l_orderkey:17(int!null) l_partkey:18(int!null) l_suppkey:19(int!null) l_quantity:21(float!null) l_extendedprice:22(float!null) l_discount:23(float!null) ps_partkey:33(int!null) ps_suppkey:34(int!null) ps_supplycost:36(float!null) o_orderkey:38(int!null) o_orderdate:42(date!null) - │ │ │ │ ├── key columns: [17] = [38] + │ │ │ │ ├── columns: p_partkey:1(int!null) p_name:2(varchar!null) l_orderkey:19(int!null) l_partkey:20(int!null) l_suppkey:21(int!null) l_quantity:23(float!null) l_extendedprice:24(float!null) l_discount:25(float!null) ps_partkey:36(int!null) ps_suppkey:37(int!null) ps_supplycost:39(float!null) o_orderkey:42(int!null) o_orderdate:46(date!null) + │ │ │ │ ├── key columns: [19] = [42] │ │ │ │ ├── lookup columns are key - │ │ │ │ ├── stats: [rows=2430.81298, distinct(1)=1535.46099, null(1)=0, distinct(2)=2387.04794, null(2)=0, distinct(17)=1535.60004, null(17)=0, distinct(18)=1535.46099, null(18)=0, distinct(19)=1220.18199, null(19)=0, distinct(21)=50, null(21)=0, distinct(22)=1219.78652, null(22)=0, distinct(23)=11, null(23)=0, distinct(33)=1220.18199, null(33)=0, distinct(34)=1220.18199, null(34)=0, distinct(36)=1216.52365, null(36)=0, distinct(38)=1535.60004, null(38)=0, distinct(42)=1217.29478, null(42)=0] - │ │ │ │ ├── fd: (1)-->(2), (33,34)-->(36), (19)==(34), (34)==(19), (18)==(1,33), (33)==(1,18), (38)-->(42), (17)==(38), (38)==(17), (1)==(18,33) + │ │ │ │ ├── stats: [rows=2430.81298, distinct(1)=1535.46099, null(1)=0, distinct(2)=2387.04794, null(2)=0, distinct(19)=1535.60004, null(19)=0, distinct(20)=1535.46099, null(20)=0, distinct(21)=1220.18199, null(21)=0, distinct(23)=50, null(23)=0, distinct(24)=1219.78652, null(24)=0, distinct(25)=11, null(25)=0, distinct(36)=1220.18199, null(36)=0, distinct(37)=1220.18199, null(37)=0, distinct(39)=1216.52365, null(39)=0, distinct(42)=1535.60004, null(42)=0, distinct(46)=1217.29478, null(46)=0] + │ │ │ │ ├── fd: (1)-->(2), (36,37)-->(39), (21)==(37), (37)==(21), (20)==(1,36), (36)==(1,20), (42)-->(46), (19)==(42), (42)==(19), (1)==(20,36) │ │ │ │ ├── inner-join (lookup lineitem) │ │ │ │ │ ├── save-table-name: q9_lookup_join_8 - │ │ │ │ │ ├── columns: p_partkey:1(int!null) p_name:2(varchar!null) l_orderkey:17(int!null) l_partkey:18(int!null) l_suppkey:19(int!null) l_quantity:21(float!null) l_extendedprice:22(float!null) l_discount:23(float!null) ps_partkey:33(int!null) ps_suppkey:34(int!null) ps_supplycost:36(float!null) - │ │ │ │ │ ├── key columns: [17 20] = [17 20] + │ │ │ │ │ ├── columns: p_partkey:1(int!null) p_name:2(varchar!null) l_orderkey:19(int!null) l_partkey:20(int!null) l_suppkey:21(int!null) l_quantity:23(float!null) l_extendedprice:24(float!null) l_discount:25(float!null) ps_partkey:36(int!null) ps_suppkey:37(int!null) ps_supplycost:39(float!null) + │ │ │ │ │ ├── key columns: [19 22] = [19 22] │ │ │ │ │ ├── lookup columns are key - │ │ │ │ │ ├── stats: [rows=2430.81298, distinct(1)=2429.06305, null(1)=0, distinct(2)=2387.04794, null(2)=0, distinct(17)=1535.60004, null(17)=0, distinct(18)=2429.06305, null(18)=0, distinct(19)=1536.11093, null(19)=0, distinct(21)=50, null(21)=0, distinct(22)=1535.26821, null(22)=0, distinct(23)=11, null(23)=0, distinct(33)=1536.11093, null(33)=0, distinct(34)=1536.11093, null(34)=0, distinct(36)=1528.33438, null(36)=0] - │ │ │ │ │ ├── fd: (1)-->(2), (33,34)-->(36), (19)==(34), (34)==(19), (18)==(1,33), (33)==(1,18), (1)==(18,33) + │ │ │ │ │ ├── stats: [rows=2430.81298, distinct(1)=2429.06305, null(1)=0, distinct(2)=2387.04794, null(2)=0, distinct(19)=1535.60004, null(19)=0, distinct(20)=2429.06305, null(20)=0, distinct(21)=1536.11093, null(21)=0, distinct(23)=50, null(23)=0, distinct(24)=1535.26821, null(24)=0, distinct(25)=11, null(25)=0, distinct(36)=1536.11093, null(36)=0, distinct(37)=1536.11093, null(37)=0, distinct(39)=1528.33438, null(39)=0] + │ │ │ │ │ ├── fd: (1)-->(2), (36,37)-->(39), (21)==(37), (37)==(21), (20)==(1,36), (36)==(1,20), (1)==(20,36) │ │ │ │ │ ├── inner-join (lookup lineitem@l_pk_sk) │ │ │ │ │ │ ├── save-table-name: q9_lookup_join_9 - │ │ │ │ │ │ ├── columns: p_partkey:1(int!null) p_name:2(varchar!null) l_orderkey:17(int!null) l_partkey:18(int!null) l_suppkey:19(int!null) l_linenumber:20(int!null) ps_partkey:33(int!null) ps_suppkey:34(int!null) ps_supplycost:36(float!null) - │ │ │ │ │ │ ├── key columns: [33 34] = [18 19] - │ │ │ │ │ │ ├── stats: [rows=812.772154, distinct(1)=807.83427, null(1)=0, distinct(2)=807.74754, null(2)=0, distinct(17)=812.555998, null(17)=0, distinct(18)=812.772154, null(18)=0, distinct(19)=812.772154, null(19)=0, distinct(20)=7, null(20)=0, distinct(33)=812.772154, null(33)=0, distinct(34)=812.772154, null(34)=0, distinct(36)=809.246246, null(36)=0] - │ │ │ │ │ │ ├── key: (17,20) - │ │ │ │ │ │ ├── fd: (1)-->(2), (33,34)-->(36), (1)==(18,33), (33)==(1,18), (17,20)-->(18,19), (18)==(1,33), (19)==(34), (34)==(19) + │ │ │ │ │ │ ├── columns: p_partkey:1(int!null) p_name:2(varchar!null) l_orderkey:19(int!null) l_partkey:20(int!null) l_suppkey:21(int!null) l_linenumber:22(int!null) ps_partkey:36(int!null) ps_suppkey:37(int!null) ps_supplycost:39(float!null) + │ │ │ │ │ │ ├── key columns: [36 37] = [20 21] + │ │ │ │ │ │ ├── stats: [rows=812.772154, distinct(1)=807.83427, null(1)=0, distinct(2)=807.74754, null(2)=0, distinct(19)=812.555998, null(19)=0, distinct(20)=812.772154, null(20)=0, distinct(21)=812.772154, null(21)=0, distinct(22)=7, null(22)=0, distinct(36)=812.772154, null(36)=0, distinct(37)=812.772154, null(37)=0, distinct(39)=809.246246, null(39)=0] + │ │ │ │ │ │ ├── key: (19,22) + │ │ │ │ │ │ ├── fd: (1)-->(2), (36,37)-->(39), (1)==(20,36), (36)==(1,20), (19,22)-->(20,21), (20)==(1,36), (21)==(37), (37)==(21) │ │ │ │ │ │ ├── inner-join (merge) │ │ │ │ │ │ │ ├── save-table-name: q9_merge_join_10 - │ │ │ │ │ │ │ ├── columns: p_partkey:1(int!null) p_name:2(varchar!null) ps_partkey:33(int!null) ps_suppkey:34(int!null) ps_supplycost:36(float!null) + │ │ │ │ │ │ │ ├── columns: p_partkey:1(int!null) p_name:2(varchar!null) ps_partkey:36(int!null) ps_suppkey:37(int!null) ps_supplycost:39(float!null) │ │ │ │ │ │ │ ├── left ordering: +1 - │ │ │ │ │ │ │ ├── right ordering: +33 - │ │ │ │ │ │ │ ├── stats: [rows=267682.522, distinct(1)=66618.6736, null(1)=0, distinct(2)=65464.1012, null(2)=0, distinct(33)=66618.6736, null(33)=0, distinct(34)=9920, null(34)=0, distinct(36)=93404.7278, null(36)=0] - │ │ │ │ │ │ │ ├── key: (33,34) - │ │ │ │ │ │ │ ├── fd: (1)-->(2), (33,34)-->(36), (1)==(33), (33)==(1) + │ │ │ │ │ │ │ ├── right ordering: +36 + │ │ │ │ │ │ │ ├── stats: [rows=267682.522, distinct(1)=66618.6736, null(1)=0, distinct(2)=65464.1012, null(2)=0, distinct(36)=66618.6736, null(36)=0, distinct(37)=9920, null(37)=0, distinct(39)=93404.7278, null(39)=0] + │ │ │ │ │ │ │ ├── key: (36,37) + │ │ │ │ │ │ │ ├── fd: (1)-->(2), (36,37)-->(39), (1)==(36), (36)==(1) │ │ │ │ │ │ │ ├── select │ │ │ │ │ │ │ │ ├── save-table-name: q9_select_11 │ │ │ │ │ │ │ │ ├── columns: p_partkey:1(int!null) p_name:2(varchar!null) @@ -145,37 +145,37 @@ sort │ │ │ │ │ │ │ │ └── p_name:2 LIKE '%green%' [type=bool, outer=(2), constraints=(/2: (/NULL - ])] │ │ │ │ │ │ │ ├── scan partsupp │ │ │ │ │ │ │ │ ├── save-table-name: q9_scan_13 - │ │ │ │ │ │ │ │ ├── columns: ps_partkey:33(int!null) ps_suppkey:34(int!null) ps_supplycost:36(float!null) - │ │ │ │ │ │ │ │ ├── stats: [rows=800000, distinct(33)=199241, null(33)=0, distinct(34)=9920, null(34)=0, distinct(36)=100379, null(36)=0] - │ │ │ │ │ │ │ │ │ histogram(33)= 0 80 3920 80 3920 80 3920 80 3920 80 3840 160 3840 160 3920 80 3920 80 3920 80 3920 80 3920 160 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3840 160 3920 80 3920 80 3920 80 3920 80 3920 160 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3840 160 3920 80 3920 80 3920 160 3920 80 3920 160 3920 80 3920 80 3920 80 3920 80 3920 80 3920 160 3920 80 3920 80 3920 80 3920 80 3920 80 3920 160 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 3920 160 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 + │ │ │ │ │ │ │ │ ├── columns: ps_partkey:36(int!null) ps_suppkey:37(int!null) ps_supplycost:39(float!null) + │ │ │ │ │ │ │ │ ├── stats: [rows=800000, distinct(36)=199241, null(36)=0, distinct(37)=9920, null(37)=0, distinct(39)=100379, null(39)=0] + │ │ │ │ │ │ │ │ │ histogram(36)= 0 80 3920 80 3920 80 3920 80 3920 80 3840 160 3840 160 3920 80 3920 80 3920 80 3920 80 3920 160 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3840 160 3920 80 3920 80 3920 80 3920 80 3920 160 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3840 160 3920 80 3920 80 3920 160 3920 80 3920 160 3920 80 3920 80 3920 80 3920 80 3920 80 3920 160 3920 80 3920 80 3920 80 3920 80 3920 80 3920 160 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 3920 160 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 │ │ │ │ │ │ │ │ │ <--- 4 ------ 793 ------ 1856 ------ 2808 ------ 3809 ------ 4923 ------ 5975 ------ 6944 ------ 8085 ------ 8945 ------ 9831 ------ 10952 ------ 11932 ------ 12775 ------ 13849 ------ 14925 ------ 16016 ------ 16828 ------ 17768 ------ 18705 ------ 19631 ------ 20600 ------ 21491 ------ 22254 ------ 23327 ------ 24491 ------ 25470 ------ 26331 ------ 27327 ------ 28308 ------ 29359 ------ 30542 ------ 31592 ------ 32495 ------ 33408 ------ 34624 ------ 35726 ------ 36676 ------ 37637 ------ 38485 ------ 39368 ------ 40372 ------ 41034 ------ 42185 ------ 43333 ------ 44466 ------ 45440 ------ 46571 ------ 47469 ------ 48346 ------ 49379 ------ 50571 ------ 51830 ------ 52967 ------ 53673 ------ 54656 ------ 55504 ------ 56539 ------ 57393 ------ 58366 ------ 59577 ------ 60559 ------ 61676 ------ 62471 ------ 63421 ------ 64449 ------ 65409 ------ 66254 ------ 67127 ------ 68127 ------ 69177 ------ 70183 ------ 71209 ------ 72101 ------ 73102 ------ 73994 ------ 74899 ------ 76016 ------ 77098 ------ 77842 ------ 79137 ------ 80242 ------ 81364 ------ 82331 ------ 83158 ------ 84283 ------ 85282 ------ 86437 ------ 87450 ------ 88419 ------ 89493 ------ 90478 ------ 91468 ------ 92552 ------ 93200 ------ 94191 ------ 95067 ------ 96272 ------ 97228 ------ 98126 ------ 99198 ------ 100219 ------ 101057 ------ 102038 ------ 102775 ------ 103711 ------ 104623 ------ 105710 ------ 106734 ------ 107932 ------ 109255 ------ 110220 ------ 111235 ------ 112174 ------ 113260 ------ 114081 ------ 115103 ------ 115864 ------ 116794 ------ 117741 ------ 118712 ------ 119470 ------ 120528 ------ 121572 ------ 122536 ------ 123629 ------ 124404 ------ 125301 ------ 126257 ------ 127139 ------ 128267 ------ 129258 ------ 130442 ------ 131845 ------ 133164 ------ 134005 ------ 135076 ------ 135868 ------ 137297 ------ 138777 ------ 139806 ------ 140741 ------ 141896 ------ 142719 ------ 143727 ------ 144645 ------ 145510 ------ 146507 ------ 147449 ------ 148467 ------ 149635 ------ 150563 ------ 151751 ------ 152613 ------ 153416 ------ 154612 ------ 155853 ------ 156866 ------ 158311 ------ 159230 ------ 160390 ------ 161455 ------ 162555 ------ 163435 ------ 164549 ------ 165663 ------ 166891 ------ 167757 ------ 168732 ------ 169644 ------ 170532 ------ 171671 ------ 172778 ------ 173599 ------ 174321 ------ 175624 ------ 176663 ------ 177632 ------ 178555 ------ 179551 ------ 180510 ------ 181682 ------ 182648 ------ 183408 ------ 184543 ------ 185722 ------ 186713 ------ 187787 ------ 188730 ------ 189604 ------ 190711 ------ 191690 ------ 192692 ------ 193702 ------ 194685 ------ 195725 ------ 196730 ------ 197724 ------ 198701 ------ 199973 - │ │ │ │ │ │ │ │ │ histogram(34)= 0 80 3920 240 3920 80 3920 160 3920 80 3920 240 3920 80 3760 320 3680 320 3920 80 3920 160 3920 240 3920 80 3920 160 3840 160 3920 80 3920 80 3760 240 3840 160 3920 80 3840 160 3680 320 3920 80 3840 160 3840 160 3760 320 3840 160 3840 160 3920 80 3840 240 3920 80 3920 80 3840 160 3760 240 3920 160 3920 80 3920 80 3920 80 3920 320 3920 80 3920 160 3840 400 3760 240 3920 160 3920 160 3600 480 3920 80 3680 320 3840 160 3840 160 3920 240 3840 160 3920 160 3920 80 3920 160 3920 80 3760 240 3920 80 3920 80 3840 320 3840 160 3840 160 3920 240 3840 480 3920 160 3840 240 3920 160 3920 160 3920 80 3840 160 3920 80 3920 80 3920 80 3920 80 3840 240 3840 240 3920 80 3840 320 3920 80 3920 80 3920 240 3840 240 3920 160 3920 80 3840 160 3840 240 3920 240 3840 80 3680 320 3920 160 3840 160 3840 80 3920 80 3840 160 3840 160 3920 80 3920 80 3840 160 3920 80 3920 160 3840 240 3840 80 3840 160 3760 160 3920 80 3920 80 3840 240 3760 240 3840 80 3920 160 3840 80 3920 80 3920 80 3920 80 3920 160 3840 80 3920 80 3760 240 3920 80 3920 160 3760 160 3920 160 3840 80 3920 160 3840 160 3840 160 3600 320 3920 160 3840 80 3920 80 3680 320 3840 240 3760 160 3920 80 3920 80 3920 80 3920 80 3920 80 3680 320 3920 160 3840 160 3760 160 3920 240 3840 160 3840 240 3600 320 3840 80 3840 80 3920 160 3760 160 3840 160 3840 320 3840 80 3840 160 3760 240 3840 80 3840 240 3760 160 3840 160 3840 160 3920 240 3760 160 3840 80 3920 160 3680 240 3840 160 3840 160 3760 240 3920 80 3920 240 3760 160 3760 240 3840 80 3840 240 3840 240 3760 320 3760 240 3840 80 3840 160 3840 240 3760 320 3760 160 3840 160 3840 160 3840 80 3760 160 3840 80 3840 160 3920 160 3840 80 3920 80 3840 160 3920 80 3840 240 3840 80 3920 80 3760 240 3920 240 3840 80 3680 240 + │ │ │ │ │ │ │ │ │ histogram(37)= 0 80 3920 240 3920 80 3920 160 3920 80 3920 240 3920 80 3760 320 3680 320 3920 80 3920 160 3920 240 3920 80 3920 160 3840 160 3920 80 3920 80 3760 240 3840 160 3920 80 3840 160 3680 320 3920 80 3840 160 3840 160 3760 320 3840 160 3840 160 3920 80 3840 240 3920 80 3920 80 3840 160 3760 240 3920 160 3920 80 3920 80 3920 80 3920 320 3920 80 3920 160 3840 400 3760 240 3920 160 3920 160 3600 480 3920 80 3680 320 3840 160 3840 160 3920 240 3840 160 3920 160 3920 80 3920 160 3920 80 3760 240 3920 80 3920 80 3840 320 3840 160 3840 160 3920 240 3840 480 3920 160 3840 240 3920 160 3920 160 3920 80 3840 160 3920 80 3920 80 3920 80 3920 80 3840 240 3840 240 3920 80 3840 320 3920 80 3920 80 3920 240 3840 240 3920 160 3920 80 3840 160 3840 240 3920 240 3840 80 3680 320 3920 160 3840 160 3840 80 3920 80 3840 160 3840 160 3920 80 3920 80 3840 160 3920 80 3920 160 3840 240 3840 80 3840 160 3760 160 3920 80 3920 80 3840 240 3760 240 3840 80 3920 160 3840 80 3920 80 3920 80 3920 80 3920 160 3840 80 3920 80 3760 240 3920 80 3920 160 3760 160 3920 160 3840 80 3920 160 3840 160 3840 160 3600 320 3920 160 3840 80 3920 80 3680 320 3840 240 3760 160 3920 80 3920 80 3920 80 3920 80 3920 80 3680 320 3920 160 3840 160 3760 160 3920 240 3840 160 3840 240 3600 320 3840 80 3840 80 3920 160 3760 160 3840 160 3840 320 3840 80 3840 160 3760 240 3840 80 3840 240 3760 160 3840 160 3840 160 3920 240 3760 160 3840 80 3920 160 3680 240 3840 160 3840 160 3760 240 3920 80 3920 240 3760 160 3760 240 3840 80 3840 240 3840 240 3760 320 3760 240 3840 80 3840 160 3840 240 3760 320 3760 160 3840 160 3840 160 3840 80 3760 160 3840 80 3840 160 3920 160 3840 80 3920 80 3840 160 3920 80 3840 240 3840 80 3920 80 3760 240 3920 240 3840 80 3680 240 │ │ │ │ │ │ │ │ │ <--- 2 ------ 49 ------ 90 ------ 141 ------ 183 ------ 235 ------ 278 ------ 319 ------ 360 ------ 406 ------ 458 ------ 511 ------ 561 ------ 622 ------ 674 ------ 731 ------ 781 ------ 822 ------ 882 ------ 934 ------ 988 ------ 1026 ------ 1072 ------ 1114 ------ 1188 ------ 1245 ------ 1291 ------ 1335 ------ 1380 ------ 1433 ------ 1488 ------ 1537 ------ 1590 ------ 1642 ------ 1692 ------ 1751 ------ 1807 ------ 1846 ------ 1887 ------ 1939 ------ 1994 ------ 2045 ------ 2097 ------ 2139 ------ 2190 ------ 2240 ------ 2293 ------ 2344 ------ 2385 ------ 2427 ------ 2484 ------ 2538 ------ 2601 ------ 2651 ------ 2730 ------ 2781 ------ 2825 ------ 2874 ------ 2938 ------ 2978 ------ 3034 ------ 3086 ------ 3139 ------ 3179 ------ 3231 ------ 3274 ------ 3334 ------ 3387 ------ 3431 ------ 3477 ------ 3530 ------ 3581 ------ 3633 ------ 3680 ------ 3725 ------ 3775 ------ 3820 ------ 3876 ------ 3923 ------ 3982 ------ 4036 ------ 4078 ------ 4116 ------ 4183 ------ 4227 ------ 4275 ------ 4336 ------ 4376 ------ 4424 ------ 4481 ------ 4537 ------ 4582 ------ 4630 ------ 4680 ------ 4727 ------ 4779 ------ 4828 ------ 4881 ------ 4938 ------ 4986 ------ 5040 ------ 5087 ------ 5138 ------ 5188 ------ 5237 ------ 5280 ------ 5318 ------ 5358 ------ 5405 ------ 5459 ------ 5516 ------ 5561 ------ 5615 ------ 5681 ------ 5744 ------ 5790 ------ 5847 ------ 5885 ------ 5927 ------ 5991 ------ 6042 ------ 6111 ------ 6165 ------ 6207 ------ 6259 ------ 6313 ------ 6359 ------ 6418 ------ 6471 ------ 6530 ------ 6587 ------ 6626 ------ 6672 ------ 6739 ------ 6784 ------ 6837 ------ 6886 ------ 6952 ------ 6994 ------ 7040 ------ 7081 ------ 7134 ------ 7178 ------ 7232 ------ 7280 ------ 7330 ------ 7378 ------ 7435 ------ 7486 ------ 7537 ------ 7593 ------ 7636 ------ 7680 ------ 7737 ------ 7788 ------ 7836 ------ 7877 ------ 7928 ------ 7993 ------ 8036 ------ 8083 ------ 8135 ------ 8180 ------ 8221 ------ 8263 ------ 8313 ------ 8352 ------ 8399 ------ 8453 ------ 8517 ------ 8566 ------ 8612 ------ 8664 ------ 8716 ------ 8766 ------ 8821 ------ 8871 ------ 8922 ------ 8956 ------ 9007 ------ 9050 ------ 9100 ------ 9154 ------ 9203 ------ 9246 ------ 9311 ------ 9358 ------ 9407 ------ 9470 ------ 9525 ------ 9564 ------ 9633 ------ 9672 ------ 9730 ------ 9778 ------ 9824 ------ 9868 ------ 9919 ------ 9959 ------ 10000 - │ │ │ │ │ │ │ │ ├── key: (33,34) - │ │ │ │ │ │ │ │ ├── fd: (33,34)-->(36) - │ │ │ │ │ │ │ │ └── ordering: +33 + │ │ │ │ │ │ │ │ ├── key: (36,37) + │ │ │ │ │ │ │ │ ├── fd: (36,37)-->(39) + │ │ │ │ │ │ │ │ └── ordering: +36 │ │ │ │ │ │ │ └── filters (true) │ │ │ │ │ │ └── filters (true) │ │ │ │ │ └── filters (true) │ │ │ │ └── filters (true) │ │ │ └── filters - │ │ │ └── s_suppkey:10 = l_suppkey:19 [type=bool, outer=(10,19), constraints=(/10: (/NULL - ]; /19: (/NULL - ]), fd=(10)==(19), (19)==(10)] + │ │ │ └── s_suppkey:11 = l_suppkey:21 [type=bool, outer=(11,21), constraints=(/11: (/NULL - ]; /21: (/NULL - ]), fd=(11)==(21), (21)==(11)] │ │ ├── scan nation │ │ │ ├── save-table-name: q9_scan_14 - │ │ │ ├── columns: n_nationkey:47(int!null) n_name:48(char!null) - │ │ │ ├── stats: [rows=25, distinct(47)=25, null(47)=0, distinct(48)=25, null(48)=0] - │ │ │ │ histogram(47)= 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 + │ │ │ ├── columns: n_nationkey:52(int!null) n_name:53(char!null) + │ │ │ ├── stats: [rows=25, distinct(52)=25, null(52)=0, distinct(53)=25, null(53)=0] + │ │ │ │ histogram(52)= 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 --- 5 --- 6 --- 7 --- 8 --- 9 --- 10 --- 11 --- 12 --- 13 --- 14 --- 15 --- 16 --- 17 --- 18 --- 19 --- 20 --- 21 --- 22 --- 23 --- 24 - │ │ │ ├── key: (47) - │ │ │ └── fd: (47)-->(48) + │ │ │ ├── key: (52) + │ │ │ └── fd: (52)-->(53) │ │ └── filters - │ │ └── s_nationkey:13 = n_nationkey:47 [type=bool, outer=(13,47), constraints=(/13: (/NULL - ]; /47: (/NULL - ]), fd=(13)==(47), (47)==(13)] + │ │ └── s_nationkey:14 = n_nationkey:52 [type=bool, outer=(14,52), constraints=(/14: (/NULL - ]; /52: (/NULL - ]), fd=(14)==(52), (52)==(14)] │ └── projections - │ ├── extract('year', o_orderdate:42) [as=o_year:51, type=float, outer=(42), immutable] - │ └── (l_extendedprice:22 * (1.0 - l_discount:23)) - (ps_supplycost:36 * l_quantity:21) [as=amount:52, type=float, outer=(21-23,36), immutable] + │ ├── extract('year', o_orderdate:46) [as=o_year:57, type=float, outer=(46), immutable] + │ └── (l_extendedprice:24 * (1.0 - l_discount:25)) - (ps_supplycost:39 * l_quantity:23) [as=amount:58, type=float, outer=(23-25,39), immutable] └── aggregations - └── sum [as=sum:53, type=float, outer=(52)] - └── amount:52 [type=float] + └── sum [as=sum:59, type=float, outer=(58)] + └── amount:58 [type=float] stats table=q9_sort_1 ---- diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q10 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q10 index 06d9d52c37b0..5eef116902b7 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q10 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q10 @@ -54,49 +54,49 @@ LIMIT 20; ---- limit ├── save-table-name: q10_limit_1 - ├── columns: c_custkey:1(int!null) c_name:2(varchar!null) revenue:39(float!null) c_acctbal:6(float!null) n_name:35(char!null) c_address:3(varchar!null) c_phone:5(char!null) c_comment:8(varchar!null) - ├── internal-ordering: -39 + ├── columns: c_custkey:1(int!null) c_name:2(varchar!null) revenue:43(float!null) c_acctbal:6(float!null) n_name:38(char!null) c_address:3(varchar!null) c_phone:5(char!null) c_comment:8(varchar!null) + ├── internal-ordering: -43 ├── cardinality: [0 - 20] ├── immutable - ├── stats: [rows=20, distinct(1)=20, null(1)=0, distinct(2)=20, null(2)=0, distinct(3)=20, null(3)=0, distinct(5)=20, null(5)=0, distinct(6)=20, null(6)=0, distinct(8)=20, null(8)=0, distinct(35)=20, null(35)=0, distinct(39)=20, null(39)=0] + ├── stats: [rows=20, distinct(1)=20, null(1)=0, distinct(2)=20, null(2)=0, distinct(3)=20, null(3)=0, distinct(5)=20, null(5)=0, distinct(6)=20, null(6)=0, distinct(8)=20, null(8)=0, distinct(38)=20, null(38)=0, distinct(43)=20, null(43)=0] ├── key: (1) - ├── fd: (1)-->(2,3,5,6,8,35,39) - ├── ordering: -39 + ├── fd: (1)-->(2,3,5,6,8,38,43) + ├── ordering: -43 ├── sort │ ├── save-table-name: q10_sort_2 - │ ├── columns: c_custkey:1(int!null) c_name:2(varchar!null) c_address:3(varchar!null) c_phone:5(char!null) c_acctbal:6(float!null) c_comment:8(varchar!null) n_name:35(char!null) sum:39(float!null) + │ ├── columns: c_custkey:1(int!null) c_name:2(varchar!null) c_address:3(varchar!null) c_phone:5(char!null) c_acctbal:6(float!null) c_comment:8(varchar!null) n_name:38(char!null) sum:43(float!null) │ ├── immutable - │ ├── stats: [rows=34517.9842, distinct(1)=34517.9842, null(1)=0, distinct(2)=34517.9842, null(2)=0, distinct(3)=34517.9842, null(3)=0, distinct(5)=34517.9842, null(5)=0, distinct(6)=34517.9842, null(6)=0, distinct(8)=34517.9842, null(8)=0, distinct(35)=34517.9842, null(35)=0, distinct(39)=34517.9842, null(39)=0] + │ ├── stats: [rows=34517.9842, distinct(1)=34517.9842, null(1)=0, distinct(2)=34517.9842, null(2)=0, distinct(3)=34517.9842, null(3)=0, distinct(5)=34517.9842, null(5)=0, distinct(6)=34517.9842, null(6)=0, distinct(8)=34517.9842, null(8)=0, distinct(38)=34517.9842, null(38)=0, distinct(43)=34517.9842, null(43)=0] │ ├── key: (1) - │ ├── fd: (1)-->(2,3,5,6,8,35,39) - │ ├── ordering: -39 + │ ├── fd: (1)-->(2,3,5,6,8,38,43) + │ ├── ordering: -43 │ ├── limit hint: 20.00 │ └── group-by │ ├── save-table-name: q10_group_by_3 - │ ├── columns: c_custkey:1(int!null) c_name:2(varchar!null) c_address:3(varchar!null) c_phone:5(char!null) c_acctbal:6(float!null) c_comment:8(varchar!null) n_name:35(char!null) sum:39(float!null) + │ ├── columns: c_custkey:1(int!null) c_name:2(varchar!null) c_address:3(varchar!null) c_phone:5(char!null) c_acctbal:6(float!null) c_comment:8(varchar!null) n_name:38(char!null) sum:43(float!null) │ ├── grouping columns: c_custkey:1(int!null) │ ├── immutable - │ ├── stats: [rows=34517.9842, distinct(1)=34517.9842, null(1)=0, distinct(2)=34517.9842, null(2)=0, distinct(3)=34517.9842, null(3)=0, distinct(5)=34517.9842, null(5)=0, distinct(6)=34517.9842, null(6)=0, distinct(8)=34517.9842, null(8)=0, distinct(35)=34517.9842, null(35)=0, distinct(39)=34517.9842, null(39)=0] + │ ├── stats: [rows=34517.9842, distinct(1)=34517.9842, null(1)=0, distinct(2)=34517.9842, null(2)=0, distinct(3)=34517.9842, null(3)=0, distinct(5)=34517.9842, null(5)=0, distinct(6)=34517.9842, null(6)=0, distinct(8)=34517.9842, null(8)=0, distinct(38)=34517.9842, null(38)=0, distinct(43)=34517.9842, null(43)=0] │ ├── key: (1) - │ ├── fd: (1)-->(2,3,5,6,8,35,39) + │ ├── fd: (1)-->(2,3,5,6,8,38,43) │ ├── project │ │ ├── save-table-name: q10_project_4 - │ │ ├── columns: column38:38(float!null) c_custkey:1(int!null) c_name:2(varchar!null) c_address:3(varchar!null) c_phone:5(char!null) c_acctbal:6(float!null) c_comment:8(varchar!null) n_name:35(char!null) + │ │ ├── columns: column42:42(float!null) c_custkey:1(int!null) c_name:2(varchar!null) c_address:3(varchar!null) c_phone:5(char!null) c_acctbal:6(float!null) c_comment:8(varchar!null) n_name:38(char!null) │ │ ├── immutable - │ │ ├── stats: [rows=91240.8317, distinct(1)=34517.9842, null(1)=0, distinct(2)=50850.8128, null(2)=0, distinct(3)=50847.7096, null(3)=0, distinct(5)=50850.8128, null(5)=0, distinct(6)=50359.5433, null(6)=0, distinct(8)=50817.3327, null(8)=0, distinct(35)=25, null(35)=0, distinct(38)=45841.1563, null(38)=0] - │ │ ├── fd: (1)-->(2,3,5,6,8,35) + │ │ ├── stats: [rows=91240.8317, distinct(1)=34517.9842, null(1)=0, distinct(2)=50850.8128, null(2)=0, distinct(3)=50847.7096, null(3)=0, distinct(5)=50850.8128, null(5)=0, distinct(6)=50359.5433, null(6)=0, distinct(8)=50817.3327, null(8)=0, distinct(38)=25, null(38)=0, distinct(42)=45841.1563, null(42)=0] + │ │ ├── fd: (1)-->(2,3,5,6,8,38) │ │ ├── inner-join (hash) │ │ │ ├── save-table-name: q10_inner_join_5 - │ │ │ ├── columns: c_custkey:1(int!null) c_name:2(varchar!null) c_address:3(varchar!null) c_nationkey:4(int!null) c_phone:5(char!null) c_acctbal:6(float!null) c_comment:8(varchar!null) o_orderkey:9(int!null) o_custkey:10(int!null) o_orderdate:13(date!null) l_orderkey:18(int!null) l_extendedprice:23(float!null) l_discount:24(float!null) l_returnflag:26(char!null) n_nationkey:34(int!null) n_name:35(char!null) + │ │ │ ├── columns: c_custkey:1(int!null) c_name:2(varchar!null) c_address:3(varchar!null) c_nationkey:4(int!null) c_phone:5(char!null) c_acctbal:6(float!null) c_comment:8(varchar!null) o_orderkey:10(int!null) o_custkey:11(int!null) o_orderdate:14(date!null) l_orderkey:20(int!null) l_extendedprice:25(float!null) l_discount:26(float!null) l_returnflag:28(char!null) n_nationkey:37(int!null) n_name:38(char!null) │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ │ │ ├── stats: [rows=91240.8317, distinct(1)=34517.9842, null(1)=0, distinct(2)=50850.8128, null(2)=0, distinct(3)=50847.7096, null(3)=0, distinct(4)=25, null(4)=0, distinct(5)=50850.8128, null(5)=0, distinct(6)=50359.5433, null(6)=0, distinct(8)=50817.3327, null(8)=0, distinct(9)=39047.6447, null(9)=0, distinct(10)=34517.9842, null(10)=0, distinct(13)=92, null(13)=0, distinct(18)=39047.6447, null(18)=0, distinct(23)=45484.5715, null(23)=0, distinct(24)=11, null(24)=0, distinct(26)=1, null(26)=0, distinct(34)=25, null(34)=0, distinct(35)=25, null(35)=0, distinct(23,24)=45841.1563, null(23,24)=0] - │ │ │ ├── fd: ()-->(26), (1)-->(2-6,8), (9)-->(10,13), (9)==(18), (18)==(9), (1)==(10), (10)==(1), (34)-->(35), (4)==(34), (34)==(4) + │ │ │ ├── stats: [rows=91240.8317, distinct(1)=34517.9842, null(1)=0, distinct(2)=50850.8128, null(2)=0, distinct(3)=50847.7096, null(3)=0, distinct(4)=25, null(4)=0, distinct(5)=50850.8128, null(5)=0, distinct(6)=50359.5433, null(6)=0, distinct(8)=50817.3327, null(8)=0, distinct(10)=39047.6447, null(10)=0, distinct(11)=34517.9842, null(11)=0, distinct(14)=92, null(14)=0, distinct(20)=39047.6447, null(20)=0, distinct(25)=45484.5715, null(25)=0, distinct(26)=11, null(26)=0, distinct(28)=1, null(28)=0, distinct(37)=25, null(37)=0, distinct(38)=25, null(38)=0, distinct(25,26)=45841.1563, null(25,26)=0] + │ │ │ ├── fd: ()-->(28), (1)-->(2-6,8), (10)-->(11,14), (10)==(20), (20)==(10), (1)==(11), (11)==(1), (37)-->(38), (4)==(37), (37)==(4) │ │ │ ├── inner-join (hash) │ │ │ │ ├── save-table-name: q10_inner_join_6 - │ │ │ │ ├── columns: c_custkey:1(int!null) c_name:2(varchar!null) c_address:3(varchar!null) c_nationkey:4(int!null) c_phone:5(char!null) c_acctbal:6(float!null) c_comment:8(varchar!null) o_orderkey:9(int!null) o_custkey:10(int!null) o_orderdate:13(date!null) l_orderkey:18(int!null) l_extendedprice:23(float!null) l_discount:24(float!null) l_returnflag:26(char!null) + │ │ │ │ ├── columns: c_custkey:1(int!null) c_name:2(varchar!null) c_address:3(varchar!null) c_nationkey:4(int!null) c_phone:5(char!null) c_acctbal:6(float!null) c_comment:8(varchar!null) o_orderkey:10(int!null) o_custkey:11(int!null) o_orderdate:14(date!null) l_orderkey:20(int!null) l_extendedprice:25(float!null) l_discount:26(float!null) l_returnflag:28(char!null) │ │ │ │ ├── multiplicity: left-rows(zero-or-more), right-rows(exactly-one) - │ │ │ │ ├── stats: [rows=91240.8317, distinct(1)=37710.0545, null(1)=0, distinct(2)=68356.5955, null(2)=0, distinct(3)=68348.7409, null(3)=0, distinct(4)=25, null(4)=0, distinct(5)=68356.5955, null(5)=0, distinct(6)=67126.4808, null(6)=0, distinct(8)=68271.9098, null(8)=0, distinct(9)=44561.5336, null(9)=0, distinct(10)=37710.0545, null(10)=0, distinct(13)=92, null(13)=0, distinct(18)=44561.5336, null(18)=0, distinct(23)=56207.9759, null(23)=0, distinct(24)=11, null(24)=0, distinct(26)=1, null(26)=0, distinct(23,24)=56936.0051, null(23,24)=0] - │ │ │ │ ├── fd: ()-->(26), (1)-->(2-6,8), (9)-->(10,13), (9)==(18), (18)==(9), (1)==(10), (10)==(1) + │ │ │ │ ├── stats: [rows=91240.8317, distinct(1)=37710.0545, null(1)=0, distinct(2)=68356.5955, null(2)=0, distinct(3)=68348.7409, null(3)=0, distinct(4)=25, null(4)=0, distinct(5)=68356.5955, null(5)=0, distinct(6)=67126.4808, null(6)=0, distinct(8)=68271.9098, null(8)=0, distinct(10)=44561.5336, null(10)=0, distinct(11)=37710.0545, null(11)=0, distinct(14)=92, null(14)=0, distinct(20)=44561.5336, null(20)=0, distinct(25)=56207.9759, null(25)=0, distinct(26)=11, null(26)=0, distinct(28)=1, null(28)=0, distinct(25,26)=56936.0051, null(25,26)=0] + │ │ │ │ ├── fd: ()-->(28), (1)-->(2-6,8), (10)-->(11,14), (10)==(20), (20)==(10), (1)==(11), (11)==(1) │ │ │ │ ├── scan customer │ │ │ │ │ ├── save-table-name: q10_scan_7 │ │ │ │ │ ├── columns: c_custkey:1(int!null) c_name:2(varchar!null) c_address:3(varchar!null) c_nationkey:4(int!null) c_phone:5(char!null) c_acctbal:6(float!null) c_comment:8(varchar!null) @@ -109,48 +109,48 @@ limit │ │ │ │ │ └── fd: (1)-->(2-6,8) │ │ │ │ ├── inner-join (lookup lineitem) │ │ │ │ │ ├── save-table-name: q10_lookup_join_8 - │ │ │ │ │ ├── columns: o_orderkey:9(int!null) o_custkey:10(int!null) o_orderdate:13(date!null) l_orderkey:18(int!null) l_extendedprice:23(float!null) l_discount:24(float!null) l_returnflag:26(char!null) - │ │ │ │ │ ├── key columns: [9] = [18] - │ │ │ │ │ ├── stats: [rows=90518.8126, distinct(9)=55061.5385, null(9)=0, distinct(10)=37710.0545, null(10)=0, distinct(13)=92, null(13)=0, distinct(18)=55061.5385, null(18)=0, distinct(23)=85913.1168, null(23)=0, distinct(24)=11, null(24)=0, distinct(26)=1, null(26)=0, distinct(23,24)=88501.3935, null(23,24)=0] - │ │ │ │ │ ├── fd: ()-->(26), (9)-->(10,13), (9)==(18), (18)==(9) + │ │ │ │ │ ├── columns: o_orderkey:10(int!null) o_custkey:11(int!null) o_orderdate:14(date!null) l_orderkey:20(int!null) l_extendedprice:25(float!null) l_discount:26(float!null) l_returnflag:28(char!null) + │ │ │ │ │ ├── key columns: [10] = [20] + │ │ │ │ │ ├── stats: [rows=90518.8126, distinct(10)=55061.5385, null(10)=0, distinct(11)=37710.0545, null(11)=0, distinct(14)=92, null(14)=0, distinct(20)=55061.5385, null(20)=0, distinct(25)=85913.1168, null(25)=0, distinct(26)=11, null(26)=0, distinct(28)=1, null(28)=0, distinct(25,26)=88501.3935, null(25,26)=0] + │ │ │ │ │ ├── fd: ()-->(28), (10)-->(11,14), (10)==(20), (20)==(10) │ │ │ │ │ ├── index-join orders │ │ │ │ │ │ ├── save-table-name: q10_index_join_9 - │ │ │ │ │ │ ├── columns: o_orderkey:9(int!null) o_custkey:10(int!null) o_orderdate:13(date!null) - │ │ │ │ │ │ ├── stats: [rows=55061.5385, distinct(9)=55061.5385, null(9)=0, distinct(10)=42917.9526, null(10)=0, distinct(13)=92, null(13)=0] - │ │ │ │ │ │ │ histogram(13)= 0 450 6900 600 6600 1350 6900 1050 6300 1050 6750 1050 6900 450 7050 600 530.77 530.77 + │ │ │ │ │ │ ├── columns: o_orderkey:10(int!null) o_custkey:11(int!null) o_orderdate:14(date!null) + │ │ │ │ │ │ ├── stats: [rows=55061.5385, distinct(10)=55061.5385, null(10)=0, distinct(11)=42917.9526, null(11)=0, distinct(14)=92, null(14)=0] + │ │ │ │ │ │ │ histogram(14)= 0 450 6900 600 6600 1350 6900 1050 6300 1050 6750 1050 6900 450 7050 600 530.77 530.77 │ │ │ │ │ │ │ <--- '1993-10-01' ------ '1993-10-13' ------ '1993-10-22' ------ '1993-11-04' ------ '1993-11-16' ------ '1993-11-30' ------ '1993-12-16' ------ '1993-12-29' -------- '1993-12-31' - │ │ │ │ │ │ ├── key: (9) - │ │ │ │ │ │ ├── fd: (9)-->(10,13) + │ │ │ │ │ │ ├── key: (10) + │ │ │ │ │ │ ├── fd: (10)-->(11,14) │ │ │ │ │ │ └── scan orders@o_od │ │ │ │ │ │ ├── save-table-name: q10_scan_10 - │ │ │ │ │ │ ├── columns: o_orderkey:9(int!null) o_orderdate:13(date!null) - │ │ │ │ │ │ ├── constraint: /13/9: [/'1993-10-01' - /'1993-12-31'] - │ │ │ │ │ │ ├── stats: [rows=55061.5385, distinct(9)=55061.5385, null(9)=0, distinct(13)=92, null(13)=0] - │ │ │ │ │ │ │ histogram(9)= 0 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 - │ │ │ │ │ │ │ <--- 998 -------- 26593 ------- 56613 ------- 85827 ------- 115587 ------- 140512 ------- 167076 ------- 198882 ------- 228580 ------- 262243 ------- 292064 ------- 316161 ------- 344194 ------- 368550 ------- 402722 ------- 434529 ------- 468807 ------- 495107 ------- 526338 ------- 563239 ------- 597733 ------- 641894 ------- 665730 ------- 688742 ------- 721767 ------- 750050 ------- 787879 ------- 814565 ------- 837381 ------- 866657 ------- 902855 ------- 927172 ------- 952772 ------- 975840 ------- 1008646 ------- 1032066 ------- 1074656 ------- 1104896 ------- 1138369 ------- 1166689 ------- 1194912 ------- 1225222 ------- 1253284 ------- 1284803 ------- 1318499 ------- 1348609 ------- 1376609 ------- 1404933 ------- 1435745 ------- 1460325 ------- 1491648 ------- 1518852 ------- 1542567 ------- 1574277 ------- 1597958 ------- 1624964 ------- 1647555 ------- 1683651 ------- 1717189 ------- 1750563 ------- 1784775 ------- 1812449 ------- 1838850 ------- 1867751 ------- 1891783 ------- 1918790 ------- 1949827 ------- 1986021 ------- 2020134 ------- 2045829 ------- 2077894 ------- 2118912 ------- 2143264 ------- 2176486 ------- 2204838 ------- 2234146 ------- 2260484 ------- 2288512 ------- 2317121 ------- 2344321 ------- 2362567 ------- 2396133 ------- 2423623 ------- 2454884 ------- 2483879 ------- 2520162 ------- 2545574 ------- 2571874 ------- 2608417 ------- 2637477 ------- 2665607 ------- 2692231 ------- 2724261 ------- 2764451 ------- 2798656 ------- 2824775 ------- 2858690 ------- 2881990 ------- 2911814 ------- 2945056 ------- 2981315 ------- 3010306 ------- 3032320 ------- 3064389 ------- 3084166 ------- 3108832 ------- 3136032 ------- 3168484 ------- 3204039 ------- 3229922 ------- 3256549 ------- 3283169 ------- 3315236 ------- 3346756 ------- 3375879 ------- 3408007 ------- 3441827 ------- 3471204 ------- 3505414 ------- 3527938 ------- 3562561 ------- 3598630 ------- 3627270 ------- 3659266 ------- 3686468 ------- 3713505 ------- 3750817 ------- 3777696 ------- 3808129 ------- 3834533 ------- 3871968 ------- 3904931 ------- 3934215 ------- 3960643 ------- 3990336 ------- 4023203 ------- 4057826 ------- 4090691 ------- 4121216 ------- 4159681 ------- 4190726 ------- 4222624 ------- 4246567 ------- 4283653 ------- 4314339 ------- 4342592 ------- 4373829 ------- 4402372 ------- 4431332 ------- 4464452 ------- 4496327 -------- 4526789 -------- 4551109 -------- 4582401 -------- 4615974 -------- 4648482 -------- 4669601 -------- 4705891 -------- 4732869 -------- 4766145 -------- 4805862 -------- 4840677 -------- 4866787 -------- 4895332 -------- 4928100 -------- 4956323 -------- 4992161 -------- 5027008 -------- 5065797 -------- 5099011 -------- 5130592 -------- 5163463 -------- 5197314 -------- 5224994 -------- 5253892 -------- 5277638 -------- 5307105 -------- 5335750 -------- 5369828 -------- 5402528 -------- 5434183 -------- 5464227 -------- 5491072 -------- 5520679 -------- 5543047 -------- 5576708 -------- 5614951 -------- 5646055 -------- 5674721 -------- 5700295 -------- 5732066 -------- 5761255 -------- 5791233 -------- 5819651 -------- 5852291 -------- 5880258 -------- 5909062 -------- 5943111 -------- 5973926 -------- 5998752 - │ │ │ │ │ │ │ histogram(13)= 0 450 6900 600 6600 1350 6900 1050 6300 1050 6750 1050 6900 450 7050 600 530.77 530.77 + │ │ │ │ │ │ ├── columns: o_orderkey:10(int!null) o_orderdate:14(date!null) + │ │ │ │ │ │ ├── constraint: /14/10: [/'1993-10-01' - /'1993-12-31'] + │ │ │ │ │ │ ├── stats: [rows=55061.5385, distinct(10)=55061.5385, null(10)=0, distinct(14)=92, null(14)=0] + │ │ │ │ │ │ │ histogram(10)= 0 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 269.8 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 275.31 5.5062 + │ │ │ │ │ │ │ <--- 998 -------- 26593 ------- 56613 ------- 85827 ------- 115587 ------- 140512 ------- 167076 ------- 198882 ------- 228580 ------- 262243 ------- 292064 ------- 316161 ------- 344194 ------- 368550 ------- 402722 ------- 434529 ------- 468807 ------- 495107 ------- 526338 ------- 563239 ------- 597733 ------- 641894 ------- 665730 ------- 688742 ------- 721767 ------- 750050 ------- 787879 ------- 814565 ------- 837381 ------- 866657 ------- 902855 ------- 927172 ------- 952772 ------- 975840 ------- 1008646 ------- 1032066 ------- 1074656 ------- 1104896 ------- 1138369 ------- 1166689 ------- 1194912 ------- 1225222 ------- 1253284 ------- 1284803 ------- 1318499 ------- 1348609 ------- 1376609 ------- 1404933 ------- 1435745 ------- 1460325 ------- 1491648 ------- 1518852 ------- 1542567 ------- 1574277 ------- 1597958 ------- 1624964 ------- 1647555 ------- 1683651 ------- 1717189 ------- 1750563 ------- 1784775 ------- 1812449 ------- 1838850 ------- 1867751 ------- 1891783 ------- 1918790 ------- 1949827 ------- 1986021 ------- 2020134 ------- 2045829 ------- 2077894 ------- 2118912 ------- 2143264 ------- 2176486 ------- 2204838 ------- 2234146 ------- 2260484 ------- 2288512 ------- 2317121 ------- 2344321 ------- 2362567 ------- 2396133 ------- 2423623 ------- 2454884 ------- 2483879 ------- 2520162 ------- 2545574 ------- 2571874 ------- 2608417 ------- 2637477 ------- 2665607 ------- 2692231 ------- 2724261 ------- 2764451 ------- 2798656 ------- 2824775 ------- 2858690 ------- 2881990 ------- 2911814 ------- 2945056 ------- 2981315 ------- 3010306 ------- 3032320 ------- 3064389 ------- 3084166 ------- 3108832 ------- 3136032 ------- 3168484 ------- 3204039 ------- 3229922 ------- 3256549 ------- 3283169 ------- 3315236 ------- 3346756 ------- 3375879 ------- 3408007 ------- 3441827 ------- 3471204 ------- 3505414 ------- 3527938 ------- 3562561 ------- 3598630 ------- 3627270 ------- 3659266 ------- 3686468 ------- 3713505 ------- 3750817 ------- 3777696 ------- 3808129 ------- 3834533 ------- 3871968 ------- 3904931 ------- 3934215 ------- 3960643 ------- 3990336 ------- 4023203 ------- 4057826 ------- 4090691 ------- 4121216 ------- 4159681 ------- 4190726 ------- 4222624 ------- 4246567 ------- 4283653 ------- 4314339 ------- 4342592 ------- 4373829 ------- 4402372 ------- 4431332 ------- 4464452 ------- 4496327 -------- 4526789 -------- 4551109 -------- 4582401 -------- 4615974 -------- 4648482 -------- 4669601 -------- 4705891 -------- 4732869 -------- 4766145 -------- 4805862 -------- 4840677 -------- 4866787 -------- 4895332 -------- 4928100 -------- 4956323 -------- 4992161 -------- 5027008 -------- 5065797 -------- 5099011 -------- 5130592 -------- 5163463 -------- 5197314 -------- 5224994 -------- 5253892 -------- 5277638 -------- 5307105 -------- 5335750 -------- 5369828 -------- 5402528 -------- 5434183 -------- 5464227 -------- 5491072 -------- 5520679 -------- 5543047 -------- 5576708 -------- 5614951 -------- 5646055 -------- 5674721 -------- 5700295 -------- 5732066 -------- 5761255 -------- 5791233 -------- 5819651 -------- 5852291 -------- 5880258 -------- 5909062 -------- 5943111 -------- 5973926 -------- 5998752 + │ │ │ │ │ │ │ histogram(14)= 0 450 6900 600 6600 1350 6900 1050 6300 1050 6750 1050 6900 450 7050 600 530.77 530.77 │ │ │ │ │ │ │ <--- '1993-10-01' ------ '1993-10-13' ------ '1993-10-22' ------ '1993-11-04' ------ '1993-11-16' ------ '1993-11-30' ------ '1993-12-16' ------ '1993-12-29' -------- '1993-12-31' - │ │ │ │ │ │ ├── key: (9) - │ │ │ │ │ │ └── fd: (9)-->(13) + │ │ │ │ │ │ ├── key: (10) + │ │ │ │ │ │ └── fd: (10)-->(14) │ │ │ │ │ └── filters - │ │ │ │ │ └── l_returnflag:26 = 'R' [type=bool, outer=(26), constraints=(/26: [/'R' - /'R']; tight), fd=()-->(26)] + │ │ │ │ │ └── l_returnflag:28 = 'R' [type=bool, outer=(28), constraints=(/28: [/'R' - /'R']; tight), fd=()-->(28)] │ │ │ │ └── filters - │ │ │ │ └── c_custkey:1 = o_custkey:10 [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] + │ │ │ │ └── c_custkey:1 = o_custkey:11 [type=bool, outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ]), fd=(1)==(11), (11)==(1)] │ │ │ ├── scan nation │ │ │ │ ├── save-table-name: q10_scan_11 - │ │ │ │ ├── columns: n_nationkey:34(int!null) n_name:35(char!null) - │ │ │ │ ├── stats: [rows=25, distinct(34)=25, null(34)=0, distinct(35)=25, null(35)=0] - │ │ │ │ │ histogram(34)= 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 + │ │ │ │ ├── columns: n_nationkey:37(int!null) n_name:38(char!null) + │ │ │ │ ├── stats: [rows=25, distinct(37)=25, null(37)=0, distinct(38)=25, null(38)=0] + │ │ │ │ │ histogram(37)= 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 --- 5 --- 6 --- 7 --- 8 --- 9 --- 10 --- 11 --- 12 --- 13 --- 14 --- 15 --- 16 --- 17 --- 18 --- 19 --- 20 --- 21 --- 22 --- 23 --- 24 - │ │ │ │ ├── key: (34) - │ │ │ │ └── fd: (34)-->(35) + │ │ │ │ ├── key: (37) + │ │ │ │ └── fd: (37)-->(38) │ │ │ └── filters - │ │ │ └── c_nationkey:4 = n_nationkey:34 [type=bool, outer=(4,34), constraints=(/4: (/NULL - ]; /34: (/NULL - ]), fd=(4)==(34), (34)==(4)] + │ │ │ └── c_nationkey:4 = n_nationkey:37 [type=bool, outer=(4,37), constraints=(/4: (/NULL - ]; /37: (/NULL - ]), fd=(4)==(37), (37)==(4)] │ │ └── projections - │ │ └── l_extendedprice:23 * (1.0 - l_discount:24) [as=column38:38, type=float, outer=(23,24), immutable] + │ │ └── l_extendedprice:25 * (1.0 - l_discount:26) [as=column42:42, type=float, outer=(25,26), immutable] │ └── aggregations - │ ├── sum [as=sum:39, type=float, outer=(38)] - │ │ └── column38:38 [type=float] + │ ├── sum [as=sum:43, type=float, outer=(42)] + │ │ └── column42:42 [type=float] │ ├── const-agg [as=c_name:2, type=varchar, outer=(2)] │ │ └── c_name:2 [type=varchar] │ ├── const-agg [as=c_address:3, type=varchar, outer=(3)] @@ -161,8 +161,8 @@ limit │ │ └── c_acctbal:6 [type=float] │ ├── const-agg [as=c_comment:8, type=varchar, outer=(8)] │ │ └── c_comment:8 [type=varchar] - │ └── const-agg [as=n_name:35, type=char, outer=(35)] - │ └── n_name:35 [type=char] + │ └── const-agg [as=n_name:38, type=char, outer=(38)] + │ └── n_name:38 [type=char] └── 20 [type=int] stats table=q10_limit_1 @@ -240,7 +240,7 @@ column_names row_count distinct_count null_count {c_custkey} 114705 37904 0 {c_name} 114705 37859 0 {c_phone} 114705 38026 0 -{column38} 114705 114608 0 +{column42} 114705 114608 0 {n_name} 114705 25 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err @@ -250,7 +250,7 @@ column_names row_count_est row_count_err distinct_count_est distinct_count_e {c_custkey} 91241.00 1.26 34518.00 1.10 0.00 1.00 {c_name} 91241.00 1.26 50851.00 1.34 0.00 1.00 {c_phone} 91241.00 1.26 50851.00 1.34 0.00 1.00 -{column38} 91241.00 1.26 45841.00 2.50 <== 0.00 1.00 +{column42} 91241.00 1.26 45841.00 2.50 <== 0.00 1.00 {n_name} 91241.00 1.26 25.00 1.00 0.00 1.00 stats table=q10_inner_join_5 diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q11 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q11 index 81260c52ed6c..e21e3d52a4c9 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q11 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q11 @@ -45,150 +45,150 @@ ORDER BY ---- sort ├── save-table-name: q11_sort_1 - ├── columns: ps_partkey:1(int!null) value:18(float!null) + ├── columns: ps_partkey:1(int!null) value:21(float!null) ├── immutable - ├── stats: [rows=9927.82897, distinct(1)=9927.82897, null(1)=0, distinct(18)=9927.82897, null(18)=0] + ├── stats: [rows=9927.82897, distinct(1)=9927.82897, null(1)=0, distinct(21)=9927.82897, null(21)=0] ├── key: (1) - ├── fd: (1)-->(18) - ├── ordering: -18 + ├── fd: (1)-->(21) + ├── ordering: -21 └── select ├── save-table-name: q11_select_2 - ├── columns: ps_partkey:1(int!null) sum:18(float!null) + ├── columns: ps_partkey:1(int!null) sum:21(float!null) ├── immutable - ├── stats: [rows=9927.82897, distinct(1)=9927.82897, null(1)=0, distinct(18)=9927.82897, null(18)=0] + ├── stats: [rows=9927.82897, distinct(1)=9927.82897, null(1)=0, distinct(21)=9927.82897, null(21)=0] ├── key: (1) - ├── fd: (1)-->(18) + ├── fd: (1)-->(21) ├── group-by │ ├── save-table-name: q11_group_by_3 - │ ├── columns: ps_partkey:1(int!null) sum:18(float!null) + │ ├── columns: ps_partkey:1(int!null) sum:21(float!null) │ ├── grouping columns: ps_partkey:1(int!null) │ ├── immutable - │ ├── stats: [rows=29783.4869, distinct(1)=29783.4869, null(1)=0, distinct(18)=29783.4869, null(18)=0] + │ ├── stats: [rows=29783.4869, distinct(1)=29783.4869, null(1)=0, distinct(21)=29783.4869, null(21)=0] │ ├── key: (1) - │ ├── fd: (1)-->(18) + │ ├── fd: (1)-->(21) │ ├── project │ │ ├── save-table-name: q11_project_4 - │ │ ├── columns: column17:17(float!null) ps_partkey:1(int!null) + │ │ ├── columns: column20:20(float!null) ps_partkey:1(int!null) │ │ ├── immutable - │ │ ├── stats: [rows=32258.0645, distinct(1)=29783.4869, null(1)=0, distinct(17)=31617.9161, null(17)=0] + │ │ ├── stats: [rows=32258.0645, distinct(1)=29783.4869, null(1)=0, distinct(20)=31617.9161, null(20)=0] │ │ ├── inner-join (lookup partsupp) │ │ │ ├── save-table-name: q11_lookup_join_5 - │ │ │ ├── columns: ps_partkey:1(int!null) ps_suppkey:2(int!null) ps_availqty:3(int!null) ps_supplycost:4(float!null) s_suppkey:6(int!null) s_nationkey:9(int!null) n_nationkey:13(int!null) n_name:14(char!null) + │ │ │ ├── columns: ps_partkey:1(int!null) ps_suppkey:2(int!null) ps_availqty:3(int!null) ps_supplycost:4(float!null) s_suppkey:7(int!null) s_nationkey:10(int!null) n_nationkey:15(int!null) n_name:16(char!null) │ │ │ ├── key columns: [1 2] = [1 2] │ │ │ ├── lookup columns are key - │ │ │ ├── stats: [rows=32258.0645, distinct(1)=29783.4869, null(1)=0, distinct(2)=399.934613, null(2)=0, distinct(3)=9536.12259, null(3)=0, distinct(4)=27589.3232, null(4)=0, distinct(6)=399.934613, null(6)=0, distinct(9)=1, null(9)=0, distinct(13)=1, null(13)=0, distinct(14)=1, null(14)=0, distinct(3,4)=31617.9161, null(3,4)=0] - │ │ │ ├── key: (1,6) - │ │ │ ├── fd: ()-->(14), (1,2)-->(3,4), (6)-->(9), (9)==(13), (13)==(9), (2)==(6), (6)==(2) + │ │ │ ├── stats: [rows=32258.0645, distinct(1)=29783.4869, null(1)=0, distinct(2)=399.934613, null(2)=0, distinct(3)=9536.12259, null(3)=0, distinct(4)=27589.3232, null(4)=0, distinct(7)=399.934613, null(7)=0, distinct(10)=1, null(10)=0, distinct(15)=1, null(15)=0, distinct(16)=1, null(16)=0, distinct(3,4)=31617.9161, null(3,4)=0] + │ │ │ ├── key: (1,7) + │ │ │ ├── fd: ()-->(16), (1,2)-->(3,4), (7)-->(10), (10)==(15), (15)==(10), (2)==(7), (7)==(2) │ │ │ ├── inner-join (lookup partsupp@ps_sk) │ │ │ │ ├── save-table-name: q11_lookup_join_6 - │ │ │ │ ├── columns: ps_partkey:1(int!null) ps_suppkey:2(int!null) s_suppkey:6(int!null) s_nationkey:9(int!null) n_nationkey:13(int!null) n_name:14(char!null) - │ │ │ │ ├── key columns: [6] = [2] - │ │ │ │ ├── stats: [rows=32258.0645, distinct(1)=29783.4869, null(1)=0, distinct(2)=399.934613, null(2)=0, distinct(6)=399.934613, null(6)=0, distinct(9)=1, null(9)=0, distinct(13)=1, null(13)=0, distinct(14)=1, null(14)=0] - │ │ │ │ ├── key: (1,6) - │ │ │ │ ├── fd: ()-->(14), (6)-->(9), (9)==(13), (13)==(9), (2)==(6), (6)==(2) + │ │ │ │ ├── columns: ps_partkey:1(int!null) ps_suppkey:2(int!null) s_suppkey:7(int!null) s_nationkey:10(int!null) n_nationkey:15(int!null) n_name:16(char!null) + │ │ │ │ ├── key columns: [7] = [2] + │ │ │ │ ├── stats: [rows=32258.0645, distinct(1)=29783.4869, null(1)=0, distinct(2)=399.934613, null(2)=0, distinct(7)=399.934613, null(7)=0, distinct(10)=1, null(10)=0, distinct(15)=1, null(15)=0, distinct(16)=1, null(16)=0] + │ │ │ │ ├── key: (1,7) + │ │ │ │ ├── fd: ()-->(16), (7)-->(10), (10)==(15), (15)==(10), (2)==(7), (7)==(2) │ │ │ │ ├── inner-join (lookup supplier@s_nk) │ │ │ │ │ ├── save-table-name: q11_lookup_join_7 - │ │ │ │ │ ├── columns: s_suppkey:6(int!null) s_nationkey:9(int!null) n_nationkey:13(int!null) n_name:14(char!null) - │ │ │ │ │ ├── key columns: [13] = [9] - │ │ │ │ │ ├── stats: [rows=400, distinct(6)=399.934613, null(6)=0, distinct(9)=1, null(9)=0, distinct(13)=1, null(13)=0, distinct(14)=1, null(14)=0] - │ │ │ │ │ ├── key: (6) - │ │ │ │ │ ├── fd: ()-->(14), (6)-->(9), (9)==(13), (13)==(9) + │ │ │ │ │ ├── columns: s_suppkey:7(int!null) s_nationkey:10(int!null) n_nationkey:15(int!null) n_name:16(char!null) + │ │ │ │ │ ├── key columns: [15] = [10] + │ │ │ │ │ ├── stats: [rows=400, distinct(7)=399.934613, null(7)=0, distinct(10)=1, null(10)=0, distinct(15)=1, null(15)=0, distinct(16)=1, null(16)=0] + │ │ │ │ │ ├── key: (7) + │ │ │ │ │ ├── fd: ()-->(16), (7)-->(10), (10)==(15), (15)==(10) │ │ │ │ │ ├── select │ │ │ │ │ │ ├── save-table-name: q11_select_8 - │ │ │ │ │ │ ├── columns: n_nationkey:13(int!null) n_name:14(char!null) - │ │ │ │ │ │ ├── stats: [rows=1, distinct(13)=1, null(13)=0, distinct(14)=1, null(14)=0] - │ │ │ │ │ │ ├── key: (13) - │ │ │ │ │ │ ├── fd: ()-->(14) + │ │ │ │ │ │ ├── columns: n_nationkey:15(int!null) n_name:16(char!null) + │ │ │ │ │ │ ├── stats: [rows=1, distinct(15)=1, null(15)=0, distinct(16)=1, null(16)=0] + │ │ │ │ │ │ ├── key: (15) + │ │ │ │ │ │ ├── fd: ()-->(16) │ │ │ │ │ │ ├── scan nation │ │ │ │ │ │ │ ├── save-table-name: q11_scan_9 - │ │ │ │ │ │ │ ├── columns: n_nationkey:13(int!null) n_name:14(char!null) - │ │ │ │ │ │ │ ├── stats: [rows=25, distinct(13)=25, null(13)=0, distinct(14)=25, null(14)=0] - │ │ │ │ │ │ │ │ histogram(13)= 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 + │ │ │ │ │ │ │ ├── columns: n_nationkey:15(int!null) n_name:16(char!null) + │ │ │ │ │ │ │ ├── stats: [rows=25, distinct(15)=25, null(15)=0, distinct(16)=25, null(16)=0] + │ │ │ │ │ │ │ │ histogram(15)= 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 │ │ │ │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 --- 5 --- 6 --- 7 --- 8 --- 9 --- 10 --- 11 --- 12 --- 13 --- 14 --- 15 --- 16 --- 17 --- 18 --- 19 --- 20 --- 21 --- 22 --- 23 --- 24 - │ │ │ │ │ │ │ ├── key: (13) - │ │ │ │ │ │ │ └── fd: (13)-->(14) + │ │ │ │ │ │ │ ├── key: (15) + │ │ │ │ │ │ │ └── fd: (15)-->(16) │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── n_name:14 = 'GERMANY' [type=bool, outer=(14), constraints=(/14: [/'GERMANY' - /'GERMANY']; tight), fd=()-->(14)] + │ │ │ │ │ │ └── n_name:16 = 'GERMANY' [type=bool, outer=(16), constraints=(/16: [/'GERMANY' - /'GERMANY']; tight), fd=()-->(16)] │ │ │ │ │ └── filters (true) │ │ │ │ └── filters (true) │ │ │ └── filters (true) │ │ └── projections - │ │ └── ps_supplycost:4 * ps_availqty:3::FLOAT8 [as=column17:17, type=float, outer=(3,4), immutable] + │ │ └── ps_supplycost:4 * ps_availqty:3::FLOAT8 [as=column20:20, type=float, outer=(3,4), immutable] │ └── aggregations - │ └── sum [as=sum:18, type=float, outer=(17)] - │ └── column17:17 [type=float] + │ └── sum [as=sum:21, type=float, outer=(20)] + │ └── column20:20 [type=float] └── filters - └── gt [type=bool, outer=(18), immutable, subquery, constraints=(/18: (/NULL - ])] - ├── sum:18 [type=float] + └── gt [type=bool, outer=(21), immutable, subquery, constraints=(/21: (/NULL - ])] + ├── sum:21 [type=float] └── subquery [type=float] └── project ├── save-table-name: q11_project_10 - ├── columns: "?column?":37(float) + ├── columns: "?column?":43(float) ├── cardinality: [1 - 1] ├── immutable - ├── stats: [rows=1, distinct(37)=1, null(37)=0] + ├── stats: [rows=1, distinct(43)=1, null(43)=0] ├── key: () - ├── fd: ()-->(37) + ├── fd: ()-->(43) ├── scalar-group-by │ ├── save-table-name: q11_scalar_group_by_11 - │ ├── columns: sum:36(float) + │ ├── columns: sum:42(float) │ ├── cardinality: [1 - 1] │ ├── immutable - │ ├── stats: [rows=1, distinct(36)=1, null(36)=0] + │ ├── stats: [rows=1, distinct(42)=1, null(42)=0] │ ├── key: () - │ ├── fd: ()-->(36) + │ ├── fd: ()-->(42) │ ├── project │ │ ├── save-table-name: q11_project_12 - │ │ ├── columns: column35:35(float!null) + │ │ ├── columns: column41:41(float!null) │ │ ├── immutable - │ │ ├── stats: [rows=32258.0645, distinct(35)=31617.9161, null(35)=0] + │ │ ├── stats: [rows=32258.0645, distinct(41)=31617.9161, null(41)=0] │ │ ├── inner-join (lookup partsupp) │ │ │ ├── save-table-name: q11_lookup_join_13 - │ │ │ ├── columns: ps_suppkey:20(int!null) ps_availqty:21(int!null) ps_supplycost:22(float!null) s_suppkey:24(int!null) s_nationkey:27(int!null) n_nationkey:31(int!null) n_name:32(char!null) - │ │ │ ├── key columns: [19 20] = [19 20] + │ │ │ ├── columns: ps_suppkey:23(int!null) ps_availqty:24(int!null) ps_supplycost:25(float!null) s_suppkey:28(int!null) s_nationkey:31(int!null) n_nationkey:36(int!null) n_name:37(char!null) + │ │ │ ├── key columns: [22 23] = [22 23] │ │ │ ├── lookup columns are key - │ │ │ ├── stats: [rows=32258.0645, distinct(20)=399.934613, null(20)=0, distinct(21)=9536.12259, null(21)=0, distinct(22)=27589.3232, null(22)=0, distinct(24)=399.934613, null(24)=0, distinct(27)=1, null(27)=0, distinct(31)=1, null(31)=0, distinct(32)=1, null(32)=0, distinct(21,22)=31617.9161, null(21,22)=0] - │ │ │ ├── fd: ()-->(32), (24)-->(27), (27)==(31), (31)==(27), (20)==(24), (24)==(20) + │ │ │ ├── stats: [rows=32258.0645, distinct(23)=399.934613, null(23)=0, distinct(24)=9536.12259, null(24)=0, distinct(25)=27589.3232, null(25)=0, distinct(28)=399.934613, null(28)=0, distinct(31)=1, null(31)=0, distinct(36)=1, null(36)=0, distinct(37)=1, null(37)=0, distinct(24,25)=31617.9161, null(24,25)=0] + │ │ │ ├── fd: ()-->(37), (28)-->(31), (31)==(36), (36)==(31), (23)==(28), (28)==(23) │ │ │ ├── inner-join (lookup partsupp@ps_sk) │ │ │ │ ├── save-table-name: q11_lookup_join_14 - │ │ │ │ ├── columns: ps_partkey:19(int!null) ps_suppkey:20(int!null) s_suppkey:24(int!null) s_nationkey:27(int!null) n_nationkey:31(int!null) n_name:32(char!null) - │ │ │ │ ├── key columns: [24] = [20] - │ │ │ │ ├── stats: [rows=32258.0645, distinct(19)=29783.4869, null(19)=0, distinct(20)=399.934613, null(20)=0, distinct(24)=399.934613, null(24)=0, distinct(27)=1, null(27)=0, distinct(31)=1, null(31)=0, distinct(32)=1, null(32)=0] - │ │ │ │ ├── key: (19,24) - │ │ │ │ ├── fd: ()-->(32), (24)-->(27), (27)==(31), (31)==(27), (20)==(24), (24)==(20) + │ │ │ │ ├── columns: ps_partkey:22(int!null) ps_suppkey:23(int!null) s_suppkey:28(int!null) s_nationkey:31(int!null) n_nationkey:36(int!null) n_name:37(char!null) + │ │ │ │ ├── key columns: [28] = [23] + │ │ │ │ ├── stats: [rows=32258.0645, distinct(22)=29783.4869, null(22)=0, distinct(23)=399.934613, null(23)=0, distinct(28)=399.934613, null(28)=0, distinct(31)=1, null(31)=0, distinct(36)=1, null(36)=0, distinct(37)=1, null(37)=0] + │ │ │ │ ├── key: (22,28) + │ │ │ │ ├── fd: ()-->(37), (28)-->(31), (31)==(36), (36)==(31), (23)==(28), (28)==(23) │ │ │ │ ├── inner-join (lookup supplier@s_nk) │ │ │ │ │ ├── save-table-name: q11_lookup_join_15 - │ │ │ │ │ ├── columns: s_suppkey:24(int!null) s_nationkey:27(int!null) n_nationkey:31(int!null) n_name:32(char!null) - │ │ │ │ │ ├── key columns: [31] = [27] - │ │ │ │ │ ├── stats: [rows=400, distinct(24)=399.934613, null(24)=0, distinct(27)=1, null(27)=0, distinct(31)=1, null(31)=0, distinct(32)=1, null(32)=0] - │ │ │ │ │ ├── key: (24) - │ │ │ │ │ ├── fd: ()-->(32), (24)-->(27), (27)==(31), (31)==(27) + │ │ │ │ │ ├── columns: s_suppkey:28(int!null) s_nationkey:31(int!null) n_nationkey:36(int!null) n_name:37(char!null) + │ │ │ │ │ ├── key columns: [36] = [31] + │ │ │ │ │ ├── stats: [rows=400, distinct(28)=399.934613, null(28)=0, distinct(31)=1, null(31)=0, distinct(36)=1, null(36)=0, distinct(37)=1, null(37)=0] + │ │ │ │ │ ├── key: (28) + │ │ │ │ │ ├── fd: ()-->(37), (28)-->(31), (31)==(36), (36)==(31) │ │ │ │ │ ├── select │ │ │ │ │ │ ├── save-table-name: q11_select_16 - │ │ │ │ │ │ ├── columns: n_nationkey:31(int!null) n_name:32(char!null) - │ │ │ │ │ │ ├── stats: [rows=1, distinct(31)=1, null(31)=0, distinct(32)=1, null(32)=0] - │ │ │ │ │ │ ├── key: (31) - │ │ │ │ │ │ ├── fd: ()-->(32) + │ │ │ │ │ │ ├── columns: n_nationkey:36(int!null) n_name:37(char!null) + │ │ │ │ │ │ ├── stats: [rows=1, distinct(36)=1, null(36)=0, distinct(37)=1, null(37)=0] + │ │ │ │ │ │ ├── key: (36) + │ │ │ │ │ │ ├── fd: ()-->(37) │ │ │ │ │ │ ├── scan nation │ │ │ │ │ │ │ ├── save-table-name: q11_scan_17 - │ │ │ │ │ │ │ ├── columns: n_nationkey:31(int!null) n_name:32(char!null) - │ │ │ │ │ │ │ ├── stats: [rows=25, distinct(31)=25, null(31)=0, distinct(32)=25, null(32)=0] - │ │ │ │ │ │ │ │ histogram(31)= 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 + │ │ │ │ │ │ │ ├── columns: n_nationkey:36(int!null) n_name:37(char!null) + │ │ │ │ │ │ │ ├── stats: [rows=25, distinct(36)=25, null(36)=0, distinct(37)=25, null(37)=0] + │ │ │ │ │ │ │ │ histogram(36)= 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 │ │ │ │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 --- 5 --- 6 --- 7 --- 8 --- 9 --- 10 --- 11 --- 12 --- 13 --- 14 --- 15 --- 16 --- 17 --- 18 --- 19 --- 20 --- 21 --- 22 --- 23 --- 24 - │ │ │ │ │ │ │ ├── key: (31) - │ │ │ │ │ │ │ └── fd: (31)-->(32) + │ │ │ │ │ │ │ ├── key: (36) + │ │ │ │ │ │ │ └── fd: (36)-->(37) │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── n_name:32 = 'GERMANY' [type=bool, outer=(32), constraints=(/32: [/'GERMANY' - /'GERMANY']; tight), fd=()-->(32)] + │ │ │ │ │ │ └── n_name:37 = 'GERMANY' [type=bool, outer=(37), constraints=(/37: [/'GERMANY' - /'GERMANY']; tight), fd=()-->(37)] │ │ │ │ │ └── filters (true) │ │ │ │ └── filters (true) │ │ │ └── filters (true) │ │ └── projections - │ │ └── ps_supplycost:22 * ps_availqty:21::FLOAT8 [as=column35:35, type=float, outer=(21,22), immutable] + │ │ └── ps_supplycost:25 * ps_availqty:24::FLOAT8 [as=column41:41, type=float, outer=(24,25), immutable] │ └── aggregations - │ └── sum [as=sum:36, type=float, outer=(35)] - │ └── column35:35 [type=float] + │ └── sum [as=sum:42, type=float, outer=(41)] + │ └── column41:41 [type=float] └── projections - └── sum:36 * 0.0001 [as="?column?":37, type=float, outer=(36), immutable] + └── sum:42 * 0.0001 [as="?column?":43, type=float, outer=(42), immutable] stats table=q11_sort_1 ---- @@ -223,11 +223,11 @@ column_names row_count_est row_count_err distinct_count_est distinct_count_e stats table=q11_project_4 ---- column_names row_count distinct_count null_count -{column17} 31680 31888 0 +{column20} 31680 31888 0 {ps_partkey} 31680 29669 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{column17} 32258.00 1.02 31618.00 1.01 0.00 1.00 +{column20} 32258.00 1.02 31618.00 1.01 0.00 1.00 {ps_partkey} 32258.00 1.02 29783.00 1.00 0.00 1.00 stats table=q11_lookup_join_5 @@ -323,10 +323,10 @@ column_names row_count_est row_count_err distinct_count_est distinct_count_e stats table=q11_project_12 ---- column_names row_count distinct_count null_count -{column35} 31680 31888 0 +{column41} 31680 31888 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{column35} 32258.00 1.02 31618.00 1.01 0.00 1.00 +{column41} 32258.00 1.02 31618.00 1.01 0.00 1.00 stats table=q11_lookup_join_13 ---- diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q12 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q12 index 1b601a77959a..cf585d2216af 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q12 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q12 @@ -50,63 +50,63 @@ ORDER BY ---- sort ├── save-table-name: q12_sort_1 - ├── columns: l_shipmode:24(char!null) high_line_count:27(decimal!null) low_line_count:29(decimal!null) - ├── stats: [rows=2, distinct(24)=2, null(24)=0, distinct(27)=2, null(27)=0, distinct(29)=2, null(29)=0] - ├── key: (24) - ├── fd: (24)-->(27,29) - ├── ordering: +24 + ├── columns: l_shipmode:25(char!null) high_line_count:29(decimal!null) low_line_count:31(decimal!null) + ├── stats: [rows=2, distinct(25)=2, null(25)=0, distinct(29)=2, null(29)=0, distinct(31)=2, null(31)=0] + ├── key: (25) + ├── fd: (25)-->(29,31) + ├── ordering: +25 └── group-by ├── save-table-name: q12_group_by_2 - ├── columns: l_shipmode:24(char!null) sum:27(decimal!null) sum:29(decimal!null) - ├── grouping columns: l_shipmode:24(char!null) - ├── stats: [rows=2, distinct(24)=2, null(24)=0, distinct(27)=2, null(27)=0, distinct(29)=2, null(29)=0] - ├── key: (24) - ├── fd: (24)-->(27,29) + ├── columns: l_shipmode:25(char!null) sum:29(decimal!null) sum:31(decimal!null) + ├── grouping columns: l_shipmode:25(char!null) + ├── stats: [rows=2, distinct(25)=2, null(25)=0, distinct(29)=2, null(29)=0, distinct(31)=2, null(31)=0] + ├── key: (25) + ├── fd: (25)-->(29,31) ├── project │ ├── save-table-name: q12_project_3 - │ ├── columns: column26:26(int!null) column28:28(int!null) l_shipmode:24(char!null) - │ ├── stats: [rows=29822.8417, distinct(24)=2, null(24)=0, distinct(26)=5, null(26)=0, distinct(28)=5, null(28)=0] + │ ├── columns: column28:28(int!null) column30:30(int!null) l_shipmode:25(char!null) + │ ├── stats: [rows=29822.8417, distinct(25)=2, null(25)=0, distinct(28)=5, null(28)=0, distinct(30)=5, null(30)=0] │ ├── inner-join (lookup orders) │ │ ├── save-table-name: q12_lookup_join_4 - │ │ ├── columns: o_orderkey:1(int!null) o_orderpriority:6(char!null) l_orderkey:10(int!null) l_shipdate:20(date!null) l_commitdate:21(date!null) l_receiptdate:22(date!null) l_shipmode:24(char!null) - │ │ ├── key columns: [10] = [1] + │ │ ├── columns: o_orderkey:1(int!null) o_orderpriority:6(char!null) l_orderkey:11(int!null) l_shipdate:21(date!null) l_commitdate:22(date!null) l_receiptdate:23(date!null) l_shipmode:25(char!null) + │ │ ├── key columns: [11] = [1] │ │ ├── lookup columns are key - │ │ ├── stats: [rows=29822.8417, distinct(1)=29606.4627, null(1)=0, distinct(6)=5, null(6)=0, distinct(10)=29606.4627, null(10)=0, distinct(20)=2525.98116, null(20)=0, distinct(21)=2465.9862, null(21)=0, distinct(22)=365, null(22)=0, distinct(24)=2, null(24)=0] - │ │ ├── fd: (1)-->(6), (1)==(10), (10)==(1) + │ │ ├── stats: [rows=29822.8417, distinct(1)=29606.4627, null(1)=0, distinct(6)=5, null(6)=0, distinct(11)=29606.4627, null(11)=0, distinct(21)=2525.98116, null(21)=0, distinct(22)=2465.9862, null(22)=0, distinct(23)=365, null(23)=0, distinct(25)=2, null(25)=0] + │ │ ├── fd: (1)-->(6), (1)==(11), (11)==(1) │ │ ├── select │ │ │ ├── save-table-name: q12_select_5 - │ │ │ ├── columns: l_orderkey:10(int!null) l_shipdate:20(date!null) l_commitdate:21(date!null) l_receiptdate:22(date!null) l_shipmode:24(char!null) - │ │ │ ├── stats: [rows=29822.8417, distinct(10)=29606.4627, null(10)=0, distinct(20)=2526, null(20)=0, distinct(21)=2466, null(21)=0, distinct(22)=365, null(22)=0, distinct(24)=2, null(24)=0, distinct(22,24)=730, null(22,24)=0] - │ │ │ │ histogram(22)= 0 0 826.55 57.145 876.38 95.242 895.43 38.097 895.43 76.193 876.38 95.242 914.48 57.145 838.29 133.34 914.48 57.145 781.11 152.39 914.48 38.097 857.33 95.242 838.29 228.61 819.24 152.39 914.48 133.34 914.48 114.29 800.19 133.34 857.33 152.39 876.38 114.29 895.43 133.34 914.48 57.145 857.33 76.193 914.48 57.145 857.33 95.242 895.43 76.193 895.43 76.193 857.33 76.193 819.24 114.29 838.29 114.29 914.48 114.29 819.24 133.34 609.65 76.207 + │ │ │ ├── columns: l_orderkey:11(int!null) l_shipdate:21(date!null) l_commitdate:22(date!null) l_receiptdate:23(date!null) l_shipmode:25(char!null) + │ │ │ ├── stats: [rows=29822.8417, distinct(11)=29606.4627, null(11)=0, distinct(21)=2526, null(21)=0, distinct(22)=2466, null(22)=0, distinct(23)=365, null(23)=0, distinct(25)=2, null(25)=0, distinct(23,25)=730, null(23,25)=0] + │ │ │ │ histogram(23)= 0 0 826.55 57.145 876.38 95.242 895.43 38.097 895.43 76.193 876.38 95.242 914.48 57.145 838.29 133.34 914.48 57.145 781.11 152.39 914.48 38.097 857.33 95.242 838.29 228.61 819.24 152.39 914.48 133.34 914.48 114.29 800.19 133.34 857.33 152.39 876.38 114.29 895.43 133.34 914.48 57.145 857.33 76.193 914.48 57.145 857.33 95.242 895.43 76.193 895.43 76.193 857.33 76.193 819.24 114.29 838.29 114.29 914.48 114.29 819.24 133.34 609.65 76.207 │ │ │ │ <--- '1993-12-31' -------- '1994-01-13' -------- '1994-01-26' -------- '1994-02-08' -------- '1994-02-19' -------- '1994-03-04' -------- '1994-03-14' -------- '1994-03-26' -------- '1994-04-08' -------- '1994-04-21' -------- '1994-05-03' -------- '1994-05-14' -------- '1994-05-24' -------- '1994-06-03' -------- '1994-06-15' -------- '1994-06-27' -------- '1994-07-07' -------- '1994-07-16' -------- '1994-07-26' -------- '1994-08-10' -------- '1994-08-22' -------- '1994-09-03' -------- '1994-09-15' -------- '1994-09-27' -------- '1994-10-11' -------- '1994-10-22' -------- '1994-11-02' -------- '1994-11-14' -------- '1994-11-26' -------- '1994-12-10' -------- '1994-12-22' -------- '1994-12-31' │ │ │ ├── index-join lineitem │ │ │ │ ├── save-table-name: q12_index_join_6 - │ │ │ │ ├── columns: l_orderkey:10(int!null) l_shipdate:20(date!null) l_commitdate:21(date!null) l_receiptdate:22(date!null) l_shipmode:24(char!null) - │ │ │ │ ├── stats: [rows=939419.512, distinct(10)=744923.472, null(10)=0, distinct(20)=2526, null(20)=0, distinct(21)=2466, null(21)=0, distinct(22)=365, null(22)=0, distinct(24)=7, null(24)=0] + │ │ │ │ ├── columns: l_orderkey:11(int!null) l_shipdate:21(date!null) l_commitdate:22(date!null) l_receiptdate:23(date!null) l_shipmode:25(char!null) + │ │ │ │ ├── stats: [rows=939419.512, distinct(11)=744923.472, null(11)=0, distinct(21)=2526, null(21)=0, distinct(22)=2466, null(22)=0, distinct(23)=365, null(23)=0, distinct(25)=7, null(25)=0] │ │ │ │ └── scan lineitem@l_rd │ │ │ │ ├── save-table-name: q12_scan_7 - │ │ │ │ ├── columns: l_orderkey:10(int!null) l_linenumber:13(int!null) l_receiptdate:22(date!null) - │ │ │ │ ├── constraint: /22/10/13: [/'1994-01-01' - /'1994-12-31'] - │ │ │ │ ├── stats: [rows=939419.512, distinct(10)=744923.472, null(10)=0, distinct(13)=7, null(13)=0, distinct(22)=365, null(22)=0] - │ │ │ │ │ histogram(10)= 0 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 + │ │ │ │ ├── columns: l_orderkey:11(int!null) l_linenumber:14(int!null) l_receiptdate:23(date!null) + │ │ │ │ ├── constraint: /23/11/14: [/'1994-01-01' - /'1994-12-31'] + │ │ │ │ ├── stats: [rows=939419.512, distinct(11)=744923.472, null(11)=0, distinct(14)=7, null(14)=0, distinct(23)=365, null(23)=0] + │ │ │ │ │ histogram(11)= 0 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4603 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 4697.1 93.923 │ │ │ │ │ <--- 326 ------- 28929 ------ 50503 ------ 89793 ------ 115938 ------ 146944 ------ 176768 ------ 211201 ------ 237860 ------ 266885 ------ 297604 ------ 330021 ------ 365889 ------ 398951 ------ 426117 ------ 451328 ------ 472134 ------ 499590 ------ 529284 ------ 557254 ------ 589154 ------ 619394 ------ 642951 ------ 670113 ------ 692931 ------ 721157 ------ 751687 ------ 777766 ------ 804582 ------ 836740 ------ 868868 ------ 898912 ------ 922500 ------ 946403 ------ 984870 ------ 1007936 ------ 1030117 ------ 1062275 ------ 1093572 ------ 1120709 ------ 1150981 ------ 1182786 ------ 1206406 ------ 1234116 ------ 1260961 ------ 1290502 ------ 1329510 ------ 1355426 ------ 1381313 ------ 1409796 ------ 1445254 ------ 1479233 ------ 1504935 ------ 1531079 ------ 1559650 ------ 1583616 ------ 1617504 ------ 1655749 ------ 1685185 ------ 1718183 ------ 1747716 ------ 1772131 ------ 1802372 ------ 1833315 ------ 1862403 ------ 1897894 ------ 1922819 ------ 1954405 ------ 1979329 ------ 2009859 ------ 2041670 ------ 2070851 ------ 2093828 ------ 2127973 ------ 2167777 ------ 2194883 ------ 2227814 ------ 2262437 ------ 2296353 ------ 2321024 ------ 2346051 ------ 2376257 ------ 2404932 ------ 2446273 ------ 2474081 ------ 2504515 ------ 2535302 ------ 2561413 ------ 2592737 ------ 2616801 ------ 2646112 ------ 2676546 ------ 2702116 ------ 2732454 ------ 2765382 ------ 2799495 ------ 2828866 ------ 2868737 ------ 2910625 ------ 2938464 ------ 2963140 ------ 3003302 ------ 3043264 ------ 3069123 ------ 3095909 ------ 3126693 ------ 3160485 ------ 3196039 ------ 3229504 ------ 3259712 ------ 3286439 ------ 3318852 ------ 3346821 ------ 3370119 ------ 3395204 ------ 3425888 ------ 3448611 ------ 3476130 ------ 3502372 ------ 3529474 ------ 3556390 ------ 3583553 ------ 3612550 ------ 3647875 ------ 3679140 ------ 3702661 ------ 3738017 ------ 3778050 ------ 3806114 ------ 3839074 ------ 3872805 ------ 3905697 ------ 3926212 ------ 3959841 ------ 3997281 ------ 4033861 ------ 4063591 ------ 4097831 ------ 4124807 ------ 4158656 ------ 4195748 ------ 4234274 ------ 4269952 ------ 4298949 ------ 4332806 ------ 4364705 ------ 4398246 ------ 4430695 ------ 4466403 ------ 4494662 ------ 4524420 -------- 4558561 -------- 4601092 -------- 4632871 -------- 4658694 -------- 4690501 -------- 4728066 -------- 4758657 -------- 4788294 -------- 4818597 -------- 4855874 -------- 4890913 -------- 4915366 -------- 4940709 -------- 4972357 -------- 4995298 -------- 5019523 -------- 5043329 -------- 5077376 -------- 5109920 -------- 5136582 -------- 5161152 -------- 5191846 -------- 5219973 -------- 5251015 -------- 5282021 -------- 5312355 -------- 5343207 -------- 5381318 -------- 5416163 -------- 5445382 -------- 5476933 -------- 5509185 -------- 5539237 -------- 5566818 -------- 5588739 -------- 5620481 -------- 5644001 -------- 5667010 -------- 5689476 -------- 5724709 -------- 5755398 -------- 5790598 -------- 5819425 -------- 5846341 -------- 5874656 -------- 5908067 -------- 5933572 -------- 5962659 -------- 5999971 - │ │ │ │ │ histogram(22)= 0 0 26035 1800 27605 3000 28205 1200 28205 2400 27605 3000 28805 1800 26405 4200 28805 1800 24604 4800 28805 1200 27005 3000 26405 7201 25805 4800 28805 4200 28805 3600 25205 4200 27005 4800 27605 3600 28205 4200 28805 1800 27005 2400 28805 1800 27005 3000 28205 2400 28205 2400 27005 2400 25805 3600 26405 3600 28805 3600 25805 4200 19203 2400.4 + │ │ │ │ │ histogram(23)= 0 0 26035 1800 27605 3000 28205 1200 28205 2400 27605 3000 28805 1800 26405 4200 28805 1800 24604 4800 28805 1200 27005 3000 26405 7201 25805 4800 28805 4200 28805 3600 25205 4200 27005 4800 27605 3600 28205 4200 28805 1800 27005 2400 28805 1800 27005 3000 28205 2400 28205 2400 27005 2400 25805 3600 26405 3600 28805 3600 25805 4200 19203 2400.4 │ │ │ │ │ <--- '1993-12-31' ------- '1994-01-13' ------- '1994-01-26' ------- '1994-02-08' ------- '1994-02-19' ------- '1994-03-04' ------- '1994-03-14' ------- '1994-03-26' ------- '1994-04-08' ------- '1994-04-21' ------- '1994-05-03' ------- '1994-05-14' ------- '1994-05-24' ------- '1994-06-03' ------- '1994-06-15' ------- '1994-06-27' ------- '1994-07-07' ------- '1994-07-16' ------- '1994-07-26' ------- '1994-08-10' ------- '1994-08-22' ------- '1994-09-03' ------- '1994-09-15' ------- '1994-09-27' ------- '1994-10-11' ------- '1994-10-22' ------- '1994-11-02' ------- '1994-11-14' ------- '1994-11-26' ------- '1994-12-10' ------- '1994-12-22' ------- '1994-12-31' - │ │ │ │ ├── key: (10,13) - │ │ │ │ └── fd: (10,13)-->(22) + │ │ │ │ ├── key: (11,14) + │ │ │ │ └── fd: (11,14)-->(23) │ │ │ └── filters - │ │ │ ├── l_shipmode:24 IN ('MAIL', 'SHIP') [type=bool, outer=(24), constraints=(/24: [/'MAIL' - /'MAIL'] [/'SHIP' - /'SHIP']; tight)] - │ │ │ ├── l_commitdate:21 < l_receiptdate:22 [type=bool, outer=(21,22), constraints=(/21: (/NULL - ]; /22: (/NULL - ])] - │ │ │ └── l_shipdate:20 < l_commitdate:21 [type=bool, outer=(20,21), constraints=(/20: (/NULL - ]; /21: (/NULL - ])] + │ │ │ ├── l_shipmode:25 IN ('MAIL', 'SHIP') [type=bool, outer=(25), constraints=(/25: [/'MAIL' - /'MAIL'] [/'SHIP' - /'SHIP']; tight)] + │ │ │ ├── l_commitdate:22 < l_receiptdate:23 [type=bool, outer=(22,23), constraints=(/22: (/NULL - ]; /23: (/NULL - ])] + │ │ │ └── l_shipdate:21 < l_commitdate:22 [type=bool, outer=(21,22), constraints=(/21: (/NULL - ]; /22: (/NULL - ])] │ │ └── filters (true) │ └── projections - │ ├── CASE WHEN (o_orderpriority:6 = '1-URGENT') OR (o_orderpriority:6 = '2-HIGH') THEN 1 ELSE 0 END [as=column26:26, type=int, outer=(6)] - │ └── CASE WHEN (o_orderpriority:6 != '1-URGENT') AND (o_orderpriority:6 != '2-HIGH') THEN 1 ELSE 0 END [as=column28:28, type=int, outer=(6)] + │ ├── CASE WHEN (o_orderpriority:6 = '1-URGENT') OR (o_orderpriority:6 = '2-HIGH') THEN 1 ELSE 0 END [as=column28:28, type=int, outer=(6)] + │ └── CASE WHEN (o_orderpriority:6 != '1-URGENT') AND (o_orderpriority:6 != '2-HIGH') THEN 1 ELSE 0 END [as=column30:30, type=int, outer=(6)] └── aggregations - ├── sum [as=sum:27, type=decimal, outer=(26)] - │ └── column26:26 [type=int] - └── sum [as=sum:29, type=decimal, outer=(28)] - └── column28:28 [type=int] + ├── sum [as=sum:29, type=decimal, outer=(28)] + │ └── column28:28 [type=int] + └── sum [as=sum:31, type=decimal, outer=(30)] + └── column30:30 [type=int] stats table=q12_sort_1 ---- @@ -135,13 +135,13 @@ column_names row_count_est row_count_err distinct_count_est distinct_count_e stats table=q12_project_3 ---- column_names row_count distinct_count null_count -{column26} 30988 2 0 {column28} 30988 2 0 +{column30} 30988 2 0 {l_shipmode} 30988 2 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{column26} 29823.00 1.04 5.00 2.50 <== 0.00 1.00 {column28} 29823.00 1.04 5.00 2.50 <== 0.00 1.00 +{column30} 29823.00 1.04 5.00 2.50 <== 0.00 1.00 {l_shipmode} 29823.00 1.04 2.00 1.00 0.00 1.00 stats table=q12_lookup_join_4 diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q13 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q13 index bf737fc4b6f0..3a8d162c2a61 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q13 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q13 @@ -37,49 +37,49 @@ ORDER BY ---- sort ├── save-table-name: q13_sort_1 - ├── columns: c_count:18(int!null) custdist:19(int!null) - ├── stats: [rows=148813, distinct(18)=148813, null(18)=0, distinct(19)=148813, null(19)=0] - ├── key: (18) - ├── fd: (18)-->(19) - ├── ordering: -19,-18 + ├── columns: c_count:20(int!null) custdist:21(int!null) + ├── stats: [rows=148813, distinct(20)=148813, null(20)=0, distinct(21)=148813, null(21)=0] + ├── key: (20) + ├── fd: (20)-->(21) + ├── ordering: -21,-20 └── group-by ├── save-table-name: q13_group_by_2 - ├── columns: count:18(int!null) count_rows:19(int!null) - ├── grouping columns: count:18(int!null) - ├── stats: [rows=148813, distinct(18)=148813, null(18)=0, distinct(19)=148813, null(19)=0] - ├── key: (18) - ├── fd: (18)-->(19) + ├── columns: count:20(int!null) count_rows:21(int!null) + ├── grouping columns: count:20(int!null) + ├── stats: [rows=148813, distinct(20)=148813, null(20)=0, distinct(21)=148813, null(21)=0] + ├── key: (20) + ├── fd: (20)-->(21) ├── group-by │ ├── save-table-name: q13_group_by_3 - │ ├── columns: c_custkey:1(int!null) count:18(int!null) + │ ├── columns: c_custkey:1(int!null) count:20(int!null) │ ├── grouping columns: c_custkey:1(int!null) - │ ├── stats: [rows=148813, distinct(1)=148813, null(1)=0, distinct(18)=148813, null(18)=0] + │ ├── stats: [rows=148813, distinct(1)=148813, null(1)=0, distinct(20)=148813, null(20)=0] │ ├── key: (1) - │ ├── fd: (1)-->(18) + │ ├── fd: (1)-->(20) │ ├── right-join (hash) │ │ ├── save-table-name: q13_right_join_4 - │ │ ├── columns: c_custkey:1(int!null) o_orderkey:9(int) o_custkey:10(int) o_comment:17(varchar) - │ │ ├── stats: [rows=503988.227, distinct(1)=148813, null(1)=0, distinct(9)=317522.248, null(9)=0, distinct(10)=99620.1148, null(10)=0, distinct(17)=317522.248, null(17)=0] - │ │ ├── key: (1,9) - │ │ ├── fd: (9)-->(10,17) + │ │ ├── columns: c_custkey:1(int!null) o_orderkey:10(int) o_custkey:11(int) o_comment:18(varchar) + │ │ ├── stats: [rows=503988.227, distinct(1)=148813, null(1)=0, distinct(10)=317522.248, null(10)=0, distinct(11)=99620.1148, null(11)=0, distinct(18)=317522.248, null(18)=0] + │ │ ├── key: (1,10) + │ │ ├── fd: (10)-->(11,18) │ │ ├── select │ │ │ ├── save-table-name: q13_select_5 - │ │ │ ├── columns: o_orderkey:9(int!null) o_custkey:10(int!null) o_comment:17(varchar!null) - │ │ │ ├── stats: [rows=500000, distinct(9)=500000, null(9)=0, distinct(10)=99620.1148, null(10)=0, distinct(17)=500000, null(17)=0] - │ │ │ ├── key: (9) - │ │ │ ├── fd: (9)-->(10,17) + │ │ │ ├── columns: o_orderkey:10(int!null) o_custkey:11(int!null) o_comment:18(varchar!null) + │ │ │ ├── stats: [rows=500000, distinct(10)=500000, null(10)=0, distinct(11)=99620.1148, null(11)=0, distinct(18)=500000, null(18)=0] + │ │ │ ├── key: (10) + │ │ │ ├── fd: (10)-->(11,18) │ │ │ ├── scan orders │ │ │ │ ├── save-table-name: q13_scan_6 - │ │ │ │ ├── columns: o_orderkey:9(int!null) o_custkey:10(int!null) o_comment:17(varchar!null) - │ │ │ │ ├── stats: [rows=1500000, distinct(9)=1500000, null(9)=0, distinct(10)=99846, null(10)=0, distinct(17)=1469402, null(17)=0] - │ │ │ │ │ histogram(9)= 0 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 - │ │ │ │ │ <--- 998 ------ 26593 ------ 56613 ------ 85827 ------ 115587 ------ 140512 ------ 167076 ------ 198882 ------ 228580 ------ 262243 ------ 292064 ------ 316161 ------ 344194 ------ 368550 ------ 402722 ------ 434529 ------ 468807 ------ 495107 ------ 526338 ------ 563239 ------ 597733 ------ 641894 ------ 665730 ------ 688742 ------ 721767 ------ 750050 ------ 787879 ------ 814565 ------ 837381 ------ 866657 ------ 902855 ------ 927172 ------ 952772 ------ 975840 ------ 1008646 ------ 1032066 ------ 1074656 ------ 1104896 ------ 1138369 ------ 1166689 ------ 1194912 ------ 1225222 ------ 1253284 ------ 1284803 ------ 1318499 ------ 1348609 ------ 1376609 ------ 1404933 ------ 1435745 ------ 1460325 ------ 1491648 ------ 1518852 ------ 1542567 ------ 1574277 ------ 1597958 ------ 1624964 ------ 1647555 ------ 1683651 ------ 1717189 ------ 1750563 ------ 1784775 ------ 1812449 ------ 1838850 ------ 1867751 ------ 1891783 ------ 1918790 ------ 1949827 ------ 1986021 ------ 2020134 ------ 2045829 ------ 2077894 ------ 2118912 ------ 2143264 ------ 2176486 ------ 2204838 ------ 2234146 ------ 2260484 ------ 2288512 ------ 2317121 ------ 2344321 ------ 2362567 ------ 2396133 ------ 2423623 ------ 2454884 ------ 2483879 ------ 2520162 ------ 2545574 ------ 2571874 ------ 2608417 ------ 2637477 ------ 2665607 ------ 2692231 ------ 2724261 ------ 2764451 ------ 2798656 ------ 2824775 ------ 2858690 ------ 2881990 ------ 2911814 ------ 2945056 ------ 2981315 ------ 3010306 ------ 3032320 ------ 3064389 ------ 3084166 ------ 3108832 ------ 3136032 ------ 3168484 ------ 3204039 ------ 3229922 ------ 3256549 ------ 3283169 ------ 3315236 ------ 3346756 ------ 3375879 ------ 3408007 ------ 3441827 ------ 3471204 ------ 3505414 ------ 3527938 ------ 3562561 ------ 3598630 ------ 3627270 ------ 3659266 ------ 3686468 ------ 3713505 ------ 3750817 ------ 3777696 ------ 3808129 ------ 3834533 ------ 3871968 ------ 3904931 ------ 3934215 ------ 3960643 ------ 3990336 ------ 4023203 ------ 4057826 ------ 4090691 ------ 4121216 ------ 4159681 ------ 4190726 ------ 4222624 ------ 4246567 ------ 4283653 ------ 4314339 ------ 4342592 ------ 4373829 ------ 4402372 ------ 4431332 ------ 4464452 ------ 4496327 ------ 4526789 ------ 4551109 ------ 4582401 ------ 4615974 ------ 4648482 ------ 4669601 ------ 4705891 ------ 4732869 ------ 4766145 ------ 4805862 ------ 4840677 ------ 4866787 ------ 4895332 ------ 4928100 ------ 4956323 ------ 4992161 ------ 5027008 ------ 5065797 ------ 5099011 ------ 5130592 ------ 5163463 ------ 5197314 ------ 5224994 ------ 5253892 ------ 5277638 ------ 5307105 ------ 5335750 ------ 5369828 ------ 5402528 ------ 5434183 ------ 5464227 ------ 5491072 ------ 5520679 ------ 5543047 ------ 5576708 ------ 5614951 ------ 5646055 ------ 5674721 ------ 5700295 ------ 5732066 ------ 5761255 ------ 5791233 ------ 5819651 ------ 5852291 ------ 5880258 ------ 5909062 ------ 5943111 ------ 5973926 ------ 5998752 - │ │ │ │ │ histogram(10)= 0 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7200 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7200 300 7350 150 7350 150 7350 150 7350 150 7200 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7200 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 300 7350 150 7500 150 7500 150 7500 150 7500 150 7350 300 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7350 300 7500 150 7500 150 7500 150 7500 150 7500 150 7350 300 7500 150 + │ │ │ │ ├── columns: o_orderkey:10(int!null) o_custkey:11(int!null) o_comment:18(varchar!null) + │ │ │ │ ├── stats: [rows=1500000, distinct(10)=1500000, null(10)=0, distinct(11)=99846, null(11)=0, distinct(18)=1469402, null(18)=0] + │ │ │ │ │ histogram(10)= 0 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 + │ │ │ │ │ <--- 998 ------ 26593 ------ 56613 ------ 85827 ------ 115587 ------ 140512 ------ 167076 ------ 198882 ------ 228580 ------ 262243 ------ 292064 ------ 316161 ------ 344194 ------ 368550 ------ 402722 ------ 434529 ------ 468807 ------ 495107 ------ 526338 ------ 563239 ------ 597733 ------ 641894 ------ 665730 ------ 688742 ------ 721767 ------ 750050 ------ 787879 ------ 814565 ------ 837381 ------ 866657 ------ 902855 ------ 927172 ------ 952772 ------ 975840 ------ 1008646 ------ 1032066 ------ 1074656 ------ 1104896 ------ 1138369 ------ 1166689 ------ 1194912 ------ 1225222 ------ 1253284 ------ 1284803 ------ 1318499 ------ 1348609 ------ 1376609 ------ 1404933 ------ 1435745 ------ 1460325 ------ 1491648 ------ 1518852 ------ 1542567 ------ 1574277 ------ 1597958 ------ 1624964 ------ 1647555 ------ 1683651 ------ 1717189 ------ 1750563 ------ 1784775 ------ 1812449 ------ 1838850 ------ 1867751 ------ 1891783 ------ 1918790 ------ 1949827 ------ 1986021 ------ 2020134 ------ 2045829 ------ 2077894 ------ 2118912 ------ 2143264 ------ 2176486 ------ 2204838 ------ 2234146 ------ 2260484 ------ 2288512 ------ 2317121 ------ 2344321 ------ 2362567 ------ 2396133 ------ 2423623 ------ 2454884 ------ 2483879 ------ 2520162 ------ 2545574 ------ 2571874 ------ 2608417 ------ 2637477 ------ 2665607 ------ 2692231 ------ 2724261 ------ 2764451 ------ 2798656 ------ 2824775 ------ 2858690 ------ 2881990 ------ 2911814 ------ 2945056 ------ 2981315 ------ 3010306 ------ 3032320 ------ 3064389 ------ 3084166 ------ 3108832 ------ 3136032 ------ 3168484 ------ 3204039 ------ 3229922 ------ 3256549 ------ 3283169 ------ 3315236 ------ 3346756 ------ 3375879 ------ 3408007 ------ 3441827 ------ 3471204 ------ 3505414 ------ 3527938 ------ 3562561 ------ 3598630 ------ 3627270 ------ 3659266 ------ 3686468 ------ 3713505 ------ 3750817 ------ 3777696 ------ 3808129 ------ 3834533 ------ 3871968 ------ 3904931 ------ 3934215 ------ 3960643 ------ 3990336 ------ 4023203 ------ 4057826 ------ 4090691 ------ 4121216 ------ 4159681 ------ 4190726 ------ 4222624 ------ 4246567 ------ 4283653 ------ 4314339 ------ 4342592 ------ 4373829 ------ 4402372 ------ 4431332 ------ 4464452 ------ 4496327 ------ 4526789 ------ 4551109 ------ 4582401 ------ 4615974 ------ 4648482 ------ 4669601 ------ 4705891 ------ 4732869 ------ 4766145 ------ 4805862 ------ 4840677 ------ 4866787 ------ 4895332 ------ 4928100 ------ 4956323 ------ 4992161 ------ 5027008 ------ 5065797 ------ 5099011 ------ 5130592 ------ 5163463 ------ 5197314 ------ 5224994 ------ 5253892 ------ 5277638 ------ 5307105 ------ 5335750 ------ 5369828 ------ 5402528 ------ 5434183 ------ 5464227 ------ 5491072 ------ 5520679 ------ 5543047 ------ 5576708 ------ 5614951 ------ 5646055 ------ 5674721 ------ 5700295 ------ 5732066 ------ 5761255 ------ 5791233 ------ 5819651 ------ 5852291 ------ 5880258 ------ 5909062 ------ 5943111 ------ 5973926 ------ 5998752 + │ │ │ │ │ histogram(11)= 0 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7200 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7200 300 7350 150 7350 150 7350 150 7350 150 7200 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7200 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 300 7350 150 7500 150 7500 150 7500 150 7500 150 7350 300 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7350 300 7500 150 7500 150 7500 150 7500 150 7500 150 7350 300 7500 150 │ │ │ │ │ <--- 8 ------ 763 ------ 1477 ------ 2353 ------ 3229 ------ 4261 ------ 5191 ------ 5924 ------ 6406 ------ 7126 ------ 7870 ------ 8545 ------ 9301 ------ 9911 ------ 10591 ------ 11236 ------ 11854 ------ 12622 ------ 13426 ------ 14036 ------ 14708 ------ 15391 ------ 16138 ------ 16925 ------ 17681 ------ 18388 ------ 19156 ------ 20015 ------ 20660 ------ 21632 ------ 22472 ------ 23260 ------ 24055 ------ 24649 ------ 25352 ------ 25952 ------ 26774 ------ 27358 ------ 28070 ------ 28586 ------ 29296 ------ 30028 ------ 30629 ------ 31400 ------ 32102 ------ 32884 ------ 33796 ------ 34646 ------ 35479 ------ 36161 ------ 36811 ------ 37612 ------ 38314 ------ 39145 ------ 39844 ------ 40591 ------ 41254 ------ 41902 ------ 42496 ------ 43237 ------ 43948 ------ 44690 ------ 45481 ------ 46448 ------ 47242 ------ 47884 ------ 48676 ------ 49441 ------ 50155 ------ 50857 ------ 51463 ------ 52321 ------ 53014 ------ 53587 ------ 54413 ------ 55019 ------ 55771 ------ 56482 ------ 57314 ------ 58163 ------ 58894 ------ 59638 ------ 60478 ------ 61450 ------ 62417 ------ 63176 ------ 63944 ------ 64577 ------ 65344 ------ 66218 ------ 67021 ------ 67703 ------ 68477 ------ 69271 ------ 70073 ------ 70696 ------ 71512 ------ 72430 ------ 73345 ------ 74149 ------ 74780 ------ 75508 ------ 76273 ------ 76865 ------ 77666 ------ 78427 ------ 79042 ------ 79801 ------ 80557 ------ 81163 ------ 81833 ------ 82534 ------ 83368 ------ 84157 ------ 84938 ------ 85495 ------ 86257 ------ 87143 ------ 87998 ------ 88777 ------ 89566 ------ 90292 ------ 91099 ------ 91757 ------ 92401 ------ 93362 ------ 94252 ------ 94771 ------ 95527 ------ 96233 ------ 96952 ------ 97844 ------ 98411 ------ 99067 ------ 99691 ------ 100357 ------ 101009 ------ 101734 ------ 102569 ------ 103381 ------ 104114 ------ 105070 ------ 105884 ------ 106651 ------ 107393 ------ 107995 ------ 108772 ------ 109628 ------ 110383 ------ 111040 ------ 111658 ------ 112556 ------ 113338 ------ 114062 ------ 114895 ------ 115744 ------ 116554 ------ 117250 ------ 117967 ------ 118741 ------ 119540 ------ 120514 ------ 121333 ------ 122111 ------ 122989 ------ 123763 ------ 124735 ------ 125401 ------ 126022 ------ 126779 ------ 127618 ------ 128332 ------ 129167 ------ 129917 ------ 130669 ------ 131330 ------ 131911 ------ 132704 ------ 133553 ------ 134317 ------ 134960 ------ 135688 ------ 136519 ------ 137353 ------ 138061 ------ 138697 ------ 139468 ------ 140218 ------ 140942 ------ 141644 ------ 142415 ------ 143263 ------ 144352 ------ 145099 ------ 145811 ------ 146761 ------ 147643 ------ 148591 ------ 149164 ------ 149995 - │ │ │ │ ├── key: (9) - │ │ │ │ └── fd: (9)-->(10,17) + │ │ │ │ ├── key: (10) + │ │ │ │ └── fd: (10)-->(11,18) │ │ │ └── filters - │ │ │ └── o_comment:17 NOT LIKE '%special%requests%' [type=bool, outer=(17), constraints=(/17: (/NULL - ])] + │ │ │ └── o_comment:18 NOT LIKE '%special%requests%' [type=bool, outer=(18), constraints=(/18: (/NULL - ])] │ │ ├── scan customer@c_nk │ │ │ ├── save-table-name: q13_scan_7 │ │ │ ├── columns: c_custkey:1(int!null) @@ -88,12 +88,12 @@ sort │ │ │ │ <--- 2 ----- 776 ----- 1422 ----- 2189 ----- 2973 ----- 3583 ----- 4390 ----- 5154 ----- 5962 ----- 6965 ----- 7596 ----- 8303 ----- 9167 ----- 9833 ----- 10695 ----- 11397 ----- 11979 ----- 12651 ----- 13397 ----- 14144 ----- 14951 ----- 15698 ----- 16460 ----- 17203 ----- 17846 ----- 18462 ----- 19390 ----- 20189 ----- 20852 ----- 21642 ----- 22379 ----- 23009 ----- 23856 ----- 24734 ----- 25473 ----- 26231 ----- 26978 ----- 27654 ----- 28276 ----- 29054 ----- 29727 ----- 30527 ----- 31177 ----- 32126 ----- 32984 ----- 33684 ----- 34316 ----- 35070 ----- 35703 ----- 36397 ----- 37156 ----- 37709 ----- 38488 ----- 39131 ----- 39740 ----- 40736 ----- 41459 ----- 42388 ----- 42999 ----- 43844 ----- 44571 ----- 45428 ----- 46283 ----- 46979 ----- 47712 ----- 48708 ----- 49487 ----- 50275 ----- 51131 ----- 51836 ----- 52652 ----- 53389 ----- 54179 ----- 54861 ----- 55609 ----- 56492 ----- 57284 ----- 57917 ----- 58793 ----- 59665 ----- 60285 ----- 60840 ----- 61523 ----- 62354 ----- 63178 ----- 63933 ----- 64642 ----- 65282 ----- 65864 ----- 66755 ----- 67407 ----- 68099 ----- 68875 ----- 69638 ----- 70304 ----- 71016 ----- 71830 ----- 72742 ----- 73590 ----- 74434 ----- 75274 ----- 75861 ----- 76547 ----- 77252 ----- 77978 ----- 78650 ----- 79313 ----- 79925 ----- 80677 ----- 81497 ----- 82205 ----- 82962 ----- 83879 ----- 84815 ----- 85521 ----- 86272 ----- 87140 ----- 87759 ----- 88634 ----- 89452 ----- 90192 ----- 90920 ----- 91756 ----- 92690 ----- 93299 ----- 93950 ----- 94812 ----- 95569 ----- 96295 ----- 96904 ----- 97499 ----- 98144 ----- 98764 ----- 99582 ----- 100453 ----- 101098 ----- 101892 ----- 102700 ----- 103419 ----- 104297 ----- 105040 ----- 105864 ----- 106498 ----- 107196 ----- 108022 ----- 108731 ----- 109398 ----- 110145 ----- 110849 ----- 111758 ----- 112501 ----- 113222 ----- 114019 ----- 114904 ----- 115693 ----- 116350 ----- 116955 ----- 117581 ----- 118366 ----- 119159 ----- 119902 ----- 120535 ----- 121321 ----- 121993 ----- 122769 ----- 123504 ----- 124225 ----- 124992 ----- 125632 ----- 126685 ----- 127641 ----- 128303 ----- 129042 ----- 129589 ----- 130548 ----- 131374 ----- 132325 ----- 133042 ----- 133883 ----- 134716 ----- 135520 ----- 136173 ----- 136858 ----- 137584 ----- 138381 ----- 139162 ----- 139923 ----- 140738 ----- 141557 ----- 142287 ----- 143002 ----- 143794 ----- 144420 ----- 145276 ----- 146100 ----- 146977 ----- 147821 ----- 148440 ----- 149247 ----- 149978 │ │ │ └── key: (1) │ │ └── filters - │ │ └── c_custkey:1 = o_custkey:10 [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] + │ │ └── c_custkey:1 = o_custkey:11 [type=bool, outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ]), fd=(1)==(11), (11)==(1)] │ └── aggregations - │ └── count [as=count:18, type=int, outer=(9)] - │ └── o_orderkey:9 [type=int] + │ └── count [as=count:20, type=int, outer=(10)] + │ └── o_orderkey:10 [type=int] └── aggregations - └── count-rows [as=count_rows:19, type=int] + └── count-rows [as=count_rows:21, type=int] stats table=q13_sort_1 ---- diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q14 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q14 index 560d0d8c28be..7bb735d803c0 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q14 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q14 @@ -32,39 +32,39 @@ WHERE ---- project ├── save-table-name: q14_project_1 - ├── columns: promo_revenue:30(float) + ├── columns: promo_revenue:32(float) ├── cardinality: [1 - 1] ├── immutable - ├── stats: [rows=1, distinct(30)=1, null(30)=0] + ├── stats: [rows=1, distinct(32)=1, null(32)=0] ├── key: () - ├── fd: ()-->(30) + ├── fd: ()-->(32) ├── scalar-group-by │ ├── save-table-name: q14_scalar_group_by_2 - │ ├── columns: sum:27(float) sum:29(float) + │ ├── columns: sum:29(float) sum:31(float) │ ├── cardinality: [1 - 1] │ ├── immutable - │ ├── stats: [rows=1, distinct(27)=1, null(27)=0, distinct(29)=1, null(29)=0, distinct(27,29)=1, null(27,29)=0] + │ ├── stats: [rows=1, distinct(29)=1, null(29)=0, distinct(31)=1, null(31)=0, distinct(29,31)=1, null(29,31)=0] │ ├── key: () - │ ├── fd: ()-->(27,29) + │ ├── fd: ()-->(29,31) │ ├── project │ │ ├── save-table-name: q14_project_3 - │ │ ├── columns: column26:26(float!null) column28:28(float!null) + │ │ ├── columns: column28:28(float!null) column30:30(float!null) │ │ ├── immutable - │ │ ├── stats: [rows=82726.8788, distinct(26)=82726.8788, null(26)=0, distinct(28)=52210.2591, null(28)=0] + │ │ ├── stats: [rows=82726.8788, distinct(28)=82726.8788, null(28)=0, distinct(30)=52210.2591, null(30)=0] │ │ ├── inner-join (hash) │ │ │ ├── save-table-name: q14_inner_join_4 - │ │ │ ├── columns: l_partkey:2(int!null) l_extendedprice:6(float!null) l_discount:7(float!null) l_shipdate:11(date!null) p_partkey:17(int!null) p_type:21(varchar!null) + │ │ │ ├── columns: l_partkey:2(int!null) l_extendedprice:6(float!null) l_discount:7(float!null) l_shipdate:11(date!null) p_partkey:18(int!null) p_type:22(varchar!null) │ │ │ ├── multiplicity: left-rows(zero-or-more), right-rows(exactly-one) - │ │ │ ├── stats: [rows=82726.8788, distinct(2)=67871.4531, null(2)=0, distinct(6)=51381.824, null(6)=0, distinct(7)=11, null(7)=0, distinct(11)=30, null(11)=0, distinct(17)=67871.4531, null(17)=0, distinct(21)=150, null(21)=0, distinct(6,7)=52210.2591, null(6,7)=0, distinct(6,7,21)=82726.8788, null(6,7,21)=0] - │ │ │ ├── fd: (17)-->(21), (2)==(17), (17)==(2) + │ │ │ ├── stats: [rows=82726.8788, distinct(2)=67871.4531, null(2)=0, distinct(6)=51381.824, null(6)=0, distinct(7)=11, null(7)=0, distinct(11)=30, null(11)=0, distinct(18)=67871.4531, null(18)=0, distinct(22)=150, null(22)=0, distinct(6,7)=52210.2591, null(6,7)=0, distinct(6,7,22)=82726.8788, null(6,7,22)=0] + │ │ │ ├── fd: (18)-->(22), (2)==(18), (18)==(2) │ │ │ ├── scan part │ │ │ │ ├── save-table-name: q14_scan_5 - │ │ │ │ ├── columns: p_partkey:17(int!null) p_type:21(varchar!null) - │ │ │ │ ├── stats: [rows=200000, distinct(17)=199241, null(17)=0, distinct(21)=150, null(21)=0] - │ │ │ │ │ histogram(17)= 0 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 + │ │ │ │ ├── columns: p_partkey:18(int!null) p_type:22(varchar!null) + │ │ │ │ ├── stats: [rows=200000, distinct(18)=199241, null(18)=0, distinct(22)=150, null(22)=0] + │ │ │ │ │ histogram(18)= 0 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 │ │ │ │ │ <--- 27 ----- 1110 ----- 2241 ----- 3086 ----- 4134 ----- 5302 ----- 6222 ----- 7308 ----- 8249 ----- 9171 ----- 10049 ----- 10958 ----- 11832 ----- 13025 ----- 14063 ----- 14953 ----- 16249 ----- 17419 ----- 18363 ----- 19412 ----- 20257 ----- 21190 ----- 22110 ----- 23045 ----- 23956 ----- 24962 ----- 25942 ----- 26990 ----- 27934 ----- 28876 ----- 29513 ----- 30326 ----- 31259 ----- 32300 ----- 33577 ----- 34550 ----- 35562 ----- 36498 ----- 37475 ----- 38584 ----- 39641 ----- 40548 ----- 41605 ----- 42527 ----- 43612 ----- 44702 ----- 45701 ----- 46726 ----- 47795 ----- 48935 ----- 50152 ----- 51183 ----- 52001 ----- 52904 ----- 53868 ----- 54808 ----- 55986 ----- 57155 ----- 58516 ----- 59526 ----- 60557 ----- 61547 ----- 62369 ----- 63672 ----- 64583 ----- 65360 ----- 66147 ----- 67201 ----- 68142 ----- 69145 ----- 70209 ----- 71141 ----- 71923 ----- 73031 ----- 73987 ----- 74974 ----- 76170 ----- 77138 ----- 77849 ----- 78931 ----- 79832 ----- 80761 ----- 81843 ----- 82834 ----- 84032 ----- 85072 ----- 86287 ----- 87302 ----- 88422 ----- 89432 ----- 90550 ----- 91463 ----- 92249 ----- 93385 ----- 94789 ----- 96013 ----- 96893 ----- 98000 ----- 99008 ----- 100166 ----- 101263 ----- 102351 ----- 103236 ----- 104121 ----- 105363 ----- 106329 ----- 107325 ----- 108231 ----- 109054 ----- 110019 ----- 111185 ----- 112112 ----- 112908 ----- 113904 ----- 114785 ----- 115410 ----- 116526 ----- 117559 ----- 118310 ----- 119073 ----- 120034 ----- 120817 ----- 121744 ----- 122566 ----- 123720 ----- 124813 ----- 125835 ----- 126622 ----- 127651 ----- 128328 ----- 129315 ----- 130244 ----- 131450 ----- 132439 ----- 133288 ----- 134164 ----- 135298 ----- 136347 ----- 137243 ----- 138256 ----- 139427 ----- 140374 ----- 141371 ----- 142302 ----- 143322 ----- 144335 ----- 145333 ----- 146212 ----- 147321 ----- 148591 ----- 149594 ------ 150514 ------ 151361 ------ 152059 ------ 153070 ------ 154059 ------ 155259 ------ 156473 ------ 157690 ------ 158703 ------ 159675 ------ 160597 ------ 161668 ------ 162737 ------ 163955 ------ 164942 ------ 165924 ------ 167059 ------ 167866 ------ 169034 ------ 169935 ------ 170712 ------ 171806 ------ 172841 ------ 174078 ------ 175347 ------ 176430 ------ 177346 ------ 178566 ------ 179515 ------ 180677 ------ 181729 ------ 182983 ------ 183814 ------ 184892 ------ 185696 ------ 186611 ------ 187744 ------ 188974 ------ 189911 ------ 190671 ------ 191607 ------ 192820 ------ 193789 ------ 195057 ------ 196224 ------ 197231 ------ 198281 ------ 199119 ------ 199999 - │ │ │ │ ├── key: (17) - │ │ │ │ └── fd: (17)-->(21) + │ │ │ │ ├── key: (18) + │ │ │ │ └── fd: (18)-->(22) │ │ │ ├── index-join lineitem │ │ │ │ ├── save-table-name: q14_index_join_6 │ │ │ │ ├── columns: l_partkey:2(int!null) l_extendedprice:6(float!null) l_discount:7(float!null) l_shipdate:11(date!null) @@ -83,17 +83,17 @@ project │ │ │ │ ├── key: (1,4) │ │ │ │ └── fd: (1,4)-->(11) │ │ │ └── filters - │ │ │ └── l_partkey:2 = p_partkey:17 [type=bool, outer=(2,17), constraints=(/2: (/NULL - ]; /17: (/NULL - ]), fd=(2)==(17), (17)==(2)] + │ │ │ └── l_partkey:2 = p_partkey:18 [type=bool, outer=(2,18), constraints=(/2: (/NULL - ]; /18: (/NULL - ]), fd=(2)==(18), (18)==(2)] │ │ └── projections - │ │ ├── CASE WHEN p_type:21 LIKE 'PROMO%' THEN l_extendedprice:6 * (1.0 - l_discount:7) ELSE 0.0 END [as=column26:26, type=float, outer=(6,7,21), immutable] - │ │ └── l_extendedprice:6 * (1.0 - l_discount:7) [as=column28:28, type=float, outer=(6,7), immutable] + │ │ ├── CASE WHEN p_type:22 LIKE 'PROMO%' THEN l_extendedprice:6 * (1.0 - l_discount:7) ELSE 0.0 END [as=column28:28, type=float, outer=(6,7,22), immutable] + │ │ └── l_extendedprice:6 * (1.0 - l_discount:7) [as=column30:30, type=float, outer=(6,7), immutable] │ └── aggregations - │ ├── sum [as=sum:27, type=float, outer=(26)] - │ │ └── column26:26 [type=float] - │ └── sum [as=sum:29, type=float, outer=(28)] - │ └── column28:28 [type=float] + │ ├── sum [as=sum:29, type=float, outer=(28)] + │ │ └── column28:28 [type=float] + │ └── sum [as=sum:31, type=float, outer=(30)] + │ └── column30:30 [type=float] └── projections - └── (sum:27 * 100.0) / sum:29 [as=promo_revenue:30, type=float, outer=(27,29), immutable] + └── (sum:29 * 100.0) / sum:31 [as=promo_revenue:32, type=float, outer=(29,31), immutable] stats table=q14_project_1 ---- @@ -116,12 +116,12 @@ column_names row_count_est row_count_err distinct_count_est distinct_count_e stats table=q14_project_3 ---- column_names row_count distinct_count null_count -{column26} 75983 12638 0 -{column28} 75983 76207 0 +{column28} 75983 12638 0 +{column30} 75983 76207 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{column26} 82727.00 1.09 82727.00 6.55 <== 0.00 1.00 -{column28} 82727.00 1.09 52210.00 1.46 0.00 1.00 +{column28} 82727.00 1.09 82727.00 6.55 <== 0.00 1.00 +{column30} 82727.00 1.09 52210.00 1.46 0.00 1.00 stats table=q14_inner_join_4 ---- diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q15 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q15 index e18e4f141b92..89e53ceee9e0 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q15 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q15 @@ -52,22 +52,22 @@ ORDER BY ---- project ├── save-table-name: q15_project_1 - ├── columns: s_suppkey:1(int!null) s_name:2(char!null) s_address:3(varchar!null) s_phone:5(char!null) total_revenue:25(float!null) + ├── columns: s_suppkey:1(int!null) s_name:2(char!null) s_address:3(varchar!null) s_phone:5(char!null) total_revenue:27(float!null) ├── immutable - ├── stats: [rows=3333.33333, distinct(1)=3306.66667, null(1)=0, distinct(2)=2834.3606, null(2)=0, distinct(3)=2834.80729, null(3)=0, distinct(5)=2834.80729, null(5)=0, distinct(25)=2100.04396, null(25)=0] + ├── stats: [rows=3333.33333, distinct(1)=3306.66667, null(1)=0, distinct(2)=2834.3606, null(2)=0, distinct(3)=2834.80729, null(3)=0, distinct(5)=2834.80729, null(5)=0, distinct(27)=2100.04396, null(27)=0] ├── key: (1) - ├── fd: (1)-->(2,3,5,25) + ├── fd: (1)-->(2,3,5,27) ├── ordering: +1 └── inner-join (merge) ├── save-table-name: q15_merge_join_2 - ├── columns: s_suppkey:1(int!null) s_name:2(char!null) s_address:3(varchar!null) s_phone:5(char!null) l_suppkey:10(int!null) sum:25(float!null) + ├── columns: s_suppkey:1(int!null) s_name:2(char!null) s_address:3(varchar!null) s_phone:5(char!null) l_suppkey:11(int!null) sum:27(float!null) ├── left ordering: +1 - ├── right ordering: +10 + ├── right ordering: +11 ├── immutable - ├── stats: [rows=3333.33333, distinct(1)=3306.66667, null(1)=0, distinct(2)=2834.3606, null(2)=0, distinct(3)=2834.80729, null(3)=0, distinct(5)=2834.80729, null(5)=0, distinct(10)=3306.66667, null(10)=0, distinct(25)=2100.04396, null(25)=0] - ├── key: (10) - ├── fd: (1)-->(2,3,5), (10)-->(25), (1)==(10), (10)==(1) - ├── ordering: +(1|10) [actual: +1] + ├── stats: [rows=3333.33333, distinct(1)=3306.66667, null(1)=0, distinct(2)=2834.3606, null(2)=0, distinct(3)=2834.80729, null(3)=0, distinct(5)=2834.80729, null(5)=0, distinct(11)=3306.66667, null(11)=0, distinct(27)=2100.04396, null(27)=0] + ├── key: (11) + ├── fd: (1)-->(2,3,5), (11)-->(27), (1)==(11), (11)==(1) + ├── ordering: +(1|11) [actual: +1] ├── scan supplier │ ├── save-table-name: q15_scan_3 │ ├── columns: s_suppkey:1(int!null) s_name:2(char!null) s_address:3(varchar!null) s_phone:5(char!null) @@ -79,104 +79,104 @@ project │ └── ordering: +1 ├── sort │ ├── save-table-name: q15_sort_4 - │ ├── columns: l_suppkey:10(int!null) sum:25(float!null) + │ ├── columns: l_suppkey:11(int!null) sum:27(float!null) │ ├── immutable - │ ├── stats: [rows=3306.66667, distinct(10)=3306.66667, null(10)=0, distinct(25)=3306.66667, null(25)=0] - │ ├── key: (10) - │ ├── fd: (10)-->(25) - │ ├── ordering: +10 + │ ├── stats: [rows=3306.66667, distinct(11)=3306.66667, null(11)=0, distinct(27)=3306.66667, null(27)=0] + │ ├── key: (11) + │ ├── fd: (11)-->(27) + │ ├── ordering: +11 │ └── select │ ├── save-table-name: q15_select_5 - │ ├── columns: l_suppkey:10(int!null) sum:25(float!null) + │ ├── columns: l_suppkey:11(int!null) sum:27(float!null) │ ├── immutable - │ ├── stats: [rows=3306.66667, distinct(10)=3306.66667, null(10)=0, distinct(25)=3306.66667, null(25)=0] - │ ├── key: (10) - │ ├── fd: (10)-->(25) + │ ├── stats: [rows=3306.66667, distinct(11)=3306.66667, null(11)=0, distinct(27)=3306.66667, null(27)=0] + │ ├── key: (11) + │ ├── fd: (11)-->(27) │ ├── group-by │ │ ├── save-table-name: q15_group_by_6 - │ │ ├── columns: l_suppkey:10(int!null) sum:25(float!null) - │ │ ├── grouping columns: l_suppkey:10(int!null) + │ │ ├── columns: l_suppkey:11(int!null) sum:27(float!null) + │ │ ├── grouping columns: l_suppkey:11(int!null) │ │ ├── immutable - │ │ ├── stats: [rows=9920, distinct(10)=9920, null(10)=0, distinct(25)=9920, null(25)=0] - │ │ ├── key: (10) - │ │ ├── fd: (10)-->(25) + │ │ ├── stats: [rows=9920, distinct(11)=9920, null(11)=0, distinct(27)=9920, null(27)=0] + │ │ ├── key: (11) + │ │ ├── fd: (11)-->(27) │ │ ├── project │ │ │ ├── save-table-name: q15_project_7 - │ │ │ ├── columns: column24:24(float!null) l_suppkey:10(int!null) + │ │ │ ├── columns: column26:26(float!null) l_suppkey:11(int!null) │ │ │ ├── immutable - │ │ │ ├── stats: [rows=259635.063, distinct(10)=9920, null(10)=0, distinct(24)=259635.063, null(24)=0] + │ │ │ ├── stats: [rows=259635.063, distinct(11)=9920, null(11)=0, distinct(26)=259635.063, null(26)=0] │ │ │ ├── index-join lineitem │ │ │ │ ├── save-table-name: q15_index_join_8 - │ │ │ │ ├── columns: l_suppkey:10(int!null) l_extendedprice:13(float!null) l_discount:14(float!null) l_shipdate:18(date!null) - │ │ │ │ ├── stats: [rows=259635.063, distinct(10)=9920, null(10)=0, distinct(13)=230767.055, null(13)=0, distinct(14)=11, null(14)=0, distinct(18)=91, null(18)=0, distinct(13,14)=259635.063, null(13,14)=0] - │ │ │ │ │ histogram(18)= 0 0 28205 2400 25805 3600 26405 4800 25805 3000 27005 3600 27005 3000 27605 3600 27005 5401 12820 2564.1 + │ │ │ │ ├── columns: l_suppkey:11(int!null) l_extendedprice:14(float!null) l_discount:15(float!null) l_shipdate:19(date!null) + │ │ │ │ ├── stats: [rows=259635.063, distinct(11)=9920, null(11)=0, distinct(14)=230767.055, null(14)=0, distinct(15)=11, null(15)=0, distinct(19)=91, null(19)=0, distinct(14,15)=259635.063, null(14,15)=0] + │ │ │ │ │ histogram(19)= 0 0 28205 2400 25805 3600 26405 4800 25805 3000 27005 3600 27005 3000 27605 3600 27005 5401 12820 2564.1 │ │ │ │ │ <--- '1995-12-31' ------- '1996-01-12' ------- '1996-01-22' ------- '1996-02-01' ------- '1996-02-10' ------- '1996-02-21' ------- '1996-03-02' ------- '1996-03-13' ------- '1996-03-25' ------- '1996-03-31' │ │ │ │ └── scan lineitem@l_sd │ │ │ │ ├── save-table-name: q15_scan_9 - │ │ │ │ ├── columns: l_orderkey:8(int!null) l_linenumber:11(int!null) l_shipdate:18(date!null) - │ │ │ │ ├── constraint: /18/8/11: [/'1996-01-01' - /'1996-03-31'] - │ │ │ │ ├── stats: [rows=259635.063, distinct(8)=243635.718, null(8)=0, distinct(11)=7, null(11)=0, distinct(18)=91, null(18)=0] - │ │ │ │ │ histogram(8)= 0 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 + │ │ │ │ ├── columns: l_orderkey:9(int!null) l_linenumber:12(int!null) l_shipdate:19(date!null) + │ │ │ │ ├── constraint: /19/9/12: [/'1996-01-01' - /'1996-03-31'] + │ │ │ │ ├── stats: [rows=259635.063, distinct(9)=243635.718, null(9)=0, distinct(12)=7, null(12)=0, distinct(19)=91, null(19)=0] + │ │ │ │ │ histogram(9)= 0 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 │ │ │ │ │ <--- 326 --------- 28929 -------- 50503 -------- 89793 -------- 115938 -------- 146944 -------- 176768 -------- 211201 -------- 237860 -------- 266885 -------- 297604 -------- 330021 -------- 365889 -------- 398951 -------- 426117 -------- 451328 -------- 472134 -------- 499590 -------- 529284 -------- 557254 -------- 589154 -------- 619394 -------- 642951 -------- 670113 -------- 692931 -------- 721157 -------- 751687 -------- 777766 -------- 804582 -------- 836740 -------- 868868 -------- 898912 -------- 922500 -------- 946403 -------- 984870 -------- 1007936 -------- 1030117 -------- 1062275 -------- 1093572 -------- 1120709 -------- 1150981 -------- 1182786 -------- 1206406 -------- 1234116 -------- 1260961 -------- 1290502 -------- 1329510 -------- 1355426 -------- 1381313 -------- 1409796 -------- 1445254 -------- 1479233 -------- 1504935 -------- 1531079 -------- 1559650 -------- 1583616 -------- 1617504 -------- 1655749 -------- 1685185 -------- 1718183 -------- 1747716 -------- 1772131 -------- 1802372 -------- 1833315 -------- 1862403 -------- 1897894 -------- 1922819 -------- 1954405 -------- 1979329 -------- 2009859 -------- 2041670 -------- 2070851 -------- 2093828 -------- 2127973 -------- 2167777 -------- 2194883 -------- 2227814 -------- 2262437 -------- 2296353 -------- 2321024 -------- 2346051 -------- 2376257 -------- 2404932 -------- 2446273 -------- 2474081 -------- 2504515 -------- 2535302 -------- 2561413 -------- 2592737 -------- 2616801 -------- 2646112 -------- 2676546 -------- 2702116 -------- 2732454 -------- 2765382 -------- 2799495 -------- 2828866 -------- 2868737 -------- 2910625 -------- 2938464 -------- 2963140 -------- 3003302 -------- 3043264 -------- 3069123 -------- 3095909 -------- 3126693 -------- 3160485 -------- 3196039 -------- 3229504 -------- 3259712 -------- 3286439 -------- 3318852 -------- 3346821 -------- 3370119 -------- 3395204 -------- 3425888 -------- 3448611 -------- 3476130 -------- 3502372 -------- 3529474 -------- 3556390 -------- 3583553 -------- 3612550 -------- 3647875 -------- 3679140 -------- 3702661 -------- 3738017 -------- 3778050 -------- 3806114 -------- 3839074 -------- 3872805 -------- 3905697 -------- 3926212 -------- 3959841 -------- 3997281 -------- 4033861 -------- 4063591 -------- 4097831 -------- 4124807 -------- 4158656 -------- 4195748 -------- 4234274 -------- 4269952 -------- 4298949 -------- 4332806 -------- 4364705 -------- 4398246 -------- 4430695 -------- 4466403 -------- 4494662 -------- 4524420 -------- 4558561 -------- 4601092 -------- 4632871 -------- 4658694 -------- 4690501 -------- 4728066 -------- 4758657 -------- 4788294 -------- 4818597 -------- 4855874 -------- 4890913 -------- 4915366 -------- 4940709 -------- 4972357 -------- 4995298 -------- 5019523 -------- 5043329 -------- 5077376 -------- 5109920 -------- 5136582 -------- 5161152 -------- 5191846 -------- 5219973 -------- 5251015 -------- 5282021 -------- 5312355 -------- 5343207 -------- 5381318 -------- 5416163 -------- 5445382 -------- 5476933 -------- 5509185 -------- 5539237 -------- 5566818 -------- 5588739 -------- 5620481 -------- 5644001 -------- 5667010 -------- 5689476 -------- 5724709 -------- 5755398 -------- 5790598 -------- 5819425 -------- 5846341 -------- 5874656 -------- 5908067 -------- 5933572 -------- 5962659 -------- 5999971 - │ │ │ │ │ histogram(18)= 0 0 28205 2400 25805 3600 26405 4800 25805 3000 27005 3600 27005 3000 27605 3600 27005 5401 12820 2564.1 + │ │ │ │ │ histogram(19)= 0 0 28205 2400 25805 3600 26405 4800 25805 3000 27005 3600 27005 3000 27605 3600 27005 5401 12820 2564.1 │ │ │ │ │ <--- '1995-12-31' ------- '1996-01-12' ------- '1996-01-22' ------- '1996-02-01' ------- '1996-02-10' ------- '1996-02-21' ------- '1996-03-02' ------- '1996-03-13' ------- '1996-03-25' ------- '1996-03-31' - │ │ │ │ ├── key: (8,11) - │ │ │ │ └── fd: (8,11)-->(18) + │ │ │ │ ├── key: (9,12) + │ │ │ │ └── fd: (9,12)-->(19) │ │ │ └── projections - │ │ │ └── l_extendedprice:13 * (1.0 - l_discount:14) [as=column24:24, type=float, outer=(13,14), immutable] + │ │ │ └── l_extendedprice:14 * (1.0 - l_discount:15) [as=column26:26, type=float, outer=(14,15), immutable] │ │ └── aggregations - │ │ └── sum [as=sum:25, type=float, outer=(24)] - │ │ └── column24:24 [type=float] + │ │ └── sum [as=sum:27, type=float, outer=(26)] + │ │ └── column26:26 [type=float] │ └── filters - │ └── eq [type=bool, outer=(25), immutable, subquery, constraints=(/25: (/NULL - ])] - │ ├── sum:25 [type=float] + │ └── eq [type=bool, outer=(27), immutable, subquery, constraints=(/27: (/NULL - ])] + │ ├── sum:27 [type=float] │ └── subquery [type=float] │ └── scalar-group-by │ ├── save-table-name: q15_scalar_group_by_10 - │ ├── columns: max:44(float) + │ ├── columns: max:47(float) │ ├── cardinality: [1 - 1] │ ├── immutable - │ ├── stats: [rows=1, distinct(44)=1, null(44)=0] + │ ├── stats: [rows=1, distinct(47)=1, null(47)=0] │ ├── key: () - │ ├── fd: ()-->(44) + │ ├── fd: ()-->(47) │ ├── group-by │ │ ├── save-table-name: q15_group_by_11 - │ │ ├── columns: l_suppkey:28(int!null) sum:43(float!null) - │ │ ├── grouping columns: l_suppkey:28(int!null) + │ │ ├── columns: l_suppkey:30(int!null) sum:46(float!null) + │ │ ├── grouping columns: l_suppkey:30(int!null) │ │ ├── immutable - │ │ ├── stats: [rows=9920, distinct(28)=9920, null(28)=0, distinct(43)=9920, null(43)=0] - │ │ ├── key: (28) - │ │ ├── fd: (28)-->(43) + │ │ ├── stats: [rows=9920, distinct(30)=9920, null(30)=0, distinct(46)=9920, null(46)=0] + │ │ ├── key: (30) + │ │ ├── fd: (30)-->(46) │ │ ├── project │ │ │ ├── save-table-name: q15_project_12 - │ │ │ ├── columns: column42:42(float!null) l_suppkey:28(int!null) + │ │ │ ├── columns: column45:45(float!null) l_suppkey:30(int!null) │ │ │ ├── immutable - │ │ │ ├── stats: [rows=259635.063, distinct(28)=9920, null(28)=0, distinct(42)=259635.063, null(42)=0] + │ │ │ ├── stats: [rows=259635.063, distinct(30)=9920, null(30)=0, distinct(45)=259635.063, null(45)=0] │ │ │ ├── index-join lineitem │ │ │ │ ├── save-table-name: q15_index_join_13 - │ │ │ │ ├── columns: l_suppkey:28(int!null) l_extendedprice:31(float!null) l_discount:32(float!null) l_shipdate:36(date!null) - │ │ │ │ ├── stats: [rows=259635.063, distinct(28)=9920, null(28)=0, distinct(31)=230767.055, null(31)=0, distinct(32)=11, null(32)=0, distinct(36)=91, null(36)=0, distinct(31,32)=259635.063, null(31,32)=0] - │ │ │ │ │ histogram(36)= 0 0 28205 2400 25805 3600 26405 4800 25805 3000 27005 3600 27005 3000 27605 3600 27005 5401 12820 2564.1 + │ │ │ │ ├── columns: l_suppkey:30(int!null) l_extendedprice:33(float!null) l_discount:34(float!null) l_shipdate:38(date!null) + │ │ │ │ ├── stats: [rows=259635.063, distinct(30)=9920, null(30)=0, distinct(33)=230767.055, null(33)=0, distinct(34)=11, null(34)=0, distinct(38)=91, null(38)=0, distinct(33,34)=259635.063, null(33,34)=0] + │ │ │ │ │ histogram(38)= 0 0 28205 2400 25805 3600 26405 4800 25805 3000 27005 3600 27005 3000 27605 3600 27005 5401 12820 2564.1 │ │ │ │ │ <--- '1995-12-31' ------- '1996-01-12' ------- '1996-01-22' ------- '1996-02-01' ------- '1996-02-10' ------- '1996-02-21' ------- '1996-03-02' ------- '1996-03-13' ------- '1996-03-25' ------- '1996-03-31' │ │ │ │ └── scan lineitem@l_sd │ │ │ │ ├── save-table-name: q15_scan_14 - │ │ │ │ ├── columns: l_orderkey:26(int!null) l_linenumber:29(int!null) l_shipdate:36(date!null) - │ │ │ │ ├── constraint: /36/26/29: [/'1996-01-01' - /'1996-03-31'] - │ │ │ │ ├── stats: [rows=259635.063, distinct(26)=243635.718, null(26)=0, distinct(29)=7, null(29)=0, distinct(36)=91, null(36)=0] - │ │ │ │ │ histogram(26)= 0 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 + │ │ │ │ ├── columns: l_orderkey:28(int!null) l_linenumber:31(int!null) l_shipdate:38(date!null) + │ │ │ │ ├── constraint: /38/28/31: [/'1996-01-01' - /'1996-03-31'] + │ │ │ │ ├── stats: [rows=259635.063, distinct(28)=243635.718, null(28)=0, distinct(31)=7, null(31)=0, distinct(38)=91, null(38)=0] + │ │ │ │ │ histogram(28)= 0 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1272.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 1298.2 25.958 │ │ │ │ │ <--- 326 --------- 28929 -------- 50503 -------- 89793 -------- 115938 -------- 146944 -------- 176768 -------- 211201 -------- 237860 -------- 266885 -------- 297604 -------- 330021 -------- 365889 -------- 398951 -------- 426117 -------- 451328 -------- 472134 -------- 499590 -------- 529284 -------- 557254 -------- 589154 -------- 619394 -------- 642951 -------- 670113 -------- 692931 -------- 721157 -------- 751687 -------- 777766 -------- 804582 -------- 836740 -------- 868868 -------- 898912 -------- 922500 -------- 946403 -------- 984870 -------- 1007936 -------- 1030117 -------- 1062275 -------- 1093572 -------- 1120709 -------- 1150981 -------- 1182786 -------- 1206406 -------- 1234116 -------- 1260961 -------- 1290502 -------- 1329510 -------- 1355426 -------- 1381313 -------- 1409796 -------- 1445254 -------- 1479233 -------- 1504935 -------- 1531079 -------- 1559650 -------- 1583616 -------- 1617504 -------- 1655749 -------- 1685185 -------- 1718183 -------- 1747716 -------- 1772131 -------- 1802372 -------- 1833315 -------- 1862403 -------- 1897894 -------- 1922819 -------- 1954405 -------- 1979329 -------- 2009859 -------- 2041670 -------- 2070851 -------- 2093828 -------- 2127973 -------- 2167777 -------- 2194883 -------- 2227814 -------- 2262437 -------- 2296353 -------- 2321024 -------- 2346051 -------- 2376257 -------- 2404932 -------- 2446273 -------- 2474081 -------- 2504515 -------- 2535302 -------- 2561413 -------- 2592737 -------- 2616801 -------- 2646112 -------- 2676546 -------- 2702116 -------- 2732454 -------- 2765382 -------- 2799495 -------- 2828866 -------- 2868737 -------- 2910625 -------- 2938464 -------- 2963140 -------- 3003302 -------- 3043264 -------- 3069123 -------- 3095909 -------- 3126693 -------- 3160485 -------- 3196039 -------- 3229504 -------- 3259712 -------- 3286439 -------- 3318852 -------- 3346821 -------- 3370119 -------- 3395204 -------- 3425888 -------- 3448611 -------- 3476130 -------- 3502372 -------- 3529474 -------- 3556390 -------- 3583553 -------- 3612550 -------- 3647875 -------- 3679140 -------- 3702661 -------- 3738017 -------- 3778050 -------- 3806114 -------- 3839074 -------- 3872805 -------- 3905697 -------- 3926212 -------- 3959841 -------- 3997281 -------- 4033861 -------- 4063591 -------- 4097831 -------- 4124807 -------- 4158656 -------- 4195748 -------- 4234274 -------- 4269952 -------- 4298949 -------- 4332806 -------- 4364705 -------- 4398246 -------- 4430695 -------- 4466403 -------- 4494662 -------- 4524420 -------- 4558561 -------- 4601092 -------- 4632871 -------- 4658694 -------- 4690501 -------- 4728066 -------- 4758657 -------- 4788294 -------- 4818597 -------- 4855874 -------- 4890913 -------- 4915366 -------- 4940709 -------- 4972357 -------- 4995298 -------- 5019523 -------- 5043329 -------- 5077376 -------- 5109920 -------- 5136582 -------- 5161152 -------- 5191846 -------- 5219973 -------- 5251015 -------- 5282021 -------- 5312355 -------- 5343207 -------- 5381318 -------- 5416163 -------- 5445382 -------- 5476933 -------- 5509185 -------- 5539237 -------- 5566818 -------- 5588739 -------- 5620481 -------- 5644001 -------- 5667010 -------- 5689476 -------- 5724709 -------- 5755398 -------- 5790598 -------- 5819425 -------- 5846341 -------- 5874656 -------- 5908067 -------- 5933572 -------- 5962659 -------- 5999971 - │ │ │ │ │ histogram(36)= 0 0 28205 2400 25805 3600 26405 4800 25805 3000 27005 3600 27005 3000 27605 3600 27005 5401 12820 2564.1 + │ │ │ │ │ histogram(38)= 0 0 28205 2400 25805 3600 26405 4800 25805 3000 27005 3600 27005 3000 27605 3600 27005 5401 12820 2564.1 │ │ │ │ │ <--- '1995-12-31' ------- '1996-01-12' ------- '1996-01-22' ------- '1996-02-01' ------- '1996-02-10' ------- '1996-02-21' ------- '1996-03-02' ------- '1996-03-13' ------- '1996-03-25' ------- '1996-03-31' - │ │ │ │ ├── key: (26,29) - │ │ │ │ └── fd: (26,29)-->(36) + │ │ │ │ ├── key: (28,31) + │ │ │ │ └── fd: (28,31)-->(38) │ │ │ └── projections - │ │ │ └── l_extendedprice:31 * (1.0 - l_discount:32) [as=column42:42, type=float, outer=(31,32), immutable] + │ │ │ └── l_extendedprice:33 * (1.0 - l_discount:34) [as=column45:45, type=float, outer=(33,34), immutable] │ │ └── aggregations - │ │ └── sum [as=sum:43, type=float, outer=(42)] - │ │ └── column42:42 [type=float] + │ │ └── sum [as=sum:46, type=float, outer=(45)] + │ │ └── column45:45 [type=float] │ └── aggregations - │ └── max [as=max:44, type=float, outer=(43)] - │ └── sum:43 [type=float] + │ └── max [as=max:47, type=float, outer=(46)] + │ └── sum:46 [type=float] └── filters (true) stats table=q15_project_1 @@ -260,11 +260,11 @@ column_names row_count_est row_count_err distinct_count_est distinct_count_e stats table=q15_project_7 ---- column_names row_count distinct_count null_count -{column24} 225954 220864 0 +{column26} 225954 220864 0 {l_suppkey} 225954 9920 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{column24} 259635.00 1.15 259635.00 1.18 0.00 1.00 +{column26} 259635.00 1.15 259635.00 1.18 0.00 1.00 {l_suppkey} 259635.00 1.15 9920.00 1.00 0.00 1.00 stats table=q15_index_join_8 @@ -302,11 +302,11 @@ column_names row_count_est row_count_err distinct_count_est distinct_count_e stats table=q15_project_12 ---- column_names row_count distinct_count null_count -{column42} 225954 220864 0 +{column45} 225954 220864 0 {l_suppkey} 225954 9920 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{column42} 259635.00 1.15 259635.00 1.18 0.00 1.00 +{column45} 259635.00 1.15 259635.00 1.18 0.00 1.00 {l_suppkey} 259635.00 1.15 9920.00 1.00 0.00 1.00 stats table=q15_index_join_13 diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q16 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q16 index b328b67af930..866001279cbb 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q16 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q16 @@ -52,36 +52,36 @@ ORDER BY ---- sort ├── save-table-name: q16_sort_1 - ├── columns: p_brand:9(char!null) p_type:10(varchar!null) p_size:11(int!null) supplier_cnt:22(int!null) - ├── stats: [rows=3315.43068, distinct(9)=8.33333333, null(9)=0, distinct(10)=150, null(10)=0, distinct(11)=8, null(11)=0, distinct(22)=3315.43068, null(22)=0, distinct(9-11)=3315.43068, null(9-11)=0] - ├── key: (9-11) - ├── fd: (9-11)-->(22) - ├── ordering: -22,+9,+10,+11 + ├── columns: p_brand:10(char!null) p_type:11(varchar!null) p_size:12(int!null) supplier_cnt:25(int!null) + ├── stats: [rows=3315.43068, distinct(10)=8.33333333, null(10)=0, distinct(11)=150, null(11)=0, distinct(12)=8, null(12)=0, distinct(25)=3315.43068, null(25)=0, distinct(10-12)=3315.43068, null(10-12)=0] + ├── key: (10-12) + ├── fd: (10-12)-->(25) + ├── ordering: -25,+10,+11,+12 └── group-by ├── save-table-name: q16_group_by_2 - ├── columns: p_brand:9(char!null) p_type:10(varchar!null) p_size:11(int!null) count:22(int!null) - ├── grouping columns: p_brand:9(char!null) p_type:10(varchar!null) p_size:11(int!null) - ├── stats: [rows=3315.43068, distinct(9)=8.33333333, null(9)=0, distinct(10)=150, null(10)=0, distinct(11)=8, null(11)=0, distinct(22)=3315.43068, null(22)=0, distinct(9-11)=3315.43068, null(9-11)=0] - ├── key: (9-11) - ├── fd: (9-11)-->(22) + ├── columns: p_brand:10(char!null) p_type:11(varchar!null) p_size:12(int!null) count:25(int!null) + ├── grouping columns: p_brand:10(char!null) p_type:11(varchar!null) p_size:12(int!null) + ├── stats: [rows=3315.43068, distinct(10)=8.33333333, null(10)=0, distinct(11)=150, null(11)=0, distinct(12)=8, null(12)=0, distinct(25)=3315.43068, null(25)=0, distinct(10-12)=3315.43068, null(10-12)=0] + ├── key: (10-12) + ├── fd: (10-12)-->(25) ├── distinct-on │ ├── save-table-name: q16_distinct_on_3 - │ ├── columns: ps_suppkey:2(int!null) p_brand:9(char!null) p_type:10(varchar!null) p_size:11(int!null) - │ ├── grouping columns: ps_suppkey:2(int!null) p_brand:9(char!null) p_type:10(varchar!null) p_size:11(int!null) - │ ├── stats: [rows=9606.24468, distinct(2)=6153.37633, null(2)=0, distinct(9)=8.33333333, null(9)=0, distinct(10)=150, null(10)=0, distinct(11)=8, null(11)=0, distinct(9-11)=3315.43068, null(9-11)=0, distinct(2,9-11)=9606.24468, null(2,9-11)=0] - │ ├── key: (2,9-11) + │ ├── columns: ps_suppkey:2(int!null) p_brand:10(char!null) p_type:11(varchar!null) p_size:12(int!null) + │ ├── grouping columns: ps_suppkey:2(int!null) p_brand:10(char!null) p_type:11(varchar!null) p_size:12(int!null) + │ ├── stats: [rows=9606.24468, distinct(2)=6153.37633, null(2)=0, distinct(10)=8.33333333, null(10)=0, distinct(11)=150, null(11)=0, distinct(12)=8, null(12)=0, distinct(10-12)=3315.43068, null(10-12)=0, distinct(2,10-12)=9606.24468, null(2,10-12)=0] + │ ├── key: (2,10-12) │ └── inner-join (hash) │ ├── save-table-name: q16_inner_join_4 - │ ├── columns: ps_partkey:1(int!null) ps_suppkey:2(int!null) p_partkey:6(int!null) p_brand:9(char!null) p_type:10(varchar!null) p_size:11(int!null) + │ ├── columns: ps_partkey:1(int!null) ps_suppkey:2(int!null) p_partkey:7(int!null) p_brand:10(char!null) p_type:11(varchar!null) p_size:12(int!null) │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ ├── stats: [rows=9606.24468, distinct(1)=3555.43444, null(1)=0, distinct(2)=6153.37633, null(2)=0, distinct(6)=3555.43444, null(6)=0, distinct(9)=8.33333333, null(9)=0, distinct(10)=150, null(10)=0, distinct(11)=8, null(11)=0, distinct(9-11)=3315.43068, null(9-11)=0, distinct(2,9-11)=9606.24468, null(2,9-11)=0] - │ ├── key: (2,6) - │ ├── fd: (6)-->(9-11), (1)==(6), (6)==(1) + │ ├── stats: [rows=9606.24468, distinct(1)=3555.43444, null(1)=0, distinct(2)=6153.37633, null(2)=0, distinct(7)=3555.43444, null(7)=0, distinct(10)=8.33333333, null(10)=0, distinct(11)=150, null(11)=0, distinct(12)=8, null(12)=0, distinct(10-12)=3315.43068, null(10-12)=0, distinct(2,10-12)=9606.24468, null(2,10-12)=0] + │ ├── key: (2,7) + │ ├── fd: (7)-->(10-12), (1)==(7), (7)==(1) │ ├── anti-join (merge) │ │ ├── save-table-name: q16_merge_join_5 │ │ ├── columns: ps_partkey:1(int!null) ps_suppkey:2(int!null) │ │ ├── left ordering: +2 - │ │ ├── right ordering: +15 + │ │ ├── right ordering: +17 │ │ ├── stats: [rows=531592.246, distinct(1)=196758.028, null(1)=0, distinct(2)=9920, null(2)=0] │ │ ├── key: (1,2) │ │ ├── scan partsupp@ps_sk @@ -96,45 +96,45 @@ sort │ │ │ └── ordering: +2 │ │ ├── select │ │ │ ├── save-table-name: q16_select_7 - │ │ │ ├── columns: s_suppkey:15(int!null) s_comment:21(varchar!null) - │ │ │ ├── stats: [rows=3333.33333, distinct(15)=3328.25616, null(15)=0, distinct(21)=3333.33333, null(21)=0] - │ │ │ ├── key: (15) - │ │ │ ├── fd: (15)-->(21) - │ │ │ ├── ordering: +15 + │ │ │ ├── columns: s_suppkey:17(int!null) s_comment:23(varchar!null) + │ │ │ ├── stats: [rows=3333.33333, distinct(17)=3328.25616, null(17)=0, distinct(23)=3333.33333, null(23)=0] + │ │ │ ├── key: (17) + │ │ │ ├── fd: (17)-->(23) + │ │ │ ├── ordering: +17 │ │ │ ├── scan supplier │ │ │ │ ├── save-table-name: q16_scan_8 - │ │ │ │ ├── columns: s_suppkey:15(int!null) s_comment:21(varchar!null) - │ │ │ │ ├── stats: [rows=10000, distinct(15)=9920, null(15)=0, distinct(21)=9934, null(21)=0] - │ │ │ │ │ histogram(15)= 0 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 + │ │ │ │ ├── columns: s_suppkey:17(int!null) s_comment:23(varchar!null) + │ │ │ │ ├── stats: [rows=10000, distinct(17)=9920, null(17)=0, distinct(23)=9934, null(23)=0] + │ │ │ │ │ histogram(17)= 0 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 49 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 50 1 │ │ │ │ │ <--- 1 ---- 51 ---- 101 ---- 151 ---- 201 ---- 251 ---- 301 ---- 351 ---- 401 ---- 451 ---- 501 ---- 551 ---- 601 ---- 651 ---- 701 ---- 751 ---- 801 ---- 851 ---- 901 ---- 951 ---- 1001 ---- 1051 ---- 1101 ---- 1151 ---- 1201 ---- 1251 ---- 1301 ---- 1351 ---- 1401 ---- 1451 ---- 1501 ---- 1551 ---- 1601 ---- 1651 ---- 1701 ---- 1751 ---- 1801 ---- 1851 ---- 1901 ---- 1951 ---- 2001 ---- 2051 ---- 2101 ---- 2151 ---- 2201 ---- 2251 ---- 2301 ---- 2351 ---- 2401 ---- 2451 ---- 2501 ---- 2551 ---- 2601 ---- 2651 ---- 2701 ---- 2751 ---- 2801 ---- 2851 ---- 2901 ---- 2951 ---- 3001 ---- 3051 ---- 3101 ---- 3151 ---- 3201 ---- 3251 ---- 3301 ---- 3351 ---- 3401 ---- 3451 ---- 3501 ---- 3551 ---- 3601 ---- 3651 ---- 3701 ---- 3751 ---- 3801 ---- 3851 ---- 3901 ---- 3951 ---- 4001 ---- 4051 ---- 4101 ---- 4151 ---- 4201 ---- 4251 ---- 4301 ---- 4351 ---- 4401 ---- 4451 ---- 4501 ---- 4551 ---- 4601 ---- 4651 ---- 4701 ---- 4751 ---- 4801 ---- 4851 ---- 4901 ---- 4951 ---- 5001 ---- 5051 ---- 5101 ---- 5151 ---- 5201 ---- 5251 ---- 5301 ---- 5351 ---- 5401 ---- 5451 ---- 5501 ---- 5551 ---- 5601 ---- 5651 ---- 5701 ---- 5751 ---- 5801 ---- 5851 ---- 5901 ---- 5951 ---- 6001 ---- 6051 ---- 6101 ---- 6151 ---- 6201 ---- 6251 ---- 6301 ---- 6351 ---- 6401 ---- 6451 ---- 6501 ---- 6551 ---- 6601 ---- 6651 ---- 6701 ---- 6751 ---- 6801 ---- 6851 ---- 6901 ---- 6951 ---- 7001 ---- 7051 ---- 7101 ---- 7151 ---- 7201 ---- 7251 ---- 7301 ---- 7351 ---- 7401 ---- 7451 ---- 7501 ---- 7552 ---- 7603 ---- 7654 ---- 7705 ---- 7756 ---- 7807 ---- 7858 ---- 7909 ---- 7960 ---- 8011 ---- 8062 ---- 8113 ---- 8164 ---- 8215 ---- 8266 ---- 8317 ---- 8368 ---- 8419 ---- 8470 ---- 8521 ---- 8572 ---- 8623 ---- 8674 ---- 8725 ---- 8776 ---- 8827 ---- 8878 ---- 8929 ---- 8980 ---- 9031 ---- 9082 ---- 9133 ---- 9184 ---- 9235 ---- 9286 ---- 9337 ---- 9388 ---- 9439 ---- 9490 ---- 9541 ---- 9592 ---- 9643 ---- 9694 ---- 9745 ---- 9796 ---- 9847 ---- 9898 ---- 9949 ---- 10000 - │ │ │ │ ├── key: (15) - │ │ │ │ ├── fd: (15)-->(21) - │ │ │ │ └── ordering: +15 + │ │ │ │ ├── key: (17) + │ │ │ │ ├── fd: (17)-->(23) + │ │ │ │ └── ordering: +17 │ │ │ └── filters - │ │ │ └── s_comment:21 LIKE '%Customer%Complaints%' [type=bool, outer=(21), constraints=(/21: (/NULL - ])] + │ │ │ └── s_comment:23 LIKE '%Customer%Complaints%' [type=bool, outer=(23), constraints=(/23: (/NULL - ])] │ │ └── filters (true) │ ├── select │ │ ├── save-table-name: q16_select_9 - │ │ ├── columns: p_partkey:6(int!null) p_brand:9(char!null) p_type:10(varchar!null) p_size:11(int!null) - │ │ ├── stats: [rows=3555.55556, distinct(6)=3555.43444, null(6)=0, distinct(9)=8.33333333, null(9)=0, distinct(10)=150, null(10)=0, distinct(11)=8, null(11)=0, distinct(9,11)=66.6666667, null(9,11)=0, distinct(9-11)=3553.4368, null(9-11)=0] - │ │ ├── key: (6) - │ │ ├── fd: (6)-->(9-11) + │ │ ├── columns: p_partkey:7(int!null) p_brand:10(char!null) p_type:11(varchar!null) p_size:12(int!null) + │ │ ├── stats: [rows=3555.55556, distinct(7)=3555.43444, null(7)=0, distinct(10)=8.33333333, null(10)=0, distinct(11)=150, null(11)=0, distinct(12)=8, null(12)=0, distinct(10,12)=66.6666667, null(10,12)=0, distinct(10-12)=3553.4368, null(10-12)=0] + │ │ ├── key: (7) + │ │ ├── fd: (7)-->(10-12) │ │ ├── scan part │ │ │ ├── save-table-name: q16_scan_10 - │ │ │ ├── columns: p_partkey:6(int!null) p_brand:9(char!null) p_type:10(varchar!null) p_size:11(int!null) - │ │ │ ├── stats: [rows=200000, distinct(6)=199241, null(6)=0, distinct(9)=25, null(9)=0, distinct(10)=150, null(10)=0, distinct(11)=50, null(11)=0, distinct(9,11)=1250, null(9,11)=0, distinct(9-11)=187500, null(9-11)=0] - │ │ │ │ histogram(6)= 0 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 + │ │ │ ├── columns: p_partkey:7(int!null) p_brand:10(char!null) p_type:11(varchar!null) p_size:12(int!null) + │ │ │ ├── stats: [rows=200000, distinct(7)=199241, null(7)=0, distinct(10)=25, null(10)=0, distinct(11)=150, null(11)=0, distinct(12)=50, null(12)=0, distinct(10,12)=1250, null(10,12)=0, distinct(10-12)=187500, null(10-12)=0] + │ │ │ │ histogram(7)= 0 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 │ │ │ │ <--- 27 ----- 1110 ----- 2241 ----- 3086 ----- 4134 ----- 5302 ----- 6222 ----- 7308 ----- 8249 ----- 9171 ----- 10049 ----- 10958 ----- 11832 ----- 13025 ----- 14063 ----- 14953 ----- 16249 ----- 17419 ----- 18363 ----- 19412 ----- 20257 ----- 21190 ----- 22110 ----- 23045 ----- 23956 ----- 24962 ----- 25942 ----- 26990 ----- 27934 ----- 28876 ----- 29513 ----- 30326 ----- 31259 ----- 32300 ----- 33577 ----- 34550 ----- 35562 ----- 36498 ----- 37475 ----- 38584 ----- 39641 ----- 40548 ----- 41605 ----- 42527 ----- 43612 ----- 44702 ----- 45701 ----- 46726 ----- 47795 ----- 48935 ----- 50152 ----- 51183 ----- 52001 ----- 52904 ----- 53868 ----- 54808 ----- 55986 ----- 57155 ----- 58516 ----- 59526 ----- 60557 ----- 61547 ----- 62369 ----- 63672 ----- 64583 ----- 65360 ----- 66147 ----- 67201 ----- 68142 ----- 69145 ----- 70209 ----- 71141 ----- 71923 ----- 73031 ----- 73987 ----- 74974 ----- 76170 ----- 77138 ----- 77849 ----- 78931 ----- 79832 ----- 80761 ----- 81843 ----- 82834 ----- 84032 ----- 85072 ----- 86287 ----- 87302 ----- 88422 ----- 89432 ----- 90550 ----- 91463 ----- 92249 ----- 93385 ----- 94789 ----- 96013 ----- 96893 ----- 98000 ----- 99008 ----- 100166 ----- 101263 ----- 102351 ----- 103236 ----- 104121 ----- 105363 ----- 106329 ----- 107325 ----- 108231 ----- 109054 ----- 110019 ----- 111185 ----- 112112 ----- 112908 ----- 113904 ----- 114785 ----- 115410 ----- 116526 ----- 117559 ----- 118310 ----- 119073 ----- 120034 ----- 120817 ----- 121744 ----- 122566 ----- 123720 ----- 124813 ----- 125835 ----- 126622 ----- 127651 ----- 128328 ----- 129315 ----- 130244 ----- 131450 ----- 132439 ----- 133288 ----- 134164 ----- 135298 ----- 136347 ----- 137243 ----- 138256 ----- 139427 ----- 140374 ----- 141371 ----- 142302 ----- 143322 ----- 144335 ----- 145333 ----- 146212 ----- 147321 ----- 148591 ----- 149594 ------ 150514 ------ 151361 ------ 152059 ------ 153070 ------ 154059 ------ 155259 ------ 156473 ------ 157690 ------ 158703 ------ 159675 ------ 160597 ------ 161668 ------ 162737 ------ 163955 ------ 164942 ------ 165924 ------ 167059 ------ 167866 ------ 169034 ------ 169935 ------ 170712 ------ 171806 ------ 172841 ------ 174078 ------ 175347 ------ 176430 ------ 177346 ------ 178566 ------ 179515 ------ 180677 ------ 181729 ------ 182983 ------ 183814 ------ 184892 ------ 185696 ------ 186611 ------ 187744 ------ 188974 ------ 189911 ------ 190671 ------ 191607 ------ 192820 ------ 193789 ------ 195057 ------ 196224 ------ 197231 ------ 198281 ------ 199119 ------ 199999 - │ │ │ ├── key: (6) - │ │ │ └── fd: (6)-->(9-11) + │ │ │ ├── key: (7) + │ │ │ └── fd: (7)-->(10-12) │ │ └── filters - │ │ ├── p_brand:9 != 'Brand#45' [type=bool, outer=(9), constraints=(/9: (/NULL - /'Brand#45') [/e'Brand#45\x00' - ]; tight)] - │ │ ├── p_type:10 NOT LIKE 'MEDIUM POLISHED %' [type=bool, outer=(10), constraints=(/10: (/NULL - ])] - │ │ └── p_size:11 IN (3, 9, 14, 19, 23, 36, 45, 49) [type=bool, outer=(11), constraints=(/11: [/3 - /3] [/9 - /9] [/14 - /14] [/19 - /19] [/23 - /23] [/36 - /36] [/45 - /45] [/49 - /49]; tight)] + │ │ ├── p_brand:10 != 'Brand#45' [type=bool, outer=(10), constraints=(/10: (/NULL - /'Brand#45') [/e'Brand#45\x00' - ]; tight)] + │ │ ├── p_type:11 NOT LIKE 'MEDIUM POLISHED %' [type=bool, outer=(11), constraints=(/11: (/NULL - ])] + │ │ └── p_size:12 IN (3, 9, 14, 19, 23, 36, 45, 49) [type=bool, outer=(12), constraints=(/12: [/3 - /3] [/9 - /9] [/14 - /14] [/19 - /19] [/23 - /23] [/36 - /36] [/45 - /45] [/49 - /49]; tight)] │ └── filters - │ └── p_partkey:6 = ps_partkey:1 [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ └── p_partkey:7 = ps_partkey:1 [type=bool, outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] └── aggregations - └── count-rows [as=count:22, type=int] + └── count-rows [as=count:25, type=int] stats table=q16_sort_1 ---- diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q17 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q17 index 4d46d9034226..c1966c781829 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q17 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q17 @@ -41,101 +41,101 @@ WHERE ---- project ├── save-table-name: q17_project_1 - ├── columns: avg_yearly:45(float) + ├── columns: avg_yearly:48(float) ├── cardinality: [1 - 1] ├── immutable - ├── stats: [rows=1, distinct(45)=1, null(45)=0] + ├── stats: [rows=1, distinct(48)=1, null(48)=0] ├── key: () - ├── fd: ()-->(45) + ├── fd: ()-->(48) ├── scalar-group-by │ ├── save-table-name: q17_scalar_group_by_2 - │ ├── columns: sum:44(float) + │ ├── columns: sum:47(float) │ ├── cardinality: [1 - 1] │ ├── immutable - │ ├── stats: [rows=1, distinct(44)=1, null(44)=0] + │ ├── stats: [rows=1, distinct(47)=1, null(47)=0] │ ├── key: () - │ ├── fd: ()-->(44) + │ ├── fd: ()-->(47) │ ├── inner-join (lookup lineitem) │ │ ├── save-table-name: q17_lookup_join_3 - │ │ ├── columns: l_partkey:2(int!null) l_quantity:5(float!null) l_extendedprice:6(float!null) p_partkey:17(int!null) "?column?":43(float!null) + │ │ ├── columns: l_partkey:2(int!null) l_quantity:5(float!null) l_extendedprice:6(float!null) p_partkey:18(int!null) "?column?":46(float!null) │ │ ├── key columns: [1 4] = [1 4] │ │ ├── lookup columns are key │ │ ├── immutable - │ │ ├── stats: [rows=2008.02163, distinct(2)=199.999619, null(2)=0, distinct(5)=50, null(5)=0, distinct(6)=2005.84759, null(6)=0, distinct(17)=199.999619, null(17)=0, distinct(43)=199.999619, null(43)=0] - │ │ ├── fd: (17)-->(43), (2)==(17), (17)==(2) + │ │ ├── stats: [rows=2008.02163, distinct(2)=199.999619, null(2)=0, distinct(5)=50, null(5)=0, distinct(6)=2005.84759, null(6)=0, distinct(18)=199.999619, null(18)=0, distinct(46)=199.999619, null(46)=0] + │ │ ├── fd: (18)-->(46), (2)==(18), (18)==(2) │ │ ├── inner-join (lookup lineitem@l_pk) │ │ │ ├── save-table-name: q17_lookup_join_4 - │ │ │ ├── columns: l_orderkey:1(int!null) l_partkey:2(int!null) l_linenumber:4(int!null) p_partkey:17(int!null) "?column?":43(float) - │ │ │ ├── key columns: [17] = [2] + │ │ │ ├── columns: l_orderkey:1(int!null) l_partkey:2(int!null) l_linenumber:4(int!null) p_partkey:18(int!null) "?column?":46(float) + │ │ │ ├── key columns: [18] = [2] │ │ │ ├── immutable - │ │ │ ├── stats: [rows=6024.06489, distinct(1)=6012.21509, null(1)=0, distinct(2)=199.999619, null(2)=0, distinct(4)=7, null(4)=0, distinct(17)=199.999619, null(17)=0, distinct(43)=199.999619, null(43)=0] + │ │ │ ├── stats: [rows=6024.06489, distinct(1)=6012.21509, null(1)=0, distinct(2)=199.999619, null(2)=0, distinct(4)=7, null(4)=0, distinct(18)=199.999619, null(18)=0, distinct(46)=199.999619, null(46)=0] │ │ │ ├── key: (1,4) - │ │ │ ├── fd: (17)-->(43), (1,4)-->(2), (2)==(17), (17)==(2) + │ │ │ ├── fd: (18)-->(46), (1,4)-->(2), (2)==(18), (18)==(2) │ │ │ ├── project │ │ │ │ ├── save-table-name: q17_project_5 - │ │ │ │ ├── columns: "?column?":43(float) p_partkey:17(int!null) + │ │ │ │ ├── columns: "?column?":46(float) p_partkey:18(int!null) │ │ │ │ ├── immutable - │ │ │ │ ├── stats: [rows=199.999619, distinct(17)=199.999619, null(17)=0, distinct(43)=199.999619, null(43)=0] - │ │ │ │ ├── key: (17) - │ │ │ │ ├── fd: (17)-->(43) + │ │ │ │ ├── stats: [rows=199.999619, distinct(18)=199.999619, null(18)=0, distinct(46)=199.999619, null(46)=0] + │ │ │ │ ├── key: (18) + │ │ │ │ ├── fd: (18)-->(46) │ │ │ │ ├── group-by │ │ │ │ │ ├── save-table-name: q17_group_by_6 - │ │ │ │ │ ├── columns: p_partkey:17(int!null) avg:42(float) - │ │ │ │ │ ├── grouping columns: p_partkey:17(int!null) - │ │ │ │ │ ├── internal-ordering: +17 opt(20,23) - │ │ │ │ │ ├── stats: [rows=199.999619, distinct(17)=199.999619, null(17)=0, distinct(42)=199.999619, null(42)=0] - │ │ │ │ │ ├── key: (17) - │ │ │ │ │ ├── fd: (17)-->(42) + │ │ │ │ │ ├── columns: p_partkey:18(int!null) avg:45(float) + │ │ │ │ │ ├── grouping columns: p_partkey:18(int!null) + │ │ │ │ │ ├── internal-ordering: +18 opt(21,24) + │ │ │ │ │ ├── stats: [rows=199.999619, distinct(18)=199.999619, null(18)=0, distinct(45)=199.999619, null(45)=0] + │ │ │ │ │ ├── key: (18) + │ │ │ │ │ ├── fd: (18)-->(45) │ │ │ │ │ ├── left-join (lookup lineitem) │ │ │ │ │ │ ├── save-table-name: q17_lookup_join_7 - │ │ │ │ │ │ ├── columns: p_partkey:17(int!null) p_brand:20(char!null) p_container:23(char!null) l_partkey:27(int) l_quantity:30(float) - │ │ │ │ │ │ ├── key columns: [26 29] = [26 29] + │ │ │ │ │ │ ├── columns: p_partkey:18(int!null) p_brand:21(char!null) p_container:24(char!null) l_partkey:29(int) l_quantity:32(float) + │ │ │ │ │ │ ├── key columns: [28 31] = [28 31] │ │ │ │ │ │ ├── lookup columns are key - │ │ │ │ │ │ ├── stats: [rows=6024.07637, distinct(17)=199.999619, null(17)=0, distinct(20)=1, null(20)=0, distinct(23)=1, null(23)=0, distinct(27)=199.999619, null(27)=0, distinct(30)=50, null(30)=0] - │ │ │ │ │ │ ├── fd: ()-->(20,23) - │ │ │ │ │ │ ├── ordering: +17 opt(20,23) [actual: +17] + │ │ │ │ │ │ ├── stats: [rows=6024.07637, distinct(18)=199.999619, null(18)=0, distinct(21)=1, null(21)=0, distinct(24)=1, null(24)=0, distinct(29)=199.999619, null(29)=0, distinct(32)=50, null(32)=0] + │ │ │ │ │ │ ├── fd: ()-->(21,24) + │ │ │ │ │ │ ├── ordering: +18 opt(21,24) [actual: +18] │ │ │ │ │ │ ├── left-join (lookup lineitem@l_pk) │ │ │ │ │ │ │ ├── save-table-name: q17_lookup_join_8 - │ │ │ │ │ │ │ ├── columns: p_partkey:17(int!null) p_brand:20(char!null) p_container:23(char!null) l_orderkey:26(int) l_partkey:27(int) l_linenumber:29(int) - │ │ │ │ │ │ │ ├── key columns: [17] = [27] - │ │ │ │ │ │ │ ├── stats: [rows=6024.07637, distinct(17)=199.999619, null(17)=0, distinct(20)=1, null(20)=0, distinct(23)=1, null(23)=0, distinct(26)=6012.22652, null(26)=0, distinct(27)=199.999619, null(27)=0, distinct(29)=7, null(29)=0] - │ │ │ │ │ │ │ ├── key: (17,26,29) - │ │ │ │ │ │ │ ├── fd: ()-->(20,23), (26,29)-->(27) - │ │ │ │ │ │ │ ├── ordering: +17 opt(20,23) [actual: +17] + │ │ │ │ │ │ │ ├── columns: p_partkey:18(int!null) p_brand:21(char!null) p_container:24(char!null) l_orderkey:28(int) l_partkey:29(int) l_linenumber:31(int) + │ │ │ │ │ │ │ ├── key columns: [18] = [29] + │ │ │ │ │ │ │ ├── stats: [rows=6024.07637, distinct(18)=199.999619, null(18)=0, distinct(21)=1, null(21)=0, distinct(24)=1, null(24)=0, distinct(28)=6012.22652, null(28)=0, distinct(29)=199.999619, null(29)=0, distinct(31)=7, null(31)=0] + │ │ │ │ │ │ │ ├── key: (18,28,31) + │ │ │ │ │ │ │ ├── fd: ()-->(21,24), (28,31)-->(29) + │ │ │ │ │ │ │ ├── ordering: +18 opt(21,24) [actual: +18] │ │ │ │ │ │ │ ├── select │ │ │ │ │ │ │ │ ├── save-table-name: q17_select_9 - │ │ │ │ │ │ │ │ ├── columns: p_partkey:17(int!null) p_brand:20(char!null) p_container:23(char!null) - │ │ │ │ │ │ │ │ ├── stats: [rows=200, distinct(17)=199.999619, null(17)=0, distinct(20)=1, null(20)=0, distinct(23)=1, null(23)=0, distinct(20,23)=1, null(20,23)=0] - │ │ │ │ │ │ │ │ ├── key: (17) - │ │ │ │ │ │ │ │ ├── fd: ()-->(20,23) - │ │ │ │ │ │ │ │ ├── ordering: +17 opt(20,23) [actual: +17] + │ │ │ │ │ │ │ │ ├── columns: p_partkey:18(int!null) p_brand:21(char!null) p_container:24(char!null) + │ │ │ │ │ │ │ │ ├── stats: [rows=200, distinct(18)=199.999619, null(18)=0, distinct(21)=1, null(21)=0, distinct(24)=1, null(24)=0, distinct(21,24)=1, null(21,24)=0] + │ │ │ │ │ │ │ │ ├── key: (18) + │ │ │ │ │ │ │ │ ├── fd: ()-->(21,24) + │ │ │ │ │ │ │ │ ├── ordering: +18 opt(21,24) [actual: +18] │ │ │ │ │ │ │ │ ├── scan part │ │ │ │ │ │ │ │ │ ├── save-table-name: q17_scan_10 - │ │ │ │ │ │ │ │ │ ├── columns: p_partkey:17(int!null) p_brand:20(char!null) p_container:23(char!null) - │ │ │ │ │ │ │ │ │ ├── stats: [rows=200000, distinct(17)=199241, null(17)=0, distinct(20)=25, null(20)=0, distinct(23)=40, null(23)=0, distinct(20,23)=1000, null(20,23)=0] - │ │ │ │ │ │ │ │ │ │ histogram(17)= 0 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 + │ │ │ │ │ │ │ │ │ ├── columns: p_partkey:18(int!null) p_brand:21(char!null) p_container:24(char!null) + │ │ │ │ │ │ │ │ │ ├── stats: [rows=200000, distinct(18)=199241, null(18)=0, distinct(21)=25, null(21)=0, distinct(24)=40, null(24)=0, distinct(21,24)=1000, null(21,24)=0] + │ │ │ │ │ │ │ │ │ │ histogram(18)= 0 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 │ │ │ │ │ │ │ │ │ │ <--- 27 ----- 1110 ----- 2241 ----- 3086 ----- 4134 ----- 5302 ----- 6222 ----- 7308 ----- 8249 ----- 9171 ----- 10049 ----- 10958 ----- 11832 ----- 13025 ----- 14063 ----- 14953 ----- 16249 ----- 17419 ----- 18363 ----- 19412 ----- 20257 ----- 21190 ----- 22110 ----- 23045 ----- 23956 ----- 24962 ----- 25942 ----- 26990 ----- 27934 ----- 28876 ----- 29513 ----- 30326 ----- 31259 ----- 32300 ----- 33577 ----- 34550 ----- 35562 ----- 36498 ----- 37475 ----- 38584 ----- 39641 ----- 40548 ----- 41605 ----- 42527 ----- 43612 ----- 44702 ----- 45701 ----- 46726 ----- 47795 ----- 48935 ----- 50152 ----- 51183 ----- 52001 ----- 52904 ----- 53868 ----- 54808 ----- 55986 ----- 57155 ----- 58516 ----- 59526 ----- 60557 ----- 61547 ----- 62369 ----- 63672 ----- 64583 ----- 65360 ----- 66147 ----- 67201 ----- 68142 ----- 69145 ----- 70209 ----- 71141 ----- 71923 ----- 73031 ----- 73987 ----- 74974 ----- 76170 ----- 77138 ----- 77849 ----- 78931 ----- 79832 ----- 80761 ----- 81843 ----- 82834 ----- 84032 ----- 85072 ----- 86287 ----- 87302 ----- 88422 ----- 89432 ----- 90550 ----- 91463 ----- 92249 ----- 93385 ----- 94789 ----- 96013 ----- 96893 ----- 98000 ----- 99008 ----- 100166 ----- 101263 ----- 102351 ----- 103236 ----- 104121 ----- 105363 ----- 106329 ----- 107325 ----- 108231 ----- 109054 ----- 110019 ----- 111185 ----- 112112 ----- 112908 ----- 113904 ----- 114785 ----- 115410 ----- 116526 ----- 117559 ----- 118310 ----- 119073 ----- 120034 ----- 120817 ----- 121744 ----- 122566 ----- 123720 ----- 124813 ----- 125835 ----- 126622 ----- 127651 ----- 128328 ----- 129315 ----- 130244 ----- 131450 ----- 132439 ----- 133288 ----- 134164 ----- 135298 ----- 136347 ----- 137243 ----- 138256 ----- 139427 ----- 140374 ----- 141371 ----- 142302 ----- 143322 ----- 144335 ----- 145333 ----- 146212 ----- 147321 ----- 148591 ----- 149594 ------ 150514 ------ 151361 ------ 152059 ------ 153070 ------ 154059 ------ 155259 ------ 156473 ------ 157690 ------ 158703 ------ 159675 ------ 160597 ------ 161668 ------ 162737 ------ 163955 ------ 164942 ------ 165924 ------ 167059 ------ 167866 ------ 169034 ------ 169935 ------ 170712 ------ 171806 ------ 172841 ------ 174078 ------ 175347 ------ 176430 ------ 177346 ------ 178566 ------ 179515 ------ 180677 ------ 181729 ------ 182983 ------ 183814 ------ 184892 ------ 185696 ------ 186611 ------ 187744 ------ 188974 ------ 189911 ------ 190671 ------ 191607 ------ 192820 ------ 193789 ------ 195057 ------ 196224 ------ 197231 ------ 198281 ------ 199119 ------ 199999 - │ │ │ │ │ │ │ │ │ ├── key: (17) - │ │ │ │ │ │ │ │ │ ├── fd: (17)-->(20,23) - │ │ │ │ │ │ │ │ │ └── ordering: +17 opt(20,23) [actual: +17] + │ │ │ │ │ │ │ │ │ ├── key: (18) + │ │ │ │ │ │ │ │ │ ├── fd: (18)-->(21,24) + │ │ │ │ │ │ │ │ │ └── ordering: +18 opt(21,24) [actual: +18] │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ ├── p_brand:20 = 'Brand#23' [type=bool, outer=(20), constraints=(/20: [/'Brand#23' - /'Brand#23']; tight), fd=()-->(20)] - │ │ │ │ │ │ │ │ └── p_container:23 = 'MED BOX' [type=bool, outer=(23), constraints=(/23: [/'MED BOX' - /'MED BOX']; tight), fd=()-->(23)] + │ │ │ │ │ │ │ │ ├── p_brand:21 = 'Brand#23' [type=bool, outer=(21), constraints=(/21: [/'Brand#23' - /'Brand#23']; tight), fd=()-->(21)] + │ │ │ │ │ │ │ │ └── p_container:24 = 'MED BOX' [type=bool, outer=(24), constraints=(/24: [/'MED BOX' - /'MED BOX']; tight), fd=()-->(24)] │ │ │ │ │ │ │ └── filters (true) │ │ │ │ │ │ └── filters (true) │ │ │ │ │ └── aggregations - │ │ │ │ │ └── avg [as=avg:42, type=float, outer=(30)] - │ │ │ │ │ └── l_quantity:30 [type=float] + │ │ │ │ │ └── avg [as=avg:45, type=float, outer=(32)] + │ │ │ │ │ └── l_quantity:32 [type=float] │ │ │ │ └── projections - │ │ │ │ └── avg:42 * 0.2 [as="?column?":43, type=float, outer=(42), immutable] + │ │ │ │ └── avg:45 * 0.2 [as="?column?":46, type=float, outer=(45), immutable] │ │ │ └── filters (true) │ │ └── filters - │ │ └── l_quantity:5 < "?column?":43 [type=bool, outer=(5,43), constraints=(/5: (/NULL - ]; /43: (/NULL - ])] + │ │ └── l_quantity:5 < "?column?":46 [type=bool, outer=(5,46), constraints=(/5: (/NULL - ]; /46: (/NULL - ])] │ └── aggregations - │ └── sum [as=sum:44, type=float, outer=(6)] + │ └── sum [as=sum:47, type=float, outer=(6)] │ └── l_extendedprice:6 [type=float] └── projections - └── sum:44 / 7.0 [as=avg_yearly:45, type=float, outer=(44)] + └── sum:47 / 7.0 [as=avg_yearly:48, type=float, outer=(47)] stats table=q17_project_1 ---- diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q18 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q18 index 7b1737af398d..c0733ccc8a5a 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q18 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q18 @@ -52,95 +52,95 @@ LIMIT 100; ---- limit ├── save-table-name: q18_limit_1 - ├── columns: c_name:2(varchar!null) c_custkey:1(int!null) o_orderkey:9(int!null) o_orderdate:13(date!null) o_totalprice:12(float!null) sum:51(float!null) - ├── internal-ordering: -12,+13 + ├── columns: c_name:2(varchar!null) c_custkey:1(int!null) o_orderkey:10(int!null) o_orderdate:14(date!null) o_totalprice:13(float!null) sum:55(float!null) + ├── internal-ordering: -13,+14 ├── cardinality: [0 - 100] - ├── stats: [rows=100, distinct(1)=100, null(1)=0, distinct(2)=100, null(2)=0, distinct(9)=100, null(9)=0, distinct(12)=100, null(12)=0, distinct(13)=100, null(13)=0, distinct(51)=100, null(51)=0] - ├── key: (9) - ├── fd: (1)-->(2), (9)-->(1,2,12,13,51) - ├── ordering: -12,+13 + ├── stats: [rows=100, distinct(1)=100, null(1)=0, distinct(2)=100, null(2)=0, distinct(10)=100, null(10)=0, distinct(13)=100, null(13)=0, distinct(14)=100, null(14)=0, distinct(55)=100, null(55)=0] + ├── key: (10) + ├── fd: (1)-->(2), (10)-->(1,2,13,14,55) + ├── ordering: -13,+14 ├── sort │ ├── save-table-name: q18_sort_2 - │ ├── columns: c_custkey:1(int!null) c_name:2(varchar!null) o_orderkey:9(int!null) o_totalprice:12(float!null) o_orderdate:13(date!null) sum:51(float!null) - │ ├── stats: [rows=499392.239, distinct(1)=499392.239, null(1)=0, distinct(2)=499392.239, null(2)=0, distinct(9)=499392.239, null(9)=0, distinct(12)=499392.239, null(12)=0, distinct(13)=499392.239, null(13)=0, distinct(51)=499392.239, null(51)=0] - │ ├── key: (9) - │ ├── fd: (1)-->(2), (9)-->(1,2,12,13,51) - │ ├── ordering: -12,+13 + │ ├── columns: c_custkey:1(int!null) c_name:2(varchar!null) o_orderkey:10(int!null) o_totalprice:13(float!null) o_orderdate:14(date!null) sum:55(float!null) + │ ├── stats: [rows=499392.239, distinct(1)=499392.239, null(1)=0, distinct(2)=499392.239, null(2)=0, distinct(10)=499392.239, null(10)=0, distinct(13)=499392.239, null(13)=0, distinct(14)=499392.239, null(14)=0, distinct(55)=499392.239, null(55)=0] + │ ├── key: (10) + │ ├── fd: (1)-->(2), (10)-->(1,2,13,14,55) + │ ├── ordering: -13,+14 │ ├── limit hint: 100.00 │ └── group-by │ ├── save-table-name: q18_group_by_3 - │ ├── columns: c_custkey:1(int!null) c_name:2(varchar!null) o_orderkey:9(int!null) o_totalprice:12(float!null) o_orderdate:13(date!null) sum:51(float!null) - │ ├── grouping columns: o_orderkey:9(int!null) - │ ├── stats: [rows=499392.239, distinct(1)=499392.239, null(1)=0, distinct(2)=499392.239, null(2)=0, distinct(9)=499392.239, null(9)=0, distinct(12)=499392.239, null(12)=0, distinct(13)=499392.239, null(13)=0, distinct(51)=499392.239, null(51)=0] - │ ├── key: (9) - │ ├── fd: (1)-->(2), (9)-->(1,2,12,13,51) + │ ├── columns: c_custkey:1(int!null) c_name:2(varchar!null) o_orderkey:10(int!null) o_totalprice:13(float!null) o_orderdate:14(date!null) sum:55(float!null) + │ ├── grouping columns: o_orderkey:10(int!null) + │ ├── stats: [rows=499392.239, distinct(1)=499392.239, null(1)=0, distinct(2)=499392.239, null(2)=0, distinct(10)=499392.239, null(10)=0, distinct(13)=499392.239, null(13)=0, distinct(14)=499392.239, null(14)=0, distinct(55)=499392.239, null(55)=0] + │ ├── key: (10) + │ ├── fd: (1)-->(2), (10)-->(1,2,13,14,55) │ ├── inner-join (hash) │ │ ├── save-table-name: q18_inner_join_4 - │ │ ├── columns: c_custkey:1(int!null) c_name:2(varchar!null) o_orderkey:9(int!null) o_custkey:10(int!null) o_totalprice:12(float!null) o_orderdate:13(date!null) l_orderkey:18(int!null) l_quantity:22(float!null) + │ │ ├── columns: c_custkey:1(int!null) c_name:2(varchar!null) o_orderkey:10(int!null) o_custkey:11(int!null) o_totalprice:13(float!null) o_orderdate:14(date!null) l_orderkey:20(int!null) l_quantity:24(float!null) │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ ├── stats: [rows=2016361.14, distinct(1)=99649.071, null(1)=0, distinct(2)=149999.782, null(2)=0, distinct(9)=499392.239, null(9)=0, distinct(10)=99649.071, null(10)=0, distinct(12)=488043.529, null(12)=0, distinct(13)=2406, null(13)=0, distinct(18)=499392.239, null(18)=0, distinct(22)=50, null(22)=0] - │ │ ├── fd: (1)-->(2), (9)-->(10,12,13), (9)==(18), (18)==(9), (1)==(10), (10)==(1) + │ │ ├── stats: [rows=2016361.14, distinct(1)=99649.071, null(1)=0, distinct(2)=149999.782, null(2)=0, distinct(10)=499392.239, null(10)=0, distinct(11)=99649.071, null(11)=0, distinct(13)=488043.529, null(13)=0, distinct(14)=2406, null(14)=0, distinct(20)=499392.239, null(20)=0, distinct(24)=50, null(24)=0] + │ │ ├── fd: (1)-->(2), (10)-->(11,13,14), (10)==(20), (20)==(10), (1)==(11), (11)==(1) │ │ ├── scan lineitem │ │ │ ├── save-table-name: q18_scan_5 - │ │ │ ├── columns: l_orderkey:18(int!null) l_quantity:22(float!null) - │ │ │ └── stats: [rows=6001215, distinct(18)=1527270, null(18)=0, distinct(22)=50, null(22)=0] - │ │ │ histogram(18)= 0 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 + │ │ │ ├── columns: l_orderkey:20(int!null) l_quantity:24(float!null) + │ │ │ └── stats: [rows=6001215, distinct(20)=1527270, null(20)=0, distinct(24)=50, null(24)=0] + │ │ │ histogram(20)= 0 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 │ │ │ <--- 326 ------- 28929 ------- 50503 ------- 89793 ------- 115938 ------- 146944 ------- 176768 ------- 211201 ------- 237860 ------- 266885 ------- 297604 ------- 330021 ------- 365889 ------- 398951 ------- 426117 ------- 451328 ------- 472134 ------- 499590 ------- 529284 ------- 557254 ------- 589154 ------- 619394 ------- 642951 ------- 670113 ------- 692931 ------- 721157 ------- 751687 ------- 777766 ------- 804582 ------- 836740 ------- 868868 ------- 898912 ------- 922500 ------- 946403 ------- 984870 ------- 1007936 ------- 1030117 ------- 1062275 ------- 1093572 ------- 1120709 ------- 1150981 ------- 1182786 ------- 1206406 ------- 1234116 ------- 1260961 ------- 1290502 ------- 1329510 ------- 1355426 ------- 1381313 ------- 1409796 ------- 1445254 ------- 1479233 ------- 1504935 ------- 1531079 ------- 1559650 ------- 1583616 ------- 1617504 ------- 1655749 ------- 1685185 ------- 1718183 ------- 1747716 ------- 1772131 ------- 1802372 ------- 1833315 ------- 1862403 ------- 1897894 ------- 1922819 ------- 1954405 ------- 1979329 ------- 2009859 ------- 2041670 ------- 2070851 ------- 2093828 ------- 2127973 ------- 2167777 ------- 2194883 ------- 2227814 ------- 2262437 ------- 2296353 ------- 2321024 ------- 2346051 ------- 2376257 ------- 2404932 ------- 2446273 ------- 2474081 ------- 2504515 ------- 2535302 ------- 2561413 ------- 2592737 ------- 2616801 ------- 2646112 ------- 2676546 ------- 2702116 ------- 2732454 ------- 2765382 ------- 2799495 ------- 2828866 ------- 2868737 ------- 2910625 ------- 2938464 ------- 2963140 ------- 3003302 ------- 3043264 ------- 3069123 ------- 3095909 ------- 3126693 ------- 3160485 ------- 3196039 ------- 3229504 ------- 3259712 ------- 3286439 ------- 3318852 ------- 3346821 ------- 3370119 ------- 3395204 ------- 3425888 ------- 3448611 ------- 3476130 ------- 3502372 ------- 3529474 ------- 3556390 ------- 3583553 ------- 3612550 ------- 3647875 ------- 3679140 ------- 3702661 ------- 3738017 ------- 3778050 ------- 3806114 ------- 3839074 ------- 3872805 ------- 3905697 ------- 3926212 ------- 3959841 ------- 3997281 ------- 4033861 ------- 4063591 ------- 4097831 ------- 4124807 ------- 4158656 ------- 4195748 ------- 4234274 ------- 4269952 ------- 4298949 ------- 4332806 ------- 4364705 ------- 4398246 ------- 4430695 ------- 4466403 ------- 4494662 ------- 4524420 ------- 4558561 ------- 4601092 ------- 4632871 ------- 4658694 ------- 4690501 ------- 4728066 ------- 4758657 ------- 4788294 ------- 4818597 ------- 4855874 ------- 4890913 ------- 4915366 ------- 4940709 ------- 4972357 ------- 4995298 ------- 5019523 ------- 5043329 ------- 5077376 ------- 5109920 ------- 5136582 ------- 5161152 ------- 5191846 ------- 5219973 ------- 5251015 ------- 5282021 ------- 5312355 ------- 5343207 ------- 5381318 ------- 5416163 ------- 5445382 ------- 5476933 ------- 5509185 ------- 5539237 ------- 5566818 ------- 5588739 ------- 5620481 ------- 5644001 ------- 5667010 ------- 5689476 ------- 5724709 ------- 5755398 ------- 5790598 ------- 5819425 ------- 5846341 ------- 5874656 ------- 5908067 ------- 5933572 ------- 5962659 ------- 5999971 │ │ ├── inner-join (hash) │ │ │ ├── save-table-name: q18_inner_join_6 - │ │ │ ├── columns: c_custkey:1(int!null) c_name:2(varchar!null) o_orderkey:9(int!null) o_custkey:10(int!null) o_totalprice:12(float!null) o_orderdate:13(date!null) + │ │ │ ├── columns: c_custkey:1(int!null) c_name:2(varchar!null) o_orderkey:10(int!null) o_custkey:11(int!null) o_totalprice:13(float!null) o_orderdate:14(date!null) │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ │ │ ├── stats: [rows=513150.733, distinct(1)=99649.0712, null(1)=0, distinct(2)=145098.119, null(2)=0, distinct(9)=323294.803, null(9)=0, distinct(10)=99649.0712, null(10)=0, distinct(12)=322560.125, null(12)=0, distinct(13)=2406, null(13)=0] - │ │ │ ├── key: (9) - │ │ │ ├── fd: (1)-->(2), (9)-->(10,12,13), (1)==(10), (10)==(1) + │ │ │ ├── stats: [rows=513150.733, distinct(1)=99649.0712, null(1)=0, distinct(2)=145098.119, null(2)=0, distinct(10)=323294.803, null(10)=0, distinct(11)=99649.0712, null(11)=0, distinct(13)=322560.125, null(13)=0, distinct(14)=2406, null(14)=0] + │ │ │ ├── key: (10) + │ │ │ ├── fd: (1)-->(2), (10)-->(11,13,14), (1)==(11), (11)==(1) │ │ │ ├── semi-join (merge) │ │ │ │ ├── save-table-name: q18_merge_join_7 - │ │ │ │ ├── columns: o_orderkey:9(int!null) o_custkey:10(int!null) o_totalprice:12(float!null) o_orderdate:13(date!null) - │ │ │ │ ├── left ordering: +9 - │ │ │ │ ├── right ordering: +34 - │ │ │ │ ├── stats: [rows=509090, distinct(9)=509090, null(9)=0, distinct(10)=99649.0712, null(10)=0, distinct(12)=506350.486, null(12)=0, distinct(13)=2406, null(13)=0] - │ │ │ │ ├── key: (9) - │ │ │ │ ├── fd: (9)-->(10,12,13) + │ │ │ │ ├── columns: o_orderkey:10(int!null) o_custkey:11(int!null) o_totalprice:13(float!null) o_orderdate:14(date!null) + │ │ │ │ ├── left ordering: +10 + │ │ │ │ ├── right ordering: +37 + │ │ │ │ ├── stats: [rows=509090, distinct(10)=509090, null(10)=0, distinct(11)=99649.0712, null(11)=0, distinct(13)=506350.486, null(13)=0, distinct(14)=2406, null(14)=0] + │ │ │ │ ├── key: (10) + │ │ │ │ ├── fd: (10)-->(11,13,14) │ │ │ │ ├── scan orders │ │ │ │ │ ├── save-table-name: q18_scan_8 - │ │ │ │ │ ├── columns: o_orderkey:9(int!null) o_custkey:10(int!null) o_totalprice:12(float!null) o_orderdate:13(date!null) - │ │ │ │ │ ├── stats: [rows=1500000, distinct(9)=1500000, null(9)=0, distinct(10)=99846, null(10)=0, distinct(12)=1459167, null(12)=0, distinct(13)=2406, null(13)=0] - │ │ │ │ │ │ histogram(9)= 0 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 - │ │ │ │ │ │ <--- 998 ------ 26593 ------ 56613 ------ 85827 ------ 115587 ------ 140512 ------ 167076 ------ 198882 ------ 228580 ------ 262243 ------ 292064 ------ 316161 ------ 344194 ------ 368550 ------ 402722 ------ 434529 ------ 468807 ------ 495107 ------ 526338 ------ 563239 ------ 597733 ------ 641894 ------ 665730 ------ 688742 ------ 721767 ------ 750050 ------ 787879 ------ 814565 ------ 837381 ------ 866657 ------ 902855 ------ 927172 ------ 952772 ------ 975840 ------ 1008646 ------ 1032066 ------ 1074656 ------ 1104896 ------ 1138369 ------ 1166689 ------ 1194912 ------ 1225222 ------ 1253284 ------ 1284803 ------ 1318499 ------ 1348609 ------ 1376609 ------ 1404933 ------ 1435745 ------ 1460325 ------ 1491648 ------ 1518852 ------ 1542567 ------ 1574277 ------ 1597958 ------ 1624964 ------ 1647555 ------ 1683651 ------ 1717189 ------ 1750563 ------ 1784775 ------ 1812449 ------ 1838850 ------ 1867751 ------ 1891783 ------ 1918790 ------ 1949827 ------ 1986021 ------ 2020134 ------ 2045829 ------ 2077894 ------ 2118912 ------ 2143264 ------ 2176486 ------ 2204838 ------ 2234146 ------ 2260484 ------ 2288512 ------ 2317121 ------ 2344321 ------ 2362567 ------ 2396133 ------ 2423623 ------ 2454884 ------ 2483879 ------ 2520162 ------ 2545574 ------ 2571874 ------ 2608417 ------ 2637477 ------ 2665607 ------ 2692231 ------ 2724261 ------ 2764451 ------ 2798656 ------ 2824775 ------ 2858690 ------ 2881990 ------ 2911814 ------ 2945056 ------ 2981315 ------ 3010306 ------ 3032320 ------ 3064389 ------ 3084166 ------ 3108832 ------ 3136032 ------ 3168484 ------ 3204039 ------ 3229922 ------ 3256549 ------ 3283169 ------ 3315236 ------ 3346756 ------ 3375879 ------ 3408007 ------ 3441827 ------ 3471204 ------ 3505414 ------ 3527938 ------ 3562561 ------ 3598630 ------ 3627270 ------ 3659266 ------ 3686468 ------ 3713505 ------ 3750817 ------ 3777696 ------ 3808129 ------ 3834533 ------ 3871968 ------ 3904931 ------ 3934215 ------ 3960643 ------ 3990336 ------ 4023203 ------ 4057826 ------ 4090691 ------ 4121216 ------ 4159681 ------ 4190726 ------ 4222624 ------ 4246567 ------ 4283653 ------ 4314339 ------ 4342592 ------ 4373829 ------ 4402372 ------ 4431332 ------ 4464452 ------ 4496327 ------ 4526789 ------ 4551109 ------ 4582401 ------ 4615974 ------ 4648482 ------ 4669601 ------ 4705891 ------ 4732869 ------ 4766145 ------ 4805862 ------ 4840677 ------ 4866787 ------ 4895332 ------ 4928100 ------ 4956323 ------ 4992161 ------ 5027008 ------ 5065797 ------ 5099011 ------ 5130592 ------ 5163463 ------ 5197314 ------ 5224994 ------ 5253892 ------ 5277638 ------ 5307105 ------ 5335750 ------ 5369828 ------ 5402528 ------ 5434183 ------ 5464227 ------ 5491072 ------ 5520679 ------ 5543047 ------ 5576708 ------ 5614951 ------ 5646055 ------ 5674721 ------ 5700295 ------ 5732066 ------ 5761255 ------ 5791233 ------ 5819651 ------ 5852291 ------ 5880258 ------ 5909062 ------ 5943111 ------ 5973926 ------ 5998752 - │ │ │ │ │ │ histogram(10)= 0 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7200 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7200 300 7350 150 7350 150 7350 150 7350 150 7200 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7200 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 300 7350 150 7500 150 7500 150 7500 150 7500 150 7350 300 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7350 300 7500 150 7500 150 7500 150 7500 150 7500 150 7350 300 7500 150 + │ │ │ │ │ ├── columns: o_orderkey:10(int!null) o_custkey:11(int!null) o_totalprice:13(float!null) o_orderdate:14(date!null) + │ │ │ │ │ ├── stats: [rows=1500000, distinct(10)=1500000, null(10)=0, distinct(11)=99846, null(11)=0, distinct(13)=1459167, null(13)=0, distinct(14)=2406, null(14)=0] + │ │ │ │ │ │ histogram(10)= 0 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 + │ │ │ │ │ │ <--- 998 ------ 26593 ------ 56613 ------ 85827 ------ 115587 ------ 140512 ------ 167076 ------ 198882 ------ 228580 ------ 262243 ------ 292064 ------ 316161 ------ 344194 ------ 368550 ------ 402722 ------ 434529 ------ 468807 ------ 495107 ------ 526338 ------ 563239 ------ 597733 ------ 641894 ------ 665730 ------ 688742 ------ 721767 ------ 750050 ------ 787879 ------ 814565 ------ 837381 ------ 866657 ------ 902855 ------ 927172 ------ 952772 ------ 975840 ------ 1008646 ------ 1032066 ------ 1074656 ------ 1104896 ------ 1138369 ------ 1166689 ------ 1194912 ------ 1225222 ------ 1253284 ------ 1284803 ------ 1318499 ------ 1348609 ------ 1376609 ------ 1404933 ------ 1435745 ------ 1460325 ------ 1491648 ------ 1518852 ------ 1542567 ------ 1574277 ------ 1597958 ------ 1624964 ------ 1647555 ------ 1683651 ------ 1717189 ------ 1750563 ------ 1784775 ------ 1812449 ------ 1838850 ------ 1867751 ------ 1891783 ------ 1918790 ------ 1949827 ------ 1986021 ------ 2020134 ------ 2045829 ------ 2077894 ------ 2118912 ------ 2143264 ------ 2176486 ------ 2204838 ------ 2234146 ------ 2260484 ------ 2288512 ------ 2317121 ------ 2344321 ------ 2362567 ------ 2396133 ------ 2423623 ------ 2454884 ------ 2483879 ------ 2520162 ------ 2545574 ------ 2571874 ------ 2608417 ------ 2637477 ------ 2665607 ------ 2692231 ------ 2724261 ------ 2764451 ------ 2798656 ------ 2824775 ------ 2858690 ------ 2881990 ------ 2911814 ------ 2945056 ------ 2981315 ------ 3010306 ------ 3032320 ------ 3064389 ------ 3084166 ------ 3108832 ------ 3136032 ------ 3168484 ------ 3204039 ------ 3229922 ------ 3256549 ------ 3283169 ------ 3315236 ------ 3346756 ------ 3375879 ------ 3408007 ------ 3441827 ------ 3471204 ------ 3505414 ------ 3527938 ------ 3562561 ------ 3598630 ------ 3627270 ------ 3659266 ------ 3686468 ------ 3713505 ------ 3750817 ------ 3777696 ------ 3808129 ------ 3834533 ------ 3871968 ------ 3904931 ------ 3934215 ------ 3960643 ------ 3990336 ------ 4023203 ------ 4057826 ------ 4090691 ------ 4121216 ------ 4159681 ------ 4190726 ------ 4222624 ------ 4246567 ------ 4283653 ------ 4314339 ------ 4342592 ------ 4373829 ------ 4402372 ------ 4431332 ------ 4464452 ------ 4496327 ------ 4526789 ------ 4551109 ------ 4582401 ------ 4615974 ------ 4648482 ------ 4669601 ------ 4705891 ------ 4732869 ------ 4766145 ------ 4805862 ------ 4840677 ------ 4866787 ------ 4895332 ------ 4928100 ------ 4956323 ------ 4992161 ------ 5027008 ------ 5065797 ------ 5099011 ------ 5130592 ------ 5163463 ------ 5197314 ------ 5224994 ------ 5253892 ------ 5277638 ------ 5307105 ------ 5335750 ------ 5369828 ------ 5402528 ------ 5434183 ------ 5464227 ------ 5491072 ------ 5520679 ------ 5543047 ------ 5576708 ------ 5614951 ------ 5646055 ------ 5674721 ------ 5700295 ------ 5732066 ------ 5761255 ------ 5791233 ------ 5819651 ------ 5852291 ------ 5880258 ------ 5909062 ------ 5943111 ------ 5973926 ------ 5998752 + │ │ │ │ │ │ histogram(11)= 0 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7200 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7200 300 7350 150 7350 150 7350 150 7350 150 7200 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7200 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 300 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7350 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 300 7350 150 7500 150 7500 150 7500 150 7500 150 7350 300 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7500 150 7350 300 7500 150 7500 150 7500 150 7500 150 7500 150 7350 300 7500 150 │ │ │ │ │ │ <--- 8 ------ 763 ------ 1477 ------ 2353 ------ 3229 ------ 4261 ------ 5191 ------ 5924 ------ 6406 ------ 7126 ------ 7870 ------ 8545 ------ 9301 ------ 9911 ------ 10591 ------ 11236 ------ 11854 ------ 12622 ------ 13426 ------ 14036 ------ 14708 ------ 15391 ------ 16138 ------ 16925 ------ 17681 ------ 18388 ------ 19156 ------ 20015 ------ 20660 ------ 21632 ------ 22472 ------ 23260 ------ 24055 ------ 24649 ------ 25352 ------ 25952 ------ 26774 ------ 27358 ------ 28070 ------ 28586 ------ 29296 ------ 30028 ------ 30629 ------ 31400 ------ 32102 ------ 32884 ------ 33796 ------ 34646 ------ 35479 ------ 36161 ------ 36811 ------ 37612 ------ 38314 ------ 39145 ------ 39844 ------ 40591 ------ 41254 ------ 41902 ------ 42496 ------ 43237 ------ 43948 ------ 44690 ------ 45481 ------ 46448 ------ 47242 ------ 47884 ------ 48676 ------ 49441 ------ 50155 ------ 50857 ------ 51463 ------ 52321 ------ 53014 ------ 53587 ------ 54413 ------ 55019 ------ 55771 ------ 56482 ------ 57314 ------ 58163 ------ 58894 ------ 59638 ------ 60478 ------ 61450 ------ 62417 ------ 63176 ------ 63944 ------ 64577 ------ 65344 ------ 66218 ------ 67021 ------ 67703 ------ 68477 ------ 69271 ------ 70073 ------ 70696 ------ 71512 ------ 72430 ------ 73345 ------ 74149 ------ 74780 ------ 75508 ------ 76273 ------ 76865 ------ 77666 ------ 78427 ------ 79042 ------ 79801 ------ 80557 ------ 81163 ------ 81833 ------ 82534 ------ 83368 ------ 84157 ------ 84938 ------ 85495 ------ 86257 ------ 87143 ------ 87998 ------ 88777 ------ 89566 ------ 90292 ------ 91099 ------ 91757 ------ 92401 ------ 93362 ------ 94252 ------ 94771 ------ 95527 ------ 96233 ------ 96952 ------ 97844 ------ 98411 ------ 99067 ------ 99691 ------ 100357 ------ 101009 ------ 101734 ------ 102569 ------ 103381 ------ 104114 ------ 105070 ------ 105884 ------ 106651 ------ 107393 ------ 107995 ------ 108772 ------ 109628 ------ 110383 ------ 111040 ------ 111658 ------ 112556 ------ 113338 ------ 114062 ------ 114895 ------ 115744 ------ 116554 ------ 117250 ------ 117967 ------ 118741 ------ 119540 ------ 120514 ------ 121333 ------ 122111 ------ 122989 ------ 123763 ------ 124735 ------ 125401 ------ 126022 ------ 126779 ------ 127618 ------ 128332 ------ 129167 ------ 129917 ------ 130669 ------ 131330 ------ 131911 ------ 132704 ------ 133553 ------ 134317 ------ 134960 ------ 135688 ------ 136519 ------ 137353 ------ 138061 ------ 138697 ------ 139468 ------ 140218 ------ 140942 ------ 141644 ------ 142415 ------ 143263 ------ 144352 ------ 145099 ------ 145811 ------ 146761 ------ 147643 ------ 148591 ------ 149164 ------ 149995 - │ │ │ │ │ │ histogram(13)= 0 450 6600 900 6300 1350 7200 300 6750 1050 7350 300 7350 150 7200 300 7350 1200 6900 750 6600 900 6750 900 6300 1350 7350 750 7050 450 6900 900 7350 150 7200 450 7050 750 6750 750 6900 750 7350 600 6750 900 6750 750 7050 450 7050 900 6900 1050 6600 900 7350 750 6900 1050 6750 750 6750 900 7050 450 7050 750 7200 450 7050 750 6300 1800 6600 750 7200 1200 7050 600 7050 450 6600 1050 7050 450 6900 450 7200 450 6750 600 6600 900 7050 1200 6600 1200 7050 750 6600 900 6750 600 7200 450 6900 600 6600 1350 6900 1050 6300 1050 6750 1050 6900 450 7050 600 6900 450 7200 1500 7200 300 6450 900 7200 1050 6600 900 7050 600 6900 900 6600 900 6600 750 6900 1050 6450 900 7050 1350 6750 1050 6150 1500 7200 600 6900 600 6900 450 7050 300 6600 750 7200 600 6750 600 7200 300 7200 1350 7050 1200 6600 1050 7050 900 7050 450 6450 1050 6900 450 7050 1050 6750 900 6750 1200 6750 600 7200 1050 6150 1500 7200 300 7050 300 6750 900 6900 450 7200 1050 7200 450 7050 600 6750 750 6150 1200 7200 1200 6750 750 7050 450 7200 600 6450 1050 6600 600 7050 750 7050 150 6600 1200 6150 1050 6450 1050 6150 1050 6750 450 7050 300 5700 1500 7050 600 6300 1050 6900 750 6600 750 7050 1050 6600 750 6750 1050 7050 450 6900 900 7050 600 6600 750 6750 900 6900 750 6750 600 6300 1050 6750 600 6900 750 7050 900 6750 600 7050 450 6900 750 6600 600 7050 1050 6900 900 6900 750 6450 900 6750 600 6750 450 7050 450 7050 150 6600 1050 6900 600 6750 600 6300 900 6450 750 6600 600 6900 450 7050 750 6450 750 6300 900 6600 750 6450 1650 6450 750 6450 750 6900 450 5850 1350 6450 600 6300 1350 6750 750 6900 600 6900 1350 6000 1650 6600 750 6600 900 6300 1050 6300 750 6600 900 6000 900 6000 900 6600 450 6750 450 6450 750 6750 300 6450 450 6300 900 6450 450 6300 600 6300 750 6300 900 6750 1050 5850 1200 6150 1050 6300 750 6300 600 6600 750 6000 900 5550 1200 6300 450 5700 600 6000 450 + │ │ │ │ │ │ histogram(14)= 0 450 6600 900 6300 1350 7200 300 6750 1050 7350 300 7350 150 7200 300 7350 1200 6900 750 6600 900 6750 900 6300 1350 7350 750 7050 450 6900 900 7350 150 7200 450 7050 750 6750 750 6900 750 7350 600 6750 900 6750 750 7050 450 7050 900 6900 1050 6600 900 7350 750 6900 1050 6750 750 6750 900 7050 450 7050 750 7200 450 7050 750 6300 1800 6600 750 7200 1200 7050 600 7050 450 6600 1050 7050 450 6900 450 7200 450 6750 600 6600 900 7050 1200 6600 1200 7050 750 6600 900 6750 600 7200 450 6900 600 6600 1350 6900 1050 6300 1050 6750 1050 6900 450 7050 600 6900 450 7200 1500 7200 300 6450 900 7200 1050 6600 900 7050 600 6900 900 6600 900 6600 750 6900 1050 6450 900 7050 1350 6750 1050 6150 1500 7200 600 6900 600 6900 450 7050 300 6600 750 7200 600 6750 600 7200 300 7200 1350 7050 1200 6600 1050 7050 900 7050 450 6450 1050 6900 450 7050 1050 6750 900 6750 1200 6750 600 7200 1050 6150 1500 7200 300 7050 300 6750 900 6900 450 7200 1050 7200 450 7050 600 6750 750 6150 1200 7200 1200 6750 750 7050 450 7200 600 6450 1050 6600 600 7050 750 7050 150 6600 1200 6150 1050 6450 1050 6150 1050 6750 450 7050 300 5700 1500 7050 600 6300 1050 6900 750 6600 750 7050 1050 6600 750 6750 1050 7050 450 6900 900 7050 600 6600 750 6750 900 6900 750 6750 600 6300 1050 6750 600 6900 750 7050 900 6750 600 7050 450 6900 750 6600 600 7050 1050 6900 900 6900 750 6450 900 6750 600 6750 450 7050 450 7050 150 6600 1050 6900 600 6750 600 6300 900 6450 750 6600 600 6900 450 7050 750 6450 750 6300 900 6600 750 6450 1650 6450 750 6450 750 6900 450 5850 1350 6450 600 6300 1350 6750 750 6900 600 6900 1350 6000 1650 6600 750 6600 900 6300 1050 6300 750 6600 900 6000 900 6000 900 6600 450 6750 450 6450 750 6750 300 6450 450 6300 900 6450 450 6300 600 6300 750 6300 900 6750 1050 5850 1200 6150 1050 6300 750 6300 600 6600 750 6000 900 5550 1200 6300 450 5700 600 6000 450 │ │ │ │ │ │ <--- '1992-01-01' ------ '1992-01-11' ------ '1992-01-23' ------ '1992-02-04' ------ '1992-02-19' ------ '1992-03-03' ------ '1992-03-18' ------ '1992-03-29' ------ '1992-04-11' ------ '1992-04-23' ------ '1992-05-08' ------ '1992-05-20' ------ '1992-05-30' ------ '1992-06-11' ------ '1992-06-24' ------ '1992-07-05' ------ '1992-07-18' ------ '1992-07-30' ------ '1992-08-11' ------ '1992-08-23' ------ '1992-09-07' ------ '1992-09-21' ------ '1992-10-04' ------ '1992-10-18' ------ '1992-10-28' ------ '1992-11-07' ------ '1992-11-18' ------ '1992-11-30' ------ '1992-12-10' ------ '1992-12-25' ------ '1993-01-06' ------ '1993-01-19' ------ '1993-02-01' ------ '1993-02-14' ------ '1993-02-24' ------ '1993-03-08' ------ '1993-03-19' ------ '1993-04-03' ------ '1993-04-14' ------ '1993-04-25' ------ '1993-05-12' ------ '1993-05-25' ------ '1993-06-10' ------ '1993-06-18' ------ '1993-07-01' ------ '1993-07-12' ------ '1993-07-24' ------ '1993-08-05' ------ '1993-08-16' ------ '1993-08-29' ------ '1993-09-08' ------ '1993-09-19' ------ '1993-10-01' ------ '1993-10-13' ------ '1993-10-22' ------ '1993-11-04' ------ '1993-11-16' ------ '1993-11-30' ------ '1993-12-16' ------ '1993-12-29' ------ '1994-01-12' ------ '1994-01-23' ------ '1994-02-04' ------ '1994-02-16' ------ '1994-03-01' ------ '1994-03-14' ------ '1994-03-24' ------ '1994-04-06' ------ '1994-04-20' ------ '1994-05-01' ------ '1994-05-12' ------ '1994-05-23' ------ '1994-06-03' ------ '1994-06-16' ------ '1994-06-25' ------ '1994-07-08' ------ '1994-07-19' ------ '1994-08-01' ------ '1994-08-14' ------ '1994-08-25' ------ '1994-09-06' ------ '1994-09-18' ------ '1994-10-01' ------ '1994-10-17' ------ '1994-10-30' ------ '1994-11-10' ------ '1994-11-22' ------ '1994-12-02' ------ '1994-12-16' ------ '1994-12-30' ------ '1995-01-13' ------ '1995-01-25' ------ '1995-02-04' ------ '1995-02-16' ------ '1995-02-26' ------ '1995-03-10' ------ '1995-03-20' ------ '1995-04-02' ------ '1995-04-15' ------ '1995-04-28' ------ '1995-05-10' ------ '1995-05-25' ------ '1995-06-09' ------ '1995-06-20' ------ '1995-07-01' ------ '1995-07-13' ------ '1995-07-21' ------ '1995-08-01' ------ '1995-08-13' ------ '1995-08-25' ------ '1995-09-06' ------ '1995-09-18' ------ '1995-10-01' ------ '1995-10-11' ------ '1995-10-23' ------ '1995-11-01' ------ '1995-11-13' ------ '1995-11-26' ------ '1995-12-08' ------ '1995-12-22' ------ '1996-01-06' ------ '1996-01-18' ------ '1996-01-30' ------ '1996-02-13' ------ '1996-02-25' ------ '1996-03-09' ------ '1996-03-21' ------ '1996-04-04' ------ '1996-04-14' ------ '1996-04-27' ------ '1996-05-10' ------ '1996-05-22' ------ '1996-06-02' ------ '1996-06-12' ------ '1996-06-23' ------ '1996-07-07' ------ '1996-07-19' ------ '1996-08-01' ------ '1996-08-14' ------ '1996-08-25' ------ '1996-09-06' ------ '1996-09-20' ------ '1996-10-04' ------ '1996-10-15' ------ '1996-10-28' ------ '1996-11-11' ------ '1996-11-22' ------ '1996-12-04' ------ '1996-12-18' ------ '1996-12-29' ------ '1997-01-10' ------ '1997-01-22' ------ '1997-02-01' ------ '1997-02-13' ------ '1997-02-24' ------ '1997-03-09' ------ '1997-03-24' ------ '1997-04-04' ------ '1997-04-16' ------ '1997-04-26' ------ '1997-05-07' ------ '1997-05-19' ------ '1997-05-29' ------ '1997-06-08' ------ '1997-06-22' ------ '1997-07-05' ------ '1997-07-16' ------ '1997-07-28' ------ '1997-08-12' ------ '1997-08-23' ------ '1997-09-04' ------ '1997-09-17' ------ '1997-09-30' ------ '1997-10-09' ------ '1997-10-19' ------ '1997-10-31' ------ '1997-11-14' ------ '1997-11-24' ------ '1997-12-09' ------ '1997-12-24' ------ '1998-01-04' ------ '1998-01-16' ------ '1998-01-28' ------ '1998-02-07' ------ '1998-02-17' ------ '1998-03-01' ------ '1998-03-12' ------ '1998-03-24' ------ '1998-04-03' ------ '1998-04-14' ------ '1998-04-24' ------ '1998-05-03' ------ '1998-05-14' ------ '1998-05-26' ------ '1998-06-08' ------ '1998-06-20' ------ '1998-06-30' ------ '1998-07-11' ------ '1998-07-22' ------ '1998-08-02' - │ │ │ │ │ ├── key: (9) - │ │ │ │ │ ├── fd: (9)-->(10,12,13) - │ │ │ │ │ └── ordering: +9 + │ │ │ │ │ ├── key: (10) + │ │ │ │ │ ├── fd: (10)-->(11,13,14) + │ │ │ │ │ └── ordering: +10 │ │ │ │ ├── select │ │ │ │ │ ├── save-table-name: q18_select_9 - │ │ │ │ │ ├── columns: l_orderkey:34(int!null) sum:50(float!null) - │ │ │ │ │ ├── stats: [rows=509090, distinct(34)=509090, null(34)=0, distinct(50)=509090, null(50)=0] - │ │ │ │ │ ├── key: (34) - │ │ │ │ │ ├── fd: (34)-->(50) - │ │ │ │ │ ├── ordering: +34 + │ │ │ │ │ ├── columns: l_orderkey:37(int!null) sum:54(float!null) + │ │ │ │ │ ├── stats: [rows=509090, distinct(37)=509090, null(37)=0, distinct(54)=509090, null(54)=0] + │ │ │ │ │ ├── key: (37) + │ │ │ │ │ ├── fd: (37)-->(54) + │ │ │ │ │ ├── ordering: +37 │ │ │ │ │ ├── group-by │ │ │ │ │ │ ├── save-table-name: q18_group_by_10 - │ │ │ │ │ │ ├── columns: l_orderkey:34(int!null) sum:50(float!null) - │ │ │ │ │ │ ├── grouping columns: l_orderkey:34(int!null) - │ │ │ │ │ │ ├── stats: [rows=1527270, distinct(34)=1527270, null(34)=0, distinct(50)=1527270, null(50)=0] - │ │ │ │ │ │ ├── key: (34) - │ │ │ │ │ │ ├── fd: (34)-->(50) - │ │ │ │ │ │ ├── ordering: +34 + │ │ │ │ │ │ ├── columns: l_orderkey:37(int!null) sum:54(float!null) + │ │ │ │ │ │ ├── grouping columns: l_orderkey:37(int!null) + │ │ │ │ │ │ ├── stats: [rows=1527270, distinct(37)=1527270, null(37)=0, distinct(54)=1527270, null(54)=0] + │ │ │ │ │ │ ├── key: (37) + │ │ │ │ │ │ ├── fd: (37)-->(54) + │ │ │ │ │ │ ├── ordering: +37 │ │ │ │ │ │ ├── scan lineitem │ │ │ │ │ │ │ ├── save-table-name: q18_scan_11 - │ │ │ │ │ │ │ ├── columns: l_orderkey:34(int!null) l_quantity:38(float!null) - │ │ │ │ │ │ │ ├── stats: [rows=6001215, distinct(34)=1527270, null(34)=0, distinct(38)=50, null(38)=0] - │ │ │ │ │ │ │ │ histogram(34)= 0 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 + │ │ │ │ │ │ │ ├── columns: l_orderkey:37(int!null) l_quantity:41(float!null) + │ │ │ │ │ │ │ ├── stats: [rows=6001215, distinct(37)=1527270, null(37)=0, distinct(41)=50, null(41)=0] + │ │ │ │ │ │ │ │ histogram(37)= 0 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 │ │ │ │ │ │ │ │ <--- 326 ------- 28929 ------- 50503 ------- 89793 ------- 115938 ------- 146944 ------- 176768 ------- 211201 ------- 237860 ------- 266885 ------- 297604 ------- 330021 ------- 365889 ------- 398951 ------- 426117 ------- 451328 ------- 472134 ------- 499590 ------- 529284 ------- 557254 ------- 589154 ------- 619394 ------- 642951 ------- 670113 ------- 692931 ------- 721157 ------- 751687 ------- 777766 ------- 804582 ------- 836740 ------- 868868 ------- 898912 ------- 922500 ------- 946403 ------- 984870 ------- 1007936 ------- 1030117 ------- 1062275 ------- 1093572 ------- 1120709 ------- 1150981 ------- 1182786 ------- 1206406 ------- 1234116 ------- 1260961 ------- 1290502 ------- 1329510 ------- 1355426 ------- 1381313 ------- 1409796 ------- 1445254 ------- 1479233 ------- 1504935 ------- 1531079 ------- 1559650 ------- 1583616 ------- 1617504 ------- 1655749 ------- 1685185 ------- 1718183 ------- 1747716 ------- 1772131 ------- 1802372 ------- 1833315 ------- 1862403 ------- 1897894 ------- 1922819 ------- 1954405 ------- 1979329 ------- 2009859 ------- 2041670 ------- 2070851 ------- 2093828 ------- 2127973 ------- 2167777 ------- 2194883 ------- 2227814 ------- 2262437 ------- 2296353 ------- 2321024 ------- 2346051 ------- 2376257 ------- 2404932 ------- 2446273 ------- 2474081 ------- 2504515 ------- 2535302 ------- 2561413 ------- 2592737 ------- 2616801 ------- 2646112 ------- 2676546 ------- 2702116 ------- 2732454 ------- 2765382 ------- 2799495 ------- 2828866 ------- 2868737 ------- 2910625 ------- 2938464 ------- 2963140 ------- 3003302 ------- 3043264 ------- 3069123 ------- 3095909 ------- 3126693 ------- 3160485 ------- 3196039 ------- 3229504 ------- 3259712 ------- 3286439 ------- 3318852 ------- 3346821 ------- 3370119 ------- 3395204 ------- 3425888 ------- 3448611 ------- 3476130 ------- 3502372 ------- 3529474 ------- 3556390 ------- 3583553 ------- 3612550 ------- 3647875 ------- 3679140 ------- 3702661 ------- 3738017 ------- 3778050 ------- 3806114 ------- 3839074 ------- 3872805 ------- 3905697 ------- 3926212 ------- 3959841 ------- 3997281 ------- 4033861 ------- 4063591 ------- 4097831 ------- 4124807 ------- 4158656 ------- 4195748 ------- 4234274 ------- 4269952 ------- 4298949 ------- 4332806 ------- 4364705 ------- 4398246 ------- 4430695 ------- 4466403 ------- 4494662 ------- 4524420 ------- 4558561 ------- 4601092 ------- 4632871 ------- 4658694 ------- 4690501 ------- 4728066 ------- 4758657 ------- 4788294 ------- 4818597 ------- 4855874 ------- 4890913 ------- 4915366 ------- 4940709 ------- 4972357 ------- 4995298 ------- 5019523 ------- 5043329 ------- 5077376 ------- 5109920 ------- 5136582 ------- 5161152 ------- 5191846 ------- 5219973 ------- 5251015 ------- 5282021 ------- 5312355 ------- 5343207 ------- 5381318 ------- 5416163 ------- 5445382 ------- 5476933 ------- 5509185 ------- 5539237 ------- 5566818 ------- 5588739 ------- 5620481 ------- 5644001 ------- 5667010 ------- 5689476 ------- 5724709 ------- 5755398 ------- 5790598 ------- 5819425 ------- 5846341 ------- 5874656 ------- 5908067 ------- 5933572 ------- 5962659 ------- 5999971 - │ │ │ │ │ │ │ └── ordering: +34 + │ │ │ │ │ │ │ └── ordering: +37 │ │ │ │ │ │ └── aggregations - │ │ │ │ │ │ └── sum [as=sum:50, type=float, outer=(38)] - │ │ │ │ │ │ └── l_quantity:38 [type=float] + │ │ │ │ │ │ └── sum [as=sum:54, type=float, outer=(41)] + │ │ │ │ │ │ └── l_quantity:41 [type=float] │ │ │ │ │ └── filters - │ │ │ │ │ └── sum:50 > 300.0 [type=bool, outer=(50), constraints=(/50: [/300.00000000000006 - ]; tight)] + │ │ │ │ │ └── sum:54 > 300.0 [type=bool, outer=(54), constraints=(/54: [/300.00000000000006 - ]; tight)] │ │ │ │ └── filters (true) │ │ │ ├── scan customer │ │ │ │ ├── save-table-name: q18_scan_12 @@ -151,20 +151,20 @@ limit │ │ │ │ ├── key: (1) │ │ │ │ └── fd: (1)-->(2) │ │ │ └── filters - │ │ │ └── c_custkey:1 = o_custkey:10 [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] + │ │ │ └── c_custkey:1 = o_custkey:11 [type=bool, outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ]), fd=(1)==(11), (11)==(1)] │ │ └── filters - │ │ └── o_orderkey:9 = l_orderkey:18 [type=bool, outer=(9,18), constraints=(/9: (/NULL - ]; /18: (/NULL - ]), fd=(9)==(18), (18)==(9)] + │ │ └── o_orderkey:10 = l_orderkey:20 [type=bool, outer=(10,20), constraints=(/10: (/NULL - ]; /20: (/NULL - ]), fd=(10)==(20), (20)==(10)] │ └── aggregations - │ ├── sum [as=sum:51, type=float, outer=(22)] - │ │ └── l_quantity:22 [type=float] + │ ├── sum [as=sum:55, type=float, outer=(24)] + │ │ └── l_quantity:24 [type=float] │ ├── const-agg [as=c_custkey:1, type=int, outer=(1)] │ │ └── c_custkey:1 [type=int] │ ├── const-agg [as=c_name:2, type=varchar, outer=(2)] │ │ └── c_name:2 [type=varchar] - │ ├── const-agg [as=o_totalprice:12, type=float, outer=(12)] - │ │ └── o_totalprice:12 [type=float] - │ └── const-agg [as=o_orderdate:13, type=date, outer=(13)] - │ └── o_orderdate:13 [type=date] + │ ├── const-agg [as=o_totalprice:13, type=float, outer=(13)] + │ │ └── o_totalprice:13 [type=float] + │ └── const-agg [as=o_orderdate:14, type=date, outer=(14)] + │ └── o_orderdate:14 [type=date] └── 100 [type=int] stats table=q18_limit_1 diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q19 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q19 index 479cc01ca175..90a0e4247618 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q19 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q19 @@ -55,23 +55,23 @@ WHERE ---- scalar-group-by ├── save-table-name: q19_scalar_group_by_1 - ├── columns: revenue:27(float) + ├── columns: revenue:29(float) ├── cardinality: [1 - 1] ├── immutable - ├── stats: [rows=1, distinct(27)=1, null(27)=0] + ├── stats: [rows=1, distinct(29)=1, null(29)=0] ├── key: () - ├── fd: ()-->(27) + ├── fd: ()-->(29) ├── project │ ├── save-table-name: q19_project_2 - │ ├── columns: column26:26(float!null) + │ ├── columns: column28:28(float!null) │ ├── immutable - │ ├── stats: [rows=71.4087386, distinct(26)=71.402791, null(26)=0] + │ ├── stats: [rows=71.4087386, distinct(28)=71.402791, null(28)=0] │ ├── inner-join (hash) │ │ ├── save-table-name: q19_inner_join_3 - │ │ ├── columns: l_partkey:2(int!null) l_quantity:5(float!null) l_extendedprice:6(float!null) l_discount:7(float!null) l_shipinstruct:14(char!null) l_shipmode:15(char!null) p_partkey:17(int!null) p_brand:20(char!null) p_size:22(int!null) p_container:23(char!null) + │ │ ├── columns: l_partkey:2(int!null) l_quantity:5(float!null) l_extendedprice:6(float!null) l_discount:7(float!null) l_shipinstruct:14(char!null) l_shipmode:15(char!null) p_partkey:18(int!null) p_brand:21(char!null) p_size:23(int!null) p_container:24(char!null) │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ ├── stats: [rows=71.4087386, distinct(2)=71.4087386, null(2)=0, distinct(5)=5.55555556, null(5)=0, distinct(6)=71.4015197, null(6)=0, distinct(7)=10.9833244, null(7)=0, distinct(14)=1, null(14)=0, distinct(15)=2, null(15)=0, distinct(17)=71.4087386, null(17)=0, distinct(20)=3, null(20)=0, distinct(22)=5.55555556, null(22)=0, distinct(23)=12, null(23)=0, distinct(6,7)=71.402791, null(6,7)=0, distinct(20,22,23)=71.4087386, null(20,22,23)=0] - │ │ ├── fd: ()-->(14), (17)-->(20,22,23), (2)==(17), (17)==(2) + │ │ ├── stats: [rows=71.4087386, distinct(2)=71.4087386, null(2)=0, distinct(5)=5.55555556, null(5)=0, distinct(6)=71.4015197, null(6)=0, distinct(7)=10.9833244, null(7)=0, distinct(14)=1, null(14)=0, distinct(15)=2, null(15)=0, distinct(18)=71.4087386, null(18)=0, distinct(21)=3, null(21)=0, distinct(23)=5.55555556, null(23)=0, distinct(24)=12, null(24)=0, distinct(6,7)=71.402791, null(6,7)=0, distinct(21,23,24)=71.4087386, null(21,23,24)=0] + │ │ ├── fd: ()-->(14), (18)-->(21,23,24), (2)==(18), (18)==(2) │ │ ├── select │ │ │ ├── save-table-name: q19_select_4 │ │ │ ├── columns: l_partkey:2(int!null) l_quantity:5(float!null) l_extendedprice:6(float!null) l_discount:7(float!null) l_shipinstruct:14(char!null) l_shipmode:15(char!null) @@ -88,28 +88,28 @@ scalar-group-by │ │ │ └── l_shipinstruct:14 = 'DELIVER IN PERSON' [type=bool, outer=(14), constraints=(/14: [/'DELIVER IN PERSON' - /'DELIVER IN PERSON']; tight), fd=()-->(14)] │ │ ├── select │ │ │ ├── save-table-name: q19_select_6 - │ │ │ ├── columns: p_partkey:17(int!null) p_brand:20(char!null) p_size:22(int!null) p_container:23(char!null) - │ │ │ ├── stats: [rows=66666.6667, distinct(17)=66618.6736, null(17)=0, distinct(20)=25, null(20)=0, distinct(22)=16.6666667, null(22)=0, distinct(23)=40, null(23)=0, distinct(20,22,23)=16666.6667, null(20,22,23)=0] - │ │ │ ├── key: (17) - │ │ │ ├── fd: (17)-->(20,22,23) + │ │ │ ├── columns: p_partkey:18(int!null) p_brand:21(char!null) p_size:23(int!null) p_container:24(char!null) + │ │ │ ├── stats: [rows=66666.6667, distinct(18)=66618.6736, null(18)=0, distinct(21)=25, null(21)=0, distinct(23)=16.6666667, null(23)=0, distinct(24)=40, null(24)=0, distinct(21,23,24)=16666.6667, null(21,23,24)=0] + │ │ │ ├── key: (18) + │ │ │ ├── fd: (18)-->(21,23,24) │ │ │ ├── scan part │ │ │ │ ├── save-table-name: q19_scan_7 - │ │ │ │ ├── columns: p_partkey:17(int!null) p_brand:20(char!null) p_size:22(int!null) p_container:23(char!null) - │ │ │ │ ├── stats: [rows=200000, distinct(17)=199241, null(17)=0, distinct(20)=25, null(20)=0, distinct(22)=50, null(22)=0, distinct(23)=40, null(23)=0, distinct(20,22,23)=50000, null(20,22,23)=0] - │ │ │ │ │ histogram(17)= 0 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 + │ │ │ │ ├── columns: p_partkey:18(int!null) p_brand:21(char!null) p_size:23(int!null) p_container:24(char!null) + │ │ │ │ ├── stats: [rows=200000, distinct(18)=199241, null(18)=0, distinct(21)=25, null(21)=0, distinct(23)=50, null(23)=0, distinct(24)=40, null(24)=0, distinct(21,23,24)=50000, null(21,23,24)=0] + │ │ │ │ │ histogram(18)= 0 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 │ │ │ │ │ <--- 27 ----- 1110 ----- 2241 ----- 3086 ----- 4134 ----- 5302 ----- 6222 ----- 7308 ----- 8249 ----- 9171 ----- 10049 ----- 10958 ----- 11832 ----- 13025 ----- 14063 ----- 14953 ----- 16249 ----- 17419 ----- 18363 ----- 19412 ----- 20257 ----- 21190 ----- 22110 ----- 23045 ----- 23956 ----- 24962 ----- 25942 ----- 26990 ----- 27934 ----- 28876 ----- 29513 ----- 30326 ----- 31259 ----- 32300 ----- 33577 ----- 34550 ----- 35562 ----- 36498 ----- 37475 ----- 38584 ----- 39641 ----- 40548 ----- 41605 ----- 42527 ----- 43612 ----- 44702 ----- 45701 ----- 46726 ----- 47795 ----- 48935 ----- 50152 ----- 51183 ----- 52001 ----- 52904 ----- 53868 ----- 54808 ----- 55986 ----- 57155 ----- 58516 ----- 59526 ----- 60557 ----- 61547 ----- 62369 ----- 63672 ----- 64583 ----- 65360 ----- 66147 ----- 67201 ----- 68142 ----- 69145 ----- 70209 ----- 71141 ----- 71923 ----- 73031 ----- 73987 ----- 74974 ----- 76170 ----- 77138 ----- 77849 ----- 78931 ----- 79832 ----- 80761 ----- 81843 ----- 82834 ----- 84032 ----- 85072 ----- 86287 ----- 87302 ----- 88422 ----- 89432 ----- 90550 ----- 91463 ----- 92249 ----- 93385 ----- 94789 ----- 96013 ----- 96893 ----- 98000 ----- 99008 ----- 100166 ----- 101263 ----- 102351 ----- 103236 ----- 104121 ----- 105363 ----- 106329 ----- 107325 ----- 108231 ----- 109054 ----- 110019 ----- 111185 ----- 112112 ----- 112908 ----- 113904 ----- 114785 ----- 115410 ----- 116526 ----- 117559 ----- 118310 ----- 119073 ----- 120034 ----- 120817 ----- 121744 ----- 122566 ----- 123720 ----- 124813 ----- 125835 ----- 126622 ----- 127651 ----- 128328 ----- 129315 ----- 130244 ----- 131450 ----- 132439 ----- 133288 ----- 134164 ----- 135298 ----- 136347 ----- 137243 ----- 138256 ----- 139427 ----- 140374 ----- 141371 ----- 142302 ----- 143322 ----- 144335 ----- 145333 ----- 146212 ----- 147321 ----- 148591 ----- 149594 ------ 150514 ------ 151361 ------ 152059 ------ 153070 ------ 154059 ------ 155259 ------ 156473 ------ 157690 ------ 158703 ------ 159675 ------ 160597 ------ 161668 ------ 162737 ------ 163955 ------ 164942 ------ 165924 ------ 167059 ------ 167866 ------ 169034 ------ 169935 ------ 170712 ------ 171806 ------ 172841 ------ 174078 ------ 175347 ------ 176430 ------ 177346 ------ 178566 ------ 179515 ------ 180677 ------ 181729 ------ 182983 ------ 183814 ------ 184892 ------ 185696 ------ 186611 ------ 187744 ------ 188974 ------ 189911 ------ 190671 ------ 191607 ------ 192820 ------ 193789 ------ 195057 ------ 196224 ------ 197231 ------ 198281 ------ 199119 ------ 199999 - │ │ │ │ ├── key: (17) - │ │ │ │ └── fd: (17)-->(20,22,23) + │ │ │ │ ├── key: (18) + │ │ │ │ └── fd: (18)-->(21,23,24) │ │ │ └── filters - │ │ │ └── p_size:22 >= 1 [type=bool, outer=(22), constraints=(/22: [/1 - ]; tight)] + │ │ │ └── p_size:23 >= 1 [type=bool, outer=(23), constraints=(/23: [/1 - ]; tight)] │ │ └── filters - │ │ ├── p_partkey:17 = l_partkey:2 [type=bool, outer=(2,17), constraints=(/2: (/NULL - ]; /17: (/NULL - ]), fd=(2)==(17), (17)==(2)] - │ │ └── ((((((p_brand:20 = 'Brand#12') AND (p_container:23 IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG'))) AND (l_quantity:5 >= 1.0)) AND (l_quantity:5 <= 11.0)) AND (p_size:22 <= 5)) OR (((((p_brand:20 = 'Brand#23') AND (p_container:23 IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG'))) AND (l_quantity:5 >= 10.0)) AND (l_quantity:5 <= 20.0)) AND (p_size:22 <= 10))) OR (((((p_brand:20 = 'Brand#34') AND (p_container:23 IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG'))) AND (l_quantity:5 >= 20.0)) AND (l_quantity:5 <= 30.0)) AND (p_size:22 <= 15)) [type=bool, outer=(5,20,22,23), constraints=(/5: [/1.0 - /30.0]; /20: [/'Brand#12' - /'Brand#12'] [/'Brand#23' - /'Brand#23'] [/'Brand#34' - /'Brand#34']; /22: (/NULL - /15]; /23: [/'LG BOX' - /'LG BOX'] [/'LG CASE' - /'LG CASE'] [/'LG PACK' - /'LG PACK'] [/'LG PKG' - /'LG PKG'] [/'MED BAG' - /'MED BAG'] [/'MED BOX' - /'MED BOX'] [/'MED PACK' - /'MED PACK'] [/'MED PKG' - /'MED PKG'] [/'SM BOX' - /'SM BOX'] [/'SM CASE' - /'SM CASE'] [/'SM PACK' - /'SM PACK'] [/'SM PKG' - /'SM PKG'])] + │ │ ├── p_partkey:18 = l_partkey:2 [type=bool, outer=(2,18), constraints=(/2: (/NULL - ]; /18: (/NULL - ]), fd=(2)==(18), (18)==(2)] + │ │ └── ((((((p_brand:21 = 'Brand#12') AND (p_container:24 IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG'))) AND (l_quantity:5 >= 1.0)) AND (l_quantity:5 <= 11.0)) AND (p_size:23 <= 5)) OR (((((p_brand:21 = 'Brand#23') AND (p_container:24 IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG'))) AND (l_quantity:5 >= 10.0)) AND (l_quantity:5 <= 20.0)) AND (p_size:23 <= 10))) OR (((((p_brand:21 = 'Brand#34') AND (p_container:24 IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG'))) AND (l_quantity:5 >= 20.0)) AND (l_quantity:5 <= 30.0)) AND (p_size:23 <= 15)) [type=bool, outer=(5,21,23,24), constraints=(/5: [/1.0 - /30.0]; /21: [/'Brand#12' - /'Brand#12'] [/'Brand#23' - /'Brand#23'] [/'Brand#34' - /'Brand#34']; /23: (/NULL - /15]; /24: [/'LG BOX' - /'LG BOX'] [/'LG CASE' - /'LG CASE'] [/'LG PACK' - /'LG PACK'] [/'LG PKG' - /'LG PKG'] [/'MED BAG' - /'MED BAG'] [/'MED BOX' - /'MED BOX'] [/'MED PACK' - /'MED PACK'] [/'MED PKG' - /'MED PKG'] [/'SM BOX' - /'SM BOX'] [/'SM CASE' - /'SM CASE'] [/'SM PACK' - /'SM PACK'] [/'SM PKG' - /'SM PKG'])] │ └── projections - │ └── l_extendedprice:6 * (1.0 - l_discount:7) [as=column26:26, type=float, outer=(6,7), immutable] + │ └── l_extendedprice:6 * (1.0 - l_discount:7) [as=column28:28, type=float, outer=(6,7), immutable] └── aggregations - └── sum [as=sum:27, type=float, outer=(26)] - └── column26:26 [type=float] + └── sum [as=sum:29, type=float, outer=(28)] + └── column28:28 [type=float] stats table=q19_scalar_group_by_1 ---- @@ -122,10 +122,10 @@ column_names row_count_est row_count_err distinct_count_est distinct_count_e stats table=q19_project_2 ---- column_names row_count distinct_count null_count -{column26} 121 121 0 +{column28} 121 121 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{column26} 71.00 1.70 71.00 1.70 0.00 1.00 +{column28} 71.00 1.70 71.00 1.70 0.00 1.00 stats table=q19_inner_join_3 ---- diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q20 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q20 index e27e2b35670f..ea1ce9f83d79 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q20 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q20 @@ -70,12 +70,12 @@ sort ├── stats: [rows=392.749612, distinct(2)=392.742232, null(2)=0, distinct(3)=392.749612, null(3)=0] └── inner-join (hash) ├── save-table-name: q20_inner_join_3 - ├── columns: s_suppkey:1(int!null) s_name:2(char!null) s_address:3(varchar!null) s_nationkey:4(int!null) n_nationkey:8(int!null) n_name:9(char!null) + ├── columns: s_suppkey:1(int!null) s_name:2(char!null) s_address:3(varchar!null) s_nationkey:4(int!null) n_nationkey:9(int!null) n_name:10(char!null) ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) ├── immutable - ├── stats: [rows=392.749612, distinct(1)=392.685411, null(1)=0, distinct(2)=392.742232, null(2)=0, distinct(3)=392.749612, null(3)=0, distinct(4)=1, null(4)=0, distinct(8)=1, null(8)=0, distinct(9)=1, null(9)=0] + ├── stats: [rows=392.749612, distinct(1)=392.685411, null(1)=0, distinct(2)=392.742232, null(2)=0, distinct(3)=392.749612, null(3)=0, distinct(4)=1, null(4)=0, distinct(9)=1, null(9)=0, distinct(10)=1, null(10)=0] ├── key: (1) - ├── fd: ()-->(9), (1)-->(2-4), (4)==(8), (8)==(4) + ├── fd: ()-->(10), (1)-->(2-4), (4)==(9), (9)==(4) ├── semi-join (hash) │ ├── save-table-name: q20_semi_join_4 │ ├── columns: s_suppkey:1(int!null) s_name:2(char!null) s_address:3(varchar!null) s_nationkey:4(int!null) @@ -95,119 +95,119 @@ sort │ │ └── fd: (1)-->(2-4) │ ├── project │ │ ├── save-table-name: q20_project_6 - │ │ ├── columns: ps_partkey:12(int!null) ps_suppkey:13(int!null) + │ │ ├── columns: ps_partkey:14(int!null) ps_suppkey:15(int!null) │ │ ├── immutable - │ │ ├── stats: [rows=36952.1991, distinct(12)=22217.3354, null(12)=0, distinct(13)=9740.19038, null(13)=0] - │ │ ├── key: (12,13) + │ │ ├── stats: [rows=36952.1991, distinct(14)=22217.3354, null(14)=0, distinct(15)=9740.19038, null(15)=0] + │ │ ├── key: (14,15) │ │ └── project │ │ ├── save-table-name: q20_project_7 - │ │ ├── columns: ps_partkey:12(int!null) ps_suppkey:13(int!null) p_partkey:17(int!null) + │ │ ├── columns: ps_partkey:14(int!null) ps_suppkey:15(int!null) p_partkey:20(int!null) │ │ ├── immutable - │ │ ├── stats: [rows=36960.327, distinct(12)=22217.3354, null(12)=0, distinct(13)=9681.00153, null(13)=0, distinct(17)=22217.3354, null(17)=0] - │ │ ├── key: (13,17) - │ │ ├── fd: (12)==(17), (17)==(12) + │ │ ├── stats: [rows=36960.327, distinct(14)=22217.3354, null(14)=0, distinct(15)=9681.00153, null(15)=0, distinct(20)=22217.3354, null(20)=0] + │ │ ├── key: (15,20) + │ │ ├── fd: (14)==(20), (20)==(14) │ │ └── inner-join (hash) │ │ ├── save-table-name: q20_inner_join_8 - │ │ ├── columns: ps_partkey:12(int!null) ps_suppkey:13(int!null) ps_availqty:14(int!null) p_partkey:17(int!null) p_name:18(varchar!null) sum:42(float) + │ │ ├── columns: ps_partkey:14(int!null) ps_suppkey:15(int!null) ps_availqty:16(int!null) p_partkey:20(int!null) p_name:21(varchar!null) sum:47(float) │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) │ │ ├── immutable - │ │ ├── stats: [rows=36960.327, distinct(12)=22217.3354, null(12)=0, distinct(13)=9681.00153, null(13)=0, distinct(14)=34508.432, null(14)=0, distinct(17)=22217.3354, null(17)=0, distinct(18)=17907.1379, null(18)=0, distinct(42)=34508.432, null(42)=0] - │ │ ├── key: (13,17) - │ │ ├── fd: (12,13)-->(14,42), (17)-->(18), (12)==(17), (17)==(12) + │ │ ├── stats: [rows=36960.327, distinct(14)=22217.3354, null(14)=0, distinct(15)=9681.00153, null(15)=0, distinct(16)=34508.432, null(16)=0, distinct(20)=22217.3354, null(20)=0, distinct(21)=17907.1379, null(21)=0, distinct(47)=34508.432, null(47)=0] + │ │ ├── key: (15,20) + │ │ ├── fd: (14,15)-->(16,47), (20)-->(21), (14)==(20), (20)==(14) │ │ ├── select │ │ │ ├── save-table-name: q20_select_9 - │ │ │ ├── columns: ps_partkey:12(int!null) ps_suppkey:13(int!null) ps_availqty:14(int!null) sum:42(float) + │ │ │ ├── columns: ps_partkey:14(int!null) ps_suppkey:15(int!null) ps_availqty:16(int!null) sum:47(float) │ │ │ ├── immutable - │ │ │ ├── stats: [rows=266100.667, distinct(12)=159991.77, null(12)=0, distinct(13)=9920, null(13)=0, distinct(14)=266100.667, null(14)=0, distinct(42)=266100.667, null(42)=0] - │ │ │ ├── key: (12,13) - │ │ │ ├── fd: (12,13)-->(14,42) + │ │ │ ├── stats: [rows=266100.667, distinct(14)=159991.77, null(14)=0, distinct(15)=9920, null(15)=0, distinct(16)=266100.667, null(16)=0, distinct(47)=266100.667, null(47)=0] + │ │ │ ├── key: (14,15) + │ │ │ ├── fd: (14,15)-->(16,47) │ │ │ ├── group-by │ │ │ │ ├── save-table-name: q20_group_by_10 - │ │ │ │ ├── columns: ps_partkey:12(int!null) ps_suppkey:13(int!null) ps_availqty:14(int!null) sum:42(float) - │ │ │ │ ├── grouping columns: ps_partkey:12(int!null) ps_suppkey:13(int!null) - │ │ │ │ ├── stats: [rows=798302, distinct(12)=199241, null(12)=0, distinct(13)=9920, null(13)=0, distinct(14)=798302, null(14)=0, distinct(42)=798302, null(42)=0, distinct(12,13)=798302, null(12,13)=0] - │ │ │ │ ├── key: (12,13) - │ │ │ │ ├── fd: (12,13)-->(14,42) + │ │ │ │ ├── columns: ps_partkey:14(int!null) ps_suppkey:15(int!null) ps_availqty:16(int!null) sum:47(float) + │ │ │ │ ├── grouping columns: ps_partkey:14(int!null) ps_suppkey:15(int!null) + │ │ │ │ ├── stats: [rows=798302, distinct(14)=199241, null(14)=0, distinct(15)=9920, null(15)=0, distinct(16)=798302, null(16)=0, distinct(47)=798302, null(47)=0, distinct(14,15)=798302, null(14,15)=0] + │ │ │ │ ├── key: (14,15) + │ │ │ │ ├── fd: (14,15)-->(16,47) │ │ │ │ ├── right-join (hash) │ │ │ │ │ ├── save-table-name: q20_right_join_11 - │ │ │ │ │ ├── columns: ps_partkey:12(int!null) ps_suppkey:13(int!null) ps_availqty:14(int!null) l_partkey:27(int) l_suppkey:28(int) l_quantity:30(float) l_shipdate:36(date) - │ │ │ │ │ ├── stats: [rows=800000, distinct(12)=199241, null(12)=0, distinct(13)=9920, null(13)=0, distinct(14)=9920, null(14)=0, distinct(27)=379.722303, null(27)=799620.278, distinct(28)=379.722303, null(28)=799620.278, distinct(30)=49.9748381, null(30)=799620.278, distinct(36)=236.032274, null(36)=799620.278, distinct(12,13)=798302, null(12,13)=0] - │ │ │ │ │ ├── fd: (12,13)-->(14) + │ │ │ │ │ ├── columns: ps_partkey:14(int!null) ps_suppkey:15(int!null) ps_availqty:16(int!null) l_partkey:31(int) l_suppkey:32(int) l_quantity:34(float) l_shipdate:40(date) + │ │ │ │ │ ├── stats: [rows=800000, distinct(14)=199241, null(14)=0, distinct(15)=9920, null(15)=0, distinct(16)=9920, null(16)=0, distinct(31)=379.722303, null(31)=799620.278, distinct(32)=379.722303, null(32)=799620.278, distinct(34)=49.9748381, null(34)=799620.278, distinct(40)=236.032274, null(40)=799620.278, distinct(14,15)=798302, null(14,15)=0] + │ │ │ │ │ ├── fd: (14,15)-->(16) │ │ │ │ │ ├── index-join lineitem │ │ │ │ │ │ ├── save-table-name: q20_index_join_12 - │ │ │ │ │ │ ├── columns: l_partkey:27(int!null) l_suppkey:28(int!null) l_quantity:30(float!null) l_shipdate:36(date!null) - │ │ │ │ │ │ ├── stats: [rows=938137.516, distinct(27)=198050.435, null(27)=0, distinct(28)=9920, null(28)=0, distinct(30)=50, null(30)=0, distinct(36)=365, null(36)=0] - │ │ │ │ │ │ │ histogram(36)= 0 0 3150.6 4200 25805 3600 25805 3600 26405 3600 28805 1200 27605 2400 27005 3600 28805 1200 27005 4800 28205 3000 25205 4800 28805 4800 28805 600 28205 1200 24604 6601 24604 4800 28205 3600 27605 3600 27005 4200 28205 2400 28205 4200 28805 1200 28205 1800 28205 1200 28205 3000 27005 4800 28805 600 27005 3600 25805 5401 27005 2400 28205 3000 11502 2300.4 + │ │ │ │ │ │ ├── columns: l_partkey:31(int!null) l_suppkey:32(int!null) l_quantity:34(float!null) l_shipdate:40(date!null) + │ │ │ │ │ │ ├── stats: [rows=938137.516, distinct(31)=198050.435, null(31)=0, distinct(32)=9920, null(32)=0, distinct(34)=50, null(34)=0, distinct(40)=365, null(40)=0] + │ │ │ │ │ │ │ histogram(40)= 0 0 3150.6 4200 25805 3600 25805 3600 26405 3600 28805 1200 27605 2400 27005 3600 28805 1200 27005 4800 28205 3000 25205 4800 28805 4800 28805 600 28205 1200 24604 6601 24604 4800 28205 3600 27605 3600 27005 4200 28205 2400 28205 4200 28805 1200 28205 1800 28205 1200 28205 3000 27005 4800 28805 600 27005 3600 25805 5401 27005 2400 28205 3000 11502 2300.4 │ │ │ │ │ │ │ <--- '1993-12-31' -------- '1994-01-02' ------- '1994-01-16' ------- '1994-01-30' ------- '1994-02-08' ------- '1994-02-18' ------- '1994-03-04' ------- '1994-03-15' ------- '1994-03-29' ------- '1994-04-10' ------- '1994-04-24' ------- '1994-05-03' ------- '1994-05-13' ------- '1994-05-24' ------- '1994-06-04' ------- '1994-06-13' ------- '1994-06-22' ------- '1994-07-05' ------- '1994-07-16' ------- '1994-07-29' ------- '1994-08-09' ------- '1994-08-22' ------- '1994-09-04' ------- '1994-09-15' ------- '1994-09-29' ------- '1994-10-10' ------- '1994-10-20' ------- '1994-11-03' ------- '1994-11-16' ------- '1994-11-29' ------- '1994-12-11' ------- '1994-12-25' ------- '1994-12-31' │ │ │ │ │ │ └── scan lineitem@l_sd │ │ │ │ │ │ ├── save-table-name: q20_scan_13 - │ │ │ │ │ │ ├── columns: l_orderkey:26(int!null) l_linenumber:29(int!null) l_shipdate:36(date!null) - │ │ │ │ │ │ ├── constraint: /36/26/29: [/'1994-01-01' - /'1994-12-31'] - │ │ │ │ │ │ ├── stats: [rows=938137.516, distinct(26)=744144.6, null(26)=0, distinct(29)=7, null(29)=0, distinct(36)=365, null(36)=0] - │ │ │ │ │ │ │ histogram(26)= 0 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 + │ │ │ │ │ │ ├── columns: l_orderkey:30(int!null) l_linenumber:33(int!null) l_shipdate:40(date!null) + │ │ │ │ │ │ ├── constraint: /40/30/33: [/'1994-01-01' - /'1994-12-31'] + │ │ │ │ │ │ ├── stats: [rows=938137.516, distinct(30)=744144.6, null(30)=0, distinct(33)=7, null(33)=0, distinct(40)=365, null(40)=0] + │ │ │ │ │ │ │ histogram(30)= 0 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4596.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 4690.7 93.795 │ │ │ │ │ │ │ <--- 326 --------- 28929 -------- 50503 -------- 89793 -------- 115938 -------- 146944 -------- 176768 -------- 211201 -------- 237860 -------- 266885 -------- 297604 -------- 330021 -------- 365889 -------- 398951 -------- 426117 -------- 451328 -------- 472134 -------- 499590 -------- 529284 -------- 557254 -------- 589154 -------- 619394 -------- 642951 -------- 670113 -------- 692931 -------- 721157 -------- 751687 -------- 777766 -------- 804582 -------- 836740 -------- 868868 -------- 898912 -------- 922500 -------- 946403 -------- 984870 -------- 1007936 -------- 1030117 -------- 1062275 -------- 1093572 -------- 1120709 -------- 1150981 -------- 1182786 -------- 1206406 -------- 1234116 -------- 1260961 -------- 1290502 -------- 1329510 -------- 1355426 -------- 1381313 -------- 1409796 -------- 1445254 -------- 1479233 -------- 1504935 -------- 1531079 -------- 1559650 -------- 1583616 -------- 1617504 -------- 1655749 -------- 1685185 -------- 1718183 -------- 1747716 -------- 1772131 -------- 1802372 -------- 1833315 -------- 1862403 -------- 1897894 -------- 1922819 -------- 1954405 -------- 1979329 -------- 2009859 -------- 2041670 -------- 2070851 -------- 2093828 -------- 2127973 -------- 2167777 -------- 2194883 -------- 2227814 -------- 2262437 -------- 2296353 -------- 2321024 -------- 2346051 -------- 2376257 -------- 2404932 -------- 2446273 -------- 2474081 -------- 2504515 -------- 2535302 -------- 2561413 -------- 2592737 -------- 2616801 -------- 2646112 -------- 2676546 -------- 2702116 -------- 2732454 -------- 2765382 -------- 2799495 -------- 2828866 -------- 2868737 -------- 2910625 -------- 2938464 -------- 2963140 -------- 3003302 -------- 3043264 -------- 3069123 -------- 3095909 -------- 3126693 -------- 3160485 -------- 3196039 -------- 3229504 -------- 3259712 -------- 3286439 -------- 3318852 -------- 3346821 -------- 3370119 -------- 3395204 -------- 3425888 -------- 3448611 -------- 3476130 -------- 3502372 -------- 3529474 -------- 3556390 -------- 3583553 -------- 3612550 -------- 3647875 -------- 3679140 -------- 3702661 -------- 3738017 -------- 3778050 -------- 3806114 -------- 3839074 -------- 3872805 -------- 3905697 -------- 3926212 -------- 3959841 -------- 3997281 -------- 4033861 -------- 4063591 -------- 4097831 -------- 4124807 -------- 4158656 -------- 4195748 -------- 4234274 -------- 4269952 -------- 4298949 -------- 4332806 -------- 4364705 -------- 4398246 -------- 4430695 -------- 4466403 -------- 4494662 -------- 4524420 -------- 4558561 -------- 4601092 -------- 4632871 -------- 4658694 -------- 4690501 -------- 4728066 -------- 4758657 -------- 4788294 -------- 4818597 -------- 4855874 -------- 4890913 -------- 4915366 -------- 4940709 -------- 4972357 -------- 4995298 -------- 5019523 -------- 5043329 -------- 5077376 -------- 5109920 -------- 5136582 -------- 5161152 -------- 5191846 -------- 5219973 -------- 5251015 -------- 5282021 -------- 5312355 -------- 5343207 -------- 5381318 -------- 5416163 -------- 5445382 -------- 5476933 -------- 5509185 -------- 5539237 -------- 5566818 -------- 5588739 -------- 5620481 -------- 5644001 -------- 5667010 -------- 5689476 -------- 5724709 -------- 5755398 -------- 5790598 -------- 5819425 -------- 5846341 -------- 5874656 -------- 5908067 -------- 5933572 -------- 5962659 -------- 5999971 - │ │ │ │ │ │ │ histogram(36)= 0 0 3150.6 4200 25805 3600 25805 3600 26405 3600 28805 1200 27605 2400 27005 3600 28805 1200 27005 4800 28205 3000 25205 4800 28805 4800 28805 600 28205 1200 24604 6601 24604 4800 28205 3600 27605 3600 27005 4200 28205 2400 28205 4200 28805 1200 28205 1800 28205 1200 28205 3000 27005 4800 28805 600 27005 3600 25805 5401 27005 2400 28205 3000 11502 2300.4 + │ │ │ │ │ │ │ histogram(40)= 0 0 3150.6 4200 25805 3600 25805 3600 26405 3600 28805 1200 27605 2400 27005 3600 28805 1200 27005 4800 28205 3000 25205 4800 28805 4800 28805 600 28205 1200 24604 6601 24604 4800 28205 3600 27605 3600 27005 4200 28205 2400 28205 4200 28805 1200 28205 1800 28205 1200 28205 3000 27005 4800 28805 600 27005 3600 25805 5401 27005 2400 28205 3000 11502 2300.4 │ │ │ │ │ │ │ <--- '1993-12-31' -------- '1994-01-02' ------- '1994-01-16' ------- '1994-01-30' ------- '1994-02-08' ------- '1994-02-18' ------- '1994-03-04' ------- '1994-03-15' ------- '1994-03-29' ------- '1994-04-10' ------- '1994-04-24' ------- '1994-05-03' ------- '1994-05-13' ------- '1994-05-24' ------- '1994-06-04' ------- '1994-06-13' ------- '1994-06-22' ------- '1994-07-05' ------- '1994-07-16' ------- '1994-07-29' ------- '1994-08-09' ------- '1994-08-22' ------- '1994-09-04' ------- '1994-09-15' ------- '1994-09-29' ------- '1994-10-10' ------- '1994-10-20' ------- '1994-11-03' ------- '1994-11-16' ------- '1994-11-29' ------- '1994-12-11' ------- '1994-12-25' ------- '1994-12-31' - │ │ │ │ │ │ ├── key: (26,29) - │ │ │ │ │ │ └── fd: (26,29)-->(36) + │ │ │ │ │ │ ├── key: (30,33) + │ │ │ │ │ │ └── fd: (30,33)-->(40) │ │ │ │ │ ├── scan partsupp │ │ │ │ │ │ ├── save-table-name: q20_scan_14 - │ │ │ │ │ │ ├── columns: ps_partkey:12(int!null) ps_suppkey:13(int!null) ps_availqty:14(int!null) - │ │ │ │ │ │ ├── stats: [rows=800000, distinct(12)=199241, null(12)=0, distinct(13)=9920, null(13)=0, distinct(14)=9920, null(14)=0, distinct(12,13)=798302, null(12,13)=0] - │ │ │ │ │ │ │ histogram(12)= 0 80 3920 80 3920 80 3920 80 3920 80 3840 160 3840 160 3920 80 3920 80 3920 80 3920 80 3920 160 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3840 160 3920 80 3920 80 3920 80 3920 80 3920 160 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3840 160 3920 80 3920 80 3920 160 3920 80 3920 160 3920 80 3920 80 3920 80 3920 80 3920 80 3920 160 3920 80 3920 80 3920 80 3920 80 3920 80 3920 160 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 3920 160 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 + │ │ │ │ │ │ ├── columns: ps_partkey:14(int!null) ps_suppkey:15(int!null) ps_availqty:16(int!null) + │ │ │ │ │ │ ├── stats: [rows=800000, distinct(14)=199241, null(14)=0, distinct(15)=9920, null(15)=0, distinct(16)=9920, null(16)=0, distinct(14,15)=798302, null(14,15)=0] + │ │ │ │ │ │ │ histogram(14)= 0 80 3920 80 3920 80 3920 80 3920 80 3840 160 3840 160 3920 80 3920 80 3920 80 3920 80 3920 160 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3840 160 3920 80 3920 80 3920 80 3920 80 3920 160 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3840 160 3920 80 3920 80 3920 160 3920 80 3920 160 3920 80 3920 80 3920 80 3920 80 3920 80 3920 160 3920 80 3920 80 3920 80 3920 80 3920 80 3920 160 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 3920 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 3920 160 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 4000 80 │ │ │ │ │ │ │ <--- 4 ------ 793 ------ 1856 ------ 2808 ------ 3809 ------ 4923 ------ 5975 ------ 6944 ------ 8085 ------ 8945 ------ 9831 ------ 10952 ------ 11932 ------ 12775 ------ 13849 ------ 14925 ------ 16016 ------ 16828 ------ 17768 ------ 18705 ------ 19631 ------ 20600 ------ 21491 ------ 22254 ------ 23327 ------ 24491 ------ 25470 ------ 26331 ------ 27327 ------ 28308 ------ 29359 ------ 30542 ------ 31592 ------ 32495 ------ 33408 ------ 34624 ------ 35726 ------ 36676 ------ 37637 ------ 38485 ------ 39368 ------ 40372 ------ 41034 ------ 42185 ------ 43333 ------ 44466 ------ 45440 ------ 46571 ------ 47469 ------ 48346 ------ 49379 ------ 50571 ------ 51830 ------ 52967 ------ 53673 ------ 54656 ------ 55504 ------ 56539 ------ 57393 ------ 58366 ------ 59577 ------ 60559 ------ 61676 ------ 62471 ------ 63421 ------ 64449 ------ 65409 ------ 66254 ------ 67127 ------ 68127 ------ 69177 ------ 70183 ------ 71209 ------ 72101 ------ 73102 ------ 73994 ------ 74899 ------ 76016 ------ 77098 ------ 77842 ------ 79137 ------ 80242 ------ 81364 ------ 82331 ------ 83158 ------ 84283 ------ 85282 ------ 86437 ------ 87450 ------ 88419 ------ 89493 ------ 90478 ------ 91468 ------ 92552 ------ 93200 ------ 94191 ------ 95067 ------ 96272 ------ 97228 ------ 98126 ------ 99198 ------ 100219 ------ 101057 ------ 102038 ------ 102775 ------ 103711 ------ 104623 ------ 105710 ------ 106734 ------ 107932 ------ 109255 ------ 110220 ------ 111235 ------ 112174 ------ 113260 ------ 114081 ------ 115103 ------ 115864 ------ 116794 ------ 117741 ------ 118712 ------ 119470 ------ 120528 ------ 121572 ------ 122536 ------ 123629 ------ 124404 ------ 125301 ------ 126257 ------ 127139 ------ 128267 ------ 129258 ------ 130442 ------ 131845 ------ 133164 ------ 134005 ------ 135076 ------ 135868 ------ 137297 ------ 138777 ------ 139806 ------ 140741 ------ 141896 ------ 142719 ------ 143727 ------ 144645 ------ 145510 ------ 146507 ------ 147449 ------ 148467 ------ 149635 ------ 150563 ------ 151751 ------ 152613 ------ 153416 ------ 154612 ------ 155853 ------ 156866 ------ 158311 ------ 159230 ------ 160390 ------ 161455 ------ 162555 ------ 163435 ------ 164549 ------ 165663 ------ 166891 ------ 167757 ------ 168732 ------ 169644 ------ 170532 ------ 171671 ------ 172778 ------ 173599 ------ 174321 ------ 175624 ------ 176663 ------ 177632 ------ 178555 ------ 179551 ------ 180510 ------ 181682 ------ 182648 ------ 183408 ------ 184543 ------ 185722 ------ 186713 ------ 187787 ------ 188730 ------ 189604 ------ 190711 ------ 191690 ------ 192692 ------ 193702 ------ 194685 ------ 195725 ------ 196730 ------ 197724 ------ 198701 ------ 199973 - │ │ │ │ │ │ │ histogram(13)= 0 80 3920 240 3920 80 3920 160 3920 80 3920 240 3920 80 3760 320 3680 320 3920 80 3920 160 3920 240 3920 80 3920 160 3840 160 3920 80 3920 80 3760 240 3840 160 3920 80 3840 160 3680 320 3920 80 3840 160 3840 160 3760 320 3840 160 3840 160 3920 80 3840 240 3920 80 3920 80 3840 160 3760 240 3920 160 3920 80 3920 80 3920 80 3920 320 3920 80 3920 160 3840 400 3760 240 3920 160 3920 160 3600 480 3920 80 3680 320 3840 160 3840 160 3920 240 3840 160 3920 160 3920 80 3920 160 3920 80 3760 240 3920 80 3920 80 3840 320 3840 160 3840 160 3920 240 3840 480 3920 160 3840 240 3920 160 3920 160 3920 80 3840 160 3920 80 3920 80 3920 80 3920 80 3840 240 3840 240 3920 80 3840 320 3920 80 3920 80 3920 240 3840 240 3920 160 3920 80 3840 160 3840 240 3920 240 3840 80 3680 320 3920 160 3840 160 3840 80 3920 80 3840 160 3840 160 3920 80 3920 80 3840 160 3920 80 3920 160 3840 240 3840 80 3840 160 3760 160 3920 80 3920 80 3840 240 3760 240 3840 80 3920 160 3840 80 3920 80 3920 80 3920 80 3920 160 3840 80 3920 80 3760 240 3920 80 3920 160 3760 160 3920 160 3840 80 3920 160 3840 160 3840 160 3600 320 3920 160 3840 80 3920 80 3680 320 3840 240 3760 160 3920 80 3920 80 3920 80 3920 80 3920 80 3680 320 3920 160 3840 160 3760 160 3920 240 3840 160 3840 240 3600 320 3840 80 3840 80 3920 160 3760 160 3840 160 3840 320 3840 80 3840 160 3760 240 3840 80 3840 240 3760 160 3840 160 3840 160 3920 240 3760 160 3840 80 3920 160 3680 240 3840 160 3840 160 3760 240 3920 80 3920 240 3760 160 3760 240 3840 80 3840 240 3840 240 3760 320 3760 240 3840 80 3840 160 3840 240 3760 320 3760 160 3840 160 3840 160 3840 80 3760 160 3840 80 3840 160 3920 160 3840 80 3920 80 3840 160 3920 80 3840 240 3840 80 3920 80 3760 240 3920 240 3840 80 3680 240 + │ │ │ │ │ │ │ histogram(15)= 0 80 3920 240 3920 80 3920 160 3920 80 3920 240 3920 80 3760 320 3680 320 3920 80 3920 160 3920 240 3920 80 3920 160 3840 160 3920 80 3920 80 3760 240 3840 160 3920 80 3840 160 3680 320 3920 80 3840 160 3840 160 3760 320 3840 160 3840 160 3920 80 3840 240 3920 80 3920 80 3840 160 3760 240 3920 160 3920 80 3920 80 3920 80 3920 320 3920 80 3920 160 3840 400 3760 240 3920 160 3920 160 3600 480 3920 80 3680 320 3840 160 3840 160 3920 240 3840 160 3920 160 3920 80 3920 160 3920 80 3760 240 3920 80 3920 80 3840 320 3840 160 3840 160 3920 240 3840 480 3920 160 3840 240 3920 160 3920 160 3920 80 3840 160 3920 80 3920 80 3920 80 3920 80 3840 240 3840 240 3920 80 3840 320 3920 80 3920 80 3920 240 3840 240 3920 160 3920 80 3840 160 3840 240 3920 240 3840 80 3680 320 3920 160 3840 160 3840 80 3920 80 3840 160 3840 160 3920 80 3920 80 3840 160 3920 80 3920 160 3840 240 3840 80 3840 160 3760 160 3920 80 3920 80 3840 240 3760 240 3840 80 3920 160 3840 80 3920 80 3920 80 3920 80 3920 160 3840 80 3920 80 3760 240 3920 80 3920 160 3760 160 3920 160 3840 80 3920 160 3840 160 3840 160 3600 320 3920 160 3840 80 3920 80 3680 320 3840 240 3760 160 3920 80 3920 80 3920 80 3920 80 3920 80 3680 320 3920 160 3840 160 3760 160 3920 240 3840 160 3840 240 3600 320 3840 80 3840 80 3920 160 3760 160 3840 160 3840 320 3840 80 3840 160 3760 240 3840 80 3840 240 3760 160 3840 160 3840 160 3920 240 3760 160 3840 80 3920 160 3680 240 3840 160 3840 160 3760 240 3920 80 3920 240 3760 160 3760 240 3840 80 3840 240 3840 240 3760 320 3760 240 3840 80 3840 160 3840 240 3760 320 3760 160 3840 160 3840 160 3840 80 3760 160 3840 80 3840 160 3920 160 3840 80 3920 80 3840 160 3920 80 3840 240 3840 80 3920 80 3760 240 3920 240 3840 80 3680 240 │ │ │ │ │ │ │ <--- 2 ------ 49 ------ 90 ------ 141 ------ 183 ------ 235 ------ 278 ------ 319 ------ 360 ------ 406 ------ 458 ------ 511 ------ 561 ------ 622 ------ 674 ------ 731 ------ 781 ------ 822 ------ 882 ------ 934 ------ 988 ------ 1026 ------ 1072 ------ 1114 ------ 1188 ------ 1245 ------ 1291 ------ 1335 ------ 1380 ------ 1433 ------ 1488 ------ 1537 ------ 1590 ------ 1642 ------ 1692 ------ 1751 ------ 1807 ------ 1846 ------ 1887 ------ 1939 ------ 1994 ------ 2045 ------ 2097 ------ 2139 ------ 2190 ------ 2240 ------ 2293 ------ 2344 ------ 2385 ------ 2427 ------ 2484 ------ 2538 ------ 2601 ------ 2651 ------ 2730 ------ 2781 ------ 2825 ------ 2874 ------ 2938 ------ 2978 ------ 3034 ------ 3086 ------ 3139 ------ 3179 ------ 3231 ------ 3274 ------ 3334 ------ 3387 ------ 3431 ------ 3477 ------ 3530 ------ 3581 ------ 3633 ------ 3680 ------ 3725 ------ 3775 ------ 3820 ------ 3876 ------ 3923 ------ 3982 ------ 4036 ------ 4078 ------ 4116 ------ 4183 ------ 4227 ------ 4275 ------ 4336 ------ 4376 ------ 4424 ------ 4481 ------ 4537 ------ 4582 ------ 4630 ------ 4680 ------ 4727 ------ 4779 ------ 4828 ------ 4881 ------ 4938 ------ 4986 ------ 5040 ------ 5087 ------ 5138 ------ 5188 ------ 5237 ------ 5280 ------ 5318 ------ 5358 ------ 5405 ------ 5459 ------ 5516 ------ 5561 ------ 5615 ------ 5681 ------ 5744 ------ 5790 ------ 5847 ------ 5885 ------ 5927 ------ 5991 ------ 6042 ------ 6111 ------ 6165 ------ 6207 ------ 6259 ------ 6313 ------ 6359 ------ 6418 ------ 6471 ------ 6530 ------ 6587 ------ 6626 ------ 6672 ------ 6739 ------ 6784 ------ 6837 ------ 6886 ------ 6952 ------ 6994 ------ 7040 ------ 7081 ------ 7134 ------ 7178 ------ 7232 ------ 7280 ------ 7330 ------ 7378 ------ 7435 ------ 7486 ------ 7537 ------ 7593 ------ 7636 ------ 7680 ------ 7737 ------ 7788 ------ 7836 ------ 7877 ------ 7928 ------ 7993 ------ 8036 ------ 8083 ------ 8135 ------ 8180 ------ 8221 ------ 8263 ------ 8313 ------ 8352 ------ 8399 ------ 8453 ------ 8517 ------ 8566 ------ 8612 ------ 8664 ------ 8716 ------ 8766 ------ 8821 ------ 8871 ------ 8922 ------ 8956 ------ 9007 ------ 9050 ------ 9100 ------ 9154 ------ 9203 ------ 9246 ------ 9311 ------ 9358 ------ 9407 ------ 9470 ------ 9525 ------ 9564 ------ 9633 ------ 9672 ------ 9730 ------ 9778 ------ 9824 ------ 9868 ------ 9919 ------ 9959 ------ 10000 - │ │ │ │ │ │ ├── key: (12,13) - │ │ │ │ │ │ └── fd: (12,13)-->(14) + │ │ │ │ │ │ ├── key: (14,15) + │ │ │ │ │ │ └── fd: (14,15)-->(16) │ │ │ │ │ └── filters - │ │ │ │ │ ├── l_partkey:27 = ps_partkey:12 [type=bool, outer=(12,27), constraints=(/12: (/NULL - ]; /27: (/NULL - ]), fd=(12)==(27), (27)==(12)] - │ │ │ │ │ └── l_suppkey:28 = ps_suppkey:13 [type=bool, outer=(13,28), constraints=(/13: (/NULL - ]; /28: (/NULL - ]), fd=(13)==(28), (28)==(13)] + │ │ │ │ │ ├── l_partkey:31 = ps_partkey:14 [type=bool, outer=(14,31), constraints=(/14: (/NULL - ]; /31: (/NULL - ]), fd=(14)==(31), (31)==(14)] + │ │ │ │ │ └── l_suppkey:32 = ps_suppkey:15 [type=bool, outer=(15,32), constraints=(/15: (/NULL - ]; /32: (/NULL - ]), fd=(15)==(32), (32)==(15)] │ │ │ │ └── aggregations - │ │ │ │ ├── sum [as=sum:42, type=float, outer=(30)] - │ │ │ │ │ └── l_quantity:30 [type=float] - │ │ │ │ └── const-agg [as=ps_availqty:14, type=int, outer=(14)] - │ │ │ │ └── ps_availqty:14 [type=int] + │ │ │ │ ├── sum [as=sum:47, type=float, outer=(34)] + │ │ │ │ │ └── l_quantity:34 [type=float] + │ │ │ │ └── const-agg [as=ps_availqty:16, type=int, outer=(16)] + │ │ │ │ └── ps_availqty:16 [type=int] │ │ │ └── filters - │ │ │ └── ps_availqty:14 > (sum:42 * 0.5) [type=bool, outer=(14,42), immutable, constraints=(/14: (/NULL - ])] + │ │ │ └── ps_availqty:16 > (sum:47 * 0.5) [type=bool, outer=(16,47), immutable, constraints=(/16: (/NULL - ])] │ │ ├── select │ │ │ ├── save-table-name: q20_select_15 - │ │ │ ├── columns: p_partkey:17(int!null) p_name:18(varchar!null) - │ │ │ ├── stats: [rows=22222.2222, distinct(17)=22217.3354, null(17)=0, distinct(18)=22014.5556, null(18)=0] - │ │ │ ├── key: (17) - │ │ │ ├── fd: (17)-->(18) + │ │ │ ├── columns: p_partkey:20(int!null) p_name:21(varchar!null) + │ │ │ ├── stats: [rows=22222.2222, distinct(20)=22217.3354, null(20)=0, distinct(21)=22014.5556, null(21)=0] + │ │ │ ├── key: (20) + │ │ │ ├── fd: (20)-->(21) │ │ │ ├── scan part │ │ │ │ ├── save-table-name: q20_scan_16 - │ │ │ │ ├── columns: p_partkey:17(int!null) p_name:18(varchar!null) - │ │ │ │ ├── stats: [rows=200000, distinct(17)=199241, null(17)=0, distinct(18)=198131, null(18)=0] - │ │ │ │ │ histogram(17)= 0 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 + │ │ │ │ ├── columns: p_partkey:20(int!null) p_name:21(varchar!null) + │ │ │ │ ├── stats: [rows=200000, distinct(20)=199241, null(20)=0, distinct(21)=198131, null(21)=0] + │ │ │ │ │ histogram(20)= 0 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 980 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 1000 20 │ │ │ │ │ <--- 27 ----- 1110 ----- 2241 ----- 3086 ----- 4134 ----- 5302 ----- 6222 ----- 7308 ----- 8249 ----- 9171 ----- 10049 ----- 10958 ----- 11832 ----- 13025 ----- 14063 ----- 14953 ----- 16249 ----- 17419 ----- 18363 ----- 19412 ----- 20257 ----- 21190 ----- 22110 ----- 23045 ----- 23956 ----- 24962 ----- 25942 ----- 26990 ----- 27934 ----- 28876 ----- 29513 ----- 30326 ----- 31259 ----- 32300 ----- 33577 ----- 34550 ----- 35562 ----- 36498 ----- 37475 ----- 38584 ----- 39641 ----- 40548 ----- 41605 ----- 42527 ----- 43612 ----- 44702 ----- 45701 ----- 46726 ----- 47795 ----- 48935 ----- 50152 ----- 51183 ----- 52001 ----- 52904 ----- 53868 ----- 54808 ----- 55986 ----- 57155 ----- 58516 ----- 59526 ----- 60557 ----- 61547 ----- 62369 ----- 63672 ----- 64583 ----- 65360 ----- 66147 ----- 67201 ----- 68142 ----- 69145 ----- 70209 ----- 71141 ----- 71923 ----- 73031 ----- 73987 ----- 74974 ----- 76170 ----- 77138 ----- 77849 ----- 78931 ----- 79832 ----- 80761 ----- 81843 ----- 82834 ----- 84032 ----- 85072 ----- 86287 ----- 87302 ----- 88422 ----- 89432 ----- 90550 ----- 91463 ----- 92249 ----- 93385 ----- 94789 ----- 96013 ----- 96893 ----- 98000 ----- 99008 ----- 100166 ----- 101263 ----- 102351 ----- 103236 ----- 104121 ----- 105363 ----- 106329 ----- 107325 ----- 108231 ----- 109054 ----- 110019 ----- 111185 ----- 112112 ----- 112908 ----- 113904 ----- 114785 ----- 115410 ----- 116526 ----- 117559 ----- 118310 ----- 119073 ----- 120034 ----- 120817 ----- 121744 ----- 122566 ----- 123720 ----- 124813 ----- 125835 ----- 126622 ----- 127651 ----- 128328 ----- 129315 ----- 130244 ----- 131450 ----- 132439 ----- 133288 ----- 134164 ----- 135298 ----- 136347 ----- 137243 ----- 138256 ----- 139427 ----- 140374 ----- 141371 ----- 142302 ----- 143322 ----- 144335 ----- 145333 ----- 146212 ----- 147321 ----- 148591 ----- 149594 ------ 150514 ------ 151361 ------ 152059 ------ 153070 ------ 154059 ------ 155259 ------ 156473 ------ 157690 ------ 158703 ------ 159675 ------ 160597 ------ 161668 ------ 162737 ------ 163955 ------ 164942 ------ 165924 ------ 167059 ------ 167866 ------ 169034 ------ 169935 ------ 170712 ------ 171806 ------ 172841 ------ 174078 ------ 175347 ------ 176430 ------ 177346 ------ 178566 ------ 179515 ------ 180677 ------ 181729 ------ 182983 ------ 183814 ------ 184892 ------ 185696 ------ 186611 ------ 187744 ------ 188974 ------ 189911 ------ 190671 ------ 191607 ------ 192820 ------ 193789 ------ 195057 ------ 196224 ------ 197231 ------ 198281 ------ 199119 ------ 199999 - │ │ │ │ ├── key: (17) - │ │ │ │ └── fd: (17)-->(18) + │ │ │ │ ├── key: (20) + │ │ │ │ └── fd: (20)-->(21) │ │ │ └── filters - │ │ │ └── p_name:18 LIKE 'forest%' [type=bool, outer=(18), constraints=(/18: [/'forest' - /'foresu'); tight)] + │ │ │ └── p_name:21 LIKE 'forest%' [type=bool, outer=(21), constraints=(/21: [/'forest' - /'foresu'); tight)] │ │ └── filters - │ │ └── ps_partkey:12 = p_partkey:17 [type=bool, outer=(12,17), constraints=(/12: (/NULL - ]; /17: (/NULL - ]), fd=(12)==(17), (17)==(12)] + │ │ └── ps_partkey:14 = p_partkey:20 [type=bool, outer=(14,20), constraints=(/14: (/NULL - ]; /20: (/NULL - ]), fd=(14)==(20), (20)==(14)] │ └── filters - │ └── s_suppkey:1 = ps_suppkey:13 [type=bool, outer=(1,13), constraints=(/1: (/NULL - ]; /13: (/NULL - ]), fd=(1)==(13), (13)==(1)] + │ └── s_suppkey:1 = ps_suppkey:15 [type=bool, outer=(1,15), constraints=(/1: (/NULL - ]; /15: (/NULL - ]), fd=(1)==(15), (15)==(1)] ├── select │ ├── save-table-name: q20_select_17 - │ ├── columns: n_nationkey:8(int!null) n_name:9(char!null) - │ ├── stats: [rows=1, distinct(8)=1, null(8)=0, distinct(9)=1, null(9)=0] - │ ├── key: (8) - │ ├── fd: ()-->(9) + │ ├── columns: n_nationkey:9(int!null) n_name:10(char!null) + │ ├── stats: [rows=1, distinct(9)=1, null(9)=0, distinct(10)=1, null(10)=0] + │ ├── key: (9) + │ ├── fd: ()-->(10) │ ├── scan nation │ │ ├── save-table-name: q20_scan_18 - │ │ ├── columns: n_nationkey:8(int!null) n_name:9(char!null) - │ │ ├── stats: [rows=25, distinct(8)=25, null(8)=0, distinct(9)=25, null(9)=0] - │ │ │ histogram(8)= 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 + │ │ ├── columns: n_nationkey:9(int!null) n_name:10(char!null) + │ │ ├── stats: [rows=25, distinct(9)=25, null(9)=0, distinct(10)=25, null(10)=0] + │ │ │ histogram(9)= 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 --- 5 --- 6 --- 7 --- 8 --- 9 --- 10 --- 11 --- 12 --- 13 --- 14 --- 15 --- 16 --- 17 --- 18 --- 19 --- 20 --- 21 --- 22 --- 23 --- 24 - │ │ ├── key: (8) - │ │ └── fd: (8)-->(9) + │ │ ├── key: (9) + │ │ └── fd: (9)-->(10) │ └── filters - │ └── n_name:9 = 'CANADA' [type=bool, outer=(9), constraints=(/9: [/'CANADA' - /'CANADA']; tight), fd=()-->(9)] + │ └── n_name:10 = 'CANADA' [type=bool, outer=(10), constraints=(/10: [/'CANADA' - /'CANADA']; tight), fd=()-->(10)] └── filters - └── s_nationkey:4 = n_nationkey:8 [type=bool, outer=(4,8), constraints=(/4: (/NULL - ]; /8: (/NULL - ]), fd=(4)==(8), (8)==(4)] + └── s_nationkey:4 = n_nationkey:9 [type=bool, outer=(4,9), constraints=(/4: (/NULL - ]; /9: (/NULL - ]), fd=(4)==(9), (9)==(4)] stats table=q20_sort_1 ---- diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q21 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q21 index f651defada52..92d29541d2e6 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q21 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q21 @@ -58,143 +58,143 @@ LIMIT 100; ---- limit ├── save-table-name: q21_limit_1 - ├── columns: s_name:2(char!null) numwait:69(int!null) - ├── internal-ordering: -69,+2 + ├── columns: s_name:2(char!null) numwait:75(int!null) + ├── internal-ordering: -75,+2 ├── cardinality: [0 - 100] - ├── stats: [rows=100, distinct(2)=100, null(2)=0, distinct(69)=100, null(69)=0] + ├── stats: [rows=100, distinct(2)=100, null(2)=0, distinct(75)=100, null(75)=0] ├── key: (2) - ├── fd: (2)-->(69) - ├── ordering: -69,+2 + ├── fd: (2)-->(75) + ├── ordering: -75,+2 ├── sort │ ├── save-table-name: q21_sort_2 - │ ├── columns: s_name:2(char!null) count_rows:69(int!null) - │ ├── stats: [rows=8389.30056, distinct(2)=8389.30056, null(2)=0, distinct(69)=8389.30056, null(69)=0] + │ ├── columns: s_name:2(char!null) count_rows:75(int!null) + │ ├── stats: [rows=8389.30056, distinct(2)=8389.30056, null(2)=0, distinct(75)=8389.30056, null(75)=0] │ ├── key: (2) - │ ├── fd: (2)-->(69) - │ ├── ordering: -69,+2 + │ ├── fd: (2)-->(75) + │ ├── ordering: -75,+2 │ ├── limit hint: 100.00 │ └── group-by │ ├── save-table-name: q21_group_by_3 - │ ├── columns: s_name:2(char!null) count_rows:69(int!null) + │ ├── columns: s_name:2(char!null) count_rows:75(int!null) │ ├── grouping columns: s_name:2(char!null) - │ ├── stats: [rows=8389.30056, distinct(2)=8389.30056, null(2)=0, distinct(69)=8389.30056, null(69)=0] + │ ├── stats: [rows=8389.30056, distinct(2)=8389.30056, null(2)=0, distinct(75)=8389.30056, null(75)=0] │ ├── key: (2) - │ ├── fd: (2)-->(69) + │ ├── fd: (2)-->(75) │ ├── inner-join (lookup orders) │ │ ├── save-table-name: q21_lookup_join_4 - │ │ ├── columns: s_suppkey:1(int!null) s_name:2(char!null) s_nationkey:4(int!null) l1.l_orderkey:8(int!null) l1.l_suppkey:10(int!null) l1.l_commitdate:19(date!null) l1.l_receiptdate:20(date!null) o_orderkey:24(int!null) o_orderstatus:26(char!null) n_nationkey:33(int!null) n_name:34(char!null) - │ │ ├── key columns: [8] = [24] + │ │ ├── columns: s_suppkey:1(int!null) s_name:2(char!null) s_nationkey:4(int!null) l1.l_orderkey:9(int!null) l1.l_suppkey:11(int!null) l1.l_commitdate:20(date!null) l1.l_receiptdate:21(date!null) o_orderkey:26(int!null) o_orderstatus:28(char!null) n_nationkey:36(int!null) n_name:37(char!null) + │ │ ├── key columns: [9] = [26] │ │ ├── lookup columns are key - │ │ ├── stats: [rows=17924.776, distinct(1)=8350.92287, null(1)=0, distinct(2)=8389.30056, null(2)=0, distinct(4)=1, null(4)=0, distinct(8)=17713.0676, null(8)=0, distinct(10)=8350.92287, null(10)=0, distinct(19)=2464.51974, null(19)=0, distinct(20)=2552.02044, null(20)=0, distinct(24)=17713.0676, null(24)=0, distinct(26)=1, null(26)=0, distinct(33)=1, null(33)=0, distinct(34)=1, null(34)=0] - │ │ ├── fd: ()-->(26,34), (1)-->(2,4), (8)==(24), (24)==(8), (1)==(10), (10)==(1), (4)==(33), (33)==(4) + │ │ ├── stats: [rows=17924.776, distinct(1)=8350.92287, null(1)=0, distinct(2)=8389.30056, null(2)=0, distinct(4)=1, null(4)=0, distinct(9)=17713.0676, null(9)=0, distinct(11)=8350.92287, null(11)=0, distinct(20)=2464.51974, null(20)=0, distinct(21)=2552.02044, null(21)=0, distinct(26)=17713.0676, null(26)=0, distinct(28)=1, null(28)=0, distinct(36)=1, null(36)=0, distinct(37)=1, null(37)=0] + │ │ ├── fd: ()-->(28,37), (1)-->(2,4), (9)==(26), (26)==(9), (1)==(11), (11)==(1), (4)==(36), (36)==(4) │ │ ├── inner-join (hash) │ │ │ ├── save-table-name: q21_inner_join_5 - │ │ │ ├── columns: s_suppkey:1(int!null) s_name:2(char!null) s_nationkey:4(int!null) l1.l_orderkey:8(int!null) l1.l_suppkey:10(int!null) l1.l_commitdate:19(date!null) l1.l_receiptdate:20(date!null) n_nationkey:33(int!null) n_name:34(char!null) + │ │ │ ├── columns: s_suppkey:1(int!null) s_name:2(char!null) s_nationkey:4(int!null) l1.l_orderkey:9(int!null) l1.l_suppkey:11(int!null) l1.l_commitdate:20(date!null) l1.l_receiptdate:21(date!null) n_nationkey:36(int!null) n_name:37(char!null) │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ ├── stats: [rows=17924.776, distinct(1)=399.934613, null(1)=0, distinct(2)=399.991883, null(2)=0, distinct(4)=1, null(4)=0, distinct(8)=17569.066, null(8)=0, distinct(10)=399.934613, null(10)=0, distinct(19)=2464.2819, null(19)=0, distinct(20)=2551.71413, null(20)=0, distinct(33)=1, null(33)=0, distinct(34)=1, null(34)=0] - │ │ │ ├── fd: ()-->(34), (1)-->(2,4), (4)==(33), (33)==(4), (1)==(10), (10)==(1) + │ │ │ ├── stats: [rows=17924.776, distinct(1)=399.934613, null(1)=0, distinct(2)=399.991883, null(2)=0, distinct(4)=1, null(4)=0, distinct(9)=17569.066, null(9)=0, distinct(11)=399.934613, null(11)=0, distinct(20)=2464.2819, null(20)=0, distinct(21)=2551.71413, null(21)=0, distinct(36)=1, null(36)=0, distinct(37)=1, null(37)=0] + │ │ │ ├── fd: ()-->(37), (1)-->(2,4), (4)==(36), (36)==(4), (1)==(11), (11)==(1) │ │ │ ├── semi-join (hash) │ │ │ │ ├── save-table-name: q21_semi_join_6 - │ │ │ │ ├── columns: l1.l_orderkey:8(int!null) l1.l_suppkey:10(int!null) l1.l_commitdate:19(date!null) l1.l_receiptdate:20(date!null) - │ │ │ │ ├── stats: [rows=444534.444, distinct(8)=444534.444, null(8)=0, distinct(10)=9920, null(10)=0, distinct(19)=2466, null(19)=0, distinct(20)=2554, null(20)=0] + │ │ │ │ ├── columns: l1.l_orderkey:9(int!null) l1.l_suppkey:11(int!null) l1.l_commitdate:20(date!null) l1.l_receiptdate:21(date!null) + │ │ │ │ ├── stats: [rows=444534.444, distinct(9)=444534.444, null(9)=0, distinct(11)=9920, null(11)=0, distinct(20)=2466, null(20)=0, distinct(21)=2554, null(21)=0] │ │ │ │ ├── anti-join (merge) │ │ │ │ │ ├── save-table-name: q21_merge_join_7 - │ │ │ │ │ ├── columns: l1.l_orderkey:8(int!null) l1.l_suppkey:10(int!null) l1.l_commitdate:19(date!null) l1.l_receiptdate:20(date!null) - │ │ │ │ │ ├── left ordering: +8 - │ │ │ │ │ ├── right ordering: +53 - │ │ │ │ │ ├── stats: [rows=1333603.33, distinct(8)=1016901.08, null(8)=0, distinct(10)=9920, null(10)=0, distinct(19)=2466, null(19)=0, distinct(20)=2554, null(20)=0] + │ │ │ │ │ ├── columns: l1.l_orderkey:9(int!null) l1.l_suppkey:11(int!null) l1.l_commitdate:20(date!null) l1.l_receiptdate:21(date!null) + │ │ │ │ │ ├── left ordering: +9 + │ │ │ │ │ ├── right ordering: +58 + │ │ │ │ │ ├── stats: [rows=1333603.33, distinct(9)=1016901.08, null(9)=0, distinct(11)=9920, null(11)=0, distinct(20)=2466, null(20)=0, distinct(21)=2554, null(21)=0] │ │ │ │ │ ├── select │ │ │ │ │ │ ├── save-table-name: q21_select_8 - │ │ │ │ │ │ ├── columns: l1.l_orderkey:8(int!null) l1.l_suppkey:10(int!null) l1.l_commitdate:19(date!null) l1.l_receiptdate:20(date!null) - │ │ │ │ │ │ ├── stats: [rows=2000405, distinct(8)=1216823.04, null(8)=0, distinct(10)=9920, null(10)=0, distinct(19)=2466, null(19)=0, distinct(20)=2554, null(20)=0] - │ │ │ │ │ │ ├── ordering: +8 + │ │ │ │ │ │ ├── columns: l1.l_orderkey:9(int!null) l1.l_suppkey:11(int!null) l1.l_commitdate:20(date!null) l1.l_receiptdate:21(date!null) + │ │ │ │ │ │ ├── stats: [rows=2000405, distinct(9)=1216823.04, null(9)=0, distinct(11)=9920, null(11)=0, distinct(20)=2466, null(20)=0, distinct(21)=2554, null(21)=0] + │ │ │ │ │ │ ├── ordering: +9 │ │ │ │ │ │ ├── scan l1 │ │ │ │ │ │ │ ├── save-table-name: q21_scan_9 - │ │ │ │ │ │ │ ├── columns: l1.l_orderkey:8(int!null) l1.l_suppkey:10(int!null) l1.l_commitdate:19(date!null) l1.l_receiptdate:20(date!null) - │ │ │ │ │ │ │ ├── stats: [rows=6001215, distinct(8)=1527270, null(8)=0, distinct(10)=9920, null(10)=0, distinct(19)=2466, null(19)=0, distinct(20)=2554, null(20)=0] - │ │ │ │ │ │ │ │ histogram(8)= 0 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 + │ │ │ │ │ │ │ ├── columns: l1.l_orderkey:9(int!null) l1.l_suppkey:11(int!null) l1.l_commitdate:20(date!null) l1.l_receiptdate:21(date!null) + │ │ │ │ │ │ │ ├── stats: [rows=6001215, distinct(9)=1527270, null(9)=0, distinct(11)=9920, null(11)=0, distinct(20)=2466, null(20)=0, distinct(21)=2554, null(21)=0] + │ │ │ │ │ │ │ │ histogram(9)= 0 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 │ │ │ │ │ │ │ │ <--- 326 ------- 28929 ------- 50503 ------- 89793 ------- 115938 ------- 146944 ------- 176768 ------- 211201 ------- 237860 ------- 266885 ------- 297604 ------- 330021 ------- 365889 ------- 398951 ------- 426117 ------- 451328 ------- 472134 ------- 499590 ------- 529284 ------- 557254 ------- 589154 ------- 619394 ------- 642951 ------- 670113 ------- 692931 ------- 721157 ------- 751687 ------- 777766 ------- 804582 ------- 836740 ------- 868868 ------- 898912 ------- 922500 ------- 946403 ------- 984870 ------- 1007936 ------- 1030117 ------- 1062275 ------- 1093572 ------- 1120709 ------- 1150981 ------- 1182786 ------- 1206406 ------- 1234116 ------- 1260961 ------- 1290502 ------- 1329510 ------- 1355426 ------- 1381313 ------- 1409796 ------- 1445254 ------- 1479233 ------- 1504935 ------- 1531079 ------- 1559650 ------- 1583616 ------- 1617504 ------- 1655749 ------- 1685185 ------- 1718183 ------- 1747716 ------- 1772131 ------- 1802372 ------- 1833315 ------- 1862403 ------- 1897894 ------- 1922819 ------- 1954405 ------- 1979329 ------- 2009859 ------- 2041670 ------- 2070851 ------- 2093828 ------- 2127973 ------- 2167777 ------- 2194883 ------- 2227814 ------- 2262437 ------- 2296353 ------- 2321024 ------- 2346051 ------- 2376257 ------- 2404932 ------- 2446273 ------- 2474081 ------- 2504515 ------- 2535302 ------- 2561413 ------- 2592737 ------- 2616801 ------- 2646112 ------- 2676546 ------- 2702116 ------- 2732454 ------- 2765382 ------- 2799495 ------- 2828866 ------- 2868737 ------- 2910625 ------- 2938464 ------- 2963140 ------- 3003302 ------- 3043264 ------- 3069123 ------- 3095909 ------- 3126693 ------- 3160485 ------- 3196039 ------- 3229504 ------- 3259712 ------- 3286439 ------- 3318852 ------- 3346821 ------- 3370119 ------- 3395204 ------- 3425888 ------- 3448611 ------- 3476130 ------- 3502372 ------- 3529474 ------- 3556390 ------- 3583553 ------- 3612550 ------- 3647875 ------- 3679140 ------- 3702661 ------- 3738017 ------- 3778050 ------- 3806114 ------- 3839074 ------- 3872805 ------- 3905697 ------- 3926212 ------- 3959841 ------- 3997281 ------- 4033861 ------- 4063591 ------- 4097831 ------- 4124807 ------- 4158656 ------- 4195748 ------- 4234274 ------- 4269952 ------- 4298949 ------- 4332806 ------- 4364705 ------- 4398246 ------- 4430695 ------- 4466403 ------- 4494662 ------- 4524420 ------- 4558561 ------- 4601092 ------- 4632871 ------- 4658694 ------- 4690501 ------- 4728066 ------- 4758657 ------- 4788294 ------- 4818597 ------- 4855874 ------- 4890913 ------- 4915366 ------- 4940709 ------- 4972357 ------- 4995298 ------- 5019523 ------- 5043329 ------- 5077376 ------- 5109920 ------- 5136582 ------- 5161152 ------- 5191846 ------- 5219973 ------- 5251015 ------- 5282021 ------- 5312355 ------- 5343207 ------- 5381318 ------- 5416163 ------- 5445382 ------- 5476933 ------- 5509185 ------- 5539237 ------- 5566818 ------- 5588739 ------- 5620481 ------- 5644001 ------- 5667010 ------- 5689476 ------- 5724709 ------- 5755398 ------- 5790598 ------- 5819425 ------- 5846341 ------- 5874656 ------- 5908067 ------- 5933572 ------- 5962659 ------- 5999971 - │ │ │ │ │ │ │ │ histogram(10)= 0 600 28805 3000 29405 600 29405 600 29405 600 29405 600 29405 1200 29405 1200 29405 1200 28805 1200 28805 1800 29405 600 29405 1200 29405 600 29405 600 28805 1200 29405 1200 28805 1200 28205 1800 29405 1200 29405 600 29405 1200 29405 1200 29405 600 29405 1200 29405 1200 28805 1800 29405 600 28805 1800 28805 1200 28805 1200 29405 1200 28805 1800 29405 600 28205 1800 29405 600 29405 600 28805 1200 29405 600 28805 1200 29405 600 29405 600 28805 1200 28805 1800 27605 2400 29405 600 28205 1800 29405 2400 29405 600 28805 1800 28205 1800 28805 1200 29405 3000 29405 1200 29405 1800 29405 600 29405 600 28205 1800 29405 600 29405 1800 29405 1800 29405 600 29405 600 28805 1200 28805 1200 29405 1200 28205 1800 29405 600 28805 1200 27605 2400 29405 600 29405 1200 29405 1200 29405 600 29405 1800 29405 1200 28205 1800 29405 600 29405 1200 29405 1200 28805 1200 28205 1800 29405 1800 28805 1200 29405 600 29405 1800 28205 2400 29405 600 28805 1200 28205 2400 28805 1200 27605 2400 28805 1200 29405 600 28805 1800 29405 1800 28805 600 28205 1800 28805 600 29405 600 29405 1200 28805 600 28805 1200 29405 1800 27605 1800 27605 2400 27605 1800 29405 1200 28805 600 28805 1200 29405 1200 28805 2400 28805 1200 28805 600 28205 1200 28205 1200 29405 600 28805 1200 29405 600 28205 2400 28205 1200 29405 600 29405 1800 28805 1200 28205 1200 27605 2400 27605 1800 28805 1200 29405 600 28805 1200 29405 600 29405 600 28805 1200 29405 600 29405 600 29405 600 28805 1200 29405 1200 28805 600 29405 1800 28205 1200 27605 1800 29405 600 29405 600 29405 1200 28805 1200 28805 1200 28205 1200 29405 1800 28205 1800 28805 600 28805 1200 28205 1200 29405 600 28805 1800 28205 1800 27605 2400 28805 600 29405 600 28805 1200 29405 600 28805 2400 28805 1200 28805 1200 28805 600 28205 1200 28805 1200 29405 600 29405 600 29405 1800 28805 600 28805 600 28805 2400 28805 600 28805 1800 28805 1200 28805 600 27005 2400 29405 600 28205 1800 29405 1200 27605 1800 28805 1200 28805 1200 29405 600 29405 600 29405 600 29405 1200 28805 600 29405 600 27605 2400 28805 1200 29405 600 28805 1800 28805 1800 28205 1200 28805 1200 28805 600 29405 1200 28205 1200 + │ │ │ │ │ │ │ │ histogram(11)= 0 600 28805 3000 29405 600 29405 600 29405 600 29405 600 29405 1200 29405 1200 29405 1200 28805 1200 28805 1800 29405 600 29405 1200 29405 600 29405 600 28805 1200 29405 1200 28805 1200 28205 1800 29405 1200 29405 600 29405 1200 29405 1200 29405 600 29405 1200 29405 1200 28805 1800 29405 600 28805 1800 28805 1200 28805 1200 29405 1200 28805 1800 29405 600 28205 1800 29405 600 29405 600 28805 1200 29405 600 28805 1200 29405 600 29405 600 28805 1200 28805 1800 27605 2400 29405 600 28205 1800 29405 2400 29405 600 28805 1800 28205 1800 28805 1200 29405 3000 29405 1200 29405 1800 29405 600 29405 600 28205 1800 29405 600 29405 1800 29405 1800 29405 600 29405 600 28805 1200 28805 1200 29405 1200 28205 1800 29405 600 28805 1200 27605 2400 29405 600 29405 1200 29405 1200 29405 600 29405 1800 29405 1200 28205 1800 29405 600 29405 1200 29405 1200 28805 1200 28205 1800 29405 1800 28805 1200 29405 600 29405 1800 28205 2400 29405 600 28805 1200 28205 2400 28805 1200 27605 2400 28805 1200 29405 600 28805 1800 29405 1800 28805 600 28205 1800 28805 600 29405 600 29405 1200 28805 600 28805 1200 29405 1800 27605 1800 27605 2400 27605 1800 29405 1200 28805 600 28805 1200 29405 1200 28805 2400 28805 1200 28805 600 28205 1200 28205 1200 29405 600 28805 1200 29405 600 28205 2400 28205 1200 29405 600 29405 1800 28805 1200 28205 1200 27605 2400 27605 1800 28805 1200 29405 600 28805 1200 29405 600 29405 600 28805 1200 29405 600 29405 600 29405 600 28805 1200 29405 1200 28805 600 29405 1800 28205 1200 27605 1800 29405 600 29405 600 29405 1200 28805 1200 28805 1200 28205 1200 29405 1800 28205 1800 28805 600 28805 1200 28205 1200 29405 600 28805 1800 28205 1800 27605 2400 28805 600 29405 600 28805 1200 29405 600 28805 2400 28805 1200 28805 1200 28805 600 28205 1200 28805 1200 29405 600 29405 600 29405 1800 28805 600 28805 600 28805 2400 28805 600 28805 1800 28805 1200 28805 600 27005 2400 29405 600 28205 1800 29405 1200 27605 1800 28805 1200 28805 1200 29405 600 29405 600 29405 600 29405 1200 28805 600 29405 600 27605 2400 28805 1200 29405 600 28805 1800 28805 1800 28205 1200 28805 1200 28805 600 29405 1200 28205 1200 │ │ │ │ │ │ │ │ <--- 6 ------- 59 ------- 106 ------- 147 ------- 189 ------- 219 ------- 272 ------- 335 ------- 387 ------- 433 ------- 485 ------- 539 ------- 600 ------- 653 ------- 698 ------- 749 ------- 806 ------- 855 ------- 904 ------- 961 ------- 1008 ------- 1065 ------- 1115 ------- 1163 ------- 1212 ------- 1270 ------- 1329 ------- 1373 ------- 1417 ------- 1470 ------- 1514 ------- 1566 ------- 1616 ------- 1668 ------- 1719 ------- 1765 ------- 1819 ------- 1871 ------- 1926 ------- 1978 ------- 2030 ------- 2080 ------- 2128 ------- 2173 ------- 2223 ------- 2270 ------- 2318 ------- 2382 ------- 2444 ------- 2491 ------- 2534 ------- 2584 ------- 2632 ------- 2675 ------- 2736 ------- 2781 ------- 2841 ------- 2905 ------- 2940 ------- 2996 ------- 3045 ------- 3084 ------- 3124 ------- 3181 ------- 3224 ------- 3268 ------- 3318 ------- 3362 ------- 3419 ------- 3466 ------- 3515 ------- 3560 ------- 3615 ------- 3669 ------- 3725 ------- 3777 ------- 3825 ------- 3871 ------- 3923 ------- 3980 ------- 4035 ------- 4078 ------- 4135 ------- 4184 ------- 4244 ------- 4296 ------- 4346 ------- 4396 ------- 4444 ------- 4489 ------- 4540 ------- 4575 ------- 4627 ------- 4683 ------- 4731 ------- 4777 ------- 4831 ------- 4900 ------- 4953 ------- 5006 ------- 5061 ------- 5119 ------- 5161 ------- 5208 ------- 5260 ------- 5315 ------- 5361 ------- 5415 ------- 5474 ------- 5529 ------- 5572 ------- 5635 ------- 5689 ------- 5738 ------- 5792 ------- 5835 ------- 5890 ------- 5935 ------- 5997 ------- 6039 ------- 6094 ------- 6151 ------- 6201 ------- 6245 ------- 6291 ------- 6343 ------- 6387 ------- 6437 ------- 6480 ------- 6540 ------- 6580 ------- 6629 ------- 6673 ------- 6718 ------- 6771 ------- 6823 ------- 6878 ------- 6935 ------- 6974 ------- 7017 ------- 7060 ------- 7112 ------- 7170 ------- 7231 ------- 7279 ------- 7329 ------- 7373 ------- 7427 ------- 7477 ------- 7520 ------- 7564 ------- 7616 ------- 7671 ------- 7716 ------- 7760 ------- 7815 ------- 7861 ------- 7917 ------- 7969 ------- 8018 ------- 8063 ------- 8112 ------- 8180 ------- 8229 ------- 8268 ------- 8320 ------- 8376 ------- 8432 ------- 8474 ------- 8522 ------- 8569 ------- 8616 ------- 8664 ------- 8705 ------- 8765 ------- 8814 ------- 8868 ------- 8906 ------- 8948 ------- 8990 ------- 9035 ------- 9090 ------- 9137 ------- 9188 ------- 9241 ------- 9292 ------- 9354 ------- 9411 ------- 9472 ------- 9526 ------- 9569 ------- 9613 ------- 9658 ------- 9708 ------- 9737 ------- 9792 ------- 9838 ------- 9903 ------- 9952 ------- 10000 - │ │ │ │ │ │ │ │ histogram(19)= 0 600 28805 1200 29405 3600 29405 1200 29405 1800 28205 1800 28805 4200 28205 3600 27005 4800 29405 3600 29405 1800 26405 3600 29405 600 27005 3000 29405 2400 28805 3000 29405 600 25805 4800 28805 3000 27605 2400 27605 5401 28805 3600 28805 2400 27605 3600 28805 6001 28805 3600 27005 3000 25205 4800 28805 4200 28205 1200 28805 3600 28205 4200 27605 3000 28205 3600 28805 1200 27605 3000 28205 1200 27005 3000 28805 1200 28205 1800 28205 1800 27605 1800 27605 3600 27605 3600 25805 5401 28805 4200 28205 1200 28805 3000 28805 4800 28805 1200 28805 4200 28205 2400 26405 3000 28205 3000 28805 2400 28805 2400 27605 5401 28805 1800 28805 2400 28805 1200 28805 3600 27605 5401 25805 5401 28205 3600 27005 3600 28205 3000 28805 1200 28805 4800 28205 2400 24604 5401 28805 3000 27605 2400 28205 4200 27005 3000 28805 3000 28805 600 26405 5401 27605 4800 27005 2400 26405 4200 28805 2400 27605 3000 28205 3000 27005 2400 26405 3600 28205 1200 27605 4200 27605 3000 28205 1800 25805 3000 28205 2400 28205 1200 27005 2400 27605 1800 25805 3600 27005 6601 27005 1800 24004 6001 27605 1800 28205 3000 27605 2400 27605 2400 27605 3000 27605 2400 28205 3000 26405 2400 26405 2400 26405 2400 26405 3600 28205 1800 28205 2400 27605 1200 27605 3000 27605 4800 24604 4200 28205 3000 24604 6001 25805 4200 25205 4200 26405 3000 28205 3000 27605 3000 28205 600 27605 3600 25805 4200 27005 3600 28205 600 28205 3000 28205 1200 25805 3000 28205 1200 27005 3000 28205 1200 24604 4200 27605 1800 28205 4200 27605 2400 26405 4200 27005 2400 27005 1800 27605 3600 25205 4800 27605 2400 28205 1800 27005 3000 27605 3000 25205 4200 25805 3000 27605 3600 25805 3600 27605 1800 24604 4200 25805 3600 26405 1800 27005 3000 27005 1200 27005 3000 24004 5401 26405 1800 25805 2400 27605 1200 27605 600 27005 2400 27605 1200 23404 7201 25205 3600 26405 3000 27605 3600 27605 2400 24604 3600 27605 4200 27605 3600 27005 1200 26405 1200 25805 1800 25205 3600 22804 4800 24604 3000 24004 3600 26405 2400 24604 4800 26405 3000 22204 5401 25805 2400 25205 2400 27005 2400 24604 4200 25205 3000 24004 4800 25205 3000 26405 1200 27005 2400 26405 2400 26405 4800 25205 3000 25805 1800 24004 4800 24604 3000 23404 600 24004 600 + │ │ │ │ │ │ │ │ histogram(20)= 0 600 28805 1200 29405 3600 29405 1200 29405 1800 28205 1800 28805 4200 28205 3600 27005 4800 29405 3600 29405 1800 26405 3600 29405 600 27005 3000 29405 2400 28805 3000 29405 600 25805 4800 28805 3000 27605 2400 27605 5401 28805 3600 28805 2400 27605 3600 28805 6001 28805 3600 27005 3000 25205 4800 28805 4200 28205 1200 28805 3600 28205 4200 27605 3000 28205 3600 28805 1200 27605 3000 28205 1200 27005 3000 28805 1200 28205 1800 28205 1800 27605 1800 27605 3600 27605 3600 25805 5401 28805 4200 28205 1200 28805 3000 28805 4800 28805 1200 28805 4200 28205 2400 26405 3000 28205 3000 28805 2400 28805 2400 27605 5401 28805 1800 28805 2400 28805 1200 28805 3600 27605 5401 25805 5401 28205 3600 27005 3600 28205 3000 28805 1200 28805 4800 28205 2400 24604 5401 28805 3000 27605 2400 28205 4200 27005 3000 28805 3000 28805 600 26405 5401 27605 4800 27005 2400 26405 4200 28805 2400 27605 3000 28205 3000 27005 2400 26405 3600 28205 1200 27605 4200 27605 3000 28205 1800 25805 3000 28205 2400 28205 1200 27005 2400 27605 1800 25805 3600 27005 6601 27005 1800 24004 6001 27605 1800 28205 3000 27605 2400 27605 2400 27605 3000 27605 2400 28205 3000 26405 2400 26405 2400 26405 2400 26405 3600 28205 1800 28205 2400 27605 1200 27605 3000 27605 4800 24604 4200 28205 3000 24604 6001 25805 4200 25205 4200 26405 3000 28205 3000 27605 3000 28205 600 27605 3600 25805 4200 27005 3600 28205 600 28205 3000 28205 1200 25805 3000 28205 1200 27005 3000 28205 1200 24604 4200 27605 1800 28205 4200 27605 2400 26405 4200 27005 2400 27005 1800 27605 3600 25205 4800 27605 2400 28205 1800 27005 3000 27605 3000 25205 4200 25805 3000 27605 3600 25805 3600 27605 1800 24604 4200 25805 3600 26405 1800 27005 3000 27005 1200 27005 3000 24004 5401 26405 1800 25805 2400 27605 1200 27605 600 27005 2400 27605 1200 23404 7201 25205 3600 26405 3000 27605 3600 27605 2400 24604 3600 27605 4200 27605 3600 27005 1200 26405 1200 25805 1800 25205 3600 22804 4800 24604 3000 24004 3600 26405 2400 24604 4800 26405 3000 22204 5401 25805 2400 25205 2400 27005 2400 24604 4200 25205 3000 24004 4800 25205 3000 26405 1200 27005 2400 26405 2400 26405 4800 25205 3000 25805 1800 24004 4800 24604 3000 23404 600 24004 600 │ │ │ │ │ │ │ │ <--- '1992-02-06' ------- '1992-03-13' ------- '1992-03-29' ------- '1992-04-12' ------- '1992-04-24' ------- '1992-05-07' ------- '1992-05-18' ------- '1992-05-30' ------- '1992-06-15' ------- '1992-06-28' ------- '1992-07-10' ------- '1992-07-20' ------- '1992-08-04' ------- '1992-08-17' ------- '1992-08-30' ------- '1992-09-11' ------- '1992-09-25' ------- '1992-10-05' ------- '1992-10-16' ------- '1992-10-29' ------- '1992-11-10' ------- '1992-11-24' ------- '1992-12-05' ------- '1992-12-16' ------- '1992-12-31' ------- '1993-01-15' ------- '1993-01-28' ------- '1993-02-09' ------- '1993-02-21' ------- '1993-03-07' ------- '1993-03-17' ------- '1993-04-03' ------- '1993-04-17' ------- '1993-04-30' ------- '1993-05-12' ------- '1993-05-25' ------- '1993-06-08' ------- '1993-06-23' ------- '1993-07-04' ------- '1993-07-18' ------- '1993-08-02' ------- '1993-08-13' ------- '1993-08-24' ------- '1993-09-08' ------- '1993-09-20' ------- '1993-10-06' ------- '1993-10-22' ------- '1993-11-01' ------- '1993-11-14' ------- '1993-11-24' ------- '1993-12-07' ------- '1993-12-20' ------- '1994-01-01' ------- '1994-01-14' ------- '1994-01-31' ------- '1994-02-14' ------- '1994-02-26' ------- '1994-03-09' ------- '1994-03-22' ------- '1994-04-03' ------- '1994-04-14' ------- '1994-04-23' ------- '1994-05-07' ------- '1994-05-19' ------- '1994-05-31' ------- '1994-06-12' ------- '1994-06-22' ------- '1994-07-03' ------- '1994-07-14' ------- '1994-07-24' ------- '1994-08-04' ------- '1994-08-16' ------- '1994-08-29' ------- '1994-09-08' ------- '1994-09-23' ------- '1994-10-05' ------- '1994-10-16' ------- '1994-10-28' ------- '1994-11-08' ------- '1994-11-19' ------- '1994-12-05' ------- '1994-12-18' ------- '1994-12-29' ------- '1995-01-11' ------- '1995-01-23' ------- '1995-02-05' ------- '1995-02-15' ------- '1995-03-02' ------- '1995-03-14' ------- '1995-03-25' ------- '1995-04-05' ------- '1995-04-17' ------- '1995-04-30' ------- '1995-05-11' ------- '1995-05-23' ------- '1995-06-06' ------- '1995-06-20' ------- '1995-06-30' ------- '1995-07-12' ------- '1995-07-26' ------- '1995-08-07' ------- '1995-08-17' ------- '1995-08-28' ------- '1995-09-09' ------- '1995-09-21' ------- '1995-10-04' ------- '1995-10-14' ------- '1995-10-28' ------- '1995-11-07' ------- '1995-11-18' ------- '1995-11-29' ------- '1995-12-11' ------- '1995-12-21' ------- '1995-12-31' ------- '1996-01-11' ------- '1996-01-21' ------- '1996-01-31' ------- '1996-02-10' ------- '1996-02-24' ------- '1996-03-04' ------- '1996-03-13' ------- '1996-03-25' ------- '1996-04-07' ------- '1996-04-17' ------- '1996-04-28' ------- '1996-05-10' ------- '1996-05-22' ------- '1996-06-02' ------- '1996-06-15' ------- '1996-06-26' ------- '1996-07-09' ------- '1996-07-20' ------- '1996-07-31' ------- '1996-08-11' ------- '1996-08-25' ------- '1996-09-07' ------- '1996-09-21' ------- '1996-10-04' ------- '1996-10-15' ------- '1996-10-27' ------- '1996-11-07' ------- '1996-11-19' ------- '1996-11-30' ------- '1996-12-13' ------- '1996-12-27' ------- '1997-01-06' ------- '1997-01-18' ------- '1997-01-30' ------- '1997-02-12' ------- '1997-02-23' ------- '1997-03-07' ------- '1997-03-19' ------- '1997-03-31' ------- '1997-04-10' ------- '1997-04-23' ------- '1997-05-04' ------- '1997-05-17' ------- '1997-05-29' ------- '1997-06-12' ------- '1997-06-23' ------- '1997-07-04' ------- '1997-07-16' ------- '1997-07-28' ------- '1997-08-07' ------- '1997-08-19' ------- '1997-08-29' ------- '1997-09-10' ------- '1997-09-20' ------- '1997-10-03' ------- '1997-10-13' ------- '1997-10-25' ------- '1997-11-05' ------- '1997-11-17' ------- '1997-12-02' ------- '1997-12-15' ------- '1997-12-26' ------- '1998-01-07' ------- '1998-01-16' ------- '1998-01-25' ------- '1998-02-07' ------- '1998-02-20' ------- '1998-03-06' ------- '1998-03-14' ------- '1998-03-30' ------- '1998-04-08' ------- '1998-04-21' ------- '1998-05-03' ------- '1998-05-15' ------- '1998-05-27' ------- '1998-06-07' ------- '1998-06-19' ------- '1998-07-03' ------- '1998-07-15' ------- '1998-07-26' ------- '1998-08-07' ------- '1998-08-18' ------- '1998-08-30' ------- '1998-09-12' ------- '1998-10-01' ------- '1998-10-30' - │ │ │ │ │ │ │ │ histogram(20)= 0 1200 29405 1200 29405 1200 29405 1200 27605 2400 24004 6001 27605 5401 28205 4200 29405 2400 29405 1200 28205 4200 27005 3000 27005 3000 29405 4800 27605 3000 28205 1800 27605 2400 26405 3600 27605 4800 27605 3600 28805 3000 28205 6001 28205 1800 26405 3600 27005 4200 29405 4200 27605 2400 27605 1800 27005 3600 24604 4800 27605 3000 27605 1800 28205 2400 28205 3000 28805 1800 28805 600 28205 1200 27005 2400 27605 1800 26405 3000 26405 4800 28205 2400 28805 1200 28805 3600 28205 3000 28205 1200 28205 1200 25805 4200 28205 4800 26405 6001 28205 1800 27005 3000 28205 1800 27605 3000 28205 1200 28205 2400 27605 3000 28805 1800 26405 4200 28805 1800 24604 4800 28805 1200 27005 3000 26405 7201 25805 4800 28805 4200 28805 3600 25205 4200 27005 4800 27605 3600 28205 4200 28805 1800 27005 2400 28805 1800 27005 3000 28205 2400 28205 2400 27005 2400 25805 3600 26405 3600 28805 3600 25805 4200 28805 4200 28205 2400 26405 3600 27005 2400 27605 3600 28805 600 27005 4800 28205 1800 26405 4800 26405 3600 26405 3000 27005 3000 27605 2400 28205 4200 28805 1800 28205 2400 26405 4200 28805 4200 28205 1800 25205 4200 28805 4800 27605 4200 27605 3000 28805 1200 28805 3000 28205 2400 26405 4200 28805 4200 27605 1200 27605 2400 24604 4800 27605 3000 28205 3000 24604 5401 27605 4200 26405 3600 25805 4200 25805 4200 22804 6001 27605 3000 28205 2400 27005 4800 27605 1200 25205 4200 27605 2400 28205 3600 26405 2400 27005 3600 27005 1800 27605 1800 27005 2400 23404 5401 27605 3600 27605 3600 27605 4200 26405 3000 27605 1200 28205 3000 25205 4800 27605 3600 28205 3600 26405 2400 22204 6601 27005 2400 28205 4800 25805 3000 25805 3600 28205 4200 27605 3000 25205 4800 26405 2400 27005 1200 27605 2400 26405 2400 27605 3000 25205 3000 27005 1200 26405 3000 25805 4800 27605 1800 26405 4200 26405 1800 27005 2400 24604 4200 25205 3600 25205 4200 25805 4200 25805 3600 25805 4200 27005 3000 22204 6001 27605 3000 25805 1800 27005 1200 25205 4200 27005 4200 27005 3000 27005 1800 25805 3000 25805 2400 25805 1800 25805 3600 25805 1800 25205 4200 24004 3600 23404 6601 26405 1800 25205 3600 26405 3600 22804 4200 24604 2400 22204 4200 24604 1800 24004 2400 26405 600 26405 600 25805 1200 26405 600 + │ │ │ │ │ │ │ │ histogram(21)= 0 1200 29405 1200 29405 1200 29405 1200 27605 2400 24004 6001 27605 5401 28205 4200 29405 2400 29405 1200 28205 4200 27005 3000 27005 3000 29405 4800 27605 3000 28205 1800 27605 2400 26405 3600 27605 4800 27605 3600 28805 3000 28205 6001 28205 1800 26405 3600 27005 4200 29405 4200 27605 2400 27605 1800 27005 3600 24604 4800 27605 3000 27605 1800 28205 2400 28205 3000 28805 1800 28805 600 28205 1200 27005 2400 27605 1800 26405 3000 26405 4800 28205 2400 28805 1200 28805 3600 28205 3000 28205 1200 28205 1200 25805 4200 28205 4800 26405 6001 28205 1800 27005 3000 28205 1800 27605 3000 28205 1200 28205 2400 27605 3000 28805 1800 26405 4200 28805 1800 24604 4800 28805 1200 27005 3000 26405 7201 25805 4800 28805 4200 28805 3600 25205 4200 27005 4800 27605 3600 28205 4200 28805 1800 27005 2400 28805 1800 27005 3000 28205 2400 28205 2400 27005 2400 25805 3600 26405 3600 28805 3600 25805 4200 28805 4200 28205 2400 26405 3600 27005 2400 27605 3600 28805 600 27005 4800 28205 1800 26405 4800 26405 3600 26405 3000 27005 3000 27605 2400 28205 4200 28805 1800 28205 2400 26405 4200 28805 4200 28205 1800 25205 4200 28805 4800 27605 4200 27605 3000 28805 1200 28805 3000 28205 2400 26405 4200 28805 4200 27605 1200 27605 2400 24604 4800 27605 3000 28205 3000 24604 5401 27605 4200 26405 3600 25805 4200 25805 4200 22804 6001 27605 3000 28205 2400 27005 4800 27605 1200 25205 4200 27605 2400 28205 3600 26405 2400 27005 3600 27005 1800 27605 1800 27005 2400 23404 5401 27605 3600 27605 3600 27605 4200 26405 3000 27605 1200 28205 3000 25205 4800 27605 3600 28205 3600 26405 2400 22204 6601 27005 2400 28205 4800 25805 3000 25805 3600 28205 4200 27605 3000 25205 4800 26405 2400 27005 1200 27605 2400 26405 2400 27605 3000 25205 3000 27005 1200 26405 3000 25805 4800 27605 1800 26405 4200 26405 1800 27005 2400 24604 4200 25205 3600 25205 4200 25805 4200 25805 3600 25805 4200 27005 3000 22204 6001 27605 3000 25805 1800 27005 1200 25205 4200 27005 4200 27005 3000 27005 1800 25805 3000 25805 2400 25805 1800 25805 3600 25805 1800 25205 4200 24004 3600 23404 6601 26405 1800 25205 3600 26405 3600 22804 4200 24604 2400 22204 4200 24604 1800 24004 2400 26405 600 26405 600 25805 1200 26405 600 │ │ │ │ │ │ │ │ <--- '1992-01-19' ------- '1992-03-11' ------- '1992-04-04' ------- '1992-04-23' ------- '1992-05-09' ------- '1992-05-21' ------- '1992-06-06' ------- '1992-06-17' ------- '1992-07-02' ------- '1992-07-17' ------- '1992-07-28' ------- '1992-08-09' ------- '1992-08-21' ------- '1992-09-01' ------- '1992-09-13' ------- '1992-09-23' ------- '1992-10-08' ------- '1992-10-25' ------- '1992-11-06' ------- '1992-11-18' ------- '1992-12-01' ------- '1992-12-12' ------- '1992-12-24' ------- '1993-01-04' ------- '1993-01-15' ------- '1993-01-27' ------- '1993-02-09' ------- '1993-02-20' ------- '1993-03-05' ------- '1993-03-18' ------- '1993-04-01' ------- '1993-04-13' ------- '1993-04-29' ------- '1993-05-13' ------- '1993-05-27' ------- '1993-06-08' ------- '1993-06-20' ------- '1993-07-03' ------- '1993-07-15' ------- '1993-07-29' ------- '1993-08-11' ------- '1993-08-25' ------- '1993-09-08' ------- '1993-09-21' ------- '1993-10-02' ------- '1993-10-15' ------- '1993-10-29' ------- '1993-11-08' ------- '1993-11-20' ------- '1993-12-05' ------- '1993-12-17' ------- '1993-12-30' ------- '1994-01-13' ------- '1994-01-26' ------- '1994-02-08' ------- '1994-02-19' ------- '1994-03-04' ------- '1994-03-14' ------- '1994-03-26' ------- '1994-04-08' ------- '1994-04-21' ------- '1994-05-03' ------- '1994-05-14' ------- '1994-05-24' ------- '1994-06-03' ------- '1994-06-15' ------- '1994-06-27' ------- '1994-07-07' ------- '1994-07-16' ------- '1994-07-26' ------- '1994-08-10' ------- '1994-08-22' ------- '1994-09-03' ------- '1994-09-15' ------- '1994-09-27' ------- '1994-10-11' ------- '1994-10-22' ------- '1994-11-02' ------- '1994-11-14' ------- '1994-11-26' ------- '1994-12-10' ------- '1994-12-22' ------- '1995-01-04' ------- '1995-01-19' ------- '1995-01-30' ------- '1995-02-13' ------- '1995-02-23' ------- '1995-03-07' ------- '1995-03-17' ------- '1995-03-28' ------- '1995-04-11' ------- '1995-04-22' ------- '1995-05-03' ------- '1995-05-17' ------- '1995-05-29' ------- '1995-06-11' ------- '1995-06-26' ------- '1995-07-06' ------- '1995-07-18' ------- '1995-08-01' ------- '1995-08-14' ------- '1995-08-27' ------- '1995-09-09' ------- '1995-09-22' ------- '1995-10-03' ------- '1995-10-14' ------- '1995-10-26' ------- '1995-11-07' ------- '1995-11-19' ------- '1995-11-30' ------- '1995-12-12' ------- '1995-12-25' ------- '1996-01-03' ------- '1996-01-12' ------- '1996-01-24' ------- '1996-02-03' ------- '1996-02-13' ------- '1996-02-25' ------- '1996-03-06' ------- '1996-03-15' ------- '1996-03-25' ------- '1996-04-08' ------- '1996-04-21' ------- '1996-05-01' ------- '1996-05-13' ------- '1996-05-24' ------- '1996-06-03' ------- '1996-06-17' ------- '1996-06-29' ------- '1996-07-09' ------- '1996-07-22' ------- '1996-08-04' ------- '1996-08-17' ------- '1996-08-29' ------- '1996-09-09' ------- '1996-09-21' ------- '1996-10-02' ------- '1996-10-13' ------- '1996-10-27' ------- '1996-11-10' ------- '1996-11-23' ------- '1996-12-03' ------- '1996-12-17' ------- '1996-12-28' ------- '1997-01-08' ------- '1997-01-18' ------- '1997-01-30' ------- '1997-02-11' ------- '1997-02-23' ------- '1997-03-10' ------- '1997-03-20' ------- '1997-03-31' ------- '1997-04-10' ------- '1997-04-23' ------- '1997-05-06' ------- '1997-05-18' ------- '1997-05-30' ------- '1997-06-12' ------- '1997-06-24' ------- '1997-07-07' ------- '1997-07-20' ------- '1997-08-03' ------- '1997-08-16' ------- '1997-08-27' ------- '1997-09-08' ------- '1997-09-18' ------- '1997-10-01' ------- '1997-10-10' ------- '1997-10-20' ------- '1997-11-01' ------- '1997-11-13' ------- '1997-11-27' ------- '1997-12-06' ------- '1997-12-19' ------- '1997-12-29' ------- '1998-01-10' ------- '1998-01-20' ------- '1998-02-03' ------- '1998-02-17' ------- '1998-02-28' ------- '1998-03-11' ------- '1998-03-24' ------- '1998-04-05' ------- '1998-04-17' ------- '1998-05-01' ------- '1998-05-11' ------- '1998-05-22' ------- '1998-06-03' ------- '1998-06-13' ------- '1998-06-27' ------- '1998-07-12' ------- '1998-07-21' ------- '1998-07-31' ------- '1998-08-08' ------- '1998-08-22' ------- '1998-09-03' ------- '1998-09-16' ------- '1998-10-03' ------- '1998-10-26' ------- '1998-12-16' - │ │ │ │ │ │ │ └── ordering: +8 + │ │ │ │ │ │ │ └── ordering: +9 │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── l1.l_receiptdate:20 > l1.l_commitdate:19 [type=bool, outer=(19,20), constraints=(/19: (/NULL - ]; /20: (/NULL - ])] + │ │ │ │ │ │ └── l1.l_receiptdate:21 > l1.l_commitdate:20 [type=bool, outer=(20,21), constraints=(/20: (/NULL - ]; /21: (/NULL - ])] │ │ │ │ │ ├── select │ │ │ │ │ │ ├── save-table-name: q21_select_10 - │ │ │ │ │ │ ├── columns: l3.l_orderkey:53(int!null) l3.l_suppkey:55(int!null) l3.l_commitdate:64(date!null) l3.l_receiptdate:65(date!null) - │ │ │ │ │ │ ├── stats: [rows=2000405, distinct(53)=1216823.04, null(53)=0, distinct(55)=9920, null(55)=0, distinct(64)=2466, null(64)=0, distinct(65)=2554, null(65)=0] - │ │ │ │ │ │ ├── ordering: +53 + │ │ │ │ │ │ ├── columns: l3.l_orderkey:58(int!null) l3.l_suppkey:60(int!null) l3.l_commitdate:69(date!null) l3.l_receiptdate:70(date!null) + │ │ │ │ │ │ ├── stats: [rows=2000405, distinct(58)=1216823.04, null(58)=0, distinct(60)=9920, null(60)=0, distinct(69)=2466, null(69)=0, distinct(70)=2554, null(70)=0] + │ │ │ │ │ │ ├── ordering: +58 │ │ │ │ │ │ ├── scan l3 │ │ │ │ │ │ │ ├── save-table-name: q21_scan_11 - │ │ │ │ │ │ │ ├── columns: l3.l_orderkey:53(int!null) l3.l_suppkey:55(int!null) l3.l_commitdate:64(date!null) l3.l_receiptdate:65(date!null) - │ │ │ │ │ │ │ ├── stats: [rows=6001215, distinct(53)=1527270, null(53)=0, distinct(55)=9920, null(55)=0, distinct(64)=2466, null(64)=0, distinct(65)=2554, null(65)=0] - │ │ │ │ │ │ │ │ histogram(53)= 0 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 + │ │ │ │ │ │ │ ├── columns: l3.l_orderkey:58(int!null) l3.l_suppkey:60(int!null) l3.l_commitdate:69(date!null) l3.l_receiptdate:70(date!null) + │ │ │ │ │ │ │ ├── stats: [rows=6001215, distinct(58)=1527270, null(58)=0, distinct(60)=9920, null(60)=0, distinct(69)=2466, null(69)=0, distinct(70)=2554, null(70)=0] + │ │ │ │ │ │ │ │ histogram(58)= 0 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 │ │ │ │ │ │ │ │ <--- 326 ------- 28929 ------- 50503 ------- 89793 ------- 115938 ------- 146944 ------- 176768 ------- 211201 ------- 237860 ------- 266885 ------- 297604 ------- 330021 ------- 365889 ------- 398951 ------- 426117 ------- 451328 ------- 472134 ------- 499590 ------- 529284 ------- 557254 ------- 589154 ------- 619394 ------- 642951 ------- 670113 ------- 692931 ------- 721157 ------- 751687 ------- 777766 ------- 804582 ------- 836740 ------- 868868 ------- 898912 ------- 922500 ------- 946403 ------- 984870 ------- 1007936 ------- 1030117 ------- 1062275 ------- 1093572 ------- 1120709 ------- 1150981 ------- 1182786 ------- 1206406 ------- 1234116 ------- 1260961 ------- 1290502 ------- 1329510 ------- 1355426 ------- 1381313 ------- 1409796 ------- 1445254 ------- 1479233 ------- 1504935 ------- 1531079 ------- 1559650 ------- 1583616 ------- 1617504 ------- 1655749 ------- 1685185 ------- 1718183 ------- 1747716 ------- 1772131 ------- 1802372 ------- 1833315 ------- 1862403 ------- 1897894 ------- 1922819 ------- 1954405 ------- 1979329 ------- 2009859 ------- 2041670 ------- 2070851 ------- 2093828 ------- 2127973 ------- 2167777 ------- 2194883 ------- 2227814 ------- 2262437 ------- 2296353 ------- 2321024 ------- 2346051 ------- 2376257 ------- 2404932 ------- 2446273 ------- 2474081 ------- 2504515 ------- 2535302 ------- 2561413 ------- 2592737 ------- 2616801 ------- 2646112 ------- 2676546 ------- 2702116 ------- 2732454 ------- 2765382 ------- 2799495 ------- 2828866 ------- 2868737 ------- 2910625 ------- 2938464 ------- 2963140 ------- 3003302 ------- 3043264 ------- 3069123 ------- 3095909 ------- 3126693 ------- 3160485 ------- 3196039 ------- 3229504 ------- 3259712 ------- 3286439 ------- 3318852 ------- 3346821 ------- 3370119 ------- 3395204 ------- 3425888 ------- 3448611 ------- 3476130 ------- 3502372 ------- 3529474 ------- 3556390 ------- 3583553 ------- 3612550 ------- 3647875 ------- 3679140 ------- 3702661 ------- 3738017 ------- 3778050 ------- 3806114 ------- 3839074 ------- 3872805 ------- 3905697 ------- 3926212 ------- 3959841 ------- 3997281 ------- 4033861 ------- 4063591 ------- 4097831 ------- 4124807 ------- 4158656 ------- 4195748 ------- 4234274 ------- 4269952 ------- 4298949 ------- 4332806 ------- 4364705 ------- 4398246 ------- 4430695 ------- 4466403 ------- 4494662 ------- 4524420 ------- 4558561 ------- 4601092 ------- 4632871 ------- 4658694 ------- 4690501 ------- 4728066 ------- 4758657 ------- 4788294 ------- 4818597 ------- 4855874 ------- 4890913 ------- 4915366 ------- 4940709 ------- 4972357 ------- 4995298 ------- 5019523 ------- 5043329 ------- 5077376 ------- 5109920 ------- 5136582 ------- 5161152 ------- 5191846 ------- 5219973 ------- 5251015 ------- 5282021 ------- 5312355 ------- 5343207 ------- 5381318 ------- 5416163 ------- 5445382 ------- 5476933 ------- 5509185 ------- 5539237 ------- 5566818 ------- 5588739 ------- 5620481 ------- 5644001 ------- 5667010 ------- 5689476 ------- 5724709 ------- 5755398 ------- 5790598 ------- 5819425 ------- 5846341 ------- 5874656 ------- 5908067 ------- 5933572 ------- 5962659 ------- 5999971 - │ │ │ │ │ │ │ │ histogram(55)= 0 600 28805 3000 29405 600 29405 600 29405 600 29405 600 29405 1200 29405 1200 29405 1200 28805 1200 28805 1800 29405 600 29405 1200 29405 600 29405 600 28805 1200 29405 1200 28805 1200 28205 1800 29405 1200 29405 600 29405 1200 29405 1200 29405 600 29405 1200 29405 1200 28805 1800 29405 600 28805 1800 28805 1200 28805 1200 29405 1200 28805 1800 29405 600 28205 1800 29405 600 29405 600 28805 1200 29405 600 28805 1200 29405 600 29405 600 28805 1200 28805 1800 27605 2400 29405 600 28205 1800 29405 2400 29405 600 28805 1800 28205 1800 28805 1200 29405 3000 29405 1200 29405 1800 29405 600 29405 600 28205 1800 29405 600 29405 1800 29405 1800 29405 600 29405 600 28805 1200 28805 1200 29405 1200 28205 1800 29405 600 28805 1200 27605 2400 29405 600 29405 1200 29405 1200 29405 600 29405 1800 29405 1200 28205 1800 29405 600 29405 1200 29405 1200 28805 1200 28205 1800 29405 1800 28805 1200 29405 600 29405 1800 28205 2400 29405 600 28805 1200 28205 2400 28805 1200 27605 2400 28805 1200 29405 600 28805 1800 29405 1800 28805 600 28205 1800 28805 600 29405 600 29405 1200 28805 600 28805 1200 29405 1800 27605 1800 27605 2400 27605 1800 29405 1200 28805 600 28805 1200 29405 1200 28805 2400 28805 1200 28805 600 28205 1200 28205 1200 29405 600 28805 1200 29405 600 28205 2400 28205 1200 29405 600 29405 1800 28805 1200 28205 1200 27605 2400 27605 1800 28805 1200 29405 600 28805 1200 29405 600 29405 600 28805 1200 29405 600 29405 600 29405 600 28805 1200 29405 1200 28805 600 29405 1800 28205 1200 27605 1800 29405 600 29405 600 29405 1200 28805 1200 28805 1200 28205 1200 29405 1800 28205 1800 28805 600 28805 1200 28205 1200 29405 600 28805 1800 28205 1800 27605 2400 28805 600 29405 600 28805 1200 29405 600 28805 2400 28805 1200 28805 1200 28805 600 28205 1200 28805 1200 29405 600 29405 600 29405 1800 28805 600 28805 600 28805 2400 28805 600 28805 1800 28805 1200 28805 600 27005 2400 29405 600 28205 1800 29405 1200 27605 1800 28805 1200 28805 1200 29405 600 29405 600 29405 600 29405 1200 28805 600 29405 600 27605 2400 28805 1200 29405 600 28805 1800 28805 1800 28205 1200 28805 1200 28805 600 29405 1200 28205 1200 + │ │ │ │ │ │ │ │ histogram(60)= 0 600 28805 3000 29405 600 29405 600 29405 600 29405 600 29405 1200 29405 1200 29405 1200 28805 1200 28805 1800 29405 600 29405 1200 29405 600 29405 600 28805 1200 29405 1200 28805 1200 28205 1800 29405 1200 29405 600 29405 1200 29405 1200 29405 600 29405 1200 29405 1200 28805 1800 29405 600 28805 1800 28805 1200 28805 1200 29405 1200 28805 1800 29405 600 28205 1800 29405 600 29405 600 28805 1200 29405 600 28805 1200 29405 600 29405 600 28805 1200 28805 1800 27605 2400 29405 600 28205 1800 29405 2400 29405 600 28805 1800 28205 1800 28805 1200 29405 3000 29405 1200 29405 1800 29405 600 29405 600 28205 1800 29405 600 29405 1800 29405 1800 29405 600 29405 600 28805 1200 28805 1200 29405 1200 28205 1800 29405 600 28805 1200 27605 2400 29405 600 29405 1200 29405 1200 29405 600 29405 1800 29405 1200 28205 1800 29405 600 29405 1200 29405 1200 28805 1200 28205 1800 29405 1800 28805 1200 29405 600 29405 1800 28205 2400 29405 600 28805 1200 28205 2400 28805 1200 27605 2400 28805 1200 29405 600 28805 1800 29405 1800 28805 600 28205 1800 28805 600 29405 600 29405 1200 28805 600 28805 1200 29405 1800 27605 1800 27605 2400 27605 1800 29405 1200 28805 600 28805 1200 29405 1200 28805 2400 28805 1200 28805 600 28205 1200 28205 1200 29405 600 28805 1200 29405 600 28205 2400 28205 1200 29405 600 29405 1800 28805 1200 28205 1200 27605 2400 27605 1800 28805 1200 29405 600 28805 1200 29405 600 29405 600 28805 1200 29405 600 29405 600 29405 600 28805 1200 29405 1200 28805 600 29405 1800 28205 1200 27605 1800 29405 600 29405 600 29405 1200 28805 1200 28805 1200 28205 1200 29405 1800 28205 1800 28805 600 28805 1200 28205 1200 29405 600 28805 1800 28205 1800 27605 2400 28805 600 29405 600 28805 1200 29405 600 28805 2400 28805 1200 28805 1200 28805 600 28205 1200 28805 1200 29405 600 29405 600 29405 1800 28805 600 28805 600 28805 2400 28805 600 28805 1800 28805 1200 28805 600 27005 2400 29405 600 28205 1800 29405 1200 27605 1800 28805 1200 28805 1200 29405 600 29405 600 29405 600 29405 1200 28805 600 29405 600 27605 2400 28805 1200 29405 600 28805 1800 28805 1800 28205 1200 28805 1200 28805 600 29405 1200 28205 1200 │ │ │ │ │ │ │ │ <--- 6 ------- 59 ------- 106 ------- 147 ------- 189 ------- 219 ------- 272 ------- 335 ------- 387 ------- 433 ------- 485 ------- 539 ------- 600 ------- 653 ------- 698 ------- 749 ------- 806 ------- 855 ------- 904 ------- 961 ------- 1008 ------- 1065 ------- 1115 ------- 1163 ------- 1212 ------- 1270 ------- 1329 ------- 1373 ------- 1417 ------- 1470 ------- 1514 ------- 1566 ------- 1616 ------- 1668 ------- 1719 ------- 1765 ------- 1819 ------- 1871 ------- 1926 ------- 1978 ------- 2030 ------- 2080 ------- 2128 ------- 2173 ------- 2223 ------- 2270 ------- 2318 ------- 2382 ------- 2444 ------- 2491 ------- 2534 ------- 2584 ------- 2632 ------- 2675 ------- 2736 ------- 2781 ------- 2841 ------- 2905 ------- 2940 ------- 2996 ------- 3045 ------- 3084 ------- 3124 ------- 3181 ------- 3224 ------- 3268 ------- 3318 ------- 3362 ------- 3419 ------- 3466 ------- 3515 ------- 3560 ------- 3615 ------- 3669 ------- 3725 ------- 3777 ------- 3825 ------- 3871 ------- 3923 ------- 3980 ------- 4035 ------- 4078 ------- 4135 ------- 4184 ------- 4244 ------- 4296 ------- 4346 ------- 4396 ------- 4444 ------- 4489 ------- 4540 ------- 4575 ------- 4627 ------- 4683 ------- 4731 ------- 4777 ------- 4831 ------- 4900 ------- 4953 ------- 5006 ------- 5061 ------- 5119 ------- 5161 ------- 5208 ------- 5260 ------- 5315 ------- 5361 ------- 5415 ------- 5474 ------- 5529 ------- 5572 ------- 5635 ------- 5689 ------- 5738 ------- 5792 ------- 5835 ------- 5890 ------- 5935 ------- 5997 ------- 6039 ------- 6094 ------- 6151 ------- 6201 ------- 6245 ------- 6291 ------- 6343 ------- 6387 ------- 6437 ------- 6480 ------- 6540 ------- 6580 ------- 6629 ------- 6673 ------- 6718 ------- 6771 ------- 6823 ------- 6878 ------- 6935 ------- 6974 ------- 7017 ------- 7060 ------- 7112 ------- 7170 ------- 7231 ------- 7279 ------- 7329 ------- 7373 ------- 7427 ------- 7477 ------- 7520 ------- 7564 ------- 7616 ------- 7671 ------- 7716 ------- 7760 ------- 7815 ------- 7861 ------- 7917 ------- 7969 ------- 8018 ------- 8063 ------- 8112 ------- 8180 ------- 8229 ------- 8268 ------- 8320 ------- 8376 ------- 8432 ------- 8474 ------- 8522 ------- 8569 ------- 8616 ------- 8664 ------- 8705 ------- 8765 ------- 8814 ------- 8868 ------- 8906 ------- 8948 ------- 8990 ------- 9035 ------- 9090 ------- 9137 ------- 9188 ------- 9241 ------- 9292 ------- 9354 ------- 9411 ------- 9472 ------- 9526 ------- 9569 ------- 9613 ------- 9658 ------- 9708 ------- 9737 ------- 9792 ------- 9838 ------- 9903 ------- 9952 ------- 10000 - │ │ │ │ │ │ │ │ histogram(64)= 0 600 28805 1200 29405 3600 29405 1200 29405 1800 28205 1800 28805 4200 28205 3600 27005 4800 29405 3600 29405 1800 26405 3600 29405 600 27005 3000 29405 2400 28805 3000 29405 600 25805 4800 28805 3000 27605 2400 27605 5401 28805 3600 28805 2400 27605 3600 28805 6001 28805 3600 27005 3000 25205 4800 28805 4200 28205 1200 28805 3600 28205 4200 27605 3000 28205 3600 28805 1200 27605 3000 28205 1200 27005 3000 28805 1200 28205 1800 28205 1800 27605 1800 27605 3600 27605 3600 25805 5401 28805 4200 28205 1200 28805 3000 28805 4800 28805 1200 28805 4200 28205 2400 26405 3000 28205 3000 28805 2400 28805 2400 27605 5401 28805 1800 28805 2400 28805 1200 28805 3600 27605 5401 25805 5401 28205 3600 27005 3600 28205 3000 28805 1200 28805 4800 28205 2400 24604 5401 28805 3000 27605 2400 28205 4200 27005 3000 28805 3000 28805 600 26405 5401 27605 4800 27005 2400 26405 4200 28805 2400 27605 3000 28205 3000 27005 2400 26405 3600 28205 1200 27605 4200 27605 3000 28205 1800 25805 3000 28205 2400 28205 1200 27005 2400 27605 1800 25805 3600 27005 6601 27005 1800 24004 6001 27605 1800 28205 3000 27605 2400 27605 2400 27605 3000 27605 2400 28205 3000 26405 2400 26405 2400 26405 2400 26405 3600 28205 1800 28205 2400 27605 1200 27605 3000 27605 4800 24604 4200 28205 3000 24604 6001 25805 4200 25205 4200 26405 3000 28205 3000 27605 3000 28205 600 27605 3600 25805 4200 27005 3600 28205 600 28205 3000 28205 1200 25805 3000 28205 1200 27005 3000 28205 1200 24604 4200 27605 1800 28205 4200 27605 2400 26405 4200 27005 2400 27005 1800 27605 3600 25205 4800 27605 2400 28205 1800 27005 3000 27605 3000 25205 4200 25805 3000 27605 3600 25805 3600 27605 1800 24604 4200 25805 3600 26405 1800 27005 3000 27005 1200 27005 3000 24004 5401 26405 1800 25805 2400 27605 1200 27605 600 27005 2400 27605 1200 23404 7201 25205 3600 26405 3000 27605 3600 27605 2400 24604 3600 27605 4200 27605 3600 27005 1200 26405 1200 25805 1800 25205 3600 22804 4800 24604 3000 24004 3600 26405 2400 24604 4800 26405 3000 22204 5401 25805 2400 25205 2400 27005 2400 24604 4200 25205 3000 24004 4800 25205 3000 26405 1200 27005 2400 26405 2400 26405 4800 25205 3000 25805 1800 24004 4800 24604 3000 23404 600 24004 600 + │ │ │ │ │ │ │ │ histogram(69)= 0 600 28805 1200 29405 3600 29405 1200 29405 1800 28205 1800 28805 4200 28205 3600 27005 4800 29405 3600 29405 1800 26405 3600 29405 600 27005 3000 29405 2400 28805 3000 29405 600 25805 4800 28805 3000 27605 2400 27605 5401 28805 3600 28805 2400 27605 3600 28805 6001 28805 3600 27005 3000 25205 4800 28805 4200 28205 1200 28805 3600 28205 4200 27605 3000 28205 3600 28805 1200 27605 3000 28205 1200 27005 3000 28805 1200 28205 1800 28205 1800 27605 1800 27605 3600 27605 3600 25805 5401 28805 4200 28205 1200 28805 3000 28805 4800 28805 1200 28805 4200 28205 2400 26405 3000 28205 3000 28805 2400 28805 2400 27605 5401 28805 1800 28805 2400 28805 1200 28805 3600 27605 5401 25805 5401 28205 3600 27005 3600 28205 3000 28805 1200 28805 4800 28205 2400 24604 5401 28805 3000 27605 2400 28205 4200 27005 3000 28805 3000 28805 600 26405 5401 27605 4800 27005 2400 26405 4200 28805 2400 27605 3000 28205 3000 27005 2400 26405 3600 28205 1200 27605 4200 27605 3000 28205 1800 25805 3000 28205 2400 28205 1200 27005 2400 27605 1800 25805 3600 27005 6601 27005 1800 24004 6001 27605 1800 28205 3000 27605 2400 27605 2400 27605 3000 27605 2400 28205 3000 26405 2400 26405 2400 26405 2400 26405 3600 28205 1800 28205 2400 27605 1200 27605 3000 27605 4800 24604 4200 28205 3000 24604 6001 25805 4200 25205 4200 26405 3000 28205 3000 27605 3000 28205 600 27605 3600 25805 4200 27005 3600 28205 600 28205 3000 28205 1200 25805 3000 28205 1200 27005 3000 28205 1200 24604 4200 27605 1800 28205 4200 27605 2400 26405 4200 27005 2400 27005 1800 27605 3600 25205 4800 27605 2400 28205 1800 27005 3000 27605 3000 25205 4200 25805 3000 27605 3600 25805 3600 27605 1800 24604 4200 25805 3600 26405 1800 27005 3000 27005 1200 27005 3000 24004 5401 26405 1800 25805 2400 27605 1200 27605 600 27005 2400 27605 1200 23404 7201 25205 3600 26405 3000 27605 3600 27605 2400 24604 3600 27605 4200 27605 3600 27005 1200 26405 1200 25805 1800 25205 3600 22804 4800 24604 3000 24004 3600 26405 2400 24604 4800 26405 3000 22204 5401 25805 2400 25205 2400 27005 2400 24604 4200 25205 3000 24004 4800 25205 3000 26405 1200 27005 2400 26405 2400 26405 4800 25205 3000 25805 1800 24004 4800 24604 3000 23404 600 24004 600 │ │ │ │ │ │ │ │ <--- '1992-02-06' ------- '1992-03-13' ------- '1992-03-29' ------- '1992-04-12' ------- '1992-04-24' ------- '1992-05-07' ------- '1992-05-18' ------- '1992-05-30' ------- '1992-06-15' ------- '1992-06-28' ------- '1992-07-10' ------- '1992-07-20' ------- '1992-08-04' ------- '1992-08-17' ------- '1992-08-30' ------- '1992-09-11' ------- '1992-09-25' ------- '1992-10-05' ------- '1992-10-16' ------- '1992-10-29' ------- '1992-11-10' ------- '1992-11-24' ------- '1992-12-05' ------- '1992-12-16' ------- '1992-12-31' ------- '1993-01-15' ------- '1993-01-28' ------- '1993-02-09' ------- '1993-02-21' ------- '1993-03-07' ------- '1993-03-17' ------- '1993-04-03' ------- '1993-04-17' ------- '1993-04-30' ------- '1993-05-12' ------- '1993-05-25' ------- '1993-06-08' ------- '1993-06-23' ------- '1993-07-04' ------- '1993-07-18' ------- '1993-08-02' ------- '1993-08-13' ------- '1993-08-24' ------- '1993-09-08' ------- '1993-09-20' ------- '1993-10-06' ------- '1993-10-22' ------- '1993-11-01' ------- '1993-11-14' ------- '1993-11-24' ------- '1993-12-07' ------- '1993-12-20' ------- '1994-01-01' ------- '1994-01-14' ------- '1994-01-31' ------- '1994-02-14' ------- '1994-02-26' ------- '1994-03-09' ------- '1994-03-22' ------- '1994-04-03' ------- '1994-04-14' ------- '1994-04-23' ------- '1994-05-07' ------- '1994-05-19' ------- '1994-05-31' ------- '1994-06-12' ------- '1994-06-22' ------- '1994-07-03' ------- '1994-07-14' ------- '1994-07-24' ------- '1994-08-04' ------- '1994-08-16' ------- '1994-08-29' ------- '1994-09-08' ------- '1994-09-23' ------- '1994-10-05' ------- '1994-10-16' ------- '1994-10-28' ------- '1994-11-08' ------- '1994-11-19' ------- '1994-12-05' ------- '1994-12-18' ------- '1994-12-29' ------- '1995-01-11' ------- '1995-01-23' ------- '1995-02-05' ------- '1995-02-15' ------- '1995-03-02' ------- '1995-03-14' ------- '1995-03-25' ------- '1995-04-05' ------- '1995-04-17' ------- '1995-04-30' ------- '1995-05-11' ------- '1995-05-23' ------- '1995-06-06' ------- '1995-06-20' ------- '1995-06-30' ------- '1995-07-12' ------- '1995-07-26' ------- '1995-08-07' ------- '1995-08-17' ------- '1995-08-28' ------- '1995-09-09' ------- '1995-09-21' ------- '1995-10-04' ------- '1995-10-14' ------- '1995-10-28' ------- '1995-11-07' ------- '1995-11-18' ------- '1995-11-29' ------- '1995-12-11' ------- '1995-12-21' ------- '1995-12-31' ------- '1996-01-11' ------- '1996-01-21' ------- '1996-01-31' ------- '1996-02-10' ------- '1996-02-24' ------- '1996-03-04' ------- '1996-03-13' ------- '1996-03-25' ------- '1996-04-07' ------- '1996-04-17' ------- '1996-04-28' ------- '1996-05-10' ------- '1996-05-22' ------- '1996-06-02' ------- '1996-06-15' ------- '1996-06-26' ------- '1996-07-09' ------- '1996-07-20' ------- '1996-07-31' ------- '1996-08-11' ------- '1996-08-25' ------- '1996-09-07' ------- '1996-09-21' ------- '1996-10-04' ------- '1996-10-15' ------- '1996-10-27' ------- '1996-11-07' ------- '1996-11-19' ------- '1996-11-30' ------- '1996-12-13' ------- '1996-12-27' ------- '1997-01-06' ------- '1997-01-18' ------- '1997-01-30' ------- '1997-02-12' ------- '1997-02-23' ------- '1997-03-07' ------- '1997-03-19' ------- '1997-03-31' ------- '1997-04-10' ------- '1997-04-23' ------- '1997-05-04' ------- '1997-05-17' ------- '1997-05-29' ------- '1997-06-12' ------- '1997-06-23' ------- '1997-07-04' ------- '1997-07-16' ------- '1997-07-28' ------- '1997-08-07' ------- '1997-08-19' ------- '1997-08-29' ------- '1997-09-10' ------- '1997-09-20' ------- '1997-10-03' ------- '1997-10-13' ------- '1997-10-25' ------- '1997-11-05' ------- '1997-11-17' ------- '1997-12-02' ------- '1997-12-15' ------- '1997-12-26' ------- '1998-01-07' ------- '1998-01-16' ------- '1998-01-25' ------- '1998-02-07' ------- '1998-02-20' ------- '1998-03-06' ------- '1998-03-14' ------- '1998-03-30' ------- '1998-04-08' ------- '1998-04-21' ------- '1998-05-03' ------- '1998-05-15' ------- '1998-05-27' ------- '1998-06-07' ------- '1998-06-19' ------- '1998-07-03' ------- '1998-07-15' ------- '1998-07-26' ------- '1998-08-07' ------- '1998-08-18' ------- '1998-08-30' ------- '1998-09-12' ------- '1998-10-01' ------- '1998-10-30' - │ │ │ │ │ │ │ │ histogram(65)= 0 1200 29405 1200 29405 1200 29405 1200 27605 2400 24004 6001 27605 5401 28205 4200 29405 2400 29405 1200 28205 4200 27005 3000 27005 3000 29405 4800 27605 3000 28205 1800 27605 2400 26405 3600 27605 4800 27605 3600 28805 3000 28205 6001 28205 1800 26405 3600 27005 4200 29405 4200 27605 2400 27605 1800 27005 3600 24604 4800 27605 3000 27605 1800 28205 2400 28205 3000 28805 1800 28805 600 28205 1200 27005 2400 27605 1800 26405 3000 26405 4800 28205 2400 28805 1200 28805 3600 28205 3000 28205 1200 28205 1200 25805 4200 28205 4800 26405 6001 28205 1800 27005 3000 28205 1800 27605 3000 28205 1200 28205 2400 27605 3000 28805 1800 26405 4200 28805 1800 24604 4800 28805 1200 27005 3000 26405 7201 25805 4800 28805 4200 28805 3600 25205 4200 27005 4800 27605 3600 28205 4200 28805 1800 27005 2400 28805 1800 27005 3000 28205 2400 28205 2400 27005 2400 25805 3600 26405 3600 28805 3600 25805 4200 28805 4200 28205 2400 26405 3600 27005 2400 27605 3600 28805 600 27005 4800 28205 1800 26405 4800 26405 3600 26405 3000 27005 3000 27605 2400 28205 4200 28805 1800 28205 2400 26405 4200 28805 4200 28205 1800 25205 4200 28805 4800 27605 4200 27605 3000 28805 1200 28805 3000 28205 2400 26405 4200 28805 4200 27605 1200 27605 2400 24604 4800 27605 3000 28205 3000 24604 5401 27605 4200 26405 3600 25805 4200 25805 4200 22804 6001 27605 3000 28205 2400 27005 4800 27605 1200 25205 4200 27605 2400 28205 3600 26405 2400 27005 3600 27005 1800 27605 1800 27005 2400 23404 5401 27605 3600 27605 3600 27605 4200 26405 3000 27605 1200 28205 3000 25205 4800 27605 3600 28205 3600 26405 2400 22204 6601 27005 2400 28205 4800 25805 3000 25805 3600 28205 4200 27605 3000 25205 4800 26405 2400 27005 1200 27605 2400 26405 2400 27605 3000 25205 3000 27005 1200 26405 3000 25805 4800 27605 1800 26405 4200 26405 1800 27005 2400 24604 4200 25205 3600 25205 4200 25805 4200 25805 3600 25805 4200 27005 3000 22204 6001 27605 3000 25805 1800 27005 1200 25205 4200 27005 4200 27005 3000 27005 1800 25805 3000 25805 2400 25805 1800 25805 3600 25805 1800 25205 4200 24004 3600 23404 6601 26405 1800 25205 3600 26405 3600 22804 4200 24604 2400 22204 4200 24604 1800 24004 2400 26405 600 26405 600 25805 1200 26405 600 + │ │ │ │ │ │ │ │ histogram(70)= 0 1200 29405 1200 29405 1200 29405 1200 27605 2400 24004 6001 27605 5401 28205 4200 29405 2400 29405 1200 28205 4200 27005 3000 27005 3000 29405 4800 27605 3000 28205 1800 27605 2400 26405 3600 27605 4800 27605 3600 28805 3000 28205 6001 28205 1800 26405 3600 27005 4200 29405 4200 27605 2400 27605 1800 27005 3600 24604 4800 27605 3000 27605 1800 28205 2400 28205 3000 28805 1800 28805 600 28205 1200 27005 2400 27605 1800 26405 3000 26405 4800 28205 2400 28805 1200 28805 3600 28205 3000 28205 1200 28205 1200 25805 4200 28205 4800 26405 6001 28205 1800 27005 3000 28205 1800 27605 3000 28205 1200 28205 2400 27605 3000 28805 1800 26405 4200 28805 1800 24604 4800 28805 1200 27005 3000 26405 7201 25805 4800 28805 4200 28805 3600 25205 4200 27005 4800 27605 3600 28205 4200 28805 1800 27005 2400 28805 1800 27005 3000 28205 2400 28205 2400 27005 2400 25805 3600 26405 3600 28805 3600 25805 4200 28805 4200 28205 2400 26405 3600 27005 2400 27605 3600 28805 600 27005 4800 28205 1800 26405 4800 26405 3600 26405 3000 27005 3000 27605 2400 28205 4200 28805 1800 28205 2400 26405 4200 28805 4200 28205 1800 25205 4200 28805 4800 27605 4200 27605 3000 28805 1200 28805 3000 28205 2400 26405 4200 28805 4200 27605 1200 27605 2400 24604 4800 27605 3000 28205 3000 24604 5401 27605 4200 26405 3600 25805 4200 25805 4200 22804 6001 27605 3000 28205 2400 27005 4800 27605 1200 25205 4200 27605 2400 28205 3600 26405 2400 27005 3600 27005 1800 27605 1800 27005 2400 23404 5401 27605 3600 27605 3600 27605 4200 26405 3000 27605 1200 28205 3000 25205 4800 27605 3600 28205 3600 26405 2400 22204 6601 27005 2400 28205 4800 25805 3000 25805 3600 28205 4200 27605 3000 25205 4800 26405 2400 27005 1200 27605 2400 26405 2400 27605 3000 25205 3000 27005 1200 26405 3000 25805 4800 27605 1800 26405 4200 26405 1800 27005 2400 24604 4200 25205 3600 25205 4200 25805 4200 25805 3600 25805 4200 27005 3000 22204 6001 27605 3000 25805 1800 27005 1200 25205 4200 27005 4200 27005 3000 27005 1800 25805 3000 25805 2400 25805 1800 25805 3600 25805 1800 25205 4200 24004 3600 23404 6601 26405 1800 25205 3600 26405 3600 22804 4200 24604 2400 22204 4200 24604 1800 24004 2400 26405 600 26405 600 25805 1200 26405 600 │ │ │ │ │ │ │ │ <--- '1992-01-19' ------- '1992-03-11' ------- '1992-04-04' ------- '1992-04-23' ------- '1992-05-09' ------- '1992-05-21' ------- '1992-06-06' ------- '1992-06-17' ------- '1992-07-02' ------- '1992-07-17' ------- '1992-07-28' ------- '1992-08-09' ------- '1992-08-21' ------- '1992-09-01' ------- '1992-09-13' ------- '1992-09-23' ------- '1992-10-08' ------- '1992-10-25' ------- '1992-11-06' ------- '1992-11-18' ------- '1992-12-01' ------- '1992-12-12' ------- '1992-12-24' ------- '1993-01-04' ------- '1993-01-15' ------- '1993-01-27' ------- '1993-02-09' ------- '1993-02-20' ------- '1993-03-05' ------- '1993-03-18' ------- '1993-04-01' ------- '1993-04-13' ------- '1993-04-29' ------- '1993-05-13' ------- '1993-05-27' ------- '1993-06-08' ------- '1993-06-20' ------- '1993-07-03' ------- '1993-07-15' ------- '1993-07-29' ------- '1993-08-11' ------- '1993-08-25' ------- '1993-09-08' ------- '1993-09-21' ------- '1993-10-02' ------- '1993-10-15' ------- '1993-10-29' ------- '1993-11-08' ------- '1993-11-20' ------- '1993-12-05' ------- '1993-12-17' ------- '1993-12-30' ------- '1994-01-13' ------- '1994-01-26' ------- '1994-02-08' ------- '1994-02-19' ------- '1994-03-04' ------- '1994-03-14' ------- '1994-03-26' ------- '1994-04-08' ------- '1994-04-21' ------- '1994-05-03' ------- '1994-05-14' ------- '1994-05-24' ------- '1994-06-03' ------- '1994-06-15' ------- '1994-06-27' ------- '1994-07-07' ------- '1994-07-16' ------- '1994-07-26' ------- '1994-08-10' ------- '1994-08-22' ------- '1994-09-03' ------- '1994-09-15' ------- '1994-09-27' ------- '1994-10-11' ------- '1994-10-22' ------- '1994-11-02' ------- '1994-11-14' ------- '1994-11-26' ------- '1994-12-10' ------- '1994-12-22' ------- '1995-01-04' ------- '1995-01-19' ------- '1995-01-30' ------- '1995-02-13' ------- '1995-02-23' ------- '1995-03-07' ------- '1995-03-17' ------- '1995-03-28' ------- '1995-04-11' ------- '1995-04-22' ------- '1995-05-03' ------- '1995-05-17' ------- '1995-05-29' ------- '1995-06-11' ------- '1995-06-26' ------- '1995-07-06' ------- '1995-07-18' ------- '1995-08-01' ------- '1995-08-14' ------- '1995-08-27' ------- '1995-09-09' ------- '1995-09-22' ------- '1995-10-03' ------- '1995-10-14' ------- '1995-10-26' ------- '1995-11-07' ------- '1995-11-19' ------- '1995-11-30' ------- '1995-12-12' ------- '1995-12-25' ------- '1996-01-03' ------- '1996-01-12' ------- '1996-01-24' ------- '1996-02-03' ------- '1996-02-13' ------- '1996-02-25' ------- '1996-03-06' ------- '1996-03-15' ------- '1996-03-25' ------- '1996-04-08' ------- '1996-04-21' ------- '1996-05-01' ------- '1996-05-13' ------- '1996-05-24' ------- '1996-06-03' ------- '1996-06-17' ------- '1996-06-29' ------- '1996-07-09' ------- '1996-07-22' ------- '1996-08-04' ------- '1996-08-17' ------- '1996-08-29' ------- '1996-09-09' ------- '1996-09-21' ------- '1996-10-02' ------- '1996-10-13' ------- '1996-10-27' ------- '1996-11-10' ------- '1996-11-23' ------- '1996-12-03' ------- '1996-12-17' ------- '1996-12-28' ------- '1997-01-08' ------- '1997-01-18' ------- '1997-01-30' ------- '1997-02-11' ------- '1997-02-23' ------- '1997-03-10' ------- '1997-03-20' ------- '1997-03-31' ------- '1997-04-10' ------- '1997-04-23' ------- '1997-05-06' ------- '1997-05-18' ------- '1997-05-30' ------- '1997-06-12' ------- '1997-06-24' ------- '1997-07-07' ------- '1997-07-20' ------- '1997-08-03' ------- '1997-08-16' ------- '1997-08-27' ------- '1997-09-08' ------- '1997-09-18' ------- '1997-10-01' ------- '1997-10-10' ------- '1997-10-20' ------- '1997-11-01' ------- '1997-11-13' ------- '1997-11-27' ------- '1997-12-06' ------- '1997-12-19' ------- '1997-12-29' ------- '1998-01-10' ------- '1998-01-20' ------- '1998-02-03' ------- '1998-02-17' ------- '1998-02-28' ------- '1998-03-11' ------- '1998-03-24' ------- '1998-04-05' ------- '1998-04-17' ------- '1998-05-01' ------- '1998-05-11' ------- '1998-05-22' ------- '1998-06-03' ------- '1998-06-13' ------- '1998-06-27' ------- '1998-07-12' ------- '1998-07-21' ------- '1998-07-31' ------- '1998-08-08' ------- '1998-08-22' ------- '1998-09-03' ------- '1998-09-16' ------- '1998-10-03' ------- '1998-10-26' ------- '1998-12-16' - │ │ │ │ │ │ │ └── ordering: +53 + │ │ │ │ │ │ │ └── ordering: +58 │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── l3.l_receiptdate:65 > l3.l_commitdate:64 [type=bool, outer=(64,65), constraints=(/64: (/NULL - ]; /65: (/NULL - ])] + │ │ │ │ │ │ └── l3.l_receiptdate:70 > l3.l_commitdate:69 [type=bool, outer=(69,70), constraints=(/69: (/NULL - ]; /70: (/NULL - ])] │ │ │ │ │ └── filters - │ │ │ │ │ └── l3.l_suppkey:55 != l1.l_suppkey:10 [type=bool, outer=(10,55), constraints=(/10: (/NULL - ]; /55: (/NULL - ])] + │ │ │ │ │ └── l3.l_suppkey:60 != l1.l_suppkey:11 [type=bool, outer=(11,60), constraints=(/11: (/NULL - ]; /60: (/NULL - ])] │ │ │ │ ├── scan l2@l_sk │ │ │ │ │ ├── save-table-name: q21_scan_12 - │ │ │ │ │ ├── columns: l2.l_orderkey:37(int!null) l2.l_suppkey:39(int!null) - │ │ │ │ │ └── stats: [rows=6001215, distinct(37)=1527270, null(37)=0, distinct(39)=9920, null(39)=0] - │ │ │ │ │ histogram(37)= 0 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 + │ │ │ │ │ ├── columns: l2.l_orderkey:41(int!null) l2.l_suppkey:43(int!null) + │ │ │ │ │ └── stats: [rows=6001215, distinct(41)=1527270, null(41)=0, distinct(43)=9920, null(43)=0] + │ │ │ │ │ histogram(41)= 0 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 29405 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 30006 600 │ │ │ │ │ <--- 326 ------- 28929 ------- 50503 ------- 89793 ------- 115938 ------- 146944 ------- 176768 ------- 211201 ------- 237860 ------- 266885 ------- 297604 ------- 330021 ------- 365889 ------- 398951 ------- 426117 ------- 451328 ------- 472134 ------- 499590 ------- 529284 ------- 557254 ------- 589154 ------- 619394 ------- 642951 ------- 670113 ------- 692931 ------- 721157 ------- 751687 ------- 777766 ------- 804582 ------- 836740 ------- 868868 ------- 898912 ------- 922500 ------- 946403 ------- 984870 ------- 1007936 ------- 1030117 ------- 1062275 ------- 1093572 ------- 1120709 ------- 1150981 ------- 1182786 ------- 1206406 ------- 1234116 ------- 1260961 ------- 1290502 ------- 1329510 ------- 1355426 ------- 1381313 ------- 1409796 ------- 1445254 ------- 1479233 ------- 1504935 ------- 1531079 ------- 1559650 ------- 1583616 ------- 1617504 ------- 1655749 ------- 1685185 ------- 1718183 ------- 1747716 ------- 1772131 ------- 1802372 ------- 1833315 ------- 1862403 ------- 1897894 ------- 1922819 ------- 1954405 ------- 1979329 ------- 2009859 ------- 2041670 ------- 2070851 ------- 2093828 ------- 2127973 ------- 2167777 ------- 2194883 ------- 2227814 ------- 2262437 ------- 2296353 ------- 2321024 ------- 2346051 ------- 2376257 ------- 2404932 ------- 2446273 ------- 2474081 ------- 2504515 ------- 2535302 ------- 2561413 ------- 2592737 ------- 2616801 ------- 2646112 ------- 2676546 ------- 2702116 ------- 2732454 ------- 2765382 ------- 2799495 ------- 2828866 ------- 2868737 ------- 2910625 ------- 2938464 ------- 2963140 ------- 3003302 ------- 3043264 ------- 3069123 ------- 3095909 ------- 3126693 ------- 3160485 ------- 3196039 ------- 3229504 ------- 3259712 ------- 3286439 ------- 3318852 ------- 3346821 ------- 3370119 ------- 3395204 ------- 3425888 ------- 3448611 ------- 3476130 ------- 3502372 ------- 3529474 ------- 3556390 ------- 3583553 ------- 3612550 ------- 3647875 ------- 3679140 ------- 3702661 ------- 3738017 ------- 3778050 ------- 3806114 ------- 3839074 ------- 3872805 ------- 3905697 ------- 3926212 ------- 3959841 ------- 3997281 ------- 4033861 ------- 4063591 ------- 4097831 ------- 4124807 ------- 4158656 ------- 4195748 ------- 4234274 ------- 4269952 ------- 4298949 ------- 4332806 ------- 4364705 ------- 4398246 ------- 4430695 ------- 4466403 ------- 4494662 ------- 4524420 ------- 4558561 ------- 4601092 ------- 4632871 ------- 4658694 ------- 4690501 ------- 4728066 ------- 4758657 ------- 4788294 ------- 4818597 ------- 4855874 ------- 4890913 ------- 4915366 ------- 4940709 ------- 4972357 ------- 4995298 ------- 5019523 ------- 5043329 ------- 5077376 ------- 5109920 ------- 5136582 ------- 5161152 ------- 5191846 ------- 5219973 ------- 5251015 ------- 5282021 ------- 5312355 ------- 5343207 ------- 5381318 ------- 5416163 ------- 5445382 ------- 5476933 ------- 5509185 ------- 5539237 ------- 5566818 ------- 5588739 ------- 5620481 ------- 5644001 ------- 5667010 ------- 5689476 ------- 5724709 ------- 5755398 ------- 5790598 ------- 5819425 ------- 5846341 ------- 5874656 ------- 5908067 ------- 5933572 ------- 5962659 ------- 5999971 - │ │ │ │ │ histogram(39)= 0 600 28805 3000 29405 600 29405 600 29405 600 29405 600 29405 1200 29405 1200 29405 1200 28805 1200 28805 1800 29405 600 29405 1200 29405 600 29405 600 28805 1200 29405 1200 28805 1200 28205 1800 29405 1200 29405 600 29405 1200 29405 1200 29405 600 29405 1200 29405 1200 28805 1800 29405 600 28805 1800 28805 1200 28805 1200 29405 1200 28805 1800 29405 600 28205 1800 29405 600 29405 600 28805 1200 29405 600 28805 1200 29405 600 29405 600 28805 1200 28805 1800 27605 2400 29405 600 28205 1800 29405 2400 29405 600 28805 1800 28205 1800 28805 1200 29405 3000 29405 1200 29405 1800 29405 600 29405 600 28205 1800 29405 600 29405 1800 29405 1800 29405 600 29405 600 28805 1200 28805 1200 29405 1200 28205 1800 29405 600 28805 1200 27605 2400 29405 600 29405 1200 29405 1200 29405 600 29405 1800 29405 1200 28205 1800 29405 600 29405 1200 29405 1200 28805 1200 28205 1800 29405 1800 28805 1200 29405 600 29405 1800 28205 2400 29405 600 28805 1200 28205 2400 28805 1200 27605 2400 28805 1200 29405 600 28805 1800 29405 1800 28805 600 28205 1800 28805 600 29405 600 29405 1200 28805 600 28805 1200 29405 1800 27605 1800 27605 2400 27605 1800 29405 1200 28805 600 28805 1200 29405 1200 28805 2400 28805 1200 28805 600 28205 1200 28205 1200 29405 600 28805 1200 29405 600 28205 2400 28205 1200 29405 600 29405 1800 28805 1200 28205 1200 27605 2400 27605 1800 28805 1200 29405 600 28805 1200 29405 600 29405 600 28805 1200 29405 600 29405 600 29405 600 28805 1200 29405 1200 28805 600 29405 1800 28205 1200 27605 1800 29405 600 29405 600 29405 1200 28805 1200 28805 1200 28205 1200 29405 1800 28205 1800 28805 600 28805 1200 28205 1200 29405 600 28805 1800 28205 1800 27605 2400 28805 600 29405 600 28805 1200 29405 600 28805 2400 28805 1200 28805 1200 28805 600 28205 1200 28805 1200 29405 600 29405 600 29405 1800 28805 600 28805 600 28805 2400 28805 600 28805 1800 28805 1200 28805 600 27005 2400 29405 600 28205 1800 29405 1200 27605 1800 28805 1200 28805 1200 29405 600 29405 600 29405 600 29405 1200 28805 600 29405 600 27605 2400 28805 1200 29405 600 28805 1800 28805 1800 28205 1200 28805 1200 28805 600 29405 1200 28205 1200 + │ │ │ │ │ histogram(43)= 0 600 28805 3000 29405 600 29405 600 29405 600 29405 600 29405 1200 29405 1200 29405 1200 28805 1200 28805 1800 29405 600 29405 1200 29405 600 29405 600 28805 1200 29405 1200 28805 1200 28205 1800 29405 1200 29405 600 29405 1200 29405 1200 29405 600 29405 1200 29405 1200 28805 1800 29405 600 28805 1800 28805 1200 28805 1200 29405 1200 28805 1800 29405 600 28205 1800 29405 600 29405 600 28805 1200 29405 600 28805 1200 29405 600 29405 600 28805 1200 28805 1800 27605 2400 29405 600 28205 1800 29405 2400 29405 600 28805 1800 28205 1800 28805 1200 29405 3000 29405 1200 29405 1800 29405 600 29405 600 28205 1800 29405 600 29405 1800 29405 1800 29405 600 29405 600 28805 1200 28805 1200 29405 1200 28205 1800 29405 600 28805 1200 27605 2400 29405 600 29405 1200 29405 1200 29405 600 29405 1800 29405 1200 28205 1800 29405 600 29405 1200 29405 1200 28805 1200 28205 1800 29405 1800 28805 1200 29405 600 29405 1800 28205 2400 29405 600 28805 1200 28205 2400 28805 1200 27605 2400 28805 1200 29405 600 28805 1800 29405 1800 28805 600 28205 1800 28805 600 29405 600 29405 1200 28805 600 28805 1200 29405 1800 27605 1800 27605 2400 27605 1800 29405 1200 28805 600 28805 1200 29405 1200 28805 2400 28805 1200 28805 600 28205 1200 28205 1200 29405 600 28805 1200 29405 600 28205 2400 28205 1200 29405 600 29405 1800 28805 1200 28205 1200 27605 2400 27605 1800 28805 1200 29405 600 28805 1200 29405 600 29405 600 28805 1200 29405 600 29405 600 29405 600 28805 1200 29405 1200 28805 600 29405 1800 28205 1200 27605 1800 29405 600 29405 600 29405 1200 28805 1200 28805 1200 28205 1200 29405 1800 28205 1800 28805 600 28805 1200 28205 1200 29405 600 28805 1800 28205 1800 27605 2400 28805 600 29405 600 28805 1200 29405 600 28805 2400 28805 1200 28805 1200 28805 600 28205 1200 28805 1200 29405 600 29405 600 29405 1800 28805 600 28805 600 28805 2400 28805 600 28805 1800 28805 1200 28805 600 27005 2400 29405 600 28205 1800 29405 1200 27605 1800 28805 1200 28805 1200 29405 600 29405 600 29405 600 29405 1200 28805 600 29405 600 27605 2400 28805 1200 29405 600 28805 1800 28805 1800 28205 1200 28805 1200 28805 600 29405 1200 28205 1200 │ │ │ │ │ <--- 6 ------- 59 ------- 106 ------- 147 ------- 189 ------- 219 ------- 272 ------- 335 ------- 387 ------- 433 ------- 485 ------- 539 ------- 600 ------- 653 ------- 698 ------- 749 ------- 806 ------- 855 ------- 904 ------- 961 ------- 1008 ------- 1065 ------- 1115 ------- 1163 ------- 1212 ------- 1270 ------- 1329 ------- 1373 ------- 1417 ------- 1470 ------- 1514 ------- 1566 ------- 1616 ------- 1668 ------- 1719 ------- 1765 ------- 1819 ------- 1871 ------- 1926 ------- 1978 ------- 2030 ------- 2080 ------- 2128 ------- 2173 ------- 2223 ------- 2270 ------- 2318 ------- 2382 ------- 2444 ------- 2491 ------- 2534 ------- 2584 ------- 2632 ------- 2675 ------- 2736 ------- 2781 ------- 2841 ------- 2905 ------- 2940 ------- 2996 ------- 3045 ------- 3084 ------- 3124 ------- 3181 ------- 3224 ------- 3268 ------- 3318 ------- 3362 ------- 3419 ------- 3466 ------- 3515 ------- 3560 ------- 3615 ------- 3669 ------- 3725 ------- 3777 ------- 3825 ------- 3871 ------- 3923 ------- 3980 ------- 4035 ------- 4078 ------- 4135 ------- 4184 ------- 4244 ------- 4296 ------- 4346 ------- 4396 ------- 4444 ------- 4489 ------- 4540 ------- 4575 ------- 4627 ------- 4683 ------- 4731 ------- 4777 ------- 4831 ------- 4900 ------- 4953 ------- 5006 ------- 5061 ------- 5119 ------- 5161 ------- 5208 ------- 5260 ------- 5315 ------- 5361 ------- 5415 ------- 5474 ------- 5529 ------- 5572 ------- 5635 ------- 5689 ------- 5738 ------- 5792 ------- 5835 ------- 5890 ------- 5935 ------- 5997 ------- 6039 ------- 6094 ------- 6151 ------- 6201 ------- 6245 ------- 6291 ------- 6343 ------- 6387 ------- 6437 ------- 6480 ------- 6540 ------- 6580 ------- 6629 ------- 6673 ------- 6718 ------- 6771 ------- 6823 ------- 6878 ------- 6935 ------- 6974 ------- 7017 ------- 7060 ------- 7112 ------- 7170 ------- 7231 ------- 7279 ------- 7329 ------- 7373 ------- 7427 ------- 7477 ------- 7520 ------- 7564 ------- 7616 ------- 7671 ------- 7716 ------- 7760 ------- 7815 ------- 7861 ------- 7917 ------- 7969 ------- 8018 ------- 8063 ------- 8112 ------- 8180 ------- 8229 ------- 8268 ------- 8320 ------- 8376 ------- 8432 ------- 8474 ------- 8522 ------- 8569 ------- 8616 ------- 8664 ------- 8705 ------- 8765 ------- 8814 ------- 8868 ------- 8906 ------- 8948 ------- 8990 ------- 9035 ------- 9090 ------- 9137 ------- 9188 ------- 9241 ------- 9292 ------- 9354 ------- 9411 ------- 9472 ------- 9526 ------- 9569 ------- 9613 ------- 9658 ------- 9708 ------- 9737 ------- 9792 ------- 9838 ------- 9903 ------- 9952 ------- 10000 │ │ │ │ └── filters - │ │ │ │ ├── l2.l_orderkey:37 = l1.l_orderkey:8 [type=bool, outer=(8,37), constraints=(/8: (/NULL - ]; /37: (/NULL - ]), fd=(8)==(37), (37)==(8)] - │ │ │ │ └── l2.l_suppkey:39 != l1.l_suppkey:10 [type=bool, outer=(10,39), constraints=(/10: (/NULL - ]; /39: (/NULL - ])] + │ │ │ │ ├── l2.l_orderkey:41 = l1.l_orderkey:9 [type=bool, outer=(9,41), constraints=(/9: (/NULL - ]; /41: (/NULL - ]), fd=(9)==(41), (41)==(9)] + │ │ │ │ └── l2.l_suppkey:43 != l1.l_suppkey:11 [type=bool, outer=(11,43), constraints=(/11: (/NULL - ]; /43: (/NULL - ])] │ │ │ ├── inner-join (lookup supplier) │ │ │ │ ├── save-table-name: q21_lookup_join_13 - │ │ │ │ ├── columns: s_suppkey:1(int!null) s_name:2(char!null) s_nationkey:4(int!null) n_nationkey:33(int!null) n_name:34(char!null) + │ │ │ │ ├── columns: s_suppkey:1(int!null) s_name:2(char!null) s_nationkey:4(int!null) n_nationkey:36(int!null) n_name:37(char!null) │ │ │ │ ├── key columns: [1] = [1] │ │ │ │ ├── lookup columns are key - │ │ │ │ ├── stats: [rows=400, distinct(1)=399.934613, null(1)=0, distinct(2)=399.991883, null(2)=0, distinct(4)=1, null(4)=0, distinct(33)=1, null(33)=0, distinct(34)=1, null(34)=0] + │ │ │ │ ├── stats: [rows=400, distinct(1)=399.934613, null(1)=0, distinct(2)=399.991883, null(2)=0, distinct(4)=1, null(4)=0, distinct(36)=1, null(36)=0, distinct(37)=1, null(37)=0] │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: ()-->(34), (1)-->(2,4), (4)==(33), (33)==(4) + │ │ │ │ ├── fd: ()-->(37), (1)-->(2,4), (4)==(36), (36)==(4) │ │ │ │ ├── inner-join (lookup supplier@s_nk) │ │ │ │ │ ├── save-table-name: q21_lookup_join_14 - │ │ │ │ │ ├── columns: s_suppkey:1(int!null) s_nationkey:4(int!null) n_nationkey:33(int!null) n_name:34(char!null) - │ │ │ │ │ ├── key columns: [33] = [4] - │ │ │ │ │ ├── stats: [rows=400, distinct(1)=399.934613, null(1)=0, distinct(4)=1, null(4)=0, distinct(33)=1, null(33)=0, distinct(34)=1, null(34)=0] + │ │ │ │ │ ├── columns: s_suppkey:1(int!null) s_nationkey:4(int!null) n_nationkey:36(int!null) n_name:37(char!null) + │ │ │ │ │ ├── key columns: [36] = [4] + │ │ │ │ │ ├── stats: [rows=400, distinct(1)=399.934613, null(1)=0, distinct(4)=1, null(4)=0, distinct(36)=1, null(36)=0, distinct(37)=1, null(37)=0] │ │ │ │ │ ├── key: (1) - │ │ │ │ │ ├── fd: ()-->(34), (1)-->(4), (4)==(33), (33)==(4) + │ │ │ │ │ ├── fd: ()-->(37), (1)-->(4), (4)==(36), (36)==(4) │ │ │ │ │ ├── select │ │ │ │ │ │ ├── save-table-name: q21_select_15 - │ │ │ │ │ │ ├── columns: n_nationkey:33(int!null) n_name:34(char!null) - │ │ │ │ │ │ ├── stats: [rows=1, distinct(33)=1, null(33)=0, distinct(34)=1, null(34)=0] - │ │ │ │ │ │ ├── key: (33) - │ │ │ │ │ │ ├── fd: ()-->(34) + │ │ │ │ │ │ ├── columns: n_nationkey:36(int!null) n_name:37(char!null) + │ │ │ │ │ │ ├── stats: [rows=1, distinct(36)=1, null(36)=0, distinct(37)=1, null(37)=0] + │ │ │ │ │ │ ├── key: (36) + │ │ │ │ │ │ ├── fd: ()-->(37) │ │ │ │ │ │ ├── scan nation │ │ │ │ │ │ │ ├── save-table-name: q21_scan_16 - │ │ │ │ │ │ │ ├── columns: n_nationkey:33(int!null) n_name:34(char!null) - │ │ │ │ │ │ │ ├── stats: [rows=25, distinct(33)=25, null(33)=0, distinct(34)=25, null(34)=0] - │ │ │ │ │ │ │ │ histogram(33)= 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 + │ │ │ │ │ │ │ ├── columns: n_nationkey:36(int!null) n_name:37(char!null) + │ │ │ │ │ │ │ ├── stats: [rows=25, distinct(36)=25, null(36)=0, distinct(37)=25, null(37)=0] + │ │ │ │ │ │ │ │ histogram(36)= 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 0 1 │ │ │ │ │ │ │ │ <--- 0 --- 1 --- 2 --- 3 --- 4 --- 5 --- 6 --- 7 --- 8 --- 9 --- 10 --- 11 --- 12 --- 13 --- 14 --- 15 --- 16 --- 17 --- 18 --- 19 --- 20 --- 21 --- 22 --- 23 --- 24 - │ │ │ │ │ │ │ ├── key: (33) - │ │ │ │ │ │ │ └── fd: (33)-->(34) + │ │ │ │ │ │ │ ├── key: (36) + │ │ │ │ │ │ │ └── fd: (36)-->(37) │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── n_name:34 = 'SAUDI ARABIA' [type=bool, outer=(34), constraints=(/34: [/'SAUDI ARABIA' - /'SAUDI ARABIA']; tight), fd=()-->(34)] + │ │ │ │ │ │ └── n_name:37 = 'SAUDI ARABIA' [type=bool, outer=(37), constraints=(/37: [/'SAUDI ARABIA' - /'SAUDI ARABIA']; tight), fd=()-->(37)] │ │ │ │ │ └── filters (true) │ │ │ │ └── filters (true) │ │ │ └── filters - │ │ │ └── s_suppkey:1 = l1.l_suppkey:10 [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] + │ │ │ └── s_suppkey:1 = l1.l_suppkey:11 [type=bool, outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ]), fd=(1)==(11), (11)==(1)] │ │ └── filters - │ │ └── o_orderstatus:26 = 'F' [type=bool, outer=(26), constraints=(/26: [/'F' - /'F']; tight), fd=()-->(26)] + │ │ └── o_orderstatus:28 = 'F' [type=bool, outer=(28), constraints=(/28: [/'F' - /'F']; tight), fd=()-->(28)] │ └── aggregations - │ └── count-rows [as=count_rows:69, type=int] + │ └── count-rows [as=count_rows:75, type=int] └── 100 [type=int] stats table=q21_limit_1 diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q22 b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q22 index 11bf1dd29e3d..39e759675ad5 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch/q22 +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch/q22 @@ -55,29 +55,29 @@ ORDER BY ---- sort ├── save-table-name: q22_sort_1 - ├── columns: cntrycode:27(string) numcust:28(int!null) totacctbal:29(float!null) + ├── columns: cntrycode:30(string) numcust:31(int!null) totacctbal:32(float!null) ├── immutable - ├── stats: [rows=1e-10, distinct(27)=1e-10, null(27)=0, distinct(28)=1e-10, null(28)=0, distinct(29)=1e-10, null(29)=0] - ├── key: (27) - ├── fd: (27)-->(28,29) - ├── ordering: +27 + ├── stats: [rows=1e-10, distinct(30)=1e-10, null(30)=0, distinct(31)=1e-10, null(31)=0, distinct(32)=1e-10, null(32)=0] + ├── key: (30) + ├── fd: (30)-->(31,32) + ├── ordering: +30 └── group-by ├── save-table-name: q22_group_by_2 - ├── columns: cntrycode:27(string) count_rows:28(int!null) sum:29(float!null) - ├── grouping columns: cntrycode:27(string) + ├── columns: cntrycode:30(string) count_rows:31(int!null) sum:32(float!null) + ├── grouping columns: cntrycode:30(string) ├── immutable - ├── stats: [rows=1e-10, distinct(27)=1e-10, null(27)=0, distinct(28)=1e-10, null(28)=0, distinct(29)=1e-10, null(29)=0] - ├── key: (27) - ├── fd: (27)-->(28,29) + ├── stats: [rows=1e-10, distinct(30)=1e-10, null(30)=0, distinct(31)=1e-10, null(31)=0, distinct(32)=1e-10, null(32)=0] + ├── key: (30) + ├── fd: (30)-->(31,32) ├── project │ ├── save-table-name: q22_project_3 - │ ├── columns: cntrycode:27(string) c_acctbal:6(float!null) + │ ├── columns: cntrycode:30(string) c_acctbal:6(float!null) │ ├── immutable - │ ├── stats: [rows=1e-10, distinct(6)=1e-10, null(6)=0, distinct(27)=1e-10, null(27)=0] + │ ├── stats: [rows=1e-10, distinct(6)=1e-10, null(6)=0, distinct(30)=1e-10, null(30)=0] │ ├── anti-join (lookup orders@o_ck) │ │ ├── save-table-name: q22_lookup_join_4 │ │ ├── columns: c_custkey:1(int!null) c_phone:5(char!null) c_acctbal:6(float!null) - │ │ ├── key columns: [1] = [19] + │ │ ├── key columns: [1] = [21] │ │ ├── immutable │ │ ├── stats: [rows=1e-10, distinct(1)=1e-10, null(1)=0, distinct(5)=1e-10, null(5)=0, distinct(6)=1e-10, null(6)=0] │ │ ├── key: (1) @@ -104,33 +104,33 @@ sort │ │ │ └── subquery [type=float] │ │ │ └── scalar-group-by │ │ │ ├── save-table-name: q22_scalar_group_by_7 - │ │ │ ├── columns: avg:17(float) + │ │ │ ├── columns: avg:19(float) │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── immutable - │ │ │ ├── stats: [rows=1, distinct(17)=1, null(17)=0] + │ │ │ ├── stats: [rows=1, distinct(19)=1, null(19)=0] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(17) + │ │ │ ├── fd: ()-->(19) │ │ │ ├── select │ │ │ │ ├── save-table-name: q22_select_8 - │ │ │ │ ├── columns: c_phone:13(char!null) c_acctbal:14(float!null) + │ │ │ │ ├── columns: c_phone:14(char!null) c_acctbal:15(float!null) │ │ │ │ ├── immutable - │ │ │ │ ├── stats: [rows=16666.6667, distinct(13)=16666.6667, null(13)=0, distinct(14)=16666.6667, null(14)=0] + │ │ │ │ ├── stats: [rows=16666.6667, distinct(14)=16666.6667, null(14)=0, distinct(15)=16666.6667, null(15)=0] │ │ │ │ ├── scan customer │ │ │ │ │ ├── save-table-name: q22_scan_9 - │ │ │ │ │ ├── columns: c_phone:13(char!null) c_acctbal:14(float!null) - │ │ │ │ │ └── stats: [rows=150000, distinct(13)=150000, null(13)=0, distinct(14)=140628, null(14)=0] + │ │ │ │ │ ├── columns: c_phone:14(char!null) c_acctbal:15(float!null) + │ │ │ │ │ └── stats: [rows=150000, distinct(14)=150000, null(14)=0, distinct(15)=140628, null(15)=0] │ │ │ │ └── filters - │ │ │ │ ├── c_acctbal:14 > 0.0 [type=bool, outer=(14), constraints=(/14: [/5e-324 - ]; tight)] - │ │ │ │ └── substring(c_phone:13, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31') [type=bool, outer=(13), immutable] + │ │ │ │ ├── c_acctbal:15 > 0.0 [type=bool, outer=(15), constraints=(/15: [/5e-324 - ]; tight)] + │ │ │ │ └── substring(c_phone:14, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31') [type=bool, outer=(14), immutable] │ │ │ └── aggregations - │ │ │ └── avg [as=avg:17, type=float, outer=(14)] - │ │ │ └── c_acctbal:14 [type=float] + │ │ │ └── avg [as=avg:19, type=float, outer=(15)] + │ │ │ └── c_acctbal:15 [type=float] │ │ └── filters (true) │ └── projections - │ └── substring(c_phone:5, 1, 2) [as=cntrycode:27, type=string, outer=(5), immutable] + │ └── substring(c_phone:5, 1, 2) [as=cntrycode:30, type=string, outer=(5), immutable] └── aggregations - ├── count-rows [as=count_rows:28, type=int] - └── sum [as=sum:29, type=float, outer=(6)] + ├── count-rows [as=count_rows:31, type=int] + └── sum [as=sum:32, type=float, outer=(6)] └── c_acctbal:6 [type=float] stats table=q22_sort_1 diff --git a/pkg/sql/opt/memo/testdata/typing b/pkg/sql/opt/memo/testdata/typing index 6f5fa4b62472..dc28f3a69791 100644 --- a/pkg/sql/opt/memo/testdata/typing +++ b/pkg/sql/opt/memo/testdata/typing @@ -17,7 +17,7 @@ SELECT a.x FROM a project ├── columns: x:1(int!null) └── scan a - └── columns: x:1(int!null) y:2(int) + └── columns: x:1(int!null) y:2(int) crdb_internal_mvcc_timestamp:3(decimal) # Const build @@ -37,124 +37,142 @@ project build SELECT * FROM a WHERE x = $1 ---- -select +project ├── columns: x:1(int!null) y:2(int) - ├── scan a - │ └── columns: x:1(int!null) y:2(int) - └── filters - └── x:1 = $1 [type=bool] + └── select + ├── columns: x:1(int!null) y:2(int) crdb_internal_mvcc_timestamp:3(decimal) + ├── scan a + │ └── columns: x:1(int!null) y:2(int) crdb_internal_mvcc_timestamp:3(decimal) + └── filters + └── x:1 = $1 [type=bool] # Tuple, Projections build SELECT (a.x, 1.5) AS r, a.y FROM a ---- project - ├── columns: r:3(tuple{int, decimal}!null) y:2(int) + ├── columns: r:4(tuple{int, decimal}!null) y:2(int) ├── scan a - │ └── columns: x:1(int!null) y:2(int) + │ └── columns: x:1(int!null) y:2(int) crdb_internal_mvcc_timestamp:3(decimal) └── projections - └── (x:1, 1.5) [as=r:3, type=tuple{int, decimal}] + └── (x:1, 1.5) [as=r:4, type=tuple{int, decimal}] # And, Or, Not build SELECT * FROM a WHERE a.x = 1 AND NOT (a.y = 2 OR a.y = 3.5) ---- -select +project ├── columns: x:1(int!null) y:2(int) - ├── scan a - │ └── columns: x:1(int!null) y:2(int) - └── filters - └── (x:1 = 1) AND (NOT ((y:2 = 2) OR (y:2 = 3.5))) [type=bool] + └── select + ├── columns: x:1(int!null) y:2(int) crdb_internal_mvcc_timestamp:3(decimal) + ├── scan a + │ └── columns: x:1(int!null) y:2(int) crdb_internal_mvcc_timestamp:3(decimal) + └── filters + └── (x:1 = 1) AND (NOT ((y:2 = 2) OR (y:2 = 3.5))) [type=bool] # Eq, Ne build SELECT * FROM a WHERE a.x = 1 AND a.x <> 2 ---- -select +project ├── columns: x:1(int!null) y:2(int) - ├── scan a - │ └── columns: x:1(int!null) y:2(int) - └── filters - └── (x:1 = 1) AND (x:1 != 2) [type=bool] + └── select + ├── columns: x:1(int!null) y:2(int) crdb_internal_mvcc_timestamp:3(decimal) + ├── scan a + │ └── columns: x:1(int!null) y:2(int) crdb_internal_mvcc_timestamp:3(decimal) + └── filters + └── (x:1 = 1) AND (x:1 != 2) [type=bool] # Le, Ge, Lt, Gt build SELECT * FROM a WHERE a.x >= 1 AND a.x <= 10 AND a.y > 1 AND a.y < 10 ---- -select +project ├── columns: x:1(int!null) y:2(int!null) - ├── scan a - │ └── columns: x:1(int!null) y:2(int) - └── filters - └── (((x:1 >= 1) AND (x:1 <= 10)) AND (y:2 > 1)) AND (y:2 < 10) [type=bool] + └── select + ├── columns: x:1(int!null) y:2(int!null) crdb_internal_mvcc_timestamp:3(decimal) + ├── scan a + │ └── columns: x:1(int!null) y:2(int) crdb_internal_mvcc_timestamp:3(decimal) + └── filters + └── (((x:1 >= 1) AND (x:1 <= 10)) AND (y:2 > 1)) AND (y:2 < 10) [type=bool] # In, NotIn build SELECT * FROM a WHERE a.x IN (1, 2) AND a.y NOT IN (3, 4) ---- -select +project ├── columns: x:1(int!null) y:2(int) - ├── scan a - │ └── columns: x:1(int!null) y:2(int) - └── filters - └── (x:1 IN (1, 2)) AND (y:2 NOT IN (3, 4)) [type=bool] + └── select + ├── columns: x:1(int!null) y:2(int) crdb_internal_mvcc_timestamp:3(decimal) + ├── scan a + │ └── columns: x:1(int!null) y:2(int) crdb_internal_mvcc_timestamp:3(decimal) + └── filters + └── (x:1 IN (1, 2)) AND (y:2 NOT IN (3, 4)) [type=bool] # Like, NotLike build SELECT * FROM b WHERE b.x LIKE '%foo%' AND b.x NOT LIKE '%bar%' ---- -select +project ├── columns: x:1(string!null) z:2(decimal!null) - ├── scan b - │ └── columns: x:1(string!null) z:2(decimal!null) - └── filters - └── (x:1 LIKE '%foo%') AND (x:1 NOT LIKE '%bar%') [type=bool] + └── select + ├── columns: x:1(string!null) z:2(decimal!null) crdb_internal_mvcc_timestamp:3(decimal) + ├── scan b + │ └── columns: x:1(string!null) z:2(decimal!null) crdb_internal_mvcc_timestamp:3(decimal) + └── filters + └── (x:1 LIKE '%foo%') AND (x:1 NOT LIKE '%bar%') [type=bool] # ILike, INotLike build SELECT * FROM b WHERE b.x ILIKE '%foo%' AND b.x NOT ILIKE '%bar%' ---- -select +project ├── columns: x:1(string!null) z:2(decimal!null) - ├── scan b - │ └── columns: x:1(string!null) z:2(decimal!null) - └── filters - └── (x:1 ILIKE '%foo%') AND (x:1 NOT ILIKE '%bar%') [type=bool] + └── select + ├── columns: x:1(string!null) z:2(decimal!null) crdb_internal_mvcc_timestamp:3(decimal) + ├── scan b + │ └── columns: x:1(string!null) z:2(decimal!null) crdb_internal_mvcc_timestamp:3(decimal) + └── filters + └── (x:1 ILIKE '%foo%') AND (x:1 NOT ILIKE '%bar%') [type=bool] # RegMatch, NotRegMatch, RegIMatch, NotRegIMatch build SELECT * FROM b WHERE b.x ~ 'foo' AND b.x !~ 'bar' AND b.x ~* 'foo' AND b.x !~* 'bar' ---- -select +project ├── columns: x:1(string!null) z:2(decimal!null) - ├── scan b - │ └── columns: x:1(string!null) z:2(decimal!null) - └── filters - └── (((x:1 ~ 'foo') AND (x:1 !~ 'bar')) AND (x:1 ~* 'foo')) AND (x:1 !~* 'bar') [type=bool] + └── select + ├── columns: x:1(string!null) z:2(decimal!null) crdb_internal_mvcc_timestamp:3(decimal) + ├── scan b + │ └── columns: x:1(string!null) z:2(decimal!null) crdb_internal_mvcc_timestamp:3(decimal) + └── filters + └── (((x:1 ~ 'foo') AND (x:1 !~ 'bar')) AND (x:1 ~* 'foo')) AND (x:1 !~* 'bar') [type=bool] # Is, IsNot build SELECT * FROM a WHERE a.x IS DISTINCT FROM a.y AND a.x IS NULL ---- -select +project ├── columns: x:1(int!null) y:2(int) - ├── scan a - │ └── columns: x:1(int!null) y:2(int) - └── filters - └── (x:1 IS DISTINCT FROM y:2) AND (x:1 IS NULL) [type=bool] + └── select + ├── columns: x:1(int!null) y:2(int) crdb_internal_mvcc_timestamp:3(decimal) + ├── scan a + │ └── columns: x:1(int!null) y:2(int) crdb_internal_mvcc_timestamp:3(decimal) + └── filters + └── (x:1 IS DISTINCT FROM y:2) AND (x:1 IS NULL) [type=bool] # Bitand, Bitor, Bitxor build SELECT a.x & a.y AS r, a.x | a.y AS s, a.x # a.y AS t FROM a ---- project - ├── columns: r:3(int) s:4(int) t:5(int) + ├── columns: r:4(int) s:5(int) t:6(int) ├── scan a - │ └── columns: x:1(int!null) y:2(int) + │ └── columns: x:1(int!null) y:2(int) crdb_internal_mvcc_timestamp:3(decimal) └── projections - ├── x:1 & y:2 [as=r:3, type=int] - ├── x:1 | y:2 [as=s:4, type=int] - └── x:1 # y:2 [as=t:5, type=int] + ├── x:1 & y:2 [as=r:4, type=int] + ├── x:1 | y:2 [as=s:5, type=int] + └── x:1 # y:2 [as=t:6, type=int] # Plus, Minus, Mult, Div, FloorDiv build @@ -166,15 +184,15 @@ SELECT a.x + 1.5 AS r, FROM a ---- project - ├── columns: r:3(decimal!null) s:4(date!null) t:5(decimal!null) u:6(decimal) v:7(decimal!null) + ├── columns: r:4(decimal!null) s:5(date!null) t:6(decimal!null) u:7(decimal) v:8(decimal!null) ├── scan a - │ └── columns: x:1(int!null) y:2(int) + │ └── columns: x:1(int!null) y:2(int) crdb_internal_mvcc_timestamp:3(decimal) └── projections - ├── x:1 + 1.5 [as=r:3, type=decimal] - ├── '2000-01-01' - 15 [as=s:4, type=date] - ├── 10.10 * x:1 [as=t:5, type=decimal] - ├── 1 / y:2 [as=u:6, type=decimal] - └── x:1 // 1.5 [as=v:7, type=decimal] + ├── x:1 + 1.5 [as=r:4, type=decimal] + ├── '2000-01-01' - 15 [as=s:5, type=date] + ├── 10.10 * x:1 [as=t:6, type=decimal] + ├── 1 / y:2 [as=u:7, type=decimal] + └── x:1 // 1.5 [as=v:8, type=decimal] # Mod, Pow, LShift, RShift build @@ -185,14 +203,14 @@ SELECT 100.1 % a.x AS r, FROM a ---- project - ├── columns: r:3(decimal!null) s:4(decimal!null) t:5(int!null) u:6(int) + ├── columns: r:4(decimal!null) s:5(decimal!null) t:6(int!null) u:7(int) ├── scan a - │ └── columns: x:1(int!null) y:2(int) + │ └── columns: x:1(int!null) y:2(int) crdb_internal_mvcc_timestamp:3(decimal) └── projections - ├── 100.1 % x:1 [as=r:3, type=decimal] - ├── x:1 ^ 2.5 [as=s:4, type=decimal] - ├── x:1 << 3 [as=t:5, type=int] - └── y:2 >> 2 [as=u:6, type=int] + ├── 100.1 % x:1 [as=r:4, type=decimal] + ├── x:1 ^ 2.5 [as=s:5, type=decimal] + ├── x:1 << 3 [as=t:6, type=int] + └── y:2 >> 2 [as=u:7, type=int] # FetchVal, FetchText, FetchValPath, FetchTextPath build @@ -203,64 +221,64 @@ SELECT '[1, 2]'->1 AS r, FROM a ---- project - ├── columns: r:3(jsonb) s:4(string) t:5(jsonb) u:6(string) + ├── columns: r:4(jsonb) s:5(string) t:6(jsonb) u:7(string) ├── scan a - │ └── columns: x:1(int!null) y:2(int) + │ └── columns: x:1(int!null) y:2(int) crdb_internal_mvcc_timestamp:3(decimal) └── projections - ├── '[1, 2]'->1 [as=r:3, type=jsonb] - ├── '[1, 2]'->>1 [as=s:4, type=string] - ├── '{"a": 5}'#>ARRAY['a'] [as=t:5, type=jsonb] - └── '{"a": 5}'#>>ARRAY['a'] [as=u:6, type=string] + ├── '[1, 2]'->1 [as=r:4, type=jsonb] + ├── '[1, 2]'->>1 [as=s:5, type=string] + ├── '{"a": 5}'#>ARRAY['a'] [as=t:6, type=jsonb] + └── '{"a": 5}'#>>ARRAY['a'] [as=u:7, type=string] # Concat build SELECT b.x || 'more' AS r FROM b ---- project - ├── columns: r:3(string!null) + ├── columns: r:4(string!null) ├── scan b - │ └── columns: x:1(string!null) z:2(decimal!null) + │ └── columns: x:1(string!null) z:2(decimal!null) crdb_internal_mvcc_timestamp:3(decimal) └── projections - └── x:1 || 'more' [as=r:3, type=string] + └── x:1 || 'more' [as=r:4, type=string] # UnaryMinus, UnaryComplement build SELECT -a.y AS r, ~a.x AS s FROM a ---- project - ├── columns: r:3(int) s:4(int) + ├── columns: r:4(int) s:5(int) ├── scan a - │ └── columns: x:1(int!null) y:2(int) + │ └── columns: x:1(int!null) y:2(int) crdb_internal_mvcc_timestamp:3(decimal) └── projections - ├── -y:2 [as=r:3, type=int] - └── ~x:1 [as=s:4, type=int] + ├── -y:2 [as=r:4, type=int] + └── ~x:1 [as=s:5, type=int] # Array Concat build SELECT arr || arr AS r, arr || NULL AS s, NULL || arr AS t FROM unusual ---- project - ├── columns: r:3(int[]) s:4(int[]) t:5(int[]) + ├── columns: r:4(int[]) s:5(int[]) t:6(int[]) ├── scan unusual - │ └── columns: x:1(int!null) arr:2(int[]) + │ └── columns: x:1(int!null) arr:2(int[]) crdb_internal_mvcc_timestamp:3(decimal) └── projections - ├── arr:2 || arr:2 [as=r:3, type=int[]] - ├── arr:2 || NULL::INT8[] [as=s:4, type=int[]] - └── NULL::INT8[] || arr:2 [as=t:5, type=int[]] + ├── arr:2 || arr:2 [as=r:4, type=int[]] + ├── arr:2 || NULL::INT8[] [as=s:5, type=int[]] + └── NULL::INT8[] || arr:2 [as=t:6, type=int[]] # Array Element Concat build SELECT x || arr AS r, arr || x AS s, x || NULL AS t, NULL || x AS u FROM unusual ---- project - ├── columns: r:3(int[]) s:4(int[]) t:5(int[]) u:6(int[]) + ├── columns: r:4(int[]) s:5(int[]) t:6(int[]) u:7(int[]) ├── scan unusual - │ └── columns: x:1(int!null) arr:2(int[]) + │ └── columns: x:1(int!null) arr:2(int[]) crdb_internal_mvcc_timestamp:3(decimal) └── projections - ├── x:1 || arr:2 [as=r:3, type=int[]] - ├── arr:2 || x:1 [as=s:4, type=int[]] - ├── x:1 || NULL::INT8[] [as=t:5, type=int[]] - └── NULL::INT8[] || x:1 [as=u:6, type=int[]] + ├── x:1 || arr:2 [as=r:4, type=int[]] + ├── arr:2 || x:1 [as=s:5, type=int[]] + ├── x:1 || NULL::INT8[] [as=t:6, type=int[]] + └── NULL::INT8[] || x:1 [as=u:7, type=int[]] # Function with fixed return type. build @@ -315,125 +333,125 @@ SELECT FROM b ---- scalar-group-by - ├── columns: array_agg:3(decimal[]) avg:4(decimal) bool_and:6(bool) bool_or:7(bool) concat_agg:8(string) count:9(int!null) count:10(int!null) max:11(string) max:12(decimal) sum_int:14(int) sum:15(decimal) sqrdiff:16(decimal) variance:17(decimal) stddev:18(decimal) xor_agg:19(int) json_agg:21(jsonb) jsonb_agg:22(jsonb) + ├── columns: array_agg:4(decimal[]) avg:5(decimal) bool_and:7(bool) bool_or:8(bool) concat_agg:9(string) count:10(int!null) count:11(int!null) max:12(string) max:13(decimal) sum_int:15(int) sum:16(decimal) sqrdiff:17(decimal) variance:18(decimal) stddev:19(decimal) xor_agg:20(int) json_agg:22(jsonb) jsonb_agg:23(jsonb) ├── project - │ ├── columns: column5:5(bool!null) column13:13(int!null) column20:20(jsonb!null) x:1(string!null) z:2(decimal!null) + │ ├── columns: column6:6(bool!null) column14:14(int!null) column21:21(jsonb!null) x:1(string!null) z:2(decimal!null) │ ├── scan b - │ │ └── columns: x:1(string!null) z:2(decimal!null) + │ │ └── columns: x:1(string!null) z:2(decimal!null) crdb_internal_mvcc_timestamp:3(decimal) │ └── projections - │ ├── z:2 = 0 [as=column5:5, type=bool] - │ ├── x:1::INT8 [as=column13:13, type=int] - │ └── x:1::JSONB [as=column20:20, type=jsonb] + │ ├── z:2 = 0 [as=column6:6, type=bool] + │ ├── x:1::INT8 [as=column14:14, type=int] + │ └── x:1::JSONB [as=column21:21, type=jsonb] └── aggregations - ├── array-agg [as=array_agg:3, type=decimal[]] + ├── array-agg [as=array_agg:4, type=decimal[]] │ └── z:2 [type=decimal] - ├── avg [as=avg:4, type=decimal] + ├── avg [as=avg:5, type=decimal] │ └── z:2 [type=decimal] - ├── bool-and [as=bool_and:6, type=bool] - │ └── column5:5 [type=bool] - ├── bool-or [as=bool_or:7, type=bool] - │ └── column5:5 [type=bool] - ├── concat-agg [as=concat_agg:8, type=string] + ├── bool-and [as=bool_and:7, type=bool] + │ └── column6:6 [type=bool] + ├── bool-or [as=bool_or:8, type=bool] + │ └── column6:6 [type=bool] + ├── concat-agg [as=concat_agg:9, type=string] │ └── x:1 [type=string] - ├── count [as=count:9, type=int] + ├── count [as=count:10, type=int] │ └── z:2 [type=decimal] - ├── count-rows [as=count_rows:10, type=int] - ├── max [as=max:11, type=string] + ├── count-rows [as=count_rows:11, type=int] + ├── max [as=max:12, type=string] │ └── x:1 [type=string] - ├── max [as=max:12, type=decimal] + ├── max [as=max:13, type=decimal] │ └── z:2 [type=decimal] - ├── sum-int [as=sum_int:14, type=int] - │ └── column13:13 [type=int] - ├── sum [as=sum:15, type=decimal] + ├── sum-int [as=sum_int:15, type=int] + │ └── column14:14 [type=int] + ├── sum [as=sum:16, type=decimal] │ └── z:2 [type=decimal] - ├── sqr-diff [as=sqrdiff:16, type=decimal] + ├── sqr-diff [as=sqrdiff:17, type=decimal] │ └── z:2 [type=decimal] - ├── variance [as=variance:17, type=decimal] - │ └── column13:13 [type=int] - ├── std-dev [as=stddev:18, type=decimal] + ├── variance [as=variance:18, type=decimal] + │ └── column14:14 [type=int] + ├── std-dev [as=stddev:19, type=decimal] │ └── z:2 [type=decimal] - ├── xor-agg [as=xor_agg:19, type=int] - │ └── column13:13 [type=int] - ├── json-agg [as=json_agg:21, type=jsonb] - │ └── column20:20 [type=jsonb] - └── jsonb-agg [as=jsonb_agg:22, type=jsonb] - └── column20:20 [type=jsonb] + ├── xor-agg [as=xor_agg:20, type=int] + │ └── column14:14 [type=int] + ├── json-agg [as=json_agg:22, type=jsonb] + │ └── column21:21 [type=jsonb] + └── jsonb-agg [as=jsonb_agg:23, type=jsonb] + └── column21:21 [type=jsonb] # ConstAgg internal aggregate function. opt SELECT * FROM (SELECT x, x::string, y FROM a) WHERE (SELECT max(x) FROM b WHERE y=z::int) > 'foo' ---- project - ├── columns: x:1(int!null) x:3(string!null) y:2(int!null) + ├── columns: x:1(int!null) x:4(string!null) y:2(int!null) ├── select - │ ├── columns: a.x:1(int!null) y:2(int!null) max:6(string!null) + │ ├── columns: a.x:1(int!null) y:2(int!null) max:8(string!null) │ ├── group-by - │ │ ├── columns: a.x:1(int!null) y:2(int!null) max:6(string!null) + │ │ ├── columns: a.x:1(int!null) y:2(int!null) max:8(string!null) │ │ ├── grouping columns: a.x:1(int!null) │ │ ├── inner-join (hash) - │ │ │ ├── columns: a.x:1(int!null) y:2(int!null) b.x:4(string!null) column7:7(int!null) + │ │ │ ├── columns: a.x:1(int!null) y:2(int!null) b.x:5(string!null) column9:9(int!null) │ │ │ ├── scan a │ │ │ │ └── columns: a.x:1(int!null) y:2(int) │ │ │ ├── project - │ │ │ │ ├── columns: column7:7(int!null) b.x:4(string!null) + │ │ │ │ ├── columns: column9:9(int!null) b.x:5(string!null) │ │ │ │ ├── scan b - │ │ │ │ │ └── columns: b.x:4(string!null) z:5(decimal!null) + │ │ │ │ │ └── columns: b.x:5(string!null) z:6(decimal!null) │ │ │ │ └── projections - │ │ │ │ └── z:5::INT8 [as=column7:7, type=int] + │ │ │ │ └── z:6::INT8 [as=column9:9, type=int] │ │ │ └── filters - │ │ │ └── y:2 = column7:7 [type=bool] + │ │ │ └── y:2 = column9:9 [type=bool] │ │ └── aggregations - │ │ ├── max [as=max:6, type=string] - │ │ │ └── b.x:4 [type=string] + │ │ ├── max [as=max:8, type=string] + │ │ │ └── b.x:5 [type=string] │ │ └── const-agg [as=y:2, type=int] │ │ └── y:2 [type=int] │ └── filters - │ └── max:6 > 'foo' [type=bool] + │ └── max:8 > 'foo' [type=bool] └── projections - └── a.x:1::STRING [as=x:3, type=string] + └── a.x:1::STRING [as=x:4, type=string] # ConstNotNullAgg internal aggregate function. opt SELECT EXISTS(SELECT * FROM a WHERE expr<0) FROM (SELECT x+1 AS expr FROM a) ---- project - ├── columns: exists:6(bool!null) + ├── columns: exists:8(bool!null) ├── group-by - │ ├── columns: true_agg:8(bool) rownum:10(int!null) - │ ├── grouping columns: rownum:10(int!null) + │ ├── columns: true_agg:10(bool) rownum:12(int!null) + │ ├── grouping columns: rownum:12(int!null) │ ├── left-join (cross) - │ │ ├── columns: expr:3(int!null) true:7(bool) rownum:10(int!null) + │ │ ├── columns: expr:4(int!null) true:9(bool) rownum:12(int!null) │ │ ├── ordinality - │ │ │ ├── columns: expr:3(int!null) rownum:10(int!null) + │ │ │ ├── columns: expr:4(int!null) rownum:12(int!null) │ │ │ └── project - │ │ │ ├── columns: expr:3(int!null) + │ │ │ ├── columns: expr:4(int!null) │ │ │ ├── scan a │ │ │ │ └── columns: x:1(int!null) │ │ │ └── projections - │ │ │ └── x:1 + 1 [as=expr:3, type=int] + │ │ │ └── x:1 + 1 [as=expr:4, type=int] │ │ ├── project - │ │ │ ├── columns: true:7(bool!null) + │ │ │ ├── columns: true:9(bool!null) │ │ │ ├── scan a │ │ │ └── projections - │ │ │ └── true [as=true:7, type=bool] + │ │ │ └── true [as=true:9, type=bool] │ │ └── filters - │ │ └── expr:3 < 0 [type=bool] + │ │ └── expr:4 < 0 [type=bool] │ └── aggregations - │ └── const-not-null-agg [as=true_agg:8, type=bool] - │ └── true:7 [type=bool] + │ └── const-not-null-agg [as=true_agg:10, type=bool] + │ └── true:9 [type=bool] └── projections - └── true_agg:8 IS NOT NULL [as=exists:6, type=bool] + └── true_agg:10 IS NOT NULL [as=exists:8, type=bool] # Cast build SELECT x::VARCHAR(2) FROM b ---- project - ├── columns: x:3(varchar!null) + ├── columns: x:4(varchar!null) ├── scan b - │ └── columns: b.x:1(string!null) z:2(decimal!null) + │ └── columns: b.x:1(string!null) z:2(decimal!null) crdb_internal_mvcc_timestamp:3(decimal) └── projections - └── b.x:1::VARCHAR(2) [as=x:3, type=varchar] + └── b.x:1::VARCHAR(2) [as=x:4, type=varchar] # Cast same type with different precisions. # See #42571. @@ -441,10 +459,10 @@ build SELECT z::decimal(10, 3), z::decimal(10, 1), z::decimal(10, 4) FROM b ---- project - ├── columns: z:3(decimal!null) z:4(decimal!null) z:5(decimal!null) + ├── columns: z:4(decimal!null) z:5(decimal!null) z:6(decimal!null) ├── scan b - │ └── columns: x:1(string!null) b.z:2(decimal!null) + │ └── columns: x:1(string!null) b.z:2(decimal!null) crdb_internal_mvcc_timestamp:3(decimal) └── projections - ├── b.z:2::DECIMAL(10,3) [as=z:3, type=decimal] - ├── b.z:2::DECIMAL(10,1) [as=z:4, type=decimal] - └── b.z:2::DECIMAL(10,4) [as=z:5, type=decimal] + ├── b.z:2::DECIMAL(10,3) [as=z:4, type=decimal] + ├── b.z:2::DECIMAL(10,1) [as=z:5, type=decimal] + └── b.z:2::DECIMAL(10,4) [as=z:6, type=decimal] diff --git a/pkg/sql/opt/norm/prune_cols_funcs.go b/pkg/sql/opt/norm/prune_cols_funcs.go index e24784d89047..9be590e39865 100644 --- a/pkg/sql/opt/norm/prune_cols_funcs.go +++ b/pkg/sql/opt/norm/prune_cols_funcs.go @@ -170,7 +170,14 @@ func neededMutationFetchCols( if i == cat.PrimaryIndex && !keyCols.Intersects(updateCols) { addFamilyCols(updateCols) } else { - cols.UnionWith(indexCols) + // Add all of the index columns into cols. + indexCols.ForEach(func(col opt.ColumnID) { + ord := tabMeta.MetaID.ColumnOrdinal(col) + // We don't want to include system columns. + if !cat.IsSystemColumn(tabMeta.Table, ord) { + cols.Add(col) + } + }) } } diff --git a/pkg/sql/opt/norm/testdata/rules/agg b/pkg/sql/opt/norm/testdata/rules/agg index 873a5e71f233..01954fc983ad 100644 --- a/pkg/sql/opt/norm/testdata/rules/agg +++ b/pkg/sql/opt/norm/testdata/rules/agg @@ -10,30 +10,30 @@ norm expect=EliminateAggDistinct SELECT min(DISTINCT i), max(DISTINCT i), bool_and(DISTINCT i>f), bool_or(DISTINCT i>f), corr(DISTINCT k, i) FROM a ---- scalar-group-by - ├── columns: min:7 max:8 bool_and:10 bool_or:11 corr:12 + ├── columns: min:8 max:9 bool_and:11 bool_or:12 corr:13 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(7,8,10-12) + ├── fd: ()-->(8,9,11-13) ├── project - │ ├── columns: column9:9 k:1!null i:2 + │ ├── columns: column10:10 k:1!null i:2 │ ├── key: (1) - │ ├── fd: (1)-->(2,9) + │ ├── fd: (1)-->(2,10) │ ├── scan a │ │ ├── columns: k:1!null i:2 f:3 │ │ ├── key: (1) │ │ └── fd: (1)-->(2,3) │ └── projections - │ └── i:2 > f:3 [as=column9:9, outer=(2,3)] + │ └── i:2 > f:3 [as=column10:10, outer=(2,3)] └── aggregations - ├── min [as=min:7, outer=(2)] + ├── min [as=min:8, outer=(2)] │ └── i:2 - ├── max [as=max:8, outer=(2)] + ├── max [as=max:9, outer=(2)] │ └── i:2 - ├── bool-and [as=bool_and:10, outer=(9)] - │ └── column9:9 - ├── bool-or [as=bool_or:11, outer=(9)] - │ └── column9:9 - └── corr [as=corr:12, outer=(1,2)] + ├── bool-and [as=bool_and:11, outer=(10)] + │ └── column10:10 + ├── bool-or [as=bool_or:12, outer=(10)] + │ └── column10:10 + └── corr [as=corr:13, outer=(1,2)] ├── k:1 └── i:2 @@ -48,46 +48,46 @@ SELECT FROM a ---- scalar-group-by - ├── columns: min:8 max:9 bool_and:12 bool_or:14 corr:16 + ├── columns: min:9 max:10 bool_and:13 bool_or:15 corr:17 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(8,9,12,14,16) + ├── fd: ()-->(9,10,13,15,17) ├── project - │ ├── columns: column7:7 column10:10 column11:11 column13:13 column15:15 k:1!null i:2 + │ ├── columns: column8:8 column11:11 column12:12 column14:14 column16:16 k:1!null i:2 │ ├── key: (1) - │ ├── fd: (1)-->(2,7,10,11,13,15), (2)-->(7) + │ ├── fd: (1)-->(2,8,11,12,14,16), (2)-->(8) │ ├── scan a │ │ ├── columns: k:1!null i:2 f:3 │ │ ├── key: (1) │ │ └── fd: (1)-->(2,3) │ └── projections - │ ├── i:2 > 5 [as=column7:7, outer=(2)] - │ ├── i:2 > f:3 [as=column10:10, outer=(2,3)] - │ ├── f:3 > 0.0 [as=column11:11, outer=(3)] - │ ├── f:3 > 1.0 [as=column13:13, outer=(3)] - │ └── (k:1 > 5) AND (i:2 > 5) [as=column15:15, outer=(1,2)] + │ ├── i:2 > 5 [as=column8:8, outer=(2)] + │ ├── i:2 > f:3 [as=column11:11, outer=(2,3)] + │ ├── f:3 > 0.0 [as=column12:12, outer=(3)] + │ ├── f:3 > 1.0 [as=column14:14, outer=(3)] + │ └── (k:1 > 5) AND (i:2 > 5) [as=column16:16, outer=(1,2)] └── aggregations - ├── agg-filter [as=min:8, outer=(2,7)] + ├── agg-filter [as=min:9, outer=(2,8)] │ ├── min │ │ └── i:2 - │ └── column7:7 - ├── agg-filter [as=max:9, outer=(2,7)] + │ └── column8:8 + ├── agg-filter [as=max:10, outer=(2,8)] │ ├── max │ │ └── i:2 - │ └── column7:7 - ├── agg-filter [as=bool_and:12, outer=(10,11)] + │ └── column8:8 + ├── agg-filter [as=bool_and:13, outer=(11,12)] │ ├── bool-and - │ │ └── column10:10 - │ └── column11:11 - ├── agg-filter [as=bool_or:14, outer=(10,13)] + │ │ └── column11:11 + │ └── column12:12 + ├── agg-filter [as=bool_or:15, outer=(11,14)] │ ├── bool-or - │ │ └── column10:10 - │ └── column13:13 - └── agg-filter [as=corr:16, outer=(1,2,15)] + │ │ └── column11:11 + │ └── column14:14 + └── agg-filter [as=corr:17, outer=(1,2,16)] ├── corr │ ├── k:1 │ └── i:2 - └── column15:15 + └── column16:16 # The rule should not apply to these aggregations. norm expect-not=EliminateAggDistinct @@ -104,47 +104,47 @@ SELECT FROM a ---- scalar-group-by - ├── columns: count:7!null sum:9 sum_int:10 avg:11 stddev:12 variance:13 xor_agg:15 array_agg:16 json_agg:17 + ├── columns: count:8!null sum:10 sum_int:11 avg:12 stddev:13 variance:14 xor_agg:16 array_agg:17 json_agg:18 ├── cardinality: [1 - 1] ├── immutable ├── key: () - ├── fd: ()-->(7,9-13,15-17) + ├── fd: ()-->(8,10-14,16-18) ├── project - │ ├── columns: column8:8 column14:14 i:2 f:3 j:5 + │ ├── columns: column9:9 column15:15 i:2 f:3 j:5 │ ├── immutable - │ ├── fd: (2)-->(8) + │ ├── fd: (2)-->(9) │ ├── scan a │ │ └── columns: i:2 f:3 s:4 j:5 │ └── projections - │ ├── i:2 > 5 [as=column8:8, outer=(2)] - │ └── s:4::BYTES [as=column14:14, outer=(4), immutable] + │ ├── i:2 > 5 [as=column9:9, outer=(2)] + │ └── s:4::BYTES [as=column15:15, outer=(4), immutable] └── aggregations - ├── agg-distinct [as=count:7, outer=(2)] + ├── agg-distinct [as=count:8, outer=(2)] │ └── count │ └── i:2 - ├── agg-filter [as=sum:9, outer=(2,8)] + ├── agg-filter [as=sum:10, outer=(2,9)] │ ├── agg-distinct │ │ └── sum │ │ └── i:2 - │ └── column8:8 - ├── agg-distinct [as=sum_int:10, outer=(2)] + │ └── column9:9 + ├── agg-distinct [as=sum_int:11, outer=(2)] │ └── sum-int │ └── i:2 - ├── agg-distinct [as=avg:11, outer=(2)] + ├── agg-distinct [as=avg:12, outer=(2)] │ └── avg │ └── i:2 - ├── agg-distinct [as=stddev:12, outer=(3)] + ├── agg-distinct [as=stddev:13, outer=(3)] │ └── std-dev │ └── f:3 - ├── agg-distinct [as=variance:13, outer=(3)] + ├── agg-distinct [as=variance:14, outer=(3)] │ └── variance │ └── f:3 - ├── agg-distinct [as=xor_agg:15, outer=(14)] + ├── agg-distinct [as=xor_agg:16, outer=(15)] │ └── xor-agg - │ └── column14:14 - ├── agg-distinct [as=array_agg:16, outer=(2)] + │ └── column15:15 + ├── agg-distinct [as=array_agg:17, outer=(2)] │ └── array-agg │ └── i:2 - └── agg-distinct [as=json_agg:17, outer=(5)] + └── agg-distinct [as=json_agg:18, outer=(5)] └── json-agg └── j:5 diff --git a/pkg/sql/opt/norm/testdata/rules/bool b/pkg/sql/opt/norm/testdata/rules/bool index 01dd3312a510..e57db301b783 100644 --- a/pkg/sql/opt/norm/testdata/rules/bool +++ b/pkg/sql/opt/norm/testdata/rules/bool @@ -19,32 +19,32 @@ norm expect=NormalizeNestedAnds SELECT a AND (b AND (c AND (d AND e))) FROM c ---- project - ├── columns: "?column?":7 + ├── columns: "?column?":8 ├── scan c │ └── columns: a:1 b:2 c:3 d:4 e:5 └── projections - └── (((a:1 AND b:2) AND c:3) AND d:4) AND e:5 [as="?column?":7, outer=(1-5)] + └── (((a:1 AND b:2) AND c:3) AND d:4) AND e:5 [as="?column?":8, outer=(1-5)] norm expect=NormalizeNestedAnds SELECT (a AND b) AND (c AND (d OR e)) FROM c ---- project - ├── columns: "?column?":7 + ├── columns: "?column?":8 ├── scan c │ └── columns: a:1 b:2 c:3 d:4 e:5 └── projections - └── ((a:1 AND b:2) AND c:3) AND (d:4 OR e:5) [as="?column?":7, outer=(1-5)] + └── ((a:1 AND b:2) AND c:3) AND (d:4 OR e:5) [as="?column?":8, outer=(1-5)] # Already normalized. norm expect-not=NormalizeNestedAnds SELECT a AND b AND c FROM c ---- project - ├── columns: "?column?":7 + ├── columns: "?column?":8 ├── scan c │ └── columns: a:1 b:2 c:3 └── projections - └── (a:1 AND b:2) AND c:3 [as="?column?":7, outer=(1-3)] + └── (a:1 AND b:2) AND c:3 [as="?column?":8, outer=(1-3)] # -------------------------------------------------- # SimplifyTrueAnd + SimplifyAndTrue @@ -54,35 +54,35 @@ norm expect=SimplifyTrueAnd SELECT true AND k=1 AS r FROM a ---- project - ├── columns: r:6!null + ├── columns: r:7!null ├── scan a │ ├── columns: k:1!null │ └── key: (1) └── projections - └── k:1 = 1 [as=r:6, outer=(1)] + └── k:1 = 1 [as=r:7, outer=(1)] norm expect=SimplifyAndTrue SELECT k=1 AND true AS r FROM a ---- project - ├── columns: r:6!null + ├── columns: r:7!null ├── scan a │ ├── columns: k:1!null │ └── key: (1) └── projections - └── k:1 = 1 [as=r:6, outer=(1)] + └── k:1 = 1 [as=r:7, outer=(1)] norm expect=(SimplifyTrueAnd,SimplifyAndTrue) SELECT true AND k=1 AND true AND i=2 AS r FROM a ---- project - ├── columns: r:6 + ├── columns: r:7 ├── scan a │ ├── columns: k:1!null i:2 │ ├── key: (1) │ └── fd: (1)-->(2) └── projections - └── (k:1 = 1) AND (i:2 = 2) [as=r:6, outer=(1,2)] + └── (k:1 = 1) AND (i:2 = 2) [as=r:7, outer=(1,2)] # No conditions left after rule. norm expect=SimplifyTrueAnd @@ -101,31 +101,31 @@ norm expect=SimplifyFalseAnd SELECT false AND s='foo' AS r FROM a ---- project - ├── columns: r:6!null - ├── fd: ()-->(6) + ├── columns: r:7!null + ├── fd: ()-->(7) ├── scan a └── projections - └── false [as=r:6] + └── false [as=r:7] norm expect=SimplifyAndFalse SELECT s='foo' AND false AS r FROM a ---- project - ├── columns: r:6!null - ├── fd: ()-->(6) + ├── columns: r:7!null + ├── fd: ()-->(7) ├── scan a └── projections - └── false [as=r:6] + └── false [as=r:7] norm expect=(SimplifyAndFalse,SimplifyFalseAnd) SELECT k=1 AND false AND (f=3.5 AND false) AS r FROM a ---- project - ├── columns: r:6!null - ├── fd: ()-->(6) + ├── columns: r:7!null + ├── fd: ()-->(7) ├── scan a └── projections - └── false [as=r:6] + └── false [as=r:7] # -------------------------------------------------- # SimplifyTrueOr + SimplifyOrTrue @@ -135,31 +135,31 @@ norm expect=SimplifyTrueOr SELECT true OR s='foo' AS r FROM a ---- project - ├── columns: r:6!null - ├── fd: ()-->(6) + ├── columns: r:7!null + ├── fd: ()-->(7) ├── scan a └── projections - └── true [as=r:6] + └── true [as=r:7] norm expect=SimplifyOrTrue SELECT s='foo' OR true AS r FROM a ---- project - ├── columns: r:6!null - ├── fd: ()-->(6) + ├── columns: r:7!null + ├── fd: ()-->(7) ├── scan a └── projections - └── true [as=r:6] + └── true [as=r:7] norm expect=(SimplifyTrueOr,SimplifyOrTrue) SELECT k=1 OR true OR (true OR f=3.5) AS r FROM a ---- project - ├── columns: r:6!null - ├── fd: ()-->(6) + ├── columns: r:7!null + ├── fd: ()-->(7) ├── scan a └── projections - └── true [as=r:6] + └── true [as=r:7] # -------------------------------------------------- # SimplifyFalseOr + SimplifyOrFalse @@ -169,35 +169,35 @@ norm expect=SimplifyFalseOr SELECT false OR k=1 AS r FROM a ---- project - ├── columns: r:6!null + ├── columns: r:7!null ├── scan a │ ├── columns: k:1!null │ └── key: (1) └── projections - └── k:1 = 1 [as=r:6, outer=(1)] + └── k:1 = 1 [as=r:7, outer=(1)] norm expect=SimplifyOrFalse SELECT k=1 OR false AS r FROM a ---- project - ├── columns: r:6!null + ├── columns: r:7!null ├── scan a │ ├── columns: k:1!null │ └── key: (1) └── projections - └── k:1 = 1 [as=r:6, outer=(1)] + └── k:1 = 1 [as=r:7, outer=(1)] norm expect=(SimplifyFalseOr,SimplifyOrFalse) SELECT (false OR k=1) OR (i=2 OR false) AS r FROM a ---- project - ├── columns: r:6 + ├── columns: r:7 ├── scan a │ ├── columns: k:1!null i:2 │ ├── key: (1) │ └── fd: (1)-->(2) └── projections - └── (k:1 = 1) OR (i:2 = 2) [as=r:6, outer=(1,2)] + └── (k:1 = 1) OR (i:2 = 2) [as=r:7, outer=(1,2)] # No conditions left after rule. norm expect=SimplifyFalseOr @@ -216,12 +216,12 @@ norm expect=(SimplifyOrFalse,SimplifyFalseOr,SimplifyAndTrue) SELECT (k=1 OR false) AND (false OR k=2 OR false) AND true AS r FROM a ---- project - ├── columns: r:6!null + ├── columns: r:7!null ├── scan a │ ├── columns: k:1!null │ └── key: (1) └── projections - └── (k:1 = 1) AND (k:1 = 2) [as=r:6, outer=(1)] + └── (k:1 = 1) AND (k:1 = 2) [as=r:7, outer=(1)] # -------------------------------------------------- # SimplifyRange @@ -249,44 +249,44 @@ norm expect=FoldNullAndOr SELECT null and null AS r FROM a ---- project - ├── columns: r:6 - ├── fd: ()-->(6) + ├── columns: r:7 + ├── fd: ()-->(7) ├── scan a └── projections - └── CAST(NULL AS BOOL) [as=r:6] + └── CAST(NULL AS BOOL) [as=r:7] norm expect=FoldNullAndOr SELECT null or null AS r FROM a ---- project - ├── columns: r:6 - ├── fd: ()-->(6) + ├── columns: r:7 + ├── fd: ()-->(7) ├── scan a └── projections - └── CAST(NULL AS BOOL) [as=r:6] + └── CAST(NULL AS BOOL) [as=r:7] norm expect=FoldNullAndOr SELECT null or (null and null and null) or null AS r FROM a ---- project - ├── columns: r:6 - ├── fd: ()-->(6) + ├── columns: r:7 + ├── fd: ()-->(7) ├── scan a └── projections - └── CAST(NULL AS BOOL) [as=r:6] + └── CAST(NULL AS BOOL) [as=r:7] # Don't fold. norm expect-not=FoldNullAndOr SELECT (null or k=1) AS r, (null and k=1) AS s FROM a ---- project - ├── columns: r:6 s:7 + ├── columns: r:7 s:8 ├── scan a │ ├── columns: k:1!null │ └── key: (1) └── projections - ├── CAST(NULL AS BOOL) OR (k:1 = 1) [as=r:6, outer=(1)] - └── CAST(NULL AS BOOL) AND (k:1 = 1) [as=s:7, outer=(1)] + ├── CAST(NULL AS BOOL) OR (k:1 = 1) [as=r:7, outer=(1)] + └── CAST(NULL AS BOOL) AND (k:1 = 1) [as=s:8, outer=(1)] # -------------------------------------------------- # FoldNotTrue + FoldNotFalse + FoldNotNull @@ -546,51 +546,51 @@ norm expect=(ExtractRedundantConjunct) SELECT b OR b FROM c ---- project - ├── columns: "?column?":7 + ├── columns: "?column?":8 ├── scan c │ └── columns: b:2 └── projections - └── b:2 [as="?column?":7, outer=(2)] + └── b:2 [as="?column?":8, outer=(2)] norm expect=(ExtractRedundantConjunct) SELECT a OR (a AND b) OR (a AND c) FROM c ---- project - ├── columns: "?column?":7 + ├── columns: "?column?":8 ├── scan c │ └── columns: a:1 └── projections - └── a:1 [as="?column?":7, outer=(1)] + └── a:1 [as="?column?":8, outer=(1)] norm expect=(ExtractRedundantConjunct) SELECT (a AND b) OR a OR (a AND c) FROM c ---- project - ├── columns: "?column?":7 + ├── columns: "?column?":8 ├── scan c │ └── columns: a:1 └── projections - └── a:1 [as="?column?":7, outer=(1)] + └── a:1 [as="?column?":8, outer=(1)] norm expect=(ExtractRedundantConjunct) SELECT (a AND b) OR (b AND a) FROM c ---- project - ├── columns: "?column?":7 + ├── columns: "?column?":8 ├── scan c │ └── columns: a:1 b:2 └── projections - └── b:2 AND a:1 [as="?column?":7, outer=(1,2)] + └── b:2 AND a:1 [as="?column?":8, outer=(1,2)] norm expect=(ExtractRedundantConjunct) SELECT (a AND b) OR (c AND a) FROM c ---- project - ├── columns: "?column?":7 + ├── columns: "?column?":8 ├── scan c │ └── columns: a:1 b:2 c:3 └── projections - └── a:1 AND (b:2 OR c:3) [as="?column?":7, outer=(1-3)] + └── a:1 AND (b:2 OR c:3) [as="?column?":8, outer=(1-3)] norm expect=(ExtractRedundantConjunct) SELECT * FROM c WHERE (a AND b) OR (a AND b AND c) OR (b AND a) @@ -663,50 +663,50 @@ norm expect=(ExtractRedundantConjunct) SELECT null or (null and k=1) AS r FROM a ---- project - ├── columns: r:6 - ├── fd: ()-->(6) + ├── columns: r:7 + ├── fd: ()-->(7) ├── scan a └── projections - └── CAST(NULL AS BOOL) [as=r:6] + └── CAST(NULL AS BOOL) [as=r:7] norm expect=(ExtractRedundantConjunct) SELECT (null and k=2) or (null and k=1) AS r FROM a ---- project - ├── columns: r:6 + ├── columns: r:7 ├── scan a │ ├── columns: k:1!null │ └── key: (1) └── projections - └── CAST(NULL AS BOOL) AND ((k:1 = 2) OR (k:1 = 1)) [as=r:6, outer=(1)] + └── CAST(NULL AS BOOL) AND ((k:1 = 2) OR (k:1 = 1)) [as=r:7, outer=(1)] # Check that we don't match non-redundant cases. norm expect-not=(ExtractRedundantConjunct) SELECT a OR b OR b FROM c ---- project - ├── columns: "?column?":7 + ├── columns: "?column?":8 ├── scan c │ └── columns: a:1 b:2 └── projections - └── (a:1 OR b:2) OR b:2 [as="?column?":7, outer=(1,2)] + └── (a:1 OR b:2) OR b:2 [as="?column?":8, outer=(1,2)] norm expect-not=(ExtractRedundantConjunct) SELECT (a AND b) OR (a OR c) FROM c ---- project - ├── columns: "?column?":7 + ├── columns: "?column?":8 ├── scan c │ └── columns: a:1 b:2 c:3 └── projections - └── (a:1 AND b:2) OR (a:1 OR c:3) [as="?column?":7, outer=(1-3)] + └── (a:1 AND b:2) OR (a:1 OR c:3) [as="?column?":8, outer=(1-3)] norm expect-not=(ExtractRedundantConjunct) SELECT (a AND b) OR (NOT a AND c) FROM c ---- project - ├── columns: "?column?":7 + ├── columns: "?column?":8 ├── scan c │ └── columns: a:1 b:2 c:3 └── projections - └── (a:1 AND b:2) OR ((NOT a:1) AND c:3) [as="?column?":7, outer=(1-3)] + └── (a:1 AND b:2) OR ((NOT a:1) AND c:3) [as="?column?":8, outer=(1-3)] diff --git a/pkg/sql/opt/norm/testdata/rules/combo b/pkg/sql/opt/norm/testdata/rules/combo index e9f06721f3ec..fa409561a42e 100644 --- a/pkg/sql/opt/norm/testdata/rules/combo +++ b/pkg/sql/opt/norm/testdata/rules/combo @@ -21,220 +21,220 @@ SELECT s FROM a INNER JOIN xy ON a.k=xy.x AND i+1=10 ---- ================================================================================ Initial expression - Cost: 15503.40 + Cost: 15523.40 ================================================================================ project ├── columns: s:4 ├── immutable └── inner-join (cross) - ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6!null y:7 + ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 x:7!null y:8 xy.crdb_internal_mvcc_timestamp:9 ├── immutable - ├── key: (1,6) - ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (6)-->(7) + ├── key: (1,7) + ├── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6), (7)-->(8,9) ├── scan a - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 │ ├── key: (1) - │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ └── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) ├── scan xy - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 xy.crdb_internal_mvcc_timestamp:9 + │ ├── key: (7) + │ └── fd: (7)-->(8,9) └── filters - └── (k:1 = x:6) AND ((i:2 + 1) = 10) [outer=(1,2,6), immutable, constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── (k:1 = x:7) AND ((i:2 + 1) = 10) [outer=(1,2,7), immutable, constraints=(/1: (/NULL - ]; /7: (/NULL - ])] ================================================================================ NormalizeCmpPlusConst - Cost: 15470.07 + Cost: 15490.07 ================================================================================ project ├── columns: s:4 ├── immutable └── inner-join (cross) - - ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6!null y:7 - + ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:6!null y:7 + - ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 x:7!null y:8 xy.crdb_internal_mvcc_timestamp:9 + + ├── columns: k:1!null i:2!null f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 x:7!null y:8 xy.crdb_internal_mvcc_timestamp:9 ├── immutable - ├── key: (1,6) - ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (6)-->(7) + ├── key: (1,7) + ├── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6), (7)-->(8,9) ├── scan a - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 │ ├── key: (1) - │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ └── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) ├── scan xy - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 xy.crdb_internal_mvcc_timestamp:9 + │ ├── key: (7) + │ └── fd: (7)-->(8,9) └── filters - - └── (k:1 = x:6) AND ((i:2 + 1) = 10) [outer=(1,2,6), immutable, constraints=(/1: (/NULL - ]; /6: (/NULL - ])] - + └── (k:1 = x:6) AND (i:2 = (10 - 1)) [outer=(1,2,6), immutable, constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /6: (/NULL - ])] + - └── (k:1 = x:7) AND ((i:2 + 1) = 10) [outer=(1,2,7), immutable, constraints=(/1: (/NULL - ]; /7: (/NULL - ])] + + └── (k:1 = x:7) AND (i:2 = (10 - 1)) [outer=(1,2,7), immutable, constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /7: (/NULL - ])] ================================================================================ FoldBinary - Cost: 12203.40 + Cost: 12223.40 ================================================================================ project ├── columns: s:4 - ├── immutable └── inner-join (cross) - ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:6!null y:7 + ├── columns: k:1!null i:2!null f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 x:7!null y:8 xy.crdb_internal_mvcc_timestamp:9 - ├── immutable - ├── key: (1,6) - - ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (6)-->(7) - + ├── fd: ()-->(2), (1)-->(3-5), (3,4)~~>(1,5), (6)-->(7) + ├── key: (1,7) + - ├── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6), (7)-->(8,9) + + ├── fd: ()-->(2), (1)-->(3-6), (3,4)~~>(1,5,6), (7)-->(8,9) ├── scan a - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 │ ├── key: (1) - │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ └── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) ├── scan xy - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 xy.crdb_internal_mvcc_timestamp:9 + │ ├── key: (7) + │ └── fd: (7)-->(8,9) └── filters - - └── (k:1 = x:6) AND (i:2 = (10 - 1)) [outer=(1,2,6), immutable, constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /6: (/NULL - ])] - + └── (k:1 = x:6) AND (i:2 = 9) [outer=(1,2,6), constraints=(/1: (/NULL - ]; /2: [/9 - /9]; /6: (/NULL - ]), fd=()-->(2)] + - └── (k:1 = x:7) AND (i:2 = (10 - 1)) [outer=(1,2,7), immutable, constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /7: (/NULL - ])] + + └── (k:1 = x:7) AND (i:2 = 9) [outer=(1,2,7), constraints=(/1: (/NULL - ]; /2: [/9 - /9]; /7: (/NULL - ]), fd=()-->(2)] ================================================================================ SimplifyJoinFilters - Cost: 2180.18 + Cost: 2200.18 ================================================================================ project ├── columns: s:4 - └── inner-join (cross) + └── inner-join (hash) - ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:6!null y:7 - - ├── key: (1,6) - - ├── fd: ()-->(2), (1)-->(3-5), (3,4)~~>(1,5), (6)-->(7) + ├── columns: k:1!null i:2!null f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 x:7!null y:8 xy.crdb_internal_mvcc_timestamp:9 + - ├── key: (1,7) + - ├── fd: ()-->(2), (1)-->(3-6), (3,4)~~>(1,5,6), (7)-->(8,9) + ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - + ├── key: (6) - + ├── fd: ()-->(2), (1)-->(3-5), (3,4)~~>(1,5), (6)-->(7), (1)==(6), (6)==(1) + + ├── key: (7) + + ├── fd: ()-->(2), (1)-->(3-6), (3,4)~~>(1,5,6), (7)-->(8,9), (1)==(7), (7)==(1) ├── scan a - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 │ ├── key: (1) - │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ └── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) ├── scan xy - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 xy.crdb_internal_mvcc_timestamp:9 + │ ├── key: (7) + │ └── fd: (7)-->(8,9) └── filters - - └── (k:1 = x:6) AND (i:2 = 9) [outer=(1,2,6), constraints=(/1: (/NULL - ]; /2: [/9 - /9]; /6: (/NULL - ]), fd=()-->(2)] - + ├── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + - └── (k:1 = x:7) AND (i:2 = 9) [outer=(1,2,7), constraints=(/1: (/NULL - ]; /2: [/9 - /9]; /7: (/NULL - ]), fd=()-->(2)] + + ├── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + └── i:2 = 9 [outer=(2), constraints=(/2: [/9 - /9]; tight), fd=()-->(2)] ================================================================================ PushFilterIntoJoinLeft - Cost: 2167.91 + Cost: 2187.91 ================================================================================ project ├── columns: s:4 └── inner-join (hash) - ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:6!null y:7 + ├── columns: k:1!null i:2!null f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 x:7!null y:8 xy.crdb_internal_mvcc_timestamp:9 ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (6) - ├── fd: ()-->(2), (1)-->(3-5), (3,4)~~>(1,5), (6)-->(7), (1)==(6), (6)==(1) + ├── key: (7) + ├── fd: ()-->(2), (1)-->(3-6), (3,4)~~>(1,5,6), (7)-->(8,9), (1)==(7), (7)==(1) - ├── scan a - - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 + - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + ├── select - + │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 + + │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 │ ├── key: (1) - - │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) - + │ ├── fd: ()-->(2), (1)-->(3-5), (3,4)~~>(1,5) + - │ └── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) + + │ ├── fd: ()-->(2), (1)-->(3-6), (3,4)~~>(1,5,6) + │ ├── scan a - + │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 + + │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + │ │ ├── key: (1) - + │ │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + + │ │ └── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) + │ └── filters + │ └── i:2 = 9 [outer=(2), constraints=(/2: [/9 - /9]; tight), fd=()-->(2)] ├── scan xy - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 xy.crdb_internal_mvcc_timestamp:9 + │ ├── key: (7) + │ └── fd: (7)-->(8,9) └── filters - - ├── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + - ├── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] - └── i:2 = 9 [outer=(2), constraints=(/2: [/9 - /9]; tight), fd=()-->(2)] - + └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + + └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] ================================================================================ PruneJoinLeftCols - Cost: 2168.03 + Cost: 2188.03 ================================================================================ project ├── columns: s:4 └── inner-join (hash) - - ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:6!null y:7 - + ├── columns: k:1!null i:2!null s:4 x:6!null y:7 + - ├── columns: k:1!null i:2!null f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 x:7!null y:8 xy.crdb_internal_mvcc_timestamp:9 + + ├── columns: k:1!null i:2!null s:4 x:7!null y:8 xy.crdb_internal_mvcc_timestamp:9 ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (6) - - ├── fd: ()-->(2), (1)-->(3-5), (3,4)~~>(1,5), (6)-->(7), (1)==(6), (6)==(1) + ├── key: (7) + - ├── fd: ()-->(2), (1)-->(3-6), (3,4)~~>(1,5,6), (7)-->(8,9), (1)==(7), (7)==(1) - ├── select - - │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 - + ├── fd: ()-->(2), (1)-->(4), (6)-->(7), (1)==(6), (6)==(1) + - │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + + ├── fd: ()-->(2), (1)-->(4), (7)-->(8,9), (1)==(7), (7)==(1) + ├── project + │ ├── columns: k:1!null i:2!null s:4 │ ├── key: (1) - - │ ├── fd: ()-->(2), (1)-->(3-5), (3,4)~~>(1,5) + - │ ├── fd: ()-->(2), (1)-->(3-6), (3,4)~~>(1,5,6) - │ ├── scan a - - │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 + - │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 - │ │ ├── key: (1) - - │ │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + - │ │ └── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) - │ └── filters - │ └── i:2 = 9 [outer=(2), constraints=(/2: [/9 - /9]; tight), fd=()-->(2)] + │ ├── fd: ()-->(2), (1)-->(4) + │ └── select - + │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 + + │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + │ ├── key: (1) - + │ ├── fd: ()-->(2), (1)-->(3-5), (3,4)~~>(1,5) + + │ ├── fd: ()-->(2), (1)-->(3-6), (3,4)~~>(1,5,6) + │ ├── scan a - + │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 + + │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + │ │ ├── key: (1) - + │ │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + + │ │ └── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) + │ └── filters + │ └── i:2 = 9 [outer=(2), constraints=(/2: [/9 - /9]; tight), fd=()-->(2)] ├── scan xy - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 xy.crdb_internal_mvcc_timestamp:9 + │ ├── key: (7) + │ └── fd: (7)-->(8,9) └── filters - └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] ================================================================================ PruneSelectCols - Cost: 2148.03 + Cost: 2158.03 ================================================================================ project ├── columns: s:4 └── inner-join (hash) - ├── columns: k:1!null i:2!null s:4 x:6!null y:7 + ├── columns: k:1!null i:2!null s:4 x:7!null y:8 xy.crdb_internal_mvcc_timestamp:9 ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (6) - ├── fd: ()-->(2), (1)-->(4), (6)-->(7), (1)==(6), (6)==(1) + ├── key: (7) + ├── fd: ()-->(2), (1)-->(4), (7)-->(8,9), (1)==(7), (7)==(1) ├── project │ ├── columns: k:1!null i:2!null s:4 │ ├── key: (1) │ ├── fd: ()-->(2), (1)-->(4) │ └── select - - │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 + - │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + │ ├── columns: k:1!null i:2!null s:4 │ ├── key: (1) - - │ ├── fd: ()-->(2), (1)-->(3-5), (3,4)~~>(1,5) + - │ ├── fd: ()-->(2), (1)-->(3-6), (3,4)~~>(1,5,6) + │ ├── fd: ()-->(2), (1)-->(4) │ ├── scan a - - │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 + - │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + │ │ ├── columns: k:1!null i:2 s:4 │ │ ├── key: (1) - - │ │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + - │ │ └── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) + │ │ └── fd: (1)-->(2,4) │ └── filters │ └── i:2 = 9 [outer=(2), constraints=(/2: [/9 - /9]; tight), fd=()-->(2)] ├── scan xy - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 xy.crdb_internal_mvcc_timestamp:9 + │ ├── key: (7) + │ └── fd: (7)-->(8,9) └── filters - └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] ================================================================================ EliminateProject - Cost: 2147.91 + Cost: 2157.91 ================================================================================ project ├── columns: s:4 └── inner-join (hash) - ├── columns: k:1!null i:2!null s:4 x:6!null y:7 + ├── columns: k:1!null i:2!null s:4 x:7!null y:8 xy.crdb_internal_mvcc_timestamp:9 ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (6) - ├── fd: ()-->(2), (1)-->(4), (6)-->(7), (1)==(6), (6)==(1) + ├── key: (7) + ├── fd: ()-->(2), (1)-->(4), (7)-->(8,9), (1)==(7), (7)==(1) - ├── project + ├── select │ ├── columns: k:1!null i:2!null s:4 @@ -257,11 +257,11 @@ EliminateProject + │ └── filters + │ └── i:2 = 9 [outer=(2), constraints=(/2: [/9 - /9]; tight), fd=()-->(2)] ├── scan xy - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 xy.crdb_internal_mvcc_timestamp:9 + │ ├── key: (7) + │ └── fd: (7)-->(8,9) └── filters - └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] ================================================================================ PruneJoinRightCols Cost: 2137.91 @@ -269,12 +269,12 @@ PruneJoinRightCols project ├── columns: s:4 └── inner-join (hash) - - ├── columns: k:1!null i:2!null s:4 x:6!null y:7 - + ├── columns: k:1!null i:2!null s:4 x:6!null + - ├── columns: k:1!null i:2!null s:4 x:7!null y:8 xy.crdb_internal_mvcc_timestamp:9 + + ├── columns: k:1!null i:2!null s:4 x:7!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (6) - - ├── fd: ()-->(2), (1)-->(4), (6)-->(7), (1)==(6), (6)==(1) - + ├── fd: ()-->(2), (1)-->(4), (1)==(6), (6)==(1) + ├── key: (7) + - ├── fd: ()-->(2), (1)-->(4), (7)-->(8,9), (1)==(7), (7)==(1) + + ├── fd: ()-->(2), (1)-->(4), (1)==(7), (7)==(1) ├── select │ ├── columns: k:1!null i:2!null s:4 │ ├── key: (1) @@ -286,13 +286,13 @@ PruneJoinRightCols │ └── filters │ └── i:2 = 9 [outer=(2), constraints=(/2: [/9 - /9]; tight), fd=()-->(2)] ├── scan xy - - │ ├── columns: x:6!null y:7 - - │ ├── key: (6) - - │ └── fd: (6)-->(7) - + │ ├── columns: x:6!null - + │ └── key: (6) + - │ ├── columns: x:7!null y:8 xy.crdb_internal_mvcc_timestamp:9 + - │ ├── key: (7) + - │ └── fd: (7)-->(8,9) + + │ ├── columns: x:7!null + + │ └── key: (7) └── filters - └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] -------------------------------------------------------------------------------- GenerateIndexScans (no changes) -------------------------------------------------------------------------------- @@ -315,13 +315,13 @@ CommuteJoin project ├── columns: s:4 └── inner-join (hash) - ├── columns: k:1!null i:2!null s:4 x:6!null + ├── columns: k:1!null i:2!null s:4 x:7!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (6) - ├── fd: ()-->(2), (1)-->(4), (1)==(6), (6)==(1) + ├── key: (7) + ├── fd: ()-->(2), (1)-->(4), (1)==(7), (7)==(1) + ├── scan xy - + │ ├── columns: x:6!null - + │ └── key: (6) + + │ ├── columns: x:7!null + + │ └── key: (7) ├── select │ ├── columns: k:1!null i:2!null s:4 │ ├── key: (1) @@ -333,10 +333,10 @@ CommuteJoin │ └── filters │ └── i:2 = 9 [outer=(2), constraints=(/2: [/9 - /9]; tight), fd=()-->(2)] - ├── scan xy - - │ ├── columns: x:6!null - - │ └── key: (6) + - │ ├── columns: x:7!null + - │ └── key: (7) └── filters - └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] ================================================================================ GenerateMergeJoins Cost: 2130.38 @@ -345,15 +345,15 @@ GenerateMergeJoins ├── columns: s:4 - └── inner-join (hash) + └── inner-join (merge) - ├── columns: k:1!null i:2!null s:4 x:6!null + ├── columns: k:1!null i:2!null s:4 x:7!null - ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) + ├── left ordering: +1 - + ├── right ordering: +6 - ├── key: (6) - ├── fd: ()-->(2), (1)-->(4), (1)==(6), (6)==(1) + + ├── right ordering: +7 + ├── key: (7) + ├── fd: ()-->(2), (1)-->(4), (1)==(7), (7)==(1) - ├── scan xy - - │ ├── columns: x:6!null - - │ └── key: (6) + - │ ├── columns: x:7!null + - │ └── key: (7) ├── select │ ├── columns: k:1!null i:2!null s:4 │ ├── key: (1) @@ -368,11 +368,11 @@ GenerateMergeJoins │ └── filters │ └── i:2 = 9 [outer=(2), constraints=(/2: [/9 - /9]; tight), fd=()-->(2)] - └── filters - - └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + - └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + ├── scan xy - + │ ├── columns: x:6!null - + │ ├── key: (6) - + │ └── ordering: +6 + + │ ├── columns: x:7!null + + │ ├── key: (7) + + │ └── ordering: +7 + └── filters (true) ================================================================================ GenerateLookupJoins @@ -382,13 +382,13 @@ GenerateLookupJoins ├── columns: s:4 - └── inner-join (merge) + └── inner-join (lookup xy) - ├── columns: k:1!null i:2!null s:4 x:6!null + ├── columns: k:1!null i:2!null s:4 x:7!null - ├── left ordering: +1 - - ├── right ordering: +6 - + ├── key columns: [1] = [6] + - ├── right ordering: +7 + + ├── key columns: [1] = [7] + ├── lookup columns are key - ├── key: (6) - ├── fd: ()-->(2), (1)-->(4), (1)==(6), (6)==(1) + ├── key: (7) + ├── fd: ()-->(2), (1)-->(4), (1)==(7), (7)==(1) ├── select │ ├── columns: k:1!null i:2!null s:4 │ ├── key: (1) @@ -403,9 +403,9 @@ GenerateLookupJoins │ └── filters │ └── i:2 = 9 [outer=(2), constraints=(/2: [/9 - /9]; tight), fd=()-->(2)] - ├── scan xy - - │ ├── columns: x:6!null - - │ ├── key: (6) - - │ └── ordering: +6 + - │ ├── columns: x:7!null + - │ ├── key: (7) + - │ └── ordering: +7 └── filters (true) -------------------------------------------------------------------------------- CommuteJoin (no changes) @@ -417,17 +417,17 @@ GenerateMergeJoins (higher cost) ├── columns: s:4 - └── inner-join (lookup xy) + └── inner-join (merge) - ├── columns: k:1!null i:2!null s:4 x:6!null - - ├── key columns: [1] = [6] + ├── columns: k:1!null i:2!null s:4 x:7!null + - ├── key columns: [1] = [7] - ├── lookup columns are key - + ├── left ordering: +6 + + ├── left ordering: +7 + ├── right ordering: +1 - ├── key: (6) - ├── fd: ()-->(2), (1)-->(4), (1)==(6), (6)==(1) + ├── key: (7) + ├── fd: ()-->(2), (1)-->(4), (1)==(7), (7)==(1) + ├── scan xy - + │ ├── columns: x:6!null - + │ ├── key: (6) - + │ └── ordering: +6 + + │ ├── columns: x:7!null + + │ ├── key: (7) + + │ └── ordering: +7 ├── select │ ├── columns: k:1!null i:2!null s:4 │ ├── key: (1) @@ -449,17 +449,17 @@ GenerateLookupJoinsWithFilter (higher cost) ├── columns: s:4 - └── inner-join (merge) + └── inner-join (lookup a) - ├── columns: k:1!null i:2!null s:4 x:6!null - - ├── left ordering: +6 + ├── columns: k:1!null i:2!null s:4 x:7!null + - ├── left ordering: +7 - ├── right ordering: +1 - + ├── key columns: [6] = [1] + + ├── key columns: [7] = [1] + ├── lookup columns are key - ├── key: (6) - ├── fd: ()-->(2), (1)-->(4), (1)==(6), (6)==(1) + ├── key: (7) + ├── fd: ()-->(2), (1)-->(4), (1)==(7), (7)==(1) ├── scan xy - │ ├── columns: x:6!null - - │ ├── key: (6) - - │ └── ordering: +6 + │ ├── columns: x:7!null + - │ ├── key: (7) + - │ └── ordering: +7 - ├── select - │ ├── columns: k:1!null i:2!null s:4 - │ ├── key: (1) @@ -473,7 +473,7 @@ GenerateLookupJoinsWithFilter (higher cost) - │ └── filters - │ └── i:2 = 9 [outer=(2), constraints=(/2: [/9 - /9]; tight), fd=()-->(2)] - └── filters (true) - + │ └── key: (6) + + │ └── key: (7) + └── filters + └── i:2 = 9 [outer=(2), constraints=(/2: [/9 - /9]; tight), fd=()-->(2)] ================================================================================ @@ -483,11 +483,11 @@ Final best expression project ├── columns: s:4 └── inner-join (lookup xy) - ├── columns: k:1!null i:2!null s:4 x:6!null - ├── key columns: [1] = [6] + ├── columns: k:1!null i:2!null s:4 x:7!null + ├── key columns: [1] = [7] ├── lookup columns are key - ├── key: (6) - ├── fd: ()-->(2), (1)-->(4), (1)==(6), (6)==(1) + ├── key: (7) + ├── fd: ()-->(2), (1)-->(4), (1)==(7), (7)==(1) ├── select │ ├── columns: k:1!null i:2!null s:4 │ ├── key: (1) @@ -506,38 +506,38 @@ SELECT s, k FROM a WHERE s='foo' AND f>100 ---- ================================================================================ Initial expression - Cost: 1110.14 + Cost: 1120.14 ================================================================================ project ├── columns: s:4!null k:1!null ├── key: (1) ├── fd: ()-->(4) └── select - ├── columns: k:1!null i:2 f:3!null s:4!null j:5 + ├── columns: k:1!null i:2 f:3!null s:4!null j:5 crdb_internal_mvcc_timestamp:6 ├── key: (1) - ├── fd: ()-->(4), (1)-->(2,3,5), (3)-->(1,2,5) + ├── fd: ()-->(4), (1)-->(2,3,5,6), (3)-->(1,2,5,6) ├── scan a - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 crdb_internal_mvcc_timestamp:6 │ ├── key: (1) - │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ └── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) └── filters └── (s:4 = 'foo') AND (f:3 > 100.0) [outer=(3,4), constraints=(/3: [/100.00000000000001 - ]; /4: [/'foo' - /'foo']; tight), fd=()-->(4)] ================================================================================ SimplifySelectFilters - Cost: 1110.15 + Cost: 1120.15 ================================================================================ project ├── columns: s:4!null k:1!null ├── key: (1) ├── fd: ()-->(4) └── select - ├── columns: k:1!null i:2 f:3!null s:4!null j:5 + ├── columns: k:1!null i:2 f:3!null s:4!null j:5 crdb_internal_mvcc_timestamp:6 ├── key: (1) - ├── fd: ()-->(4), (1)-->(2,3,5), (3)-->(1,2,5) + ├── fd: ()-->(4), (1)-->(2,3,5,6), (3)-->(1,2,5,6) ├── scan a - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 crdb_internal_mvcc_timestamp:6 │ ├── key: (1) - │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + │ └── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) └── filters - └── (s:4 = 'foo') AND (f:3 > 100.0) [outer=(3,4), constraints=(/3: [/100.00000000000001 - ]; /4: [/'foo' - /'foo']; tight), fd=()-->(4)] + ├── s:4 = 'foo' [outer=(4), constraints=(/4: [/'foo' - /'foo']; tight), fd=()-->(4)] @@ -551,16 +551,16 @@ PruneSelectCols ├── key: (1) ├── fd: ()-->(4) └── select - - ├── columns: k:1!null i:2 f:3!null s:4!null j:5 + - ├── columns: k:1!null i:2 f:3!null s:4!null j:5 crdb_internal_mvcc_timestamp:6 + ├── columns: k:1!null f:3!null s:4!null ├── key: (1) - - ├── fd: ()-->(4), (1)-->(2,3,5), (3)-->(1,2,5) + - ├── fd: ()-->(4), (1)-->(2,3,5,6), (3)-->(1,2,5,6) + ├── fd: ()-->(4), (1)-->(3), (3)-->(1) ├── scan a - - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 + - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 crdb_internal_mvcc_timestamp:6 + │ ├── columns: k:1!null f:3 s:4 │ ├── key: (1) - - │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + - │ └── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) + │ └── fd: (1)-->(3,4), (3,4)~~>(1) └── filters ├── s:4 = 'foo' [outer=(4), constraints=(/4: [/'foo' - /'foo']; tight), fd=()-->(4)] @@ -633,187 +633,350 @@ SELECT * FROM a WHERE EXISTS(SELECT * FROM xy WHERE y=i) ---- ================================================================================ Initial expression - Cost: 2160.08 + Cost: 2193.33 ================================================================================ - select + project ├── columns: k:1!null i:2 f:3 s:4 j:5 ├── key: (1) ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) - ├── scan a - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 - │ ├── key: (1) - │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) - └── filters - └── exists [outer=(2), correlated-subquery] - └── select - ├── columns: x:6!null y:7!null - ├── outer: (2) - ├── key: (6) - ├── fd: ()-->(7) - ├── scan xy - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) - └── filters - └── y:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] + └── select + ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + ├── key: (1) + ├── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) + ├── scan a + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + │ ├── key: (1) + │ └── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) + └── filters + └── exists [outer=(2), correlated-subquery] + └── project + ├── columns: x:7!null y:8!null + ├── outer: (2) + ├── key: (7) + ├── fd: ()-->(8) + └── select + ├── columns: x:7!null y:8!null xy.crdb_internal_mvcc_timestamp:9 + ├── outer: (2) + ├── key: (7) + ├── fd: ()-->(8), (7)-->(9) + ├── scan xy + │ ├── columns: x:7!null y:8 xy.crdb_internal_mvcc_timestamp:9 + │ ├── key: (7) + │ └── fd: (7)-->(8,9) + └── filters + └── y:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] +================================================================================ +PruneSelectCols + Cost: 2183.33 +================================================================================ + project + ├── columns: k:1!null i:2 f:3 s:4 j:5 + ├── key: (1) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + └── select + ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + ├── key: (1) + ├── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) + ├── scan a + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + │ ├── key: (1) + │ └── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) + └── filters + └── exists [outer=(2), correlated-subquery] + └── project + ├── columns: x:7!null y:8!null + ├── outer: (2) + ├── key: (7) + ├── fd: ()-->(8) + └── select + - ├── columns: x:7!null y:8!null xy.crdb_internal_mvcc_timestamp:9 + + ├── columns: x:7!null y:8!null + ├── outer: (2) + ├── key: (7) + - ├── fd: ()-->(8), (7)-->(9) + + ├── fd: ()-->(8) + ├── scan xy + - │ ├── columns: x:7!null y:8 xy.crdb_internal_mvcc_timestamp:9 + + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + - │ └── fd: (7)-->(8,9) + + │ └── fd: (7)-->(8) + └── filters + └── y:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] +================================================================================ +EliminateProject + Cost: 2173.42 +================================================================================ + project + ├── columns: k:1!null i:2 f:3 s:4 j:5 + ├── key: (1) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + └── select + ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + ├── key: (1) + ├── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) + ├── scan a + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + │ ├── key: (1) + │ └── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) + └── filters + └── exists [outer=(2), correlated-subquery] + - └── project + + └── select + ├── columns: x:7!null y:8!null + ├── outer: (2) + ├── key: (7) + ├── fd: ()-->(8) + - └── select + - ├── columns: x:7!null y:8!null + - ├── outer: (2) + - ├── key: (7) + - ├── fd: ()-->(8) + - ├── scan xy + - │ ├── columns: x:7!null y:8 + - │ ├── key: (7) + - │ └── fd: (7)-->(8) + - └── filters + - └── y:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] + + ├── scan xy + + │ ├── columns: x:7!null y:8 + + │ ├── key: (7) + + │ └── fd: (7)-->(8) + + └── filters + + └── y:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] ================================================================================ HoistSelectExists - Cost: 12089.91 + Cost: 12109.92 ================================================================================ - select + project ├── columns: k:1!null i:2 f:3 s:4 j:5 ├── key: (1) ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) - - ├── scan a - + ├── semi-join-apply - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 - │ ├── key: (1) - - │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) - - └── filters - - └── exists [outer=(2), correlated-subquery] - - └── select - - ├── columns: x:6!null y:7!null - - ├── outer: (2) - - ├── key: (6) - - ├── fd: ()-->(7) - - ├── scan xy - - │ ├── columns: x:6!null y:7 - - │ ├── key: (6) - - │ └── fd: (6)-->(7) - - └── filters - - └── y:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - + │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) - + │ ├── scan a - + │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 - + │ │ ├── key: (1) - + │ │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) - + │ ├── select - + │ │ ├── columns: x:6!null y:7!null - + │ │ ├── outer: (2) - + │ │ ├── key: (6) - + │ │ ├── fd: ()-->(7) - + │ │ ├── scan xy - + │ │ │ ├── columns: x:6!null y:7 - + │ │ │ ├── key: (6) - + │ │ │ └── fd: (6)-->(7) - + │ │ └── filters - + │ │ └── y:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - + │ └── filters (true) - + └── filters (true) + └── select + ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + ├── key: (1) + ├── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) + - ├── scan a + + ├── semi-join-apply + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + │ ├── key: (1) + - │ └── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) + - └── filters + - └── exists [outer=(2), correlated-subquery] + - └── select + - ├── columns: x:7!null y:8!null + - ├── outer: (2) + - ├── key: (7) + - ├── fd: ()-->(8) + - ├── scan xy + - │ ├── columns: x:7!null y:8 + - │ ├── key: (7) + - │ └── fd: (7)-->(8) + - └── filters + - └── y:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] + + │ ├── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) + + │ ├── scan a + + │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + + │ │ ├── key: (1) + + │ │ └── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) + + │ ├── select + + │ │ ├── columns: x:7!null y:8!null + + │ │ ├── outer: (2) + + │ │ ├── key: (7) + + │ │ ├── fd: ()-->(8) + + │ │ ├── scan xy + + │ │ │ ├── columns: x:7!null y:8 + + │ │ │ ├── key: (7) + + │ │ │ └── fd: (7)-->(8) + + │ │ └── filters + + │ │ └── y:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] + + │ └── filters (true) + + └── filters (true) ================================================================================ TryDecorrelateSelect - Cost: 2279.97 + Cost: 2299.88 ================================================================================ - select + project - ├── columns: k:1!null i:2 f:3 s:4 j:5 + ├── columns: k:1!null i:2!null f:3 s:4 j:5 ├── key: (1) ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) - ├── semi-join-apply - - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 - + │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 - │ ├── key: (1) - │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) - │ ├── scan a - │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 - │ │ ├── key: (1) - │ │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) - - │ ├── select - - │ │ ├── columns: x:6!null y:7!null - - │ │ ├── outer: (2) - + │ ├── scan xy - + │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - - │ │ ├── fd: ()-->(7) - - │ │ ├── scan xy - - │ │ │ ├── columns: x:6!null y:7 - - │ │ │ ├── key: (6) - - │ │ │ └── fd: (6)-->(7) - - │ │ └── filters - - │ │ └── y:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - - │ └── filters (true) - + │ │ └── fd: (6)-->(7) - + │ └── filters - + │ └── y:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - └── filters (true) + └── select + - ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + + ├── columns: k:1!null i:2!null f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + ├── key: (1) + ├── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) + ├── semi-join-apply + - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + + │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + │ ├── key: (1) + │ ├── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) + │ ├── scan a + │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + │ │ ├── key: (1) + │ │ └── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) + - │ ├── select + - │ │ ├── columns: x:7!null y:8!null + - │ │ ├── outer: (2) + + │ ├── scan xy + + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + - │ │ ├── fd: ()-->(8) + - │ │ ├── scan xy + - │ │ │ ├── columns: x:7!null y:8 + - │ │ │ ├── key: (7) + - │ │ │ └── fd: (7)-->(8) + - │ │ └── filters + - │ │ └── y:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] + - │ └── filters (true) + + │ │ └── fd: (7)-->(8) + + │ └── filters + + │ └── y:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] + └── filters (true) ================================================================================ DecorrelateJoin - Cost: 2278.08 + Cost: 2298.09 ================================================================================ - select + project - ├── columns: k:1!null i:2!null f:3 s:4 j:5 + ├── columns: k:1!null i:2 f:3 s:4 j:5 ├── key: (1) ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) - - ├── semi-join-apply - - │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 - + ├── semi-join (hash) - + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 - │ ├── key: (1) - │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) - │ ├── scan a - │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 - │ │ ├── key: (1) - │ │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) - │ ├── scan xy - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) - │ └── filters - │ └── y:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - └── filters (true) + └── select + - ├── columns: k:1!null i:2!null f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + + ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + ├── key: (1) + ├── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) + - ├── semi-join-apply + - │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + + ├── semi-join (hash) + + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + │ ├── key: (1) + │ ├── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) + │ ├── scan a + │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + │ │ ├── key: (1) + │ │ └── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) + │ ├── scan xy + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) + │ └── filters + │ └── y:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] + └── filters (true) ================================================================================ PruneSemiAntiJoinRightCols - Cost: 2268.08 + Cost: 2288.09 ================================================================================ - select + project ├── columns: k:1!null i:2 f:3 s:4 j:5 ├── key: (1) ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) - ├── semi-join (hash) - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 - │ ├── key: (1) - │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) - │ ├── scan a - │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 - │ │ ├── key: (1) - │ │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) - │ ├── scan xy - - │ │ ├── columns: x:6!null y:7 - - │ │ ├── key: (6) - - │ │ └── fd: (6)-->(7) - + │ │ └── columns: y:7 - │ └── filters - │ └── y:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - └── filters (true) + └── select + ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + ├── key: (1) + ├── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) + ├── semi-join (hash) + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + │ ├── key: (1) + │ ├── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) + │ ├── scan a + │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + │ │ ├── key: (1) + │ │ └── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) + │ ├── scan xy + - │ │ ├── columns: x:7!null y:8 + - │ │ ├── key: (7) + - │ │ └── fd: (7)-->(8) + + │ │ └── columns: y:8 + │ └── filters + │ └── y:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] + └── filters (true) ================================================================================ EliminateSelect + Cost: 2278.08 +================================================================================ + project + ├── columns: k:1!null i:2 f:3 s:4 j:5 + ├── key: (1) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + - └── select + + └── semi-join (hash) + ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + ├── key: (1) + ├── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) + - ├── semi-join (hash) + + ├── scan a + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + │ ├── key: (1) + - │ ├── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) + - │ ├── scan a + - │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + - │ │ ├── key: (1) + - │ │ └── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) + - │ ├── scan xy + - │ │ └── columns: y:8 + - │ └── filters + - │ └── y:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] + - └── filters (true) + + │ └── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) + + ├── scan xy + + │ └── columns: y:8 + + └── filters + + └── y:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] +================================================================================ +PruneJoinLeftCols + Cost: 2268.08 +================================================================================ + project + ├── columns: k:1!null i:2 f:3 s:4 j:5 + ├── key: (1) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + └── semi-join (hash) + - ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + + ├── columns: k:1!null i:2 f:3 s:4 j:5 + ├── key: (1) + - ├── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) + + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + ├── scan a + - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 a.crdb_internal_mvcc_timestamp:6 + + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 + │ ├── key: (1) + - │ └── fd: (1)-->(2-6), (3,4)~~>(1,2,5,6) + + │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + ├── scan xy + │ └── columns: y:8 + └── filters + └── y:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] +================================================================================ +EliminateProject Cost: 2258.07 ================================================================================ - -select + -project +semi-join (hash) ├── columns: k:1!null i:2 f:3 s:4 j:5 ├── key: (1) ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) - - ├── semi-join (hash) + - └── semi-join (hash) + - ├── columns: k:1!null i:2 f:3 s:4 j:5 + - ├── key: (1) + - ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + - ├── scan a + - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 + - │ ├── key: (1) + - │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + - ├── scan xy + - │ └── columns: y:8 + - └── filters + - └── y:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] + ├── scan a - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 - │ ├── key: (1) - - │ ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) - - │ ├── scan a - - │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 - - │ │ ├── key: (1) - - │ │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) - - │ ├── scan xy - - │ │ └── columns: y:7 - - │ └── filters - - │ └── y:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - - └── filters (true) + + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 + + │ ├── key: (1) + │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + ├── scan xy - + │ └── columns: y:7 + + │ └── columns: y:8 + └── filters - + └── y:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] + + └── y:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] -------------------------------------------------------------------------------- GenerateIndexScans (no changes) -------------------------------------------------------------------------------- @@ -834,26 +997,26 @@ CommuteSemiJoin - │ ├── key: (1) - │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) - ├── scan xy - - │ └── columns: y:7 + - │ └── columns: y:8 - └── filters - - └── y:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] + - └── y:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] + └── inner-join (hash) - + ├── columns: k:1!null i:2!null f:3 s:4 j:5 y:7!null + + ├── columns: k:1!null i:2!null f:3 s:4 j:5 y:8!null + ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) + ├── key: (1) - + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (2)==(7), (7)==(2) + + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (2)==(8), (8)==(2) + ├── scan a + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 + │ ├── key: (1) + │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + ├── distinct-on - + │ ├── columns: y:7 - + │ ├── grouping columns: y:7 - + │ ├── key: (7) + + │ ├── columns: y:8 + + │ ├── grouping columns: y:8 + + │ ├── key: (8) + │ └── scan xy - + │ └── columns: y:7 + + │ └── columns: y:8 + └── filters - + └── y:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] + + └── y:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] -------------------------------------------------------------------------------- GenerateMergeJoins (no changes) -------------------------------------------------------------------------------- @@ -887,22 +1050,22 @@ Final best expression ├── key: (1) ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) └── inner-join (hash) - ├── columns: k:1!null i:2!null f:3 s:4 j:5 y:7!null + ├── columns: k:1!null i:2!null f:3 s:4 j:5 y:8!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) ├── key: (1) - ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (2)==(7), (7)==(2) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (2)==(8), (8)==(2) ├── scan a │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) ├── distinct-on - │ ├── columns: y:7 - │ ├── grouping columns: y:7 - │ ├── key: (7) + │ ├── columns: y:8 + │ ├── grouping columns: y:8 + │ ├── key: (8) │ └── scan xy - │ └── columns: y:7 + │ └── columns: y:8 └── filters - └── y:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] + └── y:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] # Decorrelation pattern using ANY function. optsteps @@ -910,1801 +1073,1801 @@ SELECT 5=ANY(SELECT i FROM a WHERE k=x) AS r FROM xy ---- ================================================================================ Initial expression - Cost: 2170.09 + Cost: 2190.09 ================================================================================ project - ├── columns: r:8 + ├── columns: r:10 ├── scan xy - │ ├── columns: x:1!null y:2 + │ ├── columns: x:1!null y:2 xy.crdb_internal_mvcc_timestamp:3 │ ├── key: (1) - │ └── fd: (1)-->(2) + │ └── fd: (1)-->(2,3) └── projections - └── any: eq [as=r:8, outer=(1), correlated-subquery] + └── any: eq [as=r:10, outer=(1), correlated-subquery] ├── project - │ ├── columns: i:4 + │ ├── columns: i:5 │ ├── outer: (1) │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(4) + │ ├── fd: ()-->(5) │ └── select - │ ├── columns: k:3!null i:4 f:5 s:6 j:7 + │ ├── columns: k:4!null i:5 f:6 s:7 j:8 a.crdb_internal_mvcc_timestamp:9 │ ├── outer: (1) │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(3-7) + │ ├── fd: ()-->(4-9) │ ├── scan a - │ │ ├── columns: k:3!null i:4 f:5 s:6 j:7 - │ │ ├── key: (3) - │ │ └── fd: (3)-->(4-7), (5,6)~~>(3,4,7) + │ │ ├── columns: k:4!null i:5 f:6 s:7 j:8 a.crdb_internal_mvcc_timestamp:9 + │ │ ├── key: (4) + │ │ └── fd: (4)-->(5-9), (6,7)~~>(4,5,8,9) │ └── filters - │ └── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + │ └── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] └── 5 ================================================================================ PruneSelectCols - Cost: 2140.09 + Cost: 2150.09 ================================================================================ project - ├── columns: r:8 + ├── columns: r:10 ├── scan xy - │ ├── columns: x:1!null y:2 + │ ├── columns: x:1!null y:2 xy.crdb_internal_mvcc_timestamp:3 │ ├── key: (1) - │ └── fd: (1)-->(2) + │ └── fd: (1)-->(2,3) └── projections - └── any: eq [as=r:8, outer=(1), correlated-subquery] + └── any: eq [as=r:10, outer=(1), correlated-subquery] ├── project - │ ├── columns: i:4 + │ ├── columns: i:5 │ ├── outer: (1) │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(4) + │ ├── fd: ()-->(5) │ └── select - - │ ├── columns: k:3!null i:4 f:5 s:6 j:7 - + │ ├── columns: k:3!null i:4 + - │ ├── columns: k:4!null i:5 f:6 s:7 j:8 a.crdb_internal_mvcc_timestamp:9 + + │ ├── columns: k:4!null i:5 │ ├── outer: (1) │ ├── cardinality: [0 - 1] │ ├── key: () - - │ ├── fd: ()-->(3-7) - + │ ├── fd: ()-->(3,4) + - │ ├── fd: ()-->(4-9) + + │ ├── fd: ()-->(4,5) │ ├── scan a - - │ │ ├── columns: k:3!null i:4 f:5 s:6 j:7 - + │ │ ├── columns: k:3!null i:4 - │ │ ├── key: (3) - - │ │ └── fd: (3)-->(4-7), (5,6)~~>(3,4,7) - + │ │ └── fd: (3)-->(4) + - │ │ ├── columns: k:4!null i:5 f:6 s:7 j:8 a.crdb_internal_mvcc_timestamp:9 + + │ │ ├── columns: k:4!null i:5 + │ │ ├── key: (4) + - │ │ └── fd: (4)-->(5-9), (6,7)~~>(4,5,8,9) + + │ │ └── fd: (4)-->(5) │ └── filters - │ └── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + │ └── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] └── 5 ================================================================================ PruneScanCols Cost: 2130.09 ================================================================================ project - ├── columns: r:8 + ├── columns: r:10 ├── scan xy - - │ ├── columns: x:1!null y:2 + - │ ├── columns: x:1!null y:2 xy.crdb_internal_mvcc_timestamp:3 - │ ├── key: (1) - - │ └── fd: (1)-->(2) + - │ └── fd: (1)-->(2,3) + │ ├── columns: x:1!null + │ └── key: (1) └── projections - └── any: eq [as=r:8, outer=(1), correlated-subquery] + └── any: eq [as=r:10, outer=(1), correlated-subquery] ├── project - │ ├── columns: i:4 + │ ├── columns: i:5 │ ├── outer: (1) │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(4) + │ ├── fd: ()-->(5) │ └── select - │ ├── columns: k:3!null i:4 + │ ├── columns: k:4!null i:5 │ ├── outer: (1) │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(3,4) + │ ├── fd: ()-->(4,5) │ ├── scan a - │ │ ├── columns: k:3!null i:4 - │ │ ├── key: (3) - │ │ └── fd: (3)-->(4) + │ │ ├── columns: k:4!null i:5 + │ │ ├── key: (4) + │ │ └── fd: (4)-->(5) │ └── filters - │ └── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + │ └── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] └── 5 ================================================================================ HoistProjectSubquery Cost: 2152.73 ================================================================================ project - ├── columns: r:8 + ├── columns: r:10 - ├── scan xy - │ ├── columns: x:1!null - │ └── key: (1) + ├── inner-join-apply - + │ ├── columns: x:1!null case:11 + + │ ├── columns: x:1!null case:13 + │ ├── key: (1) - + │ ├── fd: (1)-->(11) + + │ ├── fd: (1)-->(13) + │ ├── scan xy + │ │ ├── columns: x:1!null + │ │ └── key: (1) + │ ├── project - + │ │ ├── columns: case:11 + + │ │ ├── columns: case:13 + │ │ ├── outer: (1) + │ │ ├── cardinality: [1 - 1] + │ │ ├── key: () - + │ │ ├── fd: ()-->(11) + + │ │ ├── fd: ()-->(13) + │ │ ├── scalar-group-by - + │ │ │ ├── columns: bool_or:10 + + │ │ │ ├── columns: bool_or:12 + │ │ │ ├── outer: (1) + │ │ │ ├── cardinality: [1 - 1] + │ │ │ ├── key: () - + │ │ │ ├── fd: ()-->(10) + + │ │ │ ├── fd: ()-->(12) + │ │ │ ├── project - + │ │ │ │ ├── columns: notnull:9!null + + │ │ │ │ ├── columns: notnull:11!null + │ │ │ │ ├── outer: (1) + │ │ │ │ ├── cardinality: [0 - 1] + │ │ │ │ ├── key: () - + │ │ │ │ ├── fd: ()-->(9) + + │ │ │ │ ├── fd: ()-->(11) + │ │ │ │ ├── select - + │ │ │ │ │ ├── columns: i:4 + + │ │ │ │ │ ├── columns: i:5 + │ │ │ │ │ ├── outer: (1) + │ │ │ │ │ ├── cardinality: [0 - 1] + │ │ │ │ │ ├── key: () - + │ │ │ │ │ ├── fd: ()-->(4) + + │ │ │ │ │ ├── fd: ()-->(5) + │ │ │ │ │ ├── project - + │ │ │ │ │ │ ├── columns: i:4 + + │ │ │ │ │ │ ├── columns: i:5 + │ │ │ │ │ │ ├── outer: (1) + │ │ │ │ │ │ ├── cardinality: [0 - 1] + │ │ │ │ │ │ ├── key: () - + │ │ │ │ │ │ ├── fd: ()-->(4) + + │ │ │ │ │ │ ├── fd: ()-->(5) + │ │ │ │ │ │ └── select - + │ │ │ │ │ │ ├── columns: k:3!null i:4 + + │ │ │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ │ │ ├── outer: (1) + │ │ │ │ │ │ ├── cardinality: [0 - 1] + │ │ │ │ │ │ ├── key: () - + │ │ │ │ │ │ ├── fd: ()-->(3,4) + + │ │ │ │ │ │ ├── fd: ()-->(4,5) + │ │ │ │ │ │ ├── scan a - + │ │ │ │ │ │ │ ├── columns: k:3!null i:4 - + │ │ │ │ │ │ │ ├── key: (3) - + │ │ │ │ │ │ │ └── fd: (3)-->(4) + + │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + + │ │ │ │ │ │ │ ├── key: (4) + + │ │ │ │ │ │ │ └── fd: (4)-->(5) + │ │ │ │ │ │ └── filters - + │ │ │ │ │ │ └── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + + │ │ │ │ │ │ └── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + │ │ │ │ │ └── filters - + │ │ │ │ │ └── (5 = i:4) IS NOT false [outer=(4)] + + │ │ │ │ │ └── (5 = i:5) IS NOT false [outer=(5)] + │ │ │ │ └── projections - + │ │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + + │ │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] + │ │ │ └── aggregations - + │ │ │ └── bool-or [as=bool_or:10, outer=(9)] - + │ │ │ └── notnull:9 + + │ │ │ └── bool-or [as=bool_or:12, outer=(11)] + + │ │ │ └── notnull:11 + │ │ └── projections - + │ │ └── CASE WHEN bool_or:10 AND (5 IS NOT NULL) THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:11, outer=(10)] + + │ │ └── CASE WHEN bool_or:12 AND (5 IS NOT NULL) THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:13, outer=(12)] + │ └── filters (true) └── projections - - └── any: eq [as=r:8, outer=(1), correlated-subquery] + - └── any: eq [as=r:10, outer=(1), correlated-subquery] - ├── project - - │ ├── columns: i:4 + - │ ├── columns: i:5 - │ ├── outer: (1) - │ ├── cardinality: [0 - 1] - │ ├── key: () - - │ ├── fd: ()-->(4) + - │ ├── fd: ()-->(5) - │ └── select - - │ ├── columns: k:3!null i:4 + - │ ├── columns: k:4!null i:5 - │ ├── outer: (1) - │ ├── cardinality: [0 - 1] - │ ├── key: () - - │ ├── fd: ()-->(3,4) + - │ ├── fd: ()-->(4,5) - │ ├── scan a - - │ │ ├── columns: k:3!null i:4 - - │ │ ├── key: (3) - - │ │ └── fd: (3)-->(4) + - │ │ ├── columns: k:4!null i:5 + - │ │ ├── key: (4) + - │ │ └── fd: (4)-->(5) - │ └── filters - - │ └── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + - │ └── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] - └── 5 - + └── case:11 [as=r:8, outer=(11)] + + └── case:13 [as=r:10, outer=(13)] ================================================================================ CommuteVar Cost: 2152.73 ================================================================================ project - ├── columns: r:8 + ├── columns: r:10 ├── inner-join-apply - │ ├── columns: x:1!null case:11 + │ ├── columns: x:1!null case:13 │ ├── key: (1) - │ ├── fd: (1)-->(11) + │ ├── fd: (1)-->(13) │ ├── scan xy │ │ ├── columns: x:1!null │ │ └── key: (1) │ ├── project - │ │ ├── columns: case:11 + │ │ ├── columns: case:13 │ │ ├── outer: (1) │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(11) + │ │ ├── fd: ()-->(13) │ │ ├── scalar-group-by - │ │ │ ├── columns: bool_or:10 + │ │ │ ├── columns: bool_or:12 │ │ │ ├── outer: (1) │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(10) + │ │ │ ├── fd: ()-->(12) │ │ │ ├── project - │ │ │ │ ├── columns: notnull:9!null + │ │ │ │ ├── columns: notnull:11!null │ │ │ │ ├── outer: (1) │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ ├── key: () - │ │ │ │ ├── fd: ()-->(9) + │ │ │ │ ├── fd: ()-->(11) │ │ │ │ ├── select - │ │ │ │ │ ├── columns: i:4 + │ │ │ │ │ ├── columns: i:5 │ │ │ │ │ ├── outer: (1) │ │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ │ ├── key: () - │ │ │ │ │ ├── fd: ()-->(4) + │ │ │ │ │ ├── fd: ()-->(5) │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: i:4 + │ │ │ │ │ │ ├── columns: i:5 │ │ │ │ │ │ ├── outer: (1) │ │ │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ │ │ ├── key: () - │ │ │ │ │ │ ├── fd: ()-->(4) + │ │ │ │ │ │ ├── fd: ()-->(5) │ │ │ │ │ │ └── select - │ │ │ │ │ │ ├── columns: k:3!null i:4 + │ │ │ │ │ │ ├── columns: k:4!null i:5 │ │ │ │ │ │ ├── outer: (1) │ │ │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ │ │ ├── key: () - │ │ │ │ │ │ ├── fd: ()-->(3,4) + │ │ │ │ │ │ ├── fd: ()-->(4,5) │ │ │ │ │ │ ├── scan a - │ │ │ │ │ │ │ ├── columns: k:3!null i:4 - │ │ │ │ │ │ │ ├── key: (3) - │ │ │ │ │ │ │ └── fd: (3)-->(4) + │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ │ │ │ ├── key: (4) + │ │ │ │ │ │ │ └── fd: (4)-->(5) │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + │ │ │ │ │ │ └── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] │ │ │ │ │ └── filters - - │ │ │ │ │ └── (5 = i:4) IS NOT false [outer=(4)] - + │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + - │ │ │ │ │ └── (5 = i:5) IS NOT false [outer=(5)] + + │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] │ │ │ │ └── projections - │ │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + │ │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] │ │ │ └── aggregations - │ │ │ └── bool-or [as=bool_or:10, outer=(9)] - │ │ │ └── notnull:9 + │ │ │ └── bool-or [as=bool_or:12, outer=(11)] + │ │ │ └── notnull:11 │ │ └── projections - │ │ └── CASE WHEN bool_or:10 AND (5 IS NOT NULL) THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:11, outer=(10)] + │ │ └── CASE WHEN bool_or:12 AND (5 IS NOT NULL) THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:13, outer=(12)] │ └── filters (true) └── projections - └── case:11 [as=r:8, outer=(11)] + └── case:13 [as=r:10, outer=(13)] ================================================================================ PushSelectIntoProject Cost: 2152.73 ================================================================================ project - ├── columns: r:8 + ├── columns: r:10 ├── inner-join-apply - │ ├── columns: x:1!null case:11 + │ ├── columns: x:1!null case:13 │ ├── key: (1) - │ ├── fd: (1)-->(11) + │ ├── fd: (1)-->(13) │ ├── scan xy │ │ ├── columns: x:1!null │ │ └── key: (1) │ ├── project - │ │ ├── columns: case:11 + │ │ ├── columns: case:13 │ │ ├── outer: (1) │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(11) + │ │ ├── fd: ()-->(13) │ │ ├── scalar-group-by - │ │ │ ├── columns: bool_or:10 + │ │ │ ├── columns: bool_or:12 │ │ │ ├── outer: (1) │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(10) + │ │ │ ├── fd: ()-->(12) │ │ │ ├── project - │ │ │ │ ├── columns: notnull:9!null + │ │ │ │ ├── columns: notnull:11!null │ │ │ │ ├── outer: (1) │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ ├── key: () - │ │ │ │ ├── fd: ()-->(9) + │ │ │ │ ├── fd: ()-->(11) │ │ │ │ ├── select - │ │ │ │ │ ├── columns: i:4 + │ │ │ │ │ ├── columns: i:5 │ │ │ │ │ ├── outer: (1) │ │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ │ ├── key: () - │ │ │ │ │ ├── fd: ()-->(4) + │ │ │ │ │ ├── fd: ()-->(5) │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: i:4 + │ │ │ │ │ │ ├── columns: i:5 │ │ │ │ │ │ ├── outer: (1) │ │ │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ │ │ ├── key: () - │ │ │ │ │ │ ├── fd: ()-->(4) + │ │ │ │ │ │ ├── fd: ()-->(5) │ │ │ │ │ │ └── select - │ │ │ │ │ │ ├── columns: k:3!null i:4 + │ │ │ │ │ │ ├── columns: k:4!null i:5 │ │ │ │ │ │ ├── outer: (1) │ │ │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ │ │ ├── key: () - │ │ │ │ │ │ ├── fd: ()-->(3,4) + │ │ │ │ │ │ ├── fd: ()-->(4,5) - │ │ │ │ │ │ ├── scan a + │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: k:3!null i:4 - - │ │ │ │ │ │ │ ├── key: (3) - - │ │ │ │ │ │ │ └── fd: (3)-->(4) + │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + - │ │ │ │ │ │ │ ├── key: (4) + - │ │ │ │ │ │ │ └── fd: (4)-->(5) + │ │ │ │ │ │ │ ├── outer: (1) + │ │ │ │ │ │ │ ├── cardinality: [0 - 1] + │ │ │ │ │ │ │ ├── key: () - + │ │ │ │ │ │ │ ├── fd: ()-->(3,4) + + │ │ │ │ │ │ │ ├── fd: ()-->(4,5) + │ │ │ │ │ │ │ ├── scan a - + │ │ │ │ │ │ │ │ ├── columns: k:3!null i:4 - + │ │ │ │ │ │ │ │ ├── key: (3) - + │ │ │ │ │ │ │ │ └── fd: (3)-->(4) + + │ │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + + │ │ │ │ │ │ │ │ ├── key: (4) + + │ │ │ │ │ │ │ │ └── fd: (4)-->(5) + │ │ │ │ │ │ │ └── filters - + │ │ │ │ │ │ │ └── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + + │ │ │ │ │ │ │ └── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] │ │ │ │ │ │ └── filters - - │ │ │ │ │ │ └── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + - │ │ │ │ │ │ └── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] - │ │ │ │ │ └── filters - - │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] - + │ │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + - │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] + + │ │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] + │ │ │ │ │ └── filters (true) │ │ │ │ └── projections - │ │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + │ │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] │ │ │ └── aggregations - │ │ │ └── bool-or [as=bool_or:10, outer=(9)] - │ │ │ └── notnull:9 + │ │ │ └── bool-or [as=bool_or:12, outer=(11)] + │ │ │ └── notnull:11 │ │ └── projections - │ │ └── CASE WHEN bool_or:10 AND (5 IS NOT NULL) THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:11, outer=(10)] + │ │ └── CASE WHEN bool_or:12 AND (5 IS NOT NULL) THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:13, outer=(12)] │ └── filters (true) └── projections - └── case:11 [as=r:8, outer=(11)] + └── case:13 [as=r:10, outer=(13)] ================================================================================ MergeSelects Cost: 2152.75 ================================================================================ project - ├── columns: r:8 + ├── columns: r:10 ├── inner-join-apply - │ ├── columns: x:1!null case:11 + │ ├── columns: x:1!null case:13 │ ├── key: (1) - │ ├── fd: (1)-->(11) + │ ├── fd: (1)-->(13) │ ├── scan xy │ │ ├── columns: x:1!null │ │ └── key: (1) │ ├── project - │ │ ├── columns: case:11 + │ │ ├── columns: case:13 │ │ ├── outer: (1) │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(11) + │ │ ├── fd: ()-->(13) │ │ ├── scalar-group-by - │ │ │ ├── columns: bool_or:10 + │ │ │ ├── columns: bool_or:12 │ │ │ ├── outer: (1) │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(10) + │ │ │ ├── fd: ()-->(12) │ │ │ ├── project - │ │ │ │ ├── columns: notnull:9!null + │ │ │ │ ├── columns: notnull:11!null │ │ │ │ ├── outer: (1) │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ ├── key: () - │ │ │ │ ├── fd: ()-->(9) + │ │ │ │ ├── fd: ()-->(11) │ │ │ │ ├── select - │ │ │ │ │ ├── columns: i:4 + │ │ │ │ │ ├── columns: i:5 │ │ │ │ │ ├── outer: (1) │ │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ │ ├── key: () - │ │ │ │ │ ├── fd: ()-->(4) + │ │ │ │ │ ├── fd: ()-->(5) │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: i:4 + │ │ │ │ │ │ ├── columns: i:5 │ │ │ │ │ │ ├── outer: (1) │ │ │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ │ │ ├── key: () - │ │ │ │ │ │ ├── fd: ()-->(4) + │ │ │ │ │ │ ├── fd: ()-->(5) │ │ │ │ │ │ └── select - │ │ │ │ │ │ ├── columns: k:3!null i:4 + │ │ │ │ │ │ ├── columns: k:4!null i:5 │ │ │ │ │ │ ├── outer: (1) │ │ │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ │ │ ├── key: () - │ │ │ │ │ │ ├── fd: ()-->(3,4) + │ │ │ │ │ │ ├── fd: ()-->(4,5) - │ │ │ │ │ │ ├── select + │ │ │ │ │ │ ├── scan a - │ │ │ │ │ │ │ ├── columns: k:3!null i:4 + │ │ │ │ │ │ │ ├── columns: k:4!null i:5 - │ │ │ │ │ │ │ ├── outer: (1) - │ │ │ │ │ │ │ ├── cardinality: [0 - 1] - │ │ │ │ │ │ │ ├── key: () - - │ │ │ │ │ │ │ ├── fd: ()-->(3,4) + - │ │ │ │ │ │ │ ├── fd: ()-->(4,5) - │ │ │ │ │ │ │ ├── scan a - - │ │ │ │ │ │ │ │ ├── columns: k:3!null i:4 - - │ │ │ │ │ │ │ │ ├── key: (3) - - │ │ │ │ │ │ │ │ └── fd: (3)-->(4) + - │ │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + - │ │ │ │ │ │ │ │ ├── key: (4) + - │ │ │ │ │ │ │ │ └── fd: (4)-->(5) - │ │ │ │ │ │ │ └── filters - - │ │ │ │ │ │ │ └── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - + │ │ │ │ │ │ │ ├── key: (3) - + │ │ │ │ │ │ │ └── fd: (3)-->(4) + - │ │ │ │ │ │ │ └── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + + │ │ │ │ │ │ │ ├── key: (4) + + │ │ │ │ │ │ │ └── fd: (4)-->(5) │ │ │ │ │ │ └── filters - + │ │ │ │ │ │ ├── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - │ │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + + │ │ │ │ │ │ ├── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + │ │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] │ │ │ │ │ └── filters (true) │ │ │ │ └── projections - │ │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + │ │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] │ │ │ └── aggregations - │ │ │ └── bool-or [as=bool_or:10, outer=(9)] - │ │ │ └── notnull:9 + │ │ │ └── bool-or [as=bool_or:12, outer=(11)] + │ │ │ └── notnull:11 │ │ └── projections - │ │ └── CASE WHEN bool_or:10 AND (5 IS NOT NULL) THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:11, outer=(10)] + │ │ └── CASE WHEN bool_or:12 AND (5 IS NOT NULL) THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:13, outer=(12)] │ └── filters (true) └── projections - └── case:11 [as=r:8, outer=(11)] + └── case:13 [as=r:10, outer=(13)] ================================================================================ EliminateSelect Cost: 2152.73 ================================================================================ project - ├── columns: r:8 + ├── columns: r:10 ├── inner-join-apply - │ ├── columns: x:1!null case:11 + │ ├── columns: x:1!null case:13 │ ├── key: (1) - │ ├── fd: (1)-->(11) + │ ├── fd: (1)-->(13) │ ├── scan xy │ │ ├── columns: x:1!null │ │ └── key: (1) │ ├── project - │ │ ├── columns: case:11 + │ │ ├── columns: case:13 │ │ ├── outer: (1) │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(11) + │ │ ├── fd: ()-->(13) │ │ ├── scalar-group-by - │ │ │ ├── columns: bool_or:10 + │ │ │ ├── columns: bool_or:12 │ │ │ ├── outer: (1) │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(10) + │ │ │ ├── fd: ()-->(12) │ │ │ ├── project - │ │ │ │ ├── columns: notnull:9!null + │ │ │ │ ├── columns: notnull:11!null │ │ │ │ ├── outer: (1) │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ ├── key: () - │ │ │ │ ├── fd: ()-->(9) + │ │ │ │ ├── fd: ()-->(11) - │ │ │ │ ├── select + │ │ │ │ ├── project - │ │ │ │ │ ├── columns: i:4 + │ │ │ │ │ ├── columns: i:5 │ │ │ │ │ ├── outer: (1) │ │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ │ ├── key: () - │ │ │ │ │ ├── fd: ()-->(4) + │ │ │ │ │ ├── fd: ()-->(5) - │ │ │ │ │ ├── project - - │ │ │ │ │ │ ├── columns: i:4 + - │ │ │ │ │ │ ├── columns: i:5 - │ │ │ │ │ │ ├── outer: (1) - │ │ │ │ │ │ ├── cardinality: [0 - 1] - │ │ │ │ │ │ ├── key: () - - │ │ │ │ │ │ ├── fd: ()-->(4) + - │ │ │ │ │ │ ├── fd: ()-->(5) - │ │ │ │ │ │ └── select - - │ │ │ │ │ │ ├── columns: k:3!null i:4 + - │ │ │ │ │ │ ├── columns: k:4!null i:5 - │ │ │ │ │ │ ├── outer: (1) - │ │ │ │ │ │ ├── cardinality: [0 - 1] - │ │ │ │ │ │ ├── key: () - - │ │ │ │ │ │ ├── fd: ()-->(3,4) + - │ │ │ │ │ │ ├── fd: ()-->(4,5) - │ │ │ │ │ │ ├── scan a - - │ │ │ │ │ │ │ ├── columns: k:3!null i:4 - - │ │ │ │ │ │ │ ├── key: (3) - - │ │ │ │ │ │ │ └── fd: (3)-->(4) + - │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + - │ │ │ │ │ │ │ ├── key: (4) + - │ │ │ │ │ │ │ └── fd: (4)-->(5) - │ │ │ │ │ │ └── filters - - │ │ │ │ │ │ ├── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - - │ │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + - │ │ │ │ │ │ ├── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + - │ │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] - │ │ │ │ │ └── filters (true) + │ │ │ │ │ └── select - + │ │ │ │ │ ├── columns: k:3!null i:4 + + │ │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ │ ├── outer: (1) + │ │ │ │ │ ├── cardinality: [0 - 1] + │ │ │ │ │ ├── key: () - + │ │ │ │ │ ├── fd: ()-->(3,4) + + │ │ │ │ │ ├── fd: ()-->(4,5) + │ │ │ │ │ ├── scan a - + │ │ │ │ │ │ ├── columns: k:3!null i:4 - + │ │ │ │ │ │ ├── key: (3) - + │ │ │ │ │ │ └── fd: (3)-->(4) + + │ │ │ │ │ │ ├── columns: k:4!null i:5 + + │ │ │ │ │ │ ├── key: (4) + + │ │ │ │ │ │ └── fd: (4)-->(5) + │ │ │ │ │ └── filters - + │ │ │ │ │ ├── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - + │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + + │ │ │ │ │ ├── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + + │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] │ │ │ │ └── projections - │ │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + │ │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] │ │ │ └── aggregations - │ │ │ └── bool-or [as=bool_or:10, outer=(9)] - │ │ │ └── notnull:9 + │ │ │ └── bool-or [as=bool_or:12, outer=(11)] + │ │ │ └── notnull:11 │ │ └── projections - │ │ └── CASE WHEN bool_or:10 AND (5 IS NOT NULL) THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:11, outer=(10)] + │ │ └── CASE WHEN bool_or:12 AND (5 IS NOT NULL) THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:13, outer=(12)] │ └── filters (true) └── projections - └── case:11 [as=r:8, outer=(11)] + └── case:13 [as=r:10, outer=(13)] ================================================================================ MergeProjects Cost: 2152.71 ================================================================================ project - ├── columns: r:8 + ├── columns: r:10 ├── inner-join-apply - │ ├── columns: x:1!null case:11 + │ ├── columns: x:1!null case:13 │ ├── key: (1) - │ ├── fd: (1)-->(11) + │ ├── fd: (1)-->(13) │ ├── scan xy │ │ ├── columns: x:1!null │ │ └── key: (1) │ ├── project - │ │ ├── columns: case:11 + │ │ ├── columns: case:13 │ │ ├── outer: (1) │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(11) + │ │ ├── fd: ()-->(13) │ │ ├── scalar-group-by - │ │ │ ├── columns: bool_or:10 + │ │ │ ├── columns: bool_or:12 │ │ │ ├── outer: (1) │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(10) + │ │ │ ├── fd: ()-->(12) │ │ │ ├── project - │ │ │ │ ├── columns: notnull:9!null + │ │ │ │ ├── columns: notnull:11!null │ │ │ │ ├── outer: (1) │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ ├── key: () - │ │ │ │ ├── fd: ()-->(9) + │ │ │ │ ├── fd: ()-->(11) - │ │ │ │ ├── project - - │ │ │ │ │ ├── columns: i:4 + - │ │ │ │ │ ├── columns: i:5 + │ │ │ │ ├── select - + │ │ │ │ │ ├── columns: k:3!null i:4 + + │ │ │ │ │ ├── columns: k:4!null i:5 │ │ │ │ │ ├── outer: (1) │ │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ │ ├── key: () - - │ │ │ │ │ ├── fd: ()-->(4) + - │ │ │ │ │ ├── fd: ()-->(5) - │ │ │ │ │ └── select - - │ │ │ │ │ ├── columns: k:3!null i:4 + - │ │ │ │ │ ├── columns: k:4!null i:5 - │ │ │ │ │ ├── outer: (1) - │ │ │ │ │ ├── cardinality: [0 - 1] - │ │ │ │ │ ├── key: () - - │ │ │ │ │ ├── fd: ()-->(3,4) + - │ │ │ │ │ ├── fd: ()-->(4,5) - │ │ │ │ │ ├── scan a - - │ │ │ │ │ │ ├── columns: k:3!null i:4 - - │ │ │ │ │ │ ├── key: (3) - - │ │ │ │ │ │ └── fd: (3)-->(4) + - │ │ │ │ │ │ ├── columns: k:4!null i:5 + - │ │ │ │ │ │ ├── key: (4) + - │ │ │ │ │ │ └── fd: (4)-->(5) - │ │ │ │ │ └── filters - - │ │ │ │ │ ├── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - - │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] - + │ │ │ │ │ ├── fd: ()-->(3,4) + - │ │ │ │ │ ├── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + - │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] + + │ │ │ │ │ ├── fd: ()-->(4,5) + │ │ │ │ │ ├── scan a - + │ │ │ │ │ │ ├── columns: k:3!null i:4 - + │ │ │ │ │ │ ├── key: (3) - + │ │ │ │ │ │ └── fd: (3)-->(4) + + │ │ │ │ │ │ ├── columns: k:4!null i:5 + + │ │ │ │ │ │ ├── key: (4) + + │ │ │ │ │ │ └── fd: (4)-->(5) + │ │ │ │ │ └── filters - + │ │ │ │ │ ├── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - + │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + + │ │ │ │ │ ├── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + + │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] │ │ │ │ └── projections - │ │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + │ │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] │ │ │ └── aggregations - │ │ │ └── bool-or [as=bool_or:10, outer=(9)] - │ │ │ └── notnull:9 + │ │ │ └── bool-or [as=bool_or:12, outer=(11)] + │ │ │ └── notnull:11 │ │ └── projections - │ │ └── CASE WHEN bool_or:10 AND (5 IS NOT NULL) THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:11, outer=(10)] + │ │ └── CASE WHEN bool_or:12 AND (5 IS NOT NULL) THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:13, outer=(12)] │ └── filters (true) └── projections - └── case:11 [as=r:8, outer=(11)] + └── case:13 [as=r:10, outer=(13)] ================================================================================ FoldNonNullIsNotNull Cost: 2152.71 ================================================================================ project - ├── columns: r:8 + ├── columns: r:10 ├── inner-join-apply - │ ├── columns: x:1!null case:11 + │ ├── columns: x:1!null case:13 │ ├── key: (1) - │ ├── fd: (1)-->(11) + │ ├── fd: (1)-->(13) │ ├── scan xy │ │ ├── columns: x:1!null │ │ └── key: (1) │ ├── project - │ │ ├── columns: case:11 + │ │ ├── columns: case:13 │ │ ├── outer: (1) │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(11) + │ │ ├── fd: ()-->(13) │ │ ├── scalar-group-by - │ │ │ ├── columns: bool_or:10 + │ │ │ ├── columns: bool_or:12 │ │ │ ├── outer: (1) │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(10) + │ │ │ ├── fd: ()-->(12) │ │ │ ├── project - │ │ │ │ ├── columns: notnull:9!null + │ │ │ │ ├── columns: notnull:11!null │ │ │ │ ├── outer: (1) │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ ├── key: () - │ │ │ │ ├── fd: ()-->(9) + │ │ │ │ ├── fd: ()-->(11) │ │ │ │ ├── select - │ │ │ │ │ ├── columns: k:3!null i:4 + │ │ │ │ │ ├── columns: k:4!null i:5 │ │ │ │ │ ├── outer: (1) │ │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ │ ├── key: () - │ │ │ │ │ ├── fd: ()-->(3,4) + │ │ │ │ │ ├── fd: ()-->(4,5) │ │ │ │ │ ├── scan a - │ │ │ │ │ │ ├── columns: k:3!null i:4 - │ │ │ │ │ │ ├── key: (3) - │ │ │ │ │ │ └── fd: (3)-->(4) + │ │ │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ │ │ ├── key: (4) + │ │ │ │ │ │ └── fd: (4)-->(5) │ │ │ │ │ └── filters - │ │ │ │ │ ├── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + │ │ │ │ │ ├── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] │ │ │ │ └── projections - │ │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + │ │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] │ │ │ └── aggregations - │ │ │ └── bool-or [as=bool_or:10, outer=(9)] - │ │ │ └── notnull:9 + │ │ │ └── bool-or [as=bool_or:12, outer=(11)] + │ │ │ └── notnull:11 │ │ └── projections - - │ │ └── CASE WHEN bool_or:10 AND (5 IS NOT NULL) THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:11, outer=(10)] - + │ │ └── CASE WHEN bool_or:10 AND true THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:11, outer=(10)] + - │ │ └── CASE WHEN bool_or:12 AND (5 IS NOT NULL) THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:13, outer=(12)] + + │ │ └── CASE WHEN bool_or:12 AND true THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:13, outer=(12)] │ └── filters (true) └── projections - └── case:11 [as=r:8, outer=(11)] + └── case:13 [as=r:10, outer=(13)] ================================================================================ SimplifyAndTrue Cost: 2152.71 ================================================================================ project - ├── columns: r:8 + ├── columns: r:10 ├── inner-join-apply - │ ├── columns: x:1!null case:11 + │ ├── columns: x:1!null case:13 │ ├── key: (1) - │ ├── fd: (1)-->(11) + │ ├── fd: (1)-->(13) │ ├── scan xy │ │ ├── columns: x:1!null │ │ └── key: (1) │ ├── project - │ │ ├── columns: case:11 + │ │ ├── columns: case:13 │ │ ├── outer: (1) │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(11) + │ │ ├── fd: ()-->(13) │ │ ├── scalar-group-by - │ │ │ ├── columns: bool_or:10 + │ │ │ ├── columns: bool_or:12 │ │ │ ├── outer: (1) │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(10) + │ │ │ ├── fd: ()-->(12) │ │ │ ├── project - │ │ │ │ ├── columns: notnull:9!null + │ │ │ │ ├── columns: notnull:11!null │ │ │ │ ├── outer: (1) │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ ├── key: () - │ │ │ │ ├── fd: ()-->(9) + │ │ │ │ ├── fd: ()-->(11) │ │ │ │ ├── select - │ │ │ │ │ ├── columns: k:3!null i:4 + │ │ │ │ │ ├── columns: k:4!null i:5 │ │ │ │ │ ├── outer: (1) │ │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ │ ├── key: () - │ │ │ │ │ ├── fd: ()-->(3,4) + │ │ │ │ │ ├── fd: ()-->(4,5) │ │ │ │ │ ├── scan a - │ │ │ │ │ │ ├── columns: k:3!null i:4 - │ │ │ │ │ │ ├── key: (3) - │ │ │ │ │ │ └── fd: (3)-->(4) + │ │ │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ │ │ ├── key: (4) + │ │ │ │ │ │ └── fd: (4)-->(5) │ │ │ │ │ └── filters - │ │ │ │ │ ├── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + │ │ │ │ │ ├── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] │ │ │ │ └── projections - │ │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + │ │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] │ │ │ └── aggregations - │ │ │ └── bool-or [as=bool_or:10, outer=(9)] - │ │ │ └── notnull:9 + │ │ │ └── bool-or [as=bool_or:12, outer=(11)] + │ │ │ └── notnull:11 │ │ └── projections - - │ │ └── CASE WHEN bool_or:10 AND true THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:11, outer=(10)] - + │ │ └── CASE WHEN bool_or:10 THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:11, outer=(10)] + - │ │ └── CASE WHEN bool_or:12 AND true THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:13, outer=(12)] + + │ │ └── CASE WHEN bool_or:12 THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:13, outer=(12)] │ └── filters (true) └── projections - └── case:11 [as=r:8, outer=(11)] + └── case:13 [as=r:10, outer=(13)] ================================================================================ TryDecorrelateProject Cost: 2182.70 ================================================================================ project - ├── columns: r:8 + ├── columns: r:10 - ├── inner-join-apply + ├── select - │ ├── columns: x:1!null case:11 + │ ├── columns: x:1!null case:13 │ ├── key: (1) - │ ├── fd: (1)-->(11) + │ ├── fd: (1)-->(13) - │ ├── scan xy - │ │ ├── columns: x:1!null - │ │ └── key: (1) │ ├── project - - │ │ ├── columns: case:11 + - │ │ ├── columns: case:13 - │ │ ├── outer: (1) - │ │ ├── cardinality: [1 - 1] - │ │ ├── key: () - - │ │ ├── fd: ()-->(11) + - │ │ ├── fd: ()-->(13) - │ │ ├── scalar-group-by - - │ │ │ ├── columns: bool_or:10 + - │ │ │ ├── columns: bool_or:12 - │ │ │ ├── outer: (1) - │ │ │ ├── cardinality: [1 - 1] - │ │ │ ├── key: () - - │ │ │ ├── fd: ()-->(10) + - │ │ │ ├── fd: ()-->(12) - │ │ │ ├── project - - │ │ │ │ ├── columns: notnull:9!null - + │ │ ├── columns: case:11 x:1!null + - │ │ │ │ ├── columns: notnull:11!null + + │ │ ├── columns: case:13 x:1!null + │ │ ├── key: (1) - + │ │ ├── fd: (1)-->(11) + + │ │ ├── fd: (1)-->(13) + │ │ ├── inner-join-apply - + │ │ │ ├── columns: x:1!null bool_or:10 + + │ │ │ ├── columns: x:1!null bool_or:12 + │ │ │ ├── key: (1) - + │ │ │ ├── fd: (1)-->(10) + + │ │ │ ├── fd: (1)-->(12) + │ │ │ ├── scan xy + │ │ │ │ ├── columns: x:1!null + │ │ │ │ └── key: (1) + │ │ │ ├── scalar-group-by - + │ │ │ │ ├── columns: bool_or:10 + + │ │ │ │ ├── columns: bool_or:12 │ │ │ │ ├── outer: (1) - │ │ │ │ ├── cardinality: [0 - 1] + │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── key: () - - │ │ │ │ ├── fd: ()-->(9) + - │ │ │ │ ├── fd: ()-->(11) - │ │ │ │ ├── select - - │ │ │ │ │ ├── columns: k:3!null i:4 - + │ │ │ │ ├── fd: ()-->(10) + - │ │ │ │ │ ├── columns: k:4!null i:5 + + │ │ │ │ ├── fd: ()-->(12) + │ │ │ │ ├── project - + │ │ │ │ │ ├── columns: notnull:9!null + + │ │ │ │ │ ├── columns: notnull:11!null │ │ │ │ │ ├── outer: (1) │ │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ │ ├── key: () - - │ │ │ │ │ ├── fd: ()-->(3,4) + - │ │ │ │ │ ├── fd: ()-->(4,5) - │ │ │ │ │ ├── scan a - + │ │ │ │ │ ├── fd: ()-->(9) + + │ │ │ │ │ ├── fd: ()-->(11) + │ │ │ │ │ ├── select - │ │ │ │ │ │ ├── columns: k:3!null i:4 - - │ │ │ │ │ │ ├── key: (3) - - │ │ │ │ │ │ └── fd: (3)-->(4) + │ │ │ │ │ │ ├── columns: k:4!null i:5 + - │ │ │ │ │ │ ├── key: (4) + - │ │ │ │ │ │ └── fd: (4)-->(5) - │ │ │ │ │ └── filters - - │ │ │ │ │ ├── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - - │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + - │ │ │ │ │ ├── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + - │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] - │ │ │ │ └── projections - - │ │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + - │ │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] - │ │ │ └── aggregations - - │ │ │ └── bool-or [as=bool_or:10, outer=(9)] - - │ │ │ └── notnull:9 + - │ │ │ └── bool-or [as=bool_or:12, outer=(11)] + - │ │ │ └── notnull:11 + │ │ │ │ │ │ ├── outer: (1) + │ │ │ │ │ │ ├── cardinality: [0 - 1] + │ │ │ │ │ │ ├── key: () - + │ │ │ │ │ │ ├── fd: ()-->(3,4) + + │ │ │ │ │ │ ├── fd: ()-->(4,5) + │ │ │ │ │ │ ├── scan a - + │ │ │ │ │ │ │ ├── columns: k:3!null i:4 - + │ │ │ │ │ │ │ ├── key: (3) - + │ │ │ │ │ │ │ └── fd: (3)-->(4) + + │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + + │ │ │ │ │ │ │ ├── key: (4) + + │ │ │ │ │ │ │ └── fd: (4)-->(5) + │ │ │ │ │ │ └── filters - + │ │ │ │ │ │ ├── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - + │ │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + + │ │ │ │ │ │ ├── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + + │ │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] + │ │ │ │ │ └── projections - + │ │ │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + + │ │ │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] + │ │ │ │ └── aggregations - + │ │ │ │ └── bool-or [as=bool_or:10, outer=(9)] - + │ │ │ │ └── notnull:9 + + │ │ │ │ └── bool-or [as=bool_or:12, outer=(11)] + + │ │ │ │ └── notnull:11 + │ │ │ └── filters (true) │ │ └── projections - │ │ └── CASE WHEN bool_or:10 THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:11, outer=(10)] + │ │ └── CASE WHEN bool_or:12 THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:13, outer=(12)] │ └── filters (true) └── projections - └── case:11 [as=r:8, outer=(11)] + └── case:13 [as=r:10, outer=(13)] ================================================================================ TryDecorrelateScalarGroupBy Cost: 2242.70 ================================================================================ project - ├── columns: r:8 + ├── columns: r:10 ├── select - │ ├── columns: x:1!null case:11 + │ ├── columns: x:1!null case:13 │ ├── key: (1) - │ ├── fd: (1)-->(11) + │ ├── fd: (1)-->(13) │ ├── project - │ │ ├── columns: case:11 x:1!null + │ │ ├── columns: case:13 x:1!null │ │ ├── key: (1) - │ │ ├── fd: (1)-->(11) + │ │ ├── fd: (1)-->(13) - │ │ ├── inner-join-apply + │ │ ├── select - │ │ │ ├── columns: x:1!null bool_or:10 + │ │ │ ├── columns: x:1!null bool_or:12 │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(10) + │ │ │ ├── fd: (1)-->(12) - │ │ │ ├── scan xy - │ │ │ │ ├── columns: x:1!null - │ │ │ │ └── key: (1) - │ │ │ ├── scalar-group-by - - │ │ │ │ ├── columns: bool_or:10 + - │ │ │ │ ├── columns: bool_or:12 - │ │ │ │ ├── outer: (1) - │ │ │ │ ├── cardinality: [1 - 1] - │ │ │ │ ├── key: () - - │ │ │ │ ├── fd: ()-->(10) + - │ │ │ │ ├── fd: ()-->(12) - │ │ │ │ ├── project - - │ │ │ │ │ ├── columns: notnull:9!null + - │ │ │ │ │ ├── columns: notnull:11!null - │ │ │ │ │ ├── outer: (1) - │ │ │ │ │ ├── cardinality: [0 - 1] - │ │ │ │ │ ├── key: () - - │ │ │ │ │ ├── fd: ()-->(9) + - │ │ │ │ │ ├── fd: ()-->(11) - │ │ │ │ │ ├── select - - │ │ │ │ │ │ ├── columns: k:3!null i:4 + - │ │ │ │ │ │ ├── columns: k:4!null i:5 - │ │ │ │ │ │ ├── outer: (1) - │ │ │ │ │ │ ├── cardinality: [0 - 1] - │ │ │ │ │ │ ├── key: () - - │ │ │ │ │ │ ├── fd: ()-->(3,4) + - │ │ │ │ │ │ ├── fd: ()-->(4,5) - │ │ │ │ │ │ ├── scan a - - │ │ │ │ │ │ │ ├── columns: k:3!null i:4 - - │ │ │ │ │ │ │ ├── key: (3) - - │ │ │ │ │ │ │ └── fd: (3)-->(4) + - │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + - │ │ │ │ │ │ │ ├── key: (4) + - │ │ │ │ │ │ │ └── fd: (4)-->(5) - │ │ │ │ │ │ └── filters - - │ │ │ │ │ │ ├── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - - │ │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + - │ │ │ │ │ │ ├── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + - │ │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] - │ │ │ │ │ └── projections - - │ │ │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + - │ │ │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] - │ │ │ │ └── aggregations - - │ │ │ │ └── bool-or [as=bool_or:10, outer=(9)] - - │ │ │ │ └── notnull:9 + - │ │ │ │ └── bool-or [as=bool_or:12, outer=(11)] + - │ │ │ │ └── notnull:11 + │ │ │ ├── project - + │ │ │ │ ├── columns: x:1!null bool_or:10 + + │ │ │ │ ├── columns: x:1!null bool_or:12 + │ │ │ │ ├── key: (1) - + │ │ │ │ ├── fd: (1)-->(10) + + │ │ │ │ ├── fd: (1)-->(12) + │ │ │ │ └── group-by - + │ │ │ │ ├── columns: x:1!null bool_or:10 + + │ │ │ │ ├── columns: x:1!null bool_or:12 + │ │ │ │ ├── grouping columns: x:1!null + │ │ │ │ ├── key: (1) - + │ │ │ │ ├── fd: (1)-->(10) + + │ │ │ │ ├── fd: (1)-->(12) + │ │ │ │ ├── left-join-apply - + │ │ │ │ │ ├── columns: x:1!null notnull:9 + + │ │ │ │ │ ├── columns: x:1!null notnull:11 + │ │ │ │ │ ├── key: (1) - + │ │ │ │ │ ├── fd: (1)-->(9) + + │ │ │ │ │ ├── fd: (1)-->(11) + │ │ │ │ │ ├── scan xy + │ │ │ │ │ │ ├── columns: x:1!null + │ │ │ │ │ │ └── key: (1) + │ │ │ │ │ ├── project - + │ │ │ │ │ │ ├── columns: notnull:9!null + + │ │ │ │ │ │ ├── columns: notnull:11!null + │ │ │ │ │ │ ├── outer: (1) + │ │ │ │ │ │ ├── cardinality: [0 - 1] + │ │ │ │ │ │ ├── key: () - + │ │ │ │ │ │ ├── fd: ()-->(9) + + │ │ │ │ │ │ ├── fd: ()-->(11) + │ │ │ │ │ │ ├── select - + │ │ │ │ │ │ │ ├── columns: k:3!null i:4 + + │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ │ │ │ ├── outer: (1) + │ │ │ │ │ │ │ ├── cardinality: [0 - 1] + │ │ │ │ │ │ │ ├── key: () - + │ │ │ │ │ │ │ ├── fd: ()-->(3,4) + + │ │ │ │ │ │ │ ├── fd: ()-->(4,5) + │ │ │ │ │ │ │ ├── scan a - + │ │ │ │ │ │ │ │ ├── columns: k:3!null i:4 - + │ │ │ │ │ │ │ │ ├── key: (3) - + │ │ │ │ │ │ │ │ └── fd: (3)-->(4) + + │ │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + + │ │ │ │ │ │ │ │ ├── key: (4) + + │ │ │ │ │ │ │ │ └── fd: (4)-->(5) + │ │ │ │ │ │ │ └── filters - + │ │ │ │ │ │ │ ├── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - + │ │ │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + + │ │ │ │ │ │ │ ├── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + + │ │ │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] + │ │ │ │ │ │ └── projections - + │ │ │ │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + + │ │ │ │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] + │ │ │ │ │ └── filters (true) + │ │ │ │ └── aggregations - + │ │ │ │ └── bool-or [as=bool_or:10, outer=(9)] - + │ │ │ │ └── notnull:9 + + │ │ │ │ └── bool-or [as=bool_or:12, outer=(11)] + + │ │ │ │ └── notnull:11 │ │ │ └── filters (true) │ │ └── projections - │ │ └── CASE WHEN bool_or:10 THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:11, outer=(10)] + │ │ └── CASE WHEN bool_or:12 THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:13, outer=(12)] │ └── filters (true) └── projections - └── case:11 [as=r:8, outer=(11)] + └── case:13 [as=r:10, outer=(13)] ================================================================================ TryDecorrelateProjectSelect Cost: 2280.16 ================================================================================ project - ├── columns: r:8 + ├── columns: r:10 ├── select - │ ├── columns: x:1!null case:11 + │ ├── columns: x:1!null case:13 │ ├── key: (1) - │ ├── fd: (1)-->(11) + │ ├── fd: (1)-->(13) │ ├── project - │ │ ├── columns: case:11 x:1!null + │ │ ├── columns: case:13 x:1!null │ │ ├── key: (1) - │ │ ├── fd: (1)-->(11) + │ │ ├── fd: (1)-->(13) │ │ ├── select - │ │ │ ├── columns: x:1!null bool_or:10 + │ │ │ ├── columns: x:1!null bool_or:12 │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(10) + │ │ │ ├── fd: (1)-->(12) │ │ │ ├── project - │ │ │ │ ├── columns: x:1!null bool_or:10 + │ │ │ │ ├── columns: x:1!null bool_or:12 │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(10) + │ │ │ │ ├── fd: (1)-->(12) │ │ │ │ └── group-by - │ │ │ │ ├── columns: x:1!null bool_or:10 + │ │ │ │ ├── columns: x:1!null bool_or:12 │ │ │ │ ├── grouping columns: x:1!null │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(10) + │ │ │ │ ├── fd: (1)-->(12) - │ │ │ │ ├── left-join-apply + │ │ │ │ ├── project - │ │ │ │ │ ├── columns: x:1!null notnull:9 + │ │ │ │ │ ├── columns: x:1!null notnull:11 │ │ │ │ │ ├── key: (1) - │ │ │ │ │ ├── fd: (1)-->(9) + │ │ │ │ │ ├── fd: (1)-->(11) - │ │ │ │ │ ├── scan xy - │ │ │ │ │ │ ├── columns: x:1!null - │ │ │ │ │ │ └── key: (1) - │ │ │ │ │ ├── project - - │ │ │ │ │ │ ├── columns: notnull:9!null + - │ │ │ │ │ │ ├── columns: notnull:11!null - │ │ │ │ │ │ ├── outer: (1) - │ │ │ │ │ │ ├── cardinality: [0 - 1] - │ │ │ │ │ │ ├── key: () - - │ │ │ │ │ │ ├── fd: ()-->(9) + - │ │ │ │ │ │ ├── fd: ()-->(11) - │ │ │ │ │ │ ├── select - - │ │ │ │ │ │ │ ├── columns: k:3!null i:4 + - │ │ │ │ │ │ │ ├── columns: k:4!null i:5 - │ │ │ │ │ │ │ ├── outer: (1) - │ │ │ │ │ │ │ ├── cardinality: [0 - 1] - │ │ │ │ │ │ │ ├── key: () - - │ │ │ │ │ │ │ ├── fd: ()-->(3,4) + - │ │ │ │ │ │ │ ├── fd: ()-->(4,5) - │ │ │ │ │ │ │ ├── scan a - - │ │ │ │ │ │ │ │ ├── columns: k:3!null i:4 - - │ │ │ │ │ │ │ │ ├── key: (3) - - │ │ │ │ │ │ │ │ └── fd: (3)-->(4) + - │ │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + - │ │ │ │ │ │ │ │ ├── key: (4) + - │ │ │ │ │ │ │ │ └── fd: (4)-->(5) - │ │ │ │ │ │ │ └── filters - - │ │ │ │ │ │ │ ├── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - - │ │ │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + - │ │ │ │ │ │ │ ├── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + - │ │ │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] - │ │ │ │ │ │ └── projections - - │ │ │ │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + - │ │ │ │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] - │ │ │ │ │ └── filters (true) + │ │ │ │ │ └── left-join-apply - + │ │ │ │ │ ├── columns: x:1!null k:3 i:4 notnull:9 + + │ │ │ │ │ ├── columns: x:1!null k:4 i:5 notnull:11 + │ │ │ │ │ ├── key: (1) - + │ │ │ │ │ ├── fd: (1)-->(3,4,9) + + │ │ │ │ │ ├── fd: (1)-->(4,5,11) + │ │ │ │ │ ├── scan xy + │ │ │ │ │ │ ├── columns: x:1!null + │ │ │ │ │ │ └── key: (1) + │ │ │ │ │ ├── project - + │ │ │ │ │ │ ├── columns: notnull:9!null k:3!null i:4 - + │ │ │ │ │ │ ├── key: (3) - + │ │ │ │ │ │ ├── fd: (3)-->(4), (4)-->(9) + + │ │ │ │ │ │ ├── columns: notnull:11!null k:4!null i:5 + + │ │ │ │ │ │ ├── key: (4) + + │ │ │ │ │ │ ├── fd: (4)-->(5), (5)-->(11) + │ │ │ │ │ │ ├── scan a - + │ │ │ │ │ │ │ ├── columns: k:3!null i:4 - + │ │ │ │ │ │ │ ├── key: (3) - + │ │ │ │ │ │ │ └── fd: (3)-->(4) + + │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + + │ │ │ │ │ │ │ ├── key: (4) + + │ │ │ │ │ │ │ └── fd: (4)-->(5) + │ │ │ │ │ │ └── projections - + │ │ │ │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + + │ │ │ │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] + │ │ │ │ │ └── filters - + │ │ │ │ │ ├── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - + │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + + │ │ │ │ │ ├── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + + │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] │ │ │ │ └── aggregations - │ │ │ │ └── bool-or [as=bool_or:10, outer=(9)] - │ │ │ │ └── notnull:9 + │ │ │ │ └── bool-or [as=bool_or:12, outer=(11)] + │ │ │ │ └── notnull:11 │ │ │ └── filters (true) │ │ └── projections - │ │ └── CASE WHEN bool_or:10 THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:11, outer=(10)] + │ │ └── CASE WHEN bool_or:12 THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:13, outer=(12)] │ └── filters (true) └── projections - └── case:11 [as=r:8, outer=(11)] + └── case:13 [as=r:10, outer=(13)] ================================================================================ DecorrelateJoin Cost: 2280.16 ================================================================================ project - ├── columns: r:8 + ├── columns: r:10 ├── select - │ ├── columns: x:1!null case:11 + │ ├── columns: x:1!null case:13 │ ├── key: (1) - │ ├── fd: (1)-->(11) + │ ├── fd: (1)-->(13) │ ├── project - │ │ ├── columns: case:11 x:1!null + │ │ ├── columns: case:13 x:1!null │ │ ├── key: (1) - │ │ ├── fd: (1)-->(11) + │ │ ├── fd: (1)-->(13) │ │ ├── select - │ │ │ ├── columns: x:1!null bool_or:10 + │ │ │ ├── columns: x:1!null bool_or:12 │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(10) + │ │ │ ├── fd: (1)-->(12) │ │ │ ├── project - │ │ │ │ ├── columns: x:1!null bool_or:10 + │ │ │ │ ├── columns: x:1!null bool_or:12 │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(10) + │ │ │ │ ├── fd: (1)-->(12) │ │ │ │ └── group-by - │ │ │ │ ├── columns: x:1!null bool_or:10 + │ │ │ │ ├── columns: x:1!null bool_or:12 │ │ │ │ ├── grouping columns: x:1!null │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(10) + │ │ │ │ ├── fd: (1)-->(12) │ │ │ │ ├── project - │ │ │ │ │ ├── columns: x:1!null notnull:9 + │ │ │ │ │ ├── columns: x:1!null notnull:11 │ │ │ │ │ ├── key: (1) - │ │ │ │ │ ├── fd: (1)-->(9) + │ │ │ │ │ ├── fd: (1)-->(11) - │ │ │ │ │ └── left-join-apply + │ │ │ │ │ └── left-join (hash) - │ │ │ │ │ ├── columns: x:1!null k:3 i:4 notnull:9 + │ │ │ │ │ ├── columns: x:1!null k:4 i:5 notnull:11 + │ │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ │ │ │ ├── key: (1) - - │ │ │ │ │ ├── fd: (1)-->(3,4,9) - + │ │ │ │ │ ├── fd: (3)-->(4), (4)~~>(9), (1)-->(3,4,9) + - │ │ │ │ │ ├── fd: (1)-->(4,5,11) + + │ │ │ │ │ ├── fd: (4)-->(5), (5)~~>(11), (1)-->(4,5,11) │ │ │ │ │ ├── scan xy │ │ │ │ │ │ ├── columns: x:1!null │ │ │ │ │ │ └── key: (1) │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: notnull:9!null k:3!null i:4 - │ │ │ │ │ │ ├── key: (3) - │ │ │ │ │ │ ├── fd: (3)-->(4), (4)-->(9) + │ │ │ │ │ │ ├── columns: notnull:11!null k:4!null i:5 + │ │ │ │ │ │ ├── key: (4) + │ │ │ │ │ │ ├── fd: (4)-->(5), (5)-->(11) │ │ │ │ │ │ ├── scan a - │ │ │ │ │ │ │ ├── columns: k:3!null i:4 - │ │ │ │ │ │ │ ├── key: (3) - │ │ │ │ │ │ │ └── fd: (3)-->(4) + │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ │ │ │ ├── key: (4) + │ │ │ │ │ │ │ └── fd: (4)-->(5) │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + │ │ │ │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] │ │ │ │ │ └── filters - │ │ │ │ │ ├── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + │ │ │ │ │ ├── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] │ │ │ │ └── aggregations - │ │ │ │ └── bool-or [as=bool_or:10, outer=(9)] - │ │ │ │ └── notnull:9 + │ │ │ │ └── bool-or [as=bool_or:12, outer=(11)] + │ │ │ │ └── notnull:11 │ │ │ └── filters (true) │ │ └── projections - │ │ └── CASE WHEN bool_or:10 THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:11, outer=(10)] + │ │ └── CASE WHEN bool_or:12 THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:13, outer=(12)] │ └── filters (true) └── projections - └── case:11 [as=r:8, outer=(11)] + └── case:13 [as=r:10, outer=(13)] ================================================================================ PushFilterIntoJoinRight Cost: 2278.50 ================================================================================ project - ├── columns: r:8 + ├── columns: r:10 ├── select - │ ├── columns: x:1!null case:11 + │ ├── columns: x:1!null case:13 │ ├── key: (1) - │ ├── fd: (1)-->(11) + │ ├── fd: (1)-->(13) │ ├── project - │ │ ├── columns: case:11 x:1!null + │ │ ├── columns: case:13 x:1!null │ │ ├── key: (1) - │ │ ├── fd: (1)-->(11) + │ │ ├── fd: (1)-->(13) │ │ ├── select - │ │ │ ├── columns: x:1!null bool_or:10 + │ │ │ ├── columns: x:1!null bool_or:12 │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(10) + │ │ │ ├── fd: (1)-->(12) │ │ │ ├── project - │ │ │ │ ├── columns: x:1!null bool_or:10 + │ │ │ │ ├── columns: x:1!null bool_or:12 │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(10) + │ │ │ │ ├── fd: (1)-->(12) │ │ │ │ └── group-by - │ │ │ │ ├── columns: x:1!null bool_or:10 + │ │ │ │ ├── columns: x:1!null bool_or:12 │ │ │ │ ├── grouping columns: x:1!null │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(10) + │ │ │ │ ├── fd: (1)-->(12) │ │ │ │ ├── project - │ │ │ │ │ ├── columns: x:1!null notnull:9 + │ │ │ │ │ ├── columns: x:1!null notnull:11 │ │ │ │ │ ├── key: (1) - │ │ │ │ │ ├── fd: (1)-->(9) + │ │ │ │ │ ├── fd: (1)-->(11) │ │ │ │ │ └── left-join (hash) - │ │ │ │ │ ├── columns: x:1!null k:3 i:4 notnull:9 + │ │ │ │ │ ├── columns: x:1!null k:4 i:5 notnull:11 │ │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ │ │ │ ├── key: (1) - │ │ │ │ │ ├── fd: (3)-->(4), (4)~~>(9), (1)-->(3,4,9) + │ │ │ │ │ ├── fd: (4)-->(5), (5)~~>(11), (1)-->(4,5,11) │ │ │ │ │ ├── scan xy │ │ │ │ │ │ ├── columns: x:1!null │ │ │ │ │ │ └── key: (1) - │ │ │ │ │ ├── project - - │ │ │ │ │ │ ├── columns: notnull:9!null k:3!null i:4 + - │ │ │ │ │ │ ├── columns: notnull:11!null k:4!null i:5 + │ │ │ │ │ ├── select - + │ │ │ │ │ │ ├── columns: k:3!null i:4 notnull:9!null - │ │ │ │ │ │ ├── key: (3) - │ │ │ │ │ │ ├── fd: (3)-->(4), (4)-->(9) + + │ │ │ │ │ │ ├── columns: k:4!null i:5 notnull:11!null + │ │ │ │ │ │ ├── key: (4) + │ │ │ │ │ │ ├── fd: (4)-->(5), (5)-->(11) - │ │ │ │ │ │ ├── scan a - - │ │ │ │ │ │ │ ├── columns: k:3!null i:4 + - │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ │ │ ├── project - + │ │ │ │ │ │ │ ├── columns: notnull:9!null k:3!null i:4 - │ │ │ │ │ │ │ ├── key: (3) - - │ │ │ │ │ │ │ └── fd: (3)-->(4) + + │ │ │ │ │ │ │ ├── columns: notnull:11!null k:4!null i:5 + │ │ │ │ │ │ │ ├── key: (4) + - │ │ │ │ │ │ │ └── fd: (4)-->(5) - │ │ │ │ │ │ └── projections - - │ │ │ │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] - + │ │ │ │ │ │ │ ├── fd: (3)-->(4), (4)-->(9) + - │ │ │ │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] + + │ │ │ │ │ │ │ ├── fd: (4)-->(5), (5)-->(11) + │ │ │ │ │ │ │ ├── scan a - + │ │ │ │ │ │ │ │ ├── columns: k:3!null i:4 - + │ │ │ │ │ │ │ │ ├── key: (3) - + │ │ │ │ │ │ │ │ └── fd: (3)-->(4) + + │ │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + + │ │ │ │ │ │ │ │ ├── key: (4) + + │ │ │ │ │ │ │ │ └── fd: (4)-->(5) + │ │ │ │ │ │ │ └── projections - + │ │ │ │ │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + + │ │ │ │ │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] + │ │ │ │ │ │ └── filters - + │ │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + + │ │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] │ │ │ │ │ └── filters - - │ │ │ │ │ ├── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - - │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] - + │ │ │ │ │ └── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + - │ │ │ │ │ ├── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + - │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] + + │ │ │ │ │ └── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] │ │ │ │ └── aggregations - │ │ │ │ └── bool-or [as=bool_or:10, outer=(9)] - │ │ │ │ └── notnull:9 + │ │ │ │ └── bool-or [as=bool_or:12, outer=(11)] + │ │ │ │ └── notnull:11 │ │ │ └── filters (true) │ │ └── projections - │ │ └── CASE WHEN bool_or:10 THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:11, outer=(10)] + │ │ └── CASE WHEN bool_or:12 THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:13, outer=(12)] │ └── filters (true) └── projections - └── case:11 [as=r:8, outer=(11)] + └── case:13 [as=r:10, outer=(13)] ================================================================================ PushSelectIntoProject Cost: 2268.51 ================================================================================ project - ├── columns: r:8 + ├── columns: r:10 ├── select - │ ├── columns: x:1!null case:11 + │ ├── columns: x:1!null case:13 │ ├── key: (1) - │ ├── fd: (1)-->(11) + │ ├── fd: (1)-->(13) │ ├── project - │ │ ├── columns: case:11 x:1!null + │ │ ├── columns: case:13 x:1!null │ │ ├── key: (1) - │ │ ├── fd: (1)-->(11) + │ │ ├── fd: (1)-->(13) │ │ ├── select - │ │ │ ├── columns: x:1!null bool_or:10 + │ │ │ ├── columns: x:1!null bool_or:12 │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(10) + │ │ │ ├── fd: (1)-->(12) │ │ │ ├── project - │ │ │ │ ├── columns: x:1!null bool_or:10 + │ │ │ │ ├── columns: x:1!null bool_or:12 │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(10) + │ │ │ │ ├── fd: (1)-->(12) │ │ │ │ └── group-by - │ │ │ │ ├── columns: x:1!null bool_or:10 + │ │ │ │ ├── columns: x:1!null bool_or:12 │ │ │ │ ├── grouping columns: x:1!null │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(10) + │ │ │ │ ├── fd: (1)-->(12) │ │ │ │ ├── project - │ │ │ │ │ ├── columns: x:1!null notnull:9 + │ │ │ │ │ ├── columns: x:1!null notnull:11 │ │ │ │ │ ├── key: (1) - │ │ │ │ │ ├── fd: (1)-->(9) + │ │ │ │ │ ├── fd: (1)-->(11) │ │ │ │ │ └── left-join (hash) - │ │ │ │ │ ├── columns: x:1!null k:3 i:4 notnull:9 + │ │ │ │ │ ├── columns: x:1!null k:4 i:5 notnull:11 │ │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ │ │ │ ├── key: (1) - │ │ │ │ │ ├── fd: (3)-->(4), (4)~~>(9), (1)-->(3,4,9) + │ │ │ │ │ ├── fd: (4)-->(5), (5)~~>(11), (1)-->(4,5,11) │ │ │ │ │ ├── scan xy │ │ │ │ │ │ ├── columns: x:1!null │ │ │ │ │ │ └── key: (1) │ │ │ │ │ ├── select - │ │ │ │ │ │ ├── columns: k:3!null i:4 notnull:9!null - │ │ │ │ │ │ ├── key: (3) - │ │ │ │ │ │ ├── fd: (3)-->(4), (4)-->(9) + │ │ │ │ │ │ ├── columns: k:4!null i:5 notnull:11!null + │ │ │ │ │ │ ├── key: (4) + │ │ │ │ │ │ ├── fd: (4)-->(5), (5)-->(11) │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ ├── columns: notnull:9!null k:3!null i:4 - │ │ │ │ │ │ │ ├── key: (3) - │ │ │ │ │ │ │ ├── fd: (3)-->(4), (4)-->(9) + │ │ │ │ │ │ │ ├── columns: notnull:11!null k:4!null i:5 + │ │ │ │ │ │ │ ├── key: (4) + │ │ │ │ │ │ │ ├── fd: (4)-->(5), (5)-->(11) - │ │ │ │ │ │ │ ├── scan a + │ │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ │ ├── columns: k:3!null i:4 - │ │ │ │ │ │ │ │ ├── key: (3) - - │ │ │ │ │ │ │ │ └── fd: (3)-->(4) - + │ │ │ │ │ │ │ │ ├── fd: (3)-->(4) + │ │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ │ │ │ │ ├── key: (4) + - │ │ │ │ │ │ │ │ └── fd: (4)-->(5) + + │ │ │ │ │ │ │ │ ├── fd: (4)-->(5) + │ │ │ │ │ │ │ │ ├── scan a - + │ │ │ │ │ │ │ │ │ ├── columns: k:3!null i:4 - + │ │ │ │ │ │ │ │ │ ├── key: (3) - + │ │ │ │ │ │ │ │ │ └── fd: (3)-->(4) + + │ │ │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + + │ │ │ │ │ │ │ │ │ ├── key: (4) + + │ │ │ │ │ │ │ │ │ └── fd: (4)-->(5) + │ │ │ │ │ │ │ │ └── filters - + │ │ │ │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + + │ │ │ │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] │ │ │ │ │ │ │ └── projections - │ │ │ │ │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + │ │ │ │ │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] - │ │ │ │ │ │ └── filters - - │ │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + - │ │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] + │ │ │ │ │ │ └── filters (true) │ │ │ │ │ └── filters - │ │ │ │ │ └── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + │ │ │ │ │ └── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] │ │ │ │ └── aggregations - │ │ │ │ └── bool-or [as=bool_or:10, outer=(9)] - │ │ │ │ └── notnull:9 + │ │ │ │ └── bool-or [as=bool_or:12, outer=(11)] + │ │ │ │ └── notnull:11 │ │ │ └── filters (true) │ │ └── projections - │ │ └── CASE WHEN bool_or:10 THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:11, outer=(10)] + │ │ └── CASE WHEN bool_or:12 THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:13, outer=(12)] │ └── filters (true) └── projections - └── case:11 [as=r:8, outer=(11)] + └── case:13 [as=r:10, outer=(13)] ================================================================================ EliminateSelect Cost: 2265.17 ================================================================================ project - ├── columns: r:8 + ├── columns: r:10 ├── select - │ ├── columns: x:1!null case:11 + │ ├── columns: x:1!null case:13 │ ├── key: (1) - │ ├── fd: (1)-->(11) + │ ├── fd: (1)-->(13) │ ├── project - │ │ ├── columns: case:11 x:1!null + │ │ ├── columns: case:13 x:1!null │ │ ├── key: (1) - │ │ ├── fd: (1)-->(11) + │ │ ├── fd: (1)-->(13) │ │ ├── select - │ │ │ ├── columns: x:1!null bool_or:10 + │ │ │ ├── columns: x:1!null bool_or:12 │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(10) + │ │ │ ├── fd: (1)-->(12) │ │ │ ├── project - │ │ │ │ ├── columns: x:1!null bool_or:10 + │ │ │ │ ├── columns: x:1!null bool_or:12 │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(10) + │ │ │ │ ├── fd: (1)-->(12) │ │ │ │ └── group-by - │ │ │ │ ├── columns: x:1!null bool_or:10 + │ │ │ │ ├── columns: x:1!null bool_or:12 │ │ │ │ ├── grouping columns: x:1!null │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(10) + │ │ │ │ ├── fd: (1)-->(12) │ │ │ │ ├── project - │ │ │ │ │ ├── columns: x:1!null notnull:9 + │ │ │ │ │ ├── columns: x:1!null notnull:11 │ │ │ │ │ ├── key: (1) - │ │ │ │ │ ├── fd: (1)-->(9) + │ │ │ │ │ ├── fd: (1)-->(11) │ │ │ │ │ └── left-join (hash) - │ │ │ │ │ ├── columns: x:1!null k:3 i:4 notnull:9 + │ │ │ │ │ ├── columns: x:1!null k:4 i:5 notnull:11 │ │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ │ │ │ ├── key: (1) - │ │ │ │ │ ├── fd: (3)-->(4), (4)~~>(9), (1)-->(3,4,9) + │ │ │ │ │ ├── fd: (4)-->(5), (5)~~>(11), (1)-->(4,5,11) │ │ │ │ │ ├── scan xy │ │ │ │ │ │ ├── columns: x:1!null │ │ │ │ │ │ └── key: (1) - │ │ │ │ │ ├── select - - │ │ │ │ │ │ ├── columns: k:3!null i:4 notnull:9!null + - │ │ │ │ │ │ ├── columns: k:4!null i:5 notnull:11!null + │ │ │ │ │ ├── project - + │ │ │ │ │ │ ├── columns: notnull:9!null k:3!null i:4 - │ │ │ │ │ │ ├── key: (3) - │ │ │ │ │ │ ├── fd: (3)-->(4), (4)-->(9) + + │ │ │ │ │ │ ├── columns: notnull:11!null k:4!null i:5 + │ │ │ │ │ │ ├── key: (4) + │ │ │ │ │ │ ├── fd: (4)-->(5), (5)-->(11) - │ │ │ │ │ │ ├── project - - │ │ │ │ │ │ │ ├── columns: notnull:9!null k:3!null i:4 + - │ │ │ │ │ │ │ ├── columns: notnull:11!null k:4!null i:5 + │ │ │ │ │ │ ├── select - + │ │ │ │ │ │ │ ├── columns: k:3!null i:4 - │ │ │ │ │ │ │ ├── key: (3) - - │ │ │ │ │ │ │ ├── fd: (3)-->(4), (4)-->(9) + + │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ │ │ │ ├── key: (4) + - │ │ │ │ │ │ │ ├── fd: (4)-->(5), (5)-->(11) - │ │ │ │ │ │ │ ├── select - + │ │ │ │ │ │ │ ├── fd: (3)-->(4) + + │ │ │ │ │ │ │ ├── fd: (4)-->(5) + │ │ │ │ │ │ │ ├── scan a - │ │ │ │ │ │ │ │ ├── columns: k:3!null i:4 - │ │ │ │ │ │ │ │ ├── key: (3) - - │ │ │ │ │ │ │ │ ├── fd: (3)-->(4) + │ │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ │ │ │ │ ├── key: (4) + - │ │ │ │ │ │ │ │ ├── fd: (4)-->(5) - │ │ │ │ │ │ │ │ ├── scan a - - │ │ │ │ │ │ │ │ │ ├── columns: k:3!null i:4 - - │ │ │ │ │ │ │ │ │ ├── key: (3) - - │ │ │ │ │ │ │ │ │ └── fd: (3)-->(4) + - │ │ │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + - │ │ │ │ │ │ │ │ │ ├── key: (4) + - │ │ │ │ │ │ │ │ │ └── fd: (4)-->(5) - │ │ │ │ │ │ │ │ └── filters - - │ │ │ │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + - │ │ │ │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] - │ │ │ │ │ │ │ └── projections - - │ │ │ │ │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + - │ │ │ │ │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] - │ │ │ │ │ │ └── filters (true) - + │ │ │ │ │ │ │ │ └── fd: (3)-->(4) + + │ │ │ │ │ │ │ │ └── fd: (4)-->(5) + │ │ │ │ │ │ │ └── filters - + │ │ │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + + │ │ │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] + │ │ │ │ │ │ └── projections - + │ │ │ │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + + │ │ │ │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] │ │ │ │ │ └── filters - │ │ │ │ │ └── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + │ │ │ │ │ └── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] │ │ │ │ └── aggregations - │ │ │ │ └── bool-or [as=bool_or:10, outer=(9)] - │ │ │ │ └── notnull:9 + │ │ │ │ └── bool-or [as=bool_or:12, outer=(11)] + │ │ │ │ └── notnull:11 │ │ │ └── filters (true) │ │ └── projections - │ │ └── CASE WHEN bool_or:10 THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:11, outer=(10)] + │ │ └── CASE WHEN bool_or:12 THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:13, outer=(12)] │ └── filters (true) └── projections - └── case:11 [as=r:8, outer=(11)] + └── case:13 [as=r:10, outer=(13)] ================================================================================ PruneJoinRightCols Cost: 2265.17 ================================================================================ project - ├── columns: r:8 + ├── columns: r:10 ├── select - │ ├── columns: x:1!null case:11 + │ ├── columns: x:1!null case:13 │ ├── key: (1) - │ ├── fd: (1)-->(11) + │ ├── fd: (1)-->(13) │ ├── project - │ │ ├── columns: case:11 x:1!null + │ │ ├── columns: case:13 x:1!null │ │ ├── key: (1) - │ │ ├── fd: (1)-->(11) + │ │ ├── fd: (1)-->(13) │ │ ├── select - │ │ │ ├── columns: x:1!null bool_or:10 + │ │ │ ├── columns: x:1!null bool_or:12 │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(10) + │ │ │ ├── fd: (1)-->(12) │ │ │ ├── project - │ │ │ │ ├── columns: x:1!null bool_or:10 + │ │ │ │ ├── columns: x:1!null bool_or:12 │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(10) + │ │ │ │ ├── fd: (1)-->(12) │ │ │ │ └── group-by - │ │ │ │ ├── columns: x:1!null bool_or:10 + │ │ │ │ ├── columns: x:1!null bool_or:12 │ │ │ │ ├── grouping columns: x:1!null │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(10) + │ │ │ │ ├── fd: (1)-->(12) │ │ │ │ ├── project - │ │ │ │ │ ├── columns: x:1!null notnull:9 + │ │ │ │ │ ├── columns: x:1!null notnull:11 │ │ │ │ │ ├── key: (1) - │ │ │ │ │ ├── fd: (1)-->(9) + │ │ │ │ │ ├── fd: (1)-->(11) │ │ │ │ │ └── left-join (hash) - - │ │ │ │ │ ├── columns: x:1!null k:3 i:4 notnull:9 - + │ │ │ │ │ ├── columns: x:1!null k:3 notnull:9 + - │ │ │ │ │ ├── columns: x:1!null k:4 i:5 notnull:11 + + │ │ │ │ │ ├── columns: x:1!null k:4 notnull:11 │ │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ │ │ │ ├── key: (1) - - │ │ │ │ │ ├── fd: (3)-->(4), (4)~~>(9), (1)-->(3,4,9) - + │ │ │ │ │ ├── fd: (3)-->(9), (1)-->(3,9) + - │ │ │ │ │ ├── fd: (4)-->(5), (5)~~>(11), (1)-->(4,5,11) + + │ │ │ │ │ ├── fd: (4)-->(11), (1)-->(4,11) │ │ │ │ │ ├── scan xy │ │ │ │ │ │ ├── columns: x:1!null │ │ │ │ │ │ └── key: (1) │ │ │ │ │ ├── project - - │ │ │ │ │ │ ├── columns: notnull:9!null k:3!null i:4 - + │ │ │ │ │ │ ├── columns: notnull:9!null k:3!null - │ │ │ │ │ │ ├── key: (3) - - │ │ │ │ │ │ ├── fd: (3)-->(4), (4)-->(9) - + │ │ │ │ │ │ ├── fd: (3)-->(9) + - │ │ │ │ │ │ ├── columns: notnull:11!null k:4!null i:5 + + │ │ │ │ │ │ ├── columns: notnull:11!null k:4!null + │ │ │ │ │ │ ├── key: (4) + - │ │ │ │ │ │ ├── fd: (4)-->(5), (5)-->(11) + + │ │ │ │ │ │ ├── fd: (4)-->(11) │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: k:3!null i:4 - │ │ │ │ │ │ │ ├── key: (3) - │ │ │ │ │ │ │ ├── fd: (3)-->(4) + │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ │ │ │ ├── key: (4) + │ │ │ │ │ │ │ ├── fd: (4)-->(5) │ │ │ │ │ │ │ ├── scan a - │ │ │ │ │ │ │ │ ├── columns: k:3!null i:4 - │ │ │ │ │ │ │ │ ├── key: (3) - │ │ │ │ │ │ │ │ └── fd: (3)-->(4) + │ │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ │ │ │ │ ├── key: (4) + │ │ │ │ │ │ │ │ └── fd: (4)-->(5) │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + │ │ │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + │ │ │ │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] │ │ │ │ │ └── filters - │ │ │ │ │ └── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + │ │ │ │ │ └── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] │ │ │ │ └── aggregations - │ │ │ │ └── bool-or [as=bool_or:10, outer=(9)] - │ │ │ │ └── notnull:9 + │ │ │ │ └── bool-or [as=bool_or:12, outer=(11)] + │ │ │ │ └── notnull:11 │ │ │ └── filters (true) │ │ └── projections - │ │ └── CASE WHEN bool_or:10 THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:11, outer=(10)] + │ │ └── CASE WHEN bool_or:12 THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:13, outer=(12)] │ └── filters (true) └── projections - └── case:11 [as=r:8, outer=(11)] + └── case:13 [as=r:10, outer=(13)] ================================================================================ EliminateGroupByProject Cost: 2255.16 ================================================================================ project - ├── columns: r:8 + ├── columns: r:10 ├── select - │ ├── columns: x:1!null case:11 + │ ├── columns: x:1!null case:13 │ ├── key: (1) - │ ├── fd: (1)-->(11) + │ ├── fd: (1)-->(13) │ ├── project - │ │ ├── columns: case:11 x:1!null + │ │ ├── columns: case:13 x:1!null │ │ ├── key: (1) - │ │ ├── fd: (1)-->(11) + │ │ ├── fd: (1)-->(13) │ │ ├── select - │ │ │ ├── columns: x:1!null bool_or:10 + │ │ │ ├── columns: x:1!null bool_or:12 │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(10) + │ │ │ ├── fd: (1)-->(12) │ │ │ ├── project - │ │ │ │ ├── columns: x:1!null bool_or:10 + │ │ │ │ ├── columns: x:1!null bool_or:12 │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(10) + │ │ │ │ ├── fd: (1)-->(12) │ │ │ │ └── group-by - │ │ │ │ ├── columns: x:1!null bool_or:10 + │ │ │ │ ├── columns: x:1!null bool_or:12 │ │ │ │ ├── grouping columns: x:1!null │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(10) + │ │ │ │ ├── fd: (1)-->(12) - │ │ │ │ ├── project - - │ │ │ │ │ ├── columns: x:1!null notnull:9 + - │ │ │ │ │ ├── columns: x:1!null notnull:11 + │ │ │ │ ├── left-join (hash) - + │ │ │ │ │ ├── columns: x:1!null k:3 notnull:9 + + │ │ │ │ │ ├── columns: x:1!null k:4 notnull:11 + │ │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ │ │ │ ├── key: (1) - - │ │ │ │ │ ├── fd: (1)-->(9) + - │ │ │ │ │ ├── fd: (1)-->(11) - │ │ │ │ │ └── left-join (hash) - - │ │ │ │ │ ├── columns: x:1!null k:3 notnull:9 + - │ │ │ │ │ ├── columns: x:1!null k:4 notnull:11 - │ │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) - │ │ │ │ │ ├── key: (1) - - │ │ │ │ │ ├── fd: (3)-->(9), (1)-->(3,9) + - │ │ │ │ │ ├── fd: (4)-->(11), (1)-->(4,11) - │ │ │ │ │ ├── scan xy - │ │ │ │ │ │ ├── columns: x:1!null - │ │ │ │ │ │ └── key: (1) - │ │ │ │ │ ├── project - - │ │ │ │ │ │ ├── columns: notnull:9!null k:3!null - - │ │ │ │ │ │ ├── key: (3) - - │ │ │ │ │ │ ├── fd: (3)-->(9) + - │ │ │ │ │ │ ├── columns: notnull:11!null k:4!null + - │ │ │ │ │ │ ├── key: (4) + - │ │ │ │ │ │ ├── fd: (4)-->(11) - │ │ │ │ │ │ ├── select - - │ │ │ │ │ │ │ ├── columns: k:3!null i:4 - - │ │ │ │ │ │ │ ├── key: (3) - - │ │ │ │ │ │ │ ├── fd: (3)-->(4) + - │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + - │ │ │ │ │ │ │ ├── key: (4) + - │ │ │ │ │ │ │ ├── fd: (4)-->(5) - │ │ │ │ │ │ │ ├── scan a - - │ │ │ │ │ │ │ │ ├── columns: k:3!null i:4 - - │ │ │ │ │ │ │ │ ├── key: (3) - - │ │ │ │ │ │ │ │ └── fd: (3)-->(4) + - │ │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + - │ │ │ │ │ │ │ │ ├── key: (4) + - │ │ │ │ │ │ │ │ └── fd: (4)-->(5) - │ │ │ │ │ │ │ └── filters - - │ │ │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + - │ │ │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] - │ │ │ │ │ │ └── projections - - │ │ │ │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + - │ │ │ │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] - │ │ │ │ │ └── filters - - │ │ │ │ │ └── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - + │ │ │ │ │ ├── fd: (3)-->(9), (1)-->(3,9) + - │ │ │ │ │ └── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + + │ │ │ │ │ ├── fd: (4)-->(11), (1)-->(4,11) + │ │ │ │ │ ├── scan xy + │ │ │ │ │ │ ├── columns: x:1!null + │ │ │ │ │ │ └── key: (1) + │ │ │ │ │ ├── project - + │ │ │ │ │ │ ├── columns: notnull:9!null k:3!null - + │ │ │ │ │ │ ├── key: (3) - + │ │ │ │ │ │ ├── fd: (3)-->(9) + + │ │ │ │ │ │ ├── columns: notnull:11!null k:4!null + + │ │ │ │ │ │ ├── key: (4) + + │ │ │ │ │ │ ├── fd: (4)-->(11) + │ │ │ │ │ │ ├── select - + │ │ │ │ │ │ │ ├── columns: k:3!null i:4 - + │ │ │ │ │ │ │ ├── key: (3) - + │ │ │ │ │ │ │ ├── fd: (3)-->(4) + + │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + + │ │ │ │ │ │ │ ├── key: (4) + + │ │ │ │ │ │ │ ├── fd: (4)-->(5) + │ │ │ │ │ │ │ ├── scan a - + │ │ │ │ │ │ │ │ ├── columns: k:3!null i:4 - + │ │ │ │ │ │ │ │ ├── key: (3) - + │ │ │ │ │ │ │ │ └── fd: (3)-->(4) + + │ │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + + │ │ │ │ │ │ │ │ ├── key: (4) + + │ │ │ │ │ │ │ │ └── fd: (4)-->(5) + │ │ │ │ │ │ │ └── filters - + │ │ │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + + │ │ │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] + │ │ │ │ │ │ └── projections - + │ │ │ │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + + │ │ │ │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] + │ │ │ │ │ └── filters - + │ │ │ │ │ └── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + + │ │ │ │ │ └── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] │ │ │ │ └── aggregations - │ │ │ │ └── bool-or [as=bool_or:10, outer=(9)] - │ │ │ │ └── notnull:9 + │ │ │ │ └── bool-or [as=bool_or:12, outer=(11)] + │ │ │ │ └── notnull:11 │ │ │ └── filters (true) │ │ └── projections - │ │ └── CASE WHEN bool_or:10 THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:11, outer=(10)] + │ │ └── CASE WHEN bool_or:12 THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:13, outer=(12)] │ └── filters (true) └── projections - └── case:11 [as=r:8, outer=(11)] + └── case:13 [as=r:10, outer=(13)] ================================================================================ EliminateProject Cost: 2245.15 ================================================================================ project - ├── columns: r:8 + ├── columns: r:10 ├── select - │ ├── columns: x:1!null case:11 + │ ├── columns: x:1!null case:13 │ ├── key: (1) - │ ├── fd: (1)-->(11) + │ ├── fd: (1)-->(13) │ ├── project - │ │ ├── columns: case:11 x:1!null + │ │ ├── columns: case:13 x:1!null │ │ ├── key: (1) - │ │ ├── fd: (1)-->(11) + │ │ ├── fd: (1)-->(13) │ │ ├── select - │ │ │ ├── columns: x:1!null bool_or:10 + │ │ │ ├── columns: x:1!null bool_or:12 │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(10) + │ │ │ ├── fd: (1)-->(12) - │ │ │ ├── project + │ │ │ ├── group-by - │ │ │ │ ├── columns: x:1!null bool_or:10 + │ │ │ │ ├── columns: x:1!null bool_or:12 + │ │ │ │ ├── grouping columns: x:1!null │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(10) + │ │ │ │ ├── fd: (1)-->(12) - │ │ │ │ └── group-by - - │ │ │ │ ├── columns: x:1!null bool_or:10 + - │ │ │ │ ├── columns: x:1!null bool_or:12 - │ │ │ │ ├── grouping columns: x:1!null - │ │ │ │ ├── key: (1) - - │ │ │ │ ├── fd: (1)-->(10) + - │ │ │ │ ├── fd: (1)-->(12) - │ │ │ │ ├── left-join (hash) - - │ │ │ │ │ ├── columns: x:1!null k:3 notnull:9 + - │ │ │ │ │ ├── columns: x:1!null k:4 notnull:11 - │ │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) - │ │ │ │ │ ├── key: (1) - - │ │ │ │ │ ├── fd: (3)-->(9), (1)-->(3,9) + - │ │ │ │ │ ├── fd: (4)-->(11), (1)-->(4,11) - │ │ │ │ │ ├── scan xy - │ │ │ │ │ │ ├── columns: x:1!null - │ │ │ │ │ │ └── key: (1) - │ │ │ │ │ ├── project - - │ │ │ │ │ │ ├── columns: notnull:9!null k:3!null - - │ │ │ │ │ │ ├── key: (3) - - │ │ │ │ │ │ ├── fd: (3)-->(9) + - │ │ │ │ │ │ ├── columns: notnull:11!null k:4!null + - │ │ │ │ │ │ ├── key: (4) + - │ │ │ │ │ │ ├── fd: (4)-->(11) - │ │ │ │ │ │ ├── select - - │ │ │ │ │ │ │ ├── columns: k:3!null i:4 - - │ │ │ │ │ │ │ ├── key: (3) - - │ │ │ │ │ │ │ ├── fd: (3)-->(4) + - │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + - │ │ │ │ │ │ │ ├── key: (4) + - │ │ │ │ │ │ │ ├── fd: (4)-->(5) - │ │ │ │ │ │ │ ├── scan a - - │ │ │ │ │ │ │ │ ├── columns: k:3!null i:4 - - │ │ │ │ │ │ │ │ ├── key: (3) - - │ │ │ │ │ │ │ │ └── fd: (3)-->(4) + - │ │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + - │ │ │ │ │ │ │ │ ├── key: (4) + - │ │ │ │ │ │ │ │ └── fd: (4)-->(5) - │ │ │ │ │ │ │ └── filters - - │ │ │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + - │ │ │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] - │ │ │ │ │ │ └── projections - - │ │ │ │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + - │ │ │ │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] - │ │ │ │ │ └── filters - - │ │ │ │ │ └── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + - │ │ │ │ │ └── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] - │ │ │ │ └── aggregations - - │ │ │ │ └── bool-or [as=bool_or:10, outer=(9)] - - │ │ │ │ └── notnull:9 + - │ │ │ │ └── bool-or [as=bool_or:12, outer=(11)] + - │ │ │ │ └── notnull:11 + │ │ │ │ ├── left-join (hash) - + │ │ │ │ │ ├── columns: x:1!null k:3 notnull:9 + + │ │ │ │ │ ├── columns: x:1!null k:4 notnull:11 + │ │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) + │ │ │ │ │ ├── key: (1) - + │ │ │ │ │ ├── fd: (3)-->(9), (1)-->(3,9) + + │ │ │ │ │ ├── fd: (4)-->(11), (1)-->(4,11) + │ │ │ │ │ ├── scan xy + │ │ │ │ │ │ ├── columns: x:1!null + │ │ │ │ │ │ └── key: (1) + │ │ │ │ │ ├── project - + │ │ │ │ │ │ ├── columns: notnull:9!null k:3!null - + │ │ │ │ │ │ ├── key: (3) - + │ │ │ │ │ │ ├── fd: (3)-->(9) + + │ │ │ │ │ │ ├── columns: notnull:11!null k:4!null + + │ │ │ │ │ │ ├── key: (4) + + │ │ │ │ │ │ ├── fd: (4)-->(11) + │ │ │ │ │ │ ├── select - + │ │ │ │ │ │ │ ├── columns: k:3!null i:4 - + │ │ │ │ │ │ │ ├── key: (3) - + │ │ │ │ │ │ │ ├── fd: (3)-->(4) + + │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + + │ │ │ │ │ │ │ ├── key: (4) + + │ │ │ │ │ │ │ ├── fd: (4)-->(5) + │ │ │ │ │ │ │ ├── scan a - + │ │ │ │ │ │ │ │ ├── columns: k:3!null i:4 - + │ │ │ │ │ │ │ │ ├── key: (3) - + │ │ │ │ │ │ │ │ └── fd: (3)-->(4) + + │ │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + + │ │ │ │ │ │ │ │ ├── key: (4) + + │ │ │ │ │ │ │ │ └── fd: (4)-->(5) + │ │ │ │ │ │ │ └── filters - + │ │ │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + + │ │ │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] + │ │ │ │ │ │ └── projections - + │ │ │ │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + + │ │ │ │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] + │ │ │ │ │ └── filters - + │ │ │ │ │ └── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + + │ │ │ │ │ └── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + │ │ │ │ └── aggregations - + │ │ │ │ └── bool-or [as=bool_or:10, outer=(9)] - + │ │ │ │ └── notnull:9 + + │ │ │ │ └── bool-or [as=bool_or:12, outer=(11)] + + │ │ │ │ └── notnull:11 │ │ │ └── filters (true) │ │ └── projections - │ │ └── CASE WHEN bool_or:10 THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:11, outer=(10)] + │ │ └── CASE WHEN bool_or:12 THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:13, outer=(12)] │ └── filters (true) └── projections - └── case:11 [as=r:8, outer=(11)] + └── case:13 [as=r:10, outer=(13)] ================================================================================ EliminateSelect Cost: 2235.14 ================================================================================ project - ├── columns: r:8 + ├── columns: r:10 ├── select - │ ├── columns: x:1!null case:11 + │ ├── columns: x:1!null case:13 │ ├── key: (1) - │ ├── fd: (1)-->(11) + │ ├── fd: (1)-->(13) │ ├── project - │ │ ├── columns: case:11 x:1!null + │ │ ├── columns: case:13 x:1!null │ │ ├── key: (1) - │ │ ├── fd: (1)-->(11) + │ │ ├── fd: (1)-->(13) - │ │ ├── select + │ │ ├── group-by - │ │ │ ├── columns: x:1!null bool_or:10 + │ │ │ ├── columns: x:1!null bool_or:12 + │ │ │ ├── grouping columns: x:1!null │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(10) + │ │ │ ├── fd: (1)-->(12) - │ │ │ ├── group-by - - │ │ │ │ ├── columns: x:1!null bool_or:10 + - │ │ │ │ ├── columns: x:1!null bool_or:12 - │ │ │ │ ├── grouping columns: x:1!null + │ │ │ ├── left-join (hash) - + │ │ │ │ ├── columns: x:1!null k:3 notnull:9 + + │ │ │ │ ├── columns: x:1!null k:4 notnull:11 + │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ │ │ ├── key: (1) - - │ │ │ │ ├── fd: (1)-->(10) + - │ │ │ │ ├── fd: (1)-->(12) - │ │ │ │ ├── left-join (hash) - - │ │ │ │ │ ├── columns: x:1!null k:3 notnull:9 + - │ │ │ │ │ ├── columns: x:1!null k:4 notnull:11 - │ │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) - │ │ │ │ │ ├── key: (1) - - │ │ │ │ │ ├── fd: (3)-->(9), (1)-->(3,9) + - │ │ │ │ │ ├── fd: (4)-->(11), (1)-->(4,11) - │ │ │ │ │ ├── scan xy - │ │ │ │ │ │ ├── columns: x:1!null - │ │ │ │ │ │ └── key: (1) - │ │ │ │ │ ├── project - - │ │ │ │ │ │ ├── columns: notnull:9!null k:3!null - + │ │ │ │ ├── fd: (3)-->(9), (1)-->(3,9) + - │ │ │ │ │ │ ├── columns: notnull:11!null k:4!null + + │ │ │ │ ├── fd: (4)-->(11), (1)-->(4,11) + │ │ │ │ ├── scan xy + │ │ │ │ │ ├── columns: x:1!null + │ │ │ │ │ └── key: (1) + │ │ │ │ ├── project - + │ │ │ │ │ ├── columns: notnull:9!null k:3!null - + │ │ │ │ │ ├── key: (3) - + │ │ │ │ │ ├── fd: (3)-->(9) + + │ │ │ │ │ ├── columns: notnull:11!null k:4!null + + │ │ │ │ │ ├── key: (4) + + │ │ │ │ │ ├── fd: (4)-->(11) + │ │ │ │ │ ├── select - + │ │ │ │ │ │ ├── columns: k:3!null i:4 - │ │ │ │ │ │ ├── key: (3) - - │ │ │ │ │ │ ├── fd: (3)-->(9) + + │ │ │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ │ │ ├── key: (4) + - │ │ │ │ │ │ ├── fd: (4)-->(11) - │ │ │ │ │ │ ├── select - + │ │ │ │ │ │ ├── fd: (3)-->(4) + + │ │ │ │ │ │ ├── fd: (4)-->(5) + │ │ │ │ │ │ ├── scan a - │ │ │ │ │ │ │ ├── columns: k:3!null i:4 - │ │ │ │ │ │ │ ├── key: (3) - - │ │ │ │ │ │ │ ├── fd: (3)-->(4) + │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ │ │ │ ├── key: (4) + - │ │ │ │ │ │ │ ├── fd: (4)-->(5) - │ │ │ │ │ │ │ ├── scan a - - │ │ │ │ │ │ │ │ ├── columns: k:3!null i:4 - - │ │ │ │ │ │ │ │ ├── key: (3) - - │ │ │ │ │ │ │ │ └── fd: (3)-->(4) + - │ │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + - │ │ │ │ │ │ │ │ ├── key: (4) + - │ │ │ │ │ │ │ │ └── fd: (4)-->(5) - │ │ │ │ │ │ │ └── filters - - │ │ │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + - │ │ │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] - │ │ │ │ │ │ └── projections - - │ │ │ │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + - │ │ │ │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] - │ │ │ │ │ └── filters - - │ │ │ │ │ └── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + - │ │ │ │ │ └── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] - │ │ │ │ └── aggregations - - │ │ │ │ └── bool-or [as=bool_or:10, outer=(9)] - - │ │ │ │ └── notnull:9 + - │ │ │ │ └── bool-or [as=bool_or:12, outer=(11)] + - │ │ │ │ └── notnull:11 - │ │ │ └── filters (true) - + │ │ │ │ │ │ │ └── fd: (3)-->(4) + + │ │ │ │ │ │ │ └── fd: (4)-->(5) + │ │ │ │ │ │ └── filters - + │ │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + + │ │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] + │ │ │ │ │ └── projections - + │ │ │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + + │ │ │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] + │ │ │ │ └── filters - + │ │ │ │ └── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + + │ │ │ │ └── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + │ │ │ └── aggregations - + │ │ │ └── bool-or [as=bool_or:10, outer=(9)] - + │ │ │ └── notnull:9 + + │ │ │ └── bool-or [as=bool_or:12, outer=(11)] + + │ │ │ └── notnull:11 │ │ └── projections - │ │ └── CASE WHEN bool_or:10 THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:11, outer=(10)] + │ │ └── CASE WHEN bool_or:12 THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:13, outer=(12)] │ └── filters (true) └── projections - └── case:11 [as=r:8, outer=(11)] + └── case:13 [as=r:10, outer=(13)] ================================================================================ EliminateSelect Cost: 2225.13 ================================================================================ project - ├── columns: r:8 + ├── columns: r:10 - ├── select - - │ ├── columns: x:1!null case:11 + - │ ├── columns: x:1!null case:13 + ├── project - + │ ├── columns: case:11 x:1!null + + │ ├── columns: case:13 x:1!null │ ├── key: (1) - │ ├── fd: (1)-->(11) + │ ├── fd: (1)-->(13) - │ ├── project - - │ │ ├── columns: case:11 x:1!null + - │ │ ├── columns: case:13 x:1!null + │ ├── group-by - + │ │ ├── columns: x:1!null bool_or:10 + + │ │ ├── columns: x:1!null bool_or:12 + │ │ ├── grouping columns: x:1!null │ │ ├── key: (1) - - │ │ ├── fd: (1)-->(11) + - │ │ ├── fd: (1)-->(13) - │ │ ├── group-by - - │ │ │ ├── columns: x:1!null bool_or:10 + - │ │ │ ├── columns: x:1!null bool_or:12 - │ │ │ ├── grouping columns: x:1!null - + │ │ ├── fd: (1)-->(10) + + │ │ ├── fd: (1)-->(12) + │ │ ├── left-join (hash) - + │ │ │ ├── columns: x:1!null k:3 notnull:9 + + │ │ │ ├── columns: x:1!null k:4 notnull:11 + │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ │ ├── key: (1) - - │ │ │ ├── fd: (1)-->(10) + - │ │ │ ├── fd: (1)-->(12) - │ │ │ ├── left-join (hash) - - │ │ │ │ ├── columns: x:1!null k:3 notnull:9 + - │ │ │ │ ├── columns: x:1!null k:4 notnull:11 - │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) - │ │ │ │ ├── key: (1) - - │ │ │ │ ├── fd: (3)-->(9), (1)-->(3,9) + - │ │ │ │ ├── fd: (4)-->(11), (1)-->(4,11) - │ │ │ │ ├── scan xy - │ │ │ │ │ ├── columns: x:1!null - │ │ │ │ │ └── key: (1) - │ │ │ │ ├── project - - │ │ │ │ │ ├── columns: notnull:9!null k:3!null - + │ │ │ ├── fd: (3)-->(9), (1)-->(3,9) + - │ │ │ │ │ ├── columns: notnull:11!null k:4!null + + │ │ │ ├── fd: (4)-->(11), (1)-->(4,11) + │ │ │ ├── scan xy + │ │ │ │ ├── columns: x:1!null + │ │ │ │ └── key: (1) + │ │ │ ├── project - + │ │ │ │ ├── columns: notnull:9!null k:3!null - + │ │ │ │ ├── key: (3) - + │ │ │ │ ├── fd: (3)-->(9) + + │ │ │ │ ├── columns: notnull:11!null k:4!null + + │ │ │ │ ├── key: (4) + + │ │ │ │ ├── fd: (4)-->(11) + │ │ │ │ ├── select - + │ │ │ │ │ ├── columns: k:3!null i:4 - │ │ │ │ │ ├── key: (3) - - │ │ │ │ │ ├── fd: (3)-->(9) + + │ │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ │ ├── key: (4) + - │ │ │ │ │ ├── fd: (4)-->(11) - │ │ │ │ │ ├── select - + │ │ │ │ │ ├── fd: (3)-->(4) + + │ │ │ │ │ ├── fd: (4)-->(5) + │ │ │ │ │ ├── scan a - │ │ │ │ │ │ ├── columns: k:3!null i:4 - │ │ │ │ │ │ ├── key: (3) - - │ │ │ │ │ │ ├── fd: (3)-->(4) + │ │ │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ │ │ ├── key: (4) + - │ │ │ │ │ │ ├── fd: (4)-->(5) - │ │ │ │ │ │ ├── scan a - - │ │ │ │ │ │ │ ├── columns: k:3!null i:4 - - │ │ │ │ │ │ │ ├── key: (3) - - │ │ │ │ │ │ │ └── fd: (3)-->(4) + - │ │ │ │ │ │ │ ├── columns: k:4!null i:5 + - │ │ │ │ │ │ │ ├── key: (4) + - │ │ │ │ │ │ │ └── fd: (4)-->(5) - │ │ │ │ │ │ └── filters - - │ │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + - │ │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] - │ │ │ │ │ └── projections - - │ │ │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + - │ │ │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] - │ │ │ │ └── filters - - │ │ │ │ └── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + - │ │ │ │ └── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] - │ │ │ └── aggregations - - │ │ │ └── bool-or [as=bool_or:10, outer=(9)] - - │ │ │ └── notnull:9 + - │ │ │ └── bool-or [as=bool_or:12, outer=(11)] + - │ │ │ └── notnull:11 - │ │ └── projections - - │ │ └── CASE WHEN bool_or:10 THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:11, outer=(10)] + - │ │ └── CASE WHEN bool_or:12 THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:13, outer=(12)] - │ └── filters (true) - + │ │ │ │ │ │ └── fd: (3)-->(4) + + │ │ │ │ │ │ └── fd: (4)-->(5) + │ │ │ │ │ └── filters - + │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + + │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] + │ │ │ │ └── projections - + │ │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + + │ │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] + │ │ │ └── filters - + │ │ │ └── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + + │ │ │ └── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + │ │ └── aggregations - + │ │ └── bool-or [as=bool_or:10, outer=(9)] - + │ │ └── notnull:9 + + │ │ └── bool-or [as=bool_or:12, outer=(11)] + + │ │ └── notnull:11 + │ └── projections - + │ └── CASE WHEN bool_or:10 THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:11, outer=(10)] + + │ └── CASE WHEN bool_or:12 THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:13, outer=(12)] └── projections - └── case:11 [as=r:8, outer=(11)] + └── case:13 [as=r:10, outer=(13)] ================================================================================ PruneProjectCols Cost: 2225.13 ================================================================================ project - ├── columns: r:8 + ├── columns: r:10 ├── project - - │ ├── columns: case:11 x:1!null + - │ ├── columns: case:13 x:1!null - │ ├── key: (1) - - │ ├── fd: (1)-->(11) - + │ ├── columns: case:11 + - │ ├── fd: (1)-->(13) + + │ ├── columns: case:13 │ ├── group-by - │ │ ├── columns: x:1!null bool_or:10 + │ │ ├── columns: x:1!null bool_or:12 │ │ ├── grouping columns: x:1!null │ │ ├── key: (1) - │ │ ├── fd: (1)-->(10) + │ │ ├── fd: (1)-->(12) │ │ ├── left-join (hash) - │ │ │ ├── columns: x:1!null k:3 notnull:9 + │ │ │ ├── columns: x:1!null k:4 notnull:11 │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ │ ├── key: (1) - │ │ │ ├── fd: (3)-->(9), (1)-->(3,9) + │ │ │ ├── fd: (4)-->(11), (1)-->(4,11) │ │ │ ├── scan xy │ │ │ │ ├── columns: x:1!null │ │ │ │ └── key: (1) │ │ │ ├── project - │ │ │ │ ├── columns: notnull:9!null k:3!null - │ │ │ │ ├── key: (3) - │ │ │ │ ├── fd: (3)-->(9) + │ │ │ │ ├── columns: notnull:11!null k:4!null + │ │ │ │ ├── key: (4) + │ │ │ │ ├── fd: (4)-->(11) │ │ │ │ ├── select - │ │ │ │ │ ├── columns: k:3!null i:4 - │ │ │ │ │ ├── key: (3) - │ │ │ │ │ ├── fd: (3)-->(4) + │ │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ │ ├── key: (4) + │ │ │ │ │ ├── fd: (4)-->(5) │ │ │ │ │ ├── scan a - │ │ │ │ │ │ ├── columns: k:3!null i:4 - │ │ │ │ │ │ ├── key: (3) - │ │ │ │ │ │ └── fd: (3)-->(4) + │ │ │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ │ │ ├── key: (4) + │ │ │ │ │ │ └── fd: (4)-->(5) │ │ │ │ │ └── filters - │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] │ │ │ │ └── projections - │ │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + │ │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] │ │ │ └── filters - │ │ │ └── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + │ │ │ └── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] │ │ └── aggregations - │ │ └── bool-or [as=bool_or:10, outer=(9)] - │ │ └── notnull:9 + │ │ └── bool-or [as=bool_or:12, outer=(11)] + │ │ └── notnull:11 │ └── projections - │ └── CASE WHEN bool_or:10 THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:11, outer=(10)] + │ └── CASE WHEN bool_or:12 THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:13, outer=(12)] └── projections - └── case:11 [as=r:8, outer=(11)] + └── case:13 [as=r:10, outer=(13)] ================================================================================ InlineProjectInProject Cost: 2205.12 ================================================================================ project - ├── columns: r:8 + ├── columns: r:10 - ├── project - - │ ├── columns: case:11 + - │ ├── columns: case:13 - │ ├── group-by - - │ │ ├── columns: x:1!null bool_or:10 + - │ │ ├── columns: x:1!null bool_or:12 - │ │ ├── grouping columns: x:1!null + ├── group-by - + │ ├── columns: x:1!null bool_or:10 + + │ ├── columns: x:1!null bool_or:12 + │ ├── grouping columns: x:1!null + │ ├── key: (1) - + │ ├── fd: (1)-->(10) + + │ ├── fd: (1)-->(12) + │ ├── left-join (hash) - + │ │ ├── columns: x:1!null k:3 notnull:9 + + │ │ ├── columns: x:1!null k:4 notnull:11 + │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ ├── key: (1) - - │ │ ├── fd: (1)-->(10) + - │ │ ├── fd: (1)-->(12) - │ │ ├── left-join (hash) - - │ │ │ ├── columns: x:1!null k:3 notnull:9 + - │ │ │ ├── columns: x:1!null k:4 notnull:11 - │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) - │ │ │ ├── key: (1) - - │ │ │ ├── fd: (3)-->(9), (1)-->(3,9) + - │ │ │ ├── fd: (4)-->(11), (1)-->(4,11) - │ │ │ ├── scan xy - │ │ │ │ ├── columns: x:1!null - │ │ │ │ └── key: (1) - │ │ │ ├── project - - │ │ │ │ ├── columns: notnull:9!null k:3!null - + │ │ ├── fd: (3)-->(9), (1)-->(3,9) + - │ │ │ │ ├── columns: notnull:11!null k:4!null + + │ │ ├── fd: (4)-->(11), (1)-->(4,11) + │ │ ├── scan xy + │ │ │ ├── columns: x:1!null + │ │ │ └── key: (1) + │ │ ├── project - + │ │ │ ├── columns: notnull:9!null k:3!null - + │ │ │ ├── key: (3) - + │ │ │ ├── fd: (3)-->(9) + + │ │ │ ├── columns: notnull:11!null k:4!null + + │ │ │ ├── key: (4) + + │ │ │ ├── fd: (4)-->(11) + │ │ │ ├── select - + │ │ │ │ ├── columns: k:3!null i:4 - │ │ │ │ ├── key: (3) - - │ │ │ │ ├── fd: (3)-->(9) + + │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ ├── key: (4) + - │ │ │ │ ├── fd: (4)-->(11) - │ │ │ │ ├── select - + │ │ │ │ ├── fd: (3)-->(4) + + │ │ │ │ ├── fd: (4)-->(5) + │ │ │ │ ├── scan a - │ │ │ │ │ ├── columns: k:3!null i:4 - │ │ │ │ │ ├── key: (3) - - │ │ │ │ │ ├── fd: (3)-->(4) + │ │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ │ ├── key: (4) + - │ │ │ │ │ ├── fd: (4)-->(5) - │ │ │ │ │ ├── scan a - - │ │ │ │ │ │ ├── columns: k:3!null i:4 - - │ │ │ │ │ │ ├── key: (3) - - │ │ │ │ │ │ └── fd: (3)-->(4) + - │ │ │ │ │ │ ├── columns: k:4!null i:5 + - │ │ │ │ │ │ ├── key: (4) + - │ │ │ │ │ │ └── fd: (4)-->(5) - │ │ │ │ │ └── filters - - │ │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + - │ │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] - │ │ │ │ └── projections - - │ │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + - │ │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] - │ │ │ └── filters - - │ │ │ └── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + - │ │ │ └── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] - │ │ └── aggregations - - │ │ └── bool-or [as=bool_or:10, outer=(9)] - - │ │ └── notnull:9 + - │ │ └── bool-or [as=bool_or:12, outer=(11)] + - │ │ └── notnull:11 - │ └── projections - - │ └── CASE WHEN bool_or:10 THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:11, outer=(10)] - + │ │ │ │ │ └── fd: (3)-->(4) + - │ └── CASE WHEN bool_or:12 THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:13, outer=(12)] + + │ │ │ │ │ └── fd: (4)-->(5) + │ │ │ │ └── filters - + │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + + │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] + │ │ │ └── projections - + │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + + │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] + │ │ └── filters - + │ │ └── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + + │ │ └── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + │ └── aggregations - + │ └── bool-or [as=bool_or:10, outer=(9)] - + │ └── notnull:9 + + │ └── bool-or [as=bool_or:12, outer=(11)] + + │ └── notnull:11 └── projections - - └── case:11 [as=r:8, outer=(11)] - + └── CASE WHEN bool_or:10 THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=r:8, outer=(10)] + - └── case:13 [as=r:10, outer=(13)] + + └── CASE WHEN bool_or:12 THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=r:10, outer=(12)] -------------------------------------------------------------------------------- GenerateIndexScans (no changes) -------------------------------------------------------------------------------- @@ -2724,244 +2887,244 @@ GenerateConstrainedScans (no changes) CommuteLeftJoin (higher cost) -------------------------------------------------------------------------------- project - ├── columns: r:8 + ├── columns: r:10 ├── group-by - │ ├── columns: x:1!null bool_or:10 + │ ├── columns: x:1!null bool_or:12 │ ├── grouping columns: x:1!null │ ├── key: (1) - │ ├── fd: (1)-->(10) + │ ├── fd: (1)-->(12) - │ ├── left-join (hash) + │ ├── right-join (hash) - │ │ ├── columns: x:1!null k:3 notnull:9 + │ │ ├── columns: x:1!null k:4 notnull:11 - │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ ├── key: (1) - │ │ ├── fd: (3)-->(9), (1)-->(3,9) + │ │ ├── fd: (4)-->(11), (1)-->(4,11) - │ │ ├── scan xy - │ │ │ ├── columns: x:1!null - │ │ │ └── key: (1) │ │ ├── project - │ │ │ ├── columns: notnull:9!null k:3!null - │ │ │ ├── key: (3) - │ │ │ ├── fd: (3)-->(9) + │ │ │ ├── columns: notnull:11!null k:4!null + │ │ │ ├── key: (4) + │ │ │ ├── fd: (4)-->(11) │ │ │ ├── select - │ │ │ │ ├── columns: k:3!null i:4 - │ │ │ │ ├── key: (3) - │ │ │ │ ├── fd: (3)-->(4) + │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ ├── key: (4) + │ │ │ │ ├── fd: (4)-->(5) │ │ │ │ ├── scan a - │ │ │ │ │ ├── columns: k:3!null i:4 - │ │ │ │ │ ├── key: (3) - │ │ │ │ │ └── fd: (3)-->(4) + │ │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ │ ├── key: (4) + │ │ │ │ │ └── fd: (4)-->(5) │ │ │ │ └── filters - │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] │ │ │ └── projections - │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] + │ │ ├── scan xy + │ │ │ ├── columns: x:1!null + │ │ │ └── key: (1) │ │ └── filters - │ │ └── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + │ │ └── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] │ └── aggregations - │ └── bool-or [as=bool_or:10, outer=(9)] - │ └── notnull:9 + │ └── bool-or [as=bool_or:12, outer=(11)] + │ └── notnull:11 └── projections - └── CASE WHEN bool_or:10 THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=r:8, outer=(10)] + └── CASE WHEN bool_or:12 THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=r:10, outer=(12)] ================================================================================ GenerateMergeJoins Cost: 2200.11 ================================================================================ project - ├── columns: r:8 + ├── columns: r:10 ├── group-by - │ ├── columns: x:1!null bool_or:10 + │ ├── columns: x:1!null bool_or:12 │ ├── grouping columns: x:1!null │ ├── key: (1) - │ ├── fd: (1)-->(10) + │ ├── fd: (1)-->(12) - │ ├── left-join (hash) + │ ├── left-join (merge) - │ │ ├── columns: x:1!null k:3 notnull:9 + │ │ ├── columns: x:1!null k:4 notnull:11 - │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) + │ │ ├── left ordering: +1 - + │ │ ├── right ordering: +3 + + │ │ ├── right ordering: +4 │ │ ├── key: (1) - │ │ ├── fd: (3)-->(9), (1)-->(3,9) + │ │ ├── fd: (4)-->(11), (1)-->(4,11) │ │ ├── scan xy │ │ │ ├── columns: x:1!null - │ │ │ └── key: (1) + │ │ │ ├── key: (1) + │ │ │ └── ordering: +1 │ │ ├── project - │ │ │ ├── columns: notnull:9!null k:3!null - │ │ │ ├── key: (3) - │ │ │ ├── fd: (3)-->(9) - + │ │ │ ├── ordering: +3 + │ │ │ ├── columns: notnull:11!null k:4!null + │ │ │ ├── key: (4) + │ │ │ ├── fd: (4)-->(11) + + │ │ │ ├── ordering: +4 │ │ │ ├── select - │ │ │ │ ├── columns: k:3!null i:4 - │ │ │ │ ├── key: (3) - │ │ │ │ ├── fd: (3)-->(4) - + │ │ │ │ ├── ordering: +3 + │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ ├── key: (4) + │ │ │ │ ├── fd: (4)-->(5) + + │ │ │ │ ├── ordering: +4 │ │ │ │ ├── scan a - │ │ │ │ │ ├── columns: k:3!null i:4 - │ │ │ │ │ ├── key: (3) - - │ │ │ │ │ └── fd: (3)-->(4) - + │ │ │ │ │ ├── fd: (3)-->(4) - + │ │ │ │ │ └── ordering: +3 + │ │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ │ ├── key: (4) + - │ │ │ │ │ └── fd: (4)-->(5) + + │ │ │ │ │ ├── fd: (4)-->(5) + + │ │ │ │ │ └── ordering: +4 │ │ │ │ └── filters - │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] │ │ │ └── projections - │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] - │ │ └── filters - - │ │ └── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + - │ │ └── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + │ │ └── filters (true) │ └── aggregations - │ └── bool-or [as=bool_or:10, outer=(9)] - │ └── notnull:9 + │ └── bool-or [as=bool_or:12, outer=(11)] + │ └── notnull:11 └── projections - └── CASE WHEN bool_or:10 THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=r:8, outer=(10)] + └── CASE WHEN bool_or:12 THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=r:10, outer=(12)] -------------------------------------------------------------------------------- GenerateMergeJoins (higher cost) -------------------------------------------------------------------------------- project - ├── columns: r:8 + ├── columns: r:10 ├── group-by - │ ├── columns: x:1!null bool_or:10 + │ ├── columns: x:1!null bool_or:12 │ ├── grouping columns: x:1!null │ ├── key: (1) - │ ├── fd: (1)-->(10) + │ ├── fd: (1)-->(12) - │ ├── left-join (merge) + │ ├── right-join (merge) - │ │ ├── columns: x:1!null k:3 notnull:9 + │ │ ├── columns: x:1!null k:4 notnull:11 - │ │ ├── left ordering: +1 - - │ │ ├── right ordering: +3 - + │ │ ├── left ordering: +3 + - │ │ ├── right ordering: +4 + + │ │ ├── left ordering: +4 + │ │ ├── right ordering: +1 │ │ ├── key: (1) - │ │ ├── fd: (3)-->(9), (1)-->(3,9) + │ │ ├── fd: (4)-->(11), (1)-->(4,11) - │ │ ├── scan xy - │ │ │ ├── columns: x:1!null - │ │ │ ├── key: (1) - │ │ │ └── ordering: +1 │ │ ├── project - │ │ │ ├── columns: notnull:9!null k:3!null - │ │ │ ├── key: (3) - │ │ │ ├── fd: (3)-->(9) - │ │ │ ├── ordering: +3 + │ │ │ ├── columns: notnull:11!null k:4!null + │ │ │ ├── key: (4) + │ │ │ ├── fd: (4)-->(11) + │ │ │ ├── ordering: +4 │ │ │ ├── select - │ │ │ │ ├── columns: k:3!null i:4 - │ │ │ │ ├── key: (3) - │ │ │ │ ├── fd: (3)-->(4) - │ │ │ │ ├── ordering: +3 + │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ ├── key: (4) + │ │ │ │ ├── fd: (4)-->(5) + │ │ │ │ ├── ordering: +4 │ │ │ │ ├── scan a - │ │ │ │ │ ├── columns: k:3!null i:4 - │ │ │ │ │ ├── key: (3) - │ │ │ │ │ ├── fd: (3)-->(4) - │ │ │ │ │ └── ordering: +3 + │ │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ │ ├── key: (4) + │ │ │ │ │ ├── fd: (4)-->(5) + │ │ │ │ │ └── ordering: +4 │ │ │ │ └── filters - │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] │ │ │ └── projections - │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] + │ │ ├── scan xy + │ │ │ ├── columns: x:1!null + │ │ │ ├── key: (1) + │ │ │ └── ordering: +1 │ │ └── filters (true) │ └── aggregations - │ └── bool-or [as=bool_or:10, outer=(9)] - │ └── notnull:9 + │ └── bool-or [as=bool_or:12, outer=(11)] + │ └── notnull:11 └── projections - └── CASE WHEN bool_or:10 THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=r:8, outer=(10)] + └── CASE WHEN bool_or:12 THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=r:10, outer=(12)] ================================================================================ GenerateStreamingGroupBy Cost: 2190.11 ================================================================================ project - ├── columns: r:8 + ├── columns: r:10 ├── group-by - │ ├── columns: x:1!null bool_or:10 + │ ├── columns: x:1!null bool_or:12 │ ├── grouping columns: x:1!null + │ ├── internal-ordering: +1 │ ├── key: (1) - │ ├── fd: (1)-->(10) + │ ├── fd: (1)-->(12) │ ├── left-join (merge) - │ │ ├── columns: x:1!null k:3 notnull:9 + │ │ ├── columns: x:1!null k:4 notnull:11 │ │ ├── left ordering: +1 - │ │ ├── right ordering: +3 + │ │ ├── right ordering: +4 │ │ ├── key: (1) - │ │ ├── fd: (3)-->(9), (1)-->(3,9) + │ │ ├── fd: (4)-->(11), (1)-->(4,11) + │ │ ├── ordering: +1 │ │ ├── scan xy │ │ │ ├── columns: x:1!null │ │ │ ├── key: (1) │ │ │ └── ordering: +1 │ │ ├── project - │ │ │ ├── columns: notnull:9!null k:3!null - │ │ │ ├── key: (3) - │ │ │ ├── fd: (3)-->(9) - │ │ │ ├── ordering: +3 + │ │ │ ├── columns: notnull:11!null k:4!null + │ │ │ ├── key: (4) + │ │ │ ├── fd: (4)-->(11) + │ │ │ ├── ordering: +4 │ │ │ ├── select - │ │ │ │ ├── columns: k:3!null i:4 - │ │ │ │ ├── key: (3) - │ │ │ │ ├── fd: (3)-->(4) - │ │ │ │ ├── ordering: +3 + │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ ├── key: (4) + │ │ │ │ ├── fd: (4)-->(5) + │ │ │ │ ├── ordering: +4 │ │ │ │ ├── scan a - │ │ │ │ │ ├── columns: k:3!null i:4 - │ │ │ │ │ ├── key: (3) - │ │ │ │ │ ├── fd: (3)-->(4) - │ │ │ │ │ └── ordering: +3 + │ │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ │ ├── key: (4) + │ │ │ │ │ ├── fd: (4)-->(5) + │ │ │ │ │ └── ordering: +4 │ │ │ │ └── filters - │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] │ │ │ └── projections - │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] │ │ └── filters (true) │ └── aggregations - │ └── bool-or [as=bool_or:10, outer=(9)] - │ └── notnull:9 + │ └── bool-or [as=bool_or:12, outer=(11)] + │ └── notnull:11 └── projections - └── CASE WHEN bool_or:10 THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=r:8, outer=(10)] + └── CASE WHEN bool_or:12 THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=r:10, outer=(12)] ================================================================================ Final best expression Cost: 2190.11 ================================================================================ project - ├── columns: r:8 + ├── columns: r:10 ├── group-by - │ ├── columns: x:1!null bool_or:10 + │ ├── columns: x:1!null bool_or:12 │ ├── grouping columns: x:1!null │ ├── internal-ordering: +1 │ ├── key: (1) - │ ├── fd: (1)-->(10) + │ ├── fd: (1)-->(12) │ ├── left-join (merge) - │ │ ├── columns: x:1!null k:3 notnull:9 + │ │ ├── columns: x:1!null k:4 notnull:11 │ │ ├── left ordering: +1 - │ │ ├── right ordering: +3 + │ │ ├── right ordering: +4 │ │ ├── key: (1) - │ │ ├── fd: (3)-->(9), (1)-->(3,9) + │ │ ├── fd: (4)-->(11), (1)-->(4,11) │ │ ├── ordering: +1 │ │ ├── scan xy │ │ │ ├── columns: x:1!null │ │ │ ├── key: (1) │ │ │ └── ordering: +1 │ │ ├── project - │ │ │ ├── columns: notnull:9!null k:3!null - │ │ │ ├── key: (3) - │ │ │ ├── fd: (3)-->(9) - │ │ │ ├── ordering: +3 + │ │ │ ├── columns: notnull:11!null k:4!null + │ │ │ ├── key: (4) + │ │ │ ├── fd: (4)-->(11) + │ │ │ ├── ordering: +4 │ │ │ ├── select - │ │ │ │ ├── columns: k:3!null i:4 - │ │ │ │ ├── key: (3) - │ │ │ │ ├── fd: (3)-->(4) - │ │ │ │ ├── ordering: +3 + │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ ├── key: (4) + │ │ │ │ ├── fd: (4)-->(5) + │ │ │ │ ├── ordering: +4 │ │ │ │ ├── scan a - │ │ │ │ │ ├── columns: k:3!null i:4 - │ │ │ │ │ ├── key: (3) - │ │ │ │ │ ├── fd: (3)-->(4) - │ │ │ │ │ └── ordering: +3 + │ │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ │ ├── key: (4) + │ │ │ │ │ ├── fd: (4)-->(5) + │ │ │ │ │ └── ordering: +4 │ │ │ │ └── filters - │ │ │ │ └── (i:4 = 5) IS NOT false [outer=(4)] + │ │ │ │ └── (i:5 = 5) IS NOT false [outer=(5)] │ │ │ └── projections - │ │ │ └── i:4 IS NOT NULL [as=notnull:9, outer=(4)] + │ │ │ └── i:5 IS NOT NULL [as=notnull:11, outer=(5)] │ │ └── filters (true) │ └── aggregations - │ └── bool-or [as=bool_or:10, outer=(9)] - │ └── notnull:9 + │ └── bool-or [as=bool_or:12, outer=(11)] + │ └── notnull:11 └── projections - └── CASE WHEN bool_or:10 THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=r:8, outer=(10)] + └── CASE WHEN bool_or:12 THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=r:10, outer=(12)] diff --git a/pkg/sql/opt/norm/testdata/rules/comp b/pkg/sql/opt/norm/testdata/rules/comp index 4aec87105324..c2905b9181fb 100644 --- a/pkg/sql/opt/norm/testdata/rules/comp +++ b/pkg/sql/opt/norm/testdata/rules/comp @@ -386,12 +386,12 @@ norm expect-not=FoldNullTupleIsTupleNull SELECT (k, NULL) IS NULL FROM a ---- project - ├── columns: "?column?":7!null + ├── columns: "?column?":8!null ├── scan a │ ├── columns: k:1!null │ └── key: (1) └── projections - └── (k:1, NULL) IS NULL [as="?column?":7, outer=(1)] + └── (k:1, NULL) IS NULL [as="?column?":8, outer=(1)] # -------------------------------------------------- # FoldNonNullTupleIsTupleNull @@ -420,11 +420,11 @@ norm expect=FoldNonNullTupleIsTupleNull SELECT (1, k) IS NULL FROM a ---- project - ├── columns: "?column?":7!null - ├── fd: ()-->(7) + ├── columns: "?column?":8!null + ├── fd: ()-->(8) ├── scan a └── projections - └── false [as="?column?":7] + └── false [as="?column?":8] norm expect=FoldNonNullTupleIsTupleNull SELECT ((NULL, NULL), NULL) IS NULL AS r @@ -450,12 +450,12 @@ norm expect-not=FoldNonNullTupleIsTupleNull SELECT (k, NULL) IS NULL FROM a ---- project - ├── columns: "?column?":7!null + ├── columns: "?column?":8!null ├── scan a │ ├── columns: k:1!null │ └── key: (1) └── projections - └── (k:1, NULL) IS NULL [as="?column?":7, outer=(1)] + └── (k:1, NULL) IS NULL [as="?column?":8, outer=(1)] # -------------------------------------------------- # FoldIsNotNull @@ -480,16 +480,16 @@ norm expect=FoldNonNullIsNotNull SELECT 1 IS NOT NULL AS r, k IS NOT NULL AS s, i IS NOT NULL AS t FROM a ---- project - ├── columns: r:7!null s:8!null t:9!null - ├── fd: ()-->(7) + ├── columns: r:8!null s:9!null t:10!null + ├── fd: ()-->(8) ├── scan a │ ├── columns: k:1!null i:2 │ ├── key: (1) │ └── fd: (1)-->(2) └── projections - ├── true [as=r:7] - ├── k:1 IS NOT NULL [as=s:8, outer=(1)] - └── i:2 IS NOT NULL [as=t:9, outer=(2)] + ├── true [as=r:8] + ├── k:1 IS NOT NULL [as=s:9, outer=(1)] + └── i:2 IS NOT NULL [as=t:10, outer=(2)] norm expect=FoldNonNullIsNotNull SELECT (1, 2, 3) IS DISTINCT FROM NULL AS r @@ -568,12 +568,12 @@ norm expect-not=FoldNonNullTupleIsTupleNotNull SELECT (1, k) IS NOT NULL FROM a ---- project - ├── columns: "?column?":7!null + ├── columns: "?column?":8!null ├── scan a │ ├── columns: k:1!null │ └── key: (1) └── projections - └── (1, k:1) IS NOT NULL [as="?column?":7, outer=(1)] + └── (1, k:1) IS NOT NULL [as="?column?":8, outer=(1)] # -------------------------------------------------- # FoldNullTupleIsTupleNotNull @@ -592,22 +592,22 @@ norm expect=FoldNullTupleIsTupleNotNull SELECT (k, NULL) IS NOT NULL FROM a ---- project - ├── columns: "?column?":7!null - ├── fd: ()-->(7) + ├── columns: "?column?":8!null + ├── fd: ()-->(8) ├── scan a └── projections - └── false [as="?column?":7] + └── false [as="?column?":8] norm expect-not=FoldNonNullTupleIsTupleNotNull SELECT (1, k) IS NOT NULL FROM a ---- project - ├── columns: "?column?":7!null + ├── columns: "?column?":8!null ├── scan a │ ├── columns: k:1!null │ └── key: (1) └── projections - └── (1, k:1) IS NOT NULL [as="?column?":7, outer=(1)] + └── (1, k:1) IS NOT NULL [as="?column?":8, outer=(1)] # -------------------------------------------------- # CommuteNullIs @@ -633,45 +633,45 @@ norm expect=NormalizeCmpTimeZoneFunction SELECT timezone('America/Denver', ts) >= '2020-06-01 12:35:55-07' FROM t ---- project - ├── columns: "?column?":4 + ├── columns: "?column?":5 ├── scan t │ └── columns: ts:1 └── projections - └── ts:1 >= '2020-06-01 13:35:55+00:00' [as="?column?":4, outer=(1)] + └── ts:1 >= '2020-06-01 13:35:55+00:00' [as="?column?":5, outer=(1)] # Apply after commuting the inequality. norm expect=NormalizeCmpTimeZoneFunction SELECT '2020-06-01 12:35:55-07' >= timezone('America/Denver', ts) FROM t ---- project - ├── columns: "?column?":4 + ├── columns: "?column?":5 ├── scan t │ └── columns: ts:1 └── projections - └── ts:1 <= '2020-06-01 13:35:55+00:00' [as="?column?":4, outer=(1)] + └── ts:1 <= '2020-06-01 13:35:55+00:00' [as="?column?":5, outer=(1)] # Don't normalize when the right-hand-side is not a constant. norm expect-not=NormalizeCmpTimeZoneFunction SELECT timezone('America/Denver', ts) >= tz FROM t ---- project - ├── columns: "?column?":4 + ├── columns: "?column?":5 ├── immutable ├── scan t │ └── columns: ts:1 tz:2 └── projections - └── tz:2 <= timezone('America/Denver', ts:1) [as="?column?":4, outer=(1,2), immutable] + └── tz:2 <= timezone('America/Denver', ts:1) [as="?column?":5, outer=(1,2), immutable] # Don't normalize when the timezone() arguments are constants. norm expect-not=NormalizeCmpTimeZoneFunction SELECT timezone('America/Denver', '2020-06-01 12:35:55'::TIMESTAMP) >= tz FROM t ---- project - ├── columns: "?column?":4 + ├── columns: "?column?":5 ├── scan t │ └── columns: tz:2 └── projections - └── tz:2 <= '2020-06-01 18:35:55+00:00' [as="?column?":4, outer=(2)] + └── tz:2 <= '2020-06-01 18:35:55+00:00' [as="?column?":5, outer=(2)] # -------------------------------------------------- # NormalizeCmpTimeZoneFunctionTZ @@ -680,42 +680,42 @@ norm expect=NormalizeCmpTimeZoneFunctionTZ SELECT timezone('America/Denver', tz) >= '2020-06-01 12:35:55' FROM t ---- project - ├── columns: "?column?":4 + ├── columns: "?column?":5 ├── scan t │ └── columns: tz:2 └── projections - └── tz:2 >= '2020-06-01 18:35:55+00:00' [as="?column?":4, outer=(2)] + └── tz:2 >= '2020-06-01 18:35:55+00:00' [as="?column?":5, outer=(2)] # Apply after commuting the inequality. norm expect=NormalizeCmpTimeZoneFunctionTZ SELECT '2020-06-01 12:35:55' >= timezone('America/Denver', tz) FROM t ---- project - ├── columns: "?column?":4 + ├── columns: "?column?":5 ├── scan t │ └── columns: tz:2 └── projections - └── tz:2 <= '2020-06-01 18:35:55+00:00' [as="?column?":4, outer=(2)] + └── tz:2 <= '2020-06-01 18:35:55+00:00' [as="?column?":5, outer=(2)] # Don't normalize when the right-hand-side is not a constant. norm expect-not=NormalizeCmpTimeZoneFunctionTZ SELECT timezone('America/Denver', tz) >= ts FROM t ---- project - ├── columns: "?column?":4 + ├── columns: "?column?":5 ├── immutable ├── scan t │ └── columns: ts:1 tz:2 └── projections - └── ts:1 <= timezone('America/Denver', tz:2) [as="?column?":4, outer=(1,2), immutable] + └── ts:1 <= timezone('America/Denver', tz:2) [as="?column?":5, outer=(1,2), immutable] # Don't normalize when the timezone() arguments are constants. norm expect-not=NormalizeCmpTimeZoneFunctionTZ SELECT timezone('America/Denver', '2020-06-01 12:35:55-07'::TIMESTAMPTZ) >= ts FROM t ---- project - ├── columns: "?column?":4 + ├── columns: "?column?":5 ├── scan t │ └── columns: ts:1 └── projections - └── ts:1 <= '2020-06-01 13:35:55+00:00' [as="?column?":4, outer=(1)] + └── ts:1 <= '2020-06-01 13:35:55+00:00' [as="?column?":5, outer=(1)] diff --git a/pkg/sql/opt/norm/testdata/rules/decorrelate b/pkg/sql/opt/norm/testdata/rules/decorrelate index 4ac5ff4eb87d..c81e5d39b044 100644 --- a/pkg/sql/opt/norm/testdata/rules/decorrelate +++ b/pkg/sql/opt/norm/testdata/rules/decorrelate @@ -33,10 +33,10 @@ semi-join (hash) │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── scan xy - │ ├── columns: x:6!null - │ └── key: (6) + │ ├── columns: x:7!null + │ └── key: (7) └── filters - └── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] norm expect=DecorrelateJoin SELECT * FROM a WHERE NOT EXISTS(SELECT * FROM xy WHERE x=k) @@ -50,10 +50,10 @@ anti-join (hash) │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── scan xy - │ ├── columns: x:6!null - │ └── key: (6) + │ ├── columns: x:7!null + │ └── key: (7) └── filters - └── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # Decorrelate UPDATE statement. norm expect=DecorrelateJoin @@ -61,27 +61,27 @@ UPDATE xy SET (x, y)=(SELECT * FROM uv WHERE u=x) ---- update xy ├── columns: - ├── fetch columns: x:3 y:4 + ├── fetch columns: x:4 y:5 ├── update-mapping: - │ ├── u:5 => x:1 - │ └── v:6 => y:2 + │ ├── u:7 => x:1 + │ └── v:8 => y:2 ├── cardinality: [0 - 0] ├── volatile, mutations └── left-join (hash) - ├── columns: x:3!null y:4 u:5 v:6 + ├── columns: x:4!null y:5 u:7 v:8 ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) - ├── key: (3) - ├── fd: (3)-->(4-6), (5)-->(6) + ├── key: (4) + ├── fd: (4)-->(5,7,8), (7)-->(8) ├── scan xy - │ ├── columns: x:3!null y:4 - │ ├── key: (3) - │ └── fd: (3)-->(4) + │ ├── columns: x:4!null y:5 + │ ├── key: (4) + │ └── fd: (4)-->(5) ├── scan uv - │ ├── columns: u:5!null v:6 - │ ├── key: (5) - │ └── fd: (5)-->(6) + │ ├── columns: u:7!null v:8 + │ ├── key: (7) + │ └── fd: (7)-->(8) └── filters - └── u:5 = x:3 [outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ]), fd=(3)==(5), (5)==(3)] + └── u:7 = x:4 [outer=(4,7), constraints=(/4: (/NULL - ]; /7: (/NULL - ]), fd=(4)==(7), (7)==(4)] # Decorrelate INSERT..ON CONFLICT statement. norm expect=DecorrelateJoin @@ -91,53 +91,53 @@ RETURNING * ---- upsert xy ├── columns: x:1!null y:2 - ├── canary column: 5 - ├── fetch columns: x:5 y:6 + ├── canary column: 6 + ├── fetch columns: x:6 y:7 ├── insert-mapping: - │ ├── column1:3 => x:1 - │ └── column2:4 => y:2 + │ ├── column1:4 => x:1 + │ └── column2:5 => y:2 ├── update-mapping: - │ ├── upsert_x:9 => x:1 - │ └── upsert_y:10 => y:2 + │ ├── upsert_x:12 => x:1 + │ └── upsert_y:13 => y:2 ├── return-mapping: - │ ├── upsert_x:9 => x:1 - │ └── upsert_y:10 => y:2 + │ ├── upsert_x:12 => x:1 + │ └── upsert_y:13 => y:2 ├── cardinality: [2 - 2] ├── volatile, mutations └── project - ├── columns: upsert_x:9 upsert_y:10 column1:3!null column2:4!null x:5 y:6 + ├── columns: upsert_x:12 upsert_y:13 column1:4!null column2:5!null x:6 y:7 ├── cardinality: [2 - 2] - ├── fd: (5)-->(6) + ├── fd: (6)-->(7) ├── left-join (hash) - │ ├── columns: column1:3!null column2:4!null x:5 y:6 u:7 v:8 + │ ├── columns: column1:4!null column2:5!null x:6 y:7 u:9 v:10 │ ├── cardinality: [2 - 2] │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ ├── fd: (5)-->(6), (7)-->(8) + │ ├── fd: (6)-->(7), (9)-->(10) │ ├── left-join (hash) - │ │ ├── columns: column1:3!null column2:4!null x:5 y:6 + │ │ ├── columns: column1:4!null column2:5!null x:6 y:7 │ │ ├── cardinality: [2 - 2] │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ │ ├── fd: (5)-->(6) + │ │ ├── fd: (6)-->(7) │ │ ├── values - │ │ │ ├── columns: column1:3!null column2:4!null + │ │ │ ├── columns: column1:4!null column2:5!null │ │ │ ├── cardinality: [2 - 2] │ │ │ ├── (1, 2) │ │ │ └── (3, 4) │ │ ├── scan xy - │ │ │ ├── columns: x:5!null y:6 - │ │ │ ├── key: (5) - │ │ │ └── fd: (5)-->(6) + │ │ │ ├── columns: x:6!null y:7 + │ │ │ ├── key: (6) + │ │ │ └── fd: (6)-->(7) │ │ └── filters - │ │ └── column1:3 = x:5 [outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ]), fd=(3)==(5), (5)==(3)] + │ │ └── column1:4 = x:6 [outer=(4,6), constraints=(/4: (/NULL - ]; /6: (/NULL - ]), fd=(4)==(6), (6)==(4)] │ ├── scan uv - │ │ ├── columns: u:7!null v:8 - │ │ ├── key: (7) - │ │ └── fd: (7)-->(8) + │ │ ├── columns: u:9!null v:10 + │ │ ├── key: (9) + │ │ └── fd: (9)-->(10) │ └── filters - │ └── u:7 = column1:3 [outer=(3,7), constraints=(/3: (/NULL - ]; /7: (/NULL - ]), fd=(3)==(7), (7)==(3)] + │ └── u:9 = column1:4 [outer=(4,9), constraints=(/4: (/NULL - ]; /9: (/NULL - ]), fd=(4)==(9), (9)==(4)] └── projections - ├── CASE WHEN x:5 IS NULL THEN column1:3 ELSE u:7 END [as=upsert_x:9, outer=(3,5,7)] - └── CASE WHEN x:5 IS NULL THEN column2:4 ELSE v:8 END [as=upsert_y:10, outer=(4,5,8)] + ├── CASE WHEN x:6 IS NULL THEN column1:4 ELSE u:9 END [as=upsert_x:12, outer=(4,6,9)] + └── CASE WHEN x:6 IS NULL THEN column2:5 ELSE v:10 END [as=upsert_y:13, outer=(5,6,10)] # Decorrelate DELETE statement. norm expect=DecorrelateJoin @@ -145,20 +145,20 @@ DELETE FROM xy WHERE EXISTS(SELECT * FROM uv WHERE u=x) ---- delete xy ├── columns: - ├── fetch columns: x:3 + ├── fetch columns: x:4 ├── cardinality: [0 - 0] ├── volatile, mutations └── semi-join (hash) - ├── columns: x:3!null - ├── key: (3) + ├── columns: x:4!null + ├── key: (4) ├── scan xy - │ ├── columns: x:3!null - │ └── key: (3) + │ ├── columns: x:4!null + │ └── key: (4) ├── scan uv - │ ├── columns: u:5!null - │ └── key: (5) + │ ├── columns: u:7!null + │ └── key: (7) └── filters - └── u:5 = x:3 [outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ]), fd=(3)==(5), (5)==(3)] + └── u:7 = x:4 [outer=(4,7), constraints=(/4: (/NULL - ]; /7: (/NULL - ]), fd=(4)==(7), (7)==(4)] # -------------------------------------------------- # DecorrelateProjectSet @@ -168,11 +168,11 @@ norm expect=DecorrelateProjectSet SELECT generate_series(0, 5) FROM xy ---- inner-join (cross) - ├── columns: generate_series:3 + ├── columns: generate_series:4 ├── immutable ├── scan xy ├── project-set - │ ├── columns: generate_series:3 + │ ├── columns: generate_series:4 │ ├── immutable │ ├── values │ │ ├── cardinality: [1 - 1] @@ -195,12 +195,12 @@ semi-join-apply │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── inner-join (cross) - │ ├── columns: generate_series:8 + │ ├── columns: generate_series:10 │ ├── outer: (1,2) │ ├── immutable │ ├── scan xy │ ├── project-set - │ │ ├── columns: generate_series:8 + │ │ ├── columns: generate_series:10 │ │ ├── outer: (1,2) │ │ ├── immutable │ │ ├── values @@ -211,17 +211,17 @@ semi-join-apply │ │ └── generate_series(k:1, i:2) [outer=(1,2), immutable] │ └── filters (true) └── filters - └── i:2 = generate_series:8 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] + └── i:2 = generate_series:10 [outer=(2,10), constraints=(/2: (/NULL - ]; /10: (/NULL - ]), fd=(2)==(10), (10)==(2)] norm expect=DecorrelateProjectSet SELECT generate_series(0, (SELECT generate_series(1,0) FROM xy)) FROM uv ---- inner-join (cross) - ├── columns: generate_series:6 + ├── columns: generate_series:8 ├── immutable ├── scan uv ├── project-set - │ ├── columns: generate_series:6 + │ ├── columns: generate_series:8 │ ├── immutable │ ├── values │ │ ├── cardinality: [1 - 1] @@ -232,18 +232,18 @@ inner-join (cross) │ ├── 0 │ └── subquery │ └── max1-row - │ ├── columns: generate_series:5 + │ ├── columns: generate_series:7 │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── cardinality: [0 - 1] │ ├── immutable │ ├── key: () - │ ├── fd: ()-->(5) + │ ├── fd: ()-->(7) │ └── inner-join (cross) - │ ├── columns: generate_series:5 + │ ├── columns: generate_series:7 │ ├── immutable │ ├── scan xy │ ├── project-set - │ │ ├── columns: generate_series:5 + │ │ ├── columns: generate_series:7 │ │ ├── immutable │ │ ├── values │ │ │ ├── cardinality: [1 - 1] @@ -267,36 +267,36 @@ FROM WHERE i = 3 ---- project - ├── columns: u:1!null v:2 rank:8 i:4!null - ├── key: (1,8) - ├── fd: ()-->(4), (1)-->(2) + ├── columns: u:1!null v:2 rank:10 i:5!null + ├── key: (1,10) + ├── fd: ()-->(5), (1)-->(2) └── window partition=(1) - ├── columns: u:1!null v:2 k:3!null i:4!null rank:8 - ├── key: (3) - ├── fd: ()-->(4), (1)-->(2), (1)==(3), (3)==(1) + ├── columns: u:1!null v:2 k:4!null i:5!null rank:10 + ├── key: (4) + ├── fd: ()-->(5), (1)-->(2), (1)==(4), (4)==(1) ├── inner-join (hash) - │ ├── columns: u:1!null v:2 k:3!null i:4!null + │ ├── columns: u:1!null v:2 k:4!null i:5!null │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - │ ├── key: (3) - │ ├── fd: ()-->(4), (1)-->(2), (1)==(3), (3)==(1) + │ ├── key: (4) + │ ├── fd: ()-->(5), (1)-->(2), (1)==(4), (4)==(1) │ ├── scan uv │ │ ├── columns: u:1!null v:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ ├── select - │ │ ├── columns: k:3!null i:4!null - │ │ ├── key: (3) - │ │ ├── fd: ()-->(4) + │ │ ├── columns: k:4!null i:5!null + │ │ ├── key: (4) + │ │ ├── fd: ()-->(5) │ │ ├── scan a - │ │ │ ├── columns: k:3!null i:4 - │ │ │ ├── key: (3) - │ │ │ └── fd: (3)-->(4) + │ │ │ ├── columns: k:4!null i:5 + │ │ │ ├── key: (4) + │ │ │ └── fd: (4)-->(5) │ │ └── filters - │ │ └── i:4 = 3 [outer=(4), constraints=(/4: [/3 - /3]; tight), fd=()-->(4)] + │ │ └── i:5 = 3 [outer=(5), constraints=(/5: [/3 - /3]; tight), fd=()-->(5)] │ └── filters - │ └── k:3 = u:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + │ └── k:4 = u:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] └── windows - └── rank [as=rank:8] + └── rank [as=rank:10] # TryDecorrelateWindow will trigger twice here: first to pull the window above # the non-apply join, and then again the pull it above the apply join. @@ -315,40 +315,40 @@ WHERE i = 3 ---- project - ├── columns: u:1!null v:2 x:3!null ntile:10 i:6!null - ├── fd: ()-->(6), (1)-->(2) + ├── columns: u:1!null v:2 x:4!null ntile:13 i:8!null + ├── fd: ()-->(8), (1)-->(2) └── select - ├── columns: u:1!null v:2 x:3!null i:6!null ntile:10 ntile_1_arg1:11!null - ├── fd: ()-->(6), (1)-->(2), (1)==(11), (11)==(1) - ├── window partition=(1,3) - │ ├── columns: u:1!null v:2 x:3!null i:6 ntile:10 ntile_1_arg1:11!null - │ ├── fd: (1)-->(2), (1)==(11), (11)==(1) + ├── columns: u:1!null v:2 x:4!null i:8!null ntile:13 ntile_1_arg1:14!null + ├── fd: ()-->(8), (1)-->(2), (1)==(14), (14)==(1) + ├── window partition=(1,4) + │ ├── columns: u:1!null v:2 x:4!null i:8 ntile:13 ntile_1_arg1:14!null + │ ├── fd: (1)-->(2), (1)==(14), (14)==(1) │ ├── project - │ │ ├── columns: ntile_1_arg1:11!null u:1!null v:2 x:3!null i:6 - │ │ ├── fd: (1)-->(2), (1)==(11), (11)==(1) + │ │ ├── columns: ntile_1_arg1:14!null u:1!null v:2 x:4!null i:8 + │ │ ├── fd: (1)-->(2), (1)==(14), (14)==(1) │ │ ├── inner-join (cross) - │ │ │ ├── columns: u:1!null v:2 x:3!null i:6 + │ │ │ ├── columns: u:1!null v:2 x:4!null i:8 │ │ │ ├── fd: (1)-->(2) │ │ │ ├── scan uv │ │ │ │ ├── columns: u:1!null v:2 │ │ │ │ ├── key: (1) │ │ │ │ └── fd: (1)-->(2) │ │ │ ├── inner-join (cross) - │ │ │ │ ├── columns: x:3!null i:6 + │ │ │ │ ├── columns: x:4!null i:8 │ │ │ │ ├── scan xy - │ │ │ │ │ ├── columns: x:3!null - │ │ │ │ │ └── key: (3) + │ │ │ │ │ ├── columns: x:4!null + │ │ │ │ │ └── key: (4) │ │ │ │ ├── scan a - │ │ │ │ │ └── columns: i:6 + │ │ │ │ │ └── columns: i:8 │ │ │ │ └── filters (true) │ │ │ └── filters (true) │ │ └── projections - │ │ └── u:1 [as=ntile_1_arg1:11, outer=(1)] + │ │ └── u:1 [as=ntile_1_arg1:14, outer=(1)] │ └── windows - │ └── ntile [as=ntile:10, outer=(11)] - │ └── ntile_1_arg1:11 + │ └── ntile [as=ntile:13, outer=(14)] + │ └── ntile_1_arg1:14 └── filters - └── i:6 = 3 [outer=(6), constraints=(/6: [/3 - /3]; tight), fd=()-->(6)] + └── i:8 = 3 [outer=(8), constraints=(/8: [/3 - /3]; tight), fd=()-->(8)] # If the LHS has no key, we need to add one, or else identical rows would end up in the same # partition. @@ -361,25 +361,25 @@ FROM LATERAL (SELECT row_number() OVER (), i FROM (SELECT * FROM a WHERE k = x)) ---- project - ├── columns: x:1!null row_number:7 i:3 + ├── columns: x:1!null row_number:8 i:3 ├── cardinality: [0 - 3] ├── fd: (1)-->(3) - └── window partition=(8) - ├── columns: column1:1!null k:2!null i:3 row_number:7 rownum:8!null + └── window partition=(9) + ├── columns: column1:1!null k:2!null i:3 row_number:8 rownum:9!null ├── cardinality: [0 - 3] - ├── key: (8) - ├── fd: (8)-->(1), (2)-->(3), (1)==(2), (2)==(1) + ├── key: (9) + ├── fd: (9)-->(1), (2)-->(3), (1)==(2), (2)==(1) ├── inner-join (hash) - │ ├── columns: column1:1!null k:2!null i:3 rownum:8!null + │ ├── columns: column1:1!null k:2!null i:3 rownum:9!null │ ├── cardinality: [0 - 3] │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ ├── key: (8) - │ ├── fd: (8)-->(1), (2)-->(3), (1)==(2), (2)==(1) + │ ├── key: (9) + │ ├── fd: (9)-->(1), (2)-->(3), (1)==(2), (2)==(1) │ ├── ordinality - │ │ ├── columns: column1:1!null rownum:8!null + │ │ ├── columns: column1:1!null rownum:9!null │ │ ├── cardinality: [3 - 3] - │ │ ├── key: (8) - │ │ ├── fd: (8)-->(1) + │ │ ├── key: (9) + │ │ ├── fd: (9)-->(1) │ │ └── values │ │ ├── columns: column1:1!null │ │ ├── cardinality: [3 - 3] @@ -393,7 +393,7 @@ project │ └── filters │ └── k:2 = column1:1 [outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ]), fd=(1)==(2), (2)==(1)] └── windows - └── row-number [as=row_number:7] + └── row-number [as=row_number:8] norm expect=TryDecorrelateWindow SELECT @@ -403,23 +403,23 @@ FROM LATERAL (SELECT row_number() OVER (ORDER BY i), i FROM (SELECT * FROM a WHERE x > 3)) ---- project - ├── columns: x:1!null row_number:7 i:3 - └── window partition=(8) ordering=+3 opt(1,8) - ├── columns: column1:1!null i:3 row_number:7 rownum:8!null - ├── fd: (8)-->(1) + ├── columns: x:1!null row_number:8 i:3 + └── window partition=(9) ordering=+3 opt(1,9) + ├── columns: column1:1!null i:3 row_number:8 rownum:9!null + ├── fd: (9)-->(1) ├── inner-join (cross) - │ ├── columns: column1:1!null i:3 rownum:8!null - │ ├── fd: (8)-->(1) + │ ├── columns: column1:1!null i:3 rownum:9!null + │ ├── fd: (9)-->(1) │ ├── select - │ │ ├── columns: column1:1!null rownum:8!null + │ │ ├── columns: column1:1!null rownum:9!null │ │ ├── cardinality: [0 - 3] - │ │ ├── key: (8) - │ │ ├── fd: (8)-->(1) + │ │ ├── key: (9) + │ │ ├── fd: (9)-->(1) │ │ ├── ordinality - │ │ │ ├── columns: column1:1!null rownum:8!null + │ │ │ ├── columns: column1:1!null rownum:9!null │ │ │ ├── cardinality: [3 - 3] - │ │ │ ├── key: (8) - │ │ │ ├── fd: (8)-->(1) + │ │ │ ├── key: (9) + │ │ │ ├── fd: (9)-->(1) │ │ │ └── values │ │ │ ├── columns: column1:1!null │ │ │ ├── cardinality: [3 - 3] @@ -432,7 +432,7 @@ project │ │ └── columns: i:3 │ └── filters (true) └── windows - └── row-number [as=row_number:7] + └── row-number [as=row_number:8] # In this example, we introduce a key called rownum, and after TryDecorrelateWindow triggers # PARTITION BY x becomes PARTITION BY x, rownum. Then later, ReduceWindowPartitionCols triggers, @@ -446,30 +446,30 @@ FROM LATERAL (SELECT row_number() OVER (PARTITION BY x), i FROM (SELECT * FROM a WHERE k = x)) ---- project - ├── columns: x:1!null row_number:7 i:3 + ├── columns: x:1!null row_number:8 i:3 ├── cardinality: [0 - 3] ├── fd: (1)-->(3) - └── window partition=(9) - ├── columns: column1:1!null i:3 row_number:7 rownum:9!null + └── window partition=(10) + ├── columns: column1:1!null i:3 row_number:8 rownum:10!null ├── cardinality: [0 - 3] - ├── key: (9) - ├── fd: (9)-->(1), (1)-->(3) + ├── key: (10) + ├── fd: (10)-->(1), (1)-->(3) ├── project - │ ├── columns: column1:1!null i:3 rownum:9!null + │ ├── columns: column1:1!null i:3 rownum:10!null │ ├── cardinality: [0 - 3] - │ ├── key: (9) - │ ├── fd: (9)-->(1), (1)-->(3) + │ ├── key: (10) + │ ├── fd: (10)-->(1), (1)-->(3) │ └── inner-join (hash) - │ ├── columns: column1:1!null k:2!null i:3 rownum:9!null + │ ├── columns: column1:1!null k:2!null i:3 rownum:10!null │ ├── cardinality: [0 - 3] │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ ├── key: (9) - │ ├── fd: (9)-->(1), (2)-->(3), (1)==(2), (2)==(1) + │ ├── key: (10) + │ ├── fd: (10)-->(1), (2)-->(3), (1)==(2), (2)==(1) │ ├── ordinality - │ │ ├── columns: column1:1!null rownum:9!null + │ │ ├── columns: column1:1!null rownum:10!null │ │ ├── cardinality: [3 - 3] - │ │ ├── key: (9) - │ │ ├── fd: (9)-->(1) + │ │ ├── key: (10) + │ │ ├── fd: (10)-->(1) │ │ └── values │ │ ├── columns: column1:1!null │ │ ├── cardinality: [3 - 3] @@ -483,7 +483,7 @@ project │ └── filters │ └── k:2 = column1:1 [outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ]), fd=(1)==(2), (2)==(1)] └── windows - └── row-number [as=row_number:7] + └── row-number [as=row_number:8] norm expect=TryDecorrelateWindow SELECT @@ -493,30 +493,30 @@ FROM LATERAL (SELECT row_number() OVER (PARTITION BY x ORDER BY y), i FROM (SELECT * FROM a WHERE k = x)) ---- project - ├── columns: x:1!null y:2!null row_number:8 i:4 + ├── columns: x:1!null y:2!null row_number:9 i:4 ├── cardinality: [0 - 3] ├── fd: (1)-->(4) - └── window partition=(11) - ├── columns: column1:1!null column2:2!null i:4 row_number:8 rownum:11!null + └── window partition=(12) + ├── columns: column1:1!null column2:2!null i:4 row_number:9 rownum:12!null ├── cardinality: [0 - 3] - ├── key: (11) - ├── fd: (11)-->(1,2), (1)-->(4) + ├── key: (12) + ├── fd: (12)-->(1,2), (1)-->(4) ├── project - │ ├── columns: column1:1!null column2:2!null i:4 rownum:11!null + │ ├── columns: column1:1!null column2:2!null i:4 rownum:12!null │ ├── cardinality: [0 - 3] - │ ├── key: (11) - │ ├── fd: (11)-->(1,2), (1)-->(4) + │ ├── key: (12) + │ ├── fd: (12)-->(1,2), (1)-->(4) │ └── inner-join (hash) - │ ├── columns: column1:1!null column2:2!null k:3!null i:4 rownum:11!null + │ ├── columns: column1:1!null column2:2!null k:3!null i:4 rownum:12!null │ ├── cardinality: [0 - 3] │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ ├── key: (11) - │ ├── fd: (11)-->(1,2), (3)-->(4), (1)==(3), (3)==(1) + │ ├── key: (12) + │ ├── fd: (12)-->(1,2), (3)-->(4), (1)==(3), (3)==(1) │ ├── ordinality - │ │ ├── columns: column1:1!null column2:2!null rownum:11!null + │ │ ├── columns: column1:1!null column2:2!null rownum:12!null │ │ ├── cardinality: [3 - 3] - │ │ ├── key: (11) - │ │ ├── fd: (11)-->(1,2) + │ │ ├── key: (12) + │ │ ├── fd: (12)-->(1,2) │ │ └── values │ │ ├── columns: column1:1!null column2:2!null │ │ ├── cardinality: [3 - 3] @@ -530,7 +530,7 @@ project │ └── filters │ └── k:3 = column1:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] └── windows - └── row-number [as=row_number:8] + └── row-number [as=row_number:9] norm expect=TryDecorrelateWindow SELECT @@ -540,30 +540,30 @@ FROM LATERAL (SELECT row_number() OVER (PARTITION BY u), i FROM (SELECT * FROM a WHERE k = u)) ---- window partition=(1) - ├── columns: u:1!null v:2 row_number:8 i:4 + ├── columns: u:1!null v:2 row_number:10 i:5 ├── key: (1) - ├── fd: (1)-->(2,4) + ├── fd: (1)-->(2,5) ├── project - │ ├── columns: u:1!null v:2 i:4 + │ ├── columns: u:1!null v:2 i:5 │ ├── key: (1) - │ ├── fd: (1)-->(2,4) + │ ├── fd: (1)-->(2,5) │ └── inner-join (hash) - │ ├── columns: u:1!null v:2 k:3!null i:4 + │ ├── columns: u:1!null v:2 k:4!null i:5 │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - │ ├── key: (3) - │ ├── fd: (1)-->(2), (3)-->(4), (1)==(3), (3)==(1) + │ ├── key: (4) + │ ├── fd: (1)-->(2), (4)-->(5), (1)==(4), (4)==(1) │ ├── scan uv │ │ ├── columns: u:1!null v:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ ├── scan a - │ │ ├── columns: k:3!null i:4 - │ │ ├── key: (3) - │ │ └── fd: (3)-->(4) + │ │ ├── columns: k:4!null i:5 + │ │ ├── key: (4) + │ │ └── fd: (4)-->(5) │ └── filters - │ └── k:3 = u:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + │ └── k:4 = u:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] └── windows - └── row-number [as=row_number:8] + └── row-number [as=row_number:10] norm expect=TryDecorrelateWindow SELECT @@ -573,30 +573,30 @@ FROM LATERAL (SELECT row_number() OVER (PARTITION BY s), i FROM (SELECT * FROM a WHERE k = u)) ---- project - ├── columns: u:1!null v:2 row_number:8 i:4 - ├── key: (1,8) - ├── fd: (1)-->(2,4) + ├── columns: u:1!null v:2 row_number:10 i:5 + ├── key: (1,10) + ├── fd: (1)-->(2,5) └── window partition=(1) - ├── columns: u:1!null v:2 k:3!null i:4 row_number:8 - ├── key: (3) - ├── fd: (1)-->(2), (3)-->(4), (1)==(3), (3)==(1) + ├── columns: u:1!null v:2 k:4!null i:5 row_number:10 + ├── key: (4) + ├── fd: (1)-->(2), (4)-->(5), (1)==(4), (4)==(1) ├── inner-join (hash) - │ ├── columns: u:1!null v:2 k:3!null i:4 + │ ├── columns: u:1!null v:2 k:4!null i:5 │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - │ ├── key: (3) - │ ├── fd: (1)-->(2), (3)-->(4), (1)==(3), (3)==(1) + │ ├── key: (4) + │ ├── fd: (1)-->(2), (4)-->(5), (1)==(4), (4)==(1) │ ├── scan uv │ │ ├── columns: u:1!null v:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ ├── scan a - │ │ ├── columns: k:3!null i:4 - │ │ ├── key: (3) - │ │ └── fd: (3)-->(4) + │ │ ├── columns: k:4!null i:5 + │ │ ├── key: (4) + │ │ └── fd: (4)-->(5) │ └── filters - │ └── k:3 = u:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + │ └── k:4 = u:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] └── windows - └── row-number [as=row_number:8] + └── row-number [as=row_number:10] norm expect=TryDecorrelateWindow SELECT @@ -606,55 +606,55 @@ FROM LATERAL (SELECT row_number() OVER (PARTITION BY s), i FROM (SELECT * FROM a WHERE i = u)) ---- project - ├── columns: u:1!null v:2 row_number:8 i:4!null - ├── fd: (1)-->(2), (1)==(4), (4)==(1) - └── window partition=(1,6) - ├── columns: u:1!null v:2 i:4!null s:6 row_number:8 - ├── fd: (1)-->(2), (1)==(4), (4)==(1) + ├── columns: u:1!null v:2 row_number:10 i:5!null + ├── fd: (1)-->(2), (1)==(5), (5)==(1) + └── window partition=(1,7) + ├── columns: u:1!null v:2 i:5!null s:7 row_number:10 + ├── fd: (1)-->(2), (1)==(5), (5)==(1) ├── inner-join (hash) - │ ├── columns: u:1!null v:2 i:4!null s:6 + │ ├── columns: u:1!null v:2 i:5!null s:7 │ ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) - │ ├── fd: (1)-->(2), (1)==(4), (4)==(1) + │ ├── fd: (1)-->(2), (1)==(5), (5)==(1) │ ├── scan uv │ │ ├── columns: u:1!null v:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ ├── scan a - │ │ └── columns: i:4 s:6 + │ │ └── columns: i:5 s:7 │ └── filters - │ └── i:4 = u:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + │ └── i:5 = u:1 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] └── windows - └── row-number [as=row_number:8] + └── row-number [as=row_number:10] norm expect=TryDecorrelateWindow SELECT * FROM uv, LATERAL (SELECT avg(f) FILTER (WHERE u = 3) OVER (), i FROM a) ---- project - ├── columns: u:1!null v:2 avg:8 i:4 + ├── columns: u:1!null v:2 avg:10 i:5 ├── fd: (1)-->(2) └── window partition=(1) - ├── columns: u:1!null v:2 i:4 f:5 avg:8 avg_1_filter:9!null - ├── fd: (1)-->(2,9) + ├── columns: u:1!null v:2 i:5 f:6 avg:10 avg_1_filter:11!null + ├── fd: (1)-->(2,11) ├── project - │ ├── columns: avg_1_filter:9!null u:1!null v:2 i:4 f:5 - │ ├── fd: (1)-->(2,9) + │ ├── columns: avg_1_filter:11!null u:1!null v:2 i:5 f:6 + │ ├── fd: (1)-->(2,11) │ ├── inner-join (cross) - │ │ ├── columns: u:1!null v:2 i:4 f:5 + │ │ ├── columns: u:1!null v:2 i:5 f:6 │ │ ├── fd: (1)-->(2) │ │ ├── scan uv │ │ │ ├── columns: u:1!null v:2 │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2) │ │ ├── scan a - │ │ │ └── columns: i:4 f:5 + │ │ │ └── columns: i:5 f:6 │ │ └── filters (true) │ └── projections - │ └── u:1 = 3 [as=avg_1_filter:9, outer=(1)] + │ └── u:1 = 3 [as=avg_1_filter:11, outer=(1)] └── windows - └── agg-filter [as=avg:8, outer=(5,9)] + └── agg-filter [as=avg:10, outer=(6,11)] ├── avg - │ └── f:5 - └── avg_1_filter:9 + │ └── f:6 + └── avg_1_filter:11 # -------------------------------------------------- # TryDecorrelateSelect @@ -671,13 +671,13 @@ semi-join-apply │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── values - │ ├── columns: column1:6 + │ ├── columns: column1:7 │ ├── outer: (1,2) │ ├── cardinality: [2 - 2] │ ├── (k:1,) │ └── (i:2,) └── filters - └── column1:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── column1:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] norm expect=TryDecorrelateSelect SELECT * FROM a WHERE NOT EXISTS(SELECT * FROM (VALUES (k), (i)) WHERE column1=k) @@ -691,13 +691,13 @@ anti-join-apply │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── values - │ ├── columns: column1:6 + │ ├── columns: column1:7 │ ├── outer: (1,2) │ ├── cardinality: [2 - 2] │ ├── (k:1,) │ └── (i:2,) └── filters - └── column1:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── column1:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # Attempt to decorrelate query by pulling up outer select. But since limit query # cannot be decorrelated, push the outer select back down again (and make sure @@ -714,24 +714,24 @@ semi-join-apply │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── limit - │ ├── columns: y:7!null + │ ├── columns: y:8!null │ ├── outer: (1) │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(7) + │ ├── fd: ()-->(8) │ ├── select - │ │ ├── columns: y:7!null + │ │ ├── columns: y:8!null │ │ ├── outer: (1) - │ │ ├── fd: ()-->(7) + │ │ ├── fd: ()-->(8) │ │ ├── limit hint: 1.00 │ │ ├── scan xy - │ │ │ ├── columns: y:7 + │ │ │ ├── columns: y:8 │ │ │ └── limit hint: 1.01 │ │ └── filters - │ │ └── y:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + │ │ └── y:8 = k:1 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] │ └── 1 └── filters - └── y:7 = 10 [outer=(7), constraints=(/7: [/10 - /10]; tight), fd=()-->(7)] + └── y:8 = 10 [outer=(8), constraints=(/8: [/10 - /10]; tight), fd=()-->(8)] # Same as previous, but using anti-join. norm expect=TryDecorrelateSelect @@ -746,24 +746,24 @@ anti-join-apply │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── limit - │ ├── columns: y:7!null + │ ├── columns: y:8!null │ ├── outer: (1) │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(7) + │ ├── fd: ()-->(8) │ ├── select - │ │ ├── columns: y:7!null + │ │ ├── columns: y:8!null │ │ ├── outer: (1) - │ │ ├── fd: ()-->(7) + │ │ ├── fd: ()-->(8) │ │ ├── limit hint: 1.00 │ │ ├── scan xy - │ │ │ ├── columns: y:7 + │ │ │ ├── columns: y:8 │ │ │ └── limit hint: 1.01 │ │ └── filters - │ │ └── y:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + │ │ └── y:8 = k:1 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] │ └── 1 └── filters - └── y:7 = 10 [outer=(7), constraints=(/7: [/10 - /10]; tight), fd=()-->(7)] + └── y:8 = 10 [outer=(8), constraints=(/8: [/10 - /10]; tight), fd=()-->(8)] # Decorrelate Select with reference to outer column and no limit. norm expect=TryDecorrelateSelect @@ -774,10 +774,10 @@ project ├── key: (1) ├── fd: (1)-->(2-5) └── inner-join (hash) - ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:6!null + ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:7!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) ├── key: (1) - ├── fd: (1)-->(2-5), (2)==(6), (6)==(2) + ├── fd: (1)-->(2-5), (2)==(7), (7)==(2) ├── select │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 │ ├── key: (1) @@ -789,15 +789,15 @@ project │ └── filters │ └── i:2 > 100 [outer=(2), constraints=(/2: [/101 - ]; tight)] ├── select - │ ├── columns: x:6!null - │ ├── key: (6) + │ ├── columns: x:7!null + │ ├── key: (7) │ ├── scan xy - │ │ ├── columns: x:6!null - │ │ └── key: (6) + │ │ ├── columns: x:7!null + │ │ └── key: (7) │ └── filters - │ └── x:6 > 100 [outer=(6), constraints=(/6: [/101 - ]; tight)] + │ └── x:7 > 100 [outer=(7), constraints=(/7: [/101 - ]; tight)] └── filters - └── x:6 = i:2 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + └── x:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] # Decorrelate Select with LeftJoinApply. norm expect=TryDecorrelateSelect @@ -809,11 +809,11 @@ project ├── key: () ├── fd: ()-->(1-5) └── inner-join (hash) - ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6!null + ├── columns: k:1!null i:2 f:3 s:4 j:5 x:7!null ├── cardinality: [0 - 1] ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) ├── key: () - ├── fd: ()-->(1-6) + ├── fd: ()-->(1-5,7) ├── select │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ ├── key: (1) @@ -825,24 +825,24 @@ project │ └── filters │ └── k:1 > 100 [outer=(1), constraints=(/1: [/101 - ]; tight)] ├── select - │ ├── columns: x:6!null + │ ├── columns: x:7!null │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(6) + │ ├── fd: ()-->(7) │ ├── limit - │ │ ├── columns: x:6!null + │ │ ├── columns: x:7!null │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(6) + │ │ ├── fd: ()-->(7) │ │ ├── scan xy - │ │ │ ├── columns: x:6!null - │ │ │ ├── key: (6) + │ │ │ ├── columns: x:7!null + │ │ │ ├── key: (7) │ │ │ └── limit hint: 1.00 │ │ └── 1 │ └── filters - │ └── x:6 > 100 [outer=(6), constraints=(/6: [/101 - ]; tight)] + │ └── x:7 > 100 [outer=(7), constraints=(/7: [/101 - ]; tight)] └── filters - └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # Decorrelate with non-apply operator because of multi-level nesting. norm expect=TryDecorrelateSelect @@ -859,26 +859,26 @@ semi-join-apply │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── semi-join (cross) - │ ├── columns: x:6!null + │ ├── columns: x:7!null │ ├── outer: (4) - │ ├── key: (6) + │ ├── key: (7) │ ├── scan xy - │ │ ├── columns: x:6!null - │ │ └── key: (6) + │ │ ├── columns: x:7!null + │ │ └── key: (7) │ ├── select - │ │ ├── columns: u:8!null + │ │ ├── columns: u:10!null │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(8) + │ │ ├── fd: ()-->(10) │ │ ├── scan uv - │ │ │ ├── columns: u:8!null - │ │ │ └── key: (8) + │ │ │ ├── columns: u:10!null + │ │ │ └── key: (10) │ │ └── filters - │ │ └── u:8 = 10 [outer=(8), constraints=(/8: [/10 - /10]; tight), fd=()-->(8)] + │ │ └── u:10 = 10 [outer=(10), constraints=(/10: [/10 - /10]; tight), fd=()-->(10)] │ └── filters │ └── s:4 = 'foo' [outer=(4), constraints=(/4: [/'foo' - /'foo']; tight), fd=()-->(4)] └── filters - └── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # -------------------------------------------------- # TryDecorrelateProject + @@ -894,43 +894,43 @@ norm expect=(TryDecorrelateProject,TryDecorrelateProjectSelect,TryDecorrelateSca SELECT 5=ANY(SELECT y FROM xy WHERE x=k) AS r FROM a ---- project - ├── columns: r:8 + ├── columns: r:10 ├── group-by - │ ├── columns: k:1!null bool_or:10 + │ ├── columns: k:1!null bool_or:12 │ ├── grouping columns: k:1!null │ ├── key: (1) - │ ├── fd: (1)-->(10) + │ ├── fd: (1)-->(12) │ ├── left-join (hash) - │ │ ├── columns: k:1!null x:6 notnull:9 + │ │ ├── columns: k:1!null x:7 notnull:11 │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ ├── key: (1) - │ │ ├── fd: (6)-->(9), (1)-->(6,9) + │ │ ├── fd: (7)-->(11), (1)-->(7,11) │ │ ├── scan a │ │ │ ├── columns: k:1!null │ │ │ └── key: (1) │ │ ├── project - │ │ │ ├── columns: notnull:9!null x:6!null - │ │ │ ├── key: (6) - │ │ │ ├── fd: (6)-->(9) + │ │ │ ├── columns: notnull:11!null x:7!null + │ │ │ ├── key: (7) + │ │ │ ├── fd: (7)-->(11) │ │ │ ├── select - │ │ │ │ ├── columns: x:6!null y:7 - │ │ │ │ ├── key: (6) - │ │ │ │ ├── fd: (6)-->(7) + │ │ │ │ ├── columns: x:7!null y:8 + │ │ │ │ ├── key: (7) + │ │ │ │ ├── fd: (7)-->(8) │ │ │ │ ├── scan xy - │ │ │ │ │ ├── columns: x:6!null y:7 - │ │ │ │ │ ├── key: (6) - │ │ │ │ │ └── fd: (6)-->(7) + │ │ │ │ │ ├── columns: x:7!null y:8 + │ │ │ │ │ ├── key: (7) + │ │ │ │ │ └── fd: (7)-->(8) │ │ │ │ └── filters - │ │ │ │ └── (y:7 = 5) IS NOT false [outer=(7)] + │ │ │ │ └── (y:8 = 5) IS NOT false [outer=(8)] │ │ │ └── projections - │ │ │ └── y:7 IS NOT NULL [as=notnull:9, outer=(7)] + │ │ │ └── y:8 IS NOT NULL [as=notnull:11, outer=(8)] │ │ └── filters - │ │ └── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ │ └── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] │ └── aggregations - │ └── bool-or [as=bool_or:10, outer=(9)] - │ └── notnull:9 + │ └── bool-or [as=bool_or:12, outer=(11)] + │ └── notnull:11 └── projections - └── CASE WHEN bool_or:10 THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=r:8, outer=(10)] + └── CASE WHEN bool_or:12 THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=r:10, outer=(12)] # Left join caused by zero or one cardinality subquery. norm expect=TryDecorrelateProjectSelect @@ -942,170 +942,170 @@ project ├── key: () ├── fd: ()-->(1-5) └── inner-join (hash) - ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6!null y:7!null + ├── columns: k:1!null i:2 f:3 s:4 j:5 x:7!null y:8!null ├── cardinality: [0 - 1] ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) ├── key: () - ├── fd: ()-->(1-7) + ├── fd: ()-->(1-5,7,8) ├── scan a │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── select - │ ├── columns: x:6!null y:7!null + │ ├── columns: x:7!null y:8!null │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(6,7) + │ ├── fd: ()-->(7,8) │ ├── limit - │ │ ├── columns: x:6!null y:7 + │ │ ├── columns: x:7!null y:8 │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(6,7) + │ │ ├── fd: ()-->(7,8) │ │ ├── scan xy - │ │ │ ├── columns: x:6!null y:7 - │ │ │ ├── key: (6) - │ │ │ ├── fd: (6)-->(7) + │ │ │ ├── columns: x:7!null y:8 + │ │ │ ├── key: (7) + │ │ │ ├── fd: (7)-->(8) │ │ │ └── limit hint: 1.00 │ │ └── 1 │ └── filters - │ └── y:7 > 9 [outer=(7), constraints=(/7: [/10 - ]; tight)] + │ └── y:8 > 9 [outer=(8), constraints=(/8: [/10 - ]; tight)] └── filters - └── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # Any clause with constant. norm expect=(TryDecorrelateProject,TryDecorrelateProjectSelect,TryDecorrelateScalarGroupBy) SELECT 5=ANY(SELECT y FROM xy WHERE x=k) AS r FROM a ---- project - ├── columns: r:8 + ├── columns: r:10 ├── group-by - │ ├── columns: k:1!null bool_or:10 + │ ├── columns: k:1!null bool_or:12 │ ├── grouping columns: k:1!null │ ├── key: (1) - │ ├── fd: (1)-->(10) + │ ├── fd: (1)-->(12) │ ├── left-join (hash) - │ │ ├── columns: k:1!null x:6 notnull:9 + │ │ ├── columns: k:1!null x:7 notnull:11 │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ ├── key: (1) - │ │ ├── fd: (6)-->(9), (1)-->(6,9) + │ │ ├── fd: (7)-->(11), (1)-->(7,11) │ │ ├── scan a │ │ │ ├── columns: k:1!null │ │ │ └── key: (1) │ │ ├── project - │ │ │ ├── columns: notnull:9!null x:6!null - │ │ │ ├── key: (6) - │ │ │ ├── fd: (6)-->(9) + │ │ │ ├── columns: notnull:11!null x:7!null + │ │ │ ├── key: (7) + │ │ │ ├── fd: (7)-->(11) │ │ │ ├── select - │ │ │ │ ├── columns: x:6!null y:7 - │ │ │ │ ├── key: (6) - │ │ │ │ ├── fd: (6)-->(7) + │ │ │ │ ├── columns: x:7!null y:8 + │ │ │ │ ├── key: (7) + │ │ │ │ ├── fd: (7)-->(8) │ │ │ │ ├── scan xy - │ │ │ │ │ ├── columns: x:6!null y:7 - │ │ │ │ │ ├── key: (6) - │ │ │ │ │ └── fd: (6)-->(7) + │ │ │ │ │ ├── columns: x:7!null y:8 + │ │ │ │ │ ├── key: (7) + │ │ │ │ │ └── fd: (7)-->(8) │ │ │ │ └── filters - │ │ │ │ └── (y:7 = 5) IS NOT false [outer=(7)] + │ │ │ │ └── (y:8 = 5) IS NOT false [outer=(8)] │ │ │ └── projections - │ │ │ └── y:7 IS NOT NULL [as=notnull:9, outer=(7)] + │ │ │ └── y:8 IS NOT NULL [as=notnull:11, outer=(8)] │ │ └── filters - │ │ └── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ │ └── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] │ └── aggregations - │ └── bool-or [as=bool_or:10, outer=(9)] - │ └── notnull:9 + │ └── bool-or [as=bool_or:12, outer=(11)] + │ └── notnull:11 └── projections - └── CASE WHEN bool_or:10 THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=r:8, outer=(10)] + └── CASE WHEN bool_or:12 THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=r:10, outer=(12)] # Any clause with variable. norm expect=(TryDecorrelateProject,TryDecorrelateProjectSelect,TryDecorrelateScalarGroupBy) SELECT i=ANY(SELECT y FROM xy WHERE x=k) AS r FROM a ---- project - ├── columns: r:8 + ├── columns: r:10 ├── group-by - │ ├── columns: k:1!null i:2 bool_or:10 + │ ├── columns: k:1!null i:2 bool_or:12 │ ├── grouping columns: k:1!null │ ├── key: (1) - │ ├── fd: (1)-->(2,10) + │ ├── fd: (1)-->(2,12) │ ├── left-join (hash) - │ │ ├── columns: k:1!null i:2 x:6 y:7 notnull:9 + │ │ ├── columns: k:1!null i:2 x:7 y:8 notnull:11 │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2,6,7,9), (6)-->(7), (7)~~>(9) + │ │ ├── fd: (1)-->(2,7,8,11), (7)-->(8), (8)~~>(11) │ │ ├── scan a │ │ │ ├── columns: k:1!null i:2 │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2) │ │ ├── project - │ │ │ ├── columns: notnull:9!null x:6!null y:7 - │ │ │ ├── key: (6) - │ │ │ ├── fd: (6)-->(7), (7)-->(9) + │ │ │ ├── columns: notnull:11!null x:7!null y:8 + │ │ │ ├── key: (7) + │ │ │ ├── fd: (7)-->(8), (8)-->(11) │ │ │ ├── scan xy - │ │ │ │ ├── columns: x:6!null y:7 - │ │ │ │ ├── key: (6) - │ │ │ │ └── fd: (6)-->(7) + │ │ │ │ ├── columns: x:7!null y:8 + │ │ │ │ ├── key: (7) + │ │ │ │ └── fd: (7)-->(8) │ │ │ └── projections - │ │ │ └── y:7 IS NOT NULL [as=notnull:9, outer=(7)] + │ │ │ └── y:8 IS NOT NULL [as=notnull:11, outer=(8)] │ │ └── filters - │ │ ├── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ │ └── (i:2 = y:7) IS NOT false [outer=(2,7)] + │ │ ├── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + │ │ └── (i:2 = y:8) IS NOT false [outer=(2,8)] │ └── aggregations - │ ├── bool-or [as=bool_or:10, outer=(9)] - │ │ └── notnull:9 + │ ├── bool-or [as=bool_or:12, outer=(11)] + │ │ └── notnull:11 │ └── const-agg [as=i:2, outer=(2)] │ └── i:2 └── projections - └── CASE WHEN bool_or:10 AND (i:2 IS NOT NULL) THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=r:8, outer=(2,10)] + └── CASE WHEN bool_or:12 AND (i:2 IS NOT NULL) THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=r:10, outer=(2,12)] # Any clause with more complex expression that must be cached. norm expect=(TryDecorrelateProject,TryDecorrelateProjectSelect,TryDecorrelateScalarGroupBy) SELECT i*i/5=ANY(SELECT y FROM xy WHERE x=k) AS r FROM a ---- project - ├── columns: r:8 + ├── columns: r:10 ├── immutable ├── group-by - │ ├── columns: k:1!null scalar:9 bool_or:11 + │ ├── columns: k:1!null scalar:11 bool_or:13 │ ├── grouping columns: k:1!null │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(9,11) + │ ├── fd: (1)-->(11,13) │ ├── left-join (hash) - │ │ ├── columns: k:1!null x:6 y:7 scalar:9 notnull:10 + │ │ ├── columns: k:1!null x:7 y:8 scalar:11 notnull:12 │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ ├── immutable │ │ ├── key: (1) - │ │ ├── fd: (1)-->(6,7,9,10), (6)-->(7), (7)~~>(10) + │ │ ├── fd: (1)-->(7,8,11,12), (7)-->(8), (8)~~>(12) │ │ ├── project - │ │ │ ├── columns: scalar:9 k:1!null + │ │ │ ├── columns: scalar:11 k:1!null │ │ │ ├── immutable │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(9) + │ │ │ ├── fd: (1)-->(11) │ │ │ ├── scan a │ │ │ │ ├── columns: k:1!null i:2 │ │ │ │ ├── key: (1) │ │ │ │ └── fd: (1)-->(2) │ │ │ └── projections - │ │ │ └── (i:2 * i:2) / 5 [as=scalar:9, outer=(2), immutable] + │ │ │ └── (i:2 * i:2) / 5 [as=scalar:11, outer=(2), immutable] │ │ ├── project - │ │ │ ├── columns: notnull:10!null x:6!null y:7 - │ │ │ ├── key: (6) - │ │ │ ├── fd: (6)-->(7), (7)-->(10) + │ │ │ ├── columns: notnull:12!null x:7!null y:8 + │ │ │ ├── key: (7) + │ │ │ ├── fd: (7)-->(8), (8)-->(12) │ │ │ ├── scan xy - │ │ │ │ ├── columns: x:6!null y:7 - │ │ │ │ ├── key: (6) - │ │ │ │ └── fd: (6)-->(7) + │ │ │ │ ├── columns: x:7!null y:8 + │ │ │ │ ├── key: (7) + │ │ │ │ └── fd: (7)-->(8) │ │ │ └── projections - │ │ │ └── y:7 IS NOT NULL [as=notnull:10, outer=(7)] + │ │ │ └── y:8 IS NOT NULL [as=notnull:12, outer=(8)] │ │ └── filters - │ │ ├── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ │ └── (scalar:9 = y:7) IS NOT false [outer=(7,9)] + │ │ ├── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + │ │ └── (scalar:11 = y:8) IS NOT false [outer=(8,11)] │ └── aggregations - │ ├── bool-or [as=bool_or:11, outer=(10)] - │ │ └── notnull:10 - │ └── const-agg [as=scalar:9, outer=(9)] - │ └── scalar:9 + │ ├── bool-or [as=bool_or:13, outer=(12)] + │ │ └── notnull:12 + │ └── const-agg [as=scalar:11, outer=(11)] + │ └── scalar:11 └── projections - └── CASE WHEN bool_or:11 AND (scalar:9 IS NOT NULL) THEN true WHEN bool_or:11 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=r:8, outer=(9,11), immutable] + └── CASE WHEN bool_or:13 AND (scalar:11 IS NOT NULL) THEN true WHEN bool_or:13 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=r:10, outer=(11,13), immutable] # -------------------------------------------------- # TryDecorrelateProject @@ -1122,13 +1122,13 @@ distinct-on ├── grouping columns: k:1!null ├── key: (1) └── select - ├── columns: k:1!null x:6!null div:10!null - ├── fd: (6)==(10), (10)==(6) + ├── columns: k:1!null x:7!null div:13!null + ├── fd: (7)==(13), (13)==(7) ├── project - │ ├── columns: div:10!null k:1!null x:6!null + │ ├── columns: div:13!null k:1!null x:7!null │ ├── inner-join (cross) - │ │ ├── columns: k:1!null i:2!null x:6!null u:8!null - │ │ ├── key: (1,6,8) + │ │ ├── columns: k:1!null i:2!null x:7!null u:10!null + │ │ ├── key: (1,7,10) │ │ ├── fd: ()-->(2) │ │ ├── select │ │ │ ├── columns: k:1!null i:2!null @@ -1141,20 +1141,20 @@ distinct-on │ │ │ └── filters │ │ │ └── i:2 = 5 [outer=(2), constraints=(/2: [/5 - /5]; tight), fd=()-->(2)] │ │ ├── inner-join (cross) - │ │ │ ├── columns: x:6!null u:8!null - │ │ │ ├── key: (6,8) + │ │ │ ├── columns: x:7!null u:10!null + │ │ │ ├── key: (7,10) │ │ │ ├── scan xy - │ │ │ │ ├── columns: x:6!null - │ │ │ │ └── key: (6) + │ │ │ │ ├── columns: x:7!null + │ │ │ │ └── key: (7) │ │ │ ├── scan uv - │ │ │ │ ├── columns: u:8!null - │ │ │ │ └── key: (8) + │ │ │ │ ├── columns: u:10!null + │ │ │ │ └── key: (10) │ │ │ └── filters (true) │ │ └── filters (true) │ └── projections - │ └── u:8 / 1.1 [as=div:10, outer=(8)] + │ └── u:10 / 1.1 [as=div:13, outer=(10)] └── filters - └── x:6 = div:10 [outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ]), fd=(6)==(10), (10)==(6)] + └── x:7 = div:13 [outer=(7,13), constraints=(/7: (/NULL - ]; /13: (/NULL - ]), fd=(7)==(13), (13)==(7)] # Don't hoist Project operator in right join case. norm @@ -1176,28 +1176,28 @@ project │ ├── key: (1) │ └── fd: (1)-->(2) ├── left-join (cross) - │ ├── columns: x:6 div:10!null + │ ├── columns: x:7 div:13!null │ ├── outer: (2) │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) │ ├── project - │ │ ├── columns: div:10!null + │ │ ├── columns: div:13!null │ │ ├── outer: (2) │ │ ├── select - │ │ │ ├── columns: u:8!null + │ │ │ ├── columns: u:10!null │ │ │ ├── outer: (2) - │ │ │ ├── key: (8) + │ │ │ ├── key: (10) │ │ │ ├── scan uv - │ │ │ │ ├── columns: u:8!null - │ │ │ │ └── key: (8) + │ │ │ │ ├── columns: u:10!null + │ │ │ │ └── key: (10) │ │ │ └── filters │ │ │ └── i:2 = 5 [outer=(2), constraints=(/2: [/5 - /5]; tight), fd=()-->(2)] │ │ └── projections - │ │ └── u:8 / 1.1 [as=div:10, outer=(8)] + │ │ └── u:10 / 1.1 [as=div:13, outer=(10)] │ ├── scan xy - │ │ ├── columns: x:6!null - │ │ └── key: (6) + │ │ ├── columns: x:7!null + │ │ └── key: (7) │ └── filters - │ └── x:6 = div:10 [outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ]), fd=(6)==(10), (10)==(6)] + │ └── x:7 = div:13 [outer=(7,13), constraints=(/7: (/NULL - ]; /13: (/NULL - ]), fd=(7)==(13), (13)==(7)] └── filters (true) # -------------------------------------------------- @@ -1224,30 +1224,30 @@ project │ ├── key: (1) │ └── fd: (1)-->(2) ├── left-join (hash) - │ ├── columns: x:6!null plus:10 + │ ├── columns: x:7!null plus:13 │ ├── outer: (2) │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) │ ├── immutable │ ├── scan xy - │ │ ├── columns: x:6!null - │ │ └── key: (6) + │ │ ├── columns: x:7!null + │ │ └── key: (7) │ ├── project - │ │ ├── columns: plus:10!null + │ │ ├── columns: plus:13!null │ │ ├── outer: (2) │ │ ├── immutable │ │ ├── select - │ │ │ ├── columns: u:8!null + │ │ │ ├── columns: u:10!null │ │ │ ├── outer: (2) - │ │ │ ├── key: (8) + │ │ │ ├── key: (10) │ │ │ ├── scan uv - │ │ │ │ ├── columns: u:8!null - │ │ │ │ └── key: (8) + │ │ │ │ ├── columns: u:10!null + │ │ │ │ └── key: (10) │ │ │ └── filters │ │ │ └── i:2 = 5 [outer=(2), constraints=(/2: [/5 - /5]; tight), fd=()-->(2)] │ │ └── projections - │ │ └── u:8 + 1 [as=plus:10, outer=(8), immutable] + │ │ └── u:10 + 1 [as=plus:13, outer=(10), immutable] │ └── filters - │ └── x:6 = plus:10 [outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ]), fd=(6)==(10), (10)==(6)] + │ └── x:7 = plus:13 [outer=(7,13), constraints=(/7: (/NULL - ]; /13: (/NULL - ]), fd=(7)==(13), (13)==(7)] └── filters (true) # Don't decorrelate FULL JOIN case. @@ -1272,30 +1272,30 @@ project │ ├── key: (1) │ └── fd: (1)-->(2) ├── full-join (hash) - │ ├── columns: x:6 plus:10 + │ ├── columns: x:7 plus:13 │ ├── outer: (2) │ ├── multiplicity: left-rows(one-or-more), right-rows(exactly-one) │ ├── immutable │ ├── scan xy - │ │ ├── columns: x:6!null - │ │ └── key: (6) + │ │ ├── columns: x:7!null + │ │ └── key: (7) │ ├── project - │ │ ├── columns: plus:10!null + │ │ ├── columns: plus:13!null │ │ ├── outer: (2) │ │ ├── immutable │ │ ├── select - │ │ │ ├── columns: u:8!null + │ │ │ ├── columns: u:10!null │ │ │ ├── outer: (2) - │ │ │ ├── key: (8) + │ │ │ ├── key: (10) │ │ │ ├── scan uv - │ │ │ │ ├── columns: u:8!null - │ │ │ │ └── key: (8) + │ │ │ │ ├── columns: u:10!null + │ │ │ │ └── key: (10) │ │ │ └── filters │ │ │ └── i:2 = 5 [outer=(2), constraints=(/2: [/5 - /5]; tight), fd=()-->(2)] │ │ └── projections - │ │ └── u:8 + 1 [as=plus:10, outer=(8), immutable] + │ │ └── u:10 + 1 [as=plus:13, outer=(10), immutable] │ └── filters - │ └── x:6 = plus:10 [outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ]), fd=(6)==(10), (10)==(6)] + │ └── x:7 = plus:13 [outer=(7,13), constraints=(/7: (/NULL - ]; /13: (/NULL - ]), fd=(7)==(13), (13)==(7)] └── filters (true) # -------------------------------------------------- @@ -1305,52 +1305,52 @@ norm expect=TryDecorrelateProjectInnerJoin SELECT (SELECT sum(y + v) FROM xy, uv WHERE x=u AND x=k) FROM a ---- project - ├── columns: sum:12 + ├── columns: sum:15 ├── immutable ├── group-by - │ ├── columns: k:1!null sum:11 + │ ├── columns: k:1!null sum:14 │ ├── grouping columns: k:1!null │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(11) + │ ├── fd: (1)-->(14) │ ├── left-join (hash) - │ │ ├── columns: k:1!null x:6 column10:10 + │ │ ├── columns: k:1!null x:7 column13:13 │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ ├── immutable │ │ ├── key: (1) - │ │ ├── fd: (6)-->(10), (1)-->(6,10) + │ │ ├── fd: (7)-->(13), (1)-->(7,13) │ │ ├── scan a │ │ │ ├── columns: k:1!null │ │ │ └── key: (1) │ │ ├── project - │ │ │ ├── columns: column10:10 x:6!null + │ │ │ ├── columns: column13:13 x:7!null │ │ │ ├── immutable - │ │ │ ├── key: (6) - │ │ │ ├── fd: (6)-->(10) + │ │ │ ├── key: (7) + │ │ │ ├── fd: (7)-->(13) │ │ │ ├── inner-join (hash) - │ │ │ │ ├── columns: x:6!null y:7 u:8!null v:9 + │ │ │ │ ├── columns: x:7!null y:8 u:10!null v:11 │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - │ │ │ │ ├── key: (8) - │ │ │ │ ├── fd: (6)-->(7), (8)-->(9), (6)==(8), (8)==(6) + │ │ │ │ ├── key: (10) + │ │ │ │ ├── fd: (7)-->(8), (10)-->(11), (7)==(10), (10)==(7) │ │ │ │ ├── scan xy - │ │ │ │ │ ├── columns: x:6!null y:7 - │ │ │ │ │ ├── key: (6) - │ │ │ │ │ └── fd: (6)-->(7) + │ │ │ │ │ ├── columns: x:7!null y:8 + │ │ │ │ │ ├── key: (7) + │ │ │ │ │ └── fd: (7)-->(8) │ │ │ │ ├── scan uv - │ │ │ │ │ ├── columns: u:8!null v:9 - │ │ │ │ │ ├── key: (8) - │ │ │ │ │ └── fd: (8)-->(9) + │ │ │ │ │ ├── columns: u:10!null v:11 + │ │ │ │ │ ├── key: (10) + │ │ │ │ │ └── fd: (10)-->(11) │ │ │ │ └── filters - │ │ │ │ └── x:6 = u:8 [outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ]), fd=(6)==(8), (8)==(6)] + │ │ │ │ └── x:7 = u:10 [outer=(7,10), constraints=(/7: (/NULL - ]; /10: (/NULL - ]), fd=(7)==(10), (10)==(7)] │ │ │ └── projections - │ │ │ └── y:7 + v:9 [as=column10:10, outer=(7,9), immutable] + │ │ │ └── y:8 + v:11 [as=column13:13, outer=(8,11), immutable] │ │ └── filters - │ │ └── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ │ └── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] │ └── aggregations - │ └── sum [as=sum:11, outer=(10)] - │ └── column10:10 + │ └── sum [as=sum:14, outer=(13)] + │ └── column13:13 └── projections - └── sum:11 [as=sum:12, outer=(11)] + └── sum:14 [as=sum:15, outer=(14)] # -------------------------------------------------- # TryDecorrelateInnerJoin @@ -1370,20 +1370,20 @@ semi-join (hash) │ ├── columns: k:1!null │ └── key: (1) ├── inner-join (hash) - │ ├── columns: x:6!null u:8!null + │ ├── columns: x:7!null u:10!null │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - │ ├── key: (8) - │ ├── fd: (6)==(8), (8)==(6) + │ ├── key: (10) + │ ├── fd: (7)==(10), (10)==(7) │ ├── scan xy - │ │ ├── columns: x:6!null - │ │ └── key: (6) + │ │ ├── columns: x:7!null + │ │ └── key: (7) │ ├── scan uv - │ │ ├── columns: u:8!null - │ │ └── key: (8) + │ │ ├── columns: u:10!null + │ │ └── key: (10) │ └── filters - │ └── x:6 = u:8 [outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ]), fd=(6)==(8), (8)==(6)] + │ └── x:7 = u:10 [outer=(7,10), constraints=(/7: (/NULL - ]; /10: (/NULL - ]), fd=(7)==(10), (10)==(7)] └── filters - └── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # Anti-join as outer. norm expect=TryDecorrelateInnerJoin @@ -1400,20 +1400,20 @@ anti-join (hash) │ ├── columns: k:1!null │ └── key: (1) ├── inner-join (hash) - │ ├── columns: x:6!null u:8!null + │ ├── columns: x:7!null u:10!null │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - │ ├── key: (8) - │ ├── fd: (6)==(8), (8)==(6) + │ ├── key: (10) + │ ├── fd: (7)==(10), (10)==(7) │ ├── scan xy - │ │ ├── columns: x:6!null - │ │ └── key: (6) + │ │ ├── columns: x:7!null + │ │ └── key: (7) │ ├── scan uv - │ │ ├── columns: u:8!null - │ │ └── key: (8) + │ │ ├── columns: u:10!null + │ │ └── key: (10) │ └── filters - │ └── x:6 = u:8 [outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ]), fd=(6)==(8), (8)==(6)] + │ └── x:7 = u:10 [outer=(7,10), constraints=(/7: (/NULL - ]; /10: (/NULL - ]), fd=(7)==(10), (10)==(7)] └── filters - └── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # Right-join as outer. norm expect=TryDecorrelateInnerJoin @@ -1430,42 +1430,42 @@ project ├── columns: k:1!null ├── key: (1) └── select - ├── columns: k:1!null count_rows:10!null + ├── columns: k:1!null count_rows:13!null ├── key: (1) - ├── fd: (1)-->(10) + ├── fd: (1)-->(13) ├── group-by - │ ├── columns: k:1!null count_rows:10!null + │ ├── columns: k:1!null count_rows:13!null │ ├── grouping columns: k:1!null │ ├── key: (1) - │ ├── fd: (1)-->(10) + │ ├── fd: (1)-->(13) │ ├── left-join (hash) - │ │ ├── columns: k:1!null x:6 u:8 + │ │ ├── columns: k:1!null x:7 u:10 │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ ├── key: (1) - │ │ ├── fd: (6)==(8), (8)==(6), (1)-->(6,8) + │ │ ├── fd: (7)==(10), (10)==(7), (1)-->(7,10) │ │ ├── scan a │ │ │ ├── columns: k:1!null │ │ │ └── key: (1) │ │ ├── inner-join (hash) - │ │ │ ├── columns: x:6!null u:8!null + │ │ │ ├── columns: x:7!null u:10!null │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - │ │ │ ├── key: (8) - │ │ │ ├── fd: (6)==(8), (8)==(6) + │ │ │ ├── key: (10) + │ │ │ ├── fd: (7)==(10), (10)==(7) │ │ │ ├── scan xy - │ │ │ │ ├── columns: x:6!null - │ │ │ │ └── key: (6) + │ │ │ │ ├── columns: x:7!null + │ │ │ │ └── key: (7) │ │ │ ├── scan uv - │ │ │ │ ├── columns: u:8!null - │ │ │ │ └── key: (8) + │ │ │ │ ├── columns: u:10!null + │ │ │ │ └── key: (10) │ │ │ └── filters - │ │ │ └── x:6 = u:8 [outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ]), fd=(6)==(8), (8)==(6)] + │ │ │ └── x:7 = u:10 [outer=(7,10), constraints=(/7: (/NULL - ]; /10: (/NULL - ]), fd=(7)==(10), (10)==(7)] │ │ └── filters - │ │ └── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ │ └── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] │ └── aggregations - │ └── count [as=count_rows:10, outer=(6)] - │ └── x:6 + │ └── count [as=count_rows:13, outer=(7)] + │ └── x:7 └── filters - └── count_rows:10 IS DISTINCT FROM 1 [outer=(10), constraints=(/10: [ - /0] [/2 - ]; tight)] + └── count_rows:13 IS DISTINCT FROM 1 [outer=(13), constraints=(/13: [ - /0] [/2 - ]; tight)] # Can't decorrelate left-join as inner. norm @@ -1482,20 +1482,20 @@ semi-join-apply │ ├── columns: k:1!null │ └── key: (1) ├── left-join (hash) - │ ├── columns: x:6!null u:8 + │ ├── columns: x:7!null u:10 │ ├── outer: (1) │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) - │ ├── key: (6) - │ ├── fd: (6)-->(8) + │ ├── key: (7) + │ ├── fd: (7)-->(10) │ ├── scan xy - │ │ ├── columns: x:6!null - │ │ └── key: (6) + │ │ ├── columns: x:7!null + │ │ └── key: (7) │ ├── scan uv - │ │ ├── columns: u:8!null - │ │ └── key: (8) + │ │ ├── columns: u:10!null + │ │ └── key: (10) │ └── filters - │ ├── x:6 = u:8 [outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ]), fd=(6)==(8), (8)==(6)] - │ └── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ ├── x:7 = u:10 [outer=(7,10), constraints=(/7: (/NULL - ]; /10: (/NULL - ]), fd=(7)==(10), (10)==(7)] + │ └── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] └── filters (true) # Can't decorrelate semi-join as inner. @@ -1522,14 +1522,14 @@ semi-join-apply │ ├── outer: (1) │ ├── scan xy │ ├── inner-join (cross) - │ │ ├── columns: uv2.u:10!null + │ │ ├── columns: uv2.u:13!null │ │ ├── scan uv │ │ ├── scan uv2 - │ │ │ ├── columns: uv2.u:10!null - │ │ │ └── key: (10) + │ │ │ ├── columns: uv2.u:13!null + │ │ │ └── key: (13) │ │ └── filters (true) │ └── filters - │ └── uv2.u:10 = k:1 [outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] + │ └── uv2.u:13 = k:1 [outer=(1,13), constraints=(/1: (/NULL - ]; /13: (/NULL - ]), fd=(1)==(13), (13)==(1)] └── filters (true) # -------------------------------------------------- @@ -1558,27 +1558,27 @@ select └── filters └── exists [subquery] └── limit - ├── columns: k:2!null i:3!null x:7!null y:8!null + ├── columns: k:2!null i:3!null x:8!null y:9!null ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(2,3,7,8) + ├── fd: ()-->(2,3,8,9) ├── inner-join (hash) - │ ├── columns: k:2!null i:3!null x:7!null y:8!null + │ ├── columns: k:2!null i:3!null x:8!null y:9!null │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - │ ├── key: (7) - │ ├── fd: (2)-->(3), (7)-->(8), (2)==(7), (7)==(2), (3)==(8), (8)==(3) + │ ├── key: (8) + │ ├── fd: (2)-->(3), (8)-->(9), (2)==(8), (8)==(2), (3)==(9), (9)==(3) │ ├── limit hint: 1.00 │ ├── scan a │ │ ├── columns: k:2!null i:3 │ │ ├── key: (2) │ │ └── fd: (2)-->(3) │ ├── scan xy - │ │ ├── columns: x:7!null y:8 - │ │ ├── key: (7) - │ │ └── fd: (7)-->(8) + │ │ ├── columns: x:8!null y:9 + │ │ ├── key: (8) + │ │ └── fd: (8)-->(9) │ └── filters - │ ├── x:7 = k:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] - │ └── i:3 = y:8 [outer=(3,8), constraints=(/3: (/NULL - ]; /8: (/NULL - ]), fd=(3)==(8), (8)==(3)] + │ ├── x:8 = k:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] + │ └── i:3 = y:9 [outer=(3,9), constraints=(/3: (/NULL - ]; /9: (/NULL - ]), fd=(3)==(9), (9)==(3)] └── 1 norm expect=TryDecorrelateInnerLeftJoin @@ -1587,39 +1587,39 @@ FROM xy, uv WHERE (SELECT i FROM a WHERE k=x) IS DISTINCT FROM u ---- project - ├── columns: x:1!null y:2 u:3!null v:4 - ├── key: (1,3) - ├── fd: (1)-->(2), (3)-->(4) + ├── columns: x:1!null y:2 u:4!null v:5 + ├── key: (1,4) + ├── fd: (1)-->(2), (4)-->(5) └── select - ├── columns: x:1!null y:2 u:3!null v:4 k:5 i:6 - ├── key: (1,3) - ├── fd: (1)-->(2), (3)-->(4), (5)-->(6), (1,3)-->(5,6) + ├── columns: x:1!null y:2 u:4!null v:5 k:7 i:8 + ├── key: (1,4) + ├── fd: (1)-->(2), (4)-->(5), (7)-->(8), (1,4)-->(7,8) ├── left-join (hash) - │ ├── columns: x:1!null y:2 u:3!null v:4 k:5 i:6 + │ ├── columns: x:1!null y:2 u:4!null v:5 k:7 i:8 │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ ├── key: (1,3) - │ ├── fd: (1)-->(2), (3)-->(4), (5)-->(6), (1,3)-->(5,6) + │ ├── key: (1,4) + │ ├── fd: (1)-->(2), (4)-->(5), (7)-->(8), (1,4)-->(7,8) │ ├── inner-join (cross) - │ │ ├── columns: x:1!null y:2 u:3!null v:4 - │ │ ├── key: (1,3) - │ │ ├── fd: (1)-->(2), (3)-->(4) + │ │ ├── columns: x:1!null y:2 u:4!null v:5 + │ │ ├── key: (1,4) + │ │ ├── fd: (1)-->(2), (4)-->(5) │ │ ├── scan xy │ │ │ ├── columns: x:1!null y:2 │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2) │ │ ├── scan uv - │ │ │ ├── columns: u:3!null v:4 - │ │ │ ├── key: (3) - │ │ │ └── fd: (3)-->(4) + │ │ │ ├── columns: u:4!null v:5 + │ │ │ ├── key: (4) + │ │ │ └── fd: (4)-->(5) │ │ └── filters (true) │ ├── scan a - │ │ ├── columns: k:5!null i:6 - │ │ ├── key: (5) - │ │ └── fd: (5)-->(6) + │ │ ├── columns: k:7!null i:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ └── filters - │ └── k:5 = x:1 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + │ └── k:7 = x:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] └── filters - └── u:3 IS DISTINCT FROM i:6 [outer=(3,6)] + └── u:4 IS DISTINCT FROM i:8 [outer=(4,8)] # -------------------------------------------------- # TryDecorrelateGroupBy @@ -1638,16 +1638,16 @@ group-by ├── key: (1) ├── fd: ()-->(2), (1)-->(2-5) ├── select - │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:6!null v:9 count_rows:10!null - │ ├── key: (1,6,9) - │ ├── fd: ()-->(2), (1)-->(3-5), (1,6,9)-->(3-5,10), (6)==(10), (10)==(6) + │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:7!null v:11 count_rows:13!null + │ ├── key: (1,7,11) + │ ├── fd: ()-->(2), (1)-->(3-5), (1,7,11)-->(3-5,13), (7)==(13), (13)==(7) │ ├── group-by - │ │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:6!null v:9 count_rows:10!null - │ │ ├── grouping columns: k:1!null x:6!null v:9 - │ │ ├── key: (1,6,9) - │ │ ├── fd: ()-->(2), (1)-->(3-5), (1,6,9)-->(2-5,10) + │ │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:7!null v:11 count_rows:13!null + │ │ ├── grouping columns: k:1!null x:7!null v:11 + │ │ ├── key: (1,7,11) + │ │ ├── fd: ()-->(2), (1)-->(3-5), (1,7,11)-->(2-5,13) │ │ ├── inner-join (cross) - │ │ │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:6!null v:9 + │ │ │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:7!null v:11 │ │ │ ├── fd: ()-->(2), (1)-->(3-5) │ │ │ ├── select │ │ │ │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 @@ -1660,16 +1660,16 @@ group-by │ │ │ │ └── filters │ │ │ │ └── i:2 = 5 [outer=(2), constraints=(/2: [/5 - /5]; tight), fd=()-->(2)] │ │ │ ├── inner-join (cross) - │ │ │ │ ├── columns: x:6!null v:9 + │ │ │ │ ├── columns: x:7!null v:11 │ │ │ │ ├── scan xy - │ │ │ │ │ ├── columns: x:6!null - │ │ │ │ │ └── key: (6) + │ │ │ │ │ ├── columns: x:7!null + │ │ │ │ │ └── key: (7) │ │ │ │ ├── scan uv - │ │ │ │ │ └── columns: v:9 + │ │ │ │ │ └── columns: v:11 │ │ │ │ └── filters (true) │ │ │ └── filters (true) │ │ └── aggregations - │ │ ├── count-rows [as=count_rows:10] + │ │ ├── count-rows [as=count_rows:13] │ │ ├── const-agg [as=i:2, outer=(2)] │ │ │ └── i:2 │ │ ├── const-agg [as=f:3, outer=(3)] @@ -1679,7 +1679,7 @@ group-by │ │ └── const-agg [as=j:5, outer=(5)] │ │ └── j:5 │ └── filters - │ └── x:6 = count_rows:10 [outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ]), fd=(6)==(10), (10)==(6)] + │ └── x:7 = count_rows:13 [outer=(7,13), constraints=(/7: (/NULL - ]; /13: (/NULL - ]), fd=(7)==(13), (13)==(7)] └── aggregations ├── const-agg [as=i:2, outer=(2)] │ └── i:2 @@ -1704,16 +1704,16 @@ group-by ├── key: (1) ├── fd: ()-->(2), (1)-->(2-5) ├── select - │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:6!null v:9 count:10!null - │ ├── key: (1,6,9) - │ ├── fd: ()-->(2), (1)-->(3-5), (1,6,9)-->(3-5,10), (6)==(10), (10)==(6) + │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:7!null v:11 count:13!null + │ ├── key: (1,7,11) + │ ├── fd: ()-->(2), (1)-->(3-5), (1,7,11)-->(3-5,13), (7)==(13), (13)==(7) │ ├── group-by - │ │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:6!null v:9 count:10!null - │ │ ├── grouping columns: k:1!null x:6!null v:9 - │ │ ├── key: (1,6,9) - │ │ ├── fd: ()-->(2), (1)-->(3-5), (1,6,9)-->(2-5,10) + │ │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:7!null v:11 count:13!null + │ │ ├── grouping columns: k:1!null x:7!null v:11 + │ │ ├── key: (1,7,11) + │ │ ├── fd: ()-->(2), (1)-->(3-5), (1,7,11)-->(2-5,13) │ │ ├── inner-join (cross) - │ │ │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:6!null v:9 + │ │ │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:7!null v:11 │ │ │ ├── fd: ()-->(2), (1)-->(3-5) │ │ │ ├── select │ │ │ │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 @@ -1726,18 +1726,18 @@ group-by │ │ │ │ └── filters │ │ │ │ └── i:2 = 5 [outer=(2), constraints=(/2: [/5 - /5]; tight), fd=()-->(2)] │ │ │ ├── inner-join (cross) - │ │ │ │ ├── columns: x:6!null v:9 + │ │ │ │ ├── columns: x:7!null v:11 │ │ │ │ ├── scan xy - │ │ │ │ │ ├── columns: x:6!null - │ │ │ │ │ └── key: (6) + │ │ │ │ │ ├── columns: x:7!null + │ │ │ │ │ └── key: (7) │ │ │ │ ├── scan uv - │ │ │ │ │ └── columns: v:9 + │ │ │ │ │ └── columns: v:11 │ │ │ │ └── filters (true) │ │ │ └── filters (true) │ │ └── aggregations - │ │ ├── agg-distinct [as=count:10, outer=(9)] + │ │ ├── agg-distinct [as=count:13, outer=(11)] │ │ │ └── count - │ │ │ └── v:9 + │ │ │ └── v:11 │ │ ├── const-agg [as=i:2, outer=(2)] │ │ │ └── i:2 │ │ ├── const-agg [as=f:3, outer=(3)] @@ -1747,7 +1747,7 @@ group-by │ │ └── const-agg [as=j:5, outer=(5)] │ │ └── j:5 │ └── filters - │ └── x:6 = count:10 [outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ]), fd=(6)==(10), (10)==(6)] + │ └── x:7 = count:13 [outer=(7,13), constraints=(/7: (/NULL - ]; /13: (/NULL - ]), fd=(7)==(13), (13)==(7)] └── aggregations ├── const-agg [as=i:2, outer=(2)] │ └── i:2 @@ -1765,61 +1765,61 @@ FROM xy, uv WHERE x=v AND u=(SELECT max(i) FROM a WHERE k=x) ---- project - ├── columns: x:1!null y:2 u:3!null v:4!null - ├── key: (3) - ├── fd: (1)-->(2), (3)-->(1,2,4), (1)==(4), (4)==(1) + ├── columns: x:1!null y:2 u:4!null v:5!null + ├── key: (4) + ├── fd: (1)-->(2), (4)-->(1,2,5), (1)==(5), (5)==(1) └── select - ├── columns: x:1!null y:2 u:3!null v:4!null max:10!null - ├── key: (3) - ├── fd: (1)-->(2), (3)-->(1,2,4), (1)==(4), (4)==(1), (3)==(10), (10)==(3) + ├── columns: x:1!null y:2 u:4!null v:5!null max:13!null + ├── key: (4) + ├── fd: (1)-->(2), (4)-->(1,2,5), (1)==(5), (5)==(1), (4)==(13), (13)==(4) ├── group-by - │ ├── columns: x:1!null y:2 u:3!null v:4!null max:10!null - │ ├── grouping columns: u:3!null - │ ├── key: (3) - │ ├── fd: (1)-->(2), (3)-->(1,2,4,10), (1)==(4), (4)==(1) + │ ├── columns: x:1!null y:2 u:4!null v:5!null max:13!null + │ ├── grouping columns: u:4!null + │ ├── key: (4) + │ ├── fd: (1)-->(2), (4)-->(1,2,5,13), (1)==(5), (5)==(1) │ ├── inner-join (hash) - │ │ ├── columns: x:1!null y:2 u:3!null v:4!null k:5!null i:6!null + │ │ ├── columns: x:1!null y:2 u:4!null v:5!null k:7!null i:8!null │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ ├── key: (3) - │ │ ├── fd: (1)-->(2), (3)-->(4), (1)==(4,5), (4)==(1,5), (5)-->(6), (5)==(1,4) + │ │ ├── key: (4) + │ │ ├── fd: (1)-->(2), (4)-->(5), (1)==(5,7), (5)==(1,7), (7)-->(8), (7)==(1,5) │ │ ├── inner-join (hash) - │ │ │ ├── columns: x:1!null y:2 u:3!null v:4!null + │ │ │ ├── columns: x:1!null y:2 u:4!null v:5!null │ │ │ ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) - │ │ │ ├── key: (3) - │ │ │ ├── fd: (1)-->(2), (3)-->(4), (1)==(4), (4)==(1) + │ │ │ ├── key: (4) + │ │ │ ├── fd: (1)-->(2), (4)-->(5), (1)==(5), (5)==(1) │ │ │ ├── scan xy │ │ │ │ ├── columns: x:1!null y:2 │ │ │ │ ├── key: (1) │ │ │ │ └── fd: (1)-->(2) │ │ │ ├── scan uv - │ │ │ │ ├── columns: u:3!null v:4 - │ │ │ │ ├── key: (3) - │ │ │ │ └── fd: (3)-->(4) + │ │ │ │ ├── columns: u:4!null v:5 + │ │ │ │ ├── key: (4) + │ │ │ │ └── fd: (4)-->(5) │ │ │ └── filters - │ │ │ └── x:1 = v:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + │ │ │ └── x:1 = v:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] │ │ ├── select - │ │ │ ├── columns: k:5!null i:6!null - │ │ │ ├── key: (5) - │ │ │ ├── fd: (5)-->(6) + │ │ │ ├── columns: k:7!null i:8!null + │ │ │ ├── key: (7) + │ │ │ ├── fd: (7)-->(8) │ │ │ ├── scan a - │ │ │ │ ├── columns: k:5!null i:6 - │ │ │ │ ├── key: (5) - │ │ │ │ └── fd: (5)-->(6) + │ │ │ │ ├── columns: k:7!null i:8 + │ │ │ │ ├── key: (7) + │ │ │ │ └── fd: (7)-->(8) │ │ │ └── filters - │ │ │ └── i:6 IS NOT NULL [outer=(6), constraints=(/6: (/NULL - ]; tight)] + │ │ │ └── i:8 IS NOT NULL [outer=(8), constraints=(/8: (/NULL - ]; tight)] │ │ └── filters - │ │ └── k:5 = x:1 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + │ │ └── k:7 = x:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] │ └── aggregations - │ ├── max [as=max:10, outer=(6)] - │ │ └── i:6 - │ ├── const-agg [as=v:4, outer=(4)] - │ │ └── v:4 + │ ├── max [as=max:13, outer=(8)] + │ │ └── i:8 + │ ├── const-agg [as=v:5, outer=(5)] + │ │ └── v:5 │ ├── const-agg [as=y:2, outer=(2)] │ │ └── y:2 │ └── const-agg [as=x:1, outer=(1)] │ └── x:1 └── filters - └── u:3 = max:10 [outer=(3,10), constraints=(/3: (/NULL - ]; /10: (/NULL - ]), fd=(3)==(10), (10)==(3)] + └── u:4 = max:13 [outer=(4,13), constraints=(/4: (/NULL - ]; /13: (/NULL - ]), fd=(4)==(13), (13)==(4)] # Indirectly decorrelate GROUP BY after decorrelating scalar GROUP BY. Use # IS DISTINCT FROM to retain left join. @@ -1829,55 +1829,55 @@ FROM xy, uv WHERE x=v AND (SELECT max(i) FROM a WHERE k=x) IS DISTINCT FROM u ---- project - ├── columns: x:1!null y:2 u:3!null v:4!null - ├── key: (3) - ├── fd: (1)-->(2), (3)-->(1,2,4), (1)==(4), (4)==(1) + ├── columns: x:1!null y:2 u:4!null v:5!null + ├── key: (4) + ├── fd: (1)-->(2), (4)-->(1,2,5), (1)==(5), (5)==(1) └── select - ├── columns: x:1!null y:2 u:3!null v:4!null max:10 - ├── key: (3) - ├── fd: (1)-->(2), (3)-->(1,2,4,10), (1)==(4), (4)==(1) + ├── columns: x:1!null y:2 u:4!null v:5!null max:13 + ├── key: (4) + ├── fd: (1)-->(2), (4)-->(1,2,5,13), (1)==(5), (5)==(1) ├── group-by - │ ├── columns: x:1!null y:2 u:3!null v:4!null max:10 - │ ├── grouping columns: u:3!null - │ ├── key: (3) - │ ├── fd: (1)-->(2), (3)-->(1,2,4,10), (1)==(4), (4)==(1) + │ ├── columns: x:1!null y:2 u:4!null v:5!null max:13 + │ ├── grouping columns: u:4!null + │ ├── key: (4) + │ ├── fd: (1)-->(2), (4)-->(1,2,5,13), (1)==(5), (5)==(1) │ ├── left-join (hash) - │ │ ├── columns: x:1!null y:2 u:3!null v:4!null k:5 i:6 + │ │ ├── columns: x:1!null y:2 u:4!null v:5!null k:7 i:8 │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ │ ├── key: (3) - │ │ ├── fd: (1)-->(2), (3)-->(4-6), (1)==(4), (4)==(1), (5)-->(6) + │ │ ├── key: (4) + │ │ ├── fd: (1)-->(2), (4)-->(5,7,8), (1)==(5), (5)==(1), (7)-->(8) │ │ ├── inner-join (hash) - │ │ │ ├── columns: x:1!null y:2 u:3!null v:4!null + │ │ │ ├── columns: x:1!null y:2 u:4!null v:5!null │ │ │ ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) - │ │ │ ├── key: (3) - │ │ │ ├── fd: (1)-->(2), (3)-->(4), (1)==(4), (4)==(1) + │ │ │ ├── key: (4) + │ │ │ ├── fd: (1)-->(2), (4)-->(5), (1)==(5), (5)==(1) │ │ │ ├── scan xy │ │ │ │ ├── columns: x:1!null y:2 │ │ │ │ ├── key: (1) │ │ │ │ └── fd: (1)-->(2) │ │ │ ├── scan uv - │ │ │ │ ├── columns: u:3!null v:4 - │ │ │ │ ├── key: (3) - │ │ │ │ └── fd: (3)-->(4) + │ │ │ │ ├── columns: u:4!null v:5 + │ │ │ │ ├── key: (4) + │ │ │ │ └── fd: (4)-->(5) │ │ │ └── filters - │ │ │ └── x:1 = v:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + │ │ │ └── x:1 = v:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] │ │ ├── scan a - │ │ │ ├── columns: k:5!null i:6 - │ │ │ ├── key: (5) - │ │ │ └── fd: (5)-->(6) + │ │ │ ├── columns: k:7!null i:8 + │ │ │ ├── key: (7) + │ │ │ └── fd: (7)-->(8) │ │ └── filters - │ │ └── k:5 = x:1 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + │ │ └── k:7 = x:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] │ └── aggregations - │ ├── max [as=max:10, outer=(6)] - │ │ └── i:6 - │ ├── const-agg [as=v:4, outer=(4)] - │ │ └── v:4 + │ ├── max [as=max:13, outer=(8)] + │ │ └── i:8 + │ ├── const-agg [as=v:5, outer=(5)] + │ │ └── v:5 │ ├── const-agg [as=y:2, outer=(2)] │ │ └── y:2 │ └── const-agg [as=x:1, outer=(1)] │ └── x:1 └── filters - └── u:3 IS DISTINCT FROM max:10 [outer=(3,10)] + └── u:4 IS DISTINCT FROM max:13 [outer=(4,13)] # Synthesize key when one is not present. norm expect=TryDecorrelateGroupBy @@ -1889,48 +1889,48 @@ FROM WHERE 'bar'=(SELECT max(s) FROM (SELECT * FROM a LIMIT 1) WHERE k=y GROUP BY i) ---- project - ├── columns: y:2!null cst:3!null - ├── fd: ()-->(2,3) + ├── columns: y:2!null cst:4!null + ├── fd: ()-->(2,4) ├── select - │ ├── columns: x:1!null y:2!null max:9!null + │ ├── columns: x:1!null y:2!null max:11!null │ ├── key: (1) - │ ├── fd: ()-->(2,9) + │ ├── fd: ()-->(2,11) │ ├── group-by - │ │ ├── columns: x:1!null y:2!null max:9 + │ │ ├── columns: x:1!null y:2!null max:11 │ │ ├── grouping columns: x:1!null │ │ ├── key: (1) - │ │ ├── fd: ()-->(2), (1)-->(2,9) + │ │ ├── fd: ()-->(2), (1)-->(2,11) │ │ ├── inner-join (hash) - │ │ │ ├── columns: x:1!null y:2!null k:4!null s:7 + │ │ │ ├── columns: x:1!null y:2!null k:5!null s:8 │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) │ │ │ ├── key: (1) - │ │ │ ├── fd: ()-->(2,4,7), (2)==(4), (4)==(2) + │ │ │ ├── fd: ()-->(2,5,8), (2)==(5), (5)==(2) │ │ │ ├── scan xy │ │ │ │ ├── columns: x:1!null y:2 │ │ │ │ ├── key: (1) │ │ │ │ └── fd: (1)-->(2) │ │ │ ├── limit - │ │ │ │ ├── columns: k:4!null s:7 + │ │ │ │ ├── columns: k:5!null s:8 │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ ├── key: () - │ │ │ │ ├── fd: ()-->(4,7) + │ │ │ │ ├── fd: ()-->(5,8) │ │ │ │ ├── scan a - │ │ │ │ │ ├── columns: k:4!null s:7 - │ │ │ │ │ ├── key: (4) - │ │ │ │ │ ├── fd: (4)-->(7) + │ │ │ │ │ ├── columns: k:5!null s:8 + │ │ │ │ │ ├── key: (5) + │ │ │ │ │ ├── fd: (5)-->(8) │ │ │ │ │ └── limit hint: 1.00 │ │ │ │ └── 1 │ │ │ └── filters - │ │ │ └── k:4 = y:2 [outer=(2,4), constraints=(/2: (/NULL - ]; /4: (/NULL - ]), fd=(2)==(4), (4)==(2)] + │ │ │ └── k:5 = y:2 [outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ]), fd=(2)==(5), (5)==(2)] │ │ └── aggregations - │ │ ├── max [as=max:9, outer=(7)] - │ │ │ └── s:7 + │ │ ├── max [as=max:11, outer=(8)] + │ │ │ └── s:8 │ │ └── const-agg [as=y:2, outer=(2)] │ │ └── y:2 │ └── filters - │ └── max:9 = 'bar' [outer=(9), constraints=(/9: [/'bar' - /'bar']; tight), fd=()-->(9)] + │ └── max:11 = 'bar' [outer=(11), constraints=(/11: [/'bar' - /'bar']; tight), fd=()-->(11)] └── projections - └── 'foo' [as=cst:3] + └── 'foo' [as=cst:4] # Decorrelate DistinctOn. norm expect=TryDecorrelateGroupBy @@ -1949,18 +1949,18 @@ group-by ├── key: (1) ├── fd: ()-->(2), (1)-->(2-5) ├── select - │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:6!null u:8!null v:9 - │ ├── key: (1,8) - │ ├── fd: ()-->(2), (1)-->(3-5), (8)-->(9), (1,6,9)-->(3-5,8), (6)==(8), (8)==(6) + │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:7!null u:10!null v:11 + │ ├── key: (1,10) + │ ├── fd: ()-->(2), (1)-->(3-5), (10)-->(11), (1,7,11)-->(3-5,10), (7)==(10), (10)==(7) │ ├── distinct-on - │ │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:6!null u:8!null v:9 - │ │ ├── grouping columns: k:1!null x:6!null v:9 - │ │ ├── key: (1,6,8) - │ │ ├── fd: ()-->(2), (1)-->(3-5), (8)-->(9), (1,6,9)-->(2-5,8) + │ │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:7!null u:10!null v:11 + │ │ ├── grouping columns: k:1!null x:7!null v:11 + │ │ ├── key: (1,7,10) + │ │ ├── fd: ()-->(2), (1)-->(3-5), (10)-->(11), (1,7,11)-->(2-5,10) │ │ ├── inner-join (cross) - │ │ │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:6!null u:8!null v:9 - │ │ │ ├── key: (1,6,8) - │ │ │ ├── fd: ()-->(2), (1)-->(3-5), (8)-->(9) + │ │ │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:7!null u:10!null v:11 + │ │ │ ├── key: (1,7,10) + │ │ │ ├── fd: ()-->(2), (1)-->(3-5), (10)-->(11) │ │ │ ├── select │ │ │ │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 │ │ │ │ ├── key: (1) @@ -1972,21 +1972,21 @@ group-by │ │ │ │ └── filters │ │ │ │ └── i:2 = 5 [outer=(2), constraints=(/2: [/5 - /5]; tight), fd=()-->(2)] │ │ │ ├── inner-join (cross) - │ │ │ │ ├── columns: x:6!null u:8!null v:9 - │ │ │ │ ├── key: (6,8) - │ │ │ │ ├── fd: (8)-->(9) + │ │ │ │ ├── columns: x:7!null u:10!null v:11 + │ │ │ │ ├── key: (7,10) + │ │ │ │ ├── fd: (10)-->(11) │ │ │ │ ├── scan xy - │ │ │ │ │ ├── columns: x:6!null - │ │ │ │ │ └── key: (6) + │ │ │ │ │ ├── columns: x:7!null + │ │ │ │ │ └── key: (7) │ │ │ │ ├── scan uv - │ │ │ │ │ ├── columns: u:8!null v:9 - │ │ │ │ │ ├── key: (8) - │ │ │ │ │ └── fd: (8)-->(9) + │ │ │ │ │ ├── columns: u:10!null v:11 + │ │ │ │ │ ├── key: (10) + │ │ │ │ │ └── fd: (10)-->(11) │ │ │ │ └── filters (true) │ │ │ └── filters (true) │ │ └── aggregations - │ │ ├── first-agg [as=u:8, outer=(8)] - │ │ │ └── u:8 + │ │ ├── first-agg [as=u:10, outer=(10)] + │ │ │ └── u:10 │ │ ├── const-agg [as=i:2, outer=(2)] │ │ │ └── i:2 │ │ ├── const-agg [as=f:3, outer=(3)] @@ -1996,7 +1996,7 @@ group-by │ │ └── const-agg [as=j:5, outer=(5)] │ │ └── j:5 │ └── filters - │ └── x:6 = u:8 [outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ]), fd=(6)==(8), (8)==(6)] + │ └── x:7 = u:10 [outer=(7,10), constraints=(/7: (/NULL - ]; /10: (/NULL - ]), fd=(7)==(10), (10)==(7)] └── aggregations ├── const-agg [as=i:2, outer=(2)] │ └── i:2 @@ -2057,89 +2057,89 @@ WHERE ); ---- with &1 (w0) - ├── columns: "?column?":58 - ├── fd: ()-->(58) + ├── columns: "?column?":65 + ├── fd: ()-->(65) ├── values - │ ├── columns: "?column?":36 + │ ├── columns: "?column?":41 │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(36) + │ ├── fd: ()-->(41) │ └── (NULL,) └── project - ├── columns: "?column?":58 - ├── fd: ()-->(58) + ├── columns: "?column?":65 + ├── fd: ()-->(65) ├── semi-join-apply │ ├── columns: t0._string:6 │ ├── scan t0 │ │ └── columns: t0._string:6 │ ├── inner-join (hash) - │ │ ├── columns: tab_orig.rowid:14!null t1._decimal:19!null t1.rowid:21!null t2._bool:25!null t2.rowid:28!null t3._int2:29!null t3._timestamptz:31!null t3.rowid:35!null t4._int8:44!null t4._timestamptz:45!null t5._decimal:54!null + │ │ ├── columns: tab_orig.rowid:15!null t1._decimal:21!null t1.rowid:23!null t2._bool:28!null t2.rowid:31!null t3._int2:33!null t3._timestamptz:35!null t3.rowid:39!null t4._int8:49!null t4._timestamptz:50!null t5._decimal:60!null │ │ ├── outer: (6) - │ │ ├── fd: ()-->(25), (21)-->(19), (35)-->(29,31), (14,21,28,35)-->(19,29,31), (31)==(45), (45)==(31), (29)==(44), (44)==(29), (19)==(54), (54)==(19) + │ │ ├── fd: ()-->(28), (23)-->(21), (39)-->(33,35), (15,23,31,39)-->(21,33,35), (35)==(50), (50)==(35), (33)==(49), (49)==(33), (21)==(60), (60)==(21) │ │ ├── inner-join (hash) - │ │ │ ├── columns: tab_orig.rowid:14!null t1._decimal:19 t1.rowid:21!null t2._bool:25 t2.rowid:28!null t3._int2:29!null t3._timestamptz:31!null t3.rowid:35!null t4._int8:44!null t4._timestamptz:45!null + │ │ │ ├── columns: tab_orig.rowid:15!null t1._decimal:21 t1.rowid:23!null t2._bool:28 t2.rowid:31!null t3._int2:33!null t3._timestamptz:35!null t3.rowid:39!null t4._int8:49!null t4._timestamptz:50!null │ │ │ ├── outer: (6) - │ │ │ ├── fd: (21)-->(19), (28)-->(25), (35)-->(29,31), (14,21,28,35)-->(19,25,29,31), (31)==(45), (45)==(31), (29)==(44), (44)==(29) + │ │ │ ├── fd: (23)-->(21), (31)-->(28), (39)-->(33,35), (15,23,31,39)-->(21,28,33,35), (35)==(50), (50)==(35), (33)==(49), (49)==(33) │ │ │ ├── group-by - │ │ │ │ ├── columns: tab_orig.rowid:14!null t1._decimal:19 t1.rowid:21!null t2._bool:25 t2.rowid:28!null t3._int2:29 t3._timestamptz:31 t3.rowid:35!null - │ │ │ │ ├── grouping columns: tab_orig.rowid:14!null t1.rowid:21!null t2.rowid:28!null t3.rowid:35!null + │ │ │ │ ├── columns: tab_orig.rowid:15!null t1._decimal:21 t1.rowid:23!null t2._bool:28 t2.rowid:31!null t3._int2:33 t3._timestamptz:35 t3.rowid:39!null + │ │ │ │ ├── grouping columns: tab_orig.rowid:15!null t1.rowid:23!null t2.rowid:31!null t3.rowid:39!null │ │ │ │ ├── outer: (6) - │ │ │ │ ├── key: (14,21,28,35) - │ │ │ │ ├── fd: (21)-->(19), (28)-->(25), (35)-->(29,31), (14,21,28,35)-->(19,25,29,31) + │ │ │ │ ├── key: (15,23,31,39) + │ │ │ │ ├── fd: (23)-->(21), (31)-->(28), (39)-->(33,35), (15,23,31,39)-->(21,28,33,35) │ │ │ │ ├── inner-join (cross) - │ │ │ │ │ ├── columns: tab_orig.rowid:14!null t1._decimal:19 t1.rowid:21!null t2._bool:25 t2.rowid:28!null t3._int2:29 t3._timestamptz:31 t3.rowid:35!null + │ │ │ │ │ ├── columns: tab_orig.rowid:15!null t1._decimal:21 t1.rowid:23!null t2._bool:28 t2.rowid:31!null t3._int2:33 t3._timestamptz:35 t3.rowid:39!null │ │ │ │ │ ├── outer: (6) - │ │ │ │ │ ├── key: (14,21,28,35) - │ │ │ │ │ ├── fd: (21)-->(19), (28)-->(25), (35)-->(29,31) + │ │ │ │ │ ├── key: (15,23,31,39) + │ │ │ │ │ ├── fd: (23)-->(21), (31)-->(28), (39)-->(33,35) │ │ │ │ │ ├── scan tab_orig - │ │ │ │ │ │ ├── columns: tab_orig.rowid:14!null - │ │ │ │ │ │ └── key: (14) + │ │ │ │ │ │ ├── columns: tab_orig.rowid:15!null + │ │ │ │ │ │ └── key: (15) │ │ │ │ │ ├── inner-join (cross) - │ │ │ │ │ │ ├── columns: t1._decimal:19 t1.rowid:21!null t2._bool:25 t2.rowid:28!null t3._int2:29 t3._timestamptz:31 t3.rowid:35!null - │ │ │ │ │ │ ├── key: (21,28,35) - │ │ │ │ │ │ ├── fd: (21)-->(19), (28)-->(25), (35)-->(29,31) + │ │ │ │ │ │ ├── columns: t1._decimal:21 t1.rowid:23!null t2._bool:28 t2.rowid:31!null t3._int2:33 t3._timestamptz:35 t3.rowid:39!null + │ │ │ │ │ │ ├── key: (23,31,39) + │ │ │ │ │ │ ├── fd: (23)-->(21), (31)-->(28), (39)-->(33,35) │ │ │ │ │ │ ├── scan t1 - │ │ │ │ │ │ │ ├── columns: t1._decimal:19 t1.rowid:21!null - │ │ │ │ │ │ │ ├── key: (21) - │ │ │ │ │ │ │ └── fd: (21)-->(19) + │ │ │ │ │ │ │ ├── columns: t1._decimal:21 t1.rowid:23!null + │ │ │ │ │ │ │ ├── key: (23) + │ │ │ │ │ │ │ └── fd: (23)-->(21) │ │ │ │ │ │ ├── inner-join (cross) - │ │ │ │ │ │ │ ├── columns: t2._bool:25 t2.rowid:28!null t3._int2:29 t3._timestamptz:31 t3.rowid:35!null - │ │ │ │ │ │ │ ├── key: (28,35) - │ │ │ │ │ │ │ ├── fd: (28)-->(25), (35)-->(29,31) + │ │ │ │ │ │ │ ├── columns: t2._bool:28 t2.rowid:31!null t3._int2:33 t3._timestamptz:35 t3.rowid:39!null + │ │ │ │ │ │ │ ├── key: (31,39) + │ │ │ │ │ │ │ ├── fd: (31)-->(28), (39)-->(33,35) │ │ │ │ │ │ │ ├── scan t2 - │ │ │ │ │ │ │ │ ├── columns: t2._bool:25 t2.rowid:28!null - │ │ │ │ │ │ │ │ ├── key: (28) - │ │ │ │ │ │ │ │ └── fd: (28)-->(25) + │ │ │ │ │ │ │ │ ├── columns: t2._bool:28 t2.rowid:31!null + │ │ │ │ │ │ │ │ ├── key: (31) + │ │ │ │ │ │ │ │ └── fd: (31)-->(28) │ │ │ │ │ │ │ ├── scan t3 - │ │ │ │ │ │ │ │ ├── columns: t3._int2:29 t3._timestamptz:31 t3.rowid:35!null - │ │ │ │ │ │ │ │ ├── key: (35) - │ │ │ │ │ │ │ │ └── fd: (35)-->(29,31) + │ │ │ │ │ │ │ │ ├── columns: t3._int2:33 t3._timestamptz:35 t3.rowid:39!null + │ │ │ │ │ │ │ │ ├── key: (39) + │ │ │ │ │ │ │ │ └── fd: (39)-->(33,35) │ │ │ │ │ │ │ └── filters (true) │ │ │ │ │ │ └── filters (true) │ │ │ │ │ └── filters │ │ │ │ │ └── t0._string:6 = 1 [outer=(6), constraints=(/6: [/1 - /1]; tight), fd=()-->(6)] │ │ │ │ └── aggregations - │ │ │ │ ├── const-agg [as=t2._bool:25, outer=(25)] - │ │ │ │ │ └── t2._bool:25 - │ │ │ │ ├── const-agg [as=t3._int2:29, outer=(29)] - │ │ │ │ │ └── t3._int2:29 - │ │ │ │ ├── const-agg [as=t3._timestamptz:31, outer=(31)] - │ │ │ │ │ └── t3._timestamptz:31 - │ │ │ │ └── const-agg [as=t1._decimal:19, outer=(19)] - │ │ │ │ └── t1._decimal:19 + │ │ │ │ ├── const-agg [as=t2._bool:28, outer=(28)] + │ │ │ │ │ └── t2._bool:28 + │ │ │ │ ├── const-agg [as=t3._int2:33, outer=(33)] + │ │ │ │ │ └── t3._int2:33 + │ │ │ │ ├── const-agg [as=t3._timestamptz:35, outer=(35)] + │ │ │ │ │ └── t3._timestamptz:35 + │ │ │ │ └── const-agg [as=t1._decimal:21, outer=(21)] + │ │ │ │ └── t1._decimal:21 │ │ │ ├── scan t4 - │ │ │ │ └── columns: t4._int8:44 t4._timestamptz:45 + │ │ │ │ └── columns: t4._int8:49 t4._timestamptz:50 │ │ │ └── filters - │ │ │ ├── t3._timestamptz:31 = t4._timestamptz:45 [outer=(31,45), constraints=(/31: (/NULL - ]; /45: (/NULL - ]), fd=(31)==(45), (45)==(31)] - │ │ │ └── t3._int2:29 = t4._int8:44 [outer=(29,44), constraints=(/29: (/NULL - ]; /44: (/NULL - ]), fd=(29)==(44), (44)==(29)] + │ │ │ ├── t3._timestamptz:35 = t4._timestamptz:50 [outer=(35,50), constraints=(/35: (/NULL - ]; /50: (/NULL - ]), fd=(35)==(50), (50)==(35)] + │ │ │ └── t3._int2:33 = t4._int8:49 [outer=(33,49), constraints=(/33: (/NULL - ]; /49: (/NULL - ]), fd=(33)==(49), (49)==(33)] │ │ ├── scan t5 - │ │ │ └── columns: t5._decimal:54 + │ │ │ └── columns: t5._decimal:60 │ │ └── filters - │ │ ├── t2._bool:25 [outer=(25), constraints=(/25: [/true - /true]; tight), fd=()-->(25)] - │ │ └── t1._decimal:19 = t5._decimal:54 [outer=(19,54), constraints=(/19: (/NULL - ]; /54: (/NULL - ]), fd=(19)==(54), (54)==(19)] + │ │ ├── t2._bool:28 [outer=(28), constraints=(/28: [/true - /true]; tight), fd=()-->(28)] + │ │ └── t1._decimal:21 = t5._decimal:60 [outer=(21,60), constraints=(/21: (/NULL - ]; /60: (/NULL - ]), fd=(21)==(60), (60)==(21)] │ └── filters (true) └── projections - └── NULL [as="?column?":58] + └── NULL [as="?column?":65] # -------------------------------------------------- # TryDecorrelateScalarGroupBy @@ -2158,40 +2158,40 @@ group-by ├── key: (1) ├── fd: (1)-->(2-5) ├── select - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6!null count_rows:11!null - │ ├── key: (1,6) - │ ├── fd: (1)-->(2-5), (1,6)-->(2-5,11), (6)==(11), (11)==(6) + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:7!null count_rows:14!null + │ ├── key: (1,7) + │ ├── fd: (1)-->(2-5), (1,7)-->(2-5,14), (7)==(14), (14)==(7) │ ├── group-by - │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6!null count_rows:11!null - │ │ ├── grouping columns: k:1!null x:6!null - │ │ ├── key: (1,6) - │ │ ├── fd: (1)-->(2-5), (1,6)-->(2-5,11) + │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:7!null count_rows:14!null + │ │ ├── grouping columns: k:1!null x:7!null + │ │ ├── key: (1,7) + │ │ ├── fd: (1)-->(2-5), (1,7)-->(2-5,14) │ │ ├── left-join (cross) - │ │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6!null canary:12 + │ │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:7!null canary:15 │ │ │ ├── fd: (1)-->(2-5) │ │ │ ├── inner-join (cross) - │ │ │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6!null - │ │ │ │ ├── key: (1,6) + │ │ │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:7!null + │ │ │ │ ├── key: (1,7) │ │ │ │ ├── fd: (1)-->(2-5) │ │ │ │ ├── scan a │ │ │ │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ │ │ │ │ ├── key: (1) │ │ │ │ │ └── fd: (1)-->(2-5) │ │ │ │ ├── scan xy - │ │ │ │ │ ├── columns: x:6!null - │ │ │ │ │ └── key: (6) + │ │ │ │ │ ├── columns: x:7!null + │ │ │ │ │ └── key: (7) │ │ │ │ └── filters (true) │ │ │ ├── project - │ │ │ │ ├── columns: canary:12!null - │ │ │ │ ├── fd: ()-->(12) + │ │ │ │ ├── columns: canary:15!null + │ │ │ │ ├── fd: ()-->(15) │ │ │ │ ├── scan uv │ │ │ │ └── projections - │ │ │ │ └── true [as=canary:12] + │ │ │ │ └── true [as=canary:15] │ │ │ └── filters │ │ │ └── i:2 = 5 [outer=(2), constraints=(/2: [/5 - /5]; tight), fd=()-->(2)] │ │ └── aggregations - │ │ ├── count [as=count_rows:11, outer=(12)] - │ │ │ └── canary:12 + │ │ ├── count [as=count_rows:14, outer=(15)] + │ │ │ └── canary:15 │ │ ├── const-agg [as=i:2, outer=(2)] │ │ │ └── i:2 │ │ ├── const-agg [as=f:3, outer=(3)] @@ -2201,7 +2201,7 @@ group-by │ │ └── const-agg [as=j:5, outer=(5)] │ │ └── j:5 │ └── filters - │ └── x:6 = count_rows:11 [outer=(6,11), constraints=(/6: (/NULL - ]; /11: (/NULL - ]), fd=(6)==(11), (11)==(6)] + │ └── x:7 = count_rows:14 [outer=(7,14), constraints=(/7: (/NULL - ]; /14: (/NULL - ]), fd=(7)==(14), (14)==(7)] └── aggregations ├── const-agg [as=i:2, outer=(2)] │ └── i:2 @@ -2217,47 +2217,47 @@ norm expect=TryDecorrelateScalarGroupBy SELECT * FROM (SELECT i, 'foo' AS cst FROM a) WHERE 5=(SELECT max(y) FROM xy WHERE x=i) ---- project - ├── columns: i:2!null cst:6!null - ├── fd: ()-->(6) + ├── columns: i:2!null cst:7!null + ├── fd: ()-->(7) ├── select - │ ├── columns: k:1!null i:2!null max:9!null + │ ├── columns: k:1!null i:2!null max:11!null │ ├── key: (1) - │ ├── fd: ()-->(9), (1)-->(2) + │ ├── fd: ()-->(11), (1)-->(2) │ ├── group-by - │ │ ├── columns: k:1!null i:2!null max:9!null + │ │ ├── columns: k:1!null i:2!null max:11!null │ │ ├── grouping columns: k:1!null │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2,9) + │ │ ├── fd: (1)-->(2,11) │ │ ├── inner-join (hash) - │ │ │ ├── columns: k:1!null i:2!null x:7!null y:8!null + │ │ │ ├── columns: k:1!null i:2!null x:8!null y:9!null │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(2), (7)-->(8), (2)==(7), (7)==(2) + │ │ │ ├── fd: (1)-->(2), (8)-->(9), (2)==(8), (8)==(2) │ │ │ ├── scan a │ │ │ │ ├── columns: k:1!null i:2 │ │ │ │ ├── key: (1) │ │ │ │ └── fd: (1)-->(2) │ │ │ ├── select - │ │ │ │ ├── columns: x:7!null y:8!null - │ │ │ │ ├── key: (7) - │ │ │ │ ├── fd: (7)-->(8) + │ │ │ │ ├── columns: x:8!null y:9!null + │ │ │ │ ├── key: (8) + │ │ │ │ ├── fd: (8)-->(9) │ │ │ │ ├── scan xy - │ │ │ │ │ ├── columns: x:7!null y:8 - │ │ │ │ │ ├── key: (7) - │ │ │ │ │ └── fd: (7)-->(8) + │ │ │ │ │ ├── columns: x:8!null y:9 + │ │ │ │ │ ├── key: (8) + │ │ │ │ │ └── fd: (8)-->(9) │ │ │ │ └── filters - │ │ │ │ └── y:8 IS NOT NULL [outer=(8), constraints=(/8: (/NULL - ]; tight)] + │ │ │ │ └── y:9 IS NOT NULL [outer=(9), constraints=(/9: (/NULL - ]; tight)] │ │ │ └── filters - │ │ │ └── x:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] + │ │ │ └── x:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] │ │ └── aggregations - │ │ ├── max [as=max:9, outer=(8)] - │ │ │ └── y:8 + │ │ ├── max [as=max:11, outer=(9)] + │ │ │ └── y:9 │ │ └── const-agg [as=i:2, outer=(2)] │ │ └── i:2 │ └── filters - │ └── max:9 = 5 [outer=(9), constraints=(/9: [/5 - /5]; tight), fd=()-->(9)] + │ └── max:11 = 5 [outer=(11), constraints=(/11: [/5 - /5]; tight), fd=()-->(11)] └── projections - └── 'foo' [as=cst:6] + └── 'foo' [as=cst:7] # With an aggregate that can't ignore nulls. xy.y = a.k rejects nulls, so # there's no canary column to be synthesized. @@ -2265,64 +2265,64 @@ norm expect=TryDecorrelateScalarGroupBy SELECT k, (SELECT array_agg(xy.y) FROM xy WHERE xy.y = a.k) FROM a ---- project - ├── columns: k:1!null array_agg:9 + ├── columns: k:1!null array_agg:11 ├── key: (1) - ├── fd: (1)-->(9) + ├── fd: (1)-->(11) ├── group-by - │ ├── columns: k:1!null y:7 array_agg:10 + │ ├── columns: k:1!null y:8 array_agg:12 │ ├── grouping columns: k:1!null │ ├── key: (1) - │ ├── fd: (1)-->(7,10) + │ ├── fd: (1)-->(8,12) │ ├── left-join (hash) - │ │ ├── columns: k:1!null y:7 + │ │ ├── columns: k:1!null y:8 │ │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) │ │ ├── scan a │ │ │ ├── columns: k:1!null │ │ │ └── key: (1) │ │ ├── scan xy - │ │ │ └── columns: y:7 + │ │ │ └── columns: y:8 │ │ └── filters - │ │ └── y:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + │ │ └── y:8 = k:1 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] │ └── aggregations - │ ├── array-agg [as=array_agg:10, outer=(7)] - │ │ └── y:7 - │ └── any-not-null-agg [as=y:7, outer=(7)] - │ └── y:7 + │ ├── array-agg [as=array_agg:12, outer=(8)] + │ │ └── y:8 + │ └── any-not-null-agg [as=y:8, outer=(8)] + │ └── y:8 └── projections - └── CASE WHEN y:7 IS NOT NULL THEN array_agg:10 ELSE CAST(NULL AS INT8[]) END [as=array_agg:9, outer=(7,10)] + └── CASE WHEN y:8 IS NOT NULL THEN array_agg:12 ELSE CAST(NULL AS INT8[]) END [as=array_agg:11, outer=(8,12)] # With multiple columns. Without LATERAL these tests are a bit verbose. norm expect=TryDecorrelateScalarGroupBy SELECT k, (SELECT (r, q) FROM (SELECT array_agg(xy.y) r, max(xy.y) q FROM xy WHERE xy.y = a.k)) FROM a ---- project - ├── columns: k:1!null "?column?":11 + ├── columns: k:1!null "?column?":13 ├── key: (1) - ├── fd: (1)-->(11) + ├── fd: (1)-->(13) ├── group-by - │ ├── columns: k:1!null y:7 max:9 array_agg:12 + │ ├── columns: k:1!null y:8 max:11 array_agg:14 │ ├── grouping columns: k:1!null │ ├── key: (1) - │ ├── fd: (1)-->(7,9,12) + │ ├── fd: (1)-->(8,11,14) │ ├── left-join (hash) - │ │ ├── columns: k:1!null y:7 + │ │ ├── columns: k:1!null y:8 │ │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) │ │ ├── scan a │ │ │ ├── columns: k:1!null │ │ │ └── key: (1) │ │ ├── scan xy - │ │ │ └── columns: y:7 + │ │ │ └── columns: y:8 │ │ └── filters - │ │ └── y:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + │ │ └── y:8 = k:1 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] │ └── aggregations - │ ├── array-agg [as=array_agg:12, outer=(7)] - │ │ └── y:7 - │ ├── max [as=max:9, outer=(7)] - │ │ └── y:7 - │ └── any-not-null-agg [as=y:7, outer=(7)] - │ └── y:7 + │ ├── array-agg [as=array_agg:14, outer=(8)] + │ │ └── y:8 + │ ├── max [as=max:11, outer=(8)] + │ │ └── y:8 + │ └── any-not-null-agg [as=y:8, outer=(8)] + │ └── y:8 └── projections - └── (CASE WHEN y:7 IS NOT NULL THEN array_agg:12 ELSE CAST(NULL AS INT8[]) END, max:9) [as="?column?":11, outer=(7,9,12)] + └── (CASE WHEN y:8 IS NOT NULL THEN array_agg:14 ELSE CAST(NULL AS INT8[]) END, max:11) [as="?column?":13, outer=(8,11,14)] # With an aggregate that can't ignore nulls and when a non-nullable column must be synthesized. @@ -2330,79 +2330,79 @@ norm expect=TryDecorrelateScalarGroupBy SELECT k, ARRAY(SELECT y FROM xy WHERE xy.y = a.i OR xy.y IS NULL) FROM a ---- project - ├── columns: k:1!null array:9 + ├── columns: k:1!null array:11 ├── key: (1) - ├── fd: (1)-->(9) + ├── fd: (1)-->(11) ├── group-by - │ ├── columns: k:1!null canary:10 array_agg:11 + │ ├── columns: k:1!null canary:12 array_agg:13 │ ├── grouping columns: k:1!null │ ├── key: (1) - │ ├── fd: (1)-->(10,11) + │ ├── fd: (1)-->(12,13) │ ├── left-join (cross) - │ │ ├── columns: k:1!null i:2 y:7 canary:10 + │ │ ├── columns: k:1!null i:2 y:8 canary:12 │ │ ├── fd: (1)-->(2) │ │ ├── scan a │ │ │ ├── columns: k:1!null i:2 │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2) │ │ ├── project - │ │ │ ├── columns: canary:10!null y:7 - │ │ │ ├── fd: ()-->(10) + │ │ │ ├── columns: canary:12!null y:8 + │ │ │ ├── fd: ()-->(12) │ │ │ ├── scan xy - │ │ │ │ └── columns: y:7 + │ │ │ │ └── columns: y:8 │ │ │ └── projections - │ │ │ └── true [as=canary:10] + │ │ │ └── true [as=canary:12] │ │ └── filters - │ │ └── (y:7 = i:2) OR (y:7 IS NULL) [outer=(2,7)] + │ │ └── (y:8 = i:2) OR (y:8 IS NULL) [outer=(2,8)] │ └── aggregations - │ ├── array-agg [as=array_agg:11, outer=(7)] - │ │ └── y:7 - │ └── any-not-null-agg [as=canary:10, outer=(10)] - │ └── canary:10 + │ ├── array-agg [as=array_agg:13, outer=(8)] + │ │ └── y:8 + │ └── any-not-null-agg [as=canary:12, outer=(12)] + │ └── canary:12 └── projections - └── COALESCE(CASE WHEN canary:10 IS NOT NULL THEN array_agg:11 ELSE CAST(NULL AS INT8[]) END, ARRAY[]) [as=array:9, outer=(10,11)] + └── COALESCE(CASE WHEN canary:12 IS NOT NULL THEN array_agg:13 ELSE CAST(NULL AS INT8[]) END, ARRAY[]) [as=array:11, outer=(12,13)] # With an ordering. norm expect=TryDecorrelateScalarGroupBy SELECT i, ARRAY(SELECT y FROM xy WHERE xy.y = a.k OR xy.y IS NULL ORDER BY y) FROM a ---- project - ├── columns: i:2 array:9 + ├── columns: i:2 array:11 ├── group-by - │ ├── columns: k:1!null i:2 canary:10 array_agg:11 + │ ├── columns: k:1!null i:2 canary:12 array_agg:13 │ ├── grouping columns: k:1!null - │ ├── internal-ordering: +7 + │ ├── internal-ordering: +8 │ ├── key: (1) - │ ├── fd: (1)-->(2,10,11) + │ ├── fd: (1)-->(2,12,13) │ ├── sort - │ │ ├── columns: k:1!null i:2 y:7 canary:10 + │ │ ├── columns: k:1!null i:2 y:8 canary:12 │ │ ├── fd: (1)-->(2) - │ │ ├── ordering: +7 + │ │ ├── ordering: +8 │ │ └── left-join (cross) - │ │ ├── columns: k:1!null i:2 y:7 canary:10 + │ │ ├── columns: k:1!null i:2 y:8 canary:12 │ │ ├── fd: (1)-->(2) │ │ ├── scan a │ │ │ ├── columns: k:1!null i:2 │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2) │ │ ├── project - │ │ │ ├── columns: canary:10!null y:7 - │ │ │ ├── fd: ()-->(10) + │ │ │ ├── columns: canary:12!null y:8 + │ │ │ ├── fd: ()-->(12) │ │ │ ├── scan xy - │ │ │ │ └── columns: y:7 + │ │ │ │ └── columns: y:8 │ │ │ └── projections - │ │ │ └── true [as=canary:10] + │ │ │ └── true [as=canary:12] │ │ └── filters - │ │ └── (y:7 = k:1) OR (y:7 IS NULL) [outer=(1,7)] + │ │ └── (y:8 = k:1) OR (y:8 IS NULL) [outer=(1,8)] │ └── aggregations - │ ├── array-agg [as=array_agg:11, outer=(7)] - │ │ └── y:7 + │ ├── array-agg [as=array_agg:13, outer=(8)] + │ │ └── y:8 │ ├── const-agg [as=i:2, outer=(2)] │ │ └── i:2 - │ └── any-not-null-agg [as=canary:10, outer=(10)] - │ └── canary:10 + │ └── any-not-null-agg [as=canary:12, outer=(12)] + │ └── canary:12 └── projections - └── COALESCE(CASE WHEN canary:10 IS NOT NULL THEN array_agg:11 ELSE CAST(NULL AS INT8[]) END, ARRAY[]) [as=array:9, outer=(10,11)] + └── COALESCE(CASE WHEN canary:12 IS NOT NULL THEN array_agg:13 ELSE CAST(NULL AS INT8[]) END, ARRAY[]) [as=array:11, outer=(12,13)] # Nest scalar decorrelation within scalar decorrelation, using IS NULL to force # use of left joins. @@ -2424,53 +2424,53 @@ project ├── key: (1) ├── fd: (1)-->(2-5) └── select - ├── columns: k:1!null i:2 f:3 s:4 j:5 max:11 + ├── columns: k:1!null i:2 f:3 s:4 j:5 max:14 ├── key: (1) - ├── fd: ()-->(11), (1)-->(2-5) + ├── fd: ()-->(14), (1)-->(2-5) ├── group-by - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 max:11 + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 max:14 │ ├── grouping columns: k:1!null │ ├── key: (1) - │ ├── fd: (1)-->(2-5,11) + │ ├── fd: (1)-->(2-5,14) │ ├── left-join-apply - │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6 y:7 max:10 - │ │ ├── key: (1,6) - │ │ ├── fd: (1)-->(2-5), (1,6)-->(7,10) + │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:7 y:8 max:13 + │ │ ├── key: (1,7) + │ │ ├── fd: (1)-->(2-5), (1,7)-->(8,13) │ │ ├── scan a │ │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2-5) │ │ ├── group-by - │ │ │ ├── columns: x:6!null y:7 max:10 - │ │ │ ├── grouping columns: x:6!null + │ │ │ ├── columns: x:7!null y:8 max:13 + │ │ │ ├── grouping columns: x:7!null │ │ │ ├── outer: (1) - │ │ │ ├── key: (6) - │ │ │ ├── fd: (6)-->(7,10) + │ │ │ ├── key: (7) + │ │ │ ├── fd: (7)-->(8,13) │ │ │ ├── left-join (cross) - │ │ │ │ ├── columns: x:6!null y:7 u:8 v:9 + │ │ │ │ ├── columns: x:7!null y:8 u:10 v:11 │ │ │ │ ├── outer: (1) - │ │ │ │ ├── key: (6,8) - │ │ │ │ ├── fd: (6)-->(7), (8)-->(9) + │ │ │ │ ├── key: (7,10) + │ │ │ │ ├── fd: (7)-->(8), (10)-->(11) │ │ │ │ ├── scan xy - │ │ │ │ │ ├── columns: x:6!null y:7 - │ │ │ │ │ ├── key: (6) - │ │ │ │ │ └── fd: (6)-->(7) + │ │ │ │ │ ├── columns: x:7!null y:8 + │ │ │ │ │ ├── key: (7) + │ │ │ │ │ └── fd: (7)-->(8) │ │ │ │ ├── scan uv - │ │ │ │ │ ├── columns: u:8!null v:9 - │ │ │ │ │ ├── key: (8) - │ │ │ │ │ └── fd: (8)-->(9) + │ │ │ │ │ ├── columns: u:10!null v:11 + │ │ │ │ │ ├── key: (10) + │ │ │ │ │ └── fd: (10)-->(11) │ │ │ │ └── filters - │ │ │ │ └── u:8 = k:1 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] + │ │ │ │ └── u:10 = k:1 [outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] │ │ │ └── aggregations - │ │ │ ├── max [as=max:10, outer=(9)] - │ │ │ │ └── v:9 - │ │ │ └── const-agg [as=y:7, outer=(7)] - │ │ │ └── y:7 + │ │ │ ├── max [as=max:13, outer=(11)] + │ │ │ │ └── v:11 + │ │ │ └── const-agg [as=y:8, outer=(8)] + │ │ │ └── y:8 │ │ └── filters - │ │ └── max:10 IS NULL [outer=(10), constraints=(/10: [/NULL - /NULL]; tight), fd=()-->(10)] + │ │ └── max:13 IS NULL [outer=(13), constraints=(/13: [/NULL - /NULL]; tight), fd=()-->(13)] │ └── aggregations - │ ├── max [as=max:11, outer=(7)] - │ │ └── y:7 + │ ├── max [as=max:14, outer=(8)] + │ │ └── y:8 │ ├── const-agg [as=i:2, outer=(2)] │ │ └── i:2 │ ├── const-agg [as=f:3, outer=(3)] @@ -2480,7 +2480,7 @@ project │ └── const-agg [as=j:5, outer=(5)] │ └── j:5 └── filters - └── max:11 IS NULL [outer=(11), constraints=(/11: [/NULL - /NULL]; tight), fd=()-->(11)] + └── max:14 IS NULL [outer=(14), constraints=(/14: [/NULL - /NULL]; tight), fd=()-->(14)] # ScalarGroupBy with non-null ignoring and a non-nullable column. norm expect=TryDecorrelateScalarGroupBy @@ -2498,44 +2498,44 @@ project ├── key: (1) ├── fd: (1)-->(2) └── select - ├── columns: c:1!null d:2!null array_agg:5!null + ├── columns: c:1!null d:2!null array_agg:7!null ├── key: (1) - ├── fd: ()-->(5), (1)-->(2) + ├── fd: ()-->(7), (1)-->(2) ├── project - │ ├── columns: array_agg:5 c:1!null d:2!null + │ ├── columns: array_agg:7 c:1!null d:2!null │ ├── key: (1) - │ ├── fd: (1)-->(2,5) + │ ├── fd: (1)-->(2,7) │ ├── group-by - │ │ ├── columns: c:1!null d:2!null x:3 array_agg:6 + │ │ ├── columns: c:1!null d:2!null x:4 array_agg:8 │ │ ├── grouping columns: c:1!null │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2,3,6) + │ │ ├── fd: (1)-->(2,4,8) │ │ ├── left-join (hash) - │ │ │ ├── columns: c:1!null d:2!null x:3 y:4 + │ │ │ ├── columns: c:1!null d:2!null x:4 y:5 │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(2-4), (3)-->(4) + │ │ │ ├── fd: (1)-->(2,4,5), (4)-->(5) │ │ │ ├── scan cd │ │ │ │ ├── columns: c:1!null d:2!null │ │ │ │ ├── key: (1) │ │ │ │ └── fd: (1)-->(2) │ │ │ ├── scan xy - │ │ │ │ ├── columns: x:3!null y:4 - │ │ │ │ ├── key: (3) - │ │ │ │ └── fd: (3)-->(4) + │ │ │ │ ├── columns: x:4!null y:5 + │ │ │ │ ├── key: (4) + │ │ │ │ └── fd: (4)-->(5) │ │ │ └── filters - │ │ │ └── c:1 = x:3 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + │ │ │ └── c:1 = x:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] │ │ └── aggregations - │ │ ├── array-agg [as=array_agg:6, outer=(4)] - │ │ │ └── y:4 + │ │ ├── array-agg [as=array_agg:8, outer=(5)] + │ │ │ └── y:5 │ │ ├── const-agg [as=d:2, outer=(2)] │ │ │ └── d:2 - │ │ └── any-not-null-agg [as=x:3, outer=(3)] - │ │ └── x:3 + │ │ └── any-not-null-agg [as=x:4, outer=(4)] + │ │ └── x:4 │ └── projections - │ └── CASE WHEN x:3 IS NOT NULL THEN array_agg:6 ELSE CAST(NULL AS INT8[]) END [as=array_agg:5, outer=(3,6)] + │ └── CASE WHEN x:4 IS NOT NULL THEN array_agg:8 ELSE CAST(NULL AS INT8[]) END [as=array_agg:7, outer=(4,8)] └── filters - └── array_agg:5 = ARRAY[] [outer=(5), constraints=(/5: [/ARRAY[] - /ARRAY[]]; tight), fd=()-->(5)] + └── array_agg:7 = ARRAY[] [outer=(7), constraints=(/7: [/ARRAY[] - /ARRAY[]]; tight), fd=()-->(7)] norm expect=TryDecorrelateScalarGroupBy SELECT * FROM a WHERE 'foo'=(SELECT concat_agg(y::string) FROM xy WHERE x=k) @@ -2546,48 +2546,48 @@ project ├── key: (1) ├── fd: (1)-->(2-5) └── select - ├── columns: k:1!null i:2 f:3 s:4 j:5 concat_agg:9!null + ├── columns: k:1!null i:2 f:3 s:4 j:5 concat_agg:11!null ├── immutable ├── key: (1) - ├── fd: ()-->(9), (1)-->(2-5) + ├── fd: ()-->(11), (1)-->(2-5) ├── project - │ ├── columns: concat_agg:9 k:1!null i:2 f:3 s:4 j:5 + │ ├── columns: concat_agg:11 k:1!null i:2 f:3 s:4 j:5 │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(2-5,9) + │ ├── fd: (1)-->(2-5,11) │ ├── group-by - │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 canary:10 concat_agg:11 + │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 canary:12 concat_agg:13 │ │ ├── grouping columns: k:1!null │ │ ├── immutable │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2-5,10,11) + │ │ ├── fd: (1)-->(2-5,12,13) │ │ ├── left-join (hash) - │ │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6 column8:8 canary:10 + │ │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:7 column10:10 canary:12 │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ │ ├── immutable │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(2-6,8,10), (6)-->(8) + │ │ │ ├── fd: (1)-->(2-5,7,10,12), (7)-->(10) │ │ │ ├── scan a │ │ │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ │ │ │ ├── key: (1) │ │ │ │ └── fd: (1)-->(2-5) │ │ │ ├── project - │ │ │ │ ├── columns: canary:10!null column8:8 x:6!null + │ │ │ │ ├── columns: canary:12!null column10:10 x:7!null │ │ │ │ ├── immutable - │ │ │ │ ├── key: (6) - │ │ │ │ ├── fd: ()-->(10), (6)-->(8) + │ │ │ │ ├── key: (7) + │ │ │ │ ├── fd: ()-->(12), (7)-->(10) │ │ │ │ ├── scan xy - │ │ │ │ │ ├── columns: x:6!null y:7 - │ │ │ │ │ ├── key: (6) - │ │ │ │ │ └── fd: (6)-->(7) + │ │ │ │ │ ├── columns: x:7!null y:8 + │ │ │ │ │ ├── key: (7) + │ │ │ │ │ └── fd: (7)-->(8) │ │ │ │ └── projections - │ │ │ │ ├── true [as=canary:10] - │ │ │ │ └── y:7::STRING [as=column8:8, outer=(7), immutable] + │ │ │ │ ├── true [as=canary:12] + │ │ │ │ └── y:8::STRING [as=column10:10, outer=(8), immutable] │ │ │ └── filters - │ │ │ └── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ │ │ └── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] │ │ └── aggregations - │ │ ├── concat-agg [as=concat_agg:11, outer=(8)] - │ │ │ └── column8:8 + │ │ ├── concat-agg [as=concat_agg:13, outer=(10)] + │ │ │ └── column10:10 │ │ ├── const-agg [as=i:2, outer=(2)] │ │ │ └── i:2 │ │ ├── const-agg [as=f:3, outer=(3)] @@ -2596,53 +2596,53 @@ project │ │ │ └── s:4 │ │ ├── const-agg [as=j:5, outer=(5)] │ │ │ └── j:5 - │ │ └── any-not-null-agg [as=canary:10, outer=(10)] - │ │ └── canary:10 + │ │ └── any-not-null-agg [as=canary:12, outer=(12)] + │ │ └── canary:12 │ └── projections - │ └── CASE WHEN canary:10 IS NOT NULL THEN concat_agg:11 ELSE CAST(NULL AS STRING) END [as=concat_agg:9, outer=(10,11)] + │ └── CASE WHEN canary:12 IS NOT NULL THEN concat_agg:13 ELSE CAST(NULL AS STRING) END [as=concat_agg:11, outer=(12,13)] └── filters - └── concat_agg:9 = 'foo' [outer=(9), constraints=(/9: [/'foo' - /'foo']; tight), fd=()-->(9)] + └── concat_agg:11 = 'foo' [outer=(11), constraints=(/11: [/'foo' - /'foo']; tight), fd=()-->(11)] # With a multi-argument aggregate. norm expect=TryDecorrelateScalarGroupBy SELECT k, (SELECT string_agg(a.s, ',') FROM a WHERE a.k = a2.i) FROM a AS a2 ---- project - ├── columns: k:1!null string_agg:13 + ├── columns: k:1!null string_agg:15 ├── key: (1) - ├── fd: (1)-->(13) + ├── fd: (1)-->(15) ├── group-by - │ ├── columns: a2.k:1!null string_agg:12 + │ ├── columns: a2.k:1!null string_agg:14 │ ├── grouping columns: a2.k:1!null │ ├── key: (1) - │ ├── fd: (1)-->(12) + │ ├── fd: (1)-->(14) │ ├── left-join (hash) - │ │ ├── columns: a2.k:1!null a2.i:2 a.k:6 a.s:9 column11:11 + │ │ ├── columns: a2.k:1!null a2.i:2 a.k:7 a.s:10 column13:13 │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2,6,9,11), (6)-->(9) + │ │ ├── fd: (1)-->(2,7,10,13), (7)-->(10) │ │ ├── scan a2 │ │ │ ├── columns: a2.k:1!null a2.i:2 │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2) │ │ ├── project - │ │ │ ├── columns: column11:11!null a.k:6!null a.s:9 - │ │ │ ├── key: (6) - │ │ │ ├── fd: ()-->(11), (6)-->(9) + │ │ │ ├── columns: column13:13!null a.k:7!null a.s:10 + │ │ │ ├── key: (7) + │ │ │ ├── fd: ()-->(13), (7)-->(10) │ │ │ ├── scan a - │ │ │ │ ├── columns: a.k:6!null a.s:9 - │ │ │ │ ├── key: (6) - │ │ │ │ └── fd: (6)-->(9) + │ │ │ │ ├── columns: a.k:7!null a.s:10 + │ │ │ │ ├── key: (7) + │ │ │ │ └── fd: (7)-->(10) │ │ │ └── projections - │ │ │ └── ',' [as=column11:11] + │ │ │ └── ',' [as=column13:13] │ │ └── filters - │ │ └── a.k:6 = a2.i:2 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + │ │ └── a.k:7 = a2.i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] │ └── aggregations - │ └── string-agg [as=string_agg:12, outer=(9,11)] - │ ├── a.s:9 - │ └── column11:11 + │ └── string-agg [as=string_agg:14, outer=(10,13)] + │ ├── a.s:10 + │ └── column13:13 └── projections - └── string_agg:12 [as=string_agg:13, outer=(12)] + └── string_agg:14 [as=string_agg:15, outer=(14)] # -------------------------------------------------- # TryDecorrelateSemiJoin @@ -2664,57 +2664,57 @@ group-by ├── key: (1) ├── fd: (1)-->(2) ├── select - │ ├── columns: x:1!null y:2 k:3!null i:4!null max:13!null + │ ├── columns: x:1!null y:2 k:4!null i:5!null max:16!null │ ├── immutable - │ ├── key: (1,3) - │ ├── fd: (1)-->(2), (3)-->(4), (1,3)-->(2,4,13), (4)==(13), (13)==(4) + │ ├── key: (1,4) + │ ├── fd: (1)-->(2), (4)-->(5), (1,4)-->(2,5,16), (5)==(16), (16)==(5) │ ├── group-by - │ │ ├── columns: x:1!null y:2 k:3!null i:4 max:13!null - │ │ ├── grouping columns: x:1!null k:3!null + │ │ ├── columns: x:1!null y:2 k:4!null i:5 max:16!null + │ │ ├── grouping columns: x:1!null k:4!null │ │ ├── immutable - │ │ ├── key: (1,3) - │ │ ├── fd: (1)-->(2), (3)-->(4), (1,3)-->(2,4,13) + │ │ ├── key: (1,4) + │ │ ├── fd: (1)-->(2), (4)-->(5), (1,4)-->(2,5,16) │ │ ├── inner-join (cross) - │ │ │ ├── columns: x:1!null y:2 k:3!null i:4 i:9!null f:10!null column14:14!null + │ │ │ ├── columns: x:1!null y:2 k:4!null i:5 i:11!null f:12!null column17:17!null │ │ │ ├── immutable - │ │ │ ├── fd: (1)-->(2), (2)-->(14), (10)==(14), (14)==(10), (3)-->(4) + │ │ │ ├── fd: (1)-->(2), (2)-->(17), (12)==(17), (17)==(12), (4)-->(5) │ │ │ ├── inner-join (hash) - │ │ │ │ ├── columns: x:1!null y:2 i:9!null f:10!null column14:14!null + │ │ │ │ ├── columns: x:1!null y:2 i:11!null f:12!null column17:17!null │ │ │ │ ├── immutable - │ │ │ │ ├── fd: (1)-->(2), (2)-->(14), (10)==(14), (14)==(10) + │ │ │ │ ├── fd: (1)-->(2), (2)-->(17), (12)==(17), (17)==(12) │ │ │ │ ├── project - │ │ │ │ │ ├── columns: column14:14 x:1!null y:2 + │ │ │ │ │ ├── columns: column17:17 x:1!null y:2 │ │ │ │ │ ├── immutable │ │ │ │ │ ├── key: (1) - │ │ │ │ │ ├── fd: (1)-->(2), (2)-->(14) + │ │ │ │ │ ├── fd: (1)-->(2), (2)-->(17) │ │ │ │ │ ├── scan xy │ │ │ │ │ │ ├── columns: x:1!null y:2 │ │ │ │ │ │ ├── key: (1) │ │ │ │ │ │ └── fd: (1)-->(2) │ │ │ │ │ └── projections - │ │ │ │ │ └── y:2::FLOAT8 [as=column14:14, outer=(2), immutable] + │ │ │ │ │ └── y:2::FLOAT8 [as=column17:17, outer=(2), immutable] │ │ │ │ ├── select - │ │ │ │ │ ├── columns: i:9!null f:10 + │ │ │ │ │ ├── columns: i:11!null f:12 │ │ │ │ │ ├── scan a - │ │ │ │ │ │ └── columns: i:9 f:10 + │ │ │ │ │ │ └── columns: i:11 f:12 │ │ │ │ │ └── filters - │ │ │ │ │ └── i:9 IS NOT NULL [outer=(9), constraints=(/9: (/NULL - ]; tight)] + │ │ │ │ │ └── i:11 IS NOT NULL [outer=(11), constraints=(/11: (/NULL - ]; tight)] │ │ │ │ └── filters - │ │ │ │ └── column14:14 = f:10 [outer=(10,14), constraints=(/10: (/NULL - ]; /14: (/NULL - ]), fd=(10)==(14), (14)==(10)] + │ │ │ │ └── column17:17 = f:12 [outer=(12,17), constraints=(/12: (/NULL - ]; /17: (/NULL - ]), fd=(12)==(17), (17)==(12)] │ │ │ ├── scan a - │ │ │ │ ├── columns: k:3!null i:4 - │ │ │ │ ├── key: (3) - │ │ │ │ └── fd: (3)-->(4) + │ │ │ │ ├── columns: k:4!null i:5 + │ │ │ │ ├── key: (4) + │ │ │ │ └── fd: (4)-->(5) │ │ │ └── filters (true) │ │ └── aggregations - │ │ ├── max [as=max:13, outer=(9)] - │ │ │ └── i:9 - │ │ ├── const-agg [as=i:4, outer=(4)] - │ │ │ └── i:4 + │ │ ├── max [as=max:16, outer=(11)] + │ │ │ └── i:11 + │ │ ├── const-agg [as=i:5, outer=(5)] + │ │ │ └── i:5 │ │ └── const-agg [as=y:2, outer=(2)] │ │ └── y:2 │ └── filters - │ └── i:4 = max:13 [outer=(4,13), constraints=(/4: (/NULL - ]; /13: (/NULL - ]), fd=(4)==(13), (13)==(4)] + │ └── i:5 = max:16 [outer=(5,16), constraints=(/5: (/NULL - ]; /16: (/NULL - ]), fd=(5)==(16), (16)==(5)] └── aggregations └── const-agg [as=y:2, outer=(2)] └── y:2 @@ -2734,32 +2734,32 @@ group-by ├── key: (1) ├── fd: (1)-->(2) ├── select - │ ├── columns: x:1!null y:2!null i:4!null f:5!null - │ ├── key: (1,5) - │ ├── fd: (1)-->(2), (1,5)-->(2,4), (1)==(4), (4)==(1) + │ ├── columns: x:1!null y:2!null i:5!null f:6!null + │ ├── key: (1,6) + │ ├── fd: (1)-->(2), (1,6)-->(2,5), (1)==(5), (5)==(1) │ ├── distinct-on - │ │ ├── columns: x:1!null y:2!null i:4 f:5!null - │ │ ├── grouping columns: x:1!null f:5!null - │ │ ├── key: (1,5) - │ │ ├── fd: (1)-->(2), (1,5)-->(2,4) + │ │ ├── columns: x:1!null y:2!null i:5 f:6!null + │ │ ├── grouping columns: x:1!null f:6!null + │ │ ├── key: (1,6) + │ │ ├── fd: (1)-->(2), (1,6)-->(2,5) │ │ ├── inner-join (cross) - │ │ │ ├── columns: x:1!null y:2!null i:4 f:5!null + │ │ │ ├── columns: x:1!null y:2!null i:5 f:6!null │ │ │ ├── fd: (1)-->(2) │ │ │ ├── scan xy │ │ │ │ ├── columns: x:1!null y:2 │ │ │ │ ├── key: (1) │ │ │ │ └── fd: (1)-->(2) │ │ │ ├── scan a - │ │ │ │ └── columns: i:4 f:5 + │ │ │ │ └── columns: i:5 f:6 │ │ │ └── filters - │ │ │ └── y:2 > f:5 [outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ])] + │ │ │ └── y:2 > f:6 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] │ │ └── aggregations - │ │ ├── first-agg [as=i:4, outer=(4)] - │ │ │ └── i:4 + │ │ ├── first-agg [as=i:5, outer=(5)] + │ │ │ └── i:5 │ │ └── const-agg [as=y:2, outer=(2)] │ │ └── y:2 │ └── filters - │ └── x:1 = i:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + │ └── x:1 = i:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] └── aggregations └── const-agg [as=y:2, outer=(2)] └── y:2 @@ -2776,39 +2776,39 @@ project ├── columns: k:1!null ├── key: (1) └── select - ├── columns: k:1!null x:6!null computed:10!null + ├── columns: k:1!null x:7!null computed:13!null ├── key: (1) - ├── fd: (1)-->(10), (6)==(10), (10)==(6) + ├── fd: (1)-->(13), (7)==(13), (13)==(7) ├── project - │ ├── columns: computed:10 k:1!null x:6!null - │ ├── key: (1,6) - │ ├── fd: (1)-->(10) + │ ├── columns: computed:13 k:1!null x:7!null + │ ├── key: (1,7) + │ ├── fd: (1)-->(13) │ ├── inner-join (cross) - │ │ ├── columns: k:1!null i:2!null x:6!null u:8!null - │ │ ├── key: (1,6) - │ │ ├── fd: (1)-->(2), (2)==(8), (8)==(2) + │ │ ├── columns: k:1!null i:2!null x:7!null u:10!null + │ │ ├── key: (1,7) + │ │ ├── fd: (1)-->(2), (2)==(10), (10)==(2) │ │ ├── inner-join (hash) - │ │ │ ├── columns: k:1!null i:2!null u:8!null + │ │ │ ├── columns: k:1!null i:2!null u:10!null │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(2), (2)==(8), (8)==(2) + │ │ │ ├── fd: (1)-->(2), (2)==(10), (10)==(2) │ │ │ ├── scan a │ │ │ │ ├── columns: k:1!null i:2 │ │ │ │ ├── key: (1) │ │ │ │ └── fd: (1)-->(2) │ │ │ ├── scan uv - │ │ │ │ ├── columns: u:8!null - │ │ │ │ └── key: (8) + │ │ │ │ ├── columns: u:10!null + │ │ │ │ └── key: (10) │ │ │ └── filters - │ │ │ └── u:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] + │ │ │ └── u:10 = i:2 [outer=(2,10), constraints=(/2: (/NULL - ]; /10: (/NULL - ]), fd=(2)==(10), (10)==(2)] │ │ ├── scan xy - │ │ │ ├── columns: x:6!null - │ │ │ └── key: (6) + │ │ │ ├── columns: x:7!null + │ │ │ └── key: (7) │ │ └── filters (true) │ └── projections - │ └── COALESCE(u:8, 10) [as=computed:10, outer=(8)] + │ └── COALESCE(u:10, 10) [as=computed:13, outer=(10)] └── filters - └── x:6 = computed:10 [outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ]), fd=(6)==(10), (10)==(6)] + └── x:7 = computed:13 [outer=(7,13), constraints=(/7: (/NULL - ]; /13: (/NULL - ]), fd=(7)==(13), (13)==(7)] # Right input of SemiJoin is ProjectSet. norm expect=TryDecorrelateSemiJoin @@ -2821,7 +2821,7 @@ group-by ├── key: (1) ├── fd: (1)-->(2) ├── project-set - │ ├── columns: x:1!null y:2 generate_series:3 generate_series:4 + │ ├── columns: x:1!null y:2 generate_series:4 generate_series:5 │ ├── immutable │ ├── fd: (1)-->(2) │ ├── scan xy @@ -2860,52 +2860,52 @@ semi-join (hash) │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── inner-join (hash) - │ ├── columns: x:6!null u:8!null v:9 + │ ├── columns: x:7!null u:10!null v:11 │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - │ ├── key: (8) - │ ├── fd: (8)-->(9), (6)==(8), (8)==(6) + │ ├── key: (10) + │ ├── fd: (10)-->(11), (7)==(10), (10)==(7) │ ├── scan xy - │ │ ├── columns: x:6!null - │ │ └── key: (6) + │ │ ├── columns: x:7!null + │ │ └── key: (7) │ ├── scan uv - │ │ ├── columns: u:8!null v:9 - │ │ ├── key: (8) - │ │ └── fd: (8)-->(9) + │ │ ├── columns: u:10!null v:11 + │ │ ├── key: (10) + │ │ └── fd: (10)-->(11) │ └── filters - │ └── x:6 = u:8 [outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ]), fd=(6)==(8), (8)==(6)] + │ └── x:7 = u:10 [outer=(7,10), constraints=(/7: (/NULL - ]; /10: (/NULL - ]), fd=(7)==(10), (10)==(7)] └── filters - └── v:9 = i:2 [outer=(2,9), constraints=(/2: (/NULL - ]; /9: (/NULL - ]), fd=(2)==(9), (9)==(2)] + └── v:11 = i:2 [outer=(2,11), constraints=(/2: (/NULL - ]; /11: (/NULL - ]), fd=(2)==(11), (11)==(2)] # With left join. norm expect=TryDecorrelateLimitOne SELECT (SELECT x FROM xy WHERE y=i LIMIT 1) FROM a ---- project - ├── columns: x:8 + ├── columns: x:10 ├── distinct-on - │ ├── columns: k:1!null xy.x:6 + │ ├── columns: k:1!null xy.x:7 │ ├── grouping columns: k:1!null │ ├── key: (1) - │ ├── fd: (1)-->(6) + │ ├── fd: (1)-->(7) │ ├── left-join (hash) - │ │ ├── columns: k:1!null i:2 xy.x:6 y:7 - │ │ ├── key: (1,6) - │ │ ├── fd: (1)-->(2), (6)-->(7) + │ │ ├── columns: k:1!null i:2 xy.x:7 y:8 + │ │ ├── key: (1,7) + │ │ ├── fd: (1)-->(2), (7)-->(8) │ │ ├── scan a │ │ │ ├── columns: k:1!null i:2 │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2) │ │ ├── scan xy - │ │ │ ├── columns: xy.x:6!null y:7 - │ │ │ ├── key: (6) - │ │ │ └── fd: (6)-->(7) + │ │ │ ├── columns: xy.x:7!null y:8 + │ │ │ ├── key: (7) + │ │ │ └── fd: (7)-->(8) │ │ └── filters - │ │ └── y:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] + │ │ └── y:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] │ └── aggregations - │ └── first-agg [as=xy.x:6, outer=(6)] - │ └── xy.x:6 + │ └── first-agg [as=xy.x:7, outer=(7)] + │ └── xy.x:7 └── projections - └── xy.x:6 [as=x:8, outer=(6)] + └── xy.x:7 [as=x:10, outer=(7)] # With multiple limited queries. norm expect=TryDecorrelateLimitOne @@ -2916,41 +2916,41 @@ project ├── key: (1) ├── fd: (1)-->(2-5) └── select - ├── columns: k:1!null i:2 f:3 s:4 j:5 u:8!null + ├── columns: k:1!null i:2 f:3 s:4 j:5 u:10!null ├── key: (1) - ├── fd: (1)-->(2-5), (1)==(8), (8)==(1) + ├── fd: (1)-->(2-5), (1)==(10), (10)==(1) ├── distinct-on - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 u:8 + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 u:10 │ ├── grouping columns: k:1!null │ ├── key: (1) - │ ├── fd: (1)-->(2-5,8) + │ ├── fd: (1)-->(2-5,10) │ ├── left-join (hash) - │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6!null u:8 v:9 - │ │ ├── key: (1,8) - │ │ ├── fd: (1)-->(2-5), (1)==(6), (6)==(1), (8)-->(9) + │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:7!null u:10 v:11 + │ │ ├── key: (1,10) + │ │ ├── fd: (1)-->(2-5), (1)==(7), (7)==(1), (10)-->(11) │ │ ├── select - │ │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6!null + │ │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:7!null │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(2-5), (1)==(6), (6)==(1) + │ │ │ ├── fd: (1)-->(2-5), (1)==(7), (7)==(1) │ │ │ ├── distinct-on - │ │ │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6 + │ │ │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:7 │ │ │ │ ├── grouping columns: k:1!null │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(2-6) + │ │ │ │ ├── fd: (1)-->(2-5,7) │ │ │ │ ├── left-join (hash) - │ │ │ │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6 y:7 - │ │ │ │ │ ├── key: (1,6) - │ │ │ │ │ ├── fd: (1)-->(2-5), (6)-->(7) + │ │ │ │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:7 y:8 + │ │ │ │ │ ├── key: (1,7) + │ │ │ │ │ ├── fd: (1)-->(2-5), (7)-->(8) │ │ │ │ │ ├── scan a │ │ │ │ │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ │ │ │ │ │ ├── key: (1) │ │ │ │ │ │ └── fd: (1)-->(2-5) │ │ │ │ │ ├── scan xy - │ │ │ │ │ │ ├── columns: x:6!null y:7 - │ │ │ │ │ │ ├── key: (6) - │ │ │ │ │ │ └── fd: (6)-->(7) + │ │ │ │ │ │ ├── columns: x:7!null y:8 + │ │ │ │ │ │ ├── key: (7) + │ │ │ │ │ │ └── fd: (7)-->(8) │ │ │ │ │ └── filters - │ │ │ │ │ └── y:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] + │ │ │ │ │ └── y:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] │ │ │ │ └── aggregations │ │ │ │ ├── const-agg [as=i:2, outer=(2)] │ │ │ │ │ └── i:2 @@ -2960,16 +2960,16 @@ project │ │ │ │ │ └── s:4 │ │ │ │ ├── const-agg [as=j:5, outer=(5)] │ │ │ │ │ └── j:5 - │ │ │ │ └── first-agg [as=x:6, outer=(6)] - │ │ │ │ └── x:6 + │ │ │ │ └── first-agg [as=x:7, outer=(7)] + │ │ │ │ └── x:7 │ │ │ └── filters - │ │ │ └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ │ │ └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] │ │ ├── scan uv - │ │ │ ├── columns: u:8!null v:9 - │ │ │ ├── key: (8) - │ │ │ └── fd: (8)-->(9) + │ │ │ ├── columns: u:10!null v:11 + │ │ │ ├── key: (10) + │ │ │ └── fd: (10)-->(11) │ │ └── filters - │ │ └── v:9 = i:2 [outer=(2,9), constraints=(/2: (/NULL - ]; /9: (/NULL - ]), fd=(2)==(9), (9)==(2)] + │ │ └── v:11 = i:2 [outer=(2,11), constraints=(/2: (/NULL - ]; /11: (/NULL - ]), fd=(2)==(11), (11)==(2)] │ └── aggregations │ ├── const-agg [as=i:2, outer=(2)] │ │ └── i:2 @@ -2979,10 +2979,10 @@ project │ │ └── s:4 │ ├── const-agg [as=j:5, outer=(5)] │ │ └── j:5 - │ └── first-agg [as=u:8, outer=(8)] - │ └── u:8 + │ └── first-agg [as=u:10, outer=(10)] + │ └── u:10 └── filters - └── k:1 = u:8 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] + └── k:1 = u:10 [outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] # With nested limited queries. norm expect=TryDecorrelateLimitOne @@ -3004,54 +3004,54 @@ project ├── key: (1) ├── fd: (1)-->(2-5) └── select - ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6!null + ├── columns: k:1!null i:2 f:3 s:4 j:5 x:7!null ├── key: (1) - ├── fd: (1)-->(2-5), (1)==(6), (6)==(1) + ├── fd: (1)-->(2-5), (1)==(7), (7)==(1) ├── distinct-on - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6 + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:7 │ ├── grouping columns: k:1!null │ ├── key: (1) - │ ├── fd: (1)-->(2-6) + │ ├── fd: (1)-->(2-5,7) │ ├── left-join (hash) - │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6 y:7 u:8 - │ │ ├── key: (1,6) - │ │ ├── fd: (1)-->(2-5), (6)-->(7), (6)==(8), (8)==(6) + │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:7 y:8 u:10 + │ │ ├── key: (1,7) + │ │ ├── fd: (1)-->(2-5), (7)-->(8), (7)==(10), (10)==(7) │ │ ├── scan a │ │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2-5) │ │ ├── select - │ │ │ ├── columns: x:6!null y:7 u:8!null - │ │ │ ├── key: (6) - │ │ │ ├── fd: (6)-->(7), (6)==(8), (8)==(6) + │ │ │ ├── columns: x:7!null y:8 u:10!null + │ │ │ ├── key: (7) + │ │ │ ├── fd: (7)-->(8), (7)==(10), (10)==(7) │ │ │ ├── distinct-on - │ │ │ │ ├── columns: x:6!null y:7 u:8 - │ │ │ │ ├── grouping columns: x:6!null - │ │ │ │ ├── key: (6) - │ │ │ │ ├── fd: (6)-->(7,8) + │ │ │ │ ├── columns: x:7!null y:8 u:10 + │ │ │ │ ├── grouping columns: x:7!null + │ │ │ │ ├── key: (7) + │ │ │ │ ├── fd: (7)-->(8,10) │ │ │ │ ├── left-join (hash) - │ │ │ │ │ ├── columns: x:6!null y:7 u:8 v:9 - │ │ │ │ │ ├── key: (6,8) - │ │ │ │ │ ├── fd: (6)-->(7), (8)-->(9) + │ │ │ │ │ ├── columns: x:7!null y:8 u:10 v:11 + │ │ │ │ │ ├── key: (7,10) + │ │ │ │ │ ├── fd: (7)-->(8), (10)-->(11) │ │ │ │ │ ├── scan xy - │ │ │ │ │ │ ├── columns: x:6!null y:7 - │ │ │ │ │ │ ├── key: (6) - │ │ │ │ │ │ └── fd: (6)-->(7) + │ │ │ │ │ │ ├── columns: x:7!null y:8 + │ │ │ │ │ │ ├── key: (7) + │ │ │ │ │ │ └── fd: (7)-->(8) │ │ │ │ │ ├── scan uv - │ │ │ │ │ │ ├── columns: u:8!null v:9 - │ │ │ │ │ │ ├── key: (8) - │ │ │ │ │ │ └── fd: (8)-->(9) + │ │ │ │ │ │ ├── columns: u:10!null v:11 + │ │ │ │ │ │ ├── key: (10) + │ │ │ │ │ │ └── fd: (10)-->(11) │ │ │ │ │ └── filters - │ │ │ │ │ └── v:9 = y:7 [outer=(7,9), constraints=(/7: (/NULL - ]; /9: (/NULL - ]), fd=(7)==(9), (9)==(7)] + │ │ │ │ │ └── v:11 = y:8 [outer=(8,11), constraints=(/8: (/NULL - ]; /11: (/NULL - ]), fd=(8)==(11), (11)==(8)] │ │ │ │ └── aggregations - │ │ │ │ ├── const-agg [as=y:7, outer=(7)] - │ │ │ │ │ └── y:7 - │ │ │ │ └── first-agg [as=u:8, outer=(8)] - │ │ │ │ └── u:8 + │ │ │ │ ├── const-agg [as=y:8, outer=(8)] + │ │ │ │ │ └── y:8 + │ │ │ │ └── first-agg [as=u:10, outer=(10)] + │ │ │ │ └── u:10 │ │ │ └── filters - │ │ │ └── x:6 = u:8 [outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ]), fd=(6)==(8), (8)==(6)] + │ │ │ └── x:7 = u:10 [outer=(7,10), constraints=(/7: (/NULL - ]; /10: (/NULL - ]), fd=(7)==(10), (10)==(7)] │ │ └── filters - │ │ └── y:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] + │ │ └── y:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] │ └── aggregations │ ├── const-agg [as=i:2, outer=(2)] │ │ └── i:2 @@ -3061,10 +3061,10 @@ project │ │ └── s:4 │ ├── const-agg [as=j:5, outer=(5)] │ │ └── j:5 - │ └── first-agg [as=x:6, outer=(6)] - │ └── x:6 + │ └── first-agg [as=x:7, outer=(7)] + │ └── x:7 └── filters - └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # With inner join + ORDER BY. norm expect=TryDecorrelateLimitOne @@ -3079,42 +3079,42 @@ SELECT FROM xy ---- project - ├── columns: v:10 + ├── columns: v:13 ├── distinct-on - │ ├── columns: x:1!null uv.v:4 + │ ├── columns: x:1!null uv.v:5 │ ├── grouping columns: x:1!null │ ├── key: (1) - │ ├── fd: (1)-->(4) + │ ├── fd: (1)-->(5) │ ├── left-join (hash) - │ │ ├── columns: x:1!null u:3 uv.v:4 k:5 i:6 + │ │ ├── columns: x:1!null u:4 uv.v:5 k:7 i:8 │ │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) - │ │ ├── key: (1,5) - │ │ ├── fd: (3)-->(4), (5)-->(6), (3)==(5), (5)==(3) + │ │ ├── key: (1,7) + │ │ ├── fd: (4)-->(5), (7)-->(8), (4)==(7), (7)==(4) │ │ ├── scan xy │ │ │ ├── columns: x:1!null │ │ │ └── key: (1) │ │ ├── inner-join (hash) - │ │ │ ├── columns: u:3!null uv.v:4 k:5!null i:6 + │ │ │ ├── columns: u:4!null uv.v:5 k:7!null i:8 │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - │ │ │ ├── key: (5) - │ │ │ ├── fd: (3)-->(4), (5)-->(6), (3)==(5), (5)==(3) + │ │ │ ├── key: (7) + │ │ │ ├── fd: (4)-->(5), (7)-->(8), (4)==(7), (7)==(4) │ │ │ ├── scan uv - │ │ │ │ ├── columns: u:3!null uv.v:4 - │ │ │ │ ├── key: (3) - │ │ │ │ └── fd: (3)-->(4) + │ │ │ │ ├── columns: u:4!null uv.v:5 + │ │ │ │ ├── key: (4) + │ │ │ │ └── fd: (4)-->(5) │ │ │ ├── scan a - │ │ │ │ ├── columns: k:5!null i:6 - │ │ │ │ ├── key: (5) - │ │ │ │ └── fd: (5)-->(6) + │ │ │ │ ├── columns: k:7!null i:8 + │ │ │ │ ├── key: (7) + │ │ │ │ └── fd: (7)-->(8) │ │ │ └── filters - │ │ │ └── u:3 = k:5 [outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ]), fd=(3)==(5), (5)==(3)] + │ │ │ └── u:4 = k:7 [outer=(4,7), constraints=(/4: (/NULL - ]; /7: (/NULL - ]), fd=(4)==(7), (7)==(4)] │ │ └── filters - │ │ └── i:6 = x:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ │ └── i:8 = x:1 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] │ └── aggregations - │ └── first-agg [as=uv.v:4, outer=(4)] - │ └── uv.v:4 + │ └── first-agg [as=uv.v:5, outer=(5)] + │ └── uv.v:5 └── projections - └── uv.v:4 [as=v:10, outer=(4)] + └── uv.v:5 [as=v:13, outer=(5)] # With left join + ORDER BY. norm expect=TryDecorrelateLimitOne @@ -3125,41 +3125,41 @@ project ├── key: (1) ├── fd: (1)-->(2) └── select - ├── columns: x:1!null y:2 k:3!null + ├── columns: x:1!null y:2 k:4!null ├── key: (1) - ├── fd: (1)-->(2), (1)==(3), (3)==(1) + ├── fd: (1)-->(2), (1)==(4), (4)==(1) ├── distinct-on - │ ├── columns: x:1!null y:2 k:3 + │ ├── columns: x:1!null y:2 k:4 │ ├── grouping columns: x:1!null - │ ├── internal-ordering: +5,+6 opt(4) + │ ├── internal-ordering: +6,+7 │ ├── key: (1) - │ ├── fd: (1)-->(2,3) + │ ├── fd: (1)-->(2,4) │ ├── sort - │ │ ├── columns: x:1!null y:2 k:3 i:4 f:5 s:6 - │ │ ├── key: (1,3) - │ │ ├── fd: (1)-->(2), (3)-->(4-6) - │ │ ├── ordering: +5,+6 opt(4) [actual: +5,+6] + │ │ ├── columns: x:1!null y:2 k:4 i:5 f:6 s:7 + │ │ ├── key: (1,4) + │ │ ├── fd: (1)-->(2), (4)-->(5-7) + │ │ ├── ordering: +6,+7 │ │ └── left-join (hash) - │ │ ├── columns: x:1!null y:2 k:3 i:4 f:5 s:6 - │ │ ├── key: (1,3) - │ │ ├── fd: (1)-->(2), (3)-->(4-6) + │ │ ├── columns: x:1!null y:2 k:4 i:5 f:6 s:7 + │ │ ├── key: (1,4) + │ │ ├── fd: (1)-->(2), (4)-->(5-7) │ │ ├── scan xy │ │ │ ├── columns: x:1!null y:2 │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2) │ │ ├── scan a - │ │ │ ├── columns: k:3!null i:4 f:5 s:6 - │ │ │ ├── key: (3) - │ │ │ └── fd: (3)-->(4-6) + │ │ │ ├── columns: k:4!null i:5 f:6 s:7 + │ │ │ ├── key: (4) + │ │ │ └── fd: (4)-->(5-7) │ │ └── filters - │ │ └── i:4 = y:2 [outer=(2,4), constraints=(/2: (/NULL - ]; /4: (/NULL - ]), fd=(2)==(4), (4)==(2)] + │ │ └── i:5 = y:2 [outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ]), fd=(2)==(5), (5)==(2)] │ └── aggregations │ ├── const-agg [as=y:2, outer=(2)] │ │ └── y:2 - │ └── first-agg [as=k:3, outer=(3)] - │ └── k:3 + │ └── first-agg [as=k:4, outer=(4)] + │ └── k:4 └── filters - └── x:1 = k:3 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + └── x:1 = k:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] # -------------------------------------------------- # TryDecorrelateMax1Row @@ -3170,28 +3170,28 @@ norm format=show-miscprops expect=TryDecorrelateMax1Row SELECT (SELECT d FROM cd WHERE d=x) FROM xy ---- project - ├── columns: d:5 + ├── columns: d:7 ├── ensure-distinct-on - │ ├── columns: x:1!null cd.d:4 + │ ├── columns: x:1!null cd.d:5 │ ├── grouping columns: x:1!null │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── key: (1) - │ ├── fd: (1)-->(4) + │ ├── fd: (1)-->(5) │ ├── left-join (hash) - │ │ ├── columns: x:1!null cd.d:4 + │ │ ├── columns: x:1!null cd.d:5 │ │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) │ │ ├── scan xy │ │ │ ├── columns: x:1!null │ │ │ └── key: (1) │ │ ├── scan cd - │ │ │ └── columns: cd.d:4!null + │ │ │ └── columns: cd.d:5!null │ │ └── filters - │ │ └── cd.d:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + │ │ └── cd.d:5 = x:1 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] │ └── aggregations - │ └── const-agg [as=cd.d:4, outer=(4)] - │ └── cd.d:4 + │ └── const-agg [as=cd.d:5, outer=(5)] + │ └── cd.d:5 └── projections - └── cd.d:4 [as=d:5, outer=(4)] + └── cd.d:5 [as=d:7, outer=(5)] # Rule is activated on LeftJoinApply. # Case without a primary key on left side of join. @@ -3201,33 +3201,33 @@ SELECT * FROM (SELECT y FROM xy) WHERE 5 = (SELECT d FROM cd WHERE d=y) project ├── columns: y:2 └── select - ├── columns: x:1!null y:2 d:4!null + ├── columns: x:1!null y:2 d:5!null ├── key: (1) - ├── fd: ()-->(4), (1)-->(2) + ├── fd: ()-->(5), (1)-->(2) ├── ensure-distinct-on - │ ├── columns: x:1!null y:2 d:4 + │ ├── columns: x:1!null y:2 d:5 │ ├── grouping columns: x:1!null │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── key: (1) - │ ├── fd: (1)-->(2,4) + │ ├── fd: (1)-->(2,5) │ ├── left-join (hash) - │ │ ├── columns: x:1!null y:2 d:4 + │ │ ├── columns: x:1!null y:2 d:5 │ │ ├── fd: (1)-->(2) │ │ ├── scan xy │ │ │ ├── columns: x:1!null y:2 │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2) │ │ ├── scan cd - │ │ │ └── columns: d:4!null + │ │ │ └── columns: d:5!null │ │ └── filters - │ │ └── d:4 = y:2 [outer=(2,4), constraints=(/2: (/NULL - ]; /4: (/NULL - ]), fd=(2)==(4), (4)==(2)] + │ │ └── d:5 = y:2 [outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ]), fd=(2)==(5), (5)==(2)] │ └── aggregations │ ├── const-agg [as=y:2, outer=(2)] │ │ └── y:2 - │ └── const-agg [as=d:4, outer=(4)] - │ └── d:4 + │ └── const-agg [as=d:5, outer=(5)] + │ └── d:5 └── filters - └── d:4 = 5 [outer=(4), constraints=(/4: [/5 - /5]; tight), fd=()-->(4)] + └── d:5 = 5 [outer=(5), constraints=(/5: [/5 - /5]; tight), fd=()-->(5)] # Rule is activated on LeftJoinApply. # Case with multiple projected columns that will be translated to multiple @@ -3236,15 +3236,15 @@ norm format=show-miscprops expect=TryDecorrelateMax1Row SELECT (SELECT d FROM cd WHERE d=k), i, f, s, j FROM a ---- project - ├── columns: d:8 i:2 f:3 s:4 j:5 + ├── columns: d:10 i:2 f:3 s:4 j:5 ├── ensure-distinct-on - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 cd.d:7 + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 cd.d:8 │ ├── grouping columns: k:1!null │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── key: (1) - │ ├── fd: (1)-->(2-5,7) + │ ├── fd: (1)-->(2-5,8) │ ├── left-join (hash) - │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 cd.d:7 + │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 cd.d:8 │ │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) │ │ ├── fd: (1)-->(2-5) │ │ ├── scan a @@ -3252,9 +3252,9 @@ project │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2-5) │ │ ├── scan cd - │ │ │ └── columns: cd.d:7!null + │ │ │ └── columns: cd.d:8!null │ │ └── filters - │ │ └── cd.d:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + │ │ └── cd.d:8 = k:1 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] │ └── aggregations │ ├── const-agg [as=i:2, outer=(2)] │ │ └── i:2 @@ -3264,10 +3264,10 @@ project │ │ └── s:4 │ ├── const-agg [as=j:5, outer=(5)] │ │ └── j:5 - │ └── const-agg [as=cd.d:7, outer=(7)] - │ └── cd.d:7 + │ └── const-agg [as=cd.d:8, outer=(8)] + │ └── cd.d:8 └── projections - └── cd.d:7 [as=d:8, outer=(7)] + └── cd.d:8 [as=d:10, outer=(8)] # Rule is activated on LeftJoin. # This test is a bit fragile because it depends on DecorrelateJoin activating @@ -3277,54 +3277,54 @@ norm format=show-miscprops expect=TryDecorrelateMax1Row SELECT * FROM a LEFT JOIN LATERAL (SELECT * FROM uv WHERE (SELECT true FROM xy WHERE y=i)) ON true ---- project - ├── columns: k:1!null i:2 f:3 s:4 j:5 u:6 v:7 - ├── key: (1,6) - ├── fd: (1)-->(2-5), (1,6)-->(7) + ├── columns: k:1!null i:2 f:3 s:4 j:5 u:7 v:8 + ├── key: (1,7) + ├── fd: (1)-->(2-5), (1,7)-->(8) └── left-join-apply - ├── columns: k:1!null i:2 f:3 s:4 j:5 u:6 v:7 bool:10 - ├── key: (1,6) - ├── fd: (1)-->(2-5), (1,6)-->(7,10) + ├── columns: k:1!null i:2 f:3 s:4 j:5 u:7 v:8 bool:13 + ├── key: (1,7) + ├── fd: (1)-->(2-5), (1,7)-->(8,13) ├── scan a │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── ensure-distinct-on - │ ├── columns: u:6!null v:7 bool:10 - │ ├── grouping columns: u:6!null + │ ├── columns: u:7!null v:8 bool:13 + │ ├── grouping columns: u:7!null │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── outer: (2) - │ ├── key: (6) - │ ├── fd: (6)-->(7,10) + │ ├── key: (7) + │ ├── fd: (7)-->(8,13) │ ├── left-join (cross) - │ │ ├── columns: u:6!null v:7 bool:10 + │ │ ├── columns: u:7!null v:8 bool:13 │ │ ├── outer: (2) - │ │ ├── fd: (6)-->(7) + │ │ ├── fd: (7)-->(8) │ │ ├── scan uv - │ │ │ ├── columns: u:6!null v:7 - │ │ │ ├── key: (6) - │ │ │ └── fd: (6)-->(7) + │ │ │ ├── columns: u:7!null v:8 + │ │ │ ├── key: (7) + │ │ │ └── fd: (7)-->(8) │ │ ├── project - │ │ │ ├── columns: bool:10!null + │ │ │ ├── columns: bool:13!null │ │ │ ├── outer: (2) - │ │ │ ├── fd: ()-->(10) + │ │ │ ├── fd: ()-->(13) │ │ │ ├── select - │ │ │ │ ├── columns: y:9!null + │ │ │ │ ├── columns: y:11!null │ │ │ │ ├── outer: (2) - │ │ │ │ ├── fd: ()-->(9) + │ │ │ │ ├── fd: ()-->(11) │ │ │ │ ├── scan xy - │ │ │ │ │ └── columns: y:9 + │ │ │ │ │ └── columns: y:11 │ │ │ │ └── filters - │ │ │ │ └── y:9 = i:2 [outer=(2,9), constraints=(/2: (/NULL - ]; /9: (/NULL - ]), fd=(2)==(9), (9)==(2)] + │ │ │ │ └── y:11 = i:2 [outer=(2,11), constraints=(/2: (/NULL - ]; /11: (/NULL - ]), fd=(2)==(11), (11)==(2)] │ │ │ └── projections - │ │ │ └── true [as=bool:10] + │ │ │ └── true [as=bool:13] │ │ └── filters (true) │ └── aggregations - │ ├── const-agg [as=v:7, outer=(7)] - │ │ └── v:7 - │ └── const-agg [as=bool:10, outer=(10)] - │ └── bool:10 + │ ├── const-agg [as=v:8, outer=(8)] + │ │ └── v:8 + │ └── const-agg [as=bool:13, outer=(13)] + │ └── bool:13 └── filters - └── bool:10 [outer=(10), constraints=(/10: [/true - /true]; tight), fd=()-->(10)] + └── bool:13 [outer=(13), constraints=(/13: [/true - /true]; tight), fd=()-->(13)] # Rule is activated on InnerJoinApply. # An InnerJoinApply is used because subquery cardinality is guaranteed to be @@ -3333,34 +3333,34 @@ norm format=show-miscprops expect=TryDecorrelateMax1Row SELECT (SELECT t=x FROM (VALUES (1), (2)) f(t)) FROM xy ---- project - ├── columns: "?column?":5!null + ├── columns: "?column?":6!null ├── ensure-distinct-on - │ ├── columns: x:1!null "?column?":4!null + │ ├── columns: x:1!null "?column?":5!null │ ├── grouping columns: x:1!null │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── key: (1) - │ ├── fd: (1)-->(4) + │ ├── fd: (1)-->(5) │ ├── project - │ │ ├── columns: "?column?":4!null x:1!null + │ │ ├── columns: "?column?":5!null x:1!null │ │ ├── inner-join (cross) - │ │ │ ├── columns: x:1!null column1:3!null + │ │ │ ├── columns: x:1!null column1:4!null │ │ │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-more) │ │ │ ├── scan xy │ │ │ │ ├── columns: x:1!null │ │ │ │ └── key: (1) │ │ │ ├── values - │ │ │ │ ├── columns: column1:3!null + │ │ │ │ ├── columns: column1:4!null │ │ │ │ ├── cardinality: [2 - 2] │ │ │ │ ├── (1,) │ │ │ │ └── (2,) │ │ │ └── filters (true) │ │ └── projections - │ │ └── column1:3 = x:1 [as="?column?":4, outer=(1,3)] + │ │ └── column1:4 = x:1 [as="?column?":5, outer=(1,4)] │ └── aggregations - │ └── const-agg [as="?column?":4, outer=(4)] - │ └── "?column?":4 + │ └── const-agg [as="?column?":5, outer=(5)] + │ └── "?column?":5 └── projections - └── "?column?":4 [as="?column?":5, outer=(4)] + └── "?column?":5 [as="?column?":6, outer=(5)] # Rule is activated on InnerJoin. # This test is a bit fragile because it depends on DecorrelateJoin activating @@ -3369,51 +3369,51 @@ norm format=show-miscprops expect=TryDecorrelateMax1Row SELECT * FROM a LEFT JOIN LATERAL (SELECT * FROM uv WHERE (SELECT y=i FROM (VALUES (1), (2)) v(y))) ON true ---- project - ├── columns: k:1!null i:2 f:3 s:4 j:5 u:6 v:7 - ├── key: (1,6) - ├── fd: (1)-->(2-5), (1,6)-->(7) + ├── columns: k:1!null i:2 f:3 s:4 j:5 u:7 v:8 + ├── key: (1,7) + ├── fd: (1)-->(2-5), (1,7)-->(8) └── left-join-apply - ├── columns: k:1!null i:2 f:3 s:4 j:5 u:6 v:7 "?column?":9 - ├── key: (1,6) - ├── fd: (1)-->(2-5), (1,6)-->(7,9) + ├── columns: k:1!null i:2 f:3 s:4 j:5 u:7 v:8 "?column?":11 + ├── key: (1,7) + ├── fd: (1)-->(2-5), (1,7)-->(8,11) ├── scan a │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── ensure-distinct-on - │ ├── columns: u:6!null v:7 "?column?":9 - │ ├── grouping columns: u:6!null + │ ├── columns: u:7!null v:8 "?column?":11 + │ ├── grouping columns: u:7!null │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── outer: (2) - │ ├── key: (6) - │ ├── fd: (6)-->(7,9) + │ ├── key: (7) + │ ├── fd: (7)-->(8,11) │ ├── project - │ │ ├── columns: "?column?":9 u:6!null v:7 + │ │ ├── columns: "?column?":11 u:7!null v:8 │ │ ├── outer: (2) - │ │ ├── fd: (6)-->(7) + │ │ ├── fd: (7)-->(8) │ │ ├── inner-join (cross) - │ │ │ ├── columns: u:6!null v:7 column1:8!null + │ │ │ ├── columns: u:7!null v:8 column1:10!null │ │ │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-more) - │ │ │ ├── fd: (6)-->(7) + │ │ │ ├── fd: (7)-->(8) │ │ │ ├── scan uv - │ │ │ │ ├── columns: u:6!null v:7 - │ │ │ │ ├── key: (6) - │ │ │ │ └── fd: (6)-->(7) + │ │ │ │ ├── columns: u:7!null v:8 + │ │ │ │ ├── key: (7) + │ │ │ │ └── fd: (7)-->(8) │ │ │ ├── values - │ │ │ │ ├── columns: column1:8!null + │ │ │ │ ├── columns: column1:10!null │ │ │ │ ├── cardinality: [2 - 2] │ │ │ │ ├── (1,) │ │ │ │ └── (2,) │ │ │ └── filters (true) │ │ └── projections - │ │ └── column1:8 = i:2 [as="?column?":9, outer=(2,8)] + │ │ └── column1:10 = i:2 [as="?column?":11, outer=(2,10)] │ └── aggregations - │ ├── const-agg [as=v:7, outer=(7)] - │ │ └── v:7 - │ └── const-agg [as="?column?":9, outer=(9)] - │ └── "?column?":9 + │ ├── const-agg [as=v:8, outer=(8)] + │ │ └── v:8 + │ └── const-agg [as="?column?":11, outer=(11)] + │ └── "?column?":11 └── filters - └── "?column?":9 [outer=(9), constraints=(/9: [/true - /true]; tight), fd=()-->(9)] + └── "?column?":11 [outer=(11), constraints=(/11: [/true - /true]; tight), fd=()-->(11)] # Filter (y=3) must not be pushed through EnsureDistinctOn in order to ensure # expected error behavior. @@ -3423,26 +3423,26 @@ SELECT a FROM (VALUES (0), (0)) v(a) WHERE (SELECT y FROM xy WHERE x<>a)=3 project ├── columns: a:1!null └── select - ├── columns: column1:1!null y:3!null rownum:4!null - ├── key: (4) - ├── fd: ()-->(3), (4)-->(1) + ├── columns: column1:1!null y:3!null rownum:5!null + ├── key: (5) + ├── fd: ()-->(3), (5)-->(1) ├── ensure-distinct-on - │ ├── columns: column1:1!null y:3 rownum:4!null - │ ├── grouping columns: rownum:4!null + │ ├── columns: column1:1!null y:3 rownum:5!null + │ ├── grouping columns: rownum:5!null │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── cardinality: [1 - ] - │ ├── key: (4) - │ ├── fd: (4)-->(1,3) + │ ├── key: (5) + │ ├── fd: (5)-->(1,3) │ ├── left-join (cross) - │ │ ├── columns: column1:1!null x:2 y:3 rownum:4!null + │ │ ├── columns: column1:1!null x:2 y:3 rownum:5!null │ │ ├── cardinality: [2 - ] - │ │ ├── key: (2,4) - │ │ ├── fd: (4)-->(1), (2)-->(3) + │ │ ├── key: (2,5) + │ │ ├── fd: (5)-->(1), (2)-->(3) │ │ ├── ordinality - │ │ │ ├── columns: column1:1!null rownum:4!null + │ │ │ ├── columns: column1:1!null rownum:5!null │ │ │ ├── cardinality: [2 - 2] - │ │ │ ├── key: (4) - │ │ │ ├── fd: (4)-->(1) + │ │ │ ├── key: (5) + │ │ │ ├── fd: (5)-->(1) │ │ │ └── values │ │ │ ├── columns: column1:1!null │ │ │ ├── cardinality: [2 - 2] @@ -3477,10 +3477,10 @@ semi-join (hash) │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── scan xy - │ ├── columns: x:6!null - │ └── key: (6) + │ ├── columns: x:7!null + │ └── key: (7) └── filters - └── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # Ensure that EXISTS is hoisted even when it is one of several conjuncts. norm expect=HoistSelectExists @@ -3502,10 +3502,10 @@ semi-join (hash) │ ├── s:4 = 'foo' [outer=(4), constraints=(/4: [/'foo' - /'foo']; tight), fd=()-->(4)] │ └── i:2 > 1 [outer=(2), constraints=(/2: [/2 - ]; tight)] ├── scan xy - │ ├── columns: x:6!null - │ └── key: (6) + │ ├── columns: x:7!null + │ └── key: (7) └── filters - └── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # Multiple Exists operators in same Select list. norm expect=HoistSelectExists @@ -3524,15 +3524,15 @@ semi-join (hash) │ │ ├── key: (1) │ │ └── fd: (1)-->(2-5) │ ├── scan xy - │ │ ├── columns: x:8!null - │ │ └── key: (8) + │ │ ├── columns: x:10!null + │ │ └── key: (10) │ └── filters - │ └── x:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] + │ └── x:10 = i:2 [outer=(2,10), constraints=(/2: (/NULL - ]; /10: (/NULL - ]), fd=(2)==(10), (10)==(2)] ├── scan xy - │ ├── columns: x:6!null - │ └── key: (6) + │ ├── columns: x:7!null + │ └── key: (7) └── filters - └── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # Don't hoist uncorrelated subqueries. norm expect-not=HoistSelectExists @@ -3549,14 +3549,14 @@ select └── filters └── exists [subquery] └── limit - ├── columns: x:6!null y:7 + ├── columns: x:7!null y:8 ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(6,7) + ├── fd: ()-->(7,8) ├── scan xy - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ ├── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + │ ├── fd: (7)-->(8) │ └── limit hint: 1.00 └── 1 @@ -3573,18 +3573,18 @@ semi-join (hash) │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── semi-join (hash) - │ ├── columns: x:6!null - │ ├── key: (6) + │ ├── columns: x:7!null + │ ├── key: (7) │ ├── scan xy - │ │ ├── columns: x:6!null - │ │ └── key: (6) + │ │ ├── columns: x:7!null + │ │ └── key: (7) │ ├── scan uv - │ │ ├── columns: u:8!null - │ │ └── key: (8) + │ │ ├── columns: u:10!null + │ │ └── key: (10) │ └── filters - │ └── x:6 = u:8 [outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ]), fd=(6)==(8), (8)==(6)] + │ └── x:7 = u:10 [outer=(7,10), constraints=(/7: (/NULL - ]; /10: (/NULL - ]), fd=(7)==(10), (10)==(7)] └── filters - └── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # -------------------------------------------------- # HoistSelectNotExists @@ -3601,10 +3601,10 @@ anti-join (hash) │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── scan xy - │ ├── columns: x:6!null - │ └── key: (6) + │ ├── columns: x:7!null + │ └── key: (7) └── filters - └── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # Ensure that NOT EXISTS is hoisted even when one of several conjuncts. norm expect=HoistSelectNotExists @@ -3626,10 +3626,10 @@ anti-join (hash) │ ├── s:4 = 'foo' [outer=(4), constraints=(/4: [/'foo' - /'foo']; tight), fd=()-->(4)] │ └── i:2 > 1 [outer=(2), constraints=(/2: [/2 - ]; tight)] ├── scan xy - │ ├── columns: x:6!null - │ └── key: (6) + │ ├── columns: x:7!null + │ └── key: (7) └── filters - └── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # Multiple Not Exists operators in same Select list. norm expect=HoistSelectNotExists @@ -3650,15 +3650,15 @@ anti-join (hash) │ │ ├── key: (1) │ │ └── fd: (1)-->(2-5) │ ├── scan xy - │ │ ├── columns: x:8!null - │ │ └── key: (8) + │ │ ├── columns: x:10!null + │ │ └── key: (10) │ └── filters - │ └── x:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] + │ └── x:10 = i:2 [outer=(2,10), constraints=(/2: (/NULL - ]; /10: (/NULL - ]), fd=(2)==(10), (10)==(2)] ├── scan xy - │ ├── columns: x:6!null - │ └── key: (6) + │ ├── columns: x:7!null + │ └── key: (7) └── filters - └── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # Don't hoist uncorrelated subqueries. norm expect-not=HoistSelectNotExists @@ -3676,14 +3676,14 @@ select └── not [subquery] └── exists └── limit - ├── columns: x:6!null y:7 + ├── columns: x:7!null y:8 ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(6,7) + ├── fd: ()-->(7,8) ├── scan xy - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ ├── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + │ ├── fd: (7)-->(8) │ └── limit hint: 1.00 └── 1 @@ -3706,15 +3706,15 @@ semi-join (hash) │ │ ├── key: (1) │ │ └── fd: (1)-->(2-5) │ ├── scan xy - │ │ ├── columns: x:8!null - │ │ └── key: (8) + │ │ ├── columns: x:10!null + │ │ └── key: (10) │ └── filters - │ └── x:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] + │ └── x:10 = i:2 [outer=(2,10), constraints=(/2: (/NULL - ]; /10: (/NULL - ]), fd=(2)==(10), (10)==(2)] ├── scan xy - │ ├── columns: x:6!null - │ └── key: (6) + │ ├── columns: x:7!null + │ └── key: (7) └── filters - └── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # -------------------------------------------------- # HoistSelectSubquery @@ -3727,16 +3727,16 @@ project ├── key: (1) ├── fd: (1)-->(2-5) └── select - ├── columns: k:1!null i:2!null f:3 s:4 j:5 y:7!null + ├── columns: k:1!null i:2!null f:3 s:4 j:5 y:8!null ├── key: (1) - ├── fd: (1)-->(2-5,7), (2)==(7), (7)==(2) + ├── fd: (1)-->(2-5,8), (2)==(8), (8)==(2) ├── distinct-on - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 y:7 + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 y:8 │ ├── grouping columns: k:1!null │ ├── key: (1) - │ ├── fd: (1)-->(2-5,7) + │ ├── fd: (1)-->(2-5,8) │ ├── left-join (hash) - │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 y:7 + │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 y:8 │ │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) │ │ ├── fd: (1)-->(2-5) │ │ ├── scan a @@ -3744,9 +3744,9 @@ project │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2-5) │ │ ├── scan xy - │ │ │ └── columns: y:7 + │ │ │ └── columns: y:8 │ │ └── filters - │ │ └── y:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + │ │ └── y:8 = k:1 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] │ └── aggregations │ ├── const-agg [as=i:2, outer=(2)] │ │ └── i:2 @@ -3756,10 +3756,10 @@ project │ │ └── s:4 │ ├── const-agg [as=j:5, outer=(5)] │ │ └── j:5 - │ └── first-agg [as=y:7, outer=(7)] - │ └── y:7 + │ └── first-agg [as=y:8, outer=(8)] + │ └── y:8 └── filters - └── i:2 = y:7 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] + └── i:2 = y:8 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] # Multiple other conjuncts, including uncorrelated subquery (don't hoist). norm expect=HoistSelectSubquery disable=InlineConstVar @@ -3773,17 +3773,17 @@ project ├── key: () ├── fd: ()-->(1-5) └── select - ├── columns: k:1!null i:2!null f:3 s:4 j:5 y:7!null + ├── columns: k:1!null i:2!null f:3 s:4 j:5 y:8!null ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(1-5,7) + ├── fd: ()-->(1-5,8) ├── limit - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 y:7 + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 y:8 │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(1-5,7) + │ ├── fd: ()-->(1-5,8) │ ├── left-join (hash) - │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 y:7 + │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 y:8 │ │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) │ │ ├── fd: ()-->(1-5) │ │ ├── limit hint: 1.00 @@ -3801,41 +3801,41 @@ project │ │ │ └── eq [subquery] │ │ │ ├── subquery │ │ │ │ └── limit - │ │ │ │ ├── columns: x:8!null + │ │ │ │ ├── columns: x:10!null │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ ├── key: () - │ │ │ │ ├── fd: ()-->(8) + │ │ │ │ ├── fd: ()-->(10) │ │ │ │ ├── scan xy - │ │ │ │ │ ├── columns: x:8!null - │ │ │ │ │ ├── key: (8) + │ │ │ │ │ ├── columns: x:10!null + │ │ │ │ │ ├── key: (10) │ │ │ │ │ └── limit hint: 1.00 │ │ │ │ └── 1 │ │ │ └── 100 │ │ ├── select - │ │ │ ├── columns: y:7!null - │ │ │ ├── fd: ()-->(7) + │ │ │ ├── columns: y:8!null + │ │ │ ├── fd: ()-->(8) │ │ │ ├── scan xy - │ │ │ │ └── columns: y:7 + │ │ │ │ └── columns: y:8 │ │ │ └── filters - │ │ │ ├── y:7 = 10 [outer=(7), constraints=(/7: [/10 - /10]; tight), fd=()-->(7)] + │ │ │ ├── y:8 = 10 [outer=(8), constraints=(/8: [/10 - /10]; tight), fd=()-->(8)] │ │ │ └── eq [subquery] │ │ │ ├── subquery │ │ │ │ └── limit - │ │ │ │ ├── columns: x:8!null + │ │ │ │ ├── columns: x:10!null │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ ├── key: () - │ │ │ │ ├── fd: ()-->(8) + │ │ │ │ ├── fd: ()-->(10) │ │ │ │ ├── scan xy - │ │ │ │ │ ├── columns: x:8!null - │ │ │ │ │ ├── key: (8) + │ │ │ │ │ ├── columns: x:10!null + │ │ │ │ │ ├── key: (10) │ │ │ │ │ └── limit hint: 1.00 │ │ │ │ └── 1 │ │ │ └── 100 │ │ └── filters - │ │ └── y:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + │ │ └── y:8 = k:1 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] │ └── 1 └── filters - └── i:2 = y:7 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] + └── i:2 = y:8 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] # Multiple correlated subqueries. norm expect=HoistSelectSubquery @@ -3847,29 +3847,29 @@ project ├── key: (1) ├── fd: (1)-->(2-5) └── select - ├── columns: k:1!null i:2!null f:3 s:4 j:5 y:10!null + ├── columns: k:1!null i:2!null f:3 s:4 j:5 y:12!null ├── key: (1) - ├── fd: (1)-->(2-5,10), (2)==(10), (10)==(2) + ├── fd: (1)-->(2-5,12), (2)==(12), (12)==(2) ├── distinct-on - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 y:10 + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 y:12 │ ├── grouping columns: k:1!null │ ├── key: (1) - │ ├── fd: (1)-->(2-5,10) + │ ├── fd: (1)-->(2-5,12) │ ├── left-join (hash) - │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 count_rows:8!null y:10 + │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 count_rows:10!null y:12 │ │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) - │ │ ├── fd: (1)-->(2-5,8) + │ │ ├── fd: (1)-->(2-5,10) │ │ ├── select - │ │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 count_rows:8!null + │ │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 count_rows:10!null │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(2-5,8) + │ │ │ ├── fd: (1)-->(2-5,10) │ │ │ ├── group-by - │ │ │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 count_rows:8!null + │ │ │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 count_rows:10!null │ │ │ │ ├── grouping columns: k:1!null │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(2-5,8) + │ │ │ │ ├── fd: (1)-->(2-5,10) │ │ │ │ ├── left-join (hash) - │ │ │ │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 y:7 + │ │ │ │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 y:8 │ │ │ │ │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) │ │ │ │ │ ├── fd: (1)-->(2-5) │ │ │ │ │ ├── scan a @@ -3877,12 +3877,12 @@ project │ │ │ │ │ │ ├── key: (1) │ │ │ │ │ │ └── fd: (1)-->(2-5) │ │ │ │ │ ├── scan xy - │ │ │ │ │ │ └── columns: y:7 + │ │ │ │ │ │ └── columns: y:8 │ │ │ │ │ └── filters - │ │ │ │ │ └── y:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + │ │ │ │ │ └── y:8 = k:1 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] │ │ │ │ └── aggregations - │ │ │ │ ├── count [as=count_rows:8, outer=(7)] - │ │ │ │ │ └── y:7 + │ │ │ │ ├── count [as=count_rows:10, outer=(8)] + │ │ │ │ │ └── y:8 │ │ │ │ ├── const-agg [as=i:2, outer=(2)] │ │ │ │ │ └── i:2 │ │ │ │ ├── const-agg [as=f:3, outer=(3)] @@ -3892,11 +3892,11 @@ project │ │ │ │ └── const-agg [as=j:5, outer=(5)] │ │ │ │ └── j:5 │ │ │ └── filters - │ │ │ └── count_rows:8 > 0 [outer=(8), constraints=(/8: [/1 - ]; tight)] + │ │ │ └── count_rows:10 > 0 [outer=(10), constraints=(/10: [/1 - ]; tight)] │ │ ├── scan xy - │ │ │ └── columns: y:10 + │ │ │ └── columns: y:12 │ │ └── filters - │ │ └── y:10 = k:1 [outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] + │ │ └── y:12 = k:1 [outer=(1,12), constraints=(/1: (/NULL - ]; /12: (/NULL - ]), fd=(1)==(12), (12)==(1)] │ └── aggregations │ ├── const-agg [as=i:2, outer=(2)] │ │ └── i:2 @@ -3906,10 +3906,10 @@ project │ │ └── s:4 │ ├── const-agg [as=j:5, outer=(5)] │ │ └── j:5 - │ └── first-agg [as=y:10, outer=(10)] - │ └── y:10 + │ └── first-agg [as=y:12, outer=(12)] + │ └── y:12 └── filters - └── i:2 = y:10 [outer=(2,10), constraints=(/2: (/NULL - ]; /10: (/NULL - ]), fd=(2)==(10), (10)==(2)] + └── i:2 = y:12 [outer=(2,12), constraints=(/2: (/NULL - ]; /12: (/NULL - ]), fd=(2)==(12), (12)==(2)] # Subquery nested below interesting scalar operators like cast, function, tuple, # or, etc). @@ -3922,32 +3922,32 @@ project ├── key: (1) ├── fd: (1)-->(2-5) └── select - ├── columns: k:1!null i:2 f:3 s:4 j:5 count_rows:8!null + ├── columns: k:1!null i:2 f:3 s:4 j:5 count_rows:10!null ├── immutable ├── key: (1) - ├── fd: (1)-->(2-5,8) + ├── fd: (1)-->(2-5,10) ├── group-by - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 count_rows:8!null + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 count_rows:10!null │ ├── grouping columns: k:1!null │ ├── key: (1) - │ ├── fd: (1)-->(2-5,8) + │ ├── fd: (1)-->(2-5,10) │ ├── left-join (hash) - │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 u:6 + │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 u:7 │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2-6) + │ │ ├── fd: (1)-->(2-5,7) │ │ ├── scan a │ │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2-5) │ │ ├── scan uv - │ │ │ ├── columns: u:6!null - │ │ │ └── key: (6) + │ │ │ ├── columns: u:7!null + │ │ │ └── key: (7) │ │ └── filters - │ │ └── k:1 = u:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ │ └── k:1 = u:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] │ └── aggregations - │ ├── count [as=count_rows:8, outer=(6)] - │ │ └── u:6 + │ ├── count [as=count_rows:10, outer=(7)] + │ │ └── u:7 │ ├── const-agg [as=i:2, outer=(2)] │ │ └── i:2 │ ├── const-agg [as=f:3, outer=(3)] @@ -3957,7 +3957,7 @@ project │ └── const-agg [as=j:5, outer=(5)] │ └── j:5 └── filters - └── ((0, length(count_rows:8::STRING)) > (0, 1)) OR (i:2 = 1) [outer=(2,8), immutable] + └── ((0, length(count_rows:10::STRING)) > (0, 1)) OR (i:2 = 1) [outer=(2,10), immutable] # Exists within a disjunction. norm expect=HoistSelectSubquery @@ -3968,33 +3968,33 @@ project ├── key: (1) ├── fd: (1)-->(2-5) └── select - ├── columns: k:1!null i:2 f:3 s:4 j:5 true_agg:9 + ├── columns: k:1!null i:2 f:3 s:4 j:5 true_agg:11 ├── key: (1) - ├── fd: (1)-->(2-5,9) + ├── fd: (1)-->(2-5,11) ├── group-by - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 true_agg:9 + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 true_agg:11 │ ├── grouping columns: k:1!null │ ├── key: (1) - │ ├── fd: (1)-->(2-5,9) + │ ├── fd: (1)-->(2-5,11) │ ├── left-join (hash) - │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 y:7 true:8 + │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 y:8 true:10 │ │ ├── fd: (1)-->(2-5) │ │ ├── scan a │ │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2-5) │ │ ├── project - │ │ │ ├── columns: true:8!null y:7 - │ │ │ ├── fd: ()-->(8) + │ │ │ ├── columns: true:10!null y:8 + │ │ │ ├── fd: ()-->(10) │ │ │ ├── scan xy - │ │ │ │ └── columns: y:7 + │ │ │ │ └── columns: y:8 │ │ │ └── projections - │ │ │ └── true [as=true:8] + │ │ │ └── true [as=true:10] │ │ └── filters - │ │ └── y:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] + │ │ └── y:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] │ └── aggregations - │ ├── const-not-null-agg [as=true_agg:9, outer=(8)] - │ │ └── true:8 + │ ├── const-not-null-agg [as=true_agg:11, outer=(10)] + │ │ └── true:10 │ ├── const-agg [as=i:2, outer=(2)] │ │ └── i:2 │ ├── const-agg [as=f:3, outer=(3)] @@ -4004,7 +4004,7 @@ project │ └── const-agg [as=j:5, outer=(5)] │ └── j:5 └── filters - └── (i:2 = 1) OR (true_agg:9 IS NOT NULL) [outer=(2,9)] + └── (i:2 = 1) OR (true_agg:11 IS NOT NULL) [outer=(2,11)] # Any with IS NULL. norm expect=HoistSelectSubquery @@ -4015,43 +4015,43 @@ project ├── key: (1) ├── fd: (1)-->(2-5) └── select - ├── columns: k:1!null i:2 f:3 s:4 j:5 case:10 + ├── columns: k:1!null i:2 f:3 s:4 j:5 case:12 ├── key: (1) - ├── fd: ()-->(10), (1)-->(2-5) + ├── fd: ()-->(12), (1)-->(2-5) ├── project - │ ├── columns: case:10 k:1!null i:2 f:3 s:4 j:5 + │ ├── columns: case:12 k:1!null i:2 f:3 s:4 j:5 │ ├── key: (1) - │ ├── fd: (1)-->(2-5,10) + │ ├── fd: (1)-->(2-5,12) │ ├── group-by - │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 bool_or:9 + │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 bool_or:11 │ │ ├── grouping columns: k:1!null │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2-5,9) + │ │ ├── fd: (1)-->(2-5,11) │ │ ├── left-join (hash) - │ │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6 y:7 notnull:8 + │ │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:7 y:8 notnull:10 │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(2-8), (6)-->(7), (7)~~>(8) + │ │ │ ├── fd: (1)-->(2-5,7,8,10), (7)-->(8), (8)~~>(10) │ │ │ ├── scan a │ │ │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ │ │ │ ├── key: (1) │ │ │ │ └── fd: (1)-->(2-5) │ │ │ ├── project - │ │ │ │ ├── columns: notnull:8!null x:6!null y:7 - │ │ │ │ ├── key: (6) - │ │ │ │ ├── fd: (6)-->(7), (7)-->(8) + │ │ │ │ ├── columns: notnull:10!null x:7!null y:8 + │ │ │ │ ├── key: (7) + │ │ │ │ ├── fd: (7)-->(8), (8)-->(10) │ │ │ │ ├── scan xy - │ │ │ │ │ ├── columns: x:6!null y:7 - │ │ │ │ │ ├── key: (6) - │ │ │ │ │ └── fd: (6)-->(7) + │ │ │ │ │ ├── columns: x:7!null y:8 + │ │ │ │ │ ├── key: (7) + │ │ │ │ │ └── fd: (7)-->(8) │ │ │ │ └── projections - │ │ │ │ └── y:7 IS NOT NULL [as=notnull:8, outer=(7)] + │ │ │ │ └── y:8 IS NOT NULL [as=notnull:10, outer=(8)] │ │ │ └── filters - │ │ │ ├── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ │ │ └── (i:2 = y:7) IS NOT false [outer=(2,7)] + │ │ │ ├── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + │ │ │ └── (i:2 = y:8) IS NOT false [outer=(2,8)] │ │ └── aggregations - │ │ ├── bool-or [as=bool_or:9, outer=(8)] - │ │ │ └── notnull:8 + │ │ ├── bool-or [as=bool_or:11, outer=(10)] + │ │ │ └── notnull:10 │ │ ├── const-agg [as=i:2, outer=(2)] │ │ │ └── i:2 │ │ ├── const-agg [as=f:3, outer=(3)] @@ -4061,9 +4061,9 @@ project │ │ └── const-agg [as=j:5, outer=(5)] │ │ └── j:5 │ └── projections - │ └── CASE WHEN bool_or:9 AND (i:2 IS NOT NULL) THEN true WHEN bool_or:9 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:10, outer=(2,9)] + │ └── CASE WHEN bool_or:11 AND (i:2 IS NOT NULL) THEN true WHEN bool_or:11 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:12, outer=(2,11)] └── filters - └── case:10 IS NULL [outer=(10), constraints=(/10: [/NULL - /NULL]; tight), fd=()-->(10)] + └── case:12 IS NULL [outer=(12), constraints=(/12: [/NULL - /NULL]; tight), fd=()-->(12)] # Any with uncorrelated subquery (should not be hoisted). norm @@ -4081,7 +4081,7 @@ select └── is [outer=(2), correlated-subquery] ├── any: eq │ ├── scan xy - │ │ └── columns: y:7 + │ │ └── columns: y:8 │ └── i:2 └── NULL @@ -4090,53 +4090,53 @@ norm SELECT i*i/100 < ALL(SELECT y FROM xy WHERE x=k) AS r, s FROM a ---- project - ├── columns: r:8 s:4 + ├── columns: r:10 s:4 ├── immutable ├── group-by - │ ├── columns: k:1!null s:4 scalar:9 bool_or:11 + │ ├── columns: k:1!null s:4 scalar:11 bool_or:13 │ ├── grouping columns: k:1!null │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(4,9,11) + │ ├── fd: (1)-->(4,11,13) │ ├── left-join (hash) - │ │ ├── columns: k:1!null s:4 x:6 y:7 scalar:9 notnull:10 + │ │ ├── columns: k:1!null s:4 x:7 y:8 scalar:11 notnull:12 │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ ├── immutable │ │ ├── key: (1) - │ │ ├── fd: (1)-->(4,6,7,9,10), (6)-->(7), (7)~~>(10) + │ │ ├── fd: (1)-->(4,7,8,11,12), (7)-->(8), (8)~~>(12) │ │ ├── project - │ │ │ ├── columns: scalar:9 k:1!null s:4 + │ │ │ ├── columns: scalar:11 k:1!null s:4 │ │ │ ├── immutable │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(4,9) + │ │ │ ├── fd: (1)-->(4,11) │ │ │ ├── scan a │ │ │ │ ├── columns: k:1!null i:2 s:4 │ │ │ │ ├── key: (1) │ │ │ │ └── fd: (1)-->(2,4) │ │ │ └── projections - │ │ │ └── (i:2 * i:2) / 100 [as=scalar:9, outer=(2), immutable] + │ │ │ └── (i:2 * i:2) / 100 [as=scalar:11, outer=(2), immutable] │ │ ├── project - │ │ │ ├── columns: notnull:10!null x:6!null y:7 - │ │ │ ├── key: (6) - │ │ │ ├── fd: (6)-->(7), (7)-->(10) + │ │ │ ├── columns: notnull:12!null x:7!null y:8 + │ │ │ ├── key: (7) + │ │ │ ├── fd: (7)-->(8), (8)-->(12) │ │ │ ├── scan xy - │ │ │ │ ├── columns: x:6!null y:7 - │ │ │ │ ├── key: (6) - │ │ │ │ └── fd: (6)-->(7) + │ │ │ │ ├── columns: x:7!null y:8 + │ │ │ │ ├── key: (7) + │ │ │ │ └── fd: (7)-->(8) │ │ │ └── projections - │ │ │ └── y:7 IS NOT NULL [as=notnull:10, outer=(7)] + │ │ │ └── y:8 IS NOT NULL [as=notnull:12, outer=(8)] │ │ └── filters - │ │ ├── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ │ └── (scalar:9 >= y:7) IS NOT false [outer=(7,9)] + │ │ ├── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + │ │ └── (scalar:11 >= y:8) IS NOT false [outer=(8,11)] │ └── aggregations - │ ├── bool-or [as=bool_or:11, outer=(10)] - │ │ └── notnull:10 + │ ├── bool-or [as=bool_or:13, outer=(12)] + │ │ └── notnull:12 │ ├── const-agg [as=s:4, outer=(4)] │ │ └── s:4 - │ └── const-agg [as=scalar:9, outer=(9)] - │ └── scalar:9 + │ └── const-agg [as=scalar:11, outer=(11)] + │ └── scalar:11 └── projections - └── NOT CASE WHEN bool_or:11 AND (scalar:9 IS NOT NULL) THEN true WHEN bool_or:11 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=r:8, outer=(9,11), immutable] + └── NOT CASE WHEN bool_or:13 AND (scalar:11 IS NOT NULL) THEN true WHEN bool_or:13 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=r:10, outer=(11,13), immutable] # Regress issue #32270: Panic when expression contains both correlated and # uncorrelated subquery. @@ -4148,37 +4148,37 @@ project ├── key: (1) ├── fd: (1)-->(2-5) └── select - ├── columns: k:1!null i:2 f:3 s:4 j:5 true_agg:11 + ├── columns: k:1!null i:2 f:3 s:4 j:5 true_agg:14 ├── key: (1) - ├── fd: (1)-->(2-5,11) + ├── fd: (1)-->(2-5,14) ├── group-by - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 true_agg:11 + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 true_agg:14 │ ├── grouping columns: k:1!null │ ├── key: (1) - │ ├── fd: (1)-->(2-5,11) + │ ├── fd: (1)-->(2-5,14) │ ├── left-join (hash) - │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:8 true:10 + │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:10 true:13 │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2-5,8,10) + │ │ ├── fd: (1)-->(2-5,10,13) │ │ ├── scan a │ │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2-5) │ │ ├── project - │ │ │ ├── columns: true:10!null x:8!null - │ │ │ ├── key: (8) - │ │ │ ├── fd: ()-->(10) + │ │ │ ├── columns: true:13!null x:10!null + │ │ │ ├── key: (10) + │ │ │ ├── fd: ()-->(13) │ │ │ ├── scan xy - │ │ │ │ ├── columns: x:8!null - │ │ │ │ └── key: (8) + │ │ │ │ ├── columns: x:10!null + │ │ │ │ └── key: (10) │ │ │ └── projections - │ │ │ └── true [as=true:10] + │ │ │ └── true [as=true:13] │ │ └── filters - │ │ └── x:8 = k:1 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] + │ │ └── x:10 = k:1 [outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] │ └── aggregations - │ ├── const-not-null-agg [as=true_agg:11, outer=(10)] - │ │ └── true:10 + │ ├── const-not-null-agg [as=true_agg:14, outer=(13)] + │ │ └── true:13 │ ├── const-agg [as=i:2, outer=(2)] │ │ └── i:2 │ ├── const-agg [as=f:3, outer=(3)] @@ -4188,20 +4188,20 @@ project │ └── const-agg [as=j:5, outer=(5)] │ └── j:5 └── filters - └── or [outer=(11), subquery] + └── or [outer=(14), subquery] ├── exists │ └── limit - │ ├── columns: x:6!null y:7 + │ ├── columns: x:7!null y:8 │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(6,7) + │ ├── fd: ()-->(7,8) │ ├── scan xy - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ ├── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ ├── fd: (7)-->(8) │ │ └── limit hint: 1.00 │ └── 1 - └── true_agg:11 IS NOT NULL + └── true_agg:14 IS NOT NULL # -------------------------------------------------- # HoistProjectSubquery @@ -4210,22 +4210,22 @@ norm expect=HoistProjectSubquery SELECT (SELECT x FROM xy WHERE x=k) FROM a ---- project - ├── columns: x:8 + ├── columns: x:10 ├── left-join (hash) - │ ├── columns: k:1!null xy.x:6 + │ ├── columns: k:1!null xy.x:7 │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ ├── key: (1) - │ ├── fd: (1)-->(6) + │ ├── fd: (1)-->(7) │ ├── scan a │ │ ├── columns: k:1!null │ │ └── key: (1) │ ├── scan xy - │ │ ├── columns: xy.x:6!null - │ │ └── key: (6) + │ │ ├── columns: xy.x:7!null + │ │ └── key: (7) │ └── filters - │ └── xy.x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ └── xy.x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] └── projections - └── xy.x:6 [as=x:8, outer=(6)] + └── xy.x:7 [as=x:10, outer=(7)] # Mixed correlated and uncorrelated subqueries. norm expect=HoistProjectSubquery @@ -4239,195 +4239,195 @@ SELECT FROM a ---- project - ├── columns: a:17!null x:18 y:19 b:20 exists:21 count:22!null - ├── fd: ()-->(17,19-21) + ├── columns: a:23!null x:24 y:25 b:26 exists:27 count:28!null + ├── fd: ()-->(23,25-27) ├── group-by - │ ├── columns: k:1!null xy.x:6 count_rows:16!null + │ ├── columns: k:1!null xy.x:7 count_rows:22!null │ ├── grouping columns: k:1!null │ ├── key: (1) - │ ├── fd: (1)-->(6,16) + │ ├── fd: (1)-->(7,22) │ ├── left-join (hash) - │ │ ├── columns: k:1!null xy.x:6 xy.y:15 + │ │ ├── columns: k:1!null xy.x:7 xy.y:20 │ │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) - │ │ ├── fd: (1)-->(6) + │ │ ├── fd: (1)-->(7) │ │ ├── left-join (hash) - │ │ │ ├── columns: k:1!null xy.x:6 + │ │ │ ├── columns: k:1!null xy.x:7 │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(6) + │ │ │ ├── fd: (1)-->(7) │ │ │ ├── scan a │ │ │ │ ├── columns: k:1!null │ │ │ │ └── key: (1) │ │ │ ├── scan xy - │ │ │ │ ├── columns: xy.x:6!null - │ │ │ │ └── key: (6) + │ │ │ │ ├── columns: xy.x:7!null + │ │ │ │ └── key: (7) │ │ │ └── filters - │ │ │ └── xy.x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ │ │ └── xy.x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] │ │ ├── scan xy - │ │ │ └── columns: xy.y:15 + │ │ │ └── columns: xy.y:20 │ │ └── filters - │ │ └── xy.y:15 = k:1 [outer=(1,15), constraints=(/1: (/NULL - ]; /15: (/NULL - ]), fd=(1)==(15), (15)==(1)] + │ │ └── xy.y:20 = k:1 [outer=(1,20), constraints=(/1: (/NULL - ]; /20: (/NULL - ]), fd=(1)==(20), (20)==(1)] │ └── aggregations - │ ├── count [as=count_rows:16, outer=(15)] - │ │ └── xy.y:15 - │ └── const-agg [as=xy.x:6, outer=(6)] - │ └── xy.x:6 + │ ├── count [as=count_rows:22, outer=(20)] + │ │ └── xy.y:20 + │ └── const-agg [as=xy.x:7, outer=(7)] + │ └── xy.x:7 └── projections - ├── 5 [as=a:17] - ├── xy.x:6 [as=x:18, outer=(6)] - ├── subquery [as=y:19, subquery] + ├── 5 [as=a:23] + ├── xy.x:7 [as=x:24, outer=(7)] + ├── subquery [as=y:25, subquery] │ └── limit - │ ├── columns: xy.y:9 + │ ├── columns: xy.y:11 │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(9) + │ ├── fd: ()-->(11) │ ├── scan xy - │ │ ├── columns: xy.y:9 + │ │ ├── columns: xy.y:11 │ │ └── limit hint: 1.00 │ └── 1 - ├── any: eq [as=b:20, subquery] + ├── any: eq [as=b:26, subquery] │ ├── scan xy - │ │ └── columns: xy.y:11 + │ │ └── columns: xy.y:14 │ └── 5 - ├── exists [as=exists:21, subquery] + ├── exists [as=exists:27, subquery] │ └── limit - │ ├── columns: xy.x:12!null xy.y:13 + │ ├── columns: xy.x:16!null xy.y:17 │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(12,13) + │ ├── fd: ()-->(16,17) │ ├── scan xy - │ │ ├── columns: xy.x:12!null xy.y:13 - │ │ ├── key: (12) - │ │ ├── fd: (12)-->(13) + │ │ ├── columns: xy.x:16!null xy.y:17 + │ │ ├── key: (16) + │ │ ├── fd: (16)-->(17) │ │ └── limit hint: 1.00 │ └── 1 - └── count_rows:16 [as=count:22, outer=(16)] + └── count_rows:22 [as=count:28, outer=(22)] # Subquery in GroupBy aggregate (optbuilder creates correlated Project). norm expect=HoistProjectSubquery SELECT max((SELECT y FROM xy WHERE y=i)) FROM a ---- scalar-group-by - ├── columns: max:9 + ├── columns: max:11 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(9) + ├── fd: ()-->(11) ├── project - │ ├── columns: column8:8 + │ ├── columns: column10:10 │ ├── ensure-distinct-on - │ │ ├── columns: k:1!null y:7 + │ │ ├── columns: k:1!null y:8 │ │ ├── grouping columns: k:1!null │ │ ├── error: "more than one row returned by a subquery used as an expression" │ │ ├── key: (1) - │ │ ├── fd: (1)-->(7) + │ │ ├── fd: (1)-->(8) │ │ ├── left-join (hash) - │ │ │ ├── columns: k:1!null i:2 y:7 + │ │ │ ├── columns: k:1!null i:2 y:8 │ │ │ ├── fd: (1)-->(2) │ │ │ ├── scan a │ │ │ │ ├── columns: k:1!null i:2 │ │ │ │ ├── key: (1) │ │ │ │ └── fd: (1)-->(2) │ │ │ ├── scan xy - │ │ │ │ └── columns: y:7 + │ │ │ │ └── columns: y:8 │ │ │ └── filters - │ │ │ └── y:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] + │ │ │ └── y:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] │ │ └── aggregations - │ │ └── const-agg [as=y:7, outer=(7)] - │ │ └── y:7 + │ │ └── const-agg [as=y:8, outer=(8)] + │ │ └── y:8 │ └── projections - │ └── y:7 [as=column8:8, outer=(7)] + │ └── y:8 [as=column10:10, outer=(8)] └── aggregations - └── max [as=max:9, outer=(8)] - └── column8:8 + └── max [as=max:11, outer=(10)] + └── column10:10 # Exists in projection list. norm expect=HoistProjectSubquery SELECT EXISTS(SELECT * FROM xy WHERE y=i) FROM a ---- project - ├── columns: exists:8!null + ├── columns: exists:10!null ├── group-by - │ ├── columns: k:1!null true_agg:10 + │ ├── columns: k:1!null true_agg:12 │ ├── grouping columns: k:1!null │ ├── key: (1) - │ ├── fd: (1)-->(10) + │ ├── fd: (1)-->(12) │ ├── left-join (hash) - │ │ ├── columns: k:1!null i:2 y:7 true:9 + │ │ ├── columns: k:1!null i:2 y:8 true:11 │ │ ├── fd: (1)-->(2) │ │ ├── scan a │ │ │ ├── columns: k:1!null i:2 │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2) │ │ ├── project - │ │ │ ├── columns: true:9!null y:7 - │ │ │ ├── fd: ()-->(9) + │ │ │ ├── columns: true:11!null y:8 + │ │ │ ├── fd: ()-->(11) │ │ │ ├── scan xy - │ │ │ │ └── columns: y:7 + │ │ │ │ └── columns: y:8 │ │ │ └── projections - │ │ │ └── true [as=true:9] + │ │ │ └── true [as=true:11] │ │ └── filters - │ │ └── y:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] + │ │ └── y:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] │ └── aggregations - │ └── const-not-null-agg [as=true_agg:10, outer=(9)] - │ └── true:9 + │ └── const-not-null-agg [as=true_agg:12, outer=(11)] + │ └── true:11 └── projections - └── true_agg:10 IS NOT NULL [as=exists:8, outer=(10)] + └── true_agg:12 IS NOT NULL [as=exists:10, outer=(12)] # Any in projection list. norm expect=HoistProjectSubquery SELECT 5 < ANY(SELECT y FROM xy WHERE y=i) AS r FROM a ---- project - ├── columns: r:8 + ├── columns: r:10 ├── group-by - │ ├── columns: k:1!null bool_or:10 + │ ├── columns: k:1!null bool_or:12 │ ├── grouping columns: k:1!null │ ├── key: (1) - │ ├── fd: (1)-->(10) + │ ├── fd: (1)-->(12) │ ├── left-join (hash) - │ │ ├── columns: k:1!null i:2 y:7 notnull:9 - │ │ ├── fd: (1)-->(2), (7)~~>(9) + │ │ ├── columns: k:1!null i:2 y:8 notnull:11 + │ │ ├── fd: (1)-->(2), (8)~~>(11) │ │ ├── scan a │ │ │ ├── columns: k:1!null i:2 │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2) │ │ ├── project - │ │ │ ├── columns: notnull:9!null y:7 - │ │ │ ├── fd: (7)-->(9) + │ │ │ ├── columns: notnull:11!null y:8 + │ │ │ ├── fd: (8)-->(11) │ │ │ ├── select - │ │ │ │ ├── columns: y:7 + │ │ │ │ ├── columns: y:8 │ │ │ │ ├── scan xy - │ │ │ │ │ └── columns: y:7 + │ │ │ │ │ └── columns: y:8 │ │ │ │ └── filters - │ │ │ │ └── (y:7 > 5) IS NOT false [outer=(7)] + │ │ │ │ └── (y:8 > 5) IS NOT false [outer=(8)] │ │ │ └── projections - │ │ │ └── y:7 IS NOT NULL [as=notnull:9, outer=(7)] + │ │ │ └── y:8 IS NOT NULL [as=notnull:11, outer=(8)] │ │ └── filters - │ │ └── y:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] + │ │ └── y:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] │ └── aggregations - │ └── bool-or [as=bool_or:10, outer=(9)] - │ └── notnull:9 + │ └── bool-or [as=bool_or:12, outer=(11)] + │ └── notnull:11 └── projections - └── CASE WHEN bool_or:10 THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=r:8, outer=(10)] + └── CASE WHEN bool_or:12 THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=r:10, outer=(12)] # Correlated subquery nested in uncorrelated subquery. norm expect=HoistProjectSubquery SELECT EXISTS(SELECT EXISTS(SELECT * FROM xy WHERE y=i) FROM a) ---- values - ├── columns: exists:12 + ├── columns: exists:14 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(12) + ├── fd: ()-->(14) └── tuple └── exists └── limit - ├── columns: k:1!null i:2 y:7 true:9 + ├── columns: k:1!null i:2 y:8 true:11 ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(1,2,7,9) + ├── fd: ()-->(1,2,8,11) ├── left-join (hash) - │ ├── columns: k:1!null i:2 y:7 true:9 + │ ├── columns: k:1!null i:2 y:8 true:11 │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) │ ├── fd: ()-->(1,2) │ ├── limit hint: 1.00 @@ -4443,14 +4443,14 @@ values │ │ │ └── limit hint: 1.00 │ │ └── 1 │ ├── project - │ │ ├── columns: true:9!null y:7 - │ │ ├── fd: ()-->(9) + │ │ ├── columns: true:11!null y:8 + │ │ ├── fd: ()-->(11) │ │ ├── scan xy - │ │ │ └── columns: y:7 + │ │ │ └── columns: y:8 │ │ └── projections - │ │ └── true [as=true:9] + │ │ └── true [as=true:11] │ └── filters - │ └── y:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] + │ └── y:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] └── 1 # Don't hoist uncorrelated subquery. @@ -4458,13 +4458,13 @@ norm SELECT i < ANY(SELECT y FROM xy) AS r FROM a ---- project - ├── columns: r:8 + ├── columns: r:10 ├── scan a │ └── columns: i:2 └── projections - └── any: lt [as=r:8, outer=(2), correlated-subquery] + └── any: lt [as=r:10, outer=(2), correlated-subquery] ├── scan xy - │ └── columns: y:7 + │ └── columns: y:8 └── i:2 # -------------------------------------------------- @@ -4474,151 +4474,151 @@ norm expect=HoistJoinSubquery SELECT i, y FROM a INNER JOIN xy ON (SELECT k+1) = x ---- project - ├── columns: i:2 y:7 + ├── columns: i:2 y:8 ├── immutable └── inner-join-apply - ├── columns: k:1!null i:2 x:6!null y:7 "?column?":8 + ├── columns: k:1!null i:2 x:7!null y:8 "?column?":10 ├── immutable - ├── key: (1,6) - ├── fd: (1)-->(2), (1,6)-->(7,8), (6)==(8), (8)==(6) + ├── key: (1,7) + ├── fd: (1)-->(2), (1,7)-->(8,10), (7)==(10), (10)==(7) ├── scan a │ ├── columns: k:1!null i:2 │ ├── key: (1) │ └── fd: (1)-->(2) ├── inner-join (cross) - │ ├── columns: x:6!null y:7 "?column?":8 + │ ├── columns: x:7!null y:8 "?column?":10 │ ├── outer: (1) │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) │ ├── immutable - │ ├── key: (6) - │ ├── fd: ()-->(8), (6)-->(7) + │ ├── key: (7) + │ ├── fd: ()-->(10), (7)-->(8) │ ├── scan xy - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ ├── values - │ │ ├── columns: "?column?":8 + │ │ ├── columns: "?column?":10 │ │ ├── outer: (1) │ │ ├── cardinality: [1 - 1] │ │ ├── immutable │ │ ├── key: () - │ │ ├── fd: ()-->(8) + │ │ ├── fd: ()-->(10) │ │ └── (k:1 + 1,) │ └── filters (true) └── filters - └── x:6 = "?column?":8 [outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ]), fd=(6)==(8), (8)==(6)] + └── x:7 = "?column?":10 [outer=(7,10), constraints=(/7: (/NULL - ]; /10: (/NULL - ]), fd=(7)==(10), (10)==(7)] # Hoist Exists in join filter disjunction. norm expect=HoistJoinSubquery SELECT s, x FROM a INNER JOIN xy ON EXISTS(SELECT * FROM uv WHERE u=y) OR k=x ---- project - ├── columns: s:4 x:6!null + ├── columns: s:4 x:7!null └── inner-join (cross) - ├── columns: k:1!null s:4 x:6!null exists:12!null - ├── key: (1,6) - ├── fd: (1)-->(4), (6)-->(12) + ├── columns: k:1!null s:4 x:7!null exists:15!null + ├── key: (1,7) + ├── fd: (1)-->(4), (7)-->(15) ├── scan a │ ├── columns: k:1!null s:4 │ ├── key: (1) │ └── fd: (1)-->(4) ├── project - │ ├── columns: exists:12!null x:6!null - │ ├── key: (6) - │ ├── fd: (6)-->(12) + │ ├── columns: exists:15!null x:7!null + │ ├── key: (7) + │ ├── fd: (7)-->(15) │ ├── group-by - │ │ ├── columns: x:6!null true_agg:11 - │ │ ├── grouping columns: x:6!null - │ │ ├── key: (6) - │ │ ├── fd: (6)-->(11) + │ │ ├── columns: x:7!null true_agg:14 + │ │ ├── grouping columns: x:7!null + │ │ ├── key: (7) + │ │ ├── fd: (7)-->(14) │ │ ├── left-join (hash) - │ │ │ ├── columns: x:6!null y:7 u:8 true:10 + │ │ │ ├── columns: x:7!null y:8 u:10 true:13 │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ │ │ ├── key: (6) - │ │ │ ├── fd: (6)-->(7,8,10) + │ │ │ ├── key: (7) + │ │ │ ├── fd: (7)-->(8,10,13) │ │ │ ├── scan xy - │ │ │ │ ├── columns: x:6!null y:7 - │ │ │ │ ├── key: (6) - │ │ │ │ └── fd: (6)-->(7) + │ │ │ │ ├── columns: x:7!null y:8 + │ │ │ │ ├── key: (7) + │ │ │ │ └── fd: (7)-->(8) │ │ │ ├── project - │ │ │ │ ├── columns: true:10!null u:8!null - │ │ │ │ ├── key: (8) - │ │ │ │ ├── fd: ()-->(10) + │ │ │ │ ├── columns: true:13!null u:10!null + │ │ │ │ ├── key: (10) + │ │ │ │ ├── fd: ()-->(13) │ │ │ │ ├── scan uv - │ │ │ │ │ ├── columns: u:8!null - │ │ │ │ │ └── key: (8) + │ │ │ │ │ ├── columns: u:10!null + │ │ │ │ │ └── key: (10) │ │ │ │ └── projections - │ │ │ │ └── true [as=true:10] + │ │ │ │ └── true [as=true:13] │ │ │ └── filters - │ │ │ └── u:8 = y:7 [outer=(7,8), constraints=(/7: (/NULL - ]; /8: (/NULL - ]), fd=(7)==(8), (8)==(7)] + │ │ │ └── u:10 = y:8 [outer=(8,10), constraints=(/8: (/NULL - ]; /10: (/NULL - ]), fd=(8)==(10), (10)==(8)] │ │ └── aggregations - │ │ └── const-not-null-agg [as=true_agg:11, outer=(10)] - │ │ └── true:10 + │ │ └── const-not-null-agg [as=true_agg:14, outer=(13)] + │ │ └── true:13 │ └── projections - │ └── true_agg:11 IS NOT NULL [as=exists:12, outer=(11)] + │ └── true_agg:14 IS NOT NULL [as=exists:15, outer=(14)] └── filters - └── exists:12 OR (k:1 = x:6) [outer=(1,6,12)] + └── exists:15 OR (k:1 = x:7) [outer=(1,7,15)] # Any in Join filter disjunction. norm expect=HoistJoinSubquery SELECT j, y FROM a INNER JOIN xy ON x IN (SELECT v FROM uv WHERE u=y AND v=i) OR x IS NULL ---- project - ├── columns: j:5 y:7 + ├── columns: j:5 y:8 └── select - ├── columns: j:5 x:6!null y:7 case:12 - ├── fd: (6)-->(7) + ├── columns: j:5 x:7!null y:8 case:15 + ├── fd: (7)-->(8) ├── project - │ ├── columns: case:12 j:5 x:6!null y:7 - │ ├── fd: (6)-->(7) + │ ├── columns: case:15 j:5 x:7!null y:8 + │ ├── fd: (7)-->(8) │ ├── group-by - │ │ ├── columns: k:1!null j:5 x:6!null y:7 bool_or:11 - │ │ ├── grouping columns: k:1!null x:6!null - │ │ ├── key: (1,6) - │ │ ├── fd: (1)-->(5), (6)-->(7), (1,6)-->(5,7,11) + │ │ ├── columns: k:1!null j:5 x:7!null y:8 bool_or:14 + │ │ ├── grouping columns: k:1!null x:7!null + │ │ ├── key: (1,7) + │ │ ├── fd: (1)-->(5), (7)-->(8), (1,7)-->(5,8,14) │ │ ├── left-join (hash) - │ │ │ ├── columns: k:1!null i:2 j:5 x:6!null y:7 u:8 v:9 notnull:10 + │ │ │ ├── columns: k:1!null i:2 j:5 x:7!null y:8 u:10 v:11 notnull:13 │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ │ │ ├── key: (1,6) - │ │ │ ├── fd: (1)-->(2,5), (6)-->(7), (8)-->(9), (9)~~>(10), (1,6)-->(8-10) + │ │ │ ├── key: (1,7) + │ │ │ ├── fd: (1)-->(2,5), (7)-->(8), (10)-->(11), (11)~~>(13), (1,7)-->(10,11,13) │ │ │ ├── inner-join (cross) - │ │ │ │ ├── columns: k:1!null i:2 j:5 x:6!null y:7 - │ │ │ │ ├── key: (1,6) - │ │ │ │ ├── fd: (1)-->(2,5), (6)-->(7) + │ │ │ │ ├── columns: k:1!null i:2 j:5 x:7!null y:8 + │ │ │ │ ├── key: (1,7) + │ │ │ │ ├── fd: (1)-->(2,5), (7)-->(8) │ │ │ │ ├── scan a │ │ │ │ │ ├── columns: k:1!null i:2 j:5 │ │ │ │ │ ├── key: (1) │ │ │ │ │ └── fd: (1)-->(2,5) │ │ │ │ ├── scan xy - │ │ │ │ │ ├── columns: x:6!null y:7 - │ │ │ │ │ ├── key: (6) - │ │ │ │ │ └── fd: (6)-->(7) + │ │ │ │ │ ├── columns: x:7!null y:8 + │ │ │ │ │ ├── key: (7) + │ │ │ │ │ └── fd: (7)-->(8) │ │ │ │ └── filters (true) │ │ │ ├── project - │ │ │ │ ├── columns: notnull:10!null u:8!null v:9 - │ │ │ │ ├── key: (8) - │ │ │ │ ├── fd: (8)-->(9), (9)-->(10) + │ │ │ │ ├── columns: notnull:13!null u:10!null v:11 + │ │ │ │ ├── key: (10) + │ │ │ │ ├── fd: (10)-->(11), (11)-->(13) │ │ │ │ ├── scan uv - │ │ │ │ │ ├── columns: u:8!null v:9 - │ │ │ │ │ ├── key: (8) - │ │ │ │ │ └── fd: (8)-->(9) + │ │ │ │ │ ├── columns: u:10!null v:11 + │ │ │ │ │ ├── key: (10) + │ │ │ │ │ └── fd: (10)-->(11) │ │ │ │ └── projections - │ │ │ │ └── v:9 IS NOT NULL [as=notnull:10, outer=(9)] + │ │ │ │ └── v:11 IS NOT NULL [as=notnull:13, outer=(11)] │ │ │ └── filters - │ │ │ ├── u:8 = y:7 [outer=(7,8), constraints=(/7: (/NULL - ]; /8: (/NULL - ]), fd=(7)==(8), (8)==(7)] - │ │ │ ├── v:9 = i:2 [outer=(2,9), constraints=(/2: (/NULL - ]; /9: (/NULL - ]), fd=(2)==(9), (9)==(2)] - │ │ │ └── (x:6 = v:9) IS NOT false [outer=(6,9)] + │ │ │ ├── u:10 = y:8 [outer=(8,10), constraints=(/8: (/NULL - ]; /10: (/NULL - ]), fd=(8)==(10), (10)==(8)] + │ │ │ ├── v:11 = i:2 [outer=(2,11), constraints=(/2: (/NULL - ]; /11: (/NULL - ]), fd=(2)==(11), (11)==(2)] + │ │ │ └── (x:7 = v:11) IS NOT false [outer=(7,11)] │ │ └── aggregations - │ │ ├── bool-or [as=bool_or:11, outer=(10)] - │ │ │ └── notnull:10 - │ │ ├── const-agg [as=y:7, outer=(7)] - │ │ │ └── y:7 + │ │ ├── bool-or [as=bool_or:14, outer=(13)] + │ │ │ └── notnull:13 + │ │ ├── const-agg [as=y:8, outer=(8)] + │ │ │ └── y:8 │ │ └── const-agg [as=j:5, outer=(5)] │ │ └── j:5 │ └── projections - │ └── CASE WHEN bool_or:11 AND (x:6 IS NOT NULL) THEN true WHEN bool_or:11 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:12, outer=(6,11)] + │ └── CASE WHEN bool_or:14 AND (x:7 IS NOT NULL) THEN true WHEN bool_or:14 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:15, outer=(7,14)] └── filters - └── case:12 OR (x:6 IS NULL) [outer=(6,12)] + └── case:15 OR (x:7 IS NULL) [outer=(7,15)] # -------------------------------------------------- @@ -4628,17 +4628,17 @@ norm expect=HoistValuesSubquery SELECT (VALUES ((SELECT i+1 AS r)), (10), ((SELECT k+1 AS s))) FROM a ---- project - ├── columns: column1:9 + ├── columns: column1:10 ├── immutable ├── ensure-distinct-on - │ ├── columns: k:1!null column1:8 + │ ├── columns: k:1!null column1:9 │ ├── grouping columns: k:1!null │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(8) + │ ├── fd: (1)-->(9) │ ├── inner-join-apply - │ │ ├── columns: k:1!null i:2 r:6 s:7 column1:8 + │ │ ├── columns: k:1!null i:2 r:7 s:8 column1:9 │ │ ├── immutable │ │ ├── fd: (1)-->(2) │ │ ├── scan a @@ -4646,210 +4646,210 @@ project │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2) │ │ ├── inner-join-apply - │ │ │ ├── columns: r:6 s:7 column1:8 + │ │ │ ├── columns: r:7 s:8 column1:9 │ │ │ ├── outer: (1,2) │ │ │ ├── cardinality: [3 - 3] │ │ │ ├── immutable - │ │ │ ├── fd: ()-->(6,7) + │ │ │ ├── fd: ()-->(7,8) │ │ │ ├── inner-join (cross) - │ │ │ │ ├── columns: r:6 s:7 + │ │ │ │ ├── columns: r:7 s:8 │ │ │ │ ├── outer: (1,2) │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) │ │ │ │ ├── immutable │ │ │ │ ├── key: () - │ │ │ │ ├── fd: ()-->(6,7) + │ │ │ │ ├── fd: ()-->(7,8) │ │ │ │ ├── values - │ │ │ │ │ ├── columns: r:6 + │ │ │ │ │ ├── columns: r:7 │ │ │ │ │ ├── outer: (2) │ │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ │ ├── immutable │ │ │ │ │ ├── key: () - │ │ │ │ │ ├── fd: ()-->(6) + │ │ │ │ │ ├── fd: ()-->(7) │ │ │ │ │ └── (i:2 + 1,) │ │ │ │ ├── values - │ │ │ │ │ ├── columns: s:7 + │ │ │ │ │ ├── columns: s:8 │ │ │ │ │ ├── outer: (1) │ │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ │ ├── immutable │ │ │ │ │ ├── key: () - │ │ │ │ │ ├── fd: ()-->(7) + │ │ │ │ │ ├── fd: ()-->(8) │ │ │ │ │ └── (k:1 + 1,) │ │ │ │ └── filters (true) │ │ │ ├── values - │ │ │ │ ├── columns: column1:8 - │ │ │ │ ├── outer: (6,7) + │ │ │ │ ├── columns: column1:9 + │ │ │ │ ├── outer: (7,8) │ │ │ │ ├── cardinality: [3 - 3] - │ │ │ │ ├── (r:6,) + │ │ │ │ ├── (r:7,) │ │ │ │ ├── (10,) - │ │ │ │ └── (s:7,) + │ │ │ │ └── (s:8,) │ │ │ └── filters (true) │ │ └── filters (true) │ └── aggregations - │ └── const-agg [as=column1:8, outer=(8)] - │ └── column1:8 + │ └── const-agg [as=column1:9, outer=(9)] + │ └── column1:9 └── projections - └── column1:8 [as=column1:9, outer=(8)] + └── column1:9 [as=column1:10, outer=(9)] # Exists in values row. norm expect=HoistValuesSubquery SELECT (VALUES (EXISTS(SELECT * FROM xy WHERE x=k))) FROM a ---- project - ├── columns: column1:12 + ├── columns: column1:14 ├── inner-join-apply - │ ├── columns: k:1!null column1:8 exists:11!null + │ ├── columns: k:1!null column1:10 exists:13!null │ ├── key: (1) - │ ├── fd: (1)-->(8,11) + │ ├── fd: (1)-->(10,13) │ ├── scan a │ │ ├── columns: k:1!null │ │ └── key: (1) │ ├── inner-join-apply - │ │ ├── columns: column1:8 exists:11!null + │ │ ├── columns: column1:10 exists:13!null │ │ ├── outer: (1) │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(8,11) + │ │ ├── fd: ()-->(10,13) │ │ ├── project - │ │ │ ├── columns: exists:11!null + │ │ │ ├── columns: exists:13!null │ │ │ ├── outer: (1) │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(11) + │ │ │ ├── fd: ()-->(13) │ │ │ ├── group-by - │ │ │ │ ├── columns: true_agg:10 + │ │ │ │ ├── columns: true_agg:12 │ │ │ │ ├── outer: (1) │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── key: () - │ │ │ │ ├── fd: ()-->(10) + │ │ │ │ ├── fd: ()-->(12) │ │ │ │ ├── left-join (cross) - │ │ │ │ │ ├── columns: true:9 + │ │ │ │ │ ├── columns: true:11 │ │ │ │ │ ├── outer: (1) │ │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) │ │ │ │ │ ├── key: () - │ │ │ │ │ ├── fd: ()-->(9) + │ │ │ │ │ ├── fd: ()-->(11) │ │ │ │ │ ├── values │ │ │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ │ │ ├── key: () │ │ │ │ │ │ └── () │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: true:9!null + │ │ │ │ │ │ ├── columns: true:11!null │ │ │ │ │ │ ├── outer: (1) │ │ │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ │ │ ├── key: () - │ │ │ │ │ │ ├── fd: ()-->(9) + │ │ │ │ │ │ ├── fd: ()-->(11) │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: x:6!null + │ │ │ │ │ │ │ ├── columns: x:7!null │ │ │ │ │ │ │ ├── outer: (1) │ │ │ │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ │ │ │ ├── key: () - │ │ │ │ │ │ │ ├── fd: ()-->(6) + │ │ │ │ │ │ │ ├── fd: ()-->(7) │ │ │ │ │ │ │ ├── scan xy - │ │ │ │ │ │ │ │ ├── columns: x:6!null - │ │ │ │ │ │ │ │ └── key: (6) + │ │ │ │ │ │ │ │ ├── columns: x:7!null + │ │ │ │ │ │ │ │ └── key: (7) │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ │ │ │ │ │ │ └── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── true [as=true:9] + │ │ │ │ │ │ └── true [as=true:11] │ │ │ │ │ └── filters (true) │ │ │ │ └── aggregations - │ │ │ │ └── const-not-null-agg [as=true_agg:10, outer=(9)] - │ │ │ │ └── true:9 + │ │ │ │ └── const-not-null-agg [as=true_agg:12, outer=(11)] + │ │ │ │ └── true:11 │ │ │ └── projections - │ │ │ └── true_agg:10 IS NOT NULL [as=exists:11, outer=(10)] + │ │ │ └── true_agg:12 IS NOT NULL [as=exists:13, outer=(12)] │ │ ├── values - │ │ │ ├── columns: column1:8 - │ │ │ ├── outer: (11) + │ │ │ ├── columns: column1:10 + │ │ │ ├── outer: (13) │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(8) - │ │ │ └── (exists:11,) + │ │ │ ├── fd: ()-->(10) + │ │ │ └── (exists:13,) │ │ └── filters (true) │ └── filters (true) └── projections - └── column1:8 [as=column1:12, outer=(8)] + └── column1:10 [as=column1:14, outer=(10)] # Any in values row. norm expect=HoistValuesSubquery SELECT (VALUES (5 IN (SELECT y FROM xy WHERE x=k))) FROM a ---- project - ├── columns: column1:12 + ├── columns: column1:14 ├── inner-join-apply - │ ├── columns: k:1!null column1:8 case:11 + │ ├── columns: k:1!null column1:10 case:13 │ ├── key: (1) - │ ├── fd: (1)-->(8,11) + │ ├── fd: (1)-->(10,13) │ ├── scan a │ │ ├── columns: k:1!null │ │ └── key: (1) │ ├── inner-join-apply - │ │ ├── columns: column1:8 case:11 + │ │ ├── columns: column1:10 case:13 │ │ ├── outer: (1) │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(8,11) + │ │ ├── fd: ()-->(10,13) │ │ ├── project - │ │ │ ├── columns: case:11 + │ │ │ ├── columns: case:13 │ │ │ ├── outer: (1) │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(11) + │ │ │ ├── fd: ()-->(13) │ │ │ ├── group-by - │ │ │ │ ├── columns: bool_or:10 + │ │ │ │ ├── columns: bool_or:12 │ │ │ │ ├── outer: (1) │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── key: () - │ │ │ │ ├── fd: ()-->(10) + │ │ │ │ ├── fd: ()-->(12) │ │ │ │ ├── left-join (cross) - │ │ │ │ │ ├── columns: notnull:9 + │ │ │ │ │ ├── columns: notnull:11 │ │ │ │ │ ├── outer: (1) │ │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) │ │ │ │ │ ├── key: () - │ │ │ │ │ ├── fd: ()-->(9) + │ │ │ │ │ ├── fd: ()-->(11) │ │ │ │ │ ├── values │ │ │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ │ │ ├── key: () │ │ │ │ │ │ └── () │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: notnull:9!null + │ │ │ │ │ │ ├── columns: notnull:11!null │ │ │ │ │ │ ├── outer: (1) │ │ │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ │ │ ├── key: () - │ │ │ │ │ │ ├── fd: ()-->(9) + │ │ │ │ │ │ ├── fd: ()-->(11) │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: x:6!null y:7 + │ │ │ │ │ │ │ ├── columns: x:7!null y:8 │ │ │ │ │ │ │ ├── outer: (1) │ │ │ │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ │ │ │ ├── key: () - │ │ │ │ │ │ │ ├── fd: ()-->(6,7) + │ │ │ │ │ │ │ ├── fd: ()-->(7,8) │ │ │ │ │ │ │ ├── scan xy - │ │ │ │ │ │ │ │ ├── columns: x:6!null y:7 - │ │ │ │ │ │ │ │ ├── key: (6) - │ │ │ │ │ │ │ │ └── fd: (6)-->(7) + │ │ │ │ │ │ │ │ ├── columns: x:7!null y:8 + │ │ │ │ │ │ │ │ ├── key: (7) + │ │ │ │ │ │ │ │ └── fd: (7)-->(8) │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ ├── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - │ │ │ │ │ │ │ └── (y:7 = 5) IS NOT false [outer=(7)] + │ │ │ │ │ │ │ ├── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + │ │ │ │ │ │ │ └── (y:8 = 5) IS NOT false [outer=(8)] │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── y:7 IS NOT NULL [as=notnull:9, outer=(7)] + │ │ │ │ │ │ └── y:8 IS NOT NULL [as=notnull:11, outer=(8)] │ │ │ │ │ └── filters (true) │ │ │ │ └── aggregations - │ │ │ │ └── bool-or [as=bool_or:10, outer=(9)] - │ │ │ │ └── notnull:9 + │ │ │ │ └── bool-or [as=bool_or:12, outer=(11)] + │ │ │ │ └── notnull:11 │ │ │ └── projections - │ │ │ └── CASE WHEN bool_or:10 THEN true WHEN bool_or:10 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:11, outer=(10)] + │ │ │ └── CASE WHEN bool_or:12 THEN true WHEN bool_or:12 IS NULL THEN false ELSE CAST(NULL AS BOOL) END [as=case:13, outer=(12)] │ │ ├── values - │ │ │ ├── columns: column1:8 - │ │ │ ├── outer: (11) + │ │ │ ├── columns: column1:10 + │ │ │ ├── outer: (13) │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(8) - │ │ │ └── (case:11,) + │ │ │ ├── fd: ()-->(10) + │ │ │ └── (case:13,) │ │ └── filters (true) │ └── filters (true) └── projections - └── column1:8 [as=column1:12, outer=(8)] + └── column1:10 [as=column1:14, outer=(10)] # --------------------------------------------------- # HoistProjectSetSubquery + TryDecorrelateProjectSet @@ -4858,29 +4858,29 @@ norm expect=HoistProjectSetSubquery SELECT generate_series(1, (SELECT v FROM uv WHERE u=x)) FROM xy ---- project - ├── columns: generate_series:5 + ├── columns: generate_series:7 ├── immutable └── project-set - ├── columns: v:4 generate_series:5 + ├── columns: v:5 generate_series:7 ├── immutable ├── project - │ ├── columns: v:4 + │ ├── columns: v:5 │ └── left-join (hash) - │ ├── columns: x:1!null u:3 v:4 + │ ├── columns: x:1!null u:4 v:5 │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ ├── key: (1) - │ ├── fd: (3)-->(4), (1)-->(3,4) + │ ├── fd: (4)-->(5), (1)-->(4,5) │ ├── scan xy │ │ ├── columns: x:1!null │ │ └── key: (1) │ ├── scan uv - │ │ ├── columns: u:3!null v:4 - │ │ ├── key: (3) - │ │ └── fd: (3)-->(4) + │ │ ├── columns: u:4!null v:5 + │ │ ├── key: (4) + │ │ └── fd: (4)-->(5) │ └── filters - │ └── u:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + │ └── u:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] └── zip - └── generate_series(1, v:4) [outer=(4), immutable] + └── generate_series(1, v:5) [outer=(5), immutable] # Zip correlation within EXISTS. norm expect=(HoistProjectSetSubquery,TryDecorrelateSemiJoin,TryDecorrelateProjectSet) @@ -4893,30 +4893,30 @@ group-by ├── key: (1) ├── fd: (1)-->(2) ├── project-set - │ ├── columns: x:1!null y:2 v:4 generate_series:5 + │ ├── columns: x:1!null y:2 v:5 generate_series:7 │ ├── immutable - │ ├── fd: (1)-->(2,4) + │ ├── fd: (1)-->(2,5) │ ├── project - │ │ ├── columns: x:1!null y:2 v:4 + │ │ ├── columns: x:1!null y:2 v:5 │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2,4) + │ │ ├── fd: (1)-->(2,5) │ │ └── left-join (hash) - │ │ ├── columns: x:1!null y:2 u:3 v:4 + │ │ ├── columns: x:1!null y:2 u:4 v:5 │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2-4), (3)-->(4) + │ │ ├── fd: (1)-->(2,4,5), (4)-->(5) │ │ ├── scan xy │ │ │ ├── columns: x:1!null y:2 │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2) │ │ ├── scan uv - │ │ │ ├── columns: u:3!null v:4 - │ │ │ ├── key: (3) - │ │ │ └── fd: (3)-->(4) + │ │ │ ├── columns: u:4!null v:5 + │ │ │ ├── key: (4) + │ │ │ └── fd: (4)-->(5) │ │ └── filters - │ │ └── u:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + │ │ └── u:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] │ └── zip - │ └── generate_series(1, v:4) [outer=(4), immutable] + │ └── generate_series(1, v:5) [outer=(5), immutable] └── aggregations └── const-agg [as=y:2, outer=(2)] └── y:2 @@ -4926,40 +4926,40 @@ norm expect=HoistProjectSetSubquery SELECT generate_series((select y FROM xy WHERE x=k), (SELECT v FROM uv WHERE u=k)) FROM a ---- project - ├── columns: generate_series:10 + ├── columns: generate_series:13 ├── immutable └── project-set - ├── columns: y:7 v:9 generate_series:10 + ├── columns: y:8 v:11 generate_series:13 ├── immutable ├── project - │ ├── columns: y:7 v:9 + │ ├── columns: y:8 v:11 │ └── left-join (hash) - │ ├── columns: k:1!null x:6 y:7 u:8 v:9 + │ ├── columns: k:1!null x:7 y:8 u:10 v:11 │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ ├── key: (1) - │ ├── fd: (6)-->(7), (1)-->(6-9), (8)-->(9) + │ ├── fd: (7)-->(8), (1)-->(7,8,10,11), (10)-->(11) │ ├── left-join (hash) - │ │ ├── columns: k:1!null x:6 y:7 + │ │ ├── columns: k:1!null x:7 y:8 │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ ├── key: (1) - │ │ ├── fd: (6)-->(7), (1)-->(6,7) + │ │ ├── fd: (7)-->(8), (1)-->(7,8) │ │ ├── scan a │ │ │ ├── columns: k:1!null │ │ │ └── key: (1) │ │ ├── scan xy - │ │ │ ├── columns: x:6!null y:7 - │ │ │ ├── key: (6) - │ │ │ └── fd: (6)-->(7) + │ │ │ ├── columns: x:7!null y:8 + │ │ │ ├── key: (7) + │ │ │ └── fd: (7)-->(8) │ │ └── filters - │ │ └── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ │ └── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] │ ├── scan uv - │ │ ├── columns: u:8!null v:9 - │ │ ├── key: (8) - │ │ └── fd: (8)-->(9) + │ │ ├── columns: u:10!null v:11 + │ │ ├── key: (10) + │ │ └── fd: (10)-->(11) │ └── filters - │ └── u:8 = k:1 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] + │ └── u:10 = k:1 [outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] └── zip - └── generate_series(y:7, v:9) [outer=(7,9), immutable] + └── generate_series(y:8, v:11) [outer=(8,11), immutable] # Multiple functions. norm expect=HoistProjectSetSubquery @@ -4969,42 +4969,42 @@ SELECT FROM a ---- project - ├── columns: generate_series:8 information_schema._pg_expandarray:13 + ├── columns: generate_series:10 information_schema._pg_expandarray:16 ├── immutable ├── project-set - │ ├── columns: v:7 generate_series:8 xy.x:9 x:11 n:12 + │ ├── columns: v:8 generate_series:10 xy.x:11 x:14 n:15 │ ├── immutable │ ├── project - │ │ ├── columns: v:7 xy.x:9 + │ │ ├── columns: v:8 xy.x:11 │ │ └── left-join (hash) - │ │ ├── columns: k:1!null u:6 v:7 xy.x:9 + │ │ ├── columns: k:1!null u:7 v:8 xy.x:11 │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ ├── key: (1) - │ │ ├── fd: (6)-->(7), (1)-->(6,7,9) + │ │ ├── fd: (7)-->(8), (1)-->(7,8,11) │ │ ├── left-join (hash) - │ │ │ ├── columns: k:1!null u:6 v:7 + │ │ │ ├── columns: k:1!null u:7 v:8 │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ │ ├── key: (1) - │ │ │ ├── fd: (6)-->(7), (1)-->(6,7) + │ │ │ ├── fd: (7)-->(8), (1)-->(7,8) │ │ │ ├── scan a │ │ │ │ ├── columns: k:1!null │ │ │ │ └── key: (1) │ │ │ ├── scan uv - │ │ │ │ ├── columns: u:6!null v:7 - │ │ │ │ ├── key: (6) - │ │ │ │ └── fd: (6)-->(7) + │ │ │ │ ├── columns: u:7!null v:8 + │ │ │ │ ├── key: (7) + │ │ │ │ └── fd: (7)-->(8) │ │ │ └── filters - │ │ │ └── u:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ │ │ └── u:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] │ │ ├── scan xy - │ │ │ ├── columns: xy.x:9!null - │ │ │ └── key: (9) + │ │ │ ├── columns: xy.x:11!null + │ │ │ └── key: (11) │ │ └── filters - │ │ └── xy.x:9 = k:1 [outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ]), fd=(1)==(9), (9)==(1)] + │ │ └── xy.x:11 = k:1 [outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ]), fd=(1)==(11), (11)==(1)] │ └── zip - │ ├── generate_series(1, v:7) [outer=(7), immutable] - │ └── information_schema._pg_expandarray(ARRAY[xy.x:9]) [outer=(9), immutable] + │ ├── generate_series(1, v:8) [outer=(8), immutable] + │ └── information_schema._pg_expandarray(ARRAY[xy.x:11]) [outer=(11), immutable] └── projections - └── ((x:11, n:12) AS x, n) [as=information_schema._pg_expandarray:13, outer=(11,12)] + └── ((x:14, n:15) AS x, n) [as=information_schema._pg_expandarray:16, outer=(14,15)] norm expect=HoistProjectSetSubquery SELECT a, generate_series(1, (SELECT a)) FROM (VALUES (1)) AS v (a) @@ -5131,11 +5131,11 @@ limit │ ├── key: (1) │ ├── fd: (1)-->(2-9) │ ├── select - │ │ ├── columns: id:1!null body:2 description:3 title:4 slug:5 tag_list:6 user_id:7 created_at:8 updated_at:9 unnest:10!null + │ │ ├── columns: id:1!null body:2 description:3 title:4 slug:5 tag_list:6 user_id:7 created_at:8 updated_at:9 unnest:11!null │ │ ├── immutable - │ │ ├── fd: ()-->(10), (1)-->(2-9) + │ │ ├── fd: ()-->(11), (1)-->(2-9) │ │ ├── project-set - │ │ │ ├── columns: id:1!null body:2 description:3 title:4 slug:5 tag_list:6 user_id:7 created_at:8 updated_at:9 unnest:10 + │ │ │ ├── columns: id:1!null body:2 description:3 title:4 slug:5 tag_list:6 user_id:7 created_at:8 updated_at:9 unnest:11 │ │ │ ├── immutable │ │ │ ├── fd: (1)-->(2-9) │ │ │ ├── scan a0 @@ -5145,7 +5145,7 @@ limit │ │ │ └── zip │ │ │ └── unnest(tag_list:6) [outer=(6), immutable] │ │ └── filters - │ │ └── unnest:10 = 'dragons' [outer=(10), constraints=(/10: [/'dragons' - /'dragons']; tight), fd=()-->(10)] + │ │ └── unnest:11 = 'dragons' [outer=(11), constraints=(/11: [/'dragons' - /'dragons']; tight), fd=()-->(11)] │ └── aggregations │ ├── const-agg [as=body:2, outer=(2)] │ │ └── body:2 @@ -5172,65 +5172,65 @@ SELECT * FROM articles, xy WHERE EXISTS( ) ---- project - ├── columns: id:1!null body:2 description:3 title:4 slug:5 tag_list:6 user_id:7 created_at:8 updated_at:9 x:10!null y:11 + ├── columns: id:1!null body:2 description:3 title:4 slug:5 tag_list:6 user_id:7 created_at:8 updated_at:9 x:11!null y:12 ├── immutable - ├── key: (1,10) - ├── fd: (1)-->(2-9), (1,10)-->(2-9,11) + ├── key: (1,11) + ├── fd: (1)-->(2-9), (1,11)-->(2-9,12) └── select - ├── columns: id:1!null body:2 description:3 title:4 slug:5 tag_list:6 user_id:7 created_at:8 updated_at:9 x:10!null y:11 true_agg:17!null + ├── columns: id:1!null body:2 description:3 title:4 slug:5 tag_list:6 user_id:7 created_at:8 updated_at:9 x:11!null y:12 true_agg:19!null ├── immutable - ├── key: (1,10) - ├── fd: (1)-->(2-9), (1,10)-->(2-9,11,17) + ├── key: (1,11) + ├── fd: (1)-->(2-9), (1,11)-->(2-9,12,19) ├── group-by - │ ├── columns: id:1!null body:2 description:3 title:4 slug:5 tag_list:6 user_id:7 created_at:8 updated_at:9 x:10!null y:11 true_agg:17 - │ ├── grouping columns: id:1!null x:10!null + │ ├── columns: id:1!null body:2 description:3 title:4 slug:5 tag_list:6 user_id:7 created_at:8 updated_at:9 x:11!null y:12 true_agg:19 + │ ├── grouping columns: id:1!null x:11!null │ ├── immutable - │ ├── key: (1,10) - │ ├── fd: (1)-->(2-9), (1,10)-->(2-9,11,17) + │ ├── key: (1,11) + │ ├── fd: (1)-->(2-9), (1,11)-->(2-9,12,19) │ ├── inner-join-apply - │ │ ├── columns: id:1!null body:2 description:3 title:4 slug:5 tag_list:6 user_id:7 created_at:8 updated_at:9 x:10!null y:11 true:16 + │ │ ├── columns: id:1!null body:2 description:3 title:4 slug:5 tag_list:6 user_id:7 created_at:8 updated_at:9 x:11!null y:12 true:18 │ │ ├── immutable - │ │ ├── fd: (1)-->(2-9), (1,10)-->(11) + │ │ ├── fd: (1)-->(2-9), (1,11)-->(12) │ │ ├── scan articles │ │ │ ├── columns: id:1!null body:2 description:3 title:4 slug:5 tag_list:6 user_id:7 created_at:8 updated_at:9 │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2-9) │ │ ├── left-join-apply - │ │ │ ├── columns: x:10!null y:11 true:16 + │ │ │ ├── columns: x:11!null y:12 true:18 │ │ │ ├── outer: (1,4,6) │ │ │ ├── immutable - │ │ │ ├── fd: (10)-->(11) + │ │ │ ├── fd: (11)-->(12) │ │ │ ├── scan xy - │ │ │ │ ├── columns: x:10!null y:11 - │ │ │ │ ├── key: (10) - │ │ │ │ └── fd: (10)-->(11) + │ │ │ │ ├── columns: x:11!null y:12 + │ │ │ │ ├── key: (11) + │ │ │ │ └── fd: (11)-->(12) │ │ │ ├── project - │ │ │ │ ├── columns: true:16!null - │ │ │ │ ├── outer: (1,4,6,10) + │ │ │ │ ├── columns: true:18!null + │ │ │ │ ├── outer: (1,4,6,11) │ │ │ │ ├── immutable - │ │ │ │ ├── fd: ()-->(16) + │ │ │ │ ├── fd: ()-->(18) │ │ │ │ ├── project-set - │ │ │ │ │ ├── columns: generate_series:12 length:13 upper:14 unnest:15 - │ │ │ │ │ ├── outer: (1,4,6,10) + │ │ │ │ │ ├── columns: generate_series:14 length:15 upper:16 unnest:17 + │ │ │ │ │ ├── outer: (1,4,6,11) │ │ │ │ │ ├── immutable │ │ │ │ │ ├── values │ │ │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ │ │ ├── key: () │ │ │ │ │ │ └── () │ │ │ │ │ └── zip - │ │ │ │ │ ├── generate_series(x:10, id:1) [outer=(1,10), immutable] + │ │ │ │ │ ├── generate_series(x:11, id:1) [outer=(1,11), immutable] │ │ │ │ │ ├── length(title:4) [outer=(4), immutable] │ │ │ │ │ ├── upper(title:4) [outer=(4), immutable] │ │ │ │ │ └── unnest(tag_list:6) [outer=(6), immutable] │ │ │ │ └── projections - │ │ │ │ └── true [as=true:16] + │ │ │ │ └── true [as=true:18] │ │ │ └── filters (true) │ │ └── filters (true) │ └── aggregations - │ ├── const-not-null-agg [as=true_agg:17, outer=(16)] - │ │ └── true:16 - │ ├── const-agg [as=y:11, outer=(11)] - │ │ └── y:11 + │ ├── const-not-null-agg [as=true_agg:19, outer=(18)] + │ │ └── true:18 + │ ├── const-agg [as=y:12, outer=(12)] + │ │ └── y:12 │ ├── const-agg [as=body:2, outer=(2)] │ │ └── body:2 │ ├── const-agg [as=description:3, outer=(3)] @@ -5248,7 +5248,7 @@ project │ └── const-agg [as=updated_at:9, outer=(9)] │ └── updated_at:9 └── filters - └── true_agg:17 IS NOT NULL [outer=(17), constraints=(/17: (/NULL - ]; tight)] + └── true_agg:19 IS NOT NULL [outer=(19), constraints=(/19: (/NULL - ]; tight)] norm expect=TryDecorrelateProjectSet SELECT id FROM articles WHERE title = ANY( @@ -5261,11 +5261,11 @@ distinct-on ├── immutable ├── key: (1) └── select - ├── columns: id:1!null title:4!null tag_list:6 upper:10 unnest:11!null generate_series:12 lower:13 + ├── columns: id:1!null title:4!null tag_list:6 upper:11 unnest:12!null generate_series:13 lower:14 ├── immutable - ├── fd: (1)-->(4,6), (4)==(11), (11)==(4) + ├── fd: (1)-->(4,6), (4)==(12), (12)==(4) ├── project-set - │ ├── columns: id:1!null title:4 tag_list:6 upper:10 unnest:11 generate_series:12 lower:13 + │ ├── columns: id:1!null title:4 tag_list:6 upper:11 unnest:12 generate_series:13 lower:14 │ ├── immutable │ ├── fd: (1)-->(4,6) │ ├── scan articles @@ -5278,7 +5278,7 @@ distinct-on │ ├── generate_series(0, 1) [immutable] │ └── 'abc' └── filters - └── title:4 = unnest:11 [outer=(4,11), constraints=(/4: (/NULL - ]; /11: (/NULL - ]), fd=(4)==(11), (11)==(4)] + └── title:4 = unnest:12 [outer=(4,12), constraints=(/4: (/NULL - ]; /12: (/NULL - ]), fd=(4)==(12), (12)==(4)] # -------------------------------------------------- # NormalizeSelectAnyFilter + NormalizeJoinAnyFilter @@ -5295,9 +5295,9 @@ semi-join (hash) │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── scan xy - │ └── columns: y:7 + │ └── columns: y:8 └── filters - └── i:2 = y:7 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] + └── i:2 = y:8 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] # Any is one of several conjuncts. norm expect=NormalizeSelectAnyFilter @@ -5321,9 +5321,9 @@ semi-join (cross) │ ├── k:1 = 10 [outer=(1), constraints=(/1: [/10 - /10]; tight), fd=()-->(1)] │ └── s:4 = 'foo' [outer=(4), constraints=(/4: [/'foo' - /'foo']; tight), fd=()-->(4)] ├── scan xy - │ └── columns: y:7 + │ └── columns: y:8 └── filters - └── i:2 < y:7 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + └── i:2 < y:8 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ])] # Multiple ANY conjuncts. norm expect=NormalizeSelectAnyFilter @@ -5344,18 +5344,18 @@ semi-join (cross) │ │ ├── key: (1) │ │ └── fd: (1)-->(2-5) │ ├── project - │ │ ├── columns: y:10 + │ │ ├── columns: y:13 │ │ ├── immutable │ │ ├── scan xy - │ │ │ └── columns: xy.y:9 + │ │ │ └── columns: xy.y:11 │ │ └── projections - │ │ └── xy.y:9::STRING [as=y:10, outer=(9), immutable] + │ │ └── xy.y:11::STRING [as=y:13, outer=(11), immutable] │ └── filters - │ └── s:4 = y:10 [outer=(4,10), constraints=(/4: (/NULL - ]; /10: (/NULL - ]), fd=(4)==(10), (10)==(4)] + │ └── s:4 = y:13 [outer=(4,13), constraints=(/4: (/NULL - ]; /13: (/NULL - ]), fd=(4)==(13), (13)==(4)] ├── scan xy - │ └── columns: xy.y:7 + │ └── columns: xy.y:8 └── filters - └── i:2 < xy.y:7 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] + └── i:2 < xy.y:8 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ])] # Don't hoist uncorrelated ANY (but rewrite it to EXISTS). norm expect=NormalizeSelectAnyFilter @@ -5372,19 +5372,19 @@ select └── filters └── exists [subquery] └── limit - ├── columns: y:7!null + ├── columns: y:8!null ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(7) + ├── fd: ()-->(8) ├── select - │ ├── columns: y:7!null - │ ├── fd: ()-->(7) + │ ├── columns: y:8!null + │ ├── fd: ()-->(8) │ ├── limit hint: 1.00 │ ├── scan xy - │ │ ├── columns: y:7 + │ │ ├── columns: y:8 │ │ └── limit hint: 100.00 │ └── filters - │ └── y:7 = 5 [outer=(7), constraints=(/7: [/5 - /5]; tight), fd=()-->(7)] + │ └── y:8 = 5 [outer=(8), constraints=(/8: [/5 - /5]; tight), fd=()-->(8)] └── 1 # ANY in Join On condition. @@ -5392,10 +5392,10 @@ norm expect=NormalizeJoinAnyFilter SELECT * FROM a INNER JOIN xy ON i IN (SELECT v FROM uv) AND k=x ---- inner-join (hash) - ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6!null y:7 + ├── columns: k:1!null i:2 f:3 s:4 j:5 x:7!null y:8 ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (6) - ├── fd: (1)-->(2-5), (6)-->(7), (1)==(6), (6)==(1) + ├── key: (7) + ├── fd: (1)-->(2-5), (7)-->(8), (1)==(7), (7)==(1) ├── semi-join (hash) │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ ├── key: (1) @@ -5405,15 +5405,15 @@ inner-join (hash) │ │ ├── key: (1) │ │ └── fd: (1)-->(2-5) │ ├── scan uv - │ │ └── columns: v:9 + │ │ └── columns: v:11 │ └── filters - │ └── i:2 = v:9 [outer=(2,9), constraints=(/2: (/NULL - ]; /9: (/NULL - ]), fd=(2)==(9), (9)==(2)] + │ └── i:2 = v:11 [outer=(2,11), constraints=(/2: (/NULL - ]; /11: (/NULL - ]), fd=(2)==(11), (11)==(2)] ├── scan xy - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + │ └── fd: (7)-->(8) └── filters - └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # -------------------------------------------------- # NormalizeSelectNotAnyFilter + NormalizeJoinNotAnyFilter @@ -5430,9 +5430,9 @@ anti-join (cross) │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── scan xy - │ └── columns: y:7 + │ └── columns: y:8 └── filters - └── (i:2 = y:7) IS NOT false [outer=(2,7)] + └── (i:2 = y:8) IS NOT false [outer=(2,8)] # NOT ANY is one of several conjuncts. Note that i > ALL(...) gets mapped to # NOT i <= ANY(...) by optbuilder. @@ -5456,9 +5456,9 @@ anti-join (cross) │ └── filters │ └── (k:1 > 1) AND (k:1 < 5) [outer=(1), constraints=(/1: [/2 - /4]; tight)] ├── scan xy - │ └── columns: y:7 + │ └── columns: y:8 └── filters - └── (i:2 <= y:7) IS NOT false [outer=(2,7)] + └── (i:2 <= y:8) IS NOT false [outer=(2,8)] # Multiple NOT ANY conjuncts. norm expect=NormalizeSelectNotAnyFilter @@ -5479,18 +5479,18 @@ anti-join (cross) │ │ ├── key: (1) │ │ └── fd: (1)-->(2-5) │ ├── project - │ │ ├── columns: y:10 + │ │ ├── columns: y:13 │ │ ├── immutable │ │ ├── scan xy - │ │ │ └── columns: xy.y:9 + │ │ │ └── columns: xy.y:11 │ │ └── projections - │ │ └── xy.y:9::STRING [as=y:10, outer=(9), immutable] + │ │ └── xy.y:11::STRING [as=y:13, outer=(11), immutable] │ └── filters - │ └── (s:4 = y:10) IS NOT false [outer=(4,10)] + │ └── (s:4 = y:13) IS NOT false [outer=(4,13)] ├── scan xy - │ └── columns: xy.y:7 + │ └── columns: xy.y:8 └── filters - └── (i:2 >= xy.y:7) IS NOT false [outer=(2,7)] + └── (i:2 >= xy.y:8) IS NOT false [outer=(2,8)] # Don't hoist uncorrelated NOT ANY (but rewrite it to NOT EXISTS). norm expect=NormalizeSelectNotAnyFilter @@ -5508,18 +5508,18 @@ select └── not [subquery] └── exists └── limit - ├── columns: y:7 + ├── columns: y:8 ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(7) + ├── fd: ()-->(8) ├── select - │ ├── columns: y:7 + │ ├── columns: y:8 │ ├── limit hint: 1.00 │ ├── scan xy - │ │ ├── columns: y:7 + │ │ ├── columns: y:8 │ │ └── limit hint: 3.00 │ └── filters - │ └── (y:7 = 5) IS NOT false [outer=(7)] + │ └── (y:8 = 5) IS NOT false [outer=(8)] └── 1 # NOT ANY in Join On condition. @@ -5527,10 +5527,10 @@ norm expect=NormalizeJoinNotAnyFilter SELECT * FROM a INNER JOIN xy ON i NOT IN (SELECT v FROM uv) AND k=x ---- inner-join (hash) - ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6!null y:7 + ├── columns: k:1!null i:2 f:3 s:4 j:5 x:7!null y:8 ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (6) - ├── fd: (1)-->(2-5), (6)-->(7), (1)==(6), (6)==(1) + ├── key: (7) + ├── fd: (1)-->(2-5), (7)-->(8), (1)==(7), (7)==(1) ├── anti-join (cross) │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ ├── key: (1) @@ -5540,15 +5540,15 @@ inner-join (hash) │ │ ├── key: (1) │ │ └── fd: (1)-->(2-5) │ ├── scan uv - │ │ └── columns: v:9 + │ │ └── columns: v:11 │ └── filters - │ └── (i:2 = v:9) IS NOT false [outer=(2,9)] + │ └── (i:2 = v:11) IS NOT false [outer=(2,11)] ├── scan xy - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + │ └── fd: (7)-->(8) └── filters - └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # -------------------------------------------------- # NormalizeSelectAnyFilter + NormalizeSelectNotAnyFilter @@ -5571,18 +5571,18 @@ semi-join (hash) │ │ ├── key: (1) │ │ └── fd: (1)-->(2-5) │ ├── project - │ │ ├── columns: y:10 + │ │ ├── columns: y:13 │ │ ├── immutable │ │ ├── scan xy - │ │ │ └── columns: xy.y:9 + │ │ │ └── columns: xy.y:11 │ │ └── projections - │ │ └── xy.y:9::STRING [as=y:10, outer=(9), immutable] + │ │ └── xy.y:11::STRING [as=y:13, outer=(11), immutable] │ └── filters - │ └── (s:4 = y:10) IS NOT false [outer=(4,10)] + │ └── (s:4 = y:13) IS NOT false [outer=(4,13)] ├── scan xy - │ └── columns: xy.y:7 + │ └── columns: xy.y:8 └── filters - └── i:2 = xy.y:7 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] + └── i:2 = xy.y:8 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] # -------------------------------------------------- # EnsureKey @@ -5598,62 +5598,62 @@ norm SELECT (SELECT x FROM xy WHERE y=i LIMIT 1) FROM a ---- project - ├── columns: x:8 + ├── columns: x:10 ├── distinct-on - │ ├── columns: k:1!null xy.x:6 + │ ├── columns: k:1!null xy.x:7 │ ├── grouping columns: k:1!null │ ├── key: (1) - │ ├── fd: (1)-->(6) + │ ├── fd: (1)-->(7) │ ├── left-join (hash) - │ │ ├── columns: k:1!null i:2 xy.x:6 y:7 - │ │ ├── key: (1,6) - │ │ ├── fd: (1)-->(2), (6)-->(7) + │ │ ├── columns: k:1!null i:2 xy.x:7 y:8 + │ │ ├── key: (1,7) + │ │ ├── fd: (1)-->(2), (7)-->(8) │ │ ├── scan a │ │ │ ├── columns: k:1!null i:2 │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2) │ │ ├── scan xy - │ │ │ ├── columns: xy.x:6!null y:7 - │ │ │ ├── key: (6) - │ │ │ └── fd: (6)-->(7) + │ │ │ ├── columns: xy.x:7!null y:8 + │ │ │ ├── key: (7) + │ │ │ └── fd: (7)-->(8) │ │ └── filters - │ │ └── y:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] + │ │ └── y:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] │ └── aggregations - │ └── first-agg [as=xy.x:6, outer=(6)] - │ └── xy.x:6 + │ └── first-agg [as=xy.x:7, outer=(7)] + │ └── xy.x:7 └── projections - └── xy.x:6 [as=x:8, outer=(6)] + └── xy.x:7 [as=x:10, outer=(7)] # Case where EnsureKey retrieves an implicit key to add to the Scan. norm SELECT (SELECT x FROM xy WHERE y=b LIMIT 1) FROM ab ---- project - ├── columns: x:6 + ├── columns: x:8 ├── distinct-on - │ ├── columns: rowid:3!null xy.x:4 + │ ├── columns: rowid:3!null xy.x:5 │ ├── grouping columns: rowid:3!null │ ├── key: (3) - │ ├── fd: (3)-->(4) + │ ├── fd: (3)-->(5) │ ├── left-join (hash) - │ │ ├── columns: b:2 rowid:3!null xy.x:4 y:5 - │ │ ├── key: (3,4) - │ │ ├── fd: (3)-->(2), (4)-->(5) + │ │ ├── columns: b:2 rowid:3!null xy.x:5 y:6 + │ │ ├── key: (3,5) + │ │ ├── fd: (3)-->(2), (5)-->(6) │ │ ├── scan ab │ │ │ ├── columns: b:2 rowid:3!null │ │ │ ├── key: (3) │ │ │ └── fd: (3)-->(2) │ │ ├── scan xy - │ │ │ ├── columns: xy.x:4!null y:5 - │ │ │ ├── key: (4) - │ │ │ └── fd: (4)-->(5) + │ │ │ ├── columns: xy.x:5!null y:6 + │ │ │ ├── key: (5) + │ │ │ └── fd: (5)-->(6) │ │ └── filters - │ │ └── y:5 = b:2 [outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ]), fd=(2)==(5), (5)==(2)] + │ │ └── y:6 = b:2 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] │ └── aggregations - │ └── first-agg [as=xy.x:4, outer=(4)] - │ └── xy.x:4 + │ └── first-agg [as=xy.x:5, outer=(5)] + │ └── xy.x:5 └── projections - └── xy.x:4 [as=x:6, outer=(4)] + └── xy.x:5 [as=x:8, outer=(5)] # EnsureKey should construct an Ordinality operator when it is called on a Scan # over a virtual table. @@ -5661,20 +5661,20 @@ norm SELECT (SELECT x FROM xy WHERE y=version LIMIT 1) FROM information_schema.tables ---- project - ├── columns: x:10 + ├── columns: x:11 ├── distinct-on - │ ├── columns: xy.x:8 rownum:11!null - │ ├── grouping columns: rownum:11!null - │ ├── key: (11) - │ ├── fd: (11)-->(8) + │ ├── columns: xy.x:8 rownum:12!null + │ ├── grouping columns: rownum:12!null + │ ├── key: (12) + │ ├── fd: (12)-->(8) │ ├── left-join (hash) - │ │ ├── columns: version:7 xy.x:8 y:9 rownum:11!null - │ │ ├── key: (8,11) - │ │ ├── fd: (11)-->(7), (8)-->(9) + │ │ ├── columns: version:7 xy.x:8 y:9 rownum:12!null + │ │ ├── key: (8,12) + │ │ ├── fd: (12)-->(7), (8)-->(9) │ │ ├── ordinality - │ │ │ ├── columns: version:7 rownum:11!null - │ │ │ ├── key: (11) - │ │ │ ├── fd: (11)-->(7) + │ │ │ ├── columns: version:7 rownum:12!null + │ │ │ ├── key: (12) + │ │ │ ├── fd: (12)-->(7) │ │ │ └── scan information_schema.tables │ │ │ └── columns: version:7 │ │ ├── scan xy @@ -5687,4 +5687,4 @@ project │ └── first-agg [as=xy.x:8, outer=(8)] │ └── xy.x:8 └── projections - └── xy.x:8 [as=x:10, outer=(8)] + └── xy.x:8 [as=x:11, outer=(8)] diff --git a/pkg/sql/opt/norm/testdata/rules/fold_constants b/pkg/sql/opt/norm/testdata/rules/fold_constants index 39a6dbf00573..861189362895 100644 --- a/pkg/sql/opt/norm/testdata/rules/fold_constants +++ b/pkg/sql/opt/norm/testdata/rules/fold_constants @@ -33,13 +33,13 @@ norm expect=FoldNullUnary SELECT +null::int AS r, -null::int AS s, ~null::int AS t FROM a ---- project - ├── columns: r:7 s:8 t:9 - ├── fd: ()-->(7-9) + ├── columns: r:8 s:9 t:10 + ├── fd: ()-->(8-10) ├── scan a └── projections - ├── CAST(NULL AS INT8) [as=r:7] - ├── CAST(NULL AS INT8) [as=s:8] - └── CAST(NULL AS INT8) [as=t:9] + ├── CAST(NULL AS INT8) [as=r:8] + ├── CAST(NULL AS INT8) [as=s:9] + └── CAST(NULL AS INT8) [as=t:10] # -------------------------------------------------- # FoldNullBinaryLeft, FoldNullBinaryRight @@ -60,24 +60,24 @@ SELECT FROM a ---- project - ├── columns: ra:7 rb:8 sa:9 sb:10 ta:11 tb:12 ua:13 ub:14 va:15 vb:16 wa:17 wb:18 + ├── columns: ra:8 rb:9 sa:10 sb:11 ta:12 tb:13 ua:14 ub:15 va:16 vb:17 wa:18 wb:19 ├── immutable - ├── fd: ()-->(7-14) + ├── fd: ()-->(8-15) ├── scan a │ └── columns: i:2 arr:6 └── projections - ├── CAST(NULL AS INT8) [as=ra:7] - ├── CAST(NULL AS INT8) [as=rb:8] - ├── CAST(NULL AS DECIMAL) [as=sa:9] - ├── CAST(NULL AS DECIMAL) [as=sb:10] - ├── CAST(NULL AS FLOAT8) [as=ta:11] - ├── CAST(NULL AS FLOAT8) [as=tb:12] - ├── CAST(NULL AS INT8) [as=ua:13] - ├── CAST(NULL AS INT8) [as=ub:14] - ├── arr:6::DECIMAL[] || CAST(NULL AS DECIMAL[]) [as=va:15, outer=(6), immutable] - ├── CAST(NULL AS STRING[]) || arr:6::STRING[] [as=vb:16, outer=(6), immutable] - ├── i:2::DECIMAL || CAST(NULL AS DECIMAL[]) [as=wa:17, outer=(2), immutable] - └── CAST(NULL AS FLOAT8[]) || i:2::FLOAT8 [as=wb:18, outer=(2), immutable] + ├── CAST(NULL AS INT8) [as=ra:8] + ├── CAST(NULL AS INT8) [as=rb:9] + ├── CAST(NULL AS DECIMAL) [as=sa:10] + ├── CAST(NULL AS DECIMAL) [as=sb:11] + ├── CAST(NULL AS FLOAT8) [as=ta:12] + ├── CAST(NULL AS FLOAT8) [as=tb:13] + ├── CAST(NULL AS INT8) [as=ua:14] + ├── CAST(NULL AS INT8) [as=ub:15] + ├── arr:6::DECIMAL[] || CAST(NULL AS DECIMAL[]) [as=va:16, outer=(6), immutable] + ├── CAST(NULL AS STRING[]) || arr:6::STRING[] [as=vb:17, outer=(6), immutable] + ├── i:2::DECIMAL || CAST(NULL AS DECIMAL[]) [as=wa:18, outer=(2), immutable] + └── CAST(NULL AS FLOAT8[]) || i:2::FLOAT8 [as=wb:19, outer=(2), immutable] norm SELECT @@ -90,19 +90,19 @@ SELECT FROM a ---- project - ├── columns: ra:7 rb:8 sa:9 sb:10 ta:11 tb:12 ua:11 ub:12 va:13 vb:14 wa:15 wb:14 - ├── fd: ()-->(7-15) + ├── columns: ra:8 rb:9 sa:10 sb:11 ta:12 tb:13 ua:12 ub:13 va:14 vb:15 wa:16 wb:15 + ├── fd: ()-->(8-16) ├── scan a └── projections - ├── CAST(NULL AS JSONB) [as=ra:7] - ├── CAST(NULL AS JSONB) [as=rb:8] - ├── CAST(NULL AS JSONB) [as=sa:9] - ├── CAST(NULL AS JSONB) [as=sb:10] - ├── CAST(NULL AS STRING) [as=ta:11] - ├── CAST(NULL AS STRING) [as=tb:12] - ├── CAST(NULL AS JSONB) [as=va:13] - ├── NULL [as=vb:14] - └── CAST(NULL AS STRING) [as=wa:15] + ├── CAST(NULL AS JSONB) [as=ra:8] + ├── CAST(NULL AS JSONB) [as=rb:9] + ├── CAST(NULL AS JSONB) [as=sa:10] + ├── CAST(NULL AS JSONB) [as=sb:11] + ├── CAST(NULL AS STRING) [as=ta:12] + ├── CAST(NULL AS STRING) [as=tb:13] + ├── CAST(NULL AS JSONB) [as=va:14] + ├── NULL [as=vb:15] + └── CAST(NULL AS STRING) [as=wa:16] # -------------------------------------------------- # FoldNullInNonEmpty @@ -111,12 +111,12 @@ norm expect=FoldNullInNonEmpty SELECT null IN (i) AS r, null NOT IN (s) AS s FROM a ---- project - ├── columns: r:7 s:8 - ├── fd: ()-->(7,8) + ├── columns: r:8 s:9 + ├── fd: ()-->(8,9) ├── scan a └── projections - ├── CAST(NULL AS BOOL) [as=r:7] - └── CAST(NULL AS BOOL) [as=s:8] + ├── CAST(NULL AS BOOL) [as=r:8] + └── CAST(NULL AS BOOL) [as=s:9] # -------------------------------------------------- # FoldInNull @@ -125,12 +125,12 @@ norm expect=FoldInNull SELECT i IN (null, null) AS r, k NOT IN (1 * null, null::int, 1 < null) AS s FROM a ---- project - ├── columns: r:7 s:8 - ├── fd: ()-->(7,8) + ├── columns: r:8 s:9 + ├── fd: ()-->(8,9) ├── scan a └── projections - ├── CAST(NULL AS BOOL) [as=r:7] - └── CAST(NULL AS BOOL) [as=s:8] + ├── CAST(NULL AS BOOL) [as=r:8] + └── CAST(NULL AS BOOL) [as=s:9] # -------------------------------------------------- # FoldInEmpty @@ -194,23 +194,23 @@ norm expect=FoldArray SELECT ARRAY[1, 2, 3] FROM t ---- project - ├── columns: array:2!null - ├── fd: ()-->(2) + ├── columns: array:3!null + ├── fd: ()-->(3) ├── scan t └── projections - └── ARRAY[1,2,3] [as=array:2] + └── ARRAY[1,2,3] [as=array:3] # Do not fold if there is a non-constant element. norm expect-not=FoldArray SELECT ARRAY[1, 2, 3, x] FROM t ---- project - ├── columns: array:2!null + ├── columns: array:3!null ├── scan t │ ├── columns: x:1!null │ └── key: (1) └── projections - └── ARRAY[1, 2, 3, x:1] [as=array:2, outer=(1)] + └── ARRAY[1, 2, 3, x:1] [as=array:3, outer=(1)] norm expect=FoldArray SELECT ARRAY['foo', 'bar'] @@ -892,75 +892,75 @@ norm expect=FoldIndirection SELECT ARRAY[i, i + 1][1] FROM a ---- project - ├── columns: array:7 + ├── columns: array:8 ├── scan a │ └── columns: i:2 └── projections - └── i:2 [as=array:7, outer=(2)] + └── i:2 [as=array:8, outer=(2)] norm expect=FoldIndirection SELECT ARRAY[i, i + 1][2] FROM a ---- project - ├── columns: array:7 + ├── columns: array:8 ├── immutable ├── scan a │ └── columns: i:2 └── projections - └── i:2 + 1 [as=array:7, outer=(2), immutable] + └── i:2 + 1 [as=array:8, outer=(2), immutable] # Fold when input is a DArray constant. norm expect=FoldIndirection SELECT ARRAY[4, 5, 6][2] FROM a ---- project - ├── columns: array:7!null - ├── fd: ()-->(7) + ├── columns: array:8!null + ├── fd: ()-->(8) ├── scan a └── projections - └── 5 [as=array:7] + └── 5 [as=array:8] # Array bounds are out-of-range. norm expect=FoldIndirection SELECT ARRAY[s, 'foo'][0] FROM a ---- project - ├── columns: array:7 - ├── fd: ()-->(7) + ├── columns: array:8 + ├── fd: ()-->(8) ├── scan a └── projections - └── CAST(NULL AS STRING) [as=array:7] + └── CAST(NULL AS STRING) [as=array:8] norm expect=FoldIndirection SELECT ARRAY[i, i + 1][3] FROM a ---- project - ├── columns: array:7 - ├── fd: ()-->(7) + ├── columns: array:8 + ├── fd: ()-->(8) ├── scan a └── projections - └── CAST(NULL AS INT8) [as=array:7] + └── CAST(NULL AS INT8) [as=array:8] norm expect=FoldIndirection SELECT ARRAY[4, 5, 6][0] FROM a ---- project - ├── columns: array:7 - ├── fd: ()-->(7) + ├── columns: array:8 + ├── fd: ()-->(8) ├── scan a └── projections - └── CAST(NULL AS INT8) [as=array:7] + └── CAST(NULL AS INT8) [as=array:8] # Array is dynamically constructed. norm expect-not=FoldIndirection SELECT arr[0] FROM a ---- project - ├── columns: arr:7 + ├── columns: arr:8 ├── scan a │ └── columns: a.arr:6 └── projections - └── a.arr:6[0] [as=arr:7, outer=(6)] + └── a.arr:6[0] [as=arr:8, outer=(6)] # Regression test for #40404. norm expect=FoldIndirection @@ -990,21 +990,21 @@ norm expect=FoldColumnAccess SELECT (ARRAY[(('foo', i) AS foo, bar)][1]).foo FROM a ---- project - ├── columns: foo:7!null - ├── fd: ()-->(7) + ├── columns: foo:8!null + ├── fd: ()-->(8) ├── scan a └── projections - └── 'foo' [as=foo:7] + └── 'foo' [as=foo:8] norm expect=FoldColumnAccess SELECT (ARRAY[(('foo', i) AS foo, bar)][1]).bar FROM a ---- project - ├── columns: bar:7 + ├── columns: bar:8 ├── scan a │ └── columns: i:2 └── projections - └── i:2 [as=bar:7, outer=(2)] + └── i:2 [as=bar:8, outer=(2)] # Fold when input is a constant DTuple. norm expect=FoldColumnAccess diff --git a/pkg/sql/opt/norm/testdata/rules/groupby b/pkg/sql/opt/norm/testdata/rules/groupby index 577fe07dd6b9..15291dddf848 100644 --- a/pkg/sql/opt/norm/testdata/rules/groupby +++ b/pkg/sql/opt/norm/testdata/rules/groupby @@ -99,14 +99,14 @@ norm expect-not=ConvertGroupByToDistinct SELECT s, f, sum(f) FROM a GROUP BY s, f ---- group-by - ├── columns: s:4!null f:3 sum:6 + ├── columns: s:4!null f:3 sum:7 ├── grouping columns: f:3 s:4!null ├── key: (3,4) - ├── fd: (3,4)-->(6) + ├── fd: (3,4)-->(7) ├── scan a │ └── columns: f:3 s:4!null └── aggregations - └── sum [as=sum:6, outer=(3)] + └── sum [as=sum:7, outer=(3)] └── f:3 # -------------------------------------------------- @@ -128,9 +128,9 @@ norm expect=EliminateJoinUnderGroupByLeft SELECT DISTINCT ON (x) x, y FROM fks RIGHT JOIN xy ON x=v ---- scan xy - ├── columns: x:5!null y:6 - ├── key: (5) - └── fd: (5)-->(6) + ├── columns: x:6!null y:7 + ├── key: (6) + └── fd: (6)-->(7) # InnerJoin case. The Values operator in the join guarantees cardinality of at # least one, so rows from the left input are guaranteed to be included in the @@ -139,16 +139,16 @@ norm expect=EliminateJoinUnderGroupByLeft SELECT k, max(r1) FROM fks INNER JOIN (SELECT * FROM (VALUES (1), (2)) f(t)) ON True GROUP BY k ---- group-by - ├── columns: k:1!null max:6!null + ├── columns: k:1!null max:7!null ├── grouping columns: k:1!null ├── key: (1) - ├── fd: (1)-->(6) + ├── fd: (1)-->(7) ├── scan fks │ ├── columns: k:1!null r1:3!null │ ├── key: (1) │ └── fd: (1)-->(3) └── aggregations - └── max [as=max:6, outer=(3)] + └── max [as=max:7, outer=(3)] └── r1:3 # Case with ScalarGroupBy with a sum aggregate that doesn't ignore duplicates. @@ -158,15 +158,15 @@ norm expect=EliminateJoinUnderGroupByLeft disable=EliminateJoinUnderProjectLeft SELECT sum(k) FROM fks LEFT JOIN xy ON x=r1 ---- scalar-group-by - ├── columns: sum:7 + ├── columns: sum:9 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(7) + ├── fd: ()-->(9) ├── scan fks │ ├── columns: k:1!null │ └── key: (1) └── aggregations - └── sum [as=sum:7, outer=(1)] + └── sum [as=sum:9, outer=(1)] └── k:1 # LeftJoin case with possible duplicate rows. The rule can fire because the @@ -175,16 +175,16 @@ norm expect=EliminateJoinUnderGroupByLeft SELECT x, max(y) FROM xy LEFT JOIN fks ON True GROUP BY x ---- group-by - ├── columns: x:1!null max:7 + ├── columns: x:1!null max:9 ├── grouping columns: x:1!null ├── key: (1) - ├── fd: (1)-->(7) + ├── fd: (1)-->(9) ├── scan xy │ ├── columns: x:1!null y:2 │ ├── key: (1) │ └── fd: (1)-->(2) └── aggregations - └── max [as=max:7, outer=(2)] + └── max [as=max:9, outer=(2)] └── y:2 # LeftJoin case with a not-null foreign key equality filter and a sum aggregate. @@ -192,16 +192,16 @@ norm expect=EliminateJoinUnderGroupByLeft disable=EliminateJoinUnderProjectLeft SELECT k, sum(r1) FROM fks LEFT JOIN xy ON x=r1 GROUP BY k ---- group-by - ├── columns: k:1!null sum:7!null + ├── columns: k:1!null sum:9!null ├── grouping columns: k:1!null ├── key: (1) - ├── fd: (1)-->(7) + ├── fd: (1)-->(9) ├── scan fks │ ├── columns: k:1!null r1:3!null │ ├── key: (1) │ └── fd: (1)-->(3) └── aggregations - └── sum [as=sum:7, outer=(3)] + └── sum [as=sum:9, outer=(3)] └── r1:3 # The LeftJoin guarantees that all left rows will be included in the output, and @@ -211,16 +211,16 @@ norm expect=EliminateJoinUnderGroupByLeft disable=EliminateJoinUnderProjectLeft SELECT x, sum(y) FROM xy LEFT JOIN fks ON x=k GROUP BY x ---- group-by - ├── columns: x:1!null sum:7 + ├── columns: x:1!null sum:9 ├── grouping columns: x:1!null ├── key: (1) - ├── fd: (1)-->(7) + ├── fd: (1)-->(9) ├── scan xy │ ├── columns: x:1!null y:2 │ ├── key: (1) │ └── fd: (1)-->(2) └── aggregations - └── sum [as=sum:7, outer=(2)] + └── sum [as=sum:9, outer=(2)] └── y:2 # The LeftJoin guarantees that all left rows will be included in the output, and @@ -231,16 +231,16 @@ norm expect=EliminateJoinUnderGroupByLeft disable=EliminateJoinUnderProjectLeft SELECT k, sum(r1) FROM fks LEFT JOIN xy ON x=r2 GROUP BY k ---- group-by - ├── columns: k:1!null sum:7!null + ├── columns: k:1!null sum:9!null ├── grouping columns: k:1!null ├── key: (1) - ├── fd: (1)-->(7) + ├── fd: (1)-->(9) ├── scan fks │ ├── columns: k:1!null r1:3!null │ ├── key: (1) │ └── fd: (1)-->(3) └── aggregations - └── sum [as=sum:7, outer=(3)] + └── sum [as=sum:9, outer=(3)] └── r1:3 # InnerJoin case. Because r1 is a non-null foreign key that references x, the @@ -249,16 +249,16 @@ norm expect=EliminateJoinUnderGroupByLeft disable=EliminateJoinUnderProjectLeft SELECT k, sum(r1) FROM fks INNER JOIN xy ON x=r1 GROUP BY k ---- group-by - ├── columns: k:1!null sum:7!null + ├── columns: k:1!null sum:9!null ├── grouping columns: k:1!null ├── key: (1) - ├── fd: (1)-->(7) + ├── fd: (1)-->(9) ├── scan fks │ ├── columns: k:1!null r1:3!null │ ├── key: (1) │ └── fd: (1)-->(3) └── aggregations - └── sum [as=sum:7, outer=(3)] + └── sum [as=sum:9, outer=(3)] └── r1:3 # Case with an ordering on left columns. @@ -266,10 +266,10 @@ norm expect=EliminateJoinUnderGroupByLeft SELECT max(y) FROM xy LEFT JOIN fks ON True GROUP BY x ORDER BY x ---- group-by - ├── columns: max:7 [hidden: x:1!null] + ├── columns: max:9 [hidden: x:1!null] ├── grouping columns: x:1!null ├── key: (1) - ├── fd: (1)-->(7) + ├── fd: (1)-->(9) ├── ordering: +1 ├── scan xy │ ├── columns: x:1!null y:2 @@ -277,7 +277,7 @@ group-by │ ├── fd: (1)-->(2) │ └── ordering: +1 └── aggregations - └── max [as=max:7, outer=(2)] + └── max [as=max:9, outer=(2)] └── y:2 # Cross join case where neither the foreign key column nor its referenced column @@ -295,14 +295,14 @@ norm expect=EliminateJoinUnderGroupByLeft disable=EliminateJoinUnderProjectLeft SELECT max(v) FROM fks WHERE EXISTS (SELECT * FROM xy WHERE x = r1) ---- scalar-group-by - ├── columns: max:7 + ├── columns: max:9 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(7) + ├── fd: ()-->(9) ├── scan fks │ └── columns: v:2 └── aggregations - └── max [as=max:7, outer=(2)] + └── max [as=max:9, outer=(2)] └── v:2 # No-op case because the InnerJoin will return no rows if fks is empty. @@ -332,28 +332,28 @@ norm expect-not=EliminateJoinUnderGroupByLeft SELECT DISTINCT ON (x) y, k FROM xy LEFT JOIN fks ON True ---- distinct-on - ├── columns: y:2 k:3 [hidden: x:1!null] + ├── columns: y:2 k:4 [hidden: x:1!null] ├── grouping columns: x:1!null ├── key: (1) - ├── fd: (1)-->(2,3) + ├── fd: (1)-->(2,4) ├── left-join (cross) - │ ├── columns: x:1!null y:2 k:3 + │ ├── columns: x:1!null y:2 k:4 │ ├── multiplicity: left-rows(one-or-more), right-rows(one-or-more) - │ ├── key: (1,3) + │ ├── key: (1,4) │ ├── fd: (1)-->(2) │ ├── scan xy │ │ ├── columns: x:1!null y:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ ├── scan fks - │ │ ├── columns: k:3!null - │ │ └── key: (3) + │ │ ├── columns: k:4!null + │ │ └── key: (4) │ └── filters (true) └── aggregations ├── first-agg [as=y:2, outer=(2)] │ └── y:2 - └── first-agg [as=k:3, outer=(3)] - └── k:3 + └── first-agg [as=k:4, outer=(4)] + └── k:4 # No-op case because an InnerJoin on true may create duplicate rows that will # affect the output of the sum on r1. @@ -361,10 +361,10 @@ norm expect-not=EliminateJoinUnderGroupByLeft SELECT k, sum(r1) FROM fks INNER JOIN xy ON True GROUP BY k ---- group-by - ├── columns: k:1!null sum:7!null + ├── columns: k:1!null sum:9!null ├── grouping columns: k:1!null ├── key: (1) - ├── fd: (1)-->(7) + ├── fd: (1)-->(9) ├── inner-join (cross) │ ├── columns: k:1!null r1:3!null │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-more) @@ -376,7 +376,7 @@ group-by │ ├── scan xy │ └── filters (true) └── aggregations - └── sum [as=sum:7, outer=(3)] + └── sum [as=sum:9, outer=(3)] └── r1:3 # No-op case with a foreign key equality filter and a sum aggregate. No-op @@ -385,26 +385,26 @@ norm expect-not=EliminateJoinUnderGroupByLeft SELECT k, sum(r1) FROM fks INNER JOIN xy ON x=r2 GROUP BY k ---- group-by - ├── columns: k:1!null sum:7!null + ├── columns: k:1!null sum:9!null ├── grouping columns: k:1!null ├── key: (1) - ├── fd: (1)-->(7) + ├── fd: (1)-->(9) ├── inner-join (hash) - │ ├── columns: k:1!null r1:3!null r2:4!null x:5!null + │ ├── columns: k:1!null r1:3!null r2:4!null x:6!null │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) │ ├── key: (1) - │ ├── fd: (1)-->(3,4), (4)==(5), (5)==(4) + │ ├── fd: (1)-->(3,4), (4)==(6), (6)==(4) │ ├── scan fks │ │ ├── columns: k:1!null r1:3!null r2:4 │ │ ├── key: (1) │ │ └── fd: (1)-->(3,4) │ ├── scan xy - │ │ ├── columns: x:5!null - │ │ └── key: (5) + │ │ ├── columns: x:6!null + │ │ └── key: (6) │ └── filters - │ └── x:5 = r2:4 [outer=(4,5), constraints=(/4: (/NULL - ]; /5: (/NULL - ]), fd=(4)==(5), (5)==(4)] + │ └── x:6 = r2:4 [outer=(4,6), constraints=(/4: (/NULL - ]; /6: (/NULL - ]), fd=(4)==(6), (6)==(4)] └── aggregations - └── sum [as=sum:7, outer=(3)] + └── sum [as=sum:9, outer=(3)] └── r1:3 # No-op case because the ordering includes a column from the right input. @@ -412,31 +412,31 @@ norm expect-not=EliminateJoinUnderGroupByLeft SELECT x, max(y) FROM xy LEFT JOIN fks ON True GROUP BY x, k ORDER BY x, k ---- group-by - ├── columns: x:1!null max:7 [hidden: k:3] - ├── grouping columns: x:1!null k:3 - ├── key: (1,3) - ├── fd: (1,3)-->(7) - ├── ordering: +1,+3 + ├── columns: x:1!null max:9 [hidden: k:4] + ├── grouping columns: x:1!null k:4 + ├── key: (1,4) + ├── fd: (1,4)-->(9) + ├── ordering: +1,+4 ├── sort - │ ├── columns: x:1!null y:2 k:3 - │ ├── key: (1,3) + │ ├── columns: x:1!null y:2 k:4 + │ ├── key: (1,4) │ ├── fd: (1)-->(2) - │ ├── ordering: +1,+3 + │ ├── ordering: +1,+4 │ └── left-join (cross) - │ ├── columns: x:1!null y:2 k:3 + │ ├── columns: x:1!null y:2 k:4 │ ├── multiplicity: left-rows(one-or-more), right-rows(one-or-more) - │ ├── key: (1,3) + │ ├── key: (1,4) │ ├── fd: (1)-->(2) │ ├── scan xy │ │ ├── columns: x:1!null y:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ ├── scan fks - │ │ ├── columns: k:3!null - │ │ └── key: (3) + │ │ ├── columns: k:4!null + │ │ └── key: (4) │ └── filters (true) └── aggregations - └── max [as=max:7, outer=(2)] + └── max [as=max:9, outer=(2)] └── y:2 # -------------------------------------------------- @@ -448,40 +448,40 @@ norm expect=EliminateJoinUnderGroupByRight disable=EliminateJoinUnderProjectRigh SELECT k, sum(r1) FROM xy INNER JOIN fks ON x = r1 GROUP BY k ---- group-by - ├── columns: k:3!null sum:7!null - ├── grouping columns: k:3!null - ├── key: (3) - ├── fd: (3)-->(7) + ├── columns: k:4!null sum:9!null + ├── grouping columns: k:4!null + ├── key: (4) + ├── fd: (4)-->(9) ├── scan fks - │ ├── columns: k:3!null r1:5!null - │ ├── key: (3) - │ └── fd: (3)-->(5) + │ ├── columns: k:4!null r1:6!null + │ ├── key: (4) + │ └── fd: (4)-->(6) └── aggregations - └── sum [as=sum:7, outer=(5)] - └── r1:5 + └── sum [as=sum:9, outer=(6)] + └── r1:6 # No-op case because columns from the right side of a left join are being used. norm expect-not=EliminateJoinUnderGroupByRight SELECT max(r1) FROM xy LEFT JOIN fks ON x = r1 ---- scalar-group-by - ├── columns: max:7 + ├── columns: max:9 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(7) + ├── fd: ()-->(9) ├── left-join (hash) - │ ├── columns: x:1!null r1:5 + │ ├── columns: x:1!null r1:6 │ ├── multiplicity: left-rows(one-or-more), right-rows(exactly-one) │ ├── scan xy │ │ ├── columns: x:1!null │ │ └── key: (1) │ ├── scan fks - │ │ └── columns: r1:5!null + │ │ └── columns: r1:6!null │ └── filters - │ └── x:1 = r1:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + │ └── x:1 = r1:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] └── aggregations - └── max [as=max:7, outer=(5)] - └── r1:5 + └── max [as=max:9, outer=(6)] + └── r1:6 # -------------------------------------------------- # EliminateDistinct @@ -567,10 +567,10 @@ SELECT ); ---- values - ├── columns: col2:7 + ├── columns: col2:9 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(7) + ├── fd: ()-->(9) └── tuple └── subquery └── values @@ -586,30 +586,30 @@ norm expect=EliminateDistinct disable=EliminateMax1Row SELECT (SELECT y FROM xy WHERE x=k AND k=5) FROM a ---- project - ├── columns: y:8 + ├── columns: y:10 ├── left-join (cross) - │ ├── columns: k:1!null x:6 xy.y:7 + │ ├── columns: k:1!null x:7 xy.y:8 │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) │ ├── key: (1) - │ ├── fd: (1)-->(6,7) + │ ├── fd: (1)-->(7,8) │ ├── scan a │ │ ├── columns: k:1!null │ │ └── key: (1) │ ├── select - │ │ ├── columns: x:6!null xy.y:7 + │ │ ├── columns: x:7!null xy.y:8 │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(6,7) + │ │ ├── fd: ()-->(7,8) │ │ ├── scan xy - │ │ │ ├── columns: x:6!null xy.y:7 - │ │ │ ├── key: (6) - │ │ │ └── fd: (6)-->(7) + │ │ │ ├── columns: x:7!null xy.y:8 + │ │ │ ├── key: (7) + │ │ │ └── fd: (7)-->(8) │ │ └── filters - │ │ └── x:6 = 5 [outer=(6), constraints=(/6: [/5 - /5]; tight), fd=()-->(6)] + │ │ └── x:7 = 5 [outer=(7), constraints=(/7: [/5 - /5]; tight), fd=()-->(7)] │ └── filters │ └── k:1 = 5 [outer=(1), constraints=(/1: [/5 - /5]; tight), fd=()-->(1)] └── projections - └── xy.y:7 [as=y:8, outer=(7)] + └── xy.y:8 [as=y:10, outer=(8)] # -------------------------------------------------- # EliminateGroupByProject @@ -618,80 +618,80 @@ norm expect=EliminateGroupByProject SELECT min(s) FROM (SELECT i, s FROM (SELECT * FROM a UNION SELECT * FROM a)) GROUP BY i ---- project - ├── columns: min:16!null + ├── columns: min:18!null └── group-by - ├── columns: i:12!null min:16!null - ├── grouping columns: i:12!null - ├── key: (12) - ├── fd: (12)-->(16) + ├── columns: i:14!null min:18!null + ├── grouping columns: i:14!null + ├── key: (14) + ├── fd: (14)-->(18) ├── union - │ ├── columns: k:11!null i:12!null f:13 s:14!null j:15 + │ ├── columns: k:13!null i:14!null f:15 s:16!null j:17 │ ├── left columns: a.k:1 a.i:2 a.f:3 a.s:4 a.j:5 - │ ├── right columns: a.k:6 a.i:7 a.f:8 a.s:9 a.j:10 - │ ├── key: (11-15) + │ ├── right columns: a.k:7 a.i:8 a.f:9 a.s:10 a.j:11 + │ ├── key: (13-17) │ ├── scan a │ │ ├── columns: a.k:1!null a.i:2!null a.f:3 a.s:4!null a.j:5 │ │ ├── key: (1) │ │ └── fd: (1)-->(2-5), (2,4)-->(1,3,5), (2,3)~~>(1,4,5) │ └── scan a - │ ├── columns: a.k:6!null a.i:7!null a.f:8 a.s:9!null a.j:10 - │ ├── key: (6) - │ └── fd: (6)-->(7-10), (7,9)-->(6,8,10), (7,8)~~>(6,9,10) + │ ├── columns: a.k:7!null a.i:8!null a.f:9 a.s:10!null a.j:11 + │ ├── key: (7) + │ └── fd: (7)-->(8-11), (8,10)-->(7,9,11), (8,9)~~>(7,10,11) └── aggregations - └── min [as=min:16, outer=(14)] - └── s:14 + └── min [as=min:18, outer=(16)] + └── s:16 # ScalarGroupBy case. norm expect=EliminateGroupByProject SELECT min(s) FROM (SELECT i, s FROM (SELECT * FROM a UNION SELECT * FROM a)) ---- scalar-group-by - ├── columns: min:16 + ├── columns: min:18 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(16) + ├── fd: ()-->(18) ├── union - │ ├── columns: k:11!null i:12!null f:13 s:14!null j:15 + │ ├── columns: k:13!null i:14!null f:15 s:16!null j:17 │ ├── left columns: a.k:1 a.i:2 a.f:3 a.s:4 a.j:5 - │ ├── right columns: a.k:6 a.i:7 a.f:8 a.s:9 a.j:10 - │ ├── key: (11-15) + │ ├── right columns: a.k:7 a.i:8 a.f:9 a.s:10 a.j:11 + │ ├── key: (13-17) │ ├── scan a │ │ ├── columns: a.k:1!null a.i:2!null a.f:3 a.s:4!null a.j:5 │ │ ├── key: (1) │ │ └── fd: (1)-->(2-5), (2,4)-->(1,3,5), (2,3)~~>(1,4,5) │ └── scan a - │ ├── columns: a.k:6!null a.i:7!null a.f:8 a.s:9!null a.j:10 - │ ├── key: (6) - │ └── fd: (6)-->(7-10), (7,9)-->(6,8,10), (7,8)~~>(6,9,10) + │ ├── columns: a.k:7!null a.i:8!null a.f:9 a.s:10!null a.j:11 + │ ├── key: (7) + │ └── fd: (7)-->(8-11), (8,10)-->(7,9,11), (8,9)~~>(7,10,11) └── aggregations - └── min [as=min:16, outer=(14)] - └── s:14 + └── min [as=min:18, outer=(16)] + └── s:16 # DistinctOn case. norm expect=EliminateGroupByProject SELECT DISTINCT ON (i) s FROM (SELECT i, s, f FROM (SELECT * FROM a UNION SELECT * FROM a)) ---- distinct-on - ├── columns: s:14!null [hidden: i:12!null] - ├── grouping columns: i:12!null - ├── key: (12) - ├── fd: (12)-->(14) + ├── columns: s:16!null [hidden: i:14!null] + ├── grouping columns: i:14!null + ├── key: (14) + ├── fd: (14)-->(16) ├── union - │ ├── columns: k:11!null i:12!null f:13 s:14!null j:15 + │ ├── columns: k:13!null i:14!null f:15 s:16!null j:17 │ ├── left columns: a.k:1 a.i:2 a.f:3 a.s:4 a.j:5 - │ ├── right columns: a.k:6 a.i:7 a.f:8 a.s:9 a.j:10 - │ ├── key: (11-15) + │ ├── right columns: a.k:7 a.i:8 a.f:9 a.s:10 a.j:11 + │ ├── key: (13-17) │ ├── scan a │ │ ├── columns: a.k:1!null a.i:2!null a.f:3 a.s:4!null a.j:5 │ │ ├── key: (1) │ │ └── fd: (1)-->(2-5), (2,4)-->(1,3,5), (2,3)~~>(1,4,5) │ └── scan a - │ ├── columns: a.k:6!null a.i:7!null a.f:8 a.s:9!null a.j:10 - │ ├── key: (6) - │ └── fd: (6)-->(7-10), (7,9)-->(6,8,10), (7,8)~~>(6,9,10) + │ ├── columns: a.k:7!null a.i:8!null a.f:9 a.s:10!null a.j:11 + │ ├── key: (7) + │ └── fd: (7)-->(8-11), (8,10)-->(7,9,11), (8,9)~~>(7,10,11) └── aggregations - └── first-agg [as=s:14, outer=(14)] - └── s:14 + └── first-agg [as=s:16, outer=(16)] + └── s:16 # EnsureDistinctOn case. # EliminateMax1Row is disabled to ensure that an EnsureDistinctOn operator is @@ -700,38 +700,38 @@ norm expect=EliminateGroupByProject disable=EliminateMax1Row SELECT (SELECT y FROM xy WHERE x+y=k) FROM a ---- project - ├── columns: y:8 + ├── columns: y:10 ├── immutable ├── ensure-distinct-on - │ ├── columns: k:1!null xy.y:7 + │ ├── columns: k:1!null xy.y:8 │ ├── grouping columns: k:1!null │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(7) + │ ├── fd: (1)-->(8) │ ├── left-join (hash) - │ │ ├── columns: k:1!null xy.y:7 column9:9 + │ │ ├── columns: k:1!null xy.y:8 column11:11 │ │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) │ │ ├── immutable │ │ ├── scan a │ │ │ ├── columns: k:1!null │ │ │ └── key: (1) │ │ ├── project - │ │ │ ├── columns: column9:9 xy.y:7 + │ │ │ ├── columns: column11:11 xy.y:8 │ │ │ ├── immutable │ │ │ ├── scan xy - │ │ │ │ ├── columns: x:6!null xy.y:7 - │ │ │ │ ├── key: (6) - │ │ │ │ └── fd: (6)-->(7) + │ │ │ │ ├── columns: x:7!null xy.y:8 + │ │ │ │ ├── key: (7) + │ │ │ │ └── fd: (7)-->(8) │ │ │ └── projections - │ │ │ └── x:6 + xy.y:7 [as=column9:9, outer=(6,7), immutable] + │ │ │ └── x:7 + xy.y:8 [as=column11:11, outer=(7,8), immutable] │ │ └── filters - │ │ └── k:1 = column9:9 [outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ]), fd=(1)==(9), (9)==(1)] + │ │ └── k:1 = column11:11 [outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ]), fd=(1)==(11), (11)==(1)] │ └── aggregations - │ └── const-agg [as=xy.y:7, outer=(7)] - │ └── xy.y:7 + │ └── const-agg [as=xy.y:8, outer=(8)] + │ └── xy.y:8 └── projections - └── xy.y:7 [as=y:8, outer=(7)] + └── xy.y:8 [as=y:10, outer=(8)] # UpsertDistinctOn case. norm expect=EliminateGroupByProject @@ -742,55 +742,55 @@ ON CONFLICT (c1) DO NOTHING insert nullablecols ├── columns: ├── insert-mapping: - │ ├── i:6 => c1:1 - │ ├── i:6 => c2:2 - │ ├── i:6 => c3:3 - │ └── i:6 => rowid:4 + │ ├── i:7 => c1:1 + │ ├── i:7 => c2:2 + │ ├── i:7 => c3:3 + │ └── i:7 => rowid:4 ├── cardinality: [0 - 0] ├── volatile, mutations └── upsert-distinct-on - ├── columns: i:6!null - ├── grouping columns: i:6!null - ├── key: (6) + ├── columns: i:7!null + ├── grouping columns: i:7!null + ├── key: (7) └── select - ├── columns: k:5!null i:6!null c1:15 rowid:18 - ├── key: (5) - ├── fd: ()-->(15,18), (5)-->(6) + ├── columns: k:6!null i:7!null c1:18 rowid:21 + ├── key: (6) + ├── fd: ()-->(18,21), (6)-->(7) ├── left-join (hash) - │ ├── columns: k:5!null i:6!null c1:15 rowid:18 + │ ├── columns: k:6!null i:7!null c1:18 rowid:21 │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ ├── key: (5,18) - │ ├── fd: (5)-->(6), (18)-->(15), (15)~~>(18) + │ ├── key: (6,21) + │ ├── fd: (6)-->(7), (21)-->(18), (18)~~>(21) │ ├── select - │ │ ├── columns: k:5!null i:6!null - │ │ ├── key: (5) - │ │ ├── fd: (5)-->(6) + │ │ ├── columns: k:6!null i:7!null + │ │ ├── key: (6) + │ │ ├── fd: (6)-->(7) │ │ ├── scan a - │ │ │ ├── columns: k:5!null i:6!null - │ │ │ ├── key: (5) - │ │ │ └── fd: (5)-->(6) + │ │ │ ├── columns: k:6!null i:7!null + │ │ │ ├── key: (6) + │ │ │ └── fd: (6)-->(7) │ │ └── filters │ │ ├── exists [subquery] │ │ │ └── limit - │ │ │ ├── columns: k:10!null i:11!null f:12 s:13!null j:14 + │ │ │ ├── columns: k:12!null i:13!null f:14 s:15!null j:16 │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(10-14) + │ │ │ ├── fd: ()-->(12-16) │ │ │ ├── scan a - │ │ │ │ ├── columns: k:10!null i:11!null f:12 s:13!null j:14 - │ │ │ │ ├── key: (10) - │ │ │ │ ├── fd: (10)-->(11-14), (11,13)-->(10,12,14), (11,12)~~>(10,13,14) + │ │ │ │ ├── columns: k:12!null i:13!null f:14 s:15!null j:16 + │ │ │ │ ├── key: (12) + │ │ │ │ ├── fd: (12)-->(13-16), (13,15)-->(12,14,16), (13,14)~~>(12,15,16) │ │ │ │ └── limit hint: 1.00 │ │ │ └── 1 - │ │ └── k:5 > 0 [outer=(5), constraints=(/5: [/1 - ]; tight)] + │ │ └── k:6 > 0 [outer=(6), constraints=(/6: [/1 - ]; tight)] │ ├── scan nullablecols - │ │ ├── columns: c1:15 rowid:18!null - │ │ ├── key: (18) - │ │ └── fd: (18)-->(15), (15)~~>(18) + │ │ ├── columns: c1:18 rowid:21!null + │ │ ├── key: (21) + │ │ └── fd: (21)-->(18), (18)~~>(21) │ └── filters - │ └── i:6 = c1:15 [outer=(6,15), constraints=(/6: (/NULL - ]; /15: (/NULL - ]), fd=(6)==(15), (15)==(6)] + │ └── i:7 = c1:18 [outer=(7,18), constraints=(/7: (/NULL - ]; /18: (/NULL - ]), fd=(7)==(18), (18)==(7)] └── filters - └── rowid:18 IS NULL [outer=(18), constraints=(/18: [/NULL - /NULL]; tight), fd=()-->(18)] + └── rowid:21 IS NULL [outer=(21), constraints=(/21: [/NULL - /NULL]; tight), fd=()-->(21)] # EnsureUpsertDistinctOn case. norm expect=EliminateGroupByProject @@ -800,85 +800,85 @@ ON CONFLICT (c1) DO UPDATE SET c3=1 ---- upsert nullablecols ├── columns: - ├── canary column: 18 - ├── fetch columns: c1:15 c2:16 c3:17 rowid:18 + ├── canary column: 21 + ├── fetch columns: c1:18 c2:19 c3:20 rowid:21 ├── insert-mapping: - │ ├── i:6 => c1:1 - │ ├── i:6 => c2:2 - │ ├── i:6 => c3:3 - │ └── i:6 => rowid:4 + │ ├── i:7 => c1:1 + │ ├── i:7 => c2:2 + │ ├── i:7 => c3:3 + │ └── i:7 => rowid:4 ├── update-mapping: - │ └── upsert_c3:22 => c3:3 + │ └── upsert_c3:26 => c3:3 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: upsert_c3:22!null i:6!null c1:15 c2:16 c3:17 rowid:18 - ├── key: (6,18) - ├── fd: (18)-->(15-17), (15)~~>(16-18), (16,17)~~>(15,18), (6,18)-->(22) + ├── columns: upsert_c3:26!null i:7!null c1:18 c2:19 c3:20 rowid:21 + ├── key: (7,21) + ├── fd: (21)-->(18-20), (18)~~>(19-21), (19,20)~~>(18,21), (7,21)-->(26) ├── left-join (hash) - │ ├── columns: i:6!null c1:15 c2:16 c3:17 rowid:18 + │ ├── columns: i:7!null c1:18 c2:19 c3:20 rowid:21 │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) - │ ├── key: (6,18) - │ ├── fd: (18)-->(15-17), (15)~~>(16-18), (16,17)~~>(15,18) + │ ├── key: (7,21) + │ ├── fd: (21)-->(18-20), (18)~~>(19-21), (19,20)~~>(18,21) │ ├── ensure-upsert-distinct-on - │ │ ├── columns: i:6!null - │ │ ├── grouping columns: i:6!null + │ │ ├── columns: i:7!null + │ │ ├── grouping columns: i:7!null │ │ ├── error: "UPSERT or INSERT...ON CONFLICT command cannot affect row a second time" - │ │ ├── key: (6) + │ │ ├── key: (7) │ │ └── select - │ │ ├── columns: k:5!null i:6!null - │ │ ├── key: (5) - │ │ ├── fd: (5)-->(6) + │ │ ├── columns: k:6!null i:7!null + │ │ ├── key: (6) + │ │ ├── fd: (6)-->(7) │ │ ├── scan a - │ │ │ ├── columns: k:5!null i:6!null - │ │ │ ├── key: (5) - │ │ │ └── fd: (5)-->(6) + │ │ │ ├── columns: k:6!null i:7!null + │ │ │ ├── key: (6) + │ │ │ └── fd: (6)-->(7) │ │ └── filters │ │ ├── exists [subquery] │ │ │ └── limit - │ │ │ ├── columns: k:10!null i:11!null f:12 s:13!null j:14 + │ │ │ ├── columns: k:12!null i:13!null f:14 s:15!null j:16 │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(10-14) + │ │ │ ├── fd: ()-->(12-16) │ │ │ ├── scan a - │ │ │ │ ├── columns: k:10!null i:11!null f:12 s:13!null j:14 - │ │ │ │ ├── key: (10) - │ │ │ │ ├── fd: (10)-->(11-14), (11,13)-->(10,12,14), (11,12)~~>(10,13,14) + │ │ │ │ ├── columns: k:12!null i:13!null f:14 s:15!null j:16 + │ │ │ │ ├── key: (12) + │ │ │ │ ├── fd: (12)-->(13-16), (13,15)-->(12,14,16), (13,14)~~>(12,15,16) │ │ │ │ └── limit hint: 1.00 │ │ │ └── 1 - │ │ └── k:5 > 0 [outer=(5), constraints=(/5: [/1 - ]; tight)] + │ │ └── k:6 > 0 [outer=(6), constraints=(/6: [/1 - ]; tight)] │ ├── scan nullablecols - │ │ ├── columns: c1:15 c2:16 c3:17 rowid:18!null - │ │ ├── key: (18) - │ │ └── fd: (18)-->(15-17), (15)~~>(16-18), (16,17)~~>(15,18) + │ │ ├── columns: c1:18 c2:19 c3:20 rowid:21!null + │ │ ├── key: (21) + │ │ └── fd: (21)-->(18-20), (18)~~>(19-21), (19,20)~~>(18,21) │ └── filters - │ └── i:6 = c1:15 [outer=(6,15), constraints=(/6: (/NULL - ]; /15: (/NULL - ]), fd=(6)==(15), (15)==(6)] + │ └── i:7 = c1:18 [outer=(7,18), constraints=(/7: (/NULL - ]; /18: (/NULL - ]), fd=(7)==(18), (18)==(7)] └── projections - └── CASE WHEN rowid:18 IS NULL THEN i:6 ELSE 1 END [as=upsert_c3:22, outer=(6,18)] + └── CASE WHEN rowid:21 IS NULL THEN i:7 ELSE 1 END [as=upsert_c3:26, outer=(7,21)] # Don't eliminate project if it computes extra column(s). norm expect-not=EliminateGroupByProject SELECT min(s) FROM (SELECT i+1 AS i2, s FROM a) GROUP BY i2 ---- project - ├── columns: min:7!null + ├── columns: min:8!null ├── immutable └── group-by - ├── columns: i2:6!null min:7!null - ├── grouping columns: i2:6!null + ├── columns: i2:7!null min:8!null + ├── grouping columns: i2:7!null ├── immutable - ├── key: (6) - ├── fd: (6)-->(7) + ├── key: (7) + ├── fd: (7)-->(8) ├── project - │ ├── columns: i2:6!null s:4!null + │ ├── columns: i2:7!null s:4!null │ ├── immutable │ ├── scan a │ │ ├── columns: i:2!null s:4!null │ │ └── key: (2,4) │ └── projections - │ └── i:2 + 1 [as=i2:6, outer=(2), immutable] + │ └── i:2 + 1 [as=i2:7, outer=(2), immutable] └── aggregations - └── min [as=min:7, outer=(4)] + └── min [as=min:8, outer=(4)] └── s:4 # -------------------------------------------------- @@ -888,16 +888,16 @@ norm expect=ReduceGroupingCols SELECT k, min(i), f, s FROM a GROUP BY s, f, k ---- group-by - ├── columns: k:1!null min:6!null f:3 s:4!null + ├── columns: k:1!null min:7!null f:3 s:4!null ├── grouping columns: k:1!null ├── key: (1) - ├── fd: (1)-->(3,4,6) + ├── fd: (1)-->(3,4,7) ├── scan a │ ├── columns: k:1!null i:2!null f:3 s:4!null │ ├── key: (1) │ └── fd: (1)-->(2-4), (2,4)-->(1,3), (2,3)~~>(1,4) └── aggregations - ├── min [as=min:6, outer=(2)] + ├── min [as=min:7, outer=(2)] │ └── i:2 ├── const-agg [as=f:3, outer=(3)] │ └── f:3 @@ -908,10 +908,10 @@ norm expect=ReduceGroupingCols SELECT k, sum(DISTINCT i), f, s FROM a, xy GROUP BY s, f, k ---- group-by - ├── columns: k:1!null sum:8!null f:3 s:4!null + ├── columns: k:1!null sum:10!null f:3 s:4!null ├── grouping columns: k:1!null ├── key: (1) - ├── fd: (1)-->(3,4,8) + ├── fd: (1)-->(3,4,10) ├── inner-join (cross) │ ├── columns: k:1!null i:2!null f:3 s:4!null │ ├── fd: (1)-->(2-4), (2,4)-->(1,3), (2,3)~~>(1,4) @@ -922,7 +922,7 @@ group-by │ ├── scan xy │ └── filters (true) └── aggregations - ├── agg-distinct [as=sum:8, outer=(2)] + ├── agg-distinct [as=sum:10, outer=(2)] │ └── sum │ └── i:2 ├── const-agg [as=f:3, outer=(3)] @@ -935,18 +935,18 @@ norm expect=ReduceGroupingCols SELECT min(f) FROM a GROUP BY i, s, k ---- project - ├── columns: min:6 + ├── columns: min:7 └── group-by - ├── columns: i:2!null s:4!null min:6 + ├── columns: i:2!null s:4!null min:7 ├── grouping columns: i:2!null s:4!null ├── key: (2,4) - ├── fd: (2,4)-->(6) + ├── fd: (2,4)-->(7) ├── scan a │ ├── columns: i:2!null f:3 s:4!null │ ├── key: (2,4) │ └── fd: (2,4)-->(3), (2,3)~~>(4) └── aggregations - └── min [as=min:6, outer=(3)] + └── min [as=min:7, outer=(3)] └── f:3 # All grouping columns eliminated. @@ -954,10 +954,10 @@ norm expect=ReduceGroupingCols SELECT sum(f), i FROM a GROUP BY k, i, f HAVING k=1 ---- group-by - ├── columns: sum:6 i:2!null + ├── columns: sum:7 i:2!null ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(2,6) + ├── fd: ()-->(2,7) ├── select │ ├── columns: k:1!null i:2!null f:3 │ ├── cardinality: [0 - 1] @@ -970,7 +970,7 @@ group-by │ └── filters │ └── k:1 = 1 [outer=(1), constraints=(/1: [/1 - /1]; tight), fd=()-->(1)] └── aggregations - ├── sum [as=sum:6, outer=(3)] + ├── sum [as=sum:7, outer=(3)] │ └── f:3 └── const-agg [as=i:2, outer=(2)] └── i:2 @@ -979,29 +979,29 @@ norm expect=ReduceGroupingCols SELECT DISTINCT ON (k, f, s) i, f, x FROM a JOIN xy ON i=y ---- distinct-on - ├── columns: i:2!null f:3 x:6!null [hidden: k:1!null] + ├── columns: i:2!null f:3 x:7!null [hidden: k:1!null] ├── grouping columns: k:1!null ├── key: (1) - ├── fd: (1)-->(2,3,6), (2,3)~~>(1), (6)-->(2) + ├── fd: (1)-->(2,3,7), (2,3)~~>(1), (7)-->(2) ├── inner-join (hash) - │ ├── columns: k:1!null i:2!null f:3 x:6!null y:7!null - │ ├── key: (1,6) - │ ├── fd: (1)-->(2,3), (2,3)~~>(1), (6)-->(7), (2)==(7), (7)==(2) + │ ├── columns: k:1!null i:2!null f:3 x:7!null y:8!null + │ ├── key: (1,7) + │ ├── fd: (1)-->(2,3), (2,3)~~>(1), (7)-->(8), (2)==(8), (8)==(2) │ ├── scan a │ │ ├── columns: k:1!null i:2!null f:3 │ │ ├── key: (1) │ │ └── fd: (1)-->(2,3), (2,3)~~>(1) │ ├── scan xy - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ └── filters - │ └── i:2 = y:7 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] + │ └── i:2 = y:8 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] └── aggregations ├── first-agg [as=i:2, outer=(2)] │ └── i:2 - ├── first-agg [as=x:6, outer=(6)] - │ └── x:6 + ├── first-agg [as=x:7, outer=(7)] + │ └── x:7 └── const-agg [as=f:3, outer=(3)] └── f:3 @@ -1018,48 +1018,48 @@ ON CONFLICT (x) DO NOTHING insert xy ├── columns: ├── insert-mapping: - │ ├── y:4 => x:1 - │ └── column5:5 => y:2 + │ ├── y:5 => x:1 + │ └── column7:7 => y:2 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: y:4!null column5:5 + ├── columns: y:5!null column7:7 ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(4,5) + ├── fd: ()-->(5,7) └── limit - ├── columns: y:4!null column5:5 x:6 + ├── columns: y:5!null column7:7 x:8 ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(4-6) + ├── fd: ()-->(5,7,8) ├── select - │ ├── columns: y:4!null column5:5 x:6 - │ ├── fd: ()-->(4-6) + │ ├── columns: y:5!null column7:7 x:8 + │ ├── fd: ()-->(5,7,8) │ ├── limit hint: 1.00 │ ├── left-join (hash) - │ │ ├── columns: y:4!null column5:5 x:6 + │ │ ├── columns: y:5!null column7:7 x:8 │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ │ ├── fd: ()-->(4,5) + │ │ ├── fd: ()-->(5,7) │ │ ├── limit hint: 1.00 │ │ ├── project - │ │ │ ├── columns: column5:5 y:4!null - │ │ │ ├── fd: ()-->(4,5) + │ │ │ ├── columns: column7:7 y:5!null + │ │ │ ├── fd: ()-->(5,7) │ │ │ ├── select - │ │ │ │ ├── columns: y:4!null - │ │ │ │ ├── fd: ()-->(4) + │ │ │ │ ├── columns: y:5!null + │ │ │ │ ├── fd: ()-->(5) │ │ │ │ ├── scan xy - │ │ │ │ │ └── columns: y:4 + │ │ │ │ │ └── columns: y:5 │ │ │ │ └── filters - │ │ │ │ └── y:4 = 0 [outer=(4), constraints=(/4: [/0 - /0]; tight), fd=()-->(4)] + │ │ │ │ └── y:5 = 0 [outer=(5), constraints=(/5: [/0 - /0]; tight), fd=()-->(5)] │ │ │ └── projections - │ │ │ └── CAST(NULL AS INT8) [as=column5:5] + │ │ │ └── CAST(NULL AS INT8) [as=column7:7] │ │ ├── scan xy - │ │ │ ├── columns: x:6!null - │ │ │ └── key: (6) + │ │ │ ├── columns: x:8!null + │ │ │ └── key: (8) │ │ └── filters - │ │ └── y:4 = x:6 [outer=(4,6), constraints=(/4: (/NULL - ]; /6: (/NULL - ]), fd=(4)==(6), (6)==(4)] + │ │ └── y:5 = x:8 [outer=(5,8), constraints=(/5: (/NULL - ]; /8: (/NULL - ]), fd=(5)==(8), (8)==(5)] │ └── filters - │ └── x:6 IS NULL [outer=(6), constraints=(/6: [/NULL - /NULL]; tight), fd=()-->(6)] + │ └── x:8 IS NULL [outer=(8), constraints=(/8: [/NULL - /NULL]; tight), fd=()-->(8)] └── 1 # EnsureUpsertDistinctOn should reduce non-nullable constant grouping column. @@ -1070,52 +1070,52 @@ ON CONFLICT (x) DO UPDATE SET y=1 ---- upsert xy ├── columns: - ├── canary column: 6 - ├── fetch columns: x:6 y:7 + ├── canary column: 8 + ├── fetch columns: x:8 y:9 ├── insert-mapping: - │ ├── y:4 => x:1 - │ └── column5:5 => y:2 + │ ├── y:5 => x:1 + │ └── column7:7 => y:2 ├── update-mapping: - │ └── upsert_y:10 => y:2 + │ └── upsert_y:13 => y:2 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: upsert_y:10 y:4!null column5:5 x:6 y:7 + ├── columns: upsert_y:13 y:5!null column7:7 x:8 y:9 ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(4-7,10) + ├── fd: ()-->(5,7-9,13) ├── left-join (hash) - │ ├── columns: y:4!null column5:5 x:6 y:7 + │ ├── columns: y:5!null column7:7 x:8 y:9 │ ├── cardinality: [0 - 1] │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ ├── key: () - │ ├── fd: ()-->(4-7) + │ ├── fd: ()-->(5,7-9) │ ├── max1-row - │ │ ├── columns: y:4!null column5:5 + │ │ ├── columns: y:5!null column7:7 │ │ ├── error: "UPSERT or INSERT...ON CONFLICT command cannot affect row a second time" │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(4,5) + │ │ ├── fd: ()-->(5,7) │ │ └── project - │ │ ├── columns: column5:5 y:4!null - │ │ ├── fd: ()-->(4,5) + │ │ ├── columns: column7:7 y:5!null + │ │ ├── fd: ()-->(5,7) │ │ ├── select - │ │ │ ├── columns: y:4!null - │ │ │ ├── fd: ()-->(4) + │ │ │ ├── columns: y:5!null + │ │ │ ├── fd: ()-->(5) │ │ │ ├── scan xy - │ │ │ │ └── columns: y:4 + │ │ │ │ └── columns: y:5 │ │ │ └── filters - │ │ │ └── y:4 = 0 [outer=(4), constraints=(/4: [/0 - /0]; tight), fd=()-->(4)] + │ │ │ └── y:5 = 0 [outer=(5), constraints=(/5: [/0 - /0]; tight), fd=()-->(5)] │ │ └── projections - │ │ └── CAST(NULL AS INT8) [as=column5:5] + │ │ └── CAST(NULL AS INT8) [as=column7:7] │ ├── scan xy - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:8!null y:9 + │ │ ├── key: (8) + │ │ └── fd: (8)-->(9) │ └── filters - │ └── y:4 = x:6 [outer=(4,6), constraints=(/4: (/NULL - ]; /6: (/NULL - ]), fd=(4)==(6), (6)==(4)] + │ └── y:5 = x:8 [outer=(5,8), constraints=(/5: (/NULL - ]; /8: (/NULL - ]), fd=(5)==(8), (8)==(5)] └── projections - └── CASE WHEN x:6 IS NULL THEN column5:5 ELSE 1 END [as=upsert_y:10, outer=(5,6)] + └── CASE WHEN x:8 IS NULL THEN column7:7 ELSE 1 END [as=upsert_y:13, outer=(7,8)] # UpsertDistinctOn should not reduce nullable constant grouping column. norm expect-not=ReduceNotNullGroupingCols @@ -1126,44 +1126,44 @@ ON CONFLICT (x) DO NOTHING insert xy ├── columns: ├── insert-mapping: - │ ├── y:4 => x:1 - │ └── column5:5 => y:2 + │ ├── y:5 => x:1 + │ └── column7:7 => y:2 ├── cardinality: [0 - 0] ├── volatile, mutations └── upsert-distinct-on - ├── columns: y:4 column5:5 - ├── grouping columns: y:4 - ├── lax-key: (4) - ├── fd: ()-->(4,5) + ├── columns: y:5 column7:7 + ├── grouping columns: y:5 + ├── lax-key: (5) + ├── fd: ()-->(5,7) ├── select - │ ├── columns: y:4 column5:5 x:6 - │ ├── fd: ()-->(4-6) + │ ├── columns: y:5 column7:7 x:8 + │ ├── fd: ()-->(5,7,8) │ ├── left-join (hash) - │ │ ├── columns: y:4 column5:5 x:6 + │ │ ├── columns: y:5 column7:7 x:8 │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ │ ├── fd: ()-->(4,5) + │ │ ├── fd: ()-->(5,7) │ │ ├── project - │ │ │ ├── columns: column5:5 y:4 - │ │ │ ├── fd: ()-->(4,5) + │ │ │ ├── columns: column7:7 y:5 + │ │ │ ├── fd: ()-->(5,7) │ │ │ ├── select - │ │ │ │ ├── columns: y:4 - │ │ │ │ ├── fd: ()-->(4) + │ │ │ │ ├── columns: y:5 + │ │ │ │ ├── fd: ()-->(5) │ │ │ │ ├── scan xy - │ │ │ │ │ └── columns: y:4 + │ │ │ │ │ └── columns: y:5 │ │ │ │ └── filters - │ │ │ │ └── y:4 IS NULL [outer=(4), constraints=(/4: [/NULL - /NULL]; tight), fd=()-->(4)] + │ │ │ │ └── y:5 IS NULL [outer=(5), constraints=(/5: [/NULL - /NULL]; tight), fd=()-->(5)] │ │ │ └── projections - │ │ │ └── CAST(NULL AS INT8) [as=column5:5] + │ │ │ └── CAST(NULL AS INT8) [as=column7:7] │ │ ├── scan xy - │ │ │ ├── columns: x:6!null - │ │ │ └── key: (6) + │ │ │ ├── columns: x:8!null + │ │ │ └── key: (8) │ │ └── filters - │ │ └── y:4 = x:6 [outer=(4,6), constraints=(/4: (/NULL - ]; /6: (/NULL - ]), fd=(4)==(6), (6)==(4)] + │ │ └── y:5 = x:8 [outer=(5,8), constraints=(/5: (/NULL - ]; /8: (/NULL - ]), fd=(5)==(8), (8)==(5)] │ └── filters - │ └── x:6 IS NULL [outer=(6), constraints=(/6: [/NULL - /NULL]; tight), fd=()-->(6)] + │ └── x:8 IS NULL [outer=(8), constraints=(/8: [/NULL - /NULL]; tight), fd=()-->(8)] └── aggregations - └── first-agg [as=column5:5, outer=(5)] - └── column5:5 + └── first-agg [as=column7:7, outer=(7)] + └── column7:7 # EnsureUpsertDistinctOn should not reduce nullable constant grouping column. norm expect-not=ReduceNotNullGroupingCols @@ -1173,53 +1173,53 @@ ON CONFLICT (x) DO UPDATE SET y=1 ---- upsert xy ├── columns: - ├── canary column: 6 - ├── fetch columns: x:6 y:7 + ├── canary column: 8 + ├── fetch columns: x:8 y:9 ├── insert-mapping: - │ ├── y:4 => x:1 - │ └── column5:5 => y:2 + │ ├── y:5 => x:1 + │ └── column7:7 => y:2 ├── update-mapping: - │ └── upsert_y:10 => y:2 + │ └── upsert_y:13 => y:2 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: upsert_y:10 y:4 column5:5 x:6 y:7 - ├── lax-key: (4,6) - ├── fd: ()-->(4,5), (6)-->(7,10) + ├── columns: upsert_y:13 y:5 column7:7 x:8 y:9 + ├── lax-key: (5,8) + ├── fd: ()-->(5,7), (8)-->(9,13) ├── left-join (hash) - │ ├── columns: y:4 column5:5 x:6 y:7 + │ ├── columns: y:5 column7:7 x:8 y:9 │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) - │ ├── lax-key: (4,6) - │ ├── fd: ()-->(4,5), (6)-->(7) + │ ├── lax-key: (5,8) + │ ├── fd: ()-->(5,7), (8)-->(9) │ ├── ensure-upsert-distinct-on - │ │ ├── columns: y:4 column5:5 - │ │ ├── grouping columns: y:4 + │ │ ├── columns: y:5 column7:7 + │ │ ├── grouping columns: y:5 │ │ ├── error: "UPSERT or INSERT...ON CONFLICT command cannot affect row a second time" - │ │ ├── lax-key: (4) - │ │ ├── fd: ()-->(4,5) + │ │ ├── lax-key: (5) + │ │ ├── fd: ()-->(5,7) │ │ ├── project - │ │ │ ├── columns: column5:5 y:4 - │ │ │ ├── fd: ()-->(4,5) + │ │ │ ├── columns: column7:7 y:5 + │ │ │ ├── fd: ()-->(5,7) │ │ │ ├── select - │ │ │ │ ├── columns: y:4 - │ │ │ │ ├── fd: ()-->(4) + │ │ │ │ ├── columns: y:5 + │ │ │ │ ├── fd: ()-->(5) │ │ │ │ ├── scan xy - │ │ │ │ │ └── columns: y:4 + │ │ │ │ │ └── columns: y:5 │ │ │ │ └── filters - │ │ │ │ └── y:4 IS NULL [outer=(4), constraints=(/4: [/NULL - /NULL]; tight), fd=()-->(4)] + │ │ │ │ └── y:5 IS NULL [outer=(5), constraints=(/5: [/NULL - /NULL]; tight), fd=()-->(5)] │ │ │ └── projections - │ │ │ └── CAST(NULL AS INT8) [as=column5:5] + │ │ │ └── CAST(NULL AS INT8) [as=column7:7] │ │ └── aggregations - │ │ └── first-agg [as=column5:5, outer=(5)] - │ │ └── column5:5 + │ │ └── first-agg [as=column7:7, outer=(7)] + │ │ └── column7:7 │ ├── scan xy - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:8!null y:9 + │ │ ├── key: (8) + │ │ └── fd: (8)-->(9) │ └── filters - │ └── y:4 = x:6 [outer=(4,6), constraints=(/4: (/NULL - ]; /6: (/NULL - ]), fd=(4)==(6), (6)==(4)] + │ └── y:5 = x:8 [outer=(5,8), constraints=(/5: (/NULL - ]; /8: (/NULL - ]), fd=(5)==(8), (8)==(5)] └── projections - └── CASE WHEN x:6 IS NULL THEN column5:5 ELSE 1 END [as=upsert_y:10, outer=(5,6)] + └── CASE WHEN x:8 IS NULL THEN column7:7 ELSE 1 END [as=upsert_y:13, outer=(7,8)] # Test removal of 2/3 grouping columns. norm expect=ReduceNotNullGroupingCols @@ -1229,47 +1229,47 @@ ON CONFLICT (a, b, c) DO UPDATE SET a=1 ---- upsert abc ├── columns: - ├── canary column: 9 - ├── fetch columns: a:9 b:10 c:11 + ├── canary column: 11 + ├── fetch columns: a:11 b:12 c:13 ├── insert-mapping: - │ ├── "?column?":7 => a:1 - │ ├── b:5 => b:2 - │ └── "?column?":8 => c:3 + │ ├── "?column?":9 => a:1 + │ ├── b:6 => b:2 + │ └── "?column?":10 => c:3 ├── update-mapping: - │ └── "?column?":7 => a:1 + │ └── "?column?":9 => a:1 ├── cardinality: [0 - 0] ├── volatile, mutations └── left-join (hash) - ├── columns: b:5!null "?column?":7!null "?column?":8!null a:9 b:10 c:11 + ├── columns: b:6!null "?column?":9!null "?column?":10!null a:11 b:12 c:13 ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) - ├── key: (5) - ├── fd: ()-->(7,8), (5)-->(9-11) + ├── key: (6) + ├── fd: ()-->(9,10), (6)-->(11-13) ├── ensure-upsert-distinct-on - │ ├── columns: b:5!null "?column?":7!null "?column?":8!null - │ ├── grouping columns: b:5!null + │ ├── columns: b:6!null "?column?":9!null "?column?":10!null + │ ├── grouping columns: b:6!null │ ├── error: "UPSERT or INSERT...ON CONFLICT command cannot affect row a second time" - │ ├── key: (5) - │ ├── fd: ()-->(7,8) + │ ├── key: (6) + │ ├── fd: ()-->(9,10) │ ├── project - │ │ ├── columns: "?column?":7!null "?column?":8!null b:5!null - │ │ ├── fd: ()-->(7,8) + │ │ ├── columns: "?column?":9!null "?column?":10!null b:6!null + │ │ ├── fd: ()-->(9,10) │ │ ├── scan abc - │ │ │ └── columns: b:5!null + │ │ │ └── columns: b:6!null │ │ └── projections - │ │ ├── 1 [as="?column?":7] - │ │ └── 2 [as="?column?":8] + │ │ ├── 1 [as="?column?":9] + │ │ └── 2 [as="?column?":10] │ └── aggregations - │ ├── const-agg [as="?column?":7, outer=(7)] - │ │ └── "?column?":7 - │ └── const-agg [as="?column?":8, outer=(8)] - │ └── "?column?":8 + │ ├── const-agg [as="?column?":9, outer=(9)] + │ │ └── "?column?":9 + │ └── const-agg [as="?column?":10, outer=(10)] + │ └── "?column?":10 ├── scan abc - │ ├── columns: a:9!null b:10!null c:11!null - │ └── key: (9-11) + │ ├── columns: a:11!null b:12!null c:13!null + │ └── key: (11-13) └── filters - ├── "?column?":7 = a:9 [outer=(7,9), constraints=(/7: (/NULL - ]; /9: (/NULL - ]), fd=(7)==(9), (9)==(7)] - ├── b:5 = b:10 [outer=(5,10), constraints=(/5: (/NULL - ]; /10: (/NULL - ]), fd=(5)==(10), (10)==(5)] - └── "?column?":8 = c:11 [outer=(8,11), constraints=(/8: (/NULL - ]; /11: (/NULL - ]), fd=(8)==(11), (11)==(8)] + ├── "?column?":9 = a:11 [outer=(9,11), constraints=(/9: (/NULL - ]; /11: (/NULL - ]), fd=(9)==(11), (11)==(9)] + ├── b:6 = b:12 [outer=(6,12), constraints=(/6: (/NULL - ]; /12: (/NULL - ]), fd=(6)==(12), (12)==(6)] + └── "?column?":10 = c:13 [outer=(10,13), constraints=(/10: (/NULL - ]; /13: (/NULL - ]), fd=(10)==(13), (13)==(10)] # Test removal of not-null column, but not nullable column. norm expect=ReduceNotNullGroupingCols @@ -1279,55 +1279,55 @@ ON CONFLICT (a, b, c) DO UPDATE SET c=2 ---- upsert abc ├── columns: - ├── canary column: 8 - ├── fetch columns: a:8 b:9 c:10 + ├── canary column: 10 + ├── fetch columns: a:10 b:11 c:12 ├── insert-mapping: - │ ├── "?column?":7 => a:1 - │ ├── b:5 => b:2 - │ └── c:6 => c:3 + │ ├── "?column?":9 => a:1 + │ ├── b:6 => b:2 + │ └── c:7 => c:3 ├── update-mapping: - │ └── upsert_c:14 => c:3 + │ └── upsert_c:17 => c:3 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: upsert_c:14!null b:5!null c:6!null "?column?":7 a:8 b:9 c:10 - ├── lax-key: (6-10) - ├── fd: ()-->(5,7), (6,8)-->(14) + ├── columns: upsert_c:17!null b:6!null c:7!null "?column?":9 a:10 b:11 c:12 + ├── lax-key: (7,9-12) + ├── fd: ()-->(6,9), (7,10)-->(17) ├── left-join (hash) - │ ├── columns: b:5!null c:6!null "?column?":7 a:8 b:9 c:10 + │ ├── columns: b:6!null c:7!null "?column?":9 a:10 b:11 c:12 │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) - │ ├── lax-key: (6-10) - │ ├── fd: ()-->(5,7) + │ ├── lax-key: (7,9-12) + │ ├── fd: ()-->(6,9) │ ├── ensure-upsert-distinct-on - │ │ ├── columns: b:5!null c:6!null "?column?":7 - │ │ ├── grouping columns: c:6!null "?column?":7 + │ │ ├── columns: b:6!null c:7!null "?column?":9 + │ │ ├── grouping columns: c:7!null "?column?":9 │ │ ├── error: "UPSERT or INSERT...ON CONFLICT command cannot affect row a second time" - │ │ ├── lax-key: (6,7) - │ │ ├── fd: ()-->(5,7) + │ │ ├── lax-key: (7,9) + │ │ ├── fd: ()-->(6,9) │ │ ├── project - │ │ │ ├── columns: "?column?":7 b:5!null c:6!null - │ │ │ ├── fd: ()-->(5,7) + │ │ │ ├── columns: "?column?":9 b:6!null c:7!null + │ │ │ ├── fd: ()-->(6,9) │ │ │ ├── select - │ │ │ │ ├── columns: b:5!null c:6!null - │ │ │ │ ├── fd: ()-->(5) + │ │ │ │ ├── columns: b:6!null c:7!null + │ │ │ │ ├── fd: ()-->(6) │ │ │ │ ├── scan abc - │ │ │ │ │ └── columns: b:5!null c:6!null + │ │ │ │ │ └── columns: b:6!null c:7!null │ │ │ │ └── filters - │ │ │ │ └── b:5 = 1 [outer=(5), constraints=(/5: [/1 - /1]; tight), fd=()-->(5)] + │ │ │ │ └── b:6 = 1 [outer=(6), constraints=(/6: [/1 - /1]; tight), fd=()-->(6)] │ │ │ └── projections - │ │ │ └── CAST(NULL AS INT8) [as="?column?":7] + │ │ │ └── CAST(NULL AS INT8) [as="?column?":9] │ │ └── aggregations - │ │ └── const-agg [as=b:5, outer=(5)] - │ │ └── b:5 + │ │ └── const-agg [as=b:6, outer=(6)] + │ │ └── b:6 │ ├── scan abc - │ │ ├── columns: a:8!null b:9!null c:10!null - │ │ └── key: (8-10) + │ │ ├── columns: a:10!null b:11!null c:12!null + │ │ └── key: (10-12) │ └── filters - │ ├── "?column?":7 = a:8 [outer=(7,8), constraints=(/7: (/NULL - ]; /8: (/NULL - ]), fd=(7)==(8), (8)==(7)] - │ ├── b:5 = b:9 [outer=(5,9), constraints=(/5: (/NULL - ]; /9: (/NULL - ]), fd=(5)==(9), (9)==(5)] - │ └── c:6 = c:10 [outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ]), fd=(6)==(10), (10)==(6)] + │ ├── "?column?":9 = a:10 [outer=(9,10), constraints=(/9: (/NULL - ]; /10: (/NULL - ]), fd=(9)==(10), (10)==(9)] + │ ├── b:6 = b:11 [outer=(6,11), constraints=(/6: (/NULL - ]; /11: (/NULL - ]), fd=(6)==(11), (11)==(6)] + │ └── c:7 = c:12 [outer=(7,12), constraints=(/7: (/NULL - ]; /12: (/NULL - ]), fd=(7)==(12), (12)==(7)] └── projections - └── CASE WHEN a:8 IS NULL THEN c:6 ELSE 2 END [as=upsert_c:14, outer=(6,8)] + └── CASE WHEN a:10 IS NULL THEN c:7 ELSE 2 END [as=upsert_c:17, outer=(7,10)] # -------------------------------------------------- # EliminateAggDistinctForKeys @@ -1339,18 +1339,18 @@ norm expect=EliminateAggDistinctForKeys SELECT sum(DISTINCT k), sum(DISTINCT i) FROM a ---- scalar-group-by - ├── columns: sum:6 sum:7 + ├── columns: sum:7 sum:8 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(6,7) + ├── fd: ()-->(7,8) ├── scan a │ ├── columns: k:1!null i:2!null │ ├── key: (1) │ └── fd: (1)-->(2) └── aggregations - ├── sum [as=sum:6, outer=(1)] + ├── sum [as=sum:7, outer=(1)] │ └── k:1 - └── agg-distinct [as=sum:7, outer=(2)] + └── agg-distinct [as=sum:8, outer=(2)] └── sum └── i:2 @@ -1358,41 +1358,41 @@ norm expect=EliminateAggDistinctForKeys SELECT string_agg(DISTINCT s, ', ') FROM s ---- scalar-group-by - ├── columns: string_agg:3 + ├── columns: string_agg:4 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(3) + ├── fd: ()-->(4) ├── project - │ ├── columns: column2:2!null s:1!null + │ ├── columns: column3:3!null s:1!null │ ├── key: (1) - │ ├── fd: ()-->(2) + │ ├── fd: ()-->(3) │ ├── scan s │ │ ├── columns: s:1!null │ │ └── key: (1) │ └── projections - │ └── ', ' [as=column2:2] + │ └── ', ' [as=column3:3] └── aggregations - └── string-agg [as=string_agg:3, outer=(1,2)] + └── string-agg [as=string_agg:4, outer=(1,3)] ├── s:1 - └── column2:2 + └── column3:3 # GroupBy with key argument. norm expect=EliminateAggDistinctForKeys SELECT sum(DISTINCT k) FROM a GROUP BY i ---- project - ├── columns: sum:6!null + ├── columns: sum:7!null └── group-by - ├── columns: i:2!null sum:6!null + ├── columns: i:2!null sum:7!null ├── grouping columns: i:2!null ├── key: (2) - ├── fd: (2)-->(6) + ├── fd: (2)-->(7) ├── scan a │ ├── columns: k:1!null i:2!null │ ├── key: (1) │ └── fd: (1)-->(2) └── aggregations - └── sum [as=sum:6, outer=(1)] + └── sum [as=sum:7, outer=(1)] └── k:1 # GroupBy with no key. The AggDistinct is instead pushed into the GroupBy by @@ -1401,12 +1401,12 @@ norm expect-not=EliminateAggDistinctForKeys SELECT sum(DISTINCT a) FROM abc GROUP BY b ---- project - ├── columns: sum:4!null + ├── columns: sum:5!null └── group-by - ├── columns: b:2!null sum:4!null + ├── columns: b:2!null sum:5!null ├── grouping columns: b:2!null ├── key: (2) - ├── fd: (2)-->(4) + ├── fd: (2)-->(5) ├── distinct-on │ ├── columns: a:1!null b:2!null │ ├── grouping columns: a:1!null b:2!null @@ -1414,7 +1414,7 @@ project │ └── scan abc │ └── columns: a:1!null b:2!null └── aggregations - └── sum [as=sum:4, outer=(1)] + └── sum [as=sum:5, outer=(1)] └── a:1 # GroupBy with composite key formed by argument plus grouping columns. @@ -1422,17 +1422,17 @@ norm expect=EliminateAggDistinctForKeys SELECT sum(DISTINCT a) FROM abc GROUP BY b, c ---- project - ├── columns: sum:4!null + ├── columns: sum:5!null └── group-by - ├── columns: b:2!null c:3!null sum:4!null + ├── columns: b:2!null c:3!null sum:5!null ├── grouping columns: b:2!null c:3!null ├── key: (2,3) - ├── fd: (2,3)-->(4) + ├── fd: (2,3)-->(5) ├── scan abc │ ├── columns: a:1!null b:2!null c:3!null │ └── key: (1-3) └── aggregations - └── sum [as=sum:4, outer=(1)] + └── sum [as=sum:5, outer=(1)] └── a:1 # GroupBy with multiple aggregations simplified. @@ -1440,20 +1440,20 @@ norm expect=EliminateAggDistinctForKeys SELECT sum(DISTINCT i), avg(DISTINCT f) FROM a GROUP BY k ---- project - ├── columns: sum:6!null avg:7 + ├── columns: sum:7!null avg:8 └── group-by - ├── columns: k:1!null sum:6!null avg:7 + ├── columns: k:1!null sum:7!null avg:8 ├── grouping columns: k:1!null ├── key: (1) - ├── fd: (1)-->(6,7) + ├── fd: (1)-->(7,8) ├── scan a │ ├── columns: k:1!null i:2!null f:3 │ ├── key: (1) │ └── fd: (1)-->(2,3), (2,3)~~>(1) └── aggregations - ├── sum [as=sum:6, outer=(2)] + ├── sum [as=sum:7, outer=(2)] │ └── i:2 - └── avg [as=avg:7, outer=(3)] + └── avg [as=avg:8, outer=(3)] └── f:3 # GroupBy where only some aggregations are simplified (the table has @@ -1462,22 +1462,22 @@ norm expect=EliminateAggDistinctForKeys SELECT sum(DISTINCT u), stddev(DISTINCT w), avg(DISTINCT z) FROM uvwz GROUP BY v ---- project - ├── columns: sum:6!null stddev:7 avg:8!null + ├── columns: sum:7!null stddev:8 avg:9!null └── group-by - ├── columns: v:2!null sum:6!null stddev:7 avg:8!null + ├── columns: v:2!null sum:7!null stddev:8 avg:9!null ├── grouping columns: v:2!null ├── key: (2) - ├── fd: (2)-->(6-8) + ├── fd: (2)-->(7-9) ├── scan uvwz │ ├── columns: u:1!null v:2!null w:3!null z:4!null │ ├── key: (2,3) │ └── fd: (1,2)-->(3,4), (2,3)-->(1,4) └── aggregations - ├── sum [as=sum:6, outer=(1)] + ├── sum [as=sum:7, outer=(1)] │ └── u:1 - ├── std-dev [as=stddev:7, outer=(3)] + ├── std-dev [as=stddev:8, outer=(3)] │ └── w:3 - └── agg-distinct [as=avg:8, outer=(4)] + └── agg-distinct [as=avg:9, outer=(4)] └── avg └── z:4 @@ -1491,44 +1491,44 @@ norm expect=EliminateAggFilteredDistinctForKeys SELECT sum(DISTINCT k) FILTER (WHERE k > 0), sum(DISTINCT i) FILTER (WHERE i > 0) FROM a ---- scalar-group-by - ├── columns: sum:7 sum:9 + ├── columns: sum:8 sum:10 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(7,9) + ├── fd: ()-->(8,10) ├── project - │ ├── columns: column6:6!null column8:8!null k:1!null i:2!null + │ ├── columns: column7:7!null column9:9!null k:1!null i:2!null │ ├── key: (1) - │ ├── fd: (1)-->(2,6), (2)-->(8) + │ ├── fd: (1)-->(2,7), (2)-->(9) │ ├── scan a │ │ ├── columns: k:1!null i:2!null │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ └── projections - │ ├── k:1 > 0 [as=column6:6, outer=(1)] - │ └── i:2 > 0 [as=column8:8, outer=(2)] + │ ├── k:1 > 0 [as=column7:7, outer=(1)] + │ └── i:2 > 0 [as=column9:9, outer=(2)] └── aggregations - ├── agg-filter [as=sum:7, outer=(1,6)] + ├── agg-filter [as=sum:8, outer=(1,7)] │ ├── sum │ │ └── k:1 - │ └── column6:6 - └── agg-filter [as=sum:9, outer=(2,8)] + │ └── column7:7 + └── agg-filter [as=sum:10, outer=(2,9)] ├── agg-distinct │ └── sum │ └── i:2 - └── column8:8 + └── column9:9 norm expect=EliminateAggFilteredDistinctForKeys SELECT string_agg(DISTINCT s, ',') FILTER (WHERE s > 'a') FROM s ---- scalar-group-by - ├── columns: string_agg:4 + ├── columns: string_agg:5 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(4) + ├── fd: ()-->(5) ├── project - │ ├── columns: column2:2!null s:1!null + │ ├── columns: column3:3!null s:1!null │ ├── key: (1) - │ ├── fd: ()-->(2) + │ ├── fd: ()-->(3) │ ├── select │ │ ├── columns: s:1!null │ │ ├── key: (1) @@ -1538,121 +1538,121 @@ scalar-group-by │ │ └── filters │ │ └── s:1 > 'a' [outer=(1), constraints=(/1: [/e'a\x00' - ]; tight)] │ └── projections - │ └── ',' [as=column2:2] + │ └── ',' [as=column3:3] └── aggregations - └── string-agg [as=string_agg:4, outer=(1,2)] + └── string-agg [as=string_agg:5, outer=(1,3)] ├── s:1 - └── column2:2 + └── column3:3 # GroupBy with key argument. norm expect=EliminateAggFilteredDistinctForKeys SELECT sum(DISTINCT k) FILTER (WHERE f > 0) FROM a GROUP BY i ---- project - ├── columns: sum:7 + ├── columns: sum:8 └── group-by - ├── columns: i:2!null sum:7 + ├── columns: i:2!null sum:8 ├── grouping columns: i:2!null ├── key: (2) - ├── fd: (2)-->(7) + ├── fd: (2)-->(8) ├── project - │ ├── columns: column6:6 k:1!null i:2!null + │ ├── columns: column7:7 k:1!null i:2!null │ ├── key: (1) - │ ├── fd: (1)-->(2,6) + │ ├── fd: (1)-->(2,7) │ ├── scan a │ │ ├── columns: k:1!null i:2!null f:3 │ │ ├── key: (1) │ │ └── fd: (1)-->(2,3), (2,3)~~>(1) │ └── projections - │ └── f:3 > 0.0 [as=column6:6, outer=(3)] + │ └── f:3 > 0.0 [as=column7:7, outer=(3)] └── aggregations - └── agg-filter [as=sum:7, outer=(1,6)] + └── agg-filter [as=sum:8, outer=(1,7)] ├── sum │ └── k:1 - └── column6:6 + └── column7:7 # GroupBy with no key. norm expect-not=EliminateAggFilteredDistinctForKeys SELECT sum(DISTINCT a) FILTER (WHERE c > 0) FROM abc GROUP BY b ---- project - ├── columns: sum:5 + ├── columns: sum:6 └── group-by - ├── columns: b:2!null sum:5 + ├── columns: b:2!null sum:6 ├── grouping columns: b:2!null ├── key: (2) - ├── fd: (2)-->(5) + ├── fd: (2)-->(6) ├── project - │ ├── columns: column4:4!null a:1!null b:2!null + │ ├── columns: column5:5!null a:1!null b:2!null │ ├── scan abc │ │ ├── columns: a:1!null b:2!null c:3!null │ │ └── key: (1-3) │ └── projections - │ └── c:3 > 0 [as=column4:4, outer=(3)] + │ └── c:3 > 0 [as=column5:5, outer=(3)] └── aggregations - └── agg-filter [as=sum:5, outer=(1,4)] + └── agg-filter [as=sum:6, outer=(1,5)] ├── agg-distinct │ └── sum │ └── a:1 - └── column4:4 + └── column5:5 # GroupBy with composite key formed by argument plus grouping columns. norm expect=EliminateAggFilteredDistinctForKeys SELECT sum(DISTINCT a) FILTER (WHERE c > 0) FROM abc GROUP BY b, c ---- project - ├── columns: sum:5 + ├── columns: sum:6 └── group-by - ├── columns: b:2!null c:3!null sum:5 + ├── columns: b:2!null c:3!null sum:6 ├── grouping columns: b:2!null c:3!null ├── key: (2,3) - ├── fd: (2,3)-->(5) + ├── fd: (2,3)-->(6) ├── project - │ ├── columns: column4:4!null a:1!null b:2!null c:3!null + │ ├── columns: column5:5!null a:1!null b:2!null c:3!null │ ├── key: (1-3) - │ ├── fd: (3)-->(4) + │ ├── fd: (3)-->(5) │ ├── scan abc │ │ ├── columns: a:1!null b:2!null c:3!null │ │ └── key: (1-3) │ └── projections - │ └── c:3 > 0 [as=column4:4, outer=(3)] + │ └── c:3 > 0 [as=column5:5, outer=(3)] └── aggregations - └── agg-filter [as=sum:5, outer=(1,4)] + └── agg-filter [as=sum:6, outer=(1,5)] ├── sum │ └── a:1 - └── column4:4 + └── column5:5 # GroupBy with multiple aggregations simplified. norm expect=EliminateAggFilteredDistinctForKeys SELECT sum(DISTINCT i) FILTER (WHERE f > 0), avg(DISTINCT f) FILTER (WHERE i > 0) FROM a GROUP BY k ---- project - ├── columns: sum:7 avg:9 + ├── columns: sum:8 avg:10 └── group-by - ├── columns: k:1!null sum:7 avg:9 + ├── columns: k:1!null sum:8 avg:10 ├── grouping columns: k:1!null ├── key: (1) - ├── fd: (1)-->(7,9) + ├── fd: (1)-->(8,10) ├── project - │ ├── columns: column6:6 column8:8!null k:1!null i:2!null f:3 + │ ├── columns: column7:7 column9:9!null k:1!null i:2!null f:3 │ ├── key: (1) - │ ├── fd: (1)-->(2,3,6), (2,3)~~>(1), (2)-->(8) + │ ├── fd: (1)-->(2,3,7), (2,3)~~>(1), (2)-->(9) │ ├── scan a │ │ ├── columns: k:1!null i:2!null f:3 │ │ ├── key: (1) │ │ └── fd: (1)-->(2,3), (2,3)~~>(1) │ └── projections - │ ├── f:3 > 0.0 [as=column6:6, outer=(3)] - │ └── i:2 > 0 [as=column8:8, outer=(2)] + │ ├── f:3 > 0.0 [as=column7:7, outer=(3)] + │ └── i:2 > 0 [as=column9:9, outer=(2)] └── aggregations - ├── agg-filter [as=sum:7, outer=(2,6)] + ├── agg-filter [as=sum:8, outer=(2,7)] │ ├── sum │ │ └── i:2 - │ └── column6:6 - └── agg-filter [as=avg:9, outer=(3,8)] + │ └── column7:7 + └── agg-filter [as=avg:10, outer=(3,9)] ├── avg │ └── f:3 - └── column8:8 + └── column9:9 # GroupBy where only some aggregations are simplified (the table has # keys u,v and v,w). @@ -1665,38 +1665,38 @@ FROM uvwz GROUP BY v ---- project - ├── columns: sum:7 stddev:9 avg:11 + ├── columns: sum:8 stddev:10 avg:12 └── group-by - ├── columns: v:2!null sum:7 stddev:9 avg:11 + ├── columns: v:2!null sum:8 stddev:10 avg:12 ├── grouping columns: v:2!null ├── key: (2) - ├── fd: (2)-->(7,9,11) + ├── fd: (2)-->(8,10,12) ├── project - │ ├── columns: column6:6!null column8:8!null column10:10!null u:1!null v:2!null w:3!null z:4!null + │ ├── columns: column7:7!null column9:9!null column11:11!null u:1!null v:2!null w:3!null z:4!null │ ├── key: (2,3) - │ ├── fd: (1,2)-->(3,4), (2,3)-->(1,4), (1)-->(6), (3)-->(8), (4)-->(10) + │ ├── fd: (1,2)-->(3,4), (2,3)-->(1,4), (1)-->(7), (3)-->(9), (4)-->(11) │ ├── scan uvwz │ │ ├── columns: u:1!null v:2!null w:3!null z:4!null │ │ ├── key: (2,3) │ │ └── fd: (1,2)-->(3,4), (2,3)-->(1,4) │ └── projections - │ ├── u:1 > 0 [as=column6:6, outer=(1)] - │ ├── w:3 > 0 [as=column8:8, outer=(3)] - │ └── z:4 > 0 [as=column10:10, outer=(4)] + │ ├── u:1 > 0 [as=column7:7, outer=(1)] + │ ├── w:3 > 0 [as=column9:9, outer=(3)] + │ └── z:4 > 0 [as=column11:11, outer=(4)] └── aggregations - ├── agg-filter [as=sum:7, outer=(1,6)] + ├── agg-filter [as=sum:8, outer=(1,7)] │ ├── sum │ │ └── u:1 - │ └── column6:6 - ├── agg-filter [as=stddev:9, outer=(3,8)] + │ └── column7:7 + ├── agg-filter [as=stddev:10, outer=(3,9)] │ ├── std-dev │ │ └── w:3 - │ └── column8:8 - └── agg-filter [as=avg:11, outer=(4,10)] + │ └── column9:9 + └── agg-filter [as=avg:12, outer=(4,11)] ├── agg-distinct │ └── avg │ └── z:4 - └── column10:10 + └── column11:11 # -------------------------------------------------- # EliminateDistinctNoColumns @@ -1751,60 +1751,60 @@ ON CONFLICT (s, i) DO NOTHING insert a ├── columns: ├── insert-mapping: - │ ├── "?column?":11 => k:1 - │ ├── i:7 => i:2 - │ ├── column13:13 => f:3 - │ ├── "?column?":12 => s:4 - │ └── column14:14 => j:5 + │ ├── "?column?":13 => k:1 + │ ├── i:8 => i:2 + │ ├── column15:15 => f:3 + │ ├── "?column?":14 => s:4 + │ └── column16:16 => j:5 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: i:7!null "?column?":11!null "?column?":12!null column13:13 column14:14 + ├── columns: i:8!null "?column?":13!null "?column?":14!null column15:15 column16:16 ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(7,11-14) + ├── fd: ()-->(8,13-16) └── limit - ├── columns: i:7!null "?column?":11!null "?column?":12!null column13:13 column14:14 i:16 s:18 + ├── columns: i:8!null "?column?":13!null "?column?":14!null column15:15 column16:16 i:18 s:20 ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(7,11-14,16,18) + ├── fd: ()-->(8,13-16,18,20) ├── select - │ ├── columns: i:7!null "?column?":11!null "?column?":12!null column13:13 column14:14 i:16 s:18 - │ ├── fd: ()-->(7,11-14,18) + │ ├── columns: i:8!null "?column?":13!null "?column?":14!null column15:15 column16:16 i:18 s:20 + │ ├── fd: ()-->(8,13-16,20) │ ├── limit hint: 1.00 │ ├── left-join (hash) - │ │ ├── columns: i:7!null "?column?":11!null "?column?":12!null column13:13 column14:14 i:16 s:18 + │ │ ├── columns: i:8!null "?column?":13!null "?column?":14!null column15:15 column16:16 i:18 s:20 │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ │ ├── fd: ()-->(7,11-14) + │ │ ├── fd: ()-->(8,13-16) │ │ ├── limit hint: 1.00 │ │ ├── project - │ │ │ ├── columns: column13:13 column14:14 "?column?":11!null "?column?":12!null i:7!null - │ │ │ ├── fd: ()-->(7,11-14) + │ │ │ ├── columns: column15:15 column16:16 "?column?":13!null "?column?":14!null i:8!null + │ │ │ ├── fd: ()-->(8,13-16) │ │ │ ├── select - │ │ │ │ ├── columns: i:7!null - │ │ │ │ ├── fd: ()-->(7) + │ │ │ │ ├── columns: i:8!null + │ │ │ │ ├── fd: ()-->(8) │ │ │ │ ├── scan a - │ │ │ │ │ └── columns: i:7!null + │ │ │ │ │ └── columns: i:8!null │ │ │ │ └── filters - │ │ │ │ └── i:7 = 1 [outer=(7), constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] + │ │ │ │ └── i:8 = 1 [outer=(8), constraints=(/8: [/1 - /1]; tight), fd=()-->(8)] │ │ │ └── projections - │ │ │ ├── CAST(NULL AS FLOAT8) [as=column13:13] - │ │ │ ├── CAST(NULL AS JSONB) [as=column14:14] - │ │ │ ├── 1 [as="?column?":11] - │ │ │ └── 'foo' [as="?column?":12] + │ │ │ ├── CAST(NULL AS FLOAT8) [as=column15:15] + │ │ │ ├── CAST(NULL AS JSONB) [as=column16:16] + │ │ │ ├── 1 [as="?column?":13] + │ │ │ └── 'foo' [as="?column?":14] │ │ ├── select - │ │ │ ├── columns: i:16!null s:18!null - │ │ │ ├── key: (16) - │ │ │ ├── fd: ()-->(18) + │ │ │ ├── columns: i:18!null s:20!null + │ │ │ ├── key: (18) + │ │ │ ├── fd: ()-->(20) │ │ │ ├── scan a - │ │ │ │ ├── columns: i:16!null s:18!null - │ │ │ │ └── key: (16,18) + │ │ │ │ ├── columns: i:18!null s:20!null + │ │ │ │ └── key: (18,20) │ │ │ └── filters - │ │ │ └── s:18 = 'foo' [outer=(18), constraints=(/18: [/'foo' - /'foo']; tight), fd=()-->(18)] + │ │ │ └── s:20 = 'foo' [outer=(20), constraints=(/20: [/'foo' - /'foo']; tight), fd=()-->(20)] │ │ └── filters - │ │ └── i:7 = i:16 [outer=(7,16), constraints=(/7: (/NULL - ]; /16: (/NULL - ]), fd=(7)==(16), (16)==(7)] + │ │ └── i:8 = i:18 [outer=(8,18), constraints=(/8: (/NULL - ]; /18: (/NULL - ]), fd=(8)==(18), (18)==(8)] │ └── filters - │ └── s:18 IS NULL [outer=(18), constraints=(/18: [/NULL - /NULL]; tight), fd=()-->(18)] + │ └── s:20 IS NULL [outer=(20), constraints=(/20: [/NULL - /NULL]; tight), fd=()-->(20)] └── 1 # -------------------------------------------------- @@ -1816,21 +1816,21 @@ norm expect=EliminateEnsureDistinctNoColumns SELECT (SELECT x FROM xy WHERE y=i) FROM a WHERE k=5 ---- project - ├── columns: x:8 + ├── columns: x:10 ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(8) + ├── fd: ()-->(10) ├── max1-row - │ ├── columns: k:1!null i:2!null xy.x:6 y:7 + │ ├── columns: k:1!null i:2!null xy.x:7 y:8 │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(1,2,6,7) + │ ├── fd: ()-->(1,2,7,8) │ └── left-join (hash) - │ ├── columns: k:1!null i:2!null xy.x:6 y:7 + │ ├── columns: k:1!null i:2!null xy.x:7 y:8 │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) - │ ├── key: (6) - │ ├── fd: ()-->(1,2), (6)-->(7) + │ ├── key: (7) + │ ├── fd: ()-->(1,2), (7)-->(8) │ ├── select │ │ ├── columns: k:1!null i:2!null │ │ ├── cardinality: [0 - 1] @@ -1843,13 +1843,13 @@ project │ │ └── filters │ │ └── k:1 = 5 [outer=(1), constraints=(/1: [/5 - /5]; tight), fd=()-->(1)] │ ├── scan xy - │ │ ├── columns: xy.x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: xy.x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ └── filters - │ └── y:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] + │ └── y:8 = i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] └── projections - └── xy.x:6 [as=x:8, outer=(6)] + └── xy.x:7 [as=x:10, outer=(7)] # EnsureUpsertDistinctOn case. norm expect=EliminateEnsureDistinctNoColumns @@ -1858,59 +1858,59 @@ ON CONFLICT (s, i) DO UPDATE SET f=1.1 ---- upsert a ├── columns: - ├── canary column: 15 - ├── fetch columns: k:15 i:16 f:17 s:18 j:19 + ├── canary column: 17 + ├── fetch columns: k:17 i:18 f:19 s:20 j:21 ├── insert-mapping: - │ ├── "?column?":11 => k:1 - │ ├── i:7 => i:2 - │ ├── column13:13 => f:3 - │ ├── "?column?":12 => s:4 - │ └── column14:14 => j:5 + │ ├── "?column?":13 => k:1 + │ ├── i:8 => i:2 + │ ├── column15:15 => f:3 + │ ├── "?column?":14 => s:4 + │ └── column16:16 => j:5 ├── update-mapping: - │ └── upsert_f:23 => f:3 + │ └── upsert_f:26 => f:3 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: upsert_f:23 i:7!null "?column?":11!null "?column?":12!null column13:13 column14:14 k:15 i:16 f:17 s:18 j:19 + ├── columns: upsert_f:26 i:8!null "?column?":13!null "?column?":14!null column15:15 column16:16 k:17 i:18 f:19 s:20 j:21 ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(7,11-19,23) + ├── fd: ()-->(8,13-21,26) ├── left-join (hash) - │ ├── columns: i:7!null "?column?":11!null "?column?":12!null column13:13 column14:14 k:15 i:16 f:17 s:18 j:19 + │ ├── columns: i:8!null "?column?":13!null "?column?":14!null column15:15 column16:16 k:17 i:18 f:19 s:20 j:21 │ ├── cardinality: [0 - 1] │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ ├── key: () - │ ├── fd: ()-->(7,11-19) + │ ├── fd: ()-->(8,13-21) │ ├── max1-row - │ │ ├── columns: i:7!null "?column?":11!null "?column?":12!null column13:13 column14:14 + │ │ ├── columns: i:8!null "?column?":13!null "?column?":14!null column15:15 column16:16 │ │ ├── error: "UPSERT or INSERT...ON CONFLICT command cannot affect row a second time" │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(7,11-14) + │ │ ├── fd: ()-->(8,13-16) │ │ └── project - │ │ ├── columns: column13:13 column14:14 "?column?":11!null "?column?":12!null i:7!null - │ │ ├── fd: ()-->(7,11-14) + │ │ ├── columns: column15:15 column16:16 "?column?":13!null "?column?":14!null i:8!null + │ │ ├── fd: ()-->(8,13-16) │ │ ├── select - │ │ │ ├── columns: i:7!null - │ │ │ ├── fd: ()-->(7) + │ │ │ ├── columns: i:8!null + │ │ │ ├── fd: ()-->(8) │ │ │ ├── scan a - │ │ │ │ └── columns: i:7!null + │ │ │ │ └── columns: i:8!null │ │ │ └── filters - │ │ │ └── i:7 = 1 [outer=(7), constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] + │ │ │ └── i:8 = 1 [outer=(8), constraints=(/8: [/1 - /1]; tight), fd=()-->(8)] │ │ └── projections - │ │ ├── CAST(NULL AS FLOAT8) [as=column13:13] - │ │ ├── CAST(NULL AS JSONB) [as=column14:14] - │ │ ├── 1 [as="?column?":11] - │ │ └── 'foo' [as="?column?":12] + │ │ ├── CAST(NULL AS FLOAT8) [as=column15:15] + │ │ ├── CAST(NULL AS JSONB) [as=column16:16] + │ │ ├── 1 [as="?column?":13] + │ │ └── 'foo' [as="?column?":14] │ ├── scan a - │ │ ├── columns: k:15!null i:16!null f:17 s:18!null j:19 - │ │ ├── key: (15) - │ │ └── fd: (15)-->(16-19), (16,18)-->(15,17,19), (16,17)~~>(15,18,19) + │ │ ├── columns: k:17!null i:18!null f:19 s:20!null j:21 + │ │ ├── key: (17) + │ │ └── fd: (17)-->(18-21), (18,20)-->(17,19,21), (18,19)~~>(17,20,21) │ └── filters - │ ├── i:7 = i:16 [outer=(7,16), constraints=(/7: (/NULL - ]; /16: (/NULL - ]), fd=(7)==(16), (16)==(7)] - │ └── "?column?":12 = s:18 [outer=(12,18), constraints=(/12: (/NULL - ]; /18: (/NULL - ]), fd=(12)==(18), (18)==(12)] + │ ├── i:8 = i:18 [outer=(8,18), constraints=(/8: (/NULL - ]; /18: (/NULL - ]), fd=(8)==(18), (18)==(8)] + │ └── "?column?":14 = s:20 [outer=(14,20), constraints=(/14: (/NULL - ]; /20: (/NULL - ]), fd=(14)==(20), (20)==(14)] └── projections - └── CASE WHEN k:15 IS NULL THEN column13:13 ELSE 1.1 END [as=upsert_f:23, outer=(13,15)] + └── CASE WHEN k:17 IS NULL THEN column15:15 ELSE 1.1 END [as=upsert_f:26, outer=(15,17)] # -------------------------------------------------- # EliminateDistinctOnValues @@ -1936,10 +1936,10 @@ ON a=x AND b=y AND c=z WHERE x > 100 OR b > 100 ---- project - ├── columns: x:1!null y:2!null z:3!null a:4 b:5 c:6 "?column?":7!null + ├── columns: x:1!null y:2!null z:3!null a:4 b:5 c:6 "?column?":8!null ├── cardinality: [0 - 2] ├── immutable - ├── fd: (1)-->(7) + ├── fd: (1)-->(8) ├── select │ ├── columns: column1:1!null column2:2!null column3:3!null a:4 b:5 c:6 │ ├── cardinality: [0 - 2] @@ -1962,7 +1962,7 @@ project │ └── filters │ └── (column1:1 > 100) OR (b:5 > 100) [outer=(1,5)] └── projections - └── column1:1 + 1 [as="?column?":7, outer=(1), immutable] + └── column1:1 + 1 [as="?column?":8, outer=(1), immutable] # Right input of left join does not have a key, so left side may have dups. norm expect-not=EliminateDistinctOnValues @@ -2065,23 +2065,23 @@ LEFT JOIN (VALUES (1), (2)) t(x) ON k=x ---- distinct-on - ├── columns: k:1!null x:6 - ├── grouping columns: column1:6 - ├── key: (6) - ├── fd: (6)-->(1) + ├── columns: k:1!null x:7 + ├── grouping columns: column1:7 + ├── key: (7) + ├── fd: (7)-->(1) ├── left-join (hash) - │ ├── columns: k:1!null column1:6 + │ ├── columns: k:1!null column1:7 │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) │ ├── scan a │ │ ├── columns: k:1!null │ │ └── key: (1) │ ├── values - │ │ ├── columns: column1:6!null + │ │ ├── columns: column1:7!null │ │ ├── cardinality: [2 - 2] │ │ ├── (1,) │ │ └── (2,) │ └── filters - │ └── k:1 = column1:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ └── k:1 = column1:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] └── aggregations └── first-agg [as=k:1, outer=(1)] └── k:1 @@ -2202,46 +2202,46 @@ ON CONFLICT (s, i) DO NOTHING insert a ├── columns: ├── insert-mapping: - │ ├── column1:6 => k:1 - │ ├── column3:8 => i:2 - │ ├── column9:9 => f:3 - │ ├── column2:7 => s:4 - │ └── column10:10 => j:5 + │ ├── column1:7 => k:1 + │ ├── column3:9 => i:2 + │ ├── column10:10 => f:3 + │ ├── column2:8 => s:4 + │ └── column11:11 => j:5 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: column1:6!null column2:7 column3:8 column9:9 column10:10 + ├── columns: column1:7!null column2:8 column3:9 column10:10 column11:11 ├── cardinality: [0 - 2] - ├── fd: ()-->(9,10) + ├── fd: ()-->(10,11) └── select - ├── columns: column1:6!null column2:7 column3:8 column9:9 column10:10 i:12 s:14 + ├── columns: column1:7!null column2:8 column3:9 column10:10 column11:11 i:13 s:15 ├── cardinality: [0 - 2] - ├── fd: ()-->(9,10,14) + ├── fd: ()-->(10,11,15) ├── left-join (hash) - │ ├── columns: column1:6!null column2:7 column3:8 column9:9 column10:10 i:12 s:14 + │ ├── columns: column1:7!null column2:8 column3:9 column10:10 column11:11 i:13 s:15 │ ├── cardinality: [2 - 2] │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ ├── fd: ()-->(9,10) + │ ├── fd: ()-->(10,11) │ ├── project - │ │ ├── columns: column9:9 column10:10 column1:6!null column2:7 column3:8 + │ │ ├── columns: column10:10 column11:11 column1:7!null column2:8 column3:9 │ │ ├── cardinality: [2 - 2] - │ │ ├── fd: ()-->(9,10) + │ │ ├── fd: ()-->(10,11) │ │ ├── values - │ │ │ ├── columns: column1:6!null column2:7 column3:8 + │ │ │ ├── columns: column1:7!null column2:8 column3:9 │ │ │ ├── cardinality: [2 - 2] │ │ │ ├── (1, NULL, NULL) │ │ │ └── (1, NULL, NULL) │ │ └── projections - │ │ ├── CAST(NULL AS FLOAT8) [as=column9:9] - │ │ └── CAST(NULL AS JSONB) [as=column10:10] + │ │ ├── CAST(NULL AS FLOAT8) [as=column10:10] + │ │ └── CAST(NULL AS JSONB) [as=column11:11] │ ├── scan a - │ │ ├── columns: i:12!null s:14!null - │ │ └── key: (12,14) + │ │ ├── columns: i:13!null s:15!null + │ │ └── key: (13,15) │ └── filters - │ ├── column2:7 = s:14 [outer=(7,14), constraints=(/7: (/NULL - ]; /14: (/NULL - ]), fd=(7)==(14), (14)==(7)] - │ └── column3:8 = i:12 [outer=(8,12), constraints=(/8: (/NULL - ]; /12: (/NULL - ]), fd=(8)==(12), (12)==(8)] + │ ├── column2:8 = s:15 [outer=(8,15), constraints=(/8: (/NULL - ]; /15: (/NULL - ]), fd=(8)==(15), (15)==(8)] + │ └── column3:9 = i:13 [outer=(9,13), constraints=(/9: (/NULL - ]; /13: (/NULL - ]), fd=(9)==(13), (13)==(9)] └── filters - └── s:14 IS NULL [outer=(14), constraints=(/14: [/NULL - /NULL]; tight), fd=()-->(14)] + └── s:15 IS NULL [outer=(15), constraints=(/15: [/NULL - /NULL]; tight), fd=()-->(15)] # EnsureUpsertDistinctOn treats NULL values as distinct, so it can be eliminated. norm expect=EliminateDistinctOnValues @@ -2250,48 +2250,48 @@ ON CONFLICT (s, i) DO UPDATE SET f=1.0 ---- upsert a ├── columns: - ├── canary column: 11 - ├── fetch columns: k:11 i:12 f:13 s:14 j:15 + ├── canary column: 12 + ├── fetch columns: k:12 i:13 f:14 s:15 j:16 ├── insert-mapping: - │ ├── column1:6 => k:1 - │ ├── column3:8 => i:2 - │ ├── column9:9 => f:3 - │ ├── column2:7 => s:4 - │ └── column10:10 => j:5 + │ ├── column1:7 => k:1 + │ ├── column3:9 => i:2 + │ ├── column10:10 => f:3 + │ ├── column2:8 => s:4 + │ └── column11:11 => j:5 ├── update-mapping: - │ └── upsert_f:19 => f:3 + │ └── upsert_f:21 => f:3 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: upsert_f:19 column1:6!null column2:7 column3:8 column9:9 column10:10 k:11 i:12 f:13 s:14 j:15 + ├── columns: upsert_f:21 column1:7!null column2:8 column3:9 column10:10 column11:11 k:12 i:13 f:14 s:15 j:16 ├── cardinality: [2 - 2] - ├── fd: ()-->(9,10), (11)-->(12-15), (12,14)-->(11,13,15), (12,13)~~>(11,14,15) + ├── fd: ()-->(10,11), (12)-->(13-16), (13,15)-->(12,14,16), (13,14)~~>(12,15,16) ├── left-join (hash) - │ ├── columns: column1:6!null column2:7 column3:8 column9:9 column10:10 k:11 i:12 f:13 s:14 j:15 + │ ├── columns: column1:7!null column2:8 column3:9 column10:10 column11:11 k:12 i:13 f:14 s:15 j:16 │ ├── cardinality: [2 - 2] │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ ├── fd: ()-->(9,10), (11)-->(12-15), (12,14)-->(11,13,15), (12,13)~~>(11,14,15) + │ ├── fd: ()-->(10,11), (12)-->(13-16), (13,15)-->(12,14,16), (13,14)~~>(12,15,16) │ ├── project - │ │ ├── columns: column9:9 column10:10 column1:6!null column2:7 column3:8 + │ │ ├── columns: column10:10 column11:11 column1:7!null column2:8 column3:9 │ │ ├── cardinality: [2 - 2] - │ │ ├── fd: ()-->(9,10) + │ │ ├── fd: ()-->(10,11) │ │ ├── values - │ │ │ ├── columns: column1:6!null column2:7 column3:8 + │ │ │ ├── columns: column1:7!null column2:8 column3:9 │ │ │ ├── cardinality: [2 - 2] │ │ │ ├── (1, NULL, NULL) │ │ │ └── (1, NULL, NULL) │ │ └── projections - │ │ ├── CAST(NULL AS FLOAT8) [as=column9:9] - │ │ └── CAST(NULL AS JSONB) [as=column10:10] + │ │ ├── CAST(NULL AS FLOAT8) [as=column10:10] + │ │ └── CAST(NULL AS JSONB) [as=column11:11] │ ├── scan a - │ │ ├── columns: k:11!null i:12!null f:13 s:14!null j:15 - │ │ ├── key: (11) - │ │ └── fd: (11)-->(12-15), (12,14)-->(11,13,15), (12,13)~~>(11,14,15) + │ │ ├── columns: k:12!null i:13!null f:14 s:15!null j:16 + │ │ ├── key: (12) + │ │ └── fd: (12)-->(13-16), (13,15)-->(12,14,16), (13,14)~~>(12,15,16) │ └── filters - │ ├── column3:8 = i:12 [outer=(8,12), constraints=(/8: (/NULL - ]; /12: (/NULL - ]), fd=(8)==(12), (12)==(8)] - │ └── column2:7 = s:14 [outer=(7,14), constraints=(/7: (/NULL - ]; /14: (/NULL - ]), fd=(7)==(14), (14)==(7)] + │ ├── column3:9 = i:13 [outer=(9,13), constraints=(/9: (/NULL - ]; /13: (/NULL - ]), fd=(9)==(13), (13)==(9)] + │ └── column2:8 = s:15 [outer=(8,15), constraints=(/8: (/NULL - ]; /15: (/NULL - ]), fd=(8)==(15), (15)==(8)] └── projections - └── CASE WHEN k:11 IS NULL THEN column9:9 ELSE 1.0 END [as=upsert_f:19, outer=(9,11)] + └── CASE WHEN k:12 IS NULL THEN column10:10 ELSE 1.0 END [as=upsert_f:21, outer=(10,12)] # EnsureUpsertDistinctOn is not removed when there are duplicates. norm expect-not=EliminateDistinctOnValues @@ -2300,65 +2300,65 @@ ON CONFLICT (s, i) DO UPDATE SET f=1.0 ---- upsert a ├── columns: - ├── canary column: 11 - ├── fetch columns: k:11 i:12 f:13 s:14 j:15 + ├── canary column: 12 + ├── fetch columns: k:12 i:13 f:14 s:15 j:16 ├── insert-mapping: - │ ├── column1:6 => k:1 - │ ├── column3:8 => i:2 - │ ├── column9:9 => f:3 - │ ├── column2:7 => s:4 - │ └── column10:10 => j:5 + │ ├── column1:7 => k:1 + │ ├── column3:9 => i:2 + │ ├── column10:10 => f:3 + │ ├── column2:8 => s:4 + │ └── column11:11 => j:5 ├── update-mapping: - │ └── upsert_f:19 => f:3 + │ └── upsert_f:21 => f:3 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: upsert_f:19 column1:6!null column2:7!null column3:8!null column9:9 column10:10 k:11 i:12 f:13 s:14 j:15 + ├── columns: upsert_f:21 column1:7!null column2:8!null column3:9!null column10:10 column11:11 k:12 i:13 f:14 s:15 j:16 ├── cardinality: [1 - 3] - ├── key: (7,8) - ├── fd: ()-->(9,10), (7,8)-->(6,11-15,19), (11)-->(12-15), (12,14)-->(11,13,15), (12,13)~~>(11,14,15) + ├── key: (8,9) + ├── fd: ()-->(10,11), (8,9)-->(7,12-16,21), (12)-->(13-16), (13,15)-->(12,14,16), (13,14)~~>(12,15,16) ├── left-join (hash) - │ ├── columns: column1:6!null column2:7!null column3:8!null column9:9 column10:10 k:11 i:12 f:13 s:14 j:15 + │ ├── columns: column1:7!null column2:8!null column3:9!null column10:10 column11:11 k:12 i:13 f:14 s:15 j:16 │ ├── cardinality: [1 - 3] │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) - │ ├── key: (7,8) - │ ├── fd: ()-->(9,10), (7,8)-->(6,11-15), (11)-->(12-15), (12,14)-->(11,13,15), (12,13)~~>(11,14,15) + │ ├── key: (8,9) + │ ├── fd: ()-->(10,11), (8,9)-->(7,12-16), (12)-->(13-16), (13,15)-->(12,14,16), (13,14)~~>(12,15,16) │ ├── ensure-upsert-distinct-on - │ │ ├── columns: column1:6!null column2:7!null column3:8!null column9:9 column10:10 - │ │ ├── grouping columns: column2:7!null column3:8!null + │ │ ├── columns: column1:7!null column2:8!null column3:9!null column10:10 column11:11 + │ │ ├── grouping columns: column2:8!null column3:9!null │ │ ├── error: "UPSERT or INSERT...ON CONFLICT command cannot affect row a second time" │ │ ├── cardinality: [1 - 3] - │ │ ├── key: (7,8) - │ │ ├── fd: ()-->(9,10), (7,8)-->(6,9,10) + │ │ ├── key: (8,9) + │ │ ├── fd: ()-->(10,11), (8,9)-->(7,10,11) │ │ ├── project - │ │ │ ├── columns: column9:9 column10:10 column1:6!null column2:7!null column3:8!null + │ │ │ ├── columns: column10:10 column11:11 column1:7!null column2:8!null column3:9!null │ │ │ ├── cardinality: [3 - 3] - │ │ │ ├── fd: ()-->(9,10) + │ │ │ ├── fd: ()-->(10,11) │ │ │ ├── values - │ │ │ │ ├── columns: column1:6!null column2:7!null column3:8!null + │ │ │ │ ├── columns: column1:7!null column2:8!null column3:9!null │ │ │ │ ├── cardinality: [3 - 3] │ │ │ │ ├── (1, 'foo', 1) │ │ │ │ ├── (2, 'bar', 2) │ │ │ │ └── (3, 'foo', 1) │ │ │ └── projections - │ │ │ ├── CAST(NULL AS FLOAT8) [as=column9:9] - │ │ │ └── CAST(NULL AS JSONB) [as=column10:10] + │ │ │ ├── CAST(NULL AS FLOAT8) [as=column10:10] + │ │ │ └── CAST(NULL AS JSONB) [as=column11:11] │ │ └── aggregations - │ │ ├── first-agg [as=column1:6, outer=(6)] - │ │ │ └── column1:6 - │ │ ├── first-agg [as=column9:9, outer=(9)] - │ │ │ └── column9:9 - │ │ └── first-agg [as=column10:10, outer=(10)] - │ │ └── column10:10 + │ │ ├── first-agg [as=column1:7, outer=(7)] + │ │ │ └── column1:7 + │ │ ├── first-agg [as=column10:10, outer=(10)] + │ │ │ └── column10:10 + │ │ └── first-agg [as=column11:11, outer=(11)] + │ │ └── column11:11 │ ├── scan a - │ │ ├── columns: k:11!null i:12!null f:13 s:14!null j:15 - │ │ ├── key: (11) - │ │ └── fd: (11)-->(12-15), (12,14)-->(11,13,15), (12,13)~~>(11,14,15) + │ │ ├── columns: k:12!null i:13!null f:14 s:15!null j:16 + │ │ ├── key: (12) + │ │ └── fd: (12)-->(13-16), (13,15)-->(12,14,16), (13,14)~~>(12,15,16) │ └── filters - │ ├── column3:8 = i:12 [outer=(8,12), constraints=(/8: (/NULL - ]; /12: (/NULL - ]), fd=(8)==(12), (12)==(8)] - │ └── column2:7 = s:14 [outer=(7,14), constraints=(/7: (/NULL - ]; /14: (/NULL - ]), fd=(7)==(14), (14)==(7)] + │ ├── column3:9 = i:13 [outer=(9,13), constraints=(/9: (/NULL - ]; /13: (/NULL - ]), fd=(9)==(13), (13)==(9)] + │ └── column2:8 = s:15 [outer=(8,15), constraints=(/8: (/NULL - ]; /15: (/NULL - ]), fd=(8)==(15), (15)==(8)] └── projections - └── CASE WHEN k:11 IS NULL THEN column9:9 ELSE 1.0 END [as=upsert_f:19, outer=(9,11)] + └── CASE WHEN k:12 IS NULL THEN column10:10 ELSE 1.0 END [as=upsert_f:21, outer=(10,12)] # DO NOTHING case where all distinct ops can be removed. norm expect=EliminateDistinctOnValues @@ -2368,79 +2368,79 @@ ON CONFLICT DO NOTHING insert a ├── columns: ├── insert-mapping: - │ ├── column1:6 => k:1 - │ ├── column3:8 => i:2 - │ ├── column4:9 => f:3 - │ ├── column2:7 => s:4 - │ └── column10:10 => j:5 + │ ├── column1:7 => k:1 + │ ├── column3:9 => i:2 + │ ├── column4:10 => f:3 + │ ├── column2:8 => s:4 + │ └── column11:11 => j:5 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null column10:10 + ├── columns: column1:7!null column2:8!null column3:9!null column4:10!null column11:11 ├── cardinality: [0 - 3] - ├── fd: ()-->(10) + ├── fd: ()-->(11) └── select - ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null column10:10 k:11 i:17 s:19 i:22 f:23 + ├── columns: column1:7!null column2:8!null column3:9!null column4:10!null column11:11 k:12 i:19 s:21 i:25 f:26 ├── cardinality: [0 - 3] - ├── fd: ()-->(10,11,19,22) + ├── fd: ()-->(11,12,21,25) ├── left-join (hash) - │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null column10:10 k:11 i:17 s:19 i:22 f:23 + │ ├── columns: column1:7!null column2:8!null column3:9!null column4:10!null column11:11 k:12 i:19 s:21 i:25 f:26 │ ├── cardinality: [0 - 3] │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ ├── fd: ()-->(10,11,19) + │ ├── fd: ()-->(11,12,21) │ ├── select - │ │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null column10:10 k:11 i:17 s:19 + │ │ ├── columns: column1:7!null column2:8!null column3:9!null column4:10!null column11:11 k:12 i:19 s:21 │ │ ├── cardinality: [0 - 3] - │ │ ├── fd: ()-->(10,11,19) + │ │ ├── fd: ()-->(11,12,21) │ │ ├── left-join (hash) - │ │ │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null column10:10 k:11 i:17 s:19 + │ │ │ ├── columns: column1:7!null column2:8!null column3:9!null column4:10!null column11:11 k:12 i:19 s:21 │ │ │ ├── cardinality: [0 - 3] │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ │ │ ├── fd: ()-->(10,11) + │ │ │ ├── fd: ()-->(11,12) │ │ │ ├── select - │ │ │ │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null column10:10 k:11 + │ │ │ │ ├── columns: column1:7!null column2:8!null column3:9!null column4:10!null column11:11 k:12 │ │ │ │ ├── cardinality: [0 - 3] - │ │ │ │ ├── fd: ()-->(10,11) + │ │ │ │ ├── fd: ()-->(11,12) │ │ │ │ ├── left-join (hash) - │ │ │ │ │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null column10:10 k:11 + │ │ │ │ │ ├── columns: column1:7!null column2:8!null column3:9!null column4:10!null column11:11 k:12 │ │ │ │ │ ├── cardinality: [3 - 3] │ │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ │ │ │ │ ├── fd: ()-->(10) + │ │ │ │ │ ├── fd: ()-->(11) │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: column10:10 column1:6!null column2:7!null column3:8!null column4:9!null + │ │ │ │ │ │ ├── columns: column11:11 column1:7!null column2:8!null column3:9!null column4:10!null │ │ │ │ │ │ ├── cardinality: [3 - 3] - │ │ │ │ │ │ ├── fd: ()-->(10) + │ │ │ │ │ │ ├── fd: ()-->(11) │ │ │ │ │ │ ├── values - │ │ │ │ │ │ │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null + │ │ │ │ │ │ │ ├── columns: column1:7!null column2:8!null column3:9!null column4:10!null │ │ │ │ │ │ │ ├── cardinality: [3 - 3] │ │ │ │ │ │ │ ├── (1, 'foo', 1, 1.0) │ │ │ │ │ │ │ ├── (2, 'bar', 2, 2.0) │ │ │ │ │ │ │ └── (3, 'foo', 2, 1.0) │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── CAST(NULL AS JSONB) [as=column10:10] + │ │ │ │ │ │ └── CAST(NULL AS JSONB) [as=column11:11] │ │ │ │ │ ├── scan a - │ │ │ │ │ │ ├── columns: k:11!null - │ │ │ │ │ │ └── key: (11) + │ │ │ │ │ │ ├── columns: k:12!null + │ │ │ │ │ │ └── key: (12) │ │ │ │ │ └── filters - │ │ │ │ │ └── column1:6 = k:11 [outer=(6,11), constraints=(/6: (/NULL - ]; /11: (/NULL - ]), fd=(6)==(11), (11)==(6)] + │ │ │ │ │ └── column1:7 = k:12 [outer=(7,12), constraints=(/7: (/NULL - ]; /12: (/NULL - ]), fd=(7)==(12), (12)==(7)] │ │ │ │ └── filters - │ │ │ │ └── k:11 IS NULL [outer=(11), constraints=(/11: [/NULL - /NULL]; tight), fd=()-->(11)] + │ │ │ │ └── k:12 IS NULL [outer=(12), constraints=(/12: [/NULL - /NULL]; tight), fd=()-->(12)] │ │ │ ├── scan a - │ │ │ │ ├── columns: i:17!null s:19!null - │ │ │ │ └── key: (17,19) + │ │ │ │ ├── columns: i:19!null s:21!null + │ │ │ │ └── key: (19,21) │ │ │ └── filters - │ │ │ ├── column2:7 = s:19 [outer=(7,19), constraints=(/7: (/NULL - ]; /19: (/NULL - ]), fd=(7)==(19), (19)==(7)] - │ │ │ └── column3:8 = i:17 [outer=(8,17), constraints=(/8: (/NULL - ]; /17: (/NULL - ]), fd=(8)==(17), (17)==(8)] + │ │ │ ├── column2:8 = s:21 [outer=(8,21), constraints=(/8: (/NULL - ]; /21: (/NULL - ]), fd=(8)==(21), (21)==(8)] + │ │ │ └── column3:9 = i:19 [outer=(9,19), constraints=(/9: (/NULL - ]; /19: (/NULL - ]), fd=(9)==(19), (19)==(9)] │ │ └── filters - │ │ └── s:19 IS NULL [outer=(19), constraints=(/19: [/NULL - /NULL]; tight), fd=()-->(19)] + │ │ └── s:21 IS NULL [outer=(21), constraints=(/21: [/NULL - /NULL]; tight), fd=()-->(21)] │ ├── scan a - │ │ ├── columns: i:22!null f:23 - │ │ └── lax-key: (22,23) + │ │ ├── columns: i:25!null f:26 + │ │ └── lax-key: (25,26) │ └── filters - │ ├── column4:9 = f:23 [outer=(9,23), constraints=(/9: (/NULL - ]; /23: (/NULL - ]), fd=(9)==(23), (23)==(9)] - │ └── column3:8 = i:22 [outer=(8,22), constraints=(/8: (/NULL - ]; /22: (/NULL - ]), fd=(8)==(22), (22)==(8)] + │ ├── column4:10 = f:26 [outer=(10,26), constraints=(/10: (/NULL - ]; /26: (/NULL - ]), fd=(10)==(26), (26)==(10)] + │ └── column3:9 = i:25 [outer=(9,25), constraints=(/9: (/NULL - ]; /25: (/NULL - ]), fd=(9)==(25), (25)==(9)] └── filters - └── i:22 IS NULL [outer=(22), constraints=(/22: [/NULL - /NULL]; tight), fd=()-->(22)] + └── i:25 IS NULL [outer=(25), constraints=(/25: [/NULL - /NULL]; tight), fd=()-->(25)] # DO NOTHING case where one distinct op can be removed (k), but two others # can't: (s, i) and (f, i). @@ -2451,104 +2451,104 @@ ON CONFLICT DO NOTHING insert a ├── columns: ├── insert-mapping: - │ ├── column1:6 => k:1 - │ ├── column9:9 => i:2 - │ ├── column3:8 => f:3 - │ ├── column2:7 => s:4 - │ └── column10:10 => j:5 + │ ├── column1:7 => k:1 + │ ├── column10:10 => i:2 + │ ├── column3:9 => f:3 + │ ├── column2:8 => s:4 + │ └── column11:11 => j:5 ├── cardinality: [0 - 0] ├── volatile, mutations └── upsert-distinct-on - ├── columns: column1:6!null column2:7!null column3:8!null column9:9 column10:10 - ├── grouping columns: column3:8!null column9:9 + ├── columns: column1:7!null column2:8!null column3:9!null column10:10 column11:11 + ├── grouping columns: column3:9!null column10:10 ├── cardinality: [0 - 3] - ├── lax-key: (8,9) - ├── fd: ()-->(9,10), (7,9)~~>(6,8), (8,9)~~>(6,7,10) + ├── lax-key: (9,10) + ├── fd: ()-->(10,11), (8,10)~~>(7,9), (9,10)~~>(7,8,11) ├── select - │ ├── columns: column1:6!null column2:7!null column3:8!null column9:9 column10:10 i:22 f:23 + │ ├── columns: column1:7!null column2:8!null column3:9!null column10:10 column11:11 i:25 f:26 │ ├── cardinality: [0 - 3] - │ ├── lax-key: (7,9,22,23) - │ ├── fd: ()-->(9,10,22), (7,9)~~>(6,8) + │ ├── lax-key: (8,10,25,26) + │ ├── fd: ()-->(10,11,25), (8,10)~~>(7,9) │ ├── left-join (hash) - │ │ ├── columns: column1:6!null column2:7!null column3:8!null column9:9 column10:10 i:22 f:23 + │ │ ├── columns: column1:7!null column2:8!null column3:9!null column10:10 column11:11 i:25 f:26 │ │ ├── cardinality: [0 - 3] │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ │ ├── lax-key: (7,9,22,23) - │ │ ├── fd: ()-->(9,10), (7,9)~~>(6,8) + │ │ ├── lax-key: (8,10,25,26) + │ │ ├── fd: ()-->(10,11), (8,10)~~>(7,9) │ │ ├── upsert-distinct-on - │ │ │ ├── columns: column1:6!null column2:7!null column3:8!null column9:9 column10:10 - │ │ │ ├── grouping columns: column2:7!null column9:9 + │ │ │ ├── columns: column1:7!null column2:8!null column3:9!null column10:10 column11:11 + │ │ │ ├── grouping columns: column2:8!null column10:10 │ │ │ ├── cardinality: [0 - 3] - │ │ │ ├── lax-key: (7,9) - │ │ │ ├── fd: ()-->(9,10), (7,9)~~>(6,8,10) + │ │ │ ├── lax-key: (8,10) + │ │ │ ├── fd: ()-->(10,11), (8,10)~~>(7,9,11) │ │ │ ├── select - │ │ │ │ ├── columns: column1:6!null column2:7!null column3:8!null column9:9 column10:10 k:11 i:17 s:19 + │ │ │ │ ├── columns: column1:7!null column2:8!null column3:9!null column10:10 column11:11 k:12 i:19 s:21 │ │ │ │ ├── cardinality: [0 - 3] - │ │ │ │ ├── fd: ()-->(9-11,19) + │ │ │ │ ├── fd: ()-->(10-12,21) │ │ │ │ ├── left-join (hash) - │ │ │ │ │ ├── columns: column1:6!null column2:7!null column3:8!null column9:9 column10:10 k:11 i:17 s:19 + │ │ │ │ │ ├── columns: column1:7!null column2:8!null column3:9!null column10:10 column11:11 k:12 i:19 s:21 │ │ │ │ │ ├── cardinality: [0 - 3] │ │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ │ │ │ │ ├── fd: ()-->(9-11) + │ │ │ │ │ ├── fd: ()-->(10-12) │ │ │ │ │ ├── select - │ │ │ │ │ │ ├── columns: column1:6!null column2:7!null column3:8!null column9:9 column10:10 k:11 + │ │ │ │ │ │ ├── columns: column1:7!null column2:8!null column3:9!null column10:10 column11:11 k:12 │ │ │ │ │ │ ├── cardinality: [0 - 3] - │ │ │ │ │ │ ├── fd: ()-->(9-11) + │ │ │ │ │ │ ├── fd: ()-->(10-12) │ │ │ │ │ │ ├── left-join (hash) - │ │ │ │ │ │ │ ├── columns: column1:6!null column2:7!null column3:8!null column9:9 column10:10 k:11 + │ │ │ │ │ │ │ ├── columns: column1:7!null column2:8!null column3:9!null column10:10 column11:11 k:12 │ │ │ │ │ │ │ ├── cardinality: [3 - 3] │ │ │ │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ │ │ │ │ │ │ ├── fd: ()-->(9,10) + │ │ │ │ │ │ │ ├── fd: ()-->(10,11) │ │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ │ ├── columns: column9:9 column10:10 column1:6!null column2:7!null column3:8!null + │ │ │ │ │ │ │ │ ├── columns: column10:10 column11:11 column1:7!null column2:8!null column3:9!null │ │ │ │ │ │ │ │ ├── cardinality: [3 - 3] - │ │ │ │ │ │ │ │ ├── fd: ()-->(9,10) + │ │ │ │ │ │ │ │ ├── fd: ()-->(10,11) │ │ │ │ │ │ │ │ ├── values - │ │ │ │ │ │ │ │ │ ├── columns: column1:6!null column2:7!null column3:8!null + │ │ │ │ │ │ │ │ │ ├── columns: column1:7!null column2:8!null column3:9!null │ │ │ │ │ │ │ │ │ ├── cardinality: [3 - 3] │ │ │ │ │ │ │ │ │ ├── (1, 'foo', 1.0) │ │ │ │ │ │ │ │ │ ├── (2, 'bar', 2.0) │ │ │ │ │ │ │ │ │ └── (3, 'foo', 1.0) │ │ │ │ │ │ │ │ └── projections - │ │ │ │ │ │ │ │ ├── CAST(NULL AS INT8) [as=column9:9] - │ │ │ │ │ │ │ │ └── CAST(NULL AS JSONB) [as=column10:10] + │ │ │ │ │ │ │ │ ├── CAST(NULL AS INT8) [as=column10:10] + │ │ │ │ │ │ │ │ └── CAST(NULL AS JSONB) [as=column11:11] │ │ │ │ │ │ │ ├── scan a - │ │ │ │ │ │ │ │ ├── columns: k:11!null - │ │ │ │ │ │ │ │ └── key: (11) + │ │ │ │ │ │ │ │ ├── columns: k:12!null + │ │ │ │ │ │ │ │ └── key: (12) │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── column1:6 = k:11 [outer=(6,11), constraints=(/6: (/NULL - ]; /11: (/NULL - ]), fd=(6)==(11), (11)==(6)] + │ │ │ │ │ │ │ └── column1:7 = k:12 [outer=(7,12), constraints=(/7: (/NULL - ]; /12: (/NULL - ]), fd=(7)==(12), (12)==(7)] │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── k:11 IS NULL [outer=(11), constraints=(/11: [/NULL - /NULL]; tight), fd=()-->(11)] + │ │ │ │ │ │ └── k:12 IS NULL [outer=(12), constraints=(/12: [/NULL - /NULL]; tight), fd=()-->(12)] │ │ │ │ │ ├── scan a - │ │ │ │ │ │ ├── columns: i:17!null s:19!null - │ │ │ │ │ │ └── key: (17,19) + │ │ │ │ │ │ ├── columns: i:19!null s:21!null + │ │ │ │ │ │ └── key: (19,21) │ │ │ │ │ └── filters - │ │ │ │ │ ├── column2:7 = s:19 [outer=(7,19), constraints=(/7: (/NULL - ]; /19: (/NULL - ]), fd=(7)==(19), (19)==(7)] - │ │ │ │ │ └── column9:9 = i:17 [outer=(9,17), constraints=(/9: (/NULL - ]; /17: (/NULL - ]), fd=(9)==(17), (17)==(9)] + │ │ │ │ │ ├── column2:8 = s:21 [outer=(8,21), constraints=(/8: (/NULL - ]; /21: (/NULL - ]), fd=(8)==(21), (21)==(8)] + │ │ │ │ │ └── column10:10 = i:19 [outer=(10,19), constraints=(/10: (/NULL - ]; /19: (/NULL - ]), fd=(10)==(19), (19)==(10)] │ │ │ │ └── filters - │ │ │ │ └── s:19 IS NULL [outer=(19), constraints=(/19: [/NULL - /NULL]; tight), fd=()-->(19)] + │ │ │ │ └── s:21 IS NULL [outer=(21), constraints=(/21: [/NULL - /NULL]; tight), fd=()-->(21)] │ │ │ └── aggregations - │ │ │ ├── first-agg [as=column1:6, outer=(6)] - │ │ │ │ └── column1:6 - │ │ │ ├── first-agg [as=column3:8, outer=(8)] - │ │ │ │ └── column3:8 - │ │ │ └── first-agg [as=column10:10, outer=(10)] - │ │ │ └── column10:10 + │ │ │ ├── first-agg [as=column1:7, outer=(7)] + │ │ │ │ └── column1:7 + │ │ │ ├── first-agg [as=column3:9, outer=(9)] + │ │ │ │ └── column3:9 + │ │ │ └── first-agg [as=column11:11, outer=(11)] + │ │ │ └── column11:11 │ │ ├── scan a - │ │ │ ├── columns: i:22!null f:23 - │ │ │ └── lax-key: (22,23) + │ │ │ ├── columns: i:25!null f:26 + │ │ │ └── lax-key: (25,26) │ │ └── filters - │ │ ├── column3:8 = f:23 [outer=(8,23), constraints=(/8: (/NULL - ]; /23: (/NULL - ]), fd=(8)==(23), (23)==(8)] - │ │ └── column9:9 = i:22 [outer=(9,22), constraints=(/9: (/NULL - ]; /22: (/NULL - ]), fd=(9)==(22), (22)==(9)] + │ │ ├── column3:9 = f:26 [outer=(9,26), constraints=(/9: (/NULL - ]; /26: (/NULL - ]), fd=(9)==(26), (26)==(9)] + │ │ └── column10:10 = i:25 [outer=(10,25), constraints=(/10: (/NULL - ]; /25: (/NULL - ]), fd=(10)==(25), (25)==(10)] │ └── filters - │ └── i:22 IS NULL [outer=(22), constraints=(/22: [/NULL - /NULL]; tight), fd=()-->(22)] + │ └── i:25 IS NULL [outer=(25), constraints=(/25: [/NULL - /NULL]; tight), fd=()-->(25)] └── aggregations - ├── first-agg [as=column1:6, outer=(6)] - │ └── column1:6 - ├── first-agg [as=column2:7, outer=(7)] - │ └── column2:7 - └── first-agg [as=column10:10, outer=(10)] - └── column10:10 + ├── first-agg [as=column1:7, outer=(7)] + │ └── column1:7 + ├── first-agg [as=column2:8, outer=(8)] + │ └── column2:8 + └── first-agg [as=column11:11, outer=(11)] + └── column11:11 # DO NOTHING case where innermost distinct op cannot be removed (because it # groups on a non-constant column). Ensure that outer distinct ops can still be @@ -2560,107 +2560,107 @@ ON CONFLICT DO NOTHING insert a ├── columns: ├── insert-mapping: - │ ├── column1:6 => k:1 - │ ├── column3:8 => i:2 - │ ├── column4:9 => f:3 - │ ├── column2:7 => s:4 - │ └── column10:10 => j:5 + │ ├── column1:7 => k:1 + │ ├── column3:9 => i:2 + │ ├── column4:10 => f:3 + │ ├── column2:8 => s:4 + │ └── column11:11 => j:5 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: column1:6 column2:7!null column3:8!null column4:9!null column10:10 + ├── columns: column1:7 column2:8!null column3:9!null column4:10!null column11:11 ├── cardinality: [0 - 2] ├── volatile - ├── fd: ()-->(10), (6)~~>(7-9) + ├── fd: ()-->(11), (7)~~>(8-10) └── select - ├── columns: column1:6 column2:7!null column3:8!null column4:9!null column10:10 i:17 s:19 i:22 f:23 + ├── columns: column1:7 column2:8!null column3:9!null column4:10!null column11:11 i:19 s:21 i:25 f:26 ├── cardinality: [0 - 2] ├── volatile - ├── lax-key: (6,17,19,22,23) - ├── fd: ()-->(10,19,22), (6)~~>(7-9) + ├── lax-key: (7,19,21,25,26) + ├── fd: ()-->(11,21,25), (7)~~>(8-10) ├── left-join (hash) - │ ├── columns: column1:6 column2:7!null column3:8!null column4:9!null column10:10 i:17 s:19 i:22 f:23 + │ ├── columns: column1:7 column2:8!null column3:9!null column4:10!null column11:11 i:19 s:21 i:25 f:26 │ ├── cardinality: [0 - 2] │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) │ ├── volatile - │ ├── lax-key: (6,17,19,22,23) - │ ├── fd: ()-->(10,19), (6)~~>(7-9) + │ ├── lax-key: (7,19,21,25,26) + │ ├── fd: ()-->(11,21), (7)~~>(8-10) │ ├── select - │ │ ├── columns: column1:6 column2:7!null column3:8!null column4:9!null column10:10 i:17 s:19 + │ │ ├── columns: column1:7 column2:8!null column3:9!null column4:10!null column11:11 i:19 s:21 │ │ ├── cardinality: [0 - 2] │ │ ├── volatile - │ │ ├── lax-key: (6,17,19) - │ │ ├── fd: ()-->(10,19), (6)~~>(7-9) + │ │ ├── lax-key: (7,19,21) + │ │ ├── fd: ()-->(11,21), (7)~~>(8-10) │ │ ├── left-join (hash) - │ │ │ ├── columns: column1:6 column2:7!null column3:8!null column4:9!null column10:10 i:17 s:19 + │ │ │ ├── columns: column1:7 column2:8!null column3:9!null column4:10!null column11:11 i:19 s:21 │ │ │ ├── cardinality: [0 - 2] │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) │ │ │ ├── volatile - │ │ │ ├── lax-key: (6,17,19) - │ │ │ ├── fd: ()-->(10), (6)~~>(7-9) + │ │ │ ├── lax-key: (7,19,21) + │ │ │ ├── fd: ()-->(11), (7)~~>(8-10) │ │ │ ├── upsert-distinct-on - │ │ │ │ ├── columns: column1:6 column2:7!null column3:8!null column4:9!null column10:10 - │ │ │ │ ├── grouping columns: column1:6 + │ │ │ │ ├── columns: column1:7 column2:8!null column3:9!null column4:10!null column11:11 + │ │ │ │ ├── grouping columns: column1:7 │ │ │ │ ├── cardinality: [0 - 2] │ │ │ │ ├── volatile - │ │ │ │ ├── lax-key: (6) - │ │ │ │ ├── fd: ()-->(10), (6)~~>(7-10) + │ │ │ │ ├── lax-key: (7) + │ │ │ │ ├── fd: ()-->(11), (7)~~>(8-11) │ │ │ │ ├── select - │ │ │ │ │ ├── columns: column1:6 column2:7!null column3:8!null column4:9!null column10:10 k:11 + │ │ │ │ │ ├── columns: column1:7 column2:8!null column3:9!null column4:10!null column11:11 k:12 │ │ │ │ │ ├── cardinality: [0 - 2] │ │ │ │ │ ├── volatile - │ │ │ │ │ ├── fd: ()-->(10,11) + │ │ │ │ │ ├── fd: ()-->(11,12) │ │ │ │ │ ├── left-join (hash) - │ │ │ │ │ │ ├── columns: column1:6 column2:7!null column3:8!null column4:9!null column10:10 k:11 + │ │ │ │ │ │ ├── columns: column1:7 column2:8!null column3:9!null column4:10!null column11:11 k:12 │ │ │ │ │ │ ├── cardinality: [2 - 2] │ │ │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) │ │ │ │ │ │ ├── volatile - │ │ │ │ │ │ ├── fd: ()-->(10) + │ │ │ │ │ │ ├── fd: ()-->(11) │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ ├── columns: column10:10 column1:6 column2:7!null column3:8!null column4:9!null + │ │ │ │ │ │ │ ├── columns: column11:11 column1:7 column2:8!null column3:9!null column4:10!null │ │ │ │ │ │ │ ├── cardinality: [2 - 2] │ │ │ │ │ │ │ ├── volatile - │ │ │ │ │ │ │ ├── fd: ()-->(10) + │ │ │ │ │ │ │ ├── fd: ()-->(11) │ │ │ │ │ │ │ ├── values - │ │ │ │ │ │ │ │ ├── columns: column1:6 column2:7!null column3:8!null column4:9!null + │ │ │ │ │ │ │ │ ├── columns: column1:7 column2:8!null column3:9!null column4:10!null │ │ │ │ │ │ │ │ ├── cardinality: [2 - 2] │ │ │ │ │ │ │ │ ├── volatile │ │ │ │ │ │ │ │ ├── (unique_rowid(), 'foo', 1, 1.0) │ │ │ │ │ │ │ │ └── (unique_rowid(), 'bar', 2, 2.0) │ │ │ │ │ │ │ └── projections - │ │ │ │ │ │ │ └── CAST(NULL AS JSONB) [as=column10:10] + │ │ │ │ │ │ │ └── CAST(NULL AS JSONB) [as=column11:11] │ │ │ │ │ │ ├── scan a - │ │ │ │ │ │ │ ├── columns: k:11!null - │ │ │ │ │ │ │ └── key: (11) + │ │ │ │ │ │ │ ├── columns: k:12!null + │ │ │ │ │ │ │ └── key: (12) │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── column1:6 = k:11 [outer=(6,11), constraints=(/6: (/NULL - ]; /11: (/NULL - ]), fd=(6)==(11), (11)==(6)] + │ │ │ │ │ │ └── column1:7 = k:12 [outer=(7,12), constraints=(/7: (/NULL - ]; /12: (/NULL - ]), fd=(7)==(12), (12)==(7)] │ │ │ │ │ └── filters - │ │ │ │ │ └── k:11 IS NULL [outer=(11), constraints=(/11: [/NULL - /NULL]; tight), fd=()-->(11)] + │ │ │ │ │ └── k:12 IS NULL [outer=(12), constraints=(/12: [/NULL - /NULL]; tight), fd=()-->(12)] │ │ │ │ └── aggregations - │ │ │ │ ├── first-agg [as=column2:7, outer=(7)] - │ │ │ │ │ └── column2:7 - │ │ │ │ ├── first-agg [as=column3:8, outer=(8)] - │ │ │ │ │ └── column3:8 - │ │ │ │ ├── first-agg [as=column4:9, outer=(9)] - │ │ │ │ │ └── column4:9 - │ │ │ │ └── first-agg [as=column10:10, outer=(10)] - │ │ │ │ └── column10:10 + │ │ │ │ ├── first-agg [as=column2:8, outer=(8)] + │ │ │ │ │ └── column2:8 + │ │ │ │ ├── first-agg [as=column3:9, outer=(9)] + │ │ │ │ │ └── column3:9 + │ │ │ │ ├── first-agg [as=column4:10, outer=(10)] + │ │ │ │ │ └── column4:10 + │ │ │ │ └── first-agg [as=column11:11, outer=(11)] + │ │ │ │ └── column11:11 │ │ │ ├── scan a - │ │ │ │ ├── columns: i:17!null s:19!null - │ │ │ │ └── key: (17,19) + │ │ │ │ ├── columns: i:19!null s:21!null + │ │ │ │ └── key: (19,21) │ │ │ └── filters - │ │ │ ├── column2:7 = s:19 [outer=(7,19), constraints=(/7: (/NULL - ]; /19: (/NULL - ]), fd=(7)==(19), (19)==(7)] - │ │ │ └── column3:8 = i:17 [outer=(8,17), constraints=(/8: (/NULL - ]; /17: (/NULL - ]), fd=(8)==(17), (17)==(8)] + │ │ │ ├── column2:8 = s:21 [outer=(8,21), constraints=(/8: (/NULL - ]; /21: (/NULL - ]), fd=(8)==(21), (21)==(8)] + │ │ │ └── column3:9 = i:19 [outer=(9,19), constraints=(/9: (/NULL - ]; /19: (/NULL - ]), fd=(9)==(19), (19)==(9)] │ │ └── filters - │ │ └── s:19 IS NULL [outer=(19), constraints=(/19: [/NULL - /NULL]; tight), fd=()-->(19)] + │ │ └── s:21 IS NULL [outer=(21), constraints=(/21: [/NULL - /NULL]; tight), fd=()-->(21)] │ ├── scan a - │ │ ├── columns: i:22!null f:23 - │ │ └── lax-key: (22,23) + │ │ ├── columns: i:25!null f:26 + │ │ └── lax-key: (25,26) │ └── filters - │ ├── column4:9 = f:23 [outer=(9,23), constraints=(/9: (/NULL - ]; /23: (/NULL - ]), fd=(9)==(23), (23)==(9)] - │ └── column3:8 = i:22 [outer=(8,22), constraints=(/8: (/NULL - ]; /22: (/NULL - ]), fd=(8)==(22), (22)==(8)] + │ ├── column4:10 = f:26 [outer=(10,26), constraints=(/10: (/NULL - ]; /26: (/NULL - ]), fd=(10)==(26), (26)==(10)] + │ └── column3:9 = i:25 [outer=(9,25), constraints=(/9: (/NULL - ]; /25: (/NULL - ]), fd=(9)==(25), (25)==(9)] └── filters - └── i:22 IS NULL [outer=(22), constraints=(/22: [/NULL - /NULL]; tight), fd=()-->(22)] + └── i:25 IS NULL [outer=(25), constraints=(/25: [/NULL - /NULL]; tight), fd=()-->(25)] # DO NOTHING case with explicit conflict columns (only add upsert-distinct-on # for one index). @@ -2671,54 +2671,54 @@ ON CONFLICT (s, i) DO NOTHING insert a ├── columns: ├── insert-mapping: - │ ├── i:7 => k:1 - │ ├── i:7 => i:2 - │ ├── column12:12 => f:3 - │ ├── "?column?":11 => s:4 - │ └── column13:13 => j:5 + │ ├── i:8 => k:1 + │ ├── i:8 => i:2 + │ ├── column14:14 => f:3 + │ ├── "?column?":13 => s:4 + │ └── column15:15 => j:5 ├── cardinality: [0 - 0] ├── volatile, mutations └── upsert-distinct-on - ├── columns: i:7!null "?column?":11!null column12:12 column13:13 - ├── grouping columns: i:7!null - ├── key: (7) - ├── fd: ()-->(11-13) + ├── columns: i:8!null "?column?":13!null column14:14 column15:15 + ├── grouping columns: i:8!null + ├── key: (8) + ├── fd: ()-->(13-15) ├── select - │ ├── columns: i:7!null "?column?":11!null column12:12 column13:13 i:15 s:17 - │ ├── fd: ()-->(11-13,17) + │ ├── columns: i:8!null "?column?":13!null column14:14 column15:15 i:17 s:19 + │ ├── fd: ()-->(13-15,19) │ ├── left-join (hash) - │ │ ├── columns: i:7!null "?column?":11!null column12:12 column13:13 i:15 s:17 + │ │ ├── columns: i:8!null "?column?":13!null column14:14 column15:15 i:17 s:19 │ │ ├── multiplicity: left-rows(exactly-one), right-rows(one-or-more) - │ │ ├── fd: ()-->(11-13) + │ │ ├── fd: ()-->(13-15) │ │ ├── project - │ │ │ ├── columns: column12:12 column13:13 "?column?":11!null i:7!null - │ │ │ ├── fd: ()-->(11-13) + │ │ │ ├── columns: column14:14 column15:15 "?column?":13!null i:8!null + │ │ │ ├── fd: ()-->(13-15) │ │ │ ├── scan a - │ │ │ │ └── columns: i:7!null + │ │ │ │ └── columns: i:8!null │ │ │ └── projections - │ │ │ ├── CAST(NULL AS FLOAT8) [as=column12:12] - │ │ │ ├── CAST(NULL AS JSONB) [as=column13:13] - │ │ │ └── 'foo' [as="?column?":11] + │ │ │ ├── CAST(NULL AS FLOAT8) [as=column14:14] + │ │ │ ├── CAST(NULL AS JSONB) [as=column15:15] + │ │ │ └── 'foo' [as="?column?":13] │ │ ├── select - │ │ │ ├── columns: i:15!null s:17!null - │ │ │ ├── key: (15) - │ │ │ ├── fd: ()-->(17) + │ │ │ ├── columns: i:17!null s:19!null + │ │ │ ├── key: (17) + │ │ │ ├── fd: ()-->(19) │ │ │ ├── scan a - │ │ │ │ ├── columns: i:15!null s:17!null - │ │ │ │ └── key: (15,17) + │ │ │ │ ├── columns: i:17!null s:19!null + │ │ │ │ └── key: (17,19) │ │ │ └── filters - │ │ │ └── s:17 = 'foo' [outer=(17), constraints=(/17: [/'foo' - /'foo']; tight), fd=()-->(17)] + │ │ │ └── s:19 = 'foo' [outer=(19), constraints=(/19: [/'foo' - /'foo']; tight), fd=()-->(19)] │ │ └── filters - │ │ └── i:7 = i:15 [outer=(7,15), constraints=(/7: (/NULL - ]; /15: (/NULL - ]), fd=(7)==(15), (15)==(7)] + │ │ └── i:8 = i:17 [outer=(8,17), constraints=(/8: (/NULL - ]; /17: (/NULL - ]), fd=(8)==(17), (17)==(8)] │ └── filters - │ └── s:17 IS NULL [outer=(17), constraints=(/17: [/NULL - /NULL]; tight), fd=()-->(17)] + │ └── s:19 IS NULL [outer=(19), constraints=(/19: [/NULL - /NULL]; tight), fd=()-->(19)] └── aggregations - ├── first-agg [as=column12:12, outer=(12)] - │ └── column12:12 - ├── first-agg [as=column13:13, outer=(13)] - │ └── column13:13 - └── const-agg [as="?column?":11, outer=(11)] - └── "?column?":11 + ├── first-agg [as=column14:14, outer=(14)] + │ └── column14:14 + ├── first-agg [as=column15:15, outer=(15)] + │ └── column15:15 + └── const-agg [as="?column?":13, outer=(13)] + └── "?column?":13 # -------------------------------------------------- # PushAggDistinctIntoGroupBy @@ -2729,10 +2729,10 @@ norm expect=PushAggDistinctIntoGroupBy SELECT sum(DISTINCT y) FROM xyzbs ---- scalar-group-by - ├── columns: sum:6 + ├── columns: sum:7 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(6) + ├── fd: ()-->(7) ├── distinct-on │ ├── columns: y:2 │ ├── grouping columns: y:2 @@ -2740,7 +2740,7 @@ scalar-group-by │ └── scan xyzbs │ └── columns: y:2 └── aggregations - └── sum [as=sum:6, outer=(2)] + └── sum [as=sum:7, outer=(2)] └── y:2 # COUNT case. Expecting an index scan because opt command is used. @@ -2748,10 +2748,10 @@ opt expect=PushAggDistinctIntoGroupBy SELECT count(DISTINCT y) FROM xyzbs ---- scalar-group-by - ├── columns: count:6!null + ├── columns: count:7!null ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(6) + ├── fd: ()-->(7) ├── distinct-on │ ├── columns: y:2 │ ├── grouping columns: y:2 @@ -2761,7 +2761,7 @@ scalar-group-by │ ├── columns: y:2 │ └── ordering: +2 └── aggregations - └── count [as=count:6, outer=(2)] + └── count [as=count:7, outer=(2)] └── y:2 # AVG case. @@ -2769,10 +2769,10 @@ norm expect=PushAggDistinctIntoGroupBy SELECT avg(DISTINCT y) FROM xyzbs ---- scalar-group-by - ├── columns: avg:6 + ├── columns: avg:7 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(6) + ├── fd: ()-->(7) ├── distinct-on │ ├── columns: y:2 │ ├── grouping columns: y:2 @@ -2780,7 +2780,7 @@ scalar-group-by │ └── scan xyzbs │ └── columns: y:2 └── aggregations - └── avg [as=avg:6, outer=(2)] + └── avg [as=avg:7, outer=(2)] └── y:2 # JSON_AGG case. @@ -2788,10 +2788,10 @@ norm expect=PushAggDistinctIntoGroupBy SELECT json_agg(DISTINCT y) FROM xyzbs ---- scalar-group-by - ├── columns: json_agg:6 + ├── columns: json_agg:7 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(6) + ├── fd: ()-->(7) ├── distinct-on │ ├── columns: y:2 │ ├── grouping columns: y:2 @@ -2799,7 +2799,7 @@ scalar-group-by │ └── scan xyzbs │ └── columns: y:2 └── aggregations - └── json-agg [as=json_agg:6, outer=(2)] + └── json-agg [as=json_agg:7, outer=(2)] └── y:2 # CORR case. @@ -2808,10 +2808,10 @@ norm expect=PushAggDistinctIntoGroupBy SELECT corr(DISTINCT y, z) FROM xyzbs ---- scalar-group-by - ├── columns: corr:6 + ├── columns: corr:7 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(6) + ├── fd: ()-->(7) ├── distinct-on │ ├── columns: y:2 z:3!null │ ├── grouping columns: y:2 z:3!null @@ -2819,7 +2819,7 @@ scalar-group-by │ └── scan xyzbs │ └── columns: y:2 z:3!null └── aggregations - └── corr [as=corr:6, outer=(2,3)] + └── corr [as=corr:7, outer=(2,3)] ├── y:2 └── z:3 @@ -2829,29 +2829,29 @@ norm expect=PushAggDistinctIntoGroupBy SELECT string_agg(DISTINCT s, '-') FROM xyzbs ---- scalar-group-by - ├── columns: string_agg:7 + ├── columns: string_agg:8 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(7) + ├── fd: ()-->(8) ├── distinct-on - │ ├── columns: s:5 column6:6!null + │ ├── columns: s:5 column7:7!null │ ├── grouping columns: s:5 │ ├── key: (5) - │ ├── fd: ()-->(6) + │ ├── fd: ()-->(7) │ ├── project - │ │ ├── columns: column6:6!null s:5 - │ │ ├── fd: ()-->(6) + │ │ ├── columns: column7:7!null s:5 + │ │ ├── fd: ()-->(7) │ │ ├── scan xyzbs │ │ │ └── columns: s:5 │ │ └── projections - │ │ └── '-' [as=column6:6] + │ │ └── '-' [as=column7:7] │ └── aggregations - │ └── const-agg [as=column6:6, outer=(6)] - │ └── column6:6 + │ └── const-agg [as=column7:7, outer=(7)] + │ └── column7:7 └── aggregations - └── string-agg [as=string_agg:7, outer=(5,6)] + └── string-agg [as=string_agg:8, outer=(5,7)] ├── s:5 - └── column6:6 + └── column7:7 # STRING_AGG case with an ORDER BY. # Multiple input arguments for aggregate function. @@ -2859,47 +2859,47 @@ norm expect=PushAggDistinctIntoGroupBy SELECT string_agg(DISTINCT s, '-') FROM (SELECT s FROM xyzbs ORDER BY s) ---- scalar-group-by - ├── columns: string_agg:7 - ├── internal-ordering: +5 opt(6) + ├── columns: string_agg:8 + ├── internal-ordering: +5 opt(7) ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(7) + ├── fd: ()-->(8) ├── sort - │ ├── columns: s:5 column6:6!null + │ ├── columns: s:5 column7:7!null │ ├── key: (5) - │ ├── fd: ()-->(6) - │ ├── ordering: +5 opt(6) [actual: +5] + │ ├── fd: ()-->(7) + │ ├── ordering: +5 opt(7) [actual: +5] │ └── distinct-on - │ ├── columns: s:5 column6:6!null + │ ├── columns: s:5 column7:7!null │ ├── grouping columns: s:5 │ ├── key: (5) - │ ├── fd: ()-->(6) + │ ├── fd: ()-->(7) │ ├── project - │ │ ├── columns: column6:6!null s:5 - │ │ ├── fd: ()-->(6) + │ │ ├── columns: column7:7!null s:5 + │ │ ├── fd: ()-->(7) │ │ ├── scan xyzbs │ │ │ └── columns: s:5 │ │ └── projections - │ │ └── '-' [as=column6:6] + │ │ └── '-' [as=column7:7] │ └── aggregations │ ├── first-agg [as=s:5, outer=(5)] │ │ └── s:5 - │ └── const-agg [as=column6:6, outer=(6)] - │ └── column6:6 + │ └── const-agg [as=column7:7, outer=(7)] + │ └── column7:7 └── aggregations - └── string-agg [as=string_agg:7, outer=(5,6)] + └── string-agg [as=string_agg:8, outer=(5,7)] ├── s:5 - └── column6:6 + └── column7:7 # Case with a GroupBy operator. norm expect=PushAggDistinctIntoGroupBy SELECT b, count(DISTINCT y) FROM xyzbs GROUP BY b ---- group-by - ├── columns: b:4!null count:6!null + ├── columns: b:4!null count:7!null ├── grouping columns: b:4!null ├── key: (4) - ├── fd: (4)-->(6) + ├── fd: (4)-->(7) ├── distinct-on │ ├── columns: y:2 b:4!null │ ├── grouping columns: y:2 b:4!null @@ -2907,7 +2907,7 @@ group-by │ └── scan xyzbs │ └── columns: y:2 b:4!null └── aggregations - └── count [as=count:6, outer=(2)] + └── count [as=count:7, outer=(2)] └── y:2 # Case with a GroupBy operator grouping on multiple columns. @@ -2915,10 +2915,10 @@ norm expect=PushAggDistinctIntoGroupBy SELECT b, s, count(DISTINCT y) FROM xyzbs GROUP BY b, s ---- group-by - ├── columns: b:4!null s:5 count:6!null + ├── columns: b:4!null s:5 count:7!null ├── grouping columns: b:4!null s:5 ├── key: (4,5) - ├── fd: (4,5)-->(6) + ├── fd: (4,5)-->(7) ├── distinct-on │ ├── columns: y:2 b:4!null s:5 │ ├── grouping columns: y:2 b:4!null s:5 @@ -2926,7 +2926,7 @@ group-by │ └── scan xyzbs │ └── columns: y:2 b:4!null s:5 └── aggregations - └── count [as=count:6, outer=(2)] + └── count [as=count:7, outer=(2)] └── y:2 # Case with a GroupBy operator and an aggregate with multiple input columns. @@ -2934,10 +2934,10 @@ norm expect=PushAggDistinctIntoGroupBy SELECT s, corr(DISTINCT y, z) FROM xyzbs GROUP BY s ---- group-by - ├── columns: s:5 corr:6 + ├── columns: s:5 corr:7 ├── grouping columns: s:5 ├── key: (5) - ├── fd: (5)-->(6) + ├── fd: (5)-->(7) ├── distinct-on │ ├── columns: y:2 z:3!null s:5 │ ├── grouping columns: y:2 z:3!null s:5 @@ -2945,7 +2945,7 @@ group-by │ └── scan xyzbs │ └── columns: y:2 z:3!null s:5 └── aggregations - └── corr [as=corr:6, outer=(2,3)] + └── corr [as=corr:7, outer=(2,3)] ├── y:2 └── z:3 @@ -2956,13 +2956,13 @@ norm expect=PushAggDistinctIntoGroupBy SELECT array_agg(DISTINCT s) FROM (SELECT * FROM a ORDER BY i) GROUP BY f ---- project - ├── columns: array_agg:6!null + ├── columns: array_agg:7!null └── group-by - ├── columns: f:3 array_agg:6!null + ├── columns: f:3 array_agg:7!null ├── grouping columns: f:3 ├── internal-ordering: +2 opt(3) ├── key: (3) - ├── fd: (3)-->(6) + ├── fd: (3)-->(7) ├── sort │ ├── columns: i:2!null f:3 s:4!null │ ├── key: (2,4) @@ -2981,7 +2981,7 @@ project │ └── first-agg [as=i:2, outer=(2)] │ └── i:2 └── aggregations - └── array-agg [as=array_agg:6, outer=(4)] + └── array-agg [as=array_agg:7, outer=(4)] └── s:4 # No-op case where the same aggregate function is called on different @@ -2990,17 +2990,17 @@ norm expect-not=PushAggDistinctIntoGroupBy SELECT count(DISTINCT y), count(DISTINCT z) FROM xyzbs ---- scalar-group-by - ├── columns: count:6!null count:7!null + ├── columns: count:7!null count:8!null ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(6,7) + ├── fd: ()-->(7,8) ├── scan xyzbs │ └── columns: y:2 z:3!null └── aggregations - ├── agg-distinct [as=count:6, outer=(2)] + ├── agg-distinct [as=count:7, outer=(2)] │ └── count │ └── y:2 - └── agg-distinct [as=count:7, outer=(3)] + └── agg-distinct [as=count:8, outer=(3)] └── count └── z:3 @@ -3010,17 +3010,17 @@ norm expect-not=PushAggDistinctIntoGroupBy SELECT count(DISTINCT y), sum(DISTINCT y) FROM xyzbs ---- scalar-group-by - ├── columns: count:6!null sum:7 + ├── columns: count:7!null sum:8 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(6,7) + ├── fd: ()-->(7,8) ├── scan xyzbs │ └── columns: y:2 └── aggregations - ├── agg-distinct [as=count:6, outer=(2)] + ├── agg-distinct [as=count:7, outer=(2)] │ └── count │ └── y:2 - └── agg-distinct [as=sum:7, outer=(2)] + └── agg-distinct [as=sum:8, outer=(2)] └── sum └── y:2 @@ -3031,28 +3031,28 @@ norm expect-not=PushAggDistinctIntoGroupBy SELECT max(DISTINCT y) FROM xyzbs ---- scalar-group-by - ├── columns: max:6 + ├── columns: max:7 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(6) + ├── fd: ()-->(7) ├── scan xyzbs │ └── columns: y:2 └── aggregations - └── max [as=max:6, outer=(2)] + └── max [as=max:7, outer=(2)] └── y:2 norm expect-not=PushAggDistinctIntoGroupBy SELECT bool_and(DISTINCT b) FROM xyzbs ---- scalar-group-by - ├── columns: bool_and:6 + ├── columns: bool_and:7 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(6) + ├── fd: ()-->(7) ├── scan xyzbs │ └── columns: b:4!null └── aggregations - └── bool-and [as=bool_and:6, outer=(4)] + └── bool-and [as=bool_and:7, outer=(4)] └── b:4 # -------------------------------------------------- @@ -3064,10 +3064,10 @@ norm expect=PushAggFilterIntoScalarGroupBy SELECT sum(y) FILTER (WHERE y < 50) FROM xyzbs ---- scalar-group-by - ├── columns: sum:7 + ├── columns: sum:8 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(7) + ├── fd: ()-->(8) ├── select │ ├── columns: y:2!null │ ├── scan xyzbs @@ -3075,7 +3075,7 @@ scalar-group-by │ └── filters │ └── y:2 < 50 [outer=(2), constraints=(/2: (/NULL - /49]; tight)] └── aggregations - └── sum [as=sum:7, outer=(2)] + └── sum [as=sum:8, outer=(2)] └── y:2 # COUNT case. Expecting an index scan because opt command is used. @@ -3083,25 +3083,25 @@ opt expect=PushAggFilterIntoScalarGroupBy SELECT count(y) FILTER (WHERE y < 50) FROM xyzbs ---- scalar-group-by - ├── columns: count:7!null + ├── columns: count:8!null ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(7) + ├── fd: ()-->(8) ├── scan xyzbs@secondary │ ├── columns: y:2!null │ └── constraint: /2/1: (/NULL - /49] └── aggregations - └── count-rows [as=count:7] + └── count-rows [as=count:8] # AVG case. norm expect=PushAggFilterIntoScalarGroupBy SELECT avg(y) FILTER (WHERE y < 50) FROM xyzbs ---- scalar-group-by - ├── columns: avg:7 + ├── columns: avg:8 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(7) + ├── fd: ()-->(8) ├── select │ ├── columns: y:2!null │ ├── scan xyzbs @@ -3109,7 +3109,7 @@ scalar-group-by │ └── filters │ └── y:2 < 50 [outer=(2), constraints=(/2: (/NULL - /49]; tight)] └── aggregations - └── avg [as=avg:7, outer=(2)] + └── avg [as=avg:8, outer=(2)] └── y:2 # MAX case. @@ -3117,10 +3117,10 @@ norm expect=PushAggFilterIntoScalarGroupBy SELECT max(y) FILTER (WHERE y < 50) FROM xyzbs ---- scalar-group-by - ├── columns: max:7 + ├── columns: max:8 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(7) + ├── fd: ()-->(8) ├── select │ ├── columns: y:2!null │ ├── scan xyzbs @@ -3128,7 +3128,7 @@ scalar-group-by │ └── filters │ └── y:2 < 50 [outer=(2), constraints=(/2: (/NULL - /49]; tight)] └── aggregations - └── max [as=max:7, outer=(2)] + └── max [as=max:8, outer=(2)] └── y:2 # BOOL_AND case. @@ -3136,10 +3136,10 @@ norm expect=PushAggFilterIntoScalarGroupBy SELECT bool_and(b) FILTER (WHERE b) FROM xyzbs ---- scalar-group-by - ├── columns: bool_and:6 + ├── columns: bool_and:7 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(6) + ├── fd: ()-->(7) ├── select │ ├── columns: b:4!null │ ├── fd: ()-->(4) @@ -3148,7 +3148,7 @@ scalar-group-by │ └── filters │ └── b:4 [outer=(4), constraints=(/4: [/true - /true]; tight), fd=()-->(4)] └── aggregations - └── bool-and [as=bool_and:6, outer=(4)] + └── bool-and [as=bool_and:7, outer=(4)] └── b:4 # JSON_AGG case. @@ -3156,10 +3156,10 @@ norm expect=PushAggFilterIntoScalarGroupBy SELECT json_agg(y) FILTER (WHERE y < 50) FROM xyzbs ---- scalar-group-by - ├── columns: json_agg:7 + ├── columns: json_agg:8 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(7) + ├── fd: ()-->(8) ├── select │ ├── columns: y:2!null │ ├── scan xyzbs @@ -3167,7 +3167,7 @@ scalar-group-by │ └── filters │ └── y:2 < 50 [outer=(2), constraints=(/2: (/NULL - /49]; tight)] └── aggregations - └── json-agg [as=json_agg:7, outer=(2)] + └── json-agg [as=json_agg:8, outer=(2)] └── y:2 # CORR case. @@ -3176,10 +3176,10 @@ norm expect=PushAggFilterIntoScalarGroupBy SELECT corr(y, z) FILTER (WHERE y < 50) FROM xyzbs ---- scalar-group-by - ├── columns: corr:7 + ├── columns: corr:8 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(7) + ├── fd: ()-->(8) ├── select │ ├── columns: y:2!null z:3!null │ ├── scan xyzbs @@ -3187,7 +3187,7 @@ scalar-group-by │ └── filters │ └── y:2 < 50 [outer=(2), constraints=(/2: (/NULL - /49]; tight)] └── aggregations - └── corr [as=corr:7, outer=(2,3)] + └── corr [as=corr:8, outer=(2,3)] ├── y:2 └── z:3 @@ -3197,13 +3197,13 @@ norm expect=PushAggFilterIntoScalarGroupBy SELECT string_agg(s, '-') FILTER (WHERE s < 'abc') FROM xyzbs ---- scalar-group-by - ├── columns: string_agg:8 + ├── columns: string_agg:9 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(8) + ├── fd: ()-->(9) ├── project - │ ├── columns: column6:6!null s:5!null - │ ├── fd: ()-->(6) + │ ├── columns: column7:7!null s:5!null + │ ├── fd: ()-->(7) │ ├── select │ │ ├── columns: s:5!null │ │ ├── scan xyzbs @@ -3211,11 +3211,11 @@ scalar-group-by │ │ └── filters │ │ └── s:5 < 'abc' [outer=(5), constraints=(/5: (/NULL - /'abc'); tight)] │ └── projections - │ └── '-' [as=column6:6] + │ └── '-' [as=column7:7] └── aggregations - └── string-agg [as=string_agg:8, outer=(5,6)] + └── string-agg [as=string_agg:9, outer=(5,7)] ├── s:5 - └── column6:6 + └── column7:7 # STRING_AGG case with an ORDER BY. # Expecting an index scan because opt command is used. @@ -3224,35 +3224,35 @@ opt expect=PushAggFilterIntoScalarGroupBy SELECT string_agg(s, '-') FILTER (WHERE s < 'abc') FROM (SELECT s FROM xyzbs ORDER BY s) ---- scalar-group-by - ├── columns: string_agg:8 - ├── internal-ordering: +5 opt(6) + ├── columns: string_agg:9 + ├── internal-ordering: +5 opt(7) ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(8) + ├── fd: ()-->(9) ├── project - │ ├── columns: column6:6!null s:5!null - │ ├── fd: ()-->(6) - │ ├── ordering: +5 opt(6) [actual: +5] + │ ├── columns: column7:7!null s:5!null + │ ├── fd: ()-->(7) + │ ├── ordering: +5 opt(7) [actual: +5] │ ├── scan xyzbs@secondary │ │ ├── columns: s:5!null │ │ ├── constraint: /5/1: (/NULL - /'abc') │ │ └── ordering: +5 │ └── projections - │ └── '-' [as=column6:6] + │ └── '-' [as=column7:7] └── aggregations - └── string-agg [as=string_agg:8, outer=(5,6)] + └── string-agg [as=string_agg:9, outer=(5,7)] ├── s:5 - └── column6:6 + └── column7:7 # Case with multiple conditions. norm expect=PushAggFilterIntoScalarGroupBy SELECT count(y) FILTER (WHERE y < 50 AND z > 5) FROM xyzbs ---- scalar-group-by - ├── columns: count:7!null + ├── columns: count:8!null ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(7) + ├── fd: ()-->(8) ├── select │ ├── columns: y:2!null z:3!null │ ├── scan xyzbs @@ -3261,7 +3261,7 @@ scalar-group-by │ ├── y:2 < 50 [outer=(2), constraints=(/2: (/NULL - /49]; tight)] │ └── z:3 > 5 [outer=(3), constraints=(/3: [/6 - ]; tight)] └── aggregations - └── count-rows [as=count:7] + └── count-rows [as=count:8] # No-op case where the same aggregate function is called on different # columns. @@ -3269,27 +3269,27 @@ norm expect-not=PushAggFilterIntoScalarGroupBy SELECT count(y) FILTER (WHERE y < 50), count(z) FILTER (WHERE z > 50) FROM xyzbs ---- scalar-group-by - ├── columns: count:7!null count:9!null + ├── columns: count:8!null count:10!null ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(7,9) + ├── fd: ()-->(8,10) ├── project - │ ├── columns: column6:6 column8:8!null y:2 z:3!null - │ ├── fd: (2)-->(6), (3)-->(8) + │ ├── columns: column7:7 column9:9!null y:2 z:3!null + │ ├── fd: (2)-->(7), (3)-->(9) │ ├── scan xyzbs │ │ └── columns: y:2 z:3!null │ └── projections - │ ├── y:2 < 50 [as=column6:6, outer=(2)] - │ └── z:3 > 50 [as=column8:8, outer=(3)] + │ ├── y:2 < 50 [as=column7:7, outer=(2)] + │ └── z:3 > 50 [as=column9:9, outer=(3)] └── aggregations - ├── agg-filter [as=count:7, outer=(2,6)] + ├── agg-filter [as=count:8, outer=(2,7)] │ ├── count │ │ └── y:2 - │ └── column6:6 - └── agg-filter [as=count:9, outer=(3,8)] + │ └── column7:7 + └── agg-filter [as=count:10, outer=(3,9)] ├── count │ └── z:3 - └── column8:8 + └── column9:9 # No-op case where different aggregate functions are called on the same # column. @@ -3297,26 +3297,26 @@ norm expect-not=PushAggFilterIntoScalarGroupBy SELECT count(y) FILTER (WHERE y < 50), sum(y) FILTER (WHERE y < 50) FROM xyzbs ---- scalar-group-by - ├── columns: count:7!null sum:8 + ├── columns: count:8!null sum:9 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(7,8) + ├── fd: ()-->(8,9) ├── project - │ ├── columns: column6:6 y:2 - │ ├── fd: (2)-->(6) + │ ├── columns: column7:7 y:2 + │ ├── fd: (2)-->(7) │ ├── scan xyzbs │ │ └── columns: y:2 │ └── projections - │ └── y:2 < 50 [as=column6:6, outer=(2)] + │ └── y:2 < 50 [as=column7:7, outer=(2)] └── aggregations - ├── agg-filter [as=count:7, outer=(2,6)] + ├── agg-filter [as=count:8, outer=(2,7)] │ ├── count │ │ └── y:2 - │ └── column6:6 - └── agg-filter [as=sum:8, outer=(2,6)] + │ └── column7:7 + └── agg-filter [as=sum:9, outer=(2,7)] ├── sum │ └── y:2 - └── column6:6 + └── column7:7 # -------------------------------------------------- # ConvertCountToCountRows @@ -3327,83 +3327,83 @@ norm expect=ConvertCountToCountRows SELECT count(z) FROM xyzbs ---- scalar-group-by - ├── columns: count:6!null + ├── columns: count:7!null ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(6) + ├── fd: ()-->(7) ├── scan xyzbs └── aggregations - └── count-rows [as=count:6] + └── count-rows [as=count:7] norm expect=ConvertCountToCountRows SELECT count(1) FROM xyzbs ---- scalar-group-by - ├── columns: count:7!null + ├── columns: count:8!null ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(7) + ├── fd: ()-->(8) ├── scan xyzbs └── aggregations - └── count-rows [as=count:7] + └── count-rows [as=count:8] norm expect=ConvertCountToCountRows SELECT count(1 + z) FROM xyzbs ---- scalar-group-by - ├── columns: count:7!null + ├── columns: count:8!null ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(7) + ├── fd: ()-->(8) ├── scan xyzbs └── aggregations - └── count-rows [as=count:7] + └── count-rows [as=count:8] # GroupBy cases. norm expect=ConvertCountToCountRows SELECT count(z) FROM xyzbs GROUP BY s ---- project - ├── columns: count:6!null + ├── columns: count:7!null └── group-by - ├── columns: s:5 count:6!null + ├── columns: s:5 count:7!null ├── grouping columns: s:5 ├── key: (5) - ├── fd: (5)-->(6) + ├── fd: (5)-->(7) ├── scan xyzbs │ └── columns: s:5 └── aggregations - └── count-rows [as=count:6] + └── count-rows [as=count:7] norm expect=ConvertCountToCountRows SELECT count(1) FROM xyzbs GROUP BY s ---- project - ├── columns: count:7!null + ├── columns: count:8!null └── group-by - ├── columns: s:5 count:7!null + ├── columns: s:5 count:8!null ├── grouping columns: s:5 ├── key: (5) - ├── fd: (5)-->(7) + ├── fd: (5)-->(8) ├── scan xyzbs │ └── columns: s:5 └── aggregations - └── count-rows [as=count:7] + └── count-rows [as=count:8] norm expect=ConvertCountToCountRows SELECT count(1+z) FROM xyzbs GROUP BY s ---- project - ├── columns: count:7!null + ├── columns: count:8!null └── group-by - ├── columns: s:5 count:7!null + ├── columns: s:5 count:8!null ├── grouping columns: s:5 ├── key: (5) - ├── fd: (5)-->(7) + ├── fd: (5)-->(8) ├── scan xyzbs │ └── columns: s:5 └── aggregations - └── count-rows [as=count:7] + └── count-rows [as=count:8] # Case with multiple aggregate functions. # Expecting to activate on z and b but not y, because y can be null. @@ -3411,36 +3411,36 @@ norm expect=ConvertCountToCountRows SELECT count(y), corr(y, z), count(z), sum(y), count(b) FROM xyzbs ---- scalar-group-by - ├── columns: count:6!null corr:7 count:8!null sum:9 count:10!null + ├── columns: count:7!null corr:8 count:9!null sum:10 count:11!null ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(6-10) + ├── fd: ()-->(7-11) ├── scan xyzbs │ └── columns: y:2 z:3!null └── aggregations - ├── count [as=count:6, outer=(2)] + ├── count [as=count:7, outer=(2)] │ └── y:2 - ├── corr [as=corr:7, outer=(2,3)] + ├── corr [as=corr:8, outer=(2,3)] │ ├── y:2 │ └── z:3 - ├── count-rows [as=count:8] - ├── sum [as=sum:9, outer=(2)] + ├── count-rows [as=count:9] + ├── sum [as=sum:10, outer=(2)] │ └── y:2 - └── count-rows [as=count:10] + └── count-rows [as=count:11] # No-op case because y can contain nulls. norm expect-not=ConvertCountToCountRows SELECT count(y) FROM xyzbs ---- scalar-group-by - ├── columns: count:6!null + ├── columns: count:7!null ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(6) + ├── fd: ()-->(7) ├── scan xyzbs │ └── columns: y:2 └── aggregations - └── count [as=count:6, outer=(2)] + └── count [as=count:7, outer=(2)] └── y:2 # No-op case because the DISTINCT requires the count input column, so the count @@ -3449,12 +3449,12 @@ norm expect-not=ConvertCountToCountRows SELECT count(DISTINCT y) FROM xyzbs GROUP BY z ---- project - ├── columns: count:6!null + ├── columns: count:7!null └── group-by - ├── columns: z:3!null count:6!null + ├── columns: z:3!null count:7!null ├── grouping columns: z:3!null ├── key: (3) - ├── fd: (3)-->(6) + ├── fd: (3)-->(7) ├── distinct-on │ ├── columns: y:2 z:3!null │ ├── grouping columns: y:2 z:3!null @@ -3462,7 +3462,7 @@ project │ └── scan xyzbs │ └── columns: y:2 z:3!null └── aggregations - └── count [as=count:6, outer=(2)] + └── count [as=count:7, outer=(2)] └── y:2 # -------------------------------------------------- @@ -3474,15 +3474,15 @@ norm expect=FoldGroupingOperators SELECT sum(s) FROM (SELECT sum(x) FROM xy GROUP BY y) AS f(s) ---- scalar-group-by - ├── columns: sum:4 + ├── columns: sum:5 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(4) + ├── fd: ()-->(5) ├── scan xy │ ├── columns: x:1!null │ └── key: (1) └── aggregations - └── sum [as=sum:4, outer=(1)] + └── sum [as=sum:5, outer=(1)] └── x:1 # Case with count-rows aggregate. @@ -3490,27 +3490,27 @@ norm expect=FoldGroupingOperators SELECT sum_int(c) FROM (SELECT count(x) FROM xy GROUP BY y) AS f(c) ---- scalar-group-by - ├── columns: sum_int:4!null + ├── columns: sum_int:5!null ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(4) + ├── fd: ()-->(5) ├── scan xy └── aggregations - └── count-rows [as=sum_int:4] + └── count-rows [as=sum_int:5] # Case with a count aggregate. norm expect=FoldGroupingOperators SELECT sum_int(cnt) FROM (SELECT count(c2) FROM nullablecols GROUP BY c1) AS f(cnt) ---- scalar-group-by - ├── columns: sum_int:6!null + ├── columns: sum_int:7!null ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(6) + ├── fd: ()-->(7) ├── scan nullablecols │ └── columns: c2:2 └── aggregations - └── count [as=sum_int:6, outer=(2)] + └── count [as=sum_int:7, outer=(2)] └── c2:2 # Case with max aggregate. @@ -3518,15 +3518,15 @@ norm expect=FoldGroupingOperators SELECT max(m) FROM (SELECT max(x) FROM xy GROUP BY y) AS f(m) ---- scalar-group-by - ├── columns: max:4 + ├── columns: max:5 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(4) + ├── fd: ()-->(5) ├── scan xy │ ├── columns: x:1!null │ └── key: (1) └── aggregations - └── max [as=max:4, outer=(1)] + └── max [as=max:5, outer=(1)] └── x:1 # Case with bit_and aggregate. @@ -3534,15 +3534,15 @@ norm expect=FoldGroupingOperators SELECT bit_and(b) FROM (SELECT bit_and(x) FROM xy GROUP BY y) AS f(b) ---- scalar-group-by - ├── columns: bit_and:4 + ├── columns: bit_and:5 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(4) + ├── fd: ()-->(5) ├── scan xy │ ├── columns: x:1!null │ └── key: (1) └── aggregations - └── bit-and-agg [as=bit_and:4, outer=(1)] + └── bit-and-agg [as=bit_and:5, outer=(1)] └── x:1 # Case with multiple aggregates. @@ -3552,18 +3552,18 @@ FROM (SELECT sum(b), count(c), max(b) FROM abc GROUP BY a) AS f(s, c, m) ---- scalar-group-by - ├── columns: max:7 sum:8 sum_int:9!null + ├── columns: max:8 sum:9 sum_int:10!null ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(7-9) + ├── fd: ()-->(8-10) ├── scan abc │ └── columns: b:2!null └── aggregations - ├── max [as=max:7, outer=(2)] + ├── max [as=max:8, outer=(2)] │ └── b:2 - ├── sum [as=sum:8, outer=(2)] + ├── sum [as=sum:9, outer=(2)] │ └── b:2 - └── count-rows [as=sum_int:9] + └── count-rows [as=sum_int:10] # GroupBy on GroupBy case where the inner grouping columns determine the outer # grouping columns, but they do not intersect. @@ -3571,18 +3571,18 @@ norm expect=FoldGroupingOperators SELECT sum(s) FROM (SELECT y, sum(x) AS s FROM xy GROUP BY x) GROUP BY y ---- project - ├── columns: sum:4!null + ├── columns: sum:5!null └── group-by - ├── columns: y:2 sum:4!null + ├── columns: y:2 sum:5!null ├── grouping columns: y:2 ├── key: (2) - ├── fd: (2)-->(4) + ├── fd: (2)-->(5) ├── scan xy │ ├── columns: x:1!null y:2 │ ├── key: (1) │ └── fd: (1)-->(2) └── aggregations - └── sum [as=sum:4, outer=(1)] + └── sum [as=sum:5, outer=(1)] └── x:1 # GroupBy on GroupBy case with multiple-column grouping. @@ -3590,16 +3590,16 @@ norm expect=FoldGroupingOperators SELECT sum(s) FROM (SELECT a, sum(c) AS s FROM abc GROUP BY a, b) GROUP BY a ---- project - ├── columns: sum:5!null + ├── columns: sum:6!null └── group-by - ├── columns: a:1!null sum:5!null + ├── columns: a:1!null sum:6!null ├── grouping columns: a:1!null ├── key: (1) - ├── fd: (1)-->(5) + ├── fd: (1)-->(6) ├── scan abc │ └── columns: a:1!null c:3!null └── aggregations - └── sum [as=sum:5, outer=(3)] + └── sum [as=sum:6, outer=(3)] └── c:3 # No-op case with an AvgOp. Note: this query actually could be folded if the @@ -3608,55 +3608,55 @@ norm expect-not=FoldGroupingOperators SELECT sum(a) FROM (SELECT avg(x) FROM xy GROUP BY y) AS f(a) ---- scalar-group-by - ├── columns: sum:4 + ├── columns: sum:5 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(4) + ├── fd: ()-->(5) ├── group-by - │ ├── columns: y:2 avg:3!null + │ ├── columns: y:2 avg:4!null │ ├── grouping columns: y:2 │ ├── key: (2) - │ ├── fd: (2)-->(3) + │ ├── fd: (2)-->(4) │ ├── scan xy │ │ ├── columns: x:1!null y:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ └── aggregations - │ └── avg [as=avg:3, outer=(1)] + │ └── avg [as=avg:4, outer=(1)] │ └── x:1 └── aggregations - └── sum [as=sum:4, outer=(3)] - └── avg:3 + └── sum [as=sum:5, outer=(4)] + └── avg:4 # No-op case with several valid aggregate pairs and one invalid pair. norm expect-not=FoldGroupingOperators SELECT sum(c), sum(s), max(s) FROM (SELECT sum(x), count(x) FROM xy GROUP BY y) AS f(s, c) ---- scalar-group-by - ├── columns: sum:5 sum:6 max:7 + ├── columns: sum:6 sum:7 max:8 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(5-7) + ├── fd: ()-->(6-8) ├── group-by - │ ├── columns: y:2 sum:3!null count:4!null + │ ├── columns: y:2 sum:4!null count:5!null │ ├── grouping columns: y:2 │ ├── key: (2) - │ ├── fd: (2)-->(3,4) + │ ├── fd: (2)-->(4,5) │ ├── scan xy │ │ ├── columns: x:1!null y:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ └── aggregations - │ ├── sum [as=sum:3, outer=(1)] + │ ├── sum [as=sum:4, outer=(1)] │ │ └── x:1 - │ └── count-rows [as=count:4] + │ └── count-rows [as=count:5] └── aggregations - ├── sum [as=sum:5, outer=(4)] - │ └── count:4 - ├── sum [as=sum:6, outer=(3)] - │ └── sum:3 - └── max [as=max:7, outer=(3)] - └── sum:3 + ├── sum [as=sum:6, outer=(5)] + │ └── count:5 + ├── sum [as=sum:7, outer=(4)] + │ └── sum:4 + └── max [as=max:8, outer=(4)] + └── sum:4 # No-op case because the outer grouping columns are not functionally determined # by the inner grouping columns in the functional dependencies of the input of @@ -3665,29 +3665,29 @@ norm expect-not=FoldGroupingOperators SELECT max(m) FROM (SELECT max(x) AS m, sum(x) AS s FROM xy GROUP BY y) GROUP BY s ---- project - ├── columns: max:5!null + ├── columns: max:6!null └── group-by - ├── columns: sum:4!null max:5!null - ├── grouping columns: sum:4!null - ├── key: (4) - ├── fd: (4)-->(5) + ├── columns: sum:5!null max:6!null + ├── grouping columns: sum:5!null + ├── key: (5) + ├── fd: (5)-->(6) ├── group-by - │ ├── columns: y:2 max:3!null sum:4!null + │ ├── columns: y:2 max:4!null sum:5!null │ ├── grouping columns: y:2 │ ├── key: (2) - │ ├── fd: (2)-->(3,4) + │ ├── fd: (2)-->(4,5) │ ├── scan xy │ │ ├── columns: x:1!null y:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ └── aggregations - │ ├── max [as=max:3, outer=(1)] + │ ├── max [as=max:4, outer=(1)] │ │ └── x:1 - │ └── sum [as=sum:4, outer=(1)] + │ └── sum [as=sum:5, outer=(1)] │ └── x:1 └── aggregations - └── max [as=max:5, outer=(3)] - └── max:3 + └── max [as=max:6, outer=(4)] + └── max:4 # No-op case because one of the grouping operators has an internal ordering. The # array_agg ensures that the GroupBy has an internal ordering. @@ -3695,24 +3695,24 @@ norm expect-not=FoldGroupingOperators SELECT sum(s) FROM (SELECT sum(z) AS s, array_agg(z) FROM (SELECT * FROM uvwz ORDER BY w DESC) GROUP BY u) ---- scalar-group-by - ├── columns: sum:8 + ├── columns: sum:9 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(8) + ├── fd: ()-->(9) ├── group-by - │ ├── columns: u:1!null sum:6!null + │ ├── columns: u:1!null sum:7!null │ ├── grouping columns: u:1!null │ ├── internal-ordering: -3 opt(1) │ ├── key: (1) - │ ├── fd: (1)-->(6) + │ ├── fd: (1)-->(7) │ ├── sort │ │ ├── columns: u:1!null w:3!null z:4!null │ │ ├── ordering: -3 opt(1) [actual: -3] │ │ └── scan uvwz │ │ └── columns: u:1!null w:3!null z:4!null │ └── aggregations - │ └── sum [as=sum:6, outer=(4)] + │ └── sum [as=sum:7, outer=(4)] │ └── z:4 └── aggregations - └── sum [as=sum:8, outer=(6)] - └── sum:6 + └── sum [as=sum:9, outer=(7)] + └── sum:7 diff --git a/pkg/sql/opt/norm/testdata/rules/inline b/pkg/sql/opt/norm/testdata/rules/inline index ccc924e8e724..427b74f48b92 100644 --- a/pkg/sql/opt/norm/testdata/rules/inline +++ b/pkg/sql/opt/norm/testdata/rules/inline @@ -131,9 +131,9 @@ norm expect=InlineConstVar SELECT * FROM a INNER JOIN xy ON True WHERE y=10 AND i(7), (1)-->(2-5) + ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:7!null y:8!null + ├── key: (1,7) + ├── fd: ()-->(8), (1)-->(2-5) ├── select │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 │ ├── key: (1) @@ -145,15 +145,15 @@ inner-join (cross) │ └── filters │ └── i:2 < 10 [outer=(2), constraints=(/2: (/NULL - /9]; tight)] ├── select - │ ├── columns: x:6!null y:7!null - │ ├── key: (6) - │ ├── fd: ()-->(7) + │ ├── columns: x:7!null y:8!null + │ ├── key: (7) + │ ├── fd: ()-->(8) │ ├── scan xy - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ └── filters - │ └── y:7 = 10 [outer=(7), constraints=(/7: [/10 - /10]; tight), fd=()-->(7)] + │ └── y:8 = 10 [outer=(8), constraints=(/8: [/10 - /10]; tight), fd=()-->(8)] └── filters (true) # -------------------------------------------------- @@ -166,28 +166,28 @@ UPDATE computed SET a=1, b=2 ---- update computed ├── columns: - ├── fetch columns: a:4 b:5 c:6 + ├── fetch columns: a:5 b:6 c:7 ├── update-mapping: - │ ├── a_new:7 => a:1 - │ ├── b_new:8 => b:2 - │ └── column9:9 => c:3 + │ ├── a_new:9 => a:1 + │ ├── b_new:10 => b:2 + │ └── column11:11 => c:3 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: column9:9!null a_new:7!null b_new:8!null a:4!null b:5 c:6 - ├── key: (4) - ├── fd: ()-->(7-9), (4)-->(5,6) + ├── columns: column11:11!null a_new:9!null b_new:10!null a:5!null b:6 c:7 + ├── key: (5) + ├── fd: ()-->(9-11), (5)-->(6,7) ├── scan computed - │ ├── columns: a:4!null b:5 c:6 + │ ├── columns: a:5!null b:6 c:7 │ ├── computed column expressions - │ │ └── c:6 - │ │ └── (a:4 + b:5) + 1 - │ ├── key: (4) - │ └── fd: (4)-->(5,6) + │ │ └── c:7 + │ │ └── (a:5 + b:6) + 1 + │ ├── key: (5) + │ └── fd: (5)-->(6,7) └── projections - ├── 4 [as=column9:9] - ├── 1 [as=a_new:7] - └── 2 [as=b_new:8] + ├── 4 [as=column11:11] + ├── 1 [as=a_new:9] + └── 2 [as=b_new:10] # Inline constants from Values expression. norm expect=InlineProjectConstants @@ -215,16 +215,16 @@ norm expect=InlineProjectConstants SELECT one+two, x, one*two, y FROM (SELECT x, 1 AS one, y, 2 AS two FROM xy) ---- project - ├── columns: "?column?":5!null x:1!null "?column?":6!null y:2 + ├── columns: "?column?":6!null x:1!null "?column?":7!null y:2 ├── key: (1) - ├── fd: ()-->(5,6), (1)-->(2) + ├── fd: ()-->(6,7), (1)-->(2) ├── scan xy │ ├── columns: x:1!null y:2 │ ├── key: (1) │ └── fd: (1)-->(2) └── projections - ├── 3 [as="?column?":5] - └── 2 [as="?column?":6] + ├── 3 [as="?column?":6] + └── 2 [as="?column?":7] # Constant column reference within correlated subquery (which becomes # uncorrelated as a result). @@ -232,10 +232,10 @@ norm expect=InlineProjectConstants SELECT EXISTS(SELECT * FROM a WHERE k=one AND i=two) FROM (VALUES (1, 2)) AS t(one, two) ---- values - ├── columns: exists:8 + ├── columns: exists:9 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(8) + ├── fd: ()-->(9) └── tuple └── exists └── select @@ -276,11 +276,11 @@ norm expect=InlineSelectConstants SELECT * FROM (SELECT 1 AS one from xy) WHERE one > 0 ---- project - ├── columns: one:3!null - ├── fd: ()-->(3) + ├── columns: one:4!null + ├── fd: ()-->(4) ├── scan xy └── projections - └── 1 [as=one:3] + └── 1 [as=one:4] # Inline constants from Values expression. norm expect=InlineSelectConstants @@ -310,10 +310,10 @@ norm expect=InlineSelectConstants SELECT * FROM (SELECT x, 1 AS one, y, 2 AS two FROM xy) WHERE x=one AND y=two ---- project - ├── columns: x:1!null one:3!null y:2!null two:4!null + ├── columns: x:1!null one:4!null y:2!null two:5!null ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(1-4) + ├── fd: ()-->(1,2,4,5) ├── select │ ├── columns: x:1!null y:2!null │ ├── cardinality: [0 - 1] @@ -327,8 +327,8 @@ project │ ├── x:1 = 1 [outer=(1), constraints=(/1: [/1 - /1]; tight), fd=()-->(1)] │ └── y:2 = 2 [outer=(2), constraints=(/2: [/2 - /2]; tight), fd=()-->(2)] └── projections - ├── 1 [as=one:3] - └── 2 [as=two:4] + ├── 1 [as=one:4] + └── 2 [as=two:5] # Do not inline constants from Values expression with multiple rows. norm expect-not=InlineSelectConstants @@ -381,20 +381,20 @@ norm expect=InlineJoinConstantsRight SELECT * FROM a FULL JOIN (SELECT 1 AS one) ON k=one ---- full-join (cross) - ├── columns: k:1 i:2 f:3 s:4 j:5 one:6 + ├── columns: k:1 i:2 f:3 s:4 j:5 one:7 ├── cardinality: [1 - ] ├── multiplicity: left-rows(exactly-one), right-rows(one-or-more) ├── key: (1) - ├── fd: (1)-->(2-6) + ├── fd: (1)-->(2-5,7) ├── scan a │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── values - │ ├── columns: one:6!null + │ ├── columns: one:7!null │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(6) + │ ├── fd: ()-->(7) │ └── (1,) └── filters └── k:1 = 1 [outer=(1), constraints=(/1: [/1 - /1]; tight), fd=()-->(1)] @@ -413,22 +413,22 @@ norm expect-not=(InlineJoinConstantsLeft,InlineJoinConstantsRight) SELECT * FROM a INNER JOIN (SELECT 1 AS one, y FROM xy) ON k=y ---- inner-join (hash) - ├── columns: k:1!null i:2 f:3 s:4 j:5 one:8!null y:7!null + ├── columns: k:1!null i:2 f:3 s:4 j:5 one:10!null y:8!null ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) - ├── fd: ()-->(8), (1)-->(2-5), (1)==(7), (7)==(1) + ├── fd: ()-->(10), (1)-->(2-5), (1)==(8), (8)==(1) ├── scan a │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── project - │ ├── columns: one:8!null y:7 - │ ├── fd: ()-->(8) + │ ├── columns: one:10!null y:8 + │ ├── fd: ()-->(10) │ ├── scan xy - │ │ └── columns: y:7 + │ │ └── columns: y:8 │ └── projections - │ └── 1 [as=one:8] + │ └── 1 [as=one:10] └── filters - └── k:1 = y:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + └── k:1 = y:8 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] # -------------------------------------------------- # PushSelectIntoInlinableProject @@ -439,7 +439,7 @@ norm expect=PushSelectIntoInlinableProject SELECT * FROM (SELECT k=1 AS expr FROM a) a WHERE expr IS NULL ---- project - ├── columns: expr:6!null + ├── columns: expr:7!null ├── select │ ├── columns: k:1!null │ ├── key: (1) @@ -449,14 +449,14 @@ project │ └── filters │ └── (k:1 = 1) IS NULL [outer=(1)] └── projections - └── k:1 = 1 [as=expr:6, outer=(1)] + └── k:1 = 1 [as=expr:7, outer=(1)] # Inline arithmetic. norm expect=PushSelectIntoInlinableProject SELECT * FROM (SELECT k*2+1 AS expr FROM a) a WHERE expr > 10 ---- project - ├── columns: expr:6!null + ├── columns: expr:7!null ├── immutable ├── select │ ├── columns: k:1!null @@ -468,14 +468,14 @@ project │ └── filters │ └── (k:1 * 2) > 9 [outer=(1), immutable] └── projections - └── (k:1 * 2) + 1 [as=expr:6, outer=(1), immutable] + └── (k:1 * 2) + 1 [as=expr:7, outer=(1), immutable] # Inline boolean logic. norm expect=PushSelectIntoInlinableProject SELECT * FROM (SELECT NOT(k>1 AND k<=5) AS expr FROM a) a WHERE expr ---- project - ├── columns: expr:6!null + ├── columns: expr:7!null ├── select │ ├── columns: k:1!null │ ├── key: (1) @@ -485,14 +485,14 @@ project │ └── filters │ └── (k:1 <= 1) OR (k:1 > 5) [outer=(1), constraints=(/1: (/NULL - /1] [/6 - ]; tight)] └── projections - └── (k:1 <= 1) OR (k:1 > 5) [as=expr:6, outer=(1)] + └── (k:1 <= 1) OR (k:1 > 5) [as=expr:7, outer=(1)] # Inline constants. norm expect=PushSelectIntoInlinableProject SELECT * FROM (SELECT (f IS NULL OR f != 10.5) AS expr FROM a) a WHERE expr ---- project - ├── columns: expr:6 + ├── columns: expr:7 ├── select │ ├── columns: f:3 │ ├── scan a @@ -500,14 +500,14 @@ project │ └── filters │ └── (f:3 IS NULL) OR (f:3 != 10.5) [outer=(3), constraints=(/3: [/NULL - /10.499999999999998] [/10.500000000000002 - ]; tight)] └── projections - └── (f:3 IS NULL) OR (f:3 != 10.5) [as=expr:6, outer=(3)] + └── (f:3 IS NULL) OR (f:3 != 10.5) [as=expr:7, outer=(3)] # Reference the expression to inline multiple times. norm expect=PushSelectIntoInlinableProject SELECT * FROM (SELECT f+1 AS expr FROM a) a WHERE expr=expr ---- project - ├── columns: expr:6 + ├── columns: expr:7 ├── immutable ├── select │ ├── columns: f:3 @@ -517,7 +517,7 @@ project │ └── filters │ └── (f:3 + 1.0) IS DISTINCT FROM CAST(NULL AS FLOAT8) [outer=(3), immutable] └── projections - └── f:3 + 1.0 [as=expr:6, outer=(3), immutable] + └── f:3 + 1.0 [as=expr:7, outer=(3), immutable] # Use outer references in both inlined expression and in referencing expression. norm expect=PushSelectIntoInlinableProject @@ -533,10 +533,10 @@ semi-join (cross) │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── scan xy - │ ├── columns: x:6!null - │ └── key: (6) + │ ├── columns: x:7!null + │ └── key: (7) └── filters - └── (x:6 - i:2) > (i:2 * i:2) [outer=(2,6), immutable] + └── (x:7 - i:2) > (i:2 * i:2) [outer=(2,7), immutable] exec-ddl CREATE TABLE crdb_internal.zones ( @@ -589,45 +589,45 @@ LIMIT 107 ---- project - ├── columns: c0:6!null + ├── columns: c0:7!null ├── cardinality: [0 - 107] ├── volatile └── limit - ├── columns: c0:6!null c1:7!null + ├── columns: c0:7!null c1:8!null ├── cardinality: [0 - 107] ├── volatile ├── select - │ ├── columns: c0:6!null c1:7!null + │ ├── columns: c0:7!null c1:8!null │ ├── volatile │ ├── limit hint: 107.00 │ ├── project - │ │ ├── columns: c0:6!null c1:7!null + │ │ ├── columns: c0:7!null c1:8!null │ │ ├── immutable │ │ ├── limit hint: 321.00 │ │ ├── scan crdb_internal.public.zones │ │ │ ├── columns: crdb_internal.public.zones.zone_id:1!null │ │ │ └── limit hint: 321.00 │ │ └── projections - │ │ ├── crdb_internal.public.zones.zone_id:1 + 1 [as=c0:6, outer=(1), immutable] - │ │ └── crdb_internal.public.zones.zone_id:1 + 2 [as=c1:7, outer=(1), immutable] + │ │ ├── crdb_internal.public.zones.zone_id:1 + 1 [as=c0:7, outer=(1), immutable] + │ │ └── crdb_internal.public.zones.zone_id:1 + 2 [as=c1:8, outer=(1), immutable] │ └── filters - │ └── le [outer=(6,7), volatile, correlated-subquery] + │ └── le [outer=(7,8), volatile, correlated-subquery] │ ├── case │ │ ├── true │ │ ├── when - │ │ │ ├── c1:7 IS NOT NULL + │ │ │ ├── c1:8 IS NOT NULL │ │ │ └── function: extract │ │ │ ├── case │ │ │ │ ├── true │ │ │ │ ├── when │ │ │ │ │ ├── exists │ │ │ │ │ │ └── select - │ │ │ │ │ │ ├── columns: ref_1.config_yaml:10!null - │ │ │ │ │ │ ├── outer: (6) + │ │ │ │ │ │ ├── columns: ref_1.config_yaml:11!null + │ │ │ │ │ │ ├── outer: (7) │ │ │ │ │ │ ├── scan ref_1 - │ │ │ │ │ │ │ └── columns: ref_1.config_yaml:10!null + │ │ │ │ │ │ │ └── columns: ref_1.config_yaml:11!null │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── c0:6 IS NOT NULL [outer=(6), constraints=(/6: (/NULL - ]; tight)] + │ │ │ │ │ │ └── c0:7 IS NOT NULL [outer=(7), constraints=(/7: (/NULL - ]; tight)] │ │ │ │ │ └── version() │ │ │ │ └── version() │ │ │ └── '2017-05-10' @@ -642,15 +642,15 @@ norm expect=InlineProjectInProject SELECT NOT(expr), i+1 AS r FROM (SELECT k=1 AS expr, i FROM a) ---- project - ├── columns: "?column?":7!null r:8 + ├── columns: "?column?":8!null r:9 ├── immutable ├── scan a │ ├── columns: k:1!null i:2 │ ├── key: (1) │ └── fd: (1)-->(2) └── projections - ├── k:1 != 1 [as="?column?":7, outer=(1)] - └── i:2 + 1 [as=r:8, outer=(2), immutable] + ├── k:1 != 1 [as="?column?":8, outer=(1)] + └── i:2 + 1 [as=r:9, outer=(2), immutable] # Multiple synthesized column references to same inner passthrough column # (should still inline). @@ -658,16 +658,16 @@ norm expect=InlineProjectInProject SELECT x+1, x+2, y1+2 FROM (SELECT x, y+1 AS y1 FROM xy) ---- project - ├── columns: "?column?":4!null "?column?":5!null "?column?":6 + ├── columns: "?column?":5!null "?column?":6!null "?column?":7 ├── immutable ├── scan xy │ ├── columns: x:1!null y:2 │ ├── key: (1) │ └── fd: (1)-->(2) └── projections - ├── x:1 + 1 [as="?column?":4, outer=(1), immutable] - ├── x:1 + 2 [as="?column?":5, outer=(1), immutable] - └── (y:2 + 1) + 2 [as="?column?":6, outer=(2), immutable] + ├── x:1 + 1 [as="?column?":5, outer=(1), immutable] + ├── x:1 + 2 [as="?column?":6, outer=(1), immutable] + └── (y:2 + 1) + 2 [as="?column?":7, outer=(2), immutable] # Synthesized and passthrough references to same inner passthrough column # (should still inline). @@ -675,10 +675,10 @@ norm expect=InlineProjectInProject SELECT x+y1 FROM (SELECT x, y+1 AS y1 FROM xy) ORDER BY x ---- project - ├── columns: "?column?":4 [hidden: x:1!null] + ├── columns: "?column?":5 [hidden: x:1!null] ├── immutable ├── key: (1) - ├── fd: (1)-->(4) + ├── fd: (1)-->(5) ├── ordering: +1 ├── scan xy │ ├── columns: x:1!null y:2 @@ -686,126 +686,126 @@ project │ ├── fd: (1)-->(2) │ └── ordering: +1 └── projections - └── x:1 + (y:2 + 1) [as="?column?":4, outer=(1,2), immutable] + └── x:1 + (y:2 + 1) [as="?column?":5, outer=(1,2), immutable] # Inline multiple expressions. norm expect=InlineProjectInProject SELECT expr+1 AS r, i, expr2 || 'bar' AS s FROM (SELECT k+1 AS expr, s || 'foo' AS expr2, i FROM a) ---- project - ├── columns: r:8!null i:2 s:9 + ├── columns: r:9!null i:2 s:10 ├── immutable ├── scan a │ ├── columns: k:1!null i:2 a.s:4 │ ├── key: (1) │ └── fd: (1)-->(2,4) └── projections - ├── (k:1 + 1) + 1 [as=r:8, outer=(1), immutable] - └── (a.s:4 || 'foo') || 'bar' [as=s:9, outer=(4), immutable] + ├── (k:1 + 1) + 1 [as=r:9, outer=(1), immutable] + └── (a.s:4 || 'foo') || 'bar' [as=s:10, outer=(4), immutable] # Don't inline when there are multiple references. norm expect-not=InlineProjectInProject SELECT expr, expr*2 AS r FROM (SELECT k+1 AS expr FROM a) ---- project - ├── columns: expr:6!null r:7!null + ├── columns: expr:7!null r:8!null ├── immutable - ├── fd: (6)-->(7) + ├── fd: (7)-->(8) ├── project - │ ├── columns: expr:6!null + │ ├── columns: expr:7!null │ ├── immutable │ ├── scan a │ │ ├── columns: k:1!null │ │ └── key: (1) │ └── projections - │ └── k:1 + 1 [as=expr:6, outer=(1), immutable] + │ └── k:1 + 1 [as=expr:7, outer=(1), immutable] └── projections - └── expr:6 * 2 [as=r:7, outer=(6), immutable] + └── expr:7 * 2 [as=r:8, outer=(7), immutable] # Uncorrelated subquery should not block inlining. norm expect=InlineProjectInProject SELECT EXISTS(SELECT * FROM xy WHERE x=1 OR x=2), expr*2 AS r FROM (SELECT k+1 AS expr FROM a) ---- project - ├── columns: exists:9 r:10!null + ├── columns: exists:11 r:12!null ├── immutable - ├── fd: ()-->(9) + ├── fd: ()-->(11) ├── scan a │ ├── columns: k:1!null │ └── key: (1) └── projections - ├── exists [as=exists:9, subquery] + ├── exists [as=exists:11, subquery] │ └── limit - │ ├── columns: x:7!null y:8 + │ ├── columns: x:8!null y:9 │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(7,8) + │ ├── fd: ()-->(8,9) │ ├── select - │ │ ├── columns: x:7!null y:8 + │ │ ├── columns: x:8!null y:9 │ │ ├── cardinality: [0 - 2] - │ │ ├── key: (7) - │ │ ├── fd: (7)-->(8) + │ │ ├── key: (8) + │ │ ├── fd: (8)-->(9) │ │ ├── limit hint: 1.00 │ │ ├── scan xy - │ │ │ ├── columns: x:7!null y:8 - │ │ │ ├── key: (7) - │ │ │ ├── fd: (7)-->(8) + │ │ │ ├── columns: x:8!null y:9 + │ │ │ ├── key: (8) + │ │ │ ├── fd: (8)-->(9) │ │ │ └── limit hint: 500.00 │ │ └── filters - │ │ └── (x:7 = 1) OR (x:7 = 2) [outer=(7), constraints=(/7: [/1 - /1] [/2 - /2]; tight)] + │ │ └── (x:8 = 1) OR (x:8 = 2) [outer=(8), constraints=(/8: [/1 - /1] [/2 - /2]; tight)] │ └── 1 - └── (k:1 + 1) * 2 [as=r:10, outer=(1), immutable] + └── (k:1 + 1) * 2 [as=r:12, outer=(1), immutable] # Correlated subquery should be hoisted as usual. norm expect=InlineProjectInProject SELECT EXISTS(SELECT * FROM xy WHERE expr<0) FROM (SELECT k+1 AS expr FROM a) ---- project - ├── columns: exists:9!null + ├── columns: exists:11!null ├── immutable ├── group-by - │ ├── columns: true_agg:11 rownum:13!null - │ ├── grouping columns: rownum:13!null + │ ├── columns: true_agg:13 rownum:15!null + │ ├── grouping columns: rownum:15!null │ ├── immutable - │ ├── key: (13) - │ ├── fd: (13)-->(11) + │ ├── key: (15) + │ ├── fd: (15)-->(13) │ ├── left-join (cross) - │ │ ├── columns: expr:6!null true:10 rownum:13!null + │ │ ├── columns: expr:7!null true:12 rownum:15!null │ │ ├── immutable - │ │ ├── fd: (13)-->(6) + │ │ ├── fd: (15)-->(7) │ │ ├── ordinality - │ │ │ ├── columns: expr:6!null rownum:13!null + │ │ │ ├── columns: expr:7!null rownum:15!null │ │ │ ├── immutable - │ │ │ ├── key: (13) - │ │ │ ├── fd: (13)-->(6) + │ │ │ ├── key: (15) + │ │ │ ├── fd: (15)-->(7) │ │ │ └── project - │ │ │ ├── columns: expr:6!null + │ │ │ ├── columns: expr:7!null │ │ │ ├── immutable │ │ │ ├── scan a │ │ │ │ ├── columns: k:1!null │ │ │ │ └── key: (1) │ │ │ └── projections - │ │ │ └── k:1 + 1 [as=expr:6, outer=(1), immutable] + │ │ │ └── k:1 + 1 [as=expr:7, outer=(1), immutable] │ │ ├── project - │ │ │ ├── columns: true:10!null - │ │ │ ├── fd: ()-->(10) + │ │ │ ├── columns: true:12!null + │ │ │ ├── fd: ()-->(12) │ │ │ ├── scan xy │ │ │ └── projections - │ │ │ └── true [as=true:10] + │ │ │ └── true [as=true:12] │ │ └── filters - │ │ └── expr:6 < 0 [outer=(6), constraints=(/6: (/NULL - /-1]; tight)] + │ │ └── expr:7 < 0 [outer=(7), constraints=(/7: (/NULL - /-1]; tight)] │ └── aggregations - │ └── const-not-null-agg [as=true_agg:11, outer=(10)] - │ └── true:10 + │ └── const-not-null-agg [as=true_agg:13, outer=(12)] + │ └── true:12 └── projections - └── true_agg:11 IS NOT NULL [as=exists:9, outer=(11)] + └── true_agg:13 IS NOT NULL [as=exists:11, outer=(13)] # After c is replaced with k+2, (k+2) > 2 should be simplified to k > 0. norm SELECT c FROM (SELECT k+2 AS c FROM a) AS t WHERE c > 2; ---- project - ├── columns: c:6!null + ├── columns: c:7!null ├── immutable ├── select │ ├── columns: k:1!null @@ -816,4 +816,4 @@ project │ └── filters │ └── k:1 > 0 [outer=(1), constraints=(/1: [/1 - ]; tight)] └── projections - └── k:1 + 2 [as=c:6, outer=(1), immutable] + └── k:1 + 2 [as=c:7, outer=(1), immutable] diff --git a/pkg/sql/opt/norm/testdata/rules/join b/pkg/sql/opt/norm/testdata/rules/join index 9fcc645a7217..ef9bd51cb41e 100644 --- a/pkg/sql/opt/norm/testdata/rules/join +++ b/pkg/sql/opt/norm/testdata/rules/join @@ -29,19 +29,19 @@ norm SELECT * FROM a INNER JOIN b ON a.s='foo' OR b.y<10 ---- inner-join (cross) - ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:6!null y:7 - ├── key: (1,6) - ├── fd: (1)-->(2-5), (6)-->(7) + ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:7!null y:8 + ├── key: (1,7) + ├── fd: (1)-->(2-5), (7)-->(8) ├── scan a │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── scan b - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + │ └── fd: (7)-->(8) └── filters - └── (s:4 = 'foo') OR (y:7 < 10) [outer=(4,7)] + └── (s:4 = 'foo') OR (y:8 < 10) [outer=(4,8)] # -------------------------------------------------- # CommuteRightJoin @@ -51,20 +51,20 @@ norm SELECT * FROM a RIGHT JOIN b ON k=x ---- left-join (hash) - ├── columns: k:1 i:2 f:3 s:4 j:5 x:6!null y:7 + ├── columns: k:1 i:2 f:3 s:4 j:5 x:7!null y:8 ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) - ├── key: (6) - ├── fd: (6)-->(1-5,7), (1)-->(2-5) + ├── key: (7) + ├── fd: (7)-->(1-5,8), (1)-->(2-5) ├── scan b - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + │ └── fd: (7)-->(8) ├── scan a │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters - └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # -------------------------------------------------- # DetectJoinContradiction @@ -74,45 +74,45 @@ norm expect=DetectJoinContradiction SELECT * FROM a INNER JOIN b ON (k<1 AND k>2) OR (k<4 AND k>5) ---- values - ├── columns: k:1!null i:2!null f:3!null s:4!null j:5!null x:6!null y:7!null + ├── columns: k:1!null i:2!null f:3!null s:4!null j:5!null x:7!null y:8!null ├── cardinality: [0 - 0] ├── key: () - └── fd: ()-->(1-7) + └── fd: ()-->(1-5,7,8) norm expect=DetectJoinContradiction SELECT * FROM a LEFT JOIN b ON (k<1 AND k>2) OR (k<4 AND k>5) ---- left-join (cross) - ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:6 y:7 + ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:7 y:8 ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) ├── key: (1) - ├── fd: (1)-->(2-7) + ├── fd: (1)-->(2-5,7,8) ├── scan a │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── values - │ ├── columns: x:6!null y:7!null + │ ├── columns: x:7!null y:8!null │ ├── cardinality: [0 - 0] │ ├── key: () - │ └── fd: ()-->(6,7) + │ └── fd: ()-->(7,8) └── filters (true) norm expect=DetectJoinContradiction SELECT * FROM a FULL JOIN b ON i=5 AND ((k<1 AND k>2) OR (k<4 AND k>5)) AND s='foo' ---- full-join (cross) - ├── columns: k:1 i:2 f:3 s:4 j:5 x:6 y:7 - ├── key: (1,6) - ├── fd: (1)-->(2-5), (6)-->(7) + ├── columns: k:1 i:2 f:3 s:4 j:5 x:7 y:8 + ├── key: (1,7) + ├── fd: (1)-->(2-5), (7)-->(8) ├── scan a │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── scan b - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + │ └── fd: (7)-->(8) └── filters └── false @@ -123,10 +123,10 @@ norm expect=PushFilterIntoJoinLeft SELECT * FROM a INNER JOIN b ON a.k=b.x AND a.s='foo' ---- inner-join (hash) - ├── columns: k:1!null i:2 f:3!null s:4!null j:5 x:6!null y:7 + ├── columns: k:1!null i:2 f:3!null s:4!null j:5 x:7!null y:8 ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (6) - ├── fd: ()-->(4), (1)-->(2,3,5), (6)-->(7), (1)==(6), (6)==(1) + ├── key: (7) + ├── fd: ()-->(4), (1)-->(2,3,5), (7)-->(8), (1)==(7), (7)==(1) ├── select │ ├── columns: k:1!null i:2 f:3!null s:4!null j:5 │ ├── key: (1) @@ -138,31 +138,31 @@ inner-join (hash) │ └── filters │ └── s:4 = 'foo' [outer=(4), constraints=(/4: [/'foo' - /'foo']; tight), fd=()-->(4)] ├── scan b - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + │ └── fd: (7)-->(8) └── filters - └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # LEFT JOIN should not push down conditions to left side of join. norm expect-not=PushFilterIntoJoinLeft SELECT * FROM a LEFT JOIN b ON a.k=b.x AND a.i=1 ---- left-join (hash) - ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:6 y:7 + ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:7 y:8 ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) ├── key: (1) - ├── fd: (1)-->(2-7), (6)-->(7) + ├── fd: (1)-->(2-5,7,8), (7)-->(8) ├── scan a │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── scan b - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + │ └── fd: (7)-->(8) └── filters - ├── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + ├── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] └── i:2 = 1 [outer=(2), constraints=(/2: [/1 - /1]; tight), fd=()-->(2)] # Semi-join case. @@ -184,10 +184,10 @@ semi-join (hash) │ └── filters │ └── s:4 = 'foo' [outer=(4), constraints=(/4: [/'foo' - /'foo']; tight), fd=()-->(4)] ├── scan b - │ ├── columns: x:6!null - │ └── key: (6) + │ ├── columns: x:7!null + │ └── key: (7) └── filters - └── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # Do not push anti-join conditions into left input. norm expect-not=PushFilterIntoJoinLeft @@ -202,10 +202,10 @@ anti-join (hash) │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── scan b - │ ├── columns: x:6!null - │ └── key: (6) + │ ├── columns: x:7!null + │ └── key: (7) └── filters - ├── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + ├── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] └── s:4 = 'foo' [outer=(4), constraints=(/4: [/'foo' - /'foo']; tight), fd=()-->(4)] # -------------------------------------------------- @@ -215,74 +215,74 @@ norm expect=PushFilterIntoJoinRight SELECT * FROM b INNER JOIN a ON b.x=a.k AND a.s='foo' ---- inner-join (hash) - ├── columns: x:1!null y:2 k:3!null i:4 f:5!null s:6!null j:7 + ├── columns: x:1!null y:2 k:4!null i:5 f:6!null s:7!null j:8 ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (3) - ├── fd: ()-->(6), (1)-->(2), (3)-->(4,5,7), (1)==(3), (3)==(1) + ├── key: (4) + ├── fd: ()-->(7), (1)-->(2), (4)-->(5,6,8), (1)==(4), (4)==(1) ├── scan b │ ├── columns: x:1!null y:2 │ ├── key: (1) │ └── fd: (1)-->(2) ├── select - │ ├── columns: k:3!null i:4 f:5!null s:6!null j:7 - │ ├── key: (3) - │ ├── fd: ()-->(6), (3)-->(4,5,7) + │ ├── columns: k:4!null i:5 f:6!null s:7!null j:8 + │ ├── key: (4) + │ ├── fd: ()-->(7), (4)-->(5,6,8) │ ├── scan a - │ │ ├── columns: k:3!null i:4 f:5!null s:6 j:7 - │ │ ├── key: (3) - │ │ └── fd: (3)-->(4-7) + │ │ ├── columns: k:4!null i:5 f:6!null s:7 j:8 + │ │ ├── key: (4) + │ │ └── fd: (4)-->(5-8) │ └── filters - │ └── s:6 = 'foo' [outer=(6), constraints=(/6: [/'foo' - /'foo']; tight), fd=()-->(6)] + │ └── s:7 = 'foo' [outer=(7), constraints=(/7: [/'foo' - /'foo']; tight), fd=()-->(7)] └── filters - └── x:1 = k:3 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + └── x:1 = k:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] norm expect=PushFilterIntoJoinRight SELECT * FROM b LEFT JOIN a ON (a.i<0 OR a.i>10) AND b.y=1 AND a.s='foo' AND b.x=a.k ---- left-join (hash) - ├── columns: x:1!null y:2 k:3 i:4 f:5 s:6 j:7 + ├── columns: x:1!null y:2 k:4 i:5 f:6 s:7 j:8 ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) ├── key: (1) - ├── fd: (1)-->(2-7), (3)-->(4,5,7) + ├── fd: (1)-->(2,4-8), (4)-->(5,6,8) ├── scan b │ ├── columns: x:1!null y:2 │ ├── key: (1) │ └── fd: (1)-->(2) ├── select - │ ├── columns: k:3!null i:4!null f:5!null s:6!null j:7 - │ ├── key: (3) - │ ├── fd: ()-->(6), (3)-->(4,5,7) + │ ├── columns: k:4!null i:5!null f:6!null s:7!null j:8 + │ ├── key: (4) + │ ├── fd: ()-->(7), (4)-->(5,6,8) │ ├── scan a - │ │ ├── columns: k:3!null i:4 f:5!null s:6 j:7 - │ │ ├── key: (3) - │ │ └── fd: (3)-->(4-7) + │ │ ├── columns: k:4!null i:5 f:6!null s:7 j:8 + │ │ ├── key: (4) + │ │ └── fd: (4)-->(5-8) │ └── filters - │ ├── (i:4 < 0) OR (i:4 > 10) [outer=(4), constraints=(/4: (/NULL - /-1] [/11 - ]; tight)] - │ └── s:6 = 'foo' [outer=(6), constraints=(/6: [/'foo' - /'foo']; tight), fd=()-->(6)] + │ ├── (i:5 < 0) OR (i:5 > 10) [outer=(5), constraints=(/5: (/NULL - /-1] [/11 - ]; tight)] + │ └── s:7 = 'foo' [outer=(7), constraints=(/7: [/'foo' - /'foo']; tight), fd=()-->(7)] └── filters ├── y:2 = 1 [outer=(2), constraints=(/2: [/1 - /1]; tight), fd=()-->(2)] - └── x:1 = k:3 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + └── x:1 = k:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] # RIGHT JOIN should not push down conditions to right side of join. norm expect-not=PushFilterIntoJoinRight SELECT * FROM b RIGHT JOIN a ON b.x=a.k AND a.i=1 ---- left-join (hash) - ├── columns: x:1 y:2 k:3!null i:4 f:5!null s:6 j:7 + ├── columns: x:1 y:2 k:4!null i:5 f:6!null s:7 j:8 ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) - ├── key: (3) - ├── fd: (3)-->(1,2,4-7), (1)-->(2) + ├── key: (4) + ├── fd: (4)-->(1,2,5-8), (1)-->(2) ├── scan a - │ ├── columns: k:3!null i:4 f:5!null s:6 j:7 - │ ├── key: (3) - │ └── fd: (3)-->(4-7) + │ ├── columns: k:4!null i:5 f:6!null s:7 j:8 + │ ├── key: (4) + │ └── fd: (4)-->(5-8) ├── scan b │ ├── columns: x:1!null y:2 │ ├── key: (1) │ └── fd: (1)-->(2) └── filters - ├── x:1 = k:3 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] - └── i:4 = 1 [outer=(4), constraints=(/4: [/1 - /1]; tight), fd=()-->(4)] + ├── x:1 = k:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + └── i:5 = 1 [outer=(5), constraints=(/5: [/1 - /1]; tight), fd=()-->(5)] # Semi-join case. norm expect=PushFilterIntoJoinRight @@ -297,17 +297,17 @@ semi-join (hash) │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── select - │ ├── columns: x:6!null y:7!null - │ ├── key: (6) - │ ├── fd: (6)-->(7) + │ ├── columns: x:7!null y:8!null + │ ├── key: (7) + │ ├── fd: (7)-->(8) │ ├── scan b - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ └── filters - │ └── y:7 > 10 [outer=(7), constraints=(/7: [/11 - ]; tight)] + │ └── y:8 > 10 [outer=(8), constraints=(/8: [/11 - ]; tight)] └── filters - └── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # Anti-join case. norm expect=PushFilterIntoJoinRight @@ -322,17 +322,17 @@ anti-join (hash) │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── select - │ ├── columns: x:6!null y:7!null - │ ├── key: (6) - │ ├── fd: (6)-->(7) + │ ├── columns: x:7!null y:8!null + │ ├── key: (7) + │ ├── fd: (7)-->(8) │ ├── scan b - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ └── filters - │ └── y:7 > 10 [outer=(7), constraints=(/7: [/11 - ]; tight)] + │ └── y:8 > 10 [outer=(8), constraints=(/8: [/11 - ]; tight)] └── filters - └── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # ------------------------------------------------------------------------------- # PushFilterIntoJoinLeftAndRight + MapFilterIntoJoinLeft + MapFilterIntoJoinRight @@ -343,11 +343,11 @@ norm expect=(MapFilterIntoJoinLeft,MapFilterIntoJoinRight) SELECT * FROM a INNER JOIN b ON a.k=b.x AND a.k + b.y > 5 AND b.x * a.i = 3 ---- inner-join (hash) - ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:6!null y:7 + ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:7!null y:8 ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) ├── immutable - ├── key: (6) - ├── fd: (1)-->(2-5), (6)-->(7), (1)==(6), (6)==(1) + ├── key: (7) + ├── fd: (1)-->(2-5), (7)-->(8), (1)==(7), (7)==(1) ├── select │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 │ ├── immutable @@ -360,29 +360,29 @@ inner-join (hash) │ └── filters │ └── (k:1 * i:2) = 3 [outer=(1,2), immutable] ├── select - │ ├── columns: x:6!null y:7 + │ ├── columns: x:7!null y:8 │ ├── immutable - │ ├── key: (6) - │ ├── fd: (6)-->(7) + │ ├── key: (7) + │ ├── fd: (7)-->(8) │ ├── scan b - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ └── filters - │ └── (x:6 + y:7) > 5 [outer=(6,7), immutable] + │ └── (x:7 + y:8) > 5 [outer=(7,8), immutable] └── filters - └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # Multiple equivalent columns. norm expect=MapFilterIntoJoinLeft SELECT * FROM a INNER JOIN b ON a.k=b.x AND a.i=b.x AND a.i=b.y AND a.f + b.y::FLOAT > 5 AND a.s || b.x::STRING = 'foo1' ---- inner-join (hash) - ├── columns: k:1!null i:2!null f:3!null s:4 j:5 x:6!null y:7!null + ├── columns: k:1!null i:2!null f:3!null s:4 j:5 x:7!null y:8!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) ├── immutable - ├── key: (6) - ├── fd: (1)-->(3-5), (1)==(2,6,7), (2)==(1,6,7), (6)==(1,2,7), (7)==(1,2,6) + ├── key: (7) + ├── fd: (1)-->(3-5), (1)==(2,7,8), (2)==(1,7,8), (7)==(1,2,8), (8)==(1,2,7) ├── select │ ├── columns: k:1!null i:2!null f:3!null s:4 j:5 │ ├── immutable @@ -397,17 +397,17 @@ inner-join (hash) │ ├── (s:4 || k:1::STRING) = 'foo1' [outer=(1,4), immutable] │ └── k:1 = i:2 [outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ]), fd=(1)==(2), (2)==(1)] ├── select - │ ├── columns: x:6!null y:7!null - │ ├── key: (6) - │ ├── fd: (6)==(7), (7)==(6) + │ ├── columns: x:7!null y:8!null + │ ├── key: (7) + │ ├── fd: (7)==(8), (8)==(7) │ ├── scan b - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ └── filters - │ └── x:6 = y:7 [outer=(6,7), constraints=(/6: (/NULL - ]; /7: (/NULL - ]), fd=(6)==(7), (7)==(6)] + │ └── x:7 = y:8 [outer=(7,8), constraints=(/7: (/NULL - ]; /8: (/NULL - ]), fd=(7)==(8), (8)==(7)] └── filters - └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # Can push to both sides with semi-join. norm expect=(MapFilterIntoJoinLeft,MapFilterIntoJoinRight) @@ -432,18 +432,18 @@ semi-join (hash) │ └── filters │ └── (k:1 * i:2) = 3 [outer=(1,2), immutable] ├── select - │ ├── columns: x:6!null y:7 + │ ├── columns: x:7!null y:8 │ ├── immutable - │ ├── key: (6) - │ ├── fd: (6)-->(7) + │ ├── key: (7) + │ ├── fd: (7)-->(8) │ ├── scan b - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ └── filters - │ └── (x:6 + y:7) > 5 [outer=(6,7), immutable] + │ └── (x:7 + y:8) > 5 [outer=(7,8), immutable] └── filters - └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] norm expect=PushFilterIntoJoinLeftAndRight SELECT * FROM a WHERE EXISTS( @@ -467,115 +467,115 @@ semi-join (hash) │ └── filters │ └── (k:1 > 5) AND (k:1 IN (3, 7, 10)) [outer=(1), constraints=(/1: [/7 - /7] [/10 - /10]; tight)] ├── select - │ ├── columns: x:6!null + │ ├── columns: x:7!null │ ├── cardinality: [0 - 2] - │ ├── key: (6) + │ ├── key: (7) │ ├── scan b - │ │ ├── columns: x:6!null - │ │ └── key: (6) + │ │ ├── columns: x:7!null + │ │ └── key: (7) │ └── filters - │ └── (x:6 IN (3, 7, 10)) AND (x:6 > 5) [outer=(6), constraints=(/6: [/7 - /7] [/10 - /10]; tight)] + │ └── (x:7 IN (3, 7, 10)) AND (x:7 > 5) [outer=(7), constraints=(/7: [/7 - /7] [/10 - /10]; tight)] └── filters - └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # Can only push to right side with left join. norm expect=MapFilterIntoJoinRight expect-not=PushFilterIntoJoinLeftAndRight SELECT * FROM a LEFT JOIN b ON a.k=b.x AND a.k + b.y > 5 AND b.x * a.i = 3 ---- left-join (hash) - ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:6 y:7 + ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:7 y:8 ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) ├── immutable ├── key: (1) - ├── fd: (1)-->(2-7), (6)-->(7) + ├── fd: (1)-->(2-5,7,8), (7)-->(8) ├── scan a │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── select - │ ├── columns: x:6!null y:7 + │ ├── columns: x:7!null y:8 │ ├── immutable - │ ├── key: (6) - │ ├── fd: (6)-->(7) + │ ├── key: (7) + │ ├── fd: (7)-->(8) │ ├── scan b - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ └── filters - │ └── (x:6 + y:7) > 5 [outer=(6,7), immutable] + │ └── (x:7 + y:8) > 5 [outer=(7,8), immutable] └── filters - ├── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── (x:6 * i:2) = 3 [outer=(2,6), immutable] + ├── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + └── (x:7 * i:2) = 3 [outer=(2,7), immutable] norm expect=MapFilterIntoJoinRight expect-not=PushFilterIntoJoinLeftAndRight SELECT * FROM a LEFT JOIN b ON a.k=b.x AND a.k > 5 AND b.x IN (3, 7, 10) ---- left-join (hash) - ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:6 y:7 + ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:7 y:8 ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) ├── key: (1) - ├── fd: (1)-->(2-7), (6)-->(7) + ├── fd: (1)-->(2-5,7,8), (7)-->(8) ├── scan a │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── select - │ ├── columns: x:6!null y:7 + │ ├── columns: x:7!null y:8 │ ├── cardinality: [0 - 2] - │ ├── key: (6) - │ ├── fd: (6)-->(7) + │ ├── key: (7) + │ ├── fd: (7)-->(8) │ ├── scan b - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ └── filters - │ └── (x:6 IN (3, 7, 10)) AND (x:6 > 5) [outer=(6), constraints=(/6: [/7 - /7] [/10 - /10]; tight)] + │ └── (x:7 IN (3, 7, 10)) AND (x:7 > 5) [outer=(7), constraints=(/7: [/7 - /7] [/10 - /10]; tight)] └── filters - └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # Cannot push with full join. norm expect-not=(PushFilterIntoJoinLeftAndRight,MapFilterIntoJoinLeft,MapFilterIntoJoinRight) SELECT * FROM a FULL JOIN b ON a.k=b.x AND a.k + b.y > 5 AND b.x * a.i = 3 ---- full-join (hash) - ├── columns: k:1 i:2 f:3 s:4 j:5 x:6 y:7 + ├── columns: k:1 i:2 f:3 s:4 j:5 x:7 y:8 ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) ├── immutable - ├── key: (1,6) - ├── fd: (1)-->(2-5), (6)-->(7) + ├── key: (1,7) + ├── fd: (1)-->(2-5), (7)-->(8) ├── scan a │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── scan b - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + │ └── fd: (7)-->(8) └── filters - ├── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - ├── (k:1 + y:7) > 5 [outer=(1,7), immutable] - └── (x:6 * i:2) = 3 [outer=(2,6), immutable] + ├── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + ├── (k:1 + y:8) > 5 [outer=(1,8), immutable] + └── (x:7 * i:2) = 3 [outer=(2,7), immutable] norm expect-not=(PushFilterIntoJoinLeftAndRight,MapFilterIntoJoinLeft,MapFilterIntoJoinRight) SELECT * FROM a FULL JOIN b ON a.k=b.x AND a.k > 5 AND b.x IN (3, 7, 10) ---- full-join (hash) - ├── columns: k:1 i:2 f:3 s:4 j:5 x:6 y:7 + ├── columns: k:1 i:2 f:3 s:4 j:5 x:7 y:8 ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) - ├── key: (1,6) - ├── fd: (1)-->(2-5), (6)-->(7) + ├── key: (1,7) + ├── fd: (1)-->(2-5), (7)-->(8) ├── scan a │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── scan b - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + │ └── fd: (7)-->(8) └── filters - ├── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + ├── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] ├── k:1 > 5 [outer=(1), constraints=(/1: [/6 - ]; tight)] - └── x:6 IN (3, 7, 10) [outer=(6), constraints=(/6: [/3 - /3] [/7 - /7] [/10 - /10]; tight)] + └── x:7 IN (3, 7, 10) [outer=(7), constraints=(/7: [/3 - /3] [/7 - /7] [/10 - /10]; tight)] # Can only push to right side with anti-join. norm expect=MapFilterIntoJoinRight expect-not=PushFilterIntoJoinLeftAndRight @@ -593,19 +593,19 @@ anti-join (hash) │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── select - │ ├── columns: x:6!null y:7 + │ ├── columns: x:7!null y:8 │ ├── immutable - │ ├── key: (6) - │ ├── fd: (6)-->(7) + │ ├── key: (7) + │ ├── fd: (7)-->(8) │ ├── scan b - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ └── filters - │ └── (x:6 + y:7) > 5 [outer=(6,7), immutable] + │ └── (x:7 + y:8) > 5 [outer=(7,8), immutable] └── filters - ├── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── (x:6 * i:2) = 3 [outer=(2,6), immutable] + ├── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + └── (x:7 * i:2) = 3 [outer=(2,7), immutable] norm expect=MapFilterIntoJoinRight expect-not=PushFilterIntoJoinLeftAndRight SELECT * FROM a WHERE NOT EXISTS( @@ -621,27 +621,27 @@ anti-join (hash) │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── select - │ ├── columns: x:6!null + │ ├── columns: x:7!null │ ├── cardinality: [0 - 2] - │ ├── key: (6) + │ ├── key: (7) │ ├── scan b - │ │ ├── columns: x:6!null - │ │ └── key: (6) + │ │ ├── columns: x:7!null + │ │ └── key: (7) │ └── filters - │ └── (x:6 IN (3, 7, 10)) AND (x:6 > 5) [outer=(6), constraints=(/6: [/7 - /7] [/10 - /10]; tight)] + │ └── (x:7 IN (3, 7, 10)) AND (x:7 > 5) [outer=(7), constraints=(/7: [/7 - /7] [/10 - /10]; tight)] └── filters - └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # Works with a non-correlated subquery. norm expect=MapFilterIntoJoinLeft SELECT * FROM a JOIN b ON a.k = b.x AND b.x * a.i = (SELECT min(b.x) FROM b) ---- inner-join (hash) - ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:6!null y:7 + ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:7!null y:8 ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) ├── immutable - ├── key: (6) - ├── fd: (1)-->(2-5), (6)-->(7), (1)==(6), (6)==(1) + ├── key: (7) + ├── fd: (1)-->(2-5), (7)-->(8), (1)==(7), (7)==(1) ├── select │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 │ ├── immutable @@ -656,75 +656,75 @@ inner-join (hash) │ ├── k:1 * i:2 │ └── subquery │ └── scalar-group-by - │ ├── columns: min:10 + │ ├── columns: min:13 │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(10) + │ ├── fd: ()-->(13) │ ├── scan b - │ │ ├── columns: x:8!null - │ │ └── key: (8) + │ │ ├── columns: x:10!null + │ │ └── key: (10) │ └── aggregations - │ └── min [as=min:10, outer=(8)] - │ └── x:8 + │ └── min [as=min:13, outer=(10)] + │ └── x:10 ├── scan b - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + │ └── fd: (7)-->(8) └── filters - └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # Optimization does not apply with correlated suqueries. norm expect-not=(PushFilterIntoJoinLeftAndRight,MapFilterIntoJoinLeft,MapFilterIntoJoinRight) SELECT * FROM a JOIN b ON a.k = b.x AND b.x * a.i = (SELECT a.k * b.y FROM b) ---- project - ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:6!null y:7 + ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:7!null y:8 ├── immutable - ├── key: (6) - ├── fd: (1)-->(2-5), (1,6)-->(7), (1)==(6), (6)==(1) + ├── key: (7) + ├── fd: (1)-->(2-5), (1,7)-->(8), (1)==(7), (7)==(1) └── inner-join-apply - ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:6!null y:7 "?column?":10 + ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:7!null y:8 "?column?":13 ├── immutable - ├── key: (6) - ├── fd: (1)-->(2-5), (1,6)-->(7,10), (1)==(6), (6)==(1) + ├── key: (7) + ├── fd: (1)-->(2-5), (1,7)-->(8,13), (1)==(7), (7)==(1) ├── scan a │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── ensure-distinct-on - │ ├── columns: x:6!null y:7 "?column?":10 - │ ├── grouping columns: x:6!null + │ ├── columns: x:7!null y:8 "?column?":13 + │ ├── grouping columns: x:7!null │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── outer: (1) │ ├── immutable - │ ├── key: (6) - │ ├── fd: (6)-->(7,10) + │ ├── key: (7) + │ ├── fd: (7)-->(8,13) │ ├── left-join (cross) - │ │ ├── columns: x:6!null y:7 "?column?":10 + │ │ ├── columns: x:7!null y:8 "?column?":13 │ │ ├── outer: (1) │ │ ├── immutable - │ │ ├── fd: (6)-->(7) + │ │ ├── fd: (7)-->(8) │ │ ├── scan b - │ │ │ ├── columns: x:6!null y:7 - │ │ │ ├── key: (6) - │ │ │ └── fd: (6)-->(7) + │ │ │ ├── columns: x:7!null y:8 + │ │ │ ├── key: (7) + │ │ │ └── fd: (7)-->(8) │ │ ├── project - │ │ │ ├── columns: "?column?":10 + │ │ │ ├── columns: "?column?":13 │ │ │ ├── outer: (1) │ │ │ ├── immutable │ │ │ ├── scan b - │ │ │ │ └── columns: y:9 + │ │ │ │ └── columns: y:11 │ │ │ └── projections - │ │ │ └── k:1 * y:9 [as="?column?":10, outer=(1,9), immutable] + │ │ │ └── k:1 * y:11 [as="?column?":13, outer=(1,11), immutable] │ │ └── filters (true) │ └── aggregations - │ ├── const-agg [as=y:7, outer=(7)] - │ │ └── y:7 - │ └── const-agg [as="?column?":10, outer=(10)] - │ └── "?column?":10 + │ ├── const-agg [as=y:8, outer=(8)] + │ │ └── y:8 + │ └── const-agg [as="?column?":13, outer=(13)] + │ └── "?column?":13 └── filters - ├── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── "?column?":10 = (x:6 * i:2) [outer=(2,6,10), immutable, constraints=(/10: (/NULL - ])] + ├── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + └── "?column?":13 = (x:7 * i:2) [outer=(2,7,13), immutable, constraints=(/13: (/NULL - ])] # Ensure that we do not map filters for types with composite key encoding. norm expect-not=(PushFilterIntoJoinLeftAndRight,MapFilterIntoJoinLeft,MapFilterIntoJoinRight) @@ -760,10 +760,10 @@ norm expect-not=(PushFilterIntoJoinLeftAndRight,MapFilterIntoJoinLeft,MapFilterI SELECT * FROM a INNER JOIN b ON b.y=b.x AND a.k=a.i AND a.k + b.y > 5 AND b.x * a.i = 3 ---- inner-join (cross) - ├── columns: k:1!null i:2!null f:3!null s:4 j:5 x:6!null y:7!null + ├── columns: k:1!null i:2!null f:3!null s:4 j:5 x:7!null y:8!null ├── immutable - ├── key: (1,6) - ├── fd: (1)-->(3-5), (1)==(2), (2)==(1), (6)==(7), (7)==(6) + ├── key: (1,7) + ├── fd: (1)-->(3-5), (1)==(2), (2)==(1), (7)==(8), (8)==(7) ├── select │ ├── columns: k:1!null i:2!null f:3!null s:4 j:5 │ ├── key: (1) @@ -775,18 +775,18 @@ inner-join (cross) │ └── filters │ └── k:1 = i:2 [outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ]), fd=(1)==(2), (2)==(1)] ├── select - │ ├── columns: x:6!null y:7!null - │ ├── key: (6) - │ ├── fd: (6)==(7), (7)==(6) + │ ├── columns: x:7!null y:8!null + │ ├── key: (7) + │ ├── fd: (7)==(8), (8)==(7) │ ├── scan b - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ └── filters - │ └── y:7 = x:6 [outer=(6,7), constraints=(/6: (/NULL - ]; /7: (/NULL - ]), fd=(6)==(7), (7)==(6)] + │ └── y:8 = x:7 [outer=(7,8), constraints=(/7: (/NULL - ]; /8: (/NULL - ]), fd=(7)==(8), (8)==(7)] └── filters - ├── (k:1 + y:7) > 5 [outer=(1,7), immutable] - └── (x:6 * i:2) = 3 [outer=(2,6), immutable] + ├── (k:1 + y:8) > 5 [outer=(1,8), immutable] + └── (x:7 * i:2) = 3 [outer=(2,7), immutable] # Ensure that MapFilterIntoJoinRight doesn't cause cycle with decorrelation. norm expect=MapFilterIntoJoinRight @@ -799,54 +799,54 @@ SELECT FROM c ---- project - ├── columns: x:13 + ├── columns: x:17 ├── immutable ├── ensure-distinct-on - │ ├── columns: c.x:1!null b.x:4 + │ ├── columns: c.x:1!null b.x:5 │ ├── grouping columns: c.x:1!null │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(4) + │ ├── fd: (1)-->(5) │ ├── left-join-apply - │ │ ├── columns: c.x:1!null b.x:4 k:8 + │ │ ├── columns: c.x:1!null b.x:5 k:11 │ │ ├── immutable - │ │ ├── fd: (4)==(8), (8)==(4) + │ │ ├── fd: (5)==(11), (11)==(5) │ │ ├── scan c │ │ │ ├── columns: c.x:1!null │ │ │ └── key: (1) │ │ ├── inner-join (hash) - │ │ │ ├── columns: b.x:4!null k:8!null + │ │ │ ├── columns: b.x:5!null k:11!null │ │ │ ├── outer: (1) │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) │ │ │ ├── immutable - │ │ │ ├── fd: (4)==(8), (8)==(4) + │ │ │ ├── fd: (5)==(11), (11)==(5) │ │ │ ├── full-join (cross) - │ │ │ │ ├── columns: b.x:4 + │ │ │ │ ├── columns: b.x:5 │ │ │ │ ├── outer: (1) │ │ │ │ ├── scan b - │ │ │ │ │ ├── columns: b.x:4!null - │ │ │ │ │ └── key: (4) + │ │ │ │ │ ├── columns: b.x:5!null + │ │ │ │ │ └── key: (5) │ │ │ │ ├── scan b2 │ │ │ │ └── filters │ │ │ │ └── c.x:1 = 5 [outer=(1), constraints=(/1: [/5 - /5]; tight), fd=()-->(1)] │ │ │ ├── select - │ │ │ │ ├── columns: k:8!null + │ │ │ │ ├── columns: k:11!null │ │ │ │ ├── immutable - │ │ │ │ ├── key: (8) + │ │ │ │ ├── key: (11) │ │ │ │ ├── scan a - │ │ │ │ │ ├── columns: k:8!null - │ │ │ │ │ └── key: (8) + │ │ │ │ │ ├── columns: k:11!null + │ │ │ │ │ └── key: (11) │ │ │ │ └── filters - │ │ │ │ └── (k:8 + k:8) < 5 [outer=(8), immutable] + │ │ │ │ └── (k:11 + k:11) < 5 [outer=(11), immutable] │ │ │ └── filters - │ │ │ └── k:8 = b.x:4 [outer=(4,8), constraints=(/4: (/NULL - ]; /8: (/NULL - ]), fd=(4)==(8), (8)==(4)] + │ │ │ └── k:11 = b.x:5 [outer=(5,11), constraints=(/5: (/NULL - ]; /11: (/NULL - ]), fd=(5)==(11), (11)==(5)] │ │ └── filters (true) │ └── aggregations - │ └── const-agg [as=b.x:4, outer=(4)] - │ └── b.x:4 + │ └── const-agg [as=b.x:5, outer=(5)] + │ └── b.x:5 └── projections - └── b.x:4 [as=x:13, outer=(4)] + └── b.x:5 [as=x:17, outer=(5)] # Ensure that MapFilterIntoJoinLeft doesn't cause cycle with decorrelation. norm expect=MapFilterIntoJoinLeft @@ -858,54 +858,54 @@ SELECT FROM c ---- project - ├── columns: x:13 + ├── columns: x:17 ├── immutable ├── ensure-distinct-on - │ ├── columns: c.x:1!null b.x:9 + │ ├── columns: c.x:1!null b.x:11 │ ├── grouping columns: c.x:1!null │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(9) + │ ├── fd: (1)-->(11) │ ├── left-join-apply - │ │ ├── columns: c.x:1!null k:4 b.x:9 + │ │ ├── columns: c.x:1!null k:5 b.x:11 │ │ ├── immutable - │ │ ├── fd: (4)==(9), (9)==(4) + │ │ ├── fd: (5)==(11), (11)==(5) │ │ ├── scan c │ │ │ ├── columns: c.x:1!null │ │ │ └── key: (1) │ │ ├── inner-join (hash) - │ │ │ ├── columns: k:4!null b.x:9!null + │ │ │ ├── columns: k:5!null b.x:11!null │ │ │ ├── outer: (1) │ │ │ ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) │ │ │ ├── immutable - │ │ │ ├── fd: (4)==(9), (9)==(4) + │ │ │ ├── fd: (5)==(11), (11)==(5) │ │ │ ├── select - │ │ │ │ ├── columns: k:4!null + │ │ │ │ ├── columns: k:5!null │ │ │ │ ├── immutable - │ │ │ │ ├── key: (4) + │ │ │ │ ├── key: (5) │ │ │ │ ├── scan a - │ │ │ │ │ ├── columns: k:4!null - │ │ │ │ │ └── key: (4) + │ │ │ │ │ ├── columns: k:5!null + │ │ │ │ │ └── key: (5) │ │ │ │ └── filters - │ │ │ │ └── (k:4 + k:4) < 5 [outer=(4), immutable] + │ │ │ │ └── (k:5 + k:5) < 5 [outer=(5), immutable] │ │ │ ├── full-join (cross) - │ │ │ │ ├── columns: b.x:9 + │ │ │ │ ├── columns: b.x:11 │ │ │ │ ├── outer: (1) │ │ │ │ ├── scan b - │ │ │ │ │ ├── columns: b.x:9!null - │ │ │ │ │ └── key: (9) + │ │ │ │ │ ├── columns: b.x:11!null + │ │ │ │ │ └── key: (11) │ │ │ │ ├── scan b2 │ │ │ │ └── filters │ │ │ │ └── c.x:1 = 5 [outer=(1), constraints=(/1: [/5 - /5]; tight), fd=()-->(1)] │ │ │ └── filters - │ │ │ └── k:4 = b.x:9 [outer=(4,9), constraints=(/4: (/NULL - ]; /9: (/NULL - ]), fd=(4)==(9), (9)==(4)] + │ │ │ └── k:5 = b.x:11 [outer=(5,11), constraints=(/5: (/NULL - ]; /11: (/NULL - ]), fd=(5)==(11), (11)==(5)] │ │ └── filters (true) │ └── aggregations - │ └── const-agg [as=b.x:9, outer=(9)] - │ └── b.x:9 + │ └── const-agg [as=b.x:11, outer=(11)] + │ └── b.x:11 └── projections - └── b.x:9 [as=x:13, outer=(9)] + └── b.x:11 [as=x:17, outer=(11)] exec-ddl CREATE TABLE t1 (a DATE) @@ -921,20 +921,20 @@ norm SELECT * FROM t1, t2 WHERE a = b AND age(b, TIMESTAMPTZ '2017-01-01') > INTERVAL '1 day' ---- inner-join (cross) - ├── columns: a:1!null b:3!null + ├── columns: a:1!null b:4!null ├── stable - ├── fd: (1)==(3), (3)==(1) + ├── fd: (1)==(4), (4)==(1) ├── scan t1 │ └── columns: a:1 ├── select - │ ├── columns: b:3 + │ ├── columns: b:4 │ ├── immutable │ ├── scan t2 - │ │ └── columns: b:3 + │ │ └── columns: b:4 │ └── filters - │ └── age(b:3, '2017-01-01 00:00:00+00:00') > '1 day' [outer=(3), immutable] + │ └── age(b:4, '2017-01-01 00:00:00+00:00') > '1 day' [outer=(4), immutable] └── filters - └── a:1 = b:3 [outer=(1,3), stable, constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + └── a:1 = b:4 [outer=(1,4), stable, constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] # Regression for issue 28818. Try to trigger undetectable cycle between the # PushFilterIntoJoinLeftAndRight and TryDecorrelateSelect rules. @@ -956,8 +956,8 @@ WHERE EXISTS ( ) ---- project - ├── columns: "?column?":22!null - ├── fd: ()-->(22) + ├── columns: "?column?":27!null + ├── fd: ()-->(27) ├── semi-join (cross) │ ├── columns: s:4!null │ ├── select @@ -974,13 +974,13 @@ project │ │ │ │ └── eq [subquery] │ │ │ │ ├── subquery │ │ │ │ │ └── max1-row - │ │ │ │ │ ├── columns: s:19 + │ │ │ │ │ ├── columns: s:23 │ │ │ │ │ ├── error: "more than one row returned by a subquery used as an expression" │ │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ │ ├── key: () - │ │ │ │ │ ├── fd: ()-->(19) + │ │ │ │ │ ├── fd: ()-->(23) │ │ │ │ │ └── scan a - │ │ │ │ │ └── columns: s:19 + │ │ │ │ │ └── columns: s:23 │ │ │ │ └── 'foo' │ │ │ ├── select │ │ │ │ ├── scan uv @@ -988,13 +988,13 @@ project │ │ │ │ └── eq [subquery] │ │ │ │ ├── subquery │ │ │ │ │ └── max1-row - │ │ │ │ │ ├── columns: s:19 + │ │ │ │ │ ├── columns: s:23 │ │ │ │ │ ├── error: "more than one row returned by a subquery used as an expression" │ │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ │ ├── key: () - │ │ │ │ │ ├── fd: ()-->(19) + │ │ │ │ │ ├── fd: ()-->(23) │ │ │ │ │ └── scan a - │ │ │ │ │ └── columns: s:19 + │ │ │ │ │ └── columns: s:23 │ │ │ │ └── 'foo' │ │ │ └── filters (true) │ │ ├── select @@ -1003,18 +1003,18 @@ project │ │ │ └── eq [subquery] │ │ │ ├── subquery │ │ │ │ └── max1-row - │ │ │ │ ├── columns: s:19 + │ │ │ │ ├── columns: s:23 │ │ │ │ ├── error: "more than one row returned by a subquery used as an expression" │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ ├── key: () - │ │ │ │ ├── fd: ()-->(19) + │ │ │ │ ├── fd: ()-->(23) │ │ │ │ └── scan a - │ │ │ │ └── columns: s:19 + │ │ │ │ └── columns: s:23 │ │ │ └── 'foo' │ │ └── filters (true) │ └── filters (true) └── projections - └── 1 [as="?column?":22] + └── 1 [as="?column?":27] # Regression for issue 36137. Try to trigger undetectable cycle between the # PushFilterIntoJoinLeftAndRight and TryDecorrelateSelect rules. @@ -1023,38 +1023,38 @@ SELECT * FROM a JOIN b ON a.k = b.x WHERE (a.k = b.x) OR (a.k IN (SELECT 5 FROM b WHERE x = y)); ---- inner-join (hash) - ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:6!null y:7 + ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:7!null y:8 ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (6) - ├── fd: (1)-->(2-5), (6)-->(7), (1)==(6), (6)==(1) + ├── key: (7) + ├── fd: (1)-->(2-5), (7)-->(8), (1)==(7), (7)==(1) ├── scan a │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── scan b - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + │ └── fd: (7)-->(8) └── filters - ├── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── or [outer=(1,6), correlated-subquery] - ├── k:1 = x:6 + ├── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + └── or [outer=(1,7), correlated-subquery] + ├── k:1 = x:7 └── any: eq ├── project - │ ├── columns: "?column?":10!null - │ ├── fd: ()-->(10) + │ ├── columns: "?column?":13!null + │ ├── fd: ()-->(13) │ ├── select - │ │ ├── columns: x:8!null y:9!null - │ │ ├── key: (8) - │ │ ├── fd: (8)==(9), (9)==(8) + │ │ ├── columns: x:10!null y:11!null + │ │ ├── key: (10) + │ │ ├── fd: (10)==(11), (11)==(10) │ │ ├── scan b - │ │ │ ├── columns: x:8!null y:9 - │ │ │ ├── key: (8) - │ │ │ └── fd: (8)-->(9) + │ │ │ ├── columns: x:10!null y:11 + │ │ │ ├── key: (10) + │ │ │ └── fd: (10)-->(11) │ │ └── filters - │ │ └── x:8 = y:9 [outer=(8,9), constraints=(/8: (/NULL - ]; /9: (/NULL - ]), fd=(8)==(9), (9)==(8)] + │ │ └── x:10 = y:11 [outer=(10,11), constraints=(/10: (/NULL - ]; /11: (/NULL - ]), fd=(10)==(11), (11)==(10)] │ └── projections - │ └── 5 [as="?column?":10] + │ └── 5 [as="?column?":13] └── k:1 # Regression test for #43039. Use transitive equalities for filter inference. @@ -1071,29 +1071,29 @@ WHERE a.k = 3; ---- inner-join (hash) - ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:6!null y:7 x:8!null y:9!null z:10!null x:11!null y:12!null z:13!null x:14!null y:15 + ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:7!null y:8 x:10!null y:11!null z:12!null x:14!null y:15!null z:16!null x:18!null y:19 ├── cardinality: [0 - 1] ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) ├── key: () - ├── fd: ()-->(1-15) + ├── fd: ()-->(1-5,7,8,10-12,14-16,18,19) ├── inner-join (hash) - │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 b.x:6!null b.y:7 c.x:8!null c.y:9!null c.z:10!null d.x:11!null d.y:12!null d.z:13!null + │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 b.x:7!null b.y:8 c.x:10!null c.y:11!null c.z:12!null d.x:14!null d.y:15!null d.z:16!null │ ├── cardinality: [0 - 1] │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) │ ├── key: () - │ ├── fd: ()-->(1-13) + │ ├── fd: ()-->(1-5,7,8,10-12,14-16) │ ├── inner-join (hash) - │ │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 b.x:6!null b.y:7 c.x:8!null c.y:9!null c.z:10!null + │ │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 b.x:7!null b.y:8 c.x:10!null c.y:11!null c.z:12!null │ │ ├── cardinality: [0 - 1] │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) │ │ ├── key: () - │ │ ├── fd: ()-->(1-10) + │ │ ├── fd: ()-->(1-5,7,8,10-12) │ │ ├── inner-join (hash) - │ │ │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 b.x:6!null b.y:7 + │ │ │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 b.x:7!null b.y:8 │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(1-7) + │ │ │ ├── fd: ()-->(1-5,7,8) │ │ │ ├── select │ │ │ │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 │ │ │ │ ├── cardinality: [0 - 1] @@ -1106,57 +1106,57 @@ inner-join (hash) │ │ │ │ └── filters │ │ │ │ └── k:1 = 3 [outer=(1), constraints=(/1: [/3 - /3]; tight), fd=()-->(1)] │ │ │ ├── select - │ │ │ │ ├── columns: b.x:6!null b.y:7 + │ │ │ │ ├── columns: b.x:7!null b.y:8 │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ ├── key: () - │ │ │ │ ├── fd: ()-->(6,7) + │ │ │ │ ├── fd: ()-->(7,8) │ │ │ │ ├── scan b - │ │ │ │ │ ├── columns: b.x:6!null b.y:7 - │ │ │ │ │ ├── key: (6) - │ │ │ │ │ └── fd: (6)-->(7) + │ │ │ │ │ ├── columns: b.x:7!null b.y:8 + │ │ │ │ │ ├── key: (7) + │ │ │ │ │ └── fd: (7)-->(8) │ │ │ │ └── filters - │ │ │ │ └── b.x:6 = 3 [outer=(6), constraints=(/6: [/3 - /3]; tight), fd=()-->(6)] + │ │ │ │ └── b.x:7 = 3 [outer=(7), constraints=(/7: [/3 - /3]; tight), fd=()-->(7)] │ │ │ └── filters - │ │ │ └── k:1 = b.x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ │ │ └── k:1 = b.x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] │ │ ├── select - │ │ │ ├── columns: c.x:8!null c.y:9!null c.z:10!null + │ │ │ ├── columns: c.x:10!null c.y:11!null c.z:12!null │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(8-10) + │ │ │ ├── fd: ()-->(10-12) │ │ │ ├── scan c - │ │ │ │ ├── columns: c.x:8!null c.y:9!null c.z:10!null - │ │ │ │ ├── key: (8) - │ │ │ │ └── fd: (8)-->(9,10) + │ │ │ │ ├── columns: c.x:10!null c.y:11!null c.z:12!null + │ │ │ │ ├── key: (10) + │ │ │ │ └── fd: (10)-->(11,12) │ │ │ └── filters - │ │ │ └── c.x:8 = 3 [outer=(8), constraints=(/8: [/3 - /3]; tight), fd=()-->(8)] + │ │ │ └── c.x:10 = 3 [outer=(10), constraints=(/10: [/3 - /3]; tight), fd=()-->(10)] │ │ └── filters - │ │ └── b.x:6 = c.x:8 [outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ]), fd=(6)==(8), (8)==(6)] + │ │ └── b.x:7 = c.x:10 [outer=(7,10), constraints=(/7: (/NULL - ]; /10: (/NULL - ]), fd=(7)==(10), (10)==(7)] │ ├── select - │ │ ├── columns: d.x:11!null d.y:12!null d.z:13!null + │ │ ├── columns: d.x:14!null d.y:15!null d.z:16!null │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(11-13) + │ │ ├── fd: ()-->(14-16) │ │ ├── scan d - │ │ │ ├── columns: d.x:11!null d.y:12!null d.z:13!null - │ │ │ ├── key: (11) - │ │ │ └── fd: (11)-->(12,13) + │ │ │ ├── columns: d.x:14!null d.y:15!null d.z:16!null + │ │ │ ├── key: (14) + │ │ │ └── fd: (14)-->(15,16) │ │ └── filters - │ │ └── d.x:11 = 3 [outer=(11), constraints=(/11: [/3 - /3]; tight), fd=()-->(11)] + │ │ └── d.x:14 = 3 [outer=(14), constraints=(/14: [/3 - /3]; tight), fd=()-->(14)] │ └── filters - │ └── c.x:8 = d.x:11 [outer=(8,11), constraints=(/8: (/NULL - ]; /11: (/NULL - ]), fd=(8)==(11), (11)==(8)] + │ └── c.x:10 = d.x:14 [outer=(10,14), constraints=(/10: (/NULL - ]; /14: (/NULL - ]), fd=(10)==(14), (14)==(10)] ├── select - │ ├── columns: xy.x:14!null xy.y:15 + │ ├── columns: xy.x:18!null xy.y:19 │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(14,15) + │ ├── fd: ()-->(18,19) │ ├── scan xy - │ │ ├── columns: xy.x:14!null xy.y:15 - │ │ ├── key: (14) - │ │ └── fd: (14)-->(15) + │ │ ├── columns: xy.x:18!null xy.y:19 + │ │ ├── key: (18) + │ │ └── fd: (18)-->(19) │ └── filters - │ └── xy.x:14 = 3 [outer=(14), constraints=(/14: [/3 - /3]; tight), fd=()-->(14)] + │ └── xy.x:18 = 3 [outer=(18), constraints=(/18: [/3 - /3]; tight), fd=()-->(18)] └── filters - └── d.x:11 = xy.x:14 [outer=(11,14), constraints=(/11: (/NULL - ]; /14: (/NULL - ]), fd=(11)==(14), (14)==(11)] + └── d.x:14 = xy.x:18 [outer=(14,18), constraints=(/14: (/NULL - ]; /18: (/NULL - ]), fd=(14)==(18), (18)==(14)] # Regression test for #46151. Do not push down a filter with a correlated # subquery. @@ -1173,73 +1173,73 @@ SELECT (SELECT i_name FROM item LIMIT 1) OR (SELECT ol_i_id FROM order_line LIMIT 1) IS NOT NULL; ---- project - ├── columns: i_name:47 - ├── fd: ()-->(47) + ├── columns: i_name:52 + ├── fd: ()-->(52) ├── inner-join (hash) - │ ├── columns: h_data:9!null ol_o_id:10!null ol_d_id:11!null ol_w_id:12!null ol_number:13!null ol_dist_info:19!null true_agg:40 - │ ├── fd: (10-13)-->(19,40), (9)==(19), (19)==(9) + │ ├── columns: h_data:9!null ol_o_id:11!null ol_d_id:12!null ol_w_id:13!null ol_number:14!null ol_dist_info:20!null true_agg:44 + │ ├── fd: (11-14)-->(20,44), (9)==(20), (20)==(9) │ ├── scan history │ │ └── columns: h_data:9 │ ├── select - │ │ ├── columns: ol_o_id:10!null ol_d_id:11!null ol_w_id:12!null ol_number:13!null ol_dist_info:19 true_agg:40 - │ │ ├── key: (10-13) - │ │ ├── fd: (10-13)-->(19,40) + │ │ ├── columns: ol_o_id:11!null ol_d_id:12!null ol_w_id:13!null ol_number:14!null ol_dist_info:20 true_agg:44 + │ │ ├── key: (11-14) + │ │ ├── fd: (11-14)-->(20,44) │ │ ├── group-by - │ │ │ ├── columns: ol_o_id:10!null ol_d_id:11!null ol_w_id:12!null ol_number:13!null ol_dist_info:19 true_agg:40 - │ │ │ ├── grouping columns: ol_o_id:10!null ol_d_id:11!null ol_w_id:12!null ol_number:13!null - │ │ │ ├── key: (10-13) - │ │ │ ├── fd: (10-13)-->(19,40) + │ │ │ ├── columns: ol_o_id:11!null ol_d_id:12!null ol_w_id:13!null ol_number:14!null ol_dist_info:20 true_agg:44 + │ │ │ ├── grouping columns: ol_o_id:11!null ol_d_id:12!null ol_w_id:13!null ol_number:14!null + │ │ │ ├── key: (11-14) + │ │ │ ├── fd: (11-14)-->(20,44) │ │ │ ├── left-join (cross) - │ │ │ │ ├── columns: ol_o_id:10!null ol_d_id:11!null ol_w_id:12!null ol_number:13!null ol_dist_info:19 true:39 - │ │ │ │ ├── fd: (10-13)-->(19) + │ │ │ │ ├── columns: ol_o_id:11!null ol_d_id:12!null ol_w_id:13!null ol_number:14!null ol_dist_info:20 true:43 + │ │ │ │ ├── fd: (11-14)-->(20) │ │ │ │ ├── scan order_line - │ │ │ │ │ ├── columns: ol_o_id:10!null ol_d_id:11!null ol_w_id:12!null ol_number:13!null ol_dist_info:19 - │ │ │ │ │ ├── key: (10-13) - │ │ │ │ │ └── fd: (10-13)-->(19) + │ │ │ │ │ ├── columns: ol_o_id:11!null ol_d_id:12!null ol_w_id:13!null ol_number:14!null ol_dist_info:20 + │ │ │ │ │ ├── key: (11-14) + │ │ │ │ │ └── fd: (11-14)-->(20) │ │ │ │ ├── project - │ │ │ │ │ ├── columns: true:39!null - │ │ │ │ │ ├── fd: ()-->(39) + │ │ │ │ │ ├── columns: true:43!null + │ │ │ │ │ ├── fd: ()-->(43) │ │ │ │ │ ├── select - │ │ │ │ │ │ ├── columns: h_data:28!null + │ │ │ │ │ │ ├── columns: h_data:30!null │ │ │ │ │ │ ├── scan history - │ │ │ │ │ │ │ └── columns: h_data:28 + │ │ │ │ │ │ │ └── columns: h_data:30 │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── h_data:28 IS NOT NULL [outer=(28), constraints=(/28: (/NULL - ]; tight)] + │ │ │ │ │ │ └── h_data:30 IS NOT NULL [outer=(30), constraints=(/30: (/NULL - ]; tight)] │ │ │ │ │ └── projections - │ │ │ │ │ └── true [as=true:39] + │ │ │ │ │ └── true [as=true:43] │ │ │ │ └── filters - │ │ │ │ └── ol_dist_info:19 IS NOT NULL [outer=(19), constraints=(/19: (/NULL - ]; tight)] + │ │ │ │ └── ol_dist_info:20 IS NOT NULL [outer=(20), constraints=(/20: (/NULL - ]; tight)] │ │ │ └── aggregations - │ │ │ ├── const-not-null-agg [as=true_agg:40, outer=(39)] - │ │ │ │ └── true:39 - │ │ │ └── const-agg [as=ol_dist_info:19, outer=(19)] - │ │ │ └── ol_dist_info:19 + │ │ │ ├── const-not-null-agg [as=true_agg:44, outer=(43)] + │ │ │ │ └── true:43 + │ │ │ └── const-agg [as=ol_dist_info:20, outer=(20)] + │ │ │ └── ol_dist_info:20 │ │ └── filters - │ │ └── or [outer=(40), subquery] - │ │ ├── true_agg:40 IS NOT NULL + │ │ └── or [outer=(44), subquery] + │ │ ├── true_agg:44 IS NOT NULL │ │ └── is-not │ │ ├── subquery │ │ │ └── limit - │ │ │ ├── columns: ol_i_id:33!null + │ │ │ ├── columns: ol_i_id:36!null │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(33) + │ │ │ ├── fd: ()-->(36) │ │ │ ├── scan order_line - │ │ │ │ ├── columns: ol_i_id:33!null + │ │ │ │ ├── columns: ol_i_id:36!null │ │ │ │ └── limit hint: 1.00 │ │ │ └── 1 │ │ └── NULL │ └── filters - │ └── h_data:9 = ol_dist_info:19 [outer=(9,19), constraints=(/9: (/NULL - ]; /19: (/NULL - ]), fd=(9)==(19), (19)==(9)] + │ └── h_data:9 = ol_dist_info:20 [outer=(9,20), constraints=(/9: (/NULL - ]; /20: (/NULL - ]), fd=(9)==(20), (20)==(9)] └── projections - └── subquery [as=i_name:47, subquery] + └── subquery [as=i_name:52, subquery] └── limit - ├── columns: item.i_name:44 + ├── columns: item.i_name:48 ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(44) + ├── fd: ()-->(48) ├── scan item - │ ├── columns: item.i_name:44 + │ ├── columns: item.i_name:48 │ └── limit hint: 1.00 └── 1 @@ -1252,105 +1252,105 @@ SELECT * FROM (SELECT a.k AS a_k, b.x AS b_x FROM a, b) JOIN (SELECT c.x AS c_x, ON a_k = c_x AND c_x = b_x AND b_x = d_x ---- inner-join (hash) - ├── columns: a_k:1!null b_x:6!null c_x:8!null d_x:11!null + ├── columns: a_k:1!null b_x:7!null c_x:10!null d_x:14!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (11) - ├── fd: (1)==(6,8,11), (6)==(1,8,11), (8)==(1,6,11), (11)==(1,6,8) + ├── key: (14) + ├── fd: (1)==(7,10,14), (7)==(1,10,14), (10)==(1,7,14), (14)==(1,7,10) ├── inner-join (hash) - │ ├── columns: k:1!null b.x:6!null + │ ├── columns: k:1!null b.x:7!null │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - │ ├── key: (6) - │ ├── fd: (1)==(6), (6)==(1) + │ ├── key: (7) + │ ├── fd: (1)==(7), (7)==(1) │ ├── scan a │ │ ├── columns: k:1!null │ │ └── key: (1) │ ├── scan b - │ │ ├── columns: b.x:6!null - │ │ └── key: (6) + │ │ ├── columns: b.x:7!null + │ │ └── key: (7) │ └── filters - │ └── k:1 = b.x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ └── k:1 = b.x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] ├── inner-join (hash) - │ ├── columns: c.x:8!null d.x:11!null + │ ├── columns: c.x:10!null d.x:14!null │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - │ ├── key: (11) - │ ├── fd: (8)==(11), (11)==(8) + │ ├── key: (14) + │ ├── fd: (10)==(14), (14)==(10) │ ├── scan c - │ │ ├── columns: c.x:8!null - │ │ └── key: (8) + │ │ ├── columns: c.x:10!null + │ │ └── key: (10) │ ├── scan d - │ │ ├── columns: d.x:11!null - │ │ └── key: (11) + │ │ ├── columns: d.x:14!null + │ │ └── key: (14) │ └── filters - │ └── c.x:8 = d.x:11 [outer=(8,11), constraints=(/8: (/NULL - ]; /11: (/NULL - ]), fd=(8)==(11), (11)==(8)] + │ └── c.x:10 = d.x:14 [outer=(10,14), constraints=(/10: (/NULL - ]; /14: (/NULL - ]), fd=(10)==(14), (14)==(10)] └── filters - └── k:1 = c.x:8 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] + └── k:1 = c.x:10 [outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] norm expect=MapEqualityIntoJoinLeftAndRight SELECT * FROM (SELECT b.x AS b_x, c.x AS c_x FROM b, c), d WHERE b_x=d.x AND c_x=d.x ---- inner-join (hash) - ├── columns: b_x:1!null c_x:3!null x:6!null y:7!null z:8!null + ├── columns: b_x:1!null c_x:4!null x:8!null y:9!null z:10!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (6) - ├── fd: (1)==(3,6), (3)==(1,6), (6)-->(7,8), (6)==(1,3) + ├── key: (8) + ├── fd: (1)==(4,8), (4)==(1,8), (8)-->(9,10), (8)==(1,4) ├── inner-join (hash) - │ ├── columns: b.x:1!null c.x:3!null + │ ├── columns: b.x:1!null c.x:4!null │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - │ ├── key: (3) - │ ├── fd: (1)==(3), (3)==(1) + │ ├── key: (4) + │ ├── fd: (1)==(4), (4)==(1) │ ├── scan b │ │ ├── columns: b.x:1!null │ │ └── key: (1) │ ├── scan c - │ │ ├── columns: c.x:3!null - │ │ └── key: (3) + │ │ ├── columns: c.x:4!null + │ │ └── key: (4) │ └── filters - │ └── b.x:1 = c.x:3 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + │ └── b.x:1 = c.x:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] ├── scan d - │ ├── columns: d.x:6!null d.y:7!null d.z:8!null - │ ├── key: (6) - │ └── fd: (6)-->(7,8) + │ ├── columns: d.x:8!null d.y:9!null d.z:10!null + │ ├── key: (8) + │ └── fd: (8)-->(9,10) └── filters - └── b.x:1 = d.x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── b.x:1 = d.x:8 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] norm expect=MapEqualityIntoJoinLeftAndRight SELECT * FROM b, c, d WHERE b.x=c.x AND b.x=d.x ---- inner-join (hash) - ├── columns: x:1!null y:2 x:3!null y:4!null z:5!null x:6!null y:7!null z:8!null + ├── columns: x:1!null y:2 x:4!null y:5!null z:6!null x:8!null y:9!null z:10!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (6) - ├── fd: (1)-->(2), (3)-->(4,5), (6)-->(7,8), (3)==(1,6), (6)==(1,3), (1)==(3,6) + ├── key: (8) + ├── fd: (1)-->(2), (4)-->(5,6), (8)-->(9,10), (4)==(1,8), (8)==(1,4), (1)==(4,8) ├── scan b │ ├── columns: b.x:1!null b.y:2 │ ├── key: (1) │ └── fd: (1)-->(2) ├── inner-join (hash) - │ ├── columns: c.x:3!null c.y:4!null c.z:5!null d.x:6!null d.y:7!null d.z:8!null + │ ├── columns: c.x:4!null c.y:5!null c.z:6!null d.x:8!null d.y:9!null d.z:10!null │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - │ ├── key: (6) - │ ├── fd: (3)-->(4,5), (6)-->(7,8), (3)==(6), (6)==(3) + │ ├── key: (8) + │ ├── fd: (4)-->(5,6), (8)-->(9,10), (4)==(8), (8)==(4) │ ├── scan c - │ │ ├── columns: c.x:3!null c.y:4!null c.z:5!null - │ │ ├── key: (3) - │ │ └── fd: (3)-->(4,5) + │ │ ├── columns: c.x:4!null c.y:5!null c.z:6!null + │ │ ├── key: (4) + │ │ └── fd: (4)-->(5,6) │ ├── scan d - │ │ ├── columns: d.x:6!null d.y:7!null d.z:8!null - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7,8) + │ │ ├── columns: d.x:8!null d.y:9!null d.z:10!null + │ │ ├── key: (8) + │ │ └── fd: (8)-->(9,10) │ └── filters - │ └── c.x:3 = d.x:6 [outer=(3,6), constraints=(/3: (/NULL - ]; /6: (/NULL - ]), fd=(3)==(6), (6)==(3)] + │ └── c.x:4 = d.x:8 [outer=(4,8), constraints=(/4: (/NULL - ]; /8: (/NULL - ]), fd=(4)==(8), (8)==(4)] └── filters - └── b.x:1 = c.x:3 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + └── b.x:1 = c.x:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] norm expect=MapEqualityIntoJoinLeftAndRight SELECT * FROM c INNER JOIN d ON c.x = d.x AND d.x = c.y AND c.y = d.y AND d.y = c.z AND c.z = d.z AND d.z = c.x ---- inner-join (hash) - ├── columns: x:1!null y:2!null z:3!null x:4!null y:5!null z:6!null + ├── columns: x:1!null y:2!null z:3!null x:5!null y:6!null z:7!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (4) - ├── fd: (1)==(2-6), (2)==(1,3-6), (3)==(1,2,4-6), (4)==(1-3,5,6), (5)==(1-4,6), (6)==(1-5) + ├── key: (5) + ├── fd: (1)==(2,3,5-7), (2)==(1,3,5-7), (3)==(1,2,5-7), (5)==(1-3,6,7), (6)==(1-3,5,7), (7)==(1-3,5,6) ├── select │ ├── columns: c.x:1!null c.y:2!null c.z:3!null │ ├── key: (1) @@ -1363,27 +1363,27 @@ inner-join (hash) │ ├── c.x:1 = c.y:2 [outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ]), fd=(1)==(2), (2)==(1)] │ └── c.x:1 = c.z:3 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] ├── select - │ ├── columns: d.x:4!null d.y:5!null d.z:6!null - │ ├── key: (4) - │ ├── fd: (4)==(5,6), (5)==(4,6), (6)==(4,5) + │ ├── columns: d.x:5!null d.y:6!null d.z:7!null + │ ├── key: (5) + │ ├── fd: (5)==(6,7), (6)==(5,7), (7)==(5,6) │ ├── scan d - │ │ ├── columns: d.x:4!null d.y:5!null d.z:6!null - │ │ ├── key: (4) - │ │ └── fd: (4)-->(5,6) + │ │ ├── columns: d.x:5!null d.y:6!null d.z:7!null + │ │ ├── key: (5) + │ │ └── fd: (5)-->(6,7) │ └── filters - │ ├── d.x:4 = d.y:5 [outer=(4,5), constraints=(/4: (/NULL - ]; /5: (/NULL - ]), fd=(4)==(5), (5)==(4)] - │ └── d.x:4 = d.z:6 [outer=(4,6), constraints=(/4: (/NULL - ]; /6: (/NULL - ]), fd=(4)==(6), (6)==(4)] + │ ├── d.x:5 = d.y:6 [outer=(5,6), constraints=(/5: (/NULL - ]; /6: (/NULL - ]), fd=(5)==(6), (6)==(5)] + │ └── d.x:5 = d.z:7 [outer=(5,7), constraints=(/5: (/NULL - ]; /7: (/NULL - ]), fd=(5)==(7), (7)==(5)] └── filters - └── c.x:1 = d.x:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + └── c.x:1 = d.x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] norm expect=MapEqualityIntoJoinLeftAndRight SELECT * from c, d WHERE c.x = c.y AND c.x = d.x AND c.y = d.y; ---- inner-join (hash) - ├── columns: x:1!null y:2!null z:3!null x:4!null y:5!null z:6!null + ├── columns: x:1!null y:2!null z:3!null x:5!null y:6!null z:7!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (4) - ├── fd: (1)-->(3), (1)==(2,4,5), (2)==(1,4,5), (4)-->(6), (4)==(1,2,5), (5)==(1,2,4) + ├── key: (5) + ├── fd: (1)-->(3), (1)==(2,5,6), (2)==(1,5,6), (5)-->(7), (5)==(1,2,6), (6)==(1,2,5) ├── select │ ├── columns: c.x:1!null c.y:2!null c.z:3!null │ ├── key: (1) @@ -1395,26 +1395,26 @@ inner-join (hash) │ └── filters │ └── c.x:1 = c.y:2 [outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ]), fd=(1)==(2), (2)==(1)] ├── select - │ ├── columns: d.x:4!null d.y:5!null d.z:6!null - │ ├── key: (4) - │ ├── fd: (4)-->(6), (4)==(5), (5)==(4) + │ ├── columns: d.x:5!null d.y:6!null d.z:7!null + │ ├── key: (5) + │ ├── fd: (5)-->(7), (5)==(6), (6)==(5) │ ├── scan d - │ │ ├── columns: d.x:4!null d.y:5!null d.z:6!null - │ │ ├── key: (4) - │ │ └── fd: (4)-->(5,6) + │ │ ├── columns: d.x:5!null d.y:6!null d.z:7!null + │ │ ├── key: (5) + │ │ └── fd: (5)-->(6,7) │ └── filters - │ └── d.x:4 = d.y:5 [outer=(4,5), constraints=(/4: (/NULL - ]; /5: (/NULL - ]), fd=(4)==(5), (5)==(4)] + │ └── d.x:5 = d.y:6 [outer=(5,6), constraints=(/5: (/NULL - ]; /6: (/NULL - ]), fd=(5)==(6), (6)==(5)] └── filters - └── c.x:1 = d.x:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + └── c.x:1 = d.x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] norm expect=MapEqualityIntoJoinLeftAndRight SELECT * FROM c, d WHERE c.x = d.x AND d.x = c.y AND c.y = d.y ---- inner-join (hash) - ├── columns: x:1!null y:2!null z:3!null x:4!null y:5!null z:6!null + ├── columns: x:1!null y:2!null z:3!null x:5!null y:6!null z:7!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (4) - ├── fd: (1)-->(3), (1)==(2,4,5), (2)==(1,4,5), (4)-->(6), (4)==(1,2,5), (5)==(1,2,4) + ├── key: (5) + ├── fd: (1)-->(3), (1)==(2,5,6), (2)==(1,5,6), (5)-->(7), (5)==(1,2,6), (6)==(1,2,5) ├── select │ ├── columns: c.x:1!null c.y:2!null c.z:3!null │ ├── key: (1) @@ -1426,17 +1426,17 @@ inner-join (hash) │ └── filters │ └── c.x:1 = c.y:2 [outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ]), fd=(1)==(2), (2)==(1)] ├── select - │ ├── columns: d.x:4!null d.y:5!null d.z:6!null - │ ├── key: (4) - │ ├── fd: (4)-->(6), (4)==(5), (5)==(4) + │ ├── columns: d.x:5!null d.y:6!null d.z:7!null + │ ├── key: (5) + │ ├── fd: (5)-->(7), (5)==(6), (6)==(5) │ ├── scan d - │ │ ├── columns: d.x:4!null d.y:5!null d.z:6!null - │ │ ├── key: (4) - │ │ └── fd: (4)-->(5,6) + │ │ ├── columns: d.x:5!null d.y:6!null d.z:7!null + │ │ ├── key: (5) + │ │ └── fd: (5)-->(6,7) │ └── filters - │ └── d.x:4 = d.y:5 [outer=(4,5), constraints=(/4: (/NULL - ]; /5: (/NULL - ]), fd=(4)==(5), (5)==(4)] + │ └── d.x:5 = d.y:6 [outer=(5,6), constraints=(/5: (/NULL - ]; /6: (/NULL - ]), fd=(5)==(6), (6)==(5)] └── filters - └── c.x:1 = d.x:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + └── c.x:1 = d.x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] exec-ddl create table aa (a int, a1 int, a2 int) @@ -1454,8 +1454,8 @@ norm expect=MapEqualityIntoJoinLeftAndRight select * from aa, bb where a2 = b and b = a and a = b1 and b1 = a1 ---- inner-join (hash) - ├── columns: a:1!null a1:2!null a2:3!null b:5!null b1:6!null b2:7 - ├── fd: (1)==(2,3,5,6), (2)==(1,3,5,6), (3)==(1,2,5,6), (5)==(1-3,6), (6)==(1-3,5) + ├── columns: a:1!null a1:2!null a2:3!null b:6!null b1:7!null b2:8 + ├── fd: (1)==(2,3,6,7), (2)==(1,3,6,7), (3)==(1,2,6,7), (6)==(1-3,7), (7)==(1-3,6) ├── select │ ├── columns: a:1!null a1:2!null a2:3!null │ ├── fd: (1)==(2,3), (2)==(1,3), (3)==(1,2) @@ -1465,14 +1465,14 @@ inner-join (hash) │ ├── a:1 = a1:2 [outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ]), fd=(1)==(2), (2)==(1)] │ └── a:1 = a2:3 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] ├── select - │ ├── columns: b:5!null b1:6!null b2:7 - │ ├── fd: (5)==(6), (6)==(5) + │ ├── columns: b:6!null b1:7!null b2:8 + │ ├── fd: (6)==(7), (7)==(6) │ ├── scan bb - │ │ └── columns: b:5 b1:6 b2:7 + │ │ └── columns: b:6 b1:7 b2:8 │ └── filters - │ └── b:5 = b1:6 [outer=(5,6), constraints=(/5: (/NULL - ]; /6: (/NULL - ]), fd=(5)==(6), (6)==(5)] + │ └── b:6 = b1:7 [outer=(6,7), constraints=(/6: (/NULL - ]; /7: (/NULL - ]), fd=(6)==(7), (7)==(6)] └── filters - └── a:1 = b:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + └── a:1 = b:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] # -------------------------------------------------- # PushFilterIntoJoinLeft + PushFilterIntoJoinRight @@ -1482,10 +1482,10 @@ norm expect=(PushFilterIntoJoinLeft,PushFilterIntoJoinRight) SELECT * FROM a INNER JOIN b ON a.k=b.x AND a.i=1 AND b.y=1 ---- inner-join (hash) - ├── columns: k:1!null i:2!null f:3!null s:4 j:5 x:6!null y:7!null + ├── columns: k:1!null i:2!null f:3!null s:4 j:5 x:7!null y:8!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (6) - ├── fd: ()-->(2,7), (1)-->(3-5), (1)==(6), (6)==(1) + ├── key: (7) + ├── fd: ()-->(2,8), (1)-->(3-5), (1)==(7), (7)==(1) ├── select │ ├── columns: k:1!null i:2!null f:3!null s:4 j:5 │ ├── key: (1) @@ -1497,39 +1497,39 @@ inner-join (hash) │ └── filters │ └── i:2 = 1 [outer=(2), constraints=(/2: [/1 - /1]; tight), fd=()-->(2)] ├── select - │ ├── columns: x:6!null y:7!null - │ ├── key: (6) - │ ├── fd: ()-->(7) + │ ├── columns: x:7!null y:8!null + │ ├── key: (7) + │ ├── fd: ()-->(8) │ ├── scan b - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ └── filters - │ └── y:7 = 1 [outer=(7), constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] + │ └── y:8 = 1 [outer=(8), constraints=(/8: [/1 - /1]; tight), fd=()-->(8)] └── filters - └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # FULL JOIN should not push down conditions to either side of join. norm expect-not=(PushFilterIntoJoinLeft,PushFilterIntoJoinRight) SELECT * FROM a FULL JOIN b ON a.k=b.x AND a.i=1 AND b.y=1 ---- full-join (hash) - ├── columns: k:1 i:2 f:3 s:4 j:5 x:6 y:7 + ├── columns: k:1 i:2 f:3 s:4 j:5 x:7 y:8 ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) - ├── key: (1,6) - ├── fd: (1)-->(2-5), (6)-->(7) + ├── key: (1,7) + ├── fd: (1)-->(2-5), (7)-->(8) ├── scan a │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── scan b - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + │ └── fd: (7)-->(8) └── filters - ├── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + ├── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] ├── i:2 = 1 [outer=(2), constraints=(/2: [/1 - /1]; tight), fd=()-->(2)] - └── y:7 = 1 [outer=(7), constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] + └── y:8 = 1 [outer=(8), constraints=(/8: [/1 - /1]; tight), fd=()-->(8)] # Nested semi/anti-join case. norm expect=PushFilterIntoJoinRight @@ -1548,31 +1548,31 @@ semi-join-apply │ ├── key: (1) │ └── fd: (1)-->(2) ├── anti-join (cross) - │ ├── columns: k:3!null s:6!null + │ ├── columns: k:4!null s:7!null │ ├── outer: (2) - │ ├── key: (3) - │ ├── fd: ()-->(6) + │ ├── key: (4) + │ ├── fd: ()-->(7) │ ├── select - │ │ ├── columns: k:3!null s:6!null - │ │ ├── key: (3) - │ │ ├── fd: ()-->(6) + │ │ ├── columns: k:4!null s:7!null + │ │ ├── key: (4) + │ │ ├── fd: ()-->(7) │ │ ├── scan a - │ │ │ ├── columns: k:3!null s:6 - │ │ │ ├── key: (3) - │ │ │ └── fd: (3)-->(6) + │ │ │ ├── columns: k:4!null s:7 + │ │ │ ├── key: (4) + │ │ │ └── fd: (4)-->(7) │ │ └── filters - │ │ └── s:6 = 'foo' [outer=(6), constraints=(/6: [/'foo' - /'foo']; tight), fd=()-->(6)] + │ │ └── s:7 = 'foo' [outer=(7), constraints=(/7: [/'foo' - /'foo']; tight), fd=()-->(7)] │ ├── select - │ │ ├── columns: i:9!null - │ │ ├── fd: ()-->(9) + │ │ ├── columns: i:11!null + │ │ ├── fd: ()-->(11) │ │ ├── scan a - │ │ │ └── columns: i:9 + │ │ │ └── columns: i:11 │ │ └── filters - │ │ └── i:9 = 10 [outer=(9), constraints=(/9: [/10 - /10]; tight), fd=()-->(9)] + │ │ └── i:11 = 10 [outer=(11), constraints=(/11: [/10 - /10]; tight), fd=()-->(11)] │ └── filters │ └── y:2 > 100 [outer=(2), constraints=(/2: [/101 - ]; tight)] └── filters - └── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + └── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] # -------------------------------------------------- # SimplifyLeftJoin + SimplifyRightJoin @@ -1581,121 +1581,121 @@ norm expect=SimplifyLeftJoin SELECT * FROM a FULL JOIN a AS a2 ON a.k=a2.k ---- inner-join (hash) - ├── columns: k:1!null i:2 f:3!null s:4 j:5 k:6!null i:7 f:8!null s:9 j:10 + ├── columns: k:1!null i:2 f:3!null s:4 j:5 k:7!null i:8 f:9!null s:10 j:11 ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) - ├── key: (6) - ├── fd: (6)-->(7-10), (1)-->(2-5), (1)==(6), (6)==(1) + ├── key: (7) + ├── fd: (7)-->(8-11), (1)-->(2-5), (1)==(7), (7)==(1) ├── scan a2 - │ ├── columns: a2.k:6!null a2.i:7 a2.f:8!null a2.s:9 a2.j:10 - │ ├── key: (6) - │ └── fd: (6)-->(7-10) + │ ├── columns: a2.k:7!null a2.i:8 a2.f:9!null a2.s:10 a2.j:11 + │ ├── key: (7) + │ └── fd: (7)-->(8-11) ├── scan a │ ├── columns: a.k:1!null a.i:2 a.f:3!null a.s:4 a.j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters - └── a.k:1 = a2.k:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── a.k:1 = a2.k:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # Right side has partial rows, so only right-join can be simplified. norm expect=SimplifyRightJoin SELECT * FROM a FULL JOIN (SELECT * FROM a WHERE k>0) AS a2 ON a.k=a2.k ---- left-join (hash) - ├── columns: k:1!null i:2 f:3!null s:4 j:5 k:6 i:7 f:8 s:9 j:10 + ├── columns: k:1!null i:2 f:3!null s:4 j:5 k:7 i:8 f:9 s:10 j:11 ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) ├── key: (1) - ├── fd: (1)-->(2-10), (6)-->(7-10) + ├── fd: (1)-->(2-5,7-11), (7)-->(8-11) ├── scan a │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── select - │ ├── columns: k:6!null i:7 f:8!null s:9 j:10 - │ ├── key: (6) - │ ├── fd: (6)-->(7-10) + │ ├── columns: k:7!null i:8 f:9!null s:10 j:11 + │ ├── key: (7) + │ ├── fd: (7)-->(8-11) │ ├── scan a - │ │ ├── columns: k:6!null i:7 f:8!null s:9 j:10 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7-10) + │ │ ├── columns: k:7!null i:8 f:9!null s:10 j:11 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8-11) │ └── filters - │ └── k:6 > 0 [outer=(6), constraints=(/6: [/1 - ]; tight)] + │ └── k:7 > 0 [outer=(7), constraints=(/7: [/1 - ]; tight)] └── filters - └── k:1 = k:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── k:1 = k:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # Multiple equality conditions, with duplicates and reversed columns. norm expect=SimplifyLeftJoin SELECT * FROM a FULL JOIN a AS a2 ON a.k=a2.k AND a.k=a2.k AND a2.f=a.f ---- inner-join (hash) - ├── columns: k:1!null i:2 f:3!null s:4 j:5 k:6!null i:7 f:8!null s:9 j:10 + ├── columns: k:1!null i:2 f:3!null s:4 j:5 k:7!null i:8 f:9!null s:10 j:11 ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) - ├── key: (6) - ├── fd: (6)-->(7-10), (1)-->(2-5), (3)==(8), (8)==(3), (1)==(6), (6)==(1) + ├── key: (7) + ├── fd: (7)-->(8-11), (1)-->(2-5), (3)==(9), (9)==(3), (1)==(7), (7)==(1) ├── scan a2 - │ ├── columns: a2.k:6!null a2.i:7 a2.f:8!null a2.s:9 a2.j:10 - │ ├── key: (6) - │ └── fd: (6)-->(7-10) + │ ├── columns: a2.k:7!null a2.i:8 a2.f:9!null a2.s:10 a2.j:11 + │ ├── key: (7) + │ └── fd: (7)-->(8-11) ├── scan a │ ├── columns: a.k:1!null a.i:2 a.f:3!null a.s:4 a.j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters - ├── a2.f:8 = a.f:3 [outer=(3,8), constraints=(/3: (/NULL - ]; /8: (/NULL - ]), fd=(3)==(8), (8)==(3)] - └── a2.k:6 = a.k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + ├── a2.f:9 = a.f:3 [outer=(3,9), constraints=(/3: (/NULL - ]; /9: (/NULL - ]), fd=(3)==(9), (9)==(3)] + └── a2.k:7 = a.k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # Input contains Project operator. norm expect=SimplifyLeftJoin SELECT * FROM (SELECT length(s), f FROM a) AS a FULL JOIN a AS a2 ON a.f=a2.f ---- inner-join (hash) - ├── columns: length:6 f:3!null k:7!null i:8 f:9!null s:10 j:11 + ├── columns: length:7 f:3!null k:8!null i:9 f:10!null s:11 j:12 ├── multiplicity: left-rows(one-or-more), right-rows(one-or-more) ├── immutable - ├── fd: (7)-->(8-11), (3)==(9), (9)==(3) + ├── fd: (8)-->(9-12), (3)==(10), (10)==(3) ├── scan a2 - │ ├── columns: a2.k:7!null a2.i:8 a2.f:9!null a2.s:10 a2.j:11 - │ ├── key: (7) - │ └── fd: (7)-->(8-11) + │ ├── columns: a2.k:8!null a2.i:9 a2.f:10!null a2.s:11 a2.j:12 + │ ├── key: (8) + │ └── fd: (8)-->(9-12) ├── project - │ ├── columns: length:6 a.f:3!null + │ ├── columns: length:7 a.f:3!null │ ├── immutable │ ├── scan a │ │ └── columns: a.f:3!null a.s:4 │ └── projections - │ └── length(a.s:4) [as=length:6, outer=(4), immutable] + │ └── length(a.s:4) [as=length:7, outer=(4), immutable] └── filters - └── a.f:3 = a2.f:9 [outer=(3,9), constraints=(/3: (/NULL - ]; /9: (/NULL - ]), fd=(3)==(9), (9)==(3)] + └── a.f:3 = a2.f:10 [outer=(3,10), constraints=(/3: (/NULL - ]; /10: (/NULL - ]), fd=(3)==(10), (10)==(3)] # Multiple join levels. norm expect=SimplifyLeftJoin SELECT * FROM a FULL JOIN (SELECT * FROM a INNER JOIN a AS a2 ON a.k=a2.k) AS a2 ON a.f=a2.f ---- inner-join (hash) - ├── columns: k:1!null i:2 f:3!null s:4 j:5 k:6!null i:7 f:8!null s:9 j:10 k:11!null i:12 f:13!null s:14 j:15 + ├── columns: k:1!null i:2 f:3!null s:4 j:5 k:7!null i:8 f:9!null s:10 j:11 k:13!null i:14 f:15!null s:16 j:17 ├── multiplicity: left-rows(one-or-more), right-rows(one-or-more) - ├── key: (1,11) - ├── fd: (6)-->(7-10), (11)-->(12-15), (6)==(11), (11)==(6), (1)-->(2-5), (3)==(8), (8)==(3) + ├── key: (1,13) + ├── fd: (7)-->(8-11), (13)-->(14-17), (7)==(13), (13)==(7), (1)-->(2-5), (3)==(9), (9)==(3) ├── inner-join (hash) - │ ├── columns: a.k:6!null a.i:7 a.f:8!null a.s:9 a.j:10 a2.k:11!null a2.i:12 a2.f:13!null a2.s:14 a2.j:15 + │ ├── columns: a.k:7!null a.i:8 a.f:9!null a.s:10 a.j:11 a2.k:13!null a2.i:14 a2.f:15!null a2.s:16 a2.j:17 │ ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) - │ ├── key: (11) - │ ├── fd: (6)-->(7-10), (11)-->(12-15), (6)==(11), (11)==(6) + │ ├── key: (13) + │ ├── fd: (7)-->(8-11), (13)-->(14-17), (7)==(13), (13)==(7) │ ├── scan a - │ │ ├── columns: a.k:6!null a.i:7 a.f:8!null a.s:9 a.j:10 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7-10) + │ │ ├── columns: a.k:7!null a.i:8 a.f:9!null a.s:10 a.j:11 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8-11) │ ├── scan a2 - │ │ ├── columns: a2.k:11!null a2.i:12 a2.f:13!null a2.s:14 a2.j:15 - │ │ ├── key: (11) - │ │ └── fd: (11)-->(12-15) + │ │ ├── columns: a2.k:13!null a2.i:14 a2.f:15!null a2.s:16 a2.j:17 + │ │ ├── key: (13) + │ │ └── fd: (13)-->(14-17) │ └── filters - │ └── a.k:6 = a2.k:11 [outer=(6,11), constraints=(/6: (/NULL - ]; /11: (/NULL - ]), fd=(6)==(11), (11)==(6)] + │ └── a.k:7 = a2.k:13 [outer=(7,13), constraints=(/7: (/NULL - ]; /13: (/NULL - ]), fd=(7)==(13), (13)==(7)] ├── scan a │ ├── columns: a.k:1!null a.i:2 a.f:3!null a.s:4 a.j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) └── filters - └── a.f:3 = a.f:8 [outer=(3,8), constraints=(/3: (/NULL - ]; /8: (/NULL - ]), fd=(3)==(8), (8)==(3)] + └── a.f:3 = a.f:9 [outer=(3,9), constraints=(/3: (/NULL - ]; /9: (/NULL - ]), fd=(3)==(9), (9)==(3)] # Left joins on a foreign key turn into inner joins. norm expect=SimplifyLeftJoin @@ -1705,20 +1705,20 @@ LEFT OUTER JOIN a ON c.y = a.k ---- inner-join (hash) - ├── columns: x:1!null y:2!null z:3!null k:4!null i:5 f:6!null s:7 j:8 + ├── columns: x:1!null y:2!null z:3!null k:5!null i:6 f:7!null s:8 j:9 ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) ├── key: (1) - ├── fd: (1)-->(2,3), (4)-->(5-8), (2)==(4), (4)==(2) + ├── fd: (1)-->(2,3), (5)-->(6-9), (2)==(5), (5)==(2) ├── scan c │ ├── columns: x:1!null y:2!null z:3!null │ ├── key: (1) │ └── fd: (1)-->(2,3) ├── scan a - │ ├── columns: k:4!null i:5 f:6!null s:7 j:8 - │ ├── key: (4) - │ └── fd: (4)-->(5-8) + │ ├── columns: k:5!null i:6 f:7!null s:8 j:9 + │ ├── key: (5) + │ └── fd: (5)-->(6-9) └── filters - └── y:2 = k:4 [outer=(2,4), constraints=(/2: (/NULL - ]; /4: (/NULL - ]), fd=(2)==(4), (4)==(2)] + └── y:2 = k:5 [outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ]), fd=(2)==(5), (5)==(2)] # Left joins on a multiple-column foreign key turn into inner joins. norm expect=SimplifyLeftJoin @@ -1729,21 +1729,21 @@ ON d.z = c.z AND d.y = c.x ---- inner-join (hash) - ├── columns: x:1!null y:2!null z:3!null x:4!null y:5!null z:6!null + ├── columns: x:1!null y:2!null z:3!null x:5!null y:6!null z:7!null ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) ├── key: (1) - ├── fd: (1)-->(2,3), (4)-->(5,6), (3)==(6), (6)==(3), (2)==(4), (4)==(2) + ├── fd: (1)-->(2,3), (5)-->(6,7), (3)==(7), (7)==(3), (2)==(5), (5)==(2) ├── scan d │ ├── columns: d.x:1!null d.y:2!null d.z:3!null │ ├── key: (1) │ └── fd: (1)-->(2,3) ├── scan c - │ ├── columns: c.x:4!null c.y:5!null c.z:6!null - │ ├── key: (4) - │ └── fd: (4)-->(5,6) + │ ├── columns: c.x:5!null c.y:6!null c.z:7!null + │ ├── key: (5) + │ └── fd: (5)-->(6,7) └── filters - ├── d.z:3 = c.z:6 [outer=(3,6), constraints=(/3: (/NULL - ]; /6: (/NULL - ]), fd=(3)==(6), (6)==(3)] - └── d.y:2 = c.x:4 [outer=(2,4), constraints=(/2: (/NULL - ]; /4: (/NULL - ]), fd=(2)==(4), (4)==(2)] + ├── d.z:3 = c.z:7 [outer=(3,7), constraints=(/3: (/NULL - ]; /7: (/NULL - ]), fd=(3)==(7), (7)==(3)] + └── d.y:2 = c.x:5 [outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ]), fd=(2)==(5), (5)==(2)] # Left join on a part of a foreign key turns into an inner join. norm expect=SimplifyLeftJoin @@ -1753,20 +1753,20 @@ LEFT OUTER JOIN c ON d.z = c.z ---- inner-join (hash) - ├── columns: x:1!null y:2!null z:3!null x:4!null y:5!null z:6!null + ├── columns: x:1!null y:2!null z:3!null x:5!null y:6!null z:7!null ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-more) - ├── key: (1,4) - ├── fd: (1)-->(2,3), (4)-->(5,6), (3)==(6), (6)==(3) + ├── key: (1,5) + ├── fd: (1)-->(2,3), (5)-->(6,7), (3)==(7), (7)==(3) ├── scan d │ ├── columns: d.x:1!null d.y:2!null d.z:3!null │ ├── key: (1) │ └── fd: (1)-->(2,3) ├── scan c - │ ├── columns: c.x:4!null c.y:5!null c.z:6!null - │ ├── key: (4) - │ └── fd: (4)-->(5,6) + │ ├── columns: c.x:5!null c.y:6!null c.z:7!null + │ ├── key: (5) + │ └── fd: (5)-->(6,7) └── filters - └── d.z:3 = c.z:6 [outer=(3,6), constraints=(/3: (/NULL - ]; /6: (/NULL - ]), fd=(3)==(6), (6)==(3)] + └── d.z:3 = c.z:7 [outer=(3,7), constraints=(/3: (/NULL - ]; /7: (/NULL - ]), fd=(3)==(7), (7)==(3)] # Cross join case. The presence of a not-null foreign key implies that there # will be at least one right row when there is at least one left row, so left @@ -1778,41 +1778,41 @@ LEFT OUTER JOIN c ON True ---- inner-join (cross) - ├── columns: x:1!null y:2!null z:3!null x:4!null y:5!null z:6!null + ├── columns: x:1!null y:2!null z:3!null x:5!null y:6!null z:7!null ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-more) - ├── key: (1,4) - ├── fd: (1)-->(2,3), (4)-->(5,6) + ├── key: (1,5) + ├── fd: (1)-->(2,3), (5)-->(6,7) ├── scan d │ ├── columns: d.x:1!null d.y:2!null d.z:3!null │ ├── key: (1) │ └── fd: (1)-->(2,3) ├── scan c - │ ├── columns: c.x:4!null c.y:5!null c.z:6!null - │ ├── key: (4) - │ └── fd: (4)-->(5,6) + │ ├── columns: c.x:5!null c.y:6!null c.z:7!null + │ ├── key: (5) + │ └── fd: (5)-->(6,7) └── filters (true) norm expect=SimplifyRightJoin SELECT * FROM (SELECT count(*) FROM b) FULL JOIN a ON True ---- left-join (cross) - ├── columns: count:3!null k:4 i:5 f:6 s:7 j:8 + ├── columns: count:4!null k:5 i:6 f:7 s:8 j:9 ├── cardinality: [1 - ] ├── multiplicity: left-rows(one-or-more), right-rows(exactly-one) - ├── key: (4) - ├── fd: ()-->(3), (4)-->(5-8) + ├── key: (5) + ├── fd: ()-->(4), (5)-->(6-9) ├── scalar-group-by - │ ├── columns: count_rows:3!null + │ ├── columns: count_rows:4!null │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(3) + │ ├── fd: ()-->(4) │ ├── scan b │ └── aggregations - │ └── count-rows [as=count_rows:3] + │ └── count-rows [as=count_rows:4] ├── scan a - │ ├── columns: k:4!null i:5 f:6!null s:7 j:8 - │ ├── key: (4) - │ └── fd: (4)-->(5-8) + │ ├── columns: k:5!null i:6 f:7!null s:8 j:9 + │ ├── key: (5) + │ └── fd: (5)-->(6-9) └── filters (true) # Full-join. @@ -1820,23 +1820,23 @@ norm expect=SimplifyRightJoin SELECT * FROM (SELECT count(*) FROM b) FULL JOIN a ON True ---- left-join (cross) - ├── columns: count:3!null k:4 i:5 f:6 s:7 j:8 + ├── columns: count:4!null k:5 i:6 f:7 s:8 j:9 ├── cardinality: [1 - ] ├── multiplicity: left-rows(one-or-more), right-rows(exactly-one) - ├── key: (4) - ├── fd: ()-->(3), (4)-->(5-8) + ├── key: (5) + ├── fd: ()-->(4), (5)-->(6-9) ├── scalar-group-by - │ ├── columns: count_rows:3!null + │ ├── columns: count_rows:4!null │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(3) + │ ├── fd: ()-->(4) │ ├── scan b │ └── aggregations - │ └── count-rows [as=count_rows:3] + │ └── count-rows [as=count_rows:4] ├── scan a - │ ├── columns: k:4!null i:5 f:6!null s:7 j:8 - │ ├── key: (4) - │ └── fd: (4)-->(5-8) + │ ├── columns: k:5!null i:6 f:7!null s:8 j:9 + │ ├── key: (5) + │ └── fd: (5)-->(6-9) └── filters (true) # Full-join. @@ -1844,45 +1844,45 @@ norm expect=SimplifyRightJoin SELECT * FROM (SELECT count(*) FROM b) FULL JOIN a ON True ---- left-join (cross) - ├── columns: count:3!null k:4 i:5 f:6 s:7 j:8 + ├── columns: count:4!null k:5 i:6 f:7 s:8 j:9 ├── cardinality: [1 - ] ├── multiplicity: left-rows(one-or-more), right-rows(exactly-one) - ├── key: (4) - ├── fd: ()-->(3), (4)-->(5-8) + ├── key: (5) + ├── fd: ()-->(4), (5)-->(6-9) ├── scalar-group-by - │ ├── columns: count_rows:3!null + │ ├── columns: count_rows:4!null │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(3) + │ ├── fd: ()-->(4) │ ├── scan b │ └── aggregations - │ └── count-rows [as=count_rows:3] + │ └── count-rows [as=count_rows:4] ├── scan a - │ ├── columns: k:4!null i:5 f:6!null s:7 j:8 - │ ├── key: (4) - │ └── fd: (4)-->(5-8) + │ ├── columns: k:5!null i:6 f:7!null s:8 j:9 + │ ├── key: (5) + │ └── fd: (5)-->(6-9) └── filters (true) norm expect=SimplifyLeftJoin SELECT * FROM a LEFT JOIN (SELECT count(*) FROM b) ON True ---- inner-join (cross) - ├── columns: k:1!null i:2 f:3!null s:4 j:5 count:8!null + ├── columns: k:1!null i:2 f:3!null s:4 j:5 count:10!null ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) ├── key: (1) - ├── fd: ()-->(8), (1)-->(2-5) + ├── fd: ()-->(10), (1)-->(2-5) ├── scan a │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── scalar-group-by - │ ├── columns: count_rows:8!null + │ ├── columns: count_rows:10!null │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(8) + │ ├── fd: ()-->(10) │ ├── scan b │ └── aggregations - │ └── count-rows [as=count_rows:8] + │ └── count-rows [as=count_rows:10] └── filters (true) # Full-join. @@ -1890,19 +1890,19 @@ norm expect=SimplifyLeftJoin SELECT * FROM a FULL JOIN (SELECT count(*) FROM b) ON True ---- left-join (cross) - ├── columns: k:1 i:2 f:3 s:4 j:5 count:8!null + ├── columns: k:1 i:2 f:3 s:4 j:5 count:10!null ├── cardinality: [1 - ] ├── multiplicity: left-rows(one-or-more), right-rows(exactly-one) ├── key: (1) - ├── fd: ()-->(8), (1)-->(2-5) + ├── fd: ()-->(10), (1)-->(2-5) ├── scalar-group-by - │ ├── columns: count_rows:8!null + │ ├── columns: count_rows:10!null │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(8) + │ ├── fd: ()-->(10) │ ├── scan b │ └── aggregations - │ └── count-rows [as=count_rows:8] + │ └── count-rows [as=count_rows:10] ├── scan a │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 │ ├── key: (1) @@ -1919,32 +1919,32 @@ project ├── key: (1) ├── fd: (1)-->(2-5) └── select - ├── columns: k:1!null i:2 f:3!null s:4 j:5 sum:7!null + ├── columns: k:1!null i:2 f:3!null s:4 j:5 sum:8!null ├── immutable ├── key: (1) - ├── fd: ()-->(7), (1)-->(2-5) + ├── fd: ()-->(8), (1)-->(2-5) ├── group-by - │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 sum:7 + │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 sum:8 │ ├── grouping columns: k:1!null │ ├── key: (1) - │ ├── fd: (1)-->(2-5,7) + │ ├── fd: (1)-->(2-5,8) │ ├── inner-join-apply - │ │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 column1:6 + │ │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 column1:7 │ │ ├── fd: (1)-->(2-5) │ │ ├── scan a │ │ │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2-5) │ │ ├── values - │ │ │ ├── columns: column1:6 + │ │ │ ├── columns: column1:7 │ │ │ ├── outer: (1) │ │ │ ├── cardinality: [2 - 2] │ │ │ ├── (k:1,) │ │ │ └── (1,) │ │ └── filters (true) │ └── aggregations - │ ├── sum [as=sum:7, outer=(6)] - │ │ └── column1:6 + │ ├── sum [as=sum:8, outer=(7)] + │ │ └── column1:7 │ ├── const-agg [as=i:2, outer=(2)] │ │ └── i:2 │ ├── const-agg [as=f:3, outer=(3)] @@ -1954,30 +1954,30 @@ project │ └── const-agg [as=j:5, outer=(5)] │ └── j:5 └── filters - └── sum:7 = 1 [outer=(7), immutable, constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] + └── sum:8 = 1 [outer=(8), immutable, constraints=(/8: [/1 - /1]; tight), fd=()-->(8)] # Don't simplify left join norm expect-not=SimplifyRightJoin SELECT * FROM (SELECT count(*) FROM b) LEFT JOIN a ON True ---- left-join (cross) - ├── columns: count:3!null k:4 i:5 f:6 s:7 j:8 + ├── columns: count:4!null k:5 i:6 f:7 s:8 j:9 ├── cardinality: [1 - ] ├── multiplicity: left-rows(one-or-more), right-rows(exactly-one) - ├── key: (4) - ├── fd: ()-->(3), (4)-->(5-8) + ├── key: (5) + ├── fd: ()-->(4), (5)-->(6-9) ├── scalar-group-by - │ ├── columns: count_rows:3!null + │ ├── columns: count_rows:4!null │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(3) + │ ├── fd: ()-->(4) │ ├── scan b │ └── aggregations - │ └── count-rows [as=count_rows:3] + │ └── count-rows [as=count_rows:4] ├── scan a - │ ├── columns: k:4!null i:5 f:6!null s:7 j:8 - │ ├── key: (4) - │ └── fd: (4)-->(5-8) + │ ├── columns: k:5!null i:6 f:7!null s:8 j:9 + │ ├── key: (5) + │ └── fd: (5)-->(6-9) └── filters (true) # Don't simplify right join @@ -1985,19 +1985,19 @@ norm expect-not=SimplifyLeftJoin SELECT * FROM a RIGHT JOIN (SELECT count(*) FROM b) ON True ---- left-join (cross) - ├── columns: k:1 i:2 f:3 s:4 j:5 count:8!null + ├── columns: k:1 i:2 f:3 s:4 j:5 count:10!null ├── cardinality: [1 - ] ├── multiplicity: left-rows(one-or-more), right-rows(exactly-one) ├── key: (1) - ├── fd: ()-->(8), (1)-->(2-5) + ├── fd: ()-->(10), (1)-->(2-5) ├── scalar-group-by - │ ├── columns: count_rows:8!null + │ ├── columns: count_rows:10!null │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(8) + │ ├── fd: ()-->(10) │ ├── scan b │ └── aggregations - │ └── count-rows [as=count_rows:8] + │ └── count-rows [as=count_rows:10] ├── scan a │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 │ ├── key: (1) @@ -2012,20 +2012,20 @@ LEFT OUTER JOIN a ON c.z = a.k ---- left-join (hash) - ├── columns: x:1!null y:2!null z:3!null k:4 i:5 f:6 s:7 j:8 + ├── columns: x:1!null y:2!null z:3!null k:5 i:6 f:7 s:8 j:9 ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) ├── key: (1) - ├── fd: (1)-->(2-8), (4)-->(5-8) + ├── fd: (1)-->(2,3,5-9), (5)-->(6-9) ├── scan c │ ├── columns: x:1!null y:2!null z:3!null │ ├── key: (1) │ └── fd: (1)-->(2,3) ├── scan a - │ ├── columns: k:4!null i:5 f:6!null s:7 j:8 - │ ├── key: (4) - │ └── fd: (4)-->(5-8) + │ ├── columns: k:5!null i:6 f:7!null s:8 j:9 + │ ├── key: (5) + │ └── fd: (5)-->(6-9) └── filters - └── z:3 = k:4 [outer=(3,4), constraints=(/3: (/NULL - ]; /4: (/NULL - ]), fd=(3)==(4), (4)==(3)] + └── z:3 = k:5 [outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ]), fd=(3)==(5), (5)==(3)] # Can't simplify: joins on non-foreign keys still in foreign key index. norm expect-not=(SimplifyRightJoin,SimplifyLeftJoin) @@ -2035,85 +2035,85 @@ LEFT OUTER JOIN a ON c.x = a.k ---- left-join (hash) - ├── columns: x:1!null y:2!null z:3!null k:4 i:5 f:6 s:7 j:8 + ├── columns: x:1!null y:2!null z:3!null k:5 i:6 f:7 s:8 j:9 ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) ├── key: (1) - ├── fd: (1)-->(2-8), (4)-->(5-8) + ├── fd: (1)-->(2,3,5-9), (5)-->(6-9) ├── scan c │ ├── columns: x:1!null y:2!null z:3!null │ ├── key: (1) │ └── fd: (1)-->(2,3) ├── scan a - │ ├── columns: k:4!null i:5 f:6!null s:7 j:8 - │ ├── key: (4) - │ └── fd: (4)-->(5-8) + │ ├── columns: k:5!null i:6 f:7!null s:8 j:9 + │ ├── key: (5) + │ └── fd: (5)-->(6-9) └── filters - └── x:1 = k:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + └── x:1 = k:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] # Can't simplify: non-equality condition. norm expect-not=(SimplifyRightJoin,SimplifyLeftJoin) SELECT * FROM a FULL JOIN a AS a2 ON a.k(2-5), (6)-->(7-10) + ├── columns: k:1 i:2 f:3 s:4 j:5 k:7 i:8 f:9 s:10 j:11 + ├── key: (1,7) + ├── fd: (1)-->(2-5), (7)-->(8-11) ├── scan a │ ├── columns: a.k:1!null a.i:2 a.f:3!null a.s:4 a.j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── scan a2 - │ ├── columns: a2.k:6!null a2.i:7 a2.f:8!null a2.s:9 a2.j:10 - │ ├── key: (6) - │ └── fd: (6)-->(7-10) + │ ├── columns: a2.k:7!null a2.i:8 a2.f:9!null a2.s:10 a2.j:11 + │ ├── key: (7) + │ └── fd: (7)-->(8-11) └── filters - └── a.k:1 < a2.k:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── a.k:1 < a2.k:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] # Can't simplify: non-join equality condition. norm expect-not=(SimplifyRightJoin,SimplifyLeftJoin) SELECT * FROM a FULL JOIN a AS a2 ON a.f=1 AND a.f=a2.f ---- full-join (hash) - ├── columns: k:1 i:2 f:3 s:4 j:5 k:6 i:7 f:8 s:9 j:10 - ├── key: (1,6) - ├── fd: (1)-->(2-5), (6)-->(7-10) + ├── columns: k:1 i:2 f:3 s:4 j:5 k:7 i:8 f:9 s:10 j:11 + ├── key: (1,7) + ├── fd: (1)-->(2-5), (7)-->(8-11) ├── scan a │ ├── columns: a.k:1!null a.i:2 a.f:3!null a.s:4 a.j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── scan a2 - │ ├── columns: a2.k:6!null a2.i:7 a2.f:8!null a2.s:9 a2.j:10 - │ ├── key: (6) - │ └── fd: (6)-->(7-10) + │ ├── columns: a2.k:7!null a2.i:8 a2.f:9!null a2.s:10 a2.j:11 + │ ├── key: (7) + │ └── fd: (7)-->(8-11) └── filters ├── a.f:3 = 1.0 [outer=(3), constraints=(/3: [/1.0 - /1.0]; tight), fd=()-->(3)] - └── a.f:3 = a2.f:8 [outer=(3,8), constraints=(/3: (/NULL - ]; /8: (/NULL - ]), fd=(3)==(8), (8)==(3)] + └── a.f:3 = a2.f:9 [outer=(3,9), constraints=(/3: (/NULL - ]; /9: (/NULL - ]), fd=(3)==(9), (9)==(3)] # Can't simplify: non-null column. norm expect-not=(SimplifyRightJoin,SimplifyLeftJoin) SELECT * FROM a FULL JOIN a AS a2 ON a.s=a2.s ---- full-join (hash) - ├── columns: k:1 i:2 f:3 s:4 j:5 k:6 i:7 f:8 s:9 j:10 - ├── key: (1,6) - ├── fd: (1)-->(2-5), (6)-->(7-10) + ├── columns: k:1 i:2 f:3 s:4 j:5 k:7 i:8 f:9 s:10 j:11 + ├── key: (1,7) + ├── fd: (1)-->(2-5), (7)-->(8-11) ├── scan a │ ├── columns: a.k:1!null a.i:2 a.f:3!null a.s:4 a.j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── scan a2 - │ ├── columns: a2.k:6!null a2.i:7 a2.f:8!null a2.s:9 a2.j:10 - │ ├── key: (6) - │ └── fd: (6)-->(7-10) + │ ├── columns: a2.k:7!null a2.i:8 a2.f:9!null a2.s:10 a2.j:11 + │ ├── key: (7) + │ └── fd: (7)-->(8-11) └── filters - └── a.s:4 = a2.s:9 [outer=(4,9), constraints=(/4: (/NULL - ]; /9: (/NULL - ]), fd=(4)==(9), (9)==(4)] + └── a.s:4 = a2.s:10 [outer=(4,10), constraints=(/4: (/NULL - ]; /10: (/NULL - ]), fd=(4)==(10), (10)==(4)] # Can't simplify: equality column that is synthesized. norm expect-not=(SimplifyRightJoin,SimplifyLeftJoin) SELECT * FROM a FULL JOIN (SELECT k+1 AS k FROM a) AS a2 ON a.k=a2.k ---- full-join (hash) - ├── columns: k:1 i:2 f:3 s:4 j:5 k:11 + ├── columns: k:1 i:2 f:3 s:4 j:5 k:13 ├── multiplicity: left-rows(one-or-more), right-rows(exactly-one) ├── immutable ├── fd: (1)-->(2-5) @@ -2122,116 +2122,116 @@ full-join (hash) │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── project - │ ├── columns: k:11!null + │ ├── columns: k:13!null │ ├── immutable │ ├── scan a - │ │ ├── columns: a.k:6!null - │ │ └── key: (6) + │ │ ├── columns: a.k:7!null + │ │ └── key: (7) │ └── projections - │ └── a.k:6 + 1 [as=k:11, outer=(6), immutable] + │ └── a.k:7 + 1 [as=k:13, outer=(7), immutable] └── filters - └── a.k:1 = k:11 [outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ]), fd=(1)==(11), (11)==(1)] + └── a.k:1 = k:13 [outer=(1,13), constraints=(/1: (/NULL - ]; /13: (/NULL - ]), fd=(1)==(13), (13)==(1)] # Can't simplify: equality condition with different column ordinals. norm expect-not=(SimplifyRightJoin,SimplifyLeftJoin) SELECT * FROM a FULL JOIN a AS a2 ON a.k=a2.f ---- full-join (cross) - ├── columns: k:1 i:2 f:3 s:4 j:5 k:6 i:7 f:8 s:9 j:10 + ├── columns: k:1 i:2 f:3 s:4 j:5 k:7 i:8 f:9 s:10 j:11 ├── multiplicity: left-rows(one-or-more), right-rows(exactly-one) - ├── key: (1,6) - ├── fd: (1)-->(2-5), (6)-->(7-10) + ├── key: (1,7) + ├── fd: (1)-->(2-5), (7)-->(8-11) ├── scan a │ ├── columns: a.k:1!null a.i:2 a.f:3!null a.s:4 a.j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── scan a2 - │ ├── columns: a2.k:6!null a2.i:7 a2.f:8!null a2.s:9 a2.j:10 - │ ├── key: (6) - │ └── fd: (6)-->(7-10) + │ ├── columns: a2.k:7!null a2.i:8 a2.f:9!null a2.s:10 a2.j:11 + │ ├── key: (7) + │ └── fd: (7)-->(8-11) └── filters - └── a.k:1 = a2.f:8 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] + └── a.k:1 = a2.f:9 [outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ]), fd=(1)==(9), (9)==(1)] # Can't simplify: one equality condition has columns from same side of join. norm expect-not=(SimplifyRightJoin,SimplifyLeftJoin) SELECT * FROM a FULL JOIN a AS a2 ON a.k=a2.k AND a.f=a.f AND a2.f=a2.f ---- full-join (hash) - ├── columns: k:1 i:2 f:3 s:4 j:5 k:6 i:7 f:8 s:9 j:10 + ├── columns: k:1 i:2 f:3 s:4 j:5 k:7 i:8 f:9 s:10 j:11 ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) - ├── key: (1,6) - ├── fd: (1)-->(2-5), (6)-->(7-10) + ├── key: (1,7) + ├── fd: (1)-->(2-5), (7)-->(8-11) ├── scan a │ ├── columns: a.k:1!null a.i:2 a.f:3!null a.s:4 a.j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── scan a2 - │ ├── columns: a2.k:6!null a2.i:7 a2.f:8!null a2.s:9 a2.j:10 - │ ├── key: (6) - │ └── fd: (6)-->(7-10) + │ ├── columns: a2.k:7!null a2.i:8 a2.f:9!null a2.s:10 a2.j:11 + │ ├── key: (7) + │ └── fd: (7)-->(8-11) └── filters - ├── a.k:1 = a2.k:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + ├── a.k:1 = a2.k:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] ├── a.f:3 IS DISTINCT FROM CAST(NULL AS FLOAT8) [outer=(3), constraints=(/3: (/NULL - ]; tight)] - └── a2.f:8 IS DISTINCT FROM CAST(NULL AS FLOAT8) [outer=(8), constraints=(/8: (/NULL - ]; tight)] + └── a2.f:9 IS DISTINCT FROM CAST(NULL AS FLOAT8) [outer=(9), constraints=(/9: (/NULL - ]; tight)] # Can't simplify: equality conditions have columns from different tables. norm expect-not=(SimplifyRightJoin,SimplifyLeftJoin) SELECT * FROM (SELECT * FROM a, b) AS a FULL JOIN a AS a2 ON a.k=a2.k AND a.x=a2.k ---- full-join (hash) - ├── columns: k:1 i:2 f:3 s:4 j:5 x:6 y:7 k:8 i:9 f:10 s:11 j:12 + ├── columns: k:1 i:2 f:3 s:4 j:5 x:7 y:8 k:10 i:11 f:12 s:13 j:14 ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) - ├── key: (1,6,8) - ├── fd: (1)-->(2-5), (6)-->(7), (8)-->(9-12) + ├── key: (1,7,10) + ├── fd: (1)-->(2-5), (7)-->(8), (10)-->(11-14) ├── inner-join (cross) - │ ├── columns: a.k:1!null a.i:2 a.f:3!null a.s:4 a.j:5 x:6!null y:7 - │ ├── key: (1,6) - │ ├── fd: (1)-->(2-5), (6)-->(7) + │ ├── columns: a.k:1!null a.i:2 a.f:3!null a.s:4 a.j:5 x:7!null y:8 + │ ├── key: (1,7) + │ ├── fd: (1)-->(2-5), (7)-->(8) │ ├── scan a │ │ ├── columns: a.k:1!null a.i:2 a.f:3!null a.s:4 a.j:5 │ │ ├── key: (1) │ │ └── fd: (1)-->(2-5) │ ├── scan b - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ └── filters (true) ├── scan a2 - │ ├── columns: a2.k:8!null a2.i:9 a2.f:10!null a2.s:11 a2.j:12 - │ ├── key: (8) - │ └── fd: (8)-->(9-12) + │ ├── columns: a2.k:10!null a2.i:11 a2.f:12!null a2.s:13 a2.j:14 + │ ├── key: (10) + │ └── fd: (10)-->(11-14) └── filters - ├── a.k:1 = a2.k:8 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] - └── x:6 = a2.k:8 [outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ]), fd=(6)==(8), (8)==(6)] + ├── a.k:1 = a2.k:10 [outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] + └── x:7 = a2.k:10 [outer=(7,10), constraints=(/7: (/NULL - ]; /10: (/NULL - ]), fd=(7)==(10), (10)==(7)] # Can't simplify: The a2.x column is not part of unfilteredCols. norm expect-not=(SimplifyRightJoin,SimplifyLeftJoin) SELECT * FROM a LEFT JOIN (SELECT * FROM a, b) AS a2 ON a.k=a2.x ---- left-join (hash) - ├── columns: k:1!null i:2 f:3!null s:4 j:5 k:6 i:7 f:8 s:9 j:10 x:11 y:12 + ├── columns: k:1!null i:2 f:3!null s:4 j:5 k:7 i:8 f:9 s:10 j:11 x:13 y:14 ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) - ├── key: (1,6,11) - ├── fd: (1)-->(2-5), (6)-->(7-10), (11)-->(12) + ├── key: (1,7,13) + ├── fd: (1)-->(2-5), (7)-->(8-11), (13)-->(14) ├── scan a │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── inner-join (cross) - │ ├── columns: k:6!null i:7 f:8!null s:9 j:10 x:11!null y:12 - │ ├── key: (6,11) - │ ├── fd: (6)-->(7-10), (11)-->(12) + │ ├── columns: k:7!null i:8 f:9!null s:10 j:11 x:13!null y:14 + │ ├── key: (7,13) + │ ├── fd: (7)-->(8-11), (13)-->(14) │ ├── scan a - │ │ ├── columns: k:6!null i:7 f:8!null s:9 j:10 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7-10) + │ │ ├── columns: k:7!null i:8 f:9!null s:10 j:11 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8-11) │ ├── scan b - │ │ ├── columns: x:11!null y:12 - │ │ ├── key: (11) - │ │ └── fd: (11)-->(12) + │ │ ├── columns: x:13!null y:14 + │ │ ├── key: (13) + │ │ └── fd: (13)-->(14) │ └── filters (true) └── filters - └── k:1 = x:11 [outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ]), fd=(1)==(11), (11)==(1)] + └── k:1 = x:13 [outer=(1,13), constraints=(/1: (/NULL - ]; /13: (/NULL - ]), fd=(1)==(13), (13)==(1)] # Can't simplify if IGNORE_FOREIGN_KEYS hint is passed. norm expect-not=(SimplifyRightJoin,SimplifyLeftJoin) @@ -2241,20 +2241,20 @@ LEFT OUTER JOIN a ON c.y = a.k ---- left-join (hash) - ├── columns: x:1!null y:2!null z:3!null k:4 i:5 f:6 s:7 j:8 + ├── columns: x:1!null y:2!null z:3!null k:5 i:6 f:7 s:8 j:9 ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) ├── key: (1) - ├── fd: (1)-->(2-8), (4)-->(5-8) + ├── fd: (1)-->(2,3,5-9), (5)-->(6-9) ├── scan c │ ├── columns: x:1!null y:2!null z:3!null │ ├── key: (1) │ └── fd: (1)-->(2,3) ├── scan a - │ ├── columns: k:4!null i:5 f:6!null s:7 j:8 - │ ├── key: (4) - │ └── fd: (4)-->(5-8) + │ ├── columns: k:5!null i:6 f:7!null s:8 j:9 + │ ├── key: (5) + │ └── fd: (5)-->(6-9) └── filters - └── y:2 = k:4 [outer=(2,4), constraints=(/2: (/NULL - ]; /4: (/NULL - ]), fd=(2)==(4), (4)==(2)] + └── y:2 = k:5 [outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ]), fd=(2)==(5), (5)==(2)] # -------------------------------------------------- # EliminateSemiJoin @@ -2346,7 +2346,7 @@ group-by ├── key: (1) ├── fd: (1)-->(2) ├── project-set - │ ├── columns: x:1!null y:2 generate_series:3 + │ ├── columns: x:1!null y:2 generate_series:4 │ ├── immutable │ ├── fd: (1)-->(2) │ ├── scan xy @@ -2369,60 +2369,60 @@ norm expect=HoistJoinProjectRight SELECT * FROM a INNER JOIN (SELECT x FROM b WHERE y=10) ON x=k ---- project - ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:6!null - ├── key: (6) - ├── fd: (1)-->(2-5), (1)==(6), (6)==(1) + ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:7!null + ├── key: (7) + ├── fd: (1)-->(2-5), (1)==(7), (7)==(1) └── inner-join (hash) - ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:6!null y:7!null + ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:7!null y:8!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (6) - ├── fd: ()-->(7), (1)-->(2-5), (1)==(6), (6)==(1) + ├── key: (7) + ├── fd: ()-->(8), (1)-->(2-5), (1)==(7), (7)==(1) ├── scan a │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── select - │ ├── columns: x:6!null y:7!null - │ ├── key: (6) - │ ├── fd: ()-->(7) + │ ├── columns: x:7!null y:8!null + │ ├── key: (7) + │ ├── fd: ()-->(8) │ ├── scan b - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ └── filters - │ └── y:7 = 10 [outer=(7), constraints=(/7: [/10 - /10]; tight), fd=()-->(7)] + │ └── y:8 = 10 [outer=(8), constraints=(/8: [/10 - /10]; tight), fd=()-->(8)] └── filters - └── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # Left-join case. norm expect=HoistJoinProjectRight SELECT * FROM a LEFT JOIN (SELECT x FROM b WHERE y=10) ON x=k ---- project - ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:6 + ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:7 ├── key: (1) - ├── fd: (1)-->(2-6) + ├── fd: (1)-->(2-5,7) └── left-join (hash) - ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:6 y:7 + ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:7 y:8 ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) ├── key: (1) - ├── fd: (1)-->(2-7) + ├── fd: (1)-->(2-5,7,8) ├── scan a │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── select - │ ├── columns: x:6!null y:7!null - │ ├── key: (6) - │ ├── fd: ()-->(7) + │ ├── columns: x:7!null y:8!null + │ ├── key: (7) + │ ├── fd: ()-->(8) │ ├── scan b - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ └── filters - │ └── y:7 = 10 [outer=(7), constraints=(/7: [/10 - /10]; tight), fd=()-->(7)] + │ └── y:8 = 10 [outer=(8), constraints=(/8: [/10 - /10]; tight), fd=()-->(8)] └── filters - └── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # -------------------------------------------------- # HoistJoinProjectLeft @@ -2433,14 +2433,14 @@ norm expect=HoistJoinProjectLeft SELECT * FROM (SELECT x FROM b WHERE y=10) INNER JOIN a ON x=k ---- project - ├── columns: x:1!null k:3!null i:4 f:5!null s:6 j:7 - ├── key: (3) - ├── fd: (3)-->(4-7), (1)==(3), (3)==(1) + ├── columns: x:1!null k:4!null i:5 f:6!null s:7 j:8 + ├── key: (4) + ├── fd: (4)-->(5-8), (1)==(4), (4)==(1) └── inner-join (hash) - ├── columns: x:1!null y:2!null k:3!null i:4 f:5!null s:6 j:7 + ├── columns: x:1!null y:2!null k:4!null i:5 f:6!null s:7 j:8 ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (3) - ├── fd: ()-->(2), (3)-->(4-7), (1)==(3), (3)==(1) + ├── key: (4) + ├── fd: ()-->(2), (4)-->(5-8), (1)==(4), (4)==(1) ├── select │ ├── columns: x:1!null y:2!null │ ├── key: (1) @@ -2452,25 +2452,25 @@ project │ └── filters │ └── y:2 = 10 [outer=(2), constraints=(/2: [/10 - /10]; tight), fd=()-->(2)] ├── scan a - │ ├── columns: k:3!null i:4 f:5!null s:6 j:7 - │ ├── key: (3) - │ └── fd: (3)-->(4-7) + │ ├── columns: k:4!null i:5 f:6!null s:7 j:8 + │ ├── key: (4) + │ └── fd: (4)-->(5-8) └── filters - └── x:1 = k:3 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + └── x:1 = k:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] # Left-join case. norm expect=HoistJoinProjectLeft SELECT * FROM (SELECT x FROM b WHERE y=10) LEFT JOIN a ON x=k ---- project - ├── columns: x:1!null k:3 i:4 f:5 s:6 j:7 + ├── columns: x:1!null k:4 i:5 f:6 s:7 j:8 ├── key: (1) - ├── fd: (3)-->(4-7), (1)-->(3-7) + ├── fd: (4)-->(5-8), (1)-->(4-8) └── left-join (hash) - ├── columns: x:1!null y:2!null k:3 i:4 f:5 s:6 j:7 + ├── columns: x:1!null y:2!null k:4 i:5 f:6 s:7 j:8 ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) ├── key: (1) - ├── fd: ()-->(2), (3)-->(4-7), (1)-->(3-7) + ├── fd: ()-->(2), (4)-->(5-8), (1)-->(4-8) ├── select │ ├── columns: x:1!null y:2!null │ ├── key: (1) @@ -2482,11 +2482,11 @@ project │ └── filters │ └── y:2 = 10 [outer=(2), constraints=(/2: [/10 - /10]; tight), fd=()-->(2)] ├── scan a - │ ├── columns: k:3!null i:4 f:5!null s:6 j:7 - │ ├── key: (3) - │ └── fd: (3)-->(4-7) + │ ├── columns: k:4!null i:5 f:6!null s:7 j:8 + │ ├── key: (4) + │ └── fd: (4)-->(5-8) └── filters - └── x:1 = k:3 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + └── x:1 = k:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] # -------------------------------------------------- # SimplifyJoinNotNullEquality @@ -2495,100 +2495,100 @@ norm expect=SimplifyJoinNotNullEquality SELECT * FROM a INNER JOIN b ON (a.k=b.x) IS True ---- inner-join (hash) - ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:6!null y:7 + ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:7!null y:8 ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (6) - ├── fd: (1)-->(2-5), (6)-->(7), (1)==(6), (6)==(1) + ├── key: (7) + ├── fd: (1)-->(2-5), (7)-->(8), (1)==(7), (7)==(1) ├── scan a │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── scan b - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + │ └── fd: (7)-->(8) └── filters - └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] norm expect=SimplifyJoinNotNullEquality SELECT * FROM a INNER JOIN b ON (a.k=b.x) IS False ---- inner-join (cross) - ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:6!null y:7 - ├── key: (1,6) - ├── fd: (1)-->(2-5), (6)-->(7) + ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:7!null y:8 + ├── key: (1,7) + ├── fd: (1)-->(2-5), (7)-->(8) ├── scan a │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── scan b - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + │ └── fd: (7)-->(8) └── filters - └── k:1 != x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── k:1 != x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] norm expect=SimplifyJoinNotNullEquality SELECT * FROM a INNER JOIN b ON (a.k=b.x) IS Null ---- values - ├── columns: k:1!null i:2!null f:3!null s:4!null j:5!null x:6!null y:7!null + ├── columns: k:1!null i:2!null f:3!null s:4!null j:5!null x:7!null y:8!null ├── cardinality: [0 - 0] ├── key: () - └── fd: ()-->(1-7) + └── fd: ()-->(1-5,7,8) norm expect=SimplifyJoinNotNullEquality SELECT * FROM a INNER JOIN b ON (a.k=b.x) IS NOT True ---- inner-join (cross) - ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:6!null y:7 - ├── key: (1,6) - ├── fd: (1)-->(2-5), (6)-->(7) + ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:7!null y:8 + ├── key: (1,7) + ├── fd: (1)-->(2-5), (7)-->(8) ├── scan a │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── scan b - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + │ └── fd: (7)-->(8) └── filters - └── k:1 != x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── k:1 != x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] norm expect=SimplifyJoinNotNullEquality SELECT * FROM a INNER JOIN b ON (a.k=b.x) IS NOT False ---- inner-join (hash) - ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:6!null y:7 + ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:7!null y:8 ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (6) - ├── fd: (1)-->(2-5), (6)-->(7), (1)==(6), (6)==(1) + ├── key: (7) + ├── fd: (1)-->(2-5), (7)-->(8), (1)==(7), (7)==(1) ├── scan a │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── scan b - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + │ └── fd: (7)-->(8) └── filters - └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] norm expect=SimplifyJoinNotNullEquality SELECT * FROM a INNER JOIN b ON (a.k=b.x) IS NOT Null ---- inner-join (cross) - ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:6!null y:7 - ├── key: (1,6) - ├── fd: (1)-->(2-5), (6)-->(7) + ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:7!null y:8 + ├── key: (1,7) + ├── fd: (1)-->(2-5), (7)-->(8) ├── scan a │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── scan b - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + │ └── fd: (7)-->(8) └── filters (true) # Simply multiple conditions, with other conditions present as well. @@ -2599,11 +2599,11 @@ INNER JOIN (SELECT x, y, y+1 AS z FROM b WHERE y>10) AS b ON a.f>=b.z::float AND (a.k=b.x) IS True AND a.f>=b.z::float AND (a.i=b.y) IS NOT False ---- inner-join (hash) - ├── columns: k:1!null i:2!null f:3!null s:4 j:5 x:6!null y:7!null z:8!null + ├── columns: k:1!null i:2!null f:3!null s:4 j:5 x:7!null y:8!null z:10!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) ├── immutable - ├── key: (6) - ├── fd: (1)-->(2-5), (6)-->(7), (7)-->(8), (1)==(6), (6)==(1), (2)==(7), (7)==(2) + ├── key: (7) + ├── fd: (1)-->(2-5), (7)-->(8), (8)-->(10), (1)==(7), (7)==(1), (2)==(8), (8)==(2) ├── select │ ├── columns: k:1!null i:2!null f:3!null s:4 j:5 │ ├── key: (1) @@ -2615,47 +2615,47 @@ inner-join (hash) │ └── filters │ └── i:2 > 0 [outer=(2), constraints=(/2: [/1 - ]; tight)] ├── project - │ ├── columns: z:8!null x:6!null y:7!null + │ ├── columns: z:10!null x:7!null y:8!null │ ├── immutable - │ ├── key: (6) - │ ├── fd: (6)-->(7), (7)-->(8) + │ ├── key: (7) + │ ├── fd: (7)-->(8), (8)-->(10) │ ├── select - │ │ ├── columns: x:6!null y:7!null - │ │ ├── key: (6) - │ │ ├── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8!null + │ │ ├── key: (7) + │ │ ├── fd: (7)-->(8) │ │ ├── scan b - │ │ │ ├── columns: x:6!null y:7 - │ │ │ ├── key: (6) - │ │ │ └── fd: (6)-->(7) + │ │ │ ├── columns: x:7!null y:8 + │ │ │ ├── key: (7) + │ │ │ └── fd: (7)-->(8) │ │ └── filters - │ │ └── y:7 > 10 [outer=(7), constraints=(/7: [/11 - ]; tight)] + │ │ └── y:8 > 10 [outer=(8), constraints=(/8: [/11 - ]; tight)] │ └── projections - │ └── y:7 + 1 [as=z:8, outer=(7), immutable] + │ └── y:8 + 1 [as=z:10, outer=(8), immutable] └── filters - ├── f:3 >= z:8::FLOAT8 [outer=(3,8), immutable, constraints=(/3: (/NULL - ])] - ├── f:3 >= z:8::FLOAT8 [outer=(3,8), immutable, constraints=(/3: (/NULL - ])] - ├── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── i:2 = y:7 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] + ├── f:3 >= z:10::FLOAT8 [outer=(3,10), immutable, constraints=(/3: (/NULL - ])] + ├── f:3 >= z:10::FLOAT8 [outer=(3,10), immutable, constraints=(/3: (/NULL - ])] + ├── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + └── i:2 = y:8 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] # Don't trigger rule when one of the variables is nullable. norm expect-not=SimplifyJoinNotNullEquality SELECT * FROM a INNER JOIN b ON (a.k=b.y) IS True AND (a.i=b.x) IS False ---- inner-join (cross) - ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:6!null y:7 - ├── key: (1,6) - ├── fd: (1)-->(2-5), (6)-->(7) + ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:7!null y:8 + ├── key: (1,7) + ├── fd: (1)-->(2-5), (7)-->(8) ├── scan a │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── scan b - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + │ └── fd: (7)-->(8) └── filters - ├── (k:1 = y:7) IS true [outer=(1,7)] - └── (i:2 = x:6) IS false [outer=(2,6)] + ├── (k:1 = y:8) IS true [outer=(1,8)] + └── (i:2 = x:7) IS false [outer=(2,7)] # -------------------------------------------------- # ExtractJoinEqualities @@ -2665,282 +2665,282 @@ norm expect=ExtractJoinEqualities SELECT * FROM xy JOIN uv ON x+y=u ---- project - ├── columns: x:1!null y:2 u:3!null v:4 + ├── columns: x:1!null y:2 u:4!null v:5 ├── immutable ├── key: (1) - ├── fd: (1)-->(2-4), (3)-->(4) + ├── fd: (1)-->(2,4,5), (4)-->(5) └── inner-join (hash) - ├── columns: x:1!null y:2 u:3!null v:4 column5:5!null + ├── columns: x:1!null y:2 u:4!null v:5 column7:7!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) ├── immutable ├── key: (1) - ├── fd: (1)-->(2,5), (3)-->(4), (3)==(5), (5)==(3) + ├── fd: (1)-->(2,7), (4)-->(5), (4)==(7), (7)==(4) ├── project - │ ├── columns: column5:5 x:1!null y:2 + │ ├── columns: column7:7 x:1!null y:2 │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(2,5) + │ ├── fd: (1)-->(2,7) │ ├── scan xy │ │ ├── columns: x:1!null y:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ └── projections - │ └── x:1 + y:2 [as=column5:5, outer=(1,2), immutable] + │ └── x:1 + y:2 [as=column7:7, outer=(1,2), immutable] ├── scan uv - │ ├── columns: u:3!null v:4 - │ ├── key: (3) - │ └── fd: (3)-->(4) + │ ├── columns: u:4!null v:5 + │ ├── key: (4) + │ └── fd: (4)-->(5) └── filters - └── column5:5 = u:3 [outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ]), fd=(3)==(5), (5)==(3)] + └── column7:7 = u:4 [outer=(4,7), constraints=(/4: (/NULL - ]; /7: (/NULL - ]), fd=(4)==(7), (7)==(4)] norm expect=ExtractJoinEqualities SELECT * FROM xy JOIN uv ON u=x+y ---- project - ├── columns: x:1!null y:2 u:3!null v:4 + ├── columns: x:1!null y:2 u:4!null v:5 ├── immutable ├── key: (1) - ├── fd: (1)-->(2-4), (3)-->(4) + ├── fd: (1)-->(2,4,5), (4)-->(5) └── inner-join (hash) - ├── columns: x:1!null y:2 u:3!null v:4 column5:5!null + ├── columns: x:1!null y:2 u:4!null v:5 column7:7!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) ├── immutable ├── key: (1) - ├── fd: (1)-->(2,5), (3)-->(4), (3)==(5), (5)==(3) + ├── fd: (1)-->(2,7), (4)-->(5), (4)==(7), (7)==(4) ├── project - │ ├── columns: column5:5 x:1!null y:2 + │ ├── columns: column7:7 x:1!null y:2 │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(2,5) + │ ├── fd: (1)-->(2,7) │ ├── scan xy │ │ ├── columns: x:1!null y:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ └── projections - │ └── x:1 + y:2 [as=column5:5, outer=(1,2), immutable] + │ └── x:1 + y:2 [as=column7:7, outer=(1,2), immutable] ├── scan uv - │ ├── columns: u:3!null v:4 - │ ├── key: (3) - │ └── fd: (3)-->(4) + │ ├── columns: u:4!null v:5 + │ ├── key: (4) + │ └── fd: (4)-->(5) └── filters - └── column5:5 = u:3 [outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ]), fd=(3)==(5), (5)==(3)] + └── column7:7 = u:4 [outer=(4,7), constraints=(/4: (/NULL - ]; /7: (/NULL - ]), fd=(4)==(7), (7)==(4)] norm expect=ExtractJoinEqualities SELECT * FROM xy JOIN uv ON x=u+v ---- project - ├── columns: x:1!null y:2 u:3!null v:4 + ├── columns: x:1!null y:2 u:4!null v:5 ├── immutable - ├── key: (3) - ├── fd: (1)-->(2), (3)-->(1,2,4) + ├── key: (4) + ├── fd: (1)-->(2), (4)-->(1,2,5) └── inner-join (hash) - ├── columns: x:1!null y:2 u:3!null v:4 column5:5!null + ├── columns: x:1!null y:2 u:4!null v:5 column7:7!null ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) ├── immutable - ├── key: (3) - ├── fd: (1)-->(2), (3)-->(4,5), (1)==(5), (5)==(1) + ├── key: (4) + ├── fd: (1)-->(2), (4)-->(5,7), (1)==(7), (7)==(1) ├── scan xy │ ├── columns: x:1!null y:2 │ ├── key: (1) │ └── fd: (1)-->(2) ├── project - │ ├── columns: column5:5 u:3!null v:4 + │ ├── columns: column7:7 u:4!null v:5 │ ├── immutable - │ ├── key: (3) - │ ├── fd: (3)-->(4,5) + │ ├── key: (4) + │ ├── fd: (4)-->(5,7) │ ├── scan uv - │ │ ├── columns: u:3!null v:4 - │ │ ├── key: (3) - │ │ └── fd: (3)-->(4) + │ │ ├── columns: u:4!null v:5 + │ │ ├── key: (4) + │ │ └── fd: (4)-->(5) │ └── projections - │ └── u:3 + v:4 [as=column5:5, outer=(3,4), immutable] + │ └── u:4 + v:5 [as=column7:7, outer=(4,5), immutable] └── filters - └── x:1 = column5:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + └── x:1 = column7:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] norm expect=ExtractJoinEqualities SELECT * FROM xy JOIN uv ON u+v=x ---- project - ├── columns: x:1!null y:2 u:3!null v:4 + ├── columns: x:1!null y:2 u:4!null v:5 ├── immutable - ├── key: (3) - ├── fd: (1)-->(2), (3)-->(1,2,4) + ├── key: (4) + ├── fd: (1)-->(2), (4)-->(1,2,5) └── inner-join (hash) - ├── columns: x:1!null y:2 u:3!null v:4 column5:5!null + ├── columns: x:1!null y:2 u:4!null v:5 column7:7!null ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) ├── immutable - ├── key: (3) - ├── fd: (1)-->(2), (3)-->(4,5), (1)==(5), (5)==(1) + ├── key: (4) + ├── fd: (1)-->(2), (4)-->(5,7), (1)==(7), (7)==(1) ├── scan xy │ ├── columns: x:1!null y:2 │ ├── key: (1) │ └── fd: (1)-->(2) ├── project - │ ├── columns: column5:5 u:3!null v:4 + │ ├── columns: column7:7 u:4!null v:5 │ ├── immutable - │ ├── key: (3) - │ ├── fd: (3)-->(4,5) + │ ├── key: (4) + │ ├── fd: (4)-->(5,7) │ ├── scan uv - │ │ ├── columns: u:3!null v:4 - │ │ ├── key: (3) - │ │ └── fd: (3)-->(4) + │ │ ├── columns: u:4!null v:5 + │ │ ├── key: (4) + │ │ └── fd: (4)-->(5) │ └── projections - │ └── u:3 + v:4 [as=column5:5, outer=(3,4), immutable] + │ └── u:4 + v:5 [as=column7:7, outer=(4,5), immutable] └── filters - └── x:1 = column5:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + └── x:1 = column7:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] norm expect=ExtractJoinEqualities SELECT * FROM xy JOIN uv ON x+y=u+v ---- project - ├── columns: x:1!null y:2 u:3!null v:4 + ├── columns: x:1!null y:2 u:4!null v:5 ├── immutable - ├── key: (1,3) - ├── fd: (1)-->(2), (3)-->(4) + ├── key: (1,4) + ├── fd: (1)-->(2), (4)-->(5) └── inner-join (hash) - ├── columns: x:1!null y:2 u:3!null v:4 column5:5!null column6:6!null + ├── columns: x:1!null y:2 u:4!null v:5 column7:7!null column8:8!null ├── immutable - ├── key: (1,3) - ├── fd: (1)-->(2,5), (3)-->(4,6), (5)==(6), (6)==(5) + ├── key: (1,4) + ├── fd: (1)-->(2,7), (4)-->(5,8), (7)==(8), (8)==(7) ├── project - │ ├── columns: column5:5 x:1!null y:2 + │ ├── columns: column7:7 x:1!null y:2 │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(2,5) + │ ├── fd: (1)-->(2,7) │ ├── scan xy │ │ ├── columns: x:1!null y:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ └── projections - │ └── x:1 + y:2 [as=column5:5, outer=(1,2), immutable] + │ └── x:1 + y:2 [as=column7:7, outer=(1,2), immutable] ├── project - │ ├── columns: column6:6 u:3!null v:4 + │ ├── columns: column8:8 u:4!null v:5 │ ├── immutable - │ ├── key: (3) - │ ├── fd: (3)-->(4,6) + │ ├── key: (4) + │ ├── fd: (4)-->(5,8) │ ├── scan uv - │ │ ├── columns: u:3!null v:4 - │ │ ├── key: (3) - │ │ └── fd: (3)-->(4) + │ │ ├── columns: u:4!null v:5 + │ │ ├── key: (4) + │ │ └── fd: (4)-->(5) │ └── projections - │ └── u:3 + v:4 [as=column6:6, outer=(3,4), immutable] + │ └── u:4 + v:5 [as=column8:8, outer=(4,5), immutable] └── filters - └── column5:5 = column6:6 [outer=(5,6), constraints=(/5: (/NULL - ]; /6: (/NULL - ]), fd=(5)==(6), (6)==(5)] + └── column7:7 = column8:8 [outer=(7,8), constraints=(/7: (/NULL - ]; /8: (/NULL - ]), fd=(7)==(8), (8)==(7)] # Multiple extractable equalities. norm expect=ExtractJoinEqualities SELECT * FROM xy JOIN uv ON x+y=u AND x=u+v AND x*y+1=u*v+2 ---- project - ├── columns: x:1!null y:2 u:3!null v:4 + ├── columns: x:1!null y:2 u:4!null v:5 ├── immutable ├── key: (1) - ├── fd: (1)-->(2-4), (3)-->(1,2,4) + ├── fd: (1)-->(2,4,5), (4)-->(1,2,5) └── inner-join (hash) - ├── columns: x:1!null y:2 u:3!null v:4 column5:5!null column6:6!null column7:7!null column8:8!null + ├── columns: x:1!null y:2 u:4!null v:5 column7:7!null column8:8!null column9:9!null column10:10!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) ├── immutable ├── key: (1) - ├── fd: (1)-->(2,5,7), (3)-->(4,6,8), (3)==(5), (5)==(3), (1)==(6), (6)==(1), (7)==(8), (8)==(7) + ├── fd: (1)-->(2,7,9), (4)-->(5,8,10), (4)==(7), (7)==(4), (1)==(8), (8)==(1), (9)==(10), (10)==(9) ├── project - │ ├── columns: column7:7 column5:5 x:1!null y:2 + │ ├── columns: column9:9 column7:7 x:1!null y:2 │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(2,5,7) + │ ├── fd: (1)-->(2,7,9) │ ├── scan xy │ │ ├── columns: x:1!null y:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ └── projections - │ ├── (x:1 * y:2) + 1 [as=column7:7, outer=(1,2), immutable] - │ └── x:1 + y:2 [as=column5:5, outer=(1,2), immutable] + │ ├── (x:1 * y:2) + 1 [as=column9:9, outer=(1,2), immutable] + │ └── x:1 + y:2 [as=column7:7, outer=(1,2), immutable] ├── project - │ ├── columns: column8:8 column6:6 u:3!null v:4 + │ ├── columns: column10:10 column8:8 u:4!null v:5 │ ├── immutable - │ ├── key: (3) - │ ├── fd: (3)-->(4,6,8) + │ ├── key: (4) + │ ├── fd: (4)-->(5,8,10) │ ├── scan uv - │ │ ├── columns: u:3!null v:4 - │ │ ├── key: (3) - │ │ └── fd: (3)-->(4) + │ │ ├── columns: u:4!null v:5 + │ │ ├── key: (4) + │ │ └── fd: (4)-->(5) │ └── projections - │ ├── (u:3 * v:4) + 2 [as=column8:8, outer=(3,4), immutable] - │ └── u:3 + v:4 [as=column6:6, outer=(3,4), immutable] + │ ├── (u:4 * v:5) + 2 [as=column10:10, outer=(4,5), immutable] + │ └── u:4 + v:5 [as=column8:8, outer=(4,5), immutable] └── filters - ├── column5:5 = u:3 [outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ]), fd=(3)==(5), (5)==(3)] - ├── x:1 = column6:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── column7:7 = column8:8 [outer=(7,8), constraints=(/7: (/NULL - ]; /8: (/NULL - ]), fd=(7)==(8), (8)==(7)] + ├── column7:7 = u:4 [outer=(4,7), constraints=(/4: (/NULL - ]; /7: (/NULL - ]), fd=(4)==(7), (7)==(4)] + ├── x:1 = column8:8 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] + └── column9:9 = column10:10 [outer=(9,10), constraints=(/9: (/NULL - ]; /10: (/NULL - ]), fd=(9)==(10), (10)==(9)] # An extractable equality with another expression. norm expect=ExtractJoinEqualities SELECT * FROM xy JOIN uv ON x+y=u AND x+u=v ---- project - ├── columns: x:1!null y:2 u:3!null v:4!null + ├── columns: x:1!null y:2 u:4!null v:5!null ├── immutable ├── key: (1) - ├── fd: (1)-->(2-4), (3)-->(4) + ├── fd: (1)-->(2,4,5), (4)-->(5) └── inner-join (hash) - ├── columns: x:1!null y:2 u:3!null v:4!null column5:5!null + ├── columns: x:1!null y:2 u:4!null v:5!null column7:7!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) ├── immutable ├── key: (1) - ├── fd: (1)-->(2,5), (3)-->(4), (3)==(5), (5)==(3) + ├── fd: (1)-->(2,7), (4)-->(5), (4)==(7), (7)==(4) ├── project - │ ├── columns: column5:5 x:1!null y:2 + │ ├── columns: column7:7 x:1!null y:2 │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(2,5) + │ ├── fd: (1)-->(2,7) │ ├── scan xy │ │ ├── columns: x:1!null y:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ └── projections - │ └── x:1 + y:2 [as=column5:5, outer=(1,2), immutable] + │ └── x:1 + y:2 [as=column7:7, outer=(1,2), immutable] ├── scan uv - │ ├── columns: u:3!null v:4 - │ ├── key: (3) - │ └── fd: (3)-->(4) + │ ├── columns: u:4!null v:5 + │ ├── key: (4) + │ └── fd: (4)-->(5) └── filters - ├── v:4 = (x:1 + u:3) [outer=(1,3,4), immutable, constraints=(/4: (/NULL - ])] - └── column5:5 = u:3 [outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ]), fd=(3)==(5), (5)==(3)] + ├── v:5 = (x:1 + u:4) [outer=(1,4,5), immutable, constraints=(/5: (/NULL - ])] + └── column7:7 = u:4 [outer=(4,7), constraints=(/4: (/NULL - ]; /7: (/NULL - ]), fd=(4)==(7), (7)==(4)] # Cases with non-extractable equality. norm expect-not=ExtractJoinEqualities SELECT * FROM xy FULL OUTER JOIN uv ON x=u ---- full-join (hash) - ├── columns: x:1 y:2 u:3 v:4 + ├── columns: x:1 y:2 u:4 v:5 ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) - ├── key: (1,3) - ├── fd: (1)-->(2), (3)-->(4) + ├── key: (1,4) + ├── fd: (1)-->(2), (4)-->(5) ├── scan xy │ ├── columns: x:1!null y:2 │ ├── key: (1) │ └── fd: (1)-->(2) ├── scan uv - │ ├── columns: u:3!null v:4 - │ ├── key: (3) - │ └── fd: (3)-->(4) + │ ├── columns: u:4!null v:5 + │ ├── key: (4) + │ └── fd: (4)-->(5) └── filters - └── x:1 = u:3 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + └── x:1 = u:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] norm expect-not=ExtractJoinEqualities SELECT * FROM xy FULL OUTER JOIN uv ON x+y=1 ---- full-join (cross) - ├── columns: x:1 y:2 u:3 v:4 + ├── columns: x:1 y:2 u:4 v:5 ├── immutable - ├── key: (1,3) - ├── fd: (1)-->(2), (3)-->(4) + ├── key: (1,4) + ├── fd: (1)-->(2), (4)-->(5) ├── scan xy │ ├── columns: x:1!null y:2 │ ├── key: (1) │ └── fd: (1)-->(2) ├── scan uv - │ ├── columns: u:3!null v:4 - │ ├── key: (3) - │ └── fd: (3)-->(4) + │ ├── columns: u:4!null v:5 + │ ├── key: (4) + │ └── fd: (4)-->(5) └── filters └── (x:1 + y:2) = 1 [outer=(1,2), immutable] @@ -2948,110 +2948,110 @@ norm expect-not=ExtractJoinEqualities SELECT * FROM xy FULL OUTER JOIN uv ON 1=u+v ---- full-join (cross) - ├── columns: x:1 y:2 u:3 v:4 + ├── columns: x:1 y:2 u:4 v:5 ├── immutable - ├── key: (1,3) - ├── fd: (1)-->(2), (3)-->(4) + ├── key: (1,4) + ├── fd: (1)-->(2), (4)-->(5) ├── scan xy │ ├── columns: x:1!null y:2 │ ├── key: (1) │ └── fd: (1)-->(2) ├── scan uv - │ ├── columns: u:3!null v:4 - │ ├── key: (3) - │ └── fd: (3)-->(4) + │ ├── columns: u:4!null v:5 + │ ├── key: (4) + │ └── fd: (4)-->(5) └── filters - └── (u:3 + v:4) = 1 [outer=(3,4), immutable] + └── (u:4 + v:5) = 1 [outer=(4,5), immutable] norm expect-not=ExtractJoinEqualities SELECT * FROM xy INNER JOIN uv ON (SELECT k FROM a WHERE i=x)=u ---- project - ├── columns: x:1!null y:2 u:3!null v:4 - ├── key: (1,3) - ├── fd: (1)-->(2), (1,3)-->(4) + ├── columns: x:1!null y:2 u:4!null v:5 + ├── key: (1,4) + ├── fd: (1)-->(2), (1,4)-->(5) └── inner-join-apply - ├── columns: x:1!null y:2 u:3!null v:4 k:5 - ├── key: (1,3) - ├── fd: (1)-->(2), (1,3)-->(4,5), (3)==(5), (5)==(3) + ├── columns: x:1!null y:2 u:4!null v:5 k:7 + ├── key: (1,4) + ├── fd: (1)-->(2), (1,4)-->(5,7), (4)==(7), (7)==(4) ├── scan xy │ ├── columns: x:1!null y:2 │ ├── key: (1) │ └── fd: (1)-->(2) ├── ensure-distinct-on - │ ├── columns: u:3!null v:4 k:5 - │ ├── grouping columns: u:3!null + │ ├── columns: u:4!null v:5 k:7 + │ ├── grouping columns: u:4!null │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── outer: (1) - │ ├── key: (3) - │ ├── fd: (3)-->(4,5) + │ ├── key: (4) + │ ├── fd: (4)-->(5,7) │ ├── left-join (cross) - │ │ ├── columns: u:3!null v:4 k:5 i:6 + │ │ ├── columns: u:4!null v:5 k:7 i:8 │ │ ├── outer: (1) - │ │ ├── key: (3,5) - │ │ ├── fd: (3)-->(4), (5)-->(6) + │ │ ├── key: (4,7) + │ │ ├── fd: (4)-->(5), (7)-->(8) │ │ ├── scan uv - │ │ │ ├── columns: u:3!null v:4 - │ │ │ ├── key: (3) - │ │ │ └── fd: (3)-->(4) + │ │ │ ├── columns: u:4!null v:5 + │ │ │ ├── key: (4) + │ │ │ └── fd: (4)-->(5) │ │ ├── scan a - │ │ │ ├── columns: k:5!null i:6 - │ │ │ ├── key: (5) - │ │ │ └── fd: (5)-->(6) + │ │ │ ├── columns: k:7!null i:8 + │ │ │ ├── key: (7) + │ │ │ └── fd: (7)-->(8) │ │ └── filters - │ │ └── i:6 = x:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ │ └── i:8 = x:1 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] │ └── aggregations - │ ├── const-agg [as=v:4, outer=(4)] - │ │ └── v:4 - │ └── const-agg [as=k:5, outer=(5)] - │ └── k:5 + │ ├── const-agg [as=v:5, outer=(5)] + │ │ └── v:5 + │ └── const-agg [as=k:7, outer=(7)] + │ └── k:7 └── filters - └── u:3 = k:5 [outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ]), fd=(3)==(5), (5)==(3)] + └── u:4 = k:7 [outer=(4,7), constraints=(/4: (/NULL - ]; /7: (/NULL - ]), fd=(4)==(7), (7)==(4)] norm expect-not=ExtractJoinEqualities SELECT * FROM xy INNER JOIN uv ON x=(SELECT k FROM a WHERE i=u) ---- project - ├── columns: x:1!null y:2 u:3!null v:4 - ├── key: (3) - ├── fd: (1)-->(2), (3)-->(1,2,4) + ├── columns: x:1!null y:2 u:4!null v:5 + ├── key: (4) + ├── fd: (1)-->(2), (4)-->(1,2,5) └── inner-join (hash) - ├── columns: x:1!null y:2 u:3!null v:4 k:5!null + ├── columns: x:1!null y:2 u:4!null v:5 k:7!null ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) - ├── key: (3) - ├── fd: (1)-->(2), (3)-->(4,5), (1)==(5), (5)==(1) + ├── key: (4) + ├── fd: (1)-->(2), (4)-->(5,7), (1)==(7), (7)==(1) ├── scan xy │ ├── columns: x:1!null y:2 │ ├── key: (1) │ └── fd: (1)-->(2) ├── ensure-distinct-on - │ ├── columns: u:3!null v:4 k:5 - │ ├── grouping columns: u:3!null + │ ├── columns: u:4!null v:5 k:7 + │ ├── grouping columns: u:4!null │ ├── error: "more than one row returned by a subquery used as an expression" - │ ├── key: (3) - │ ├── fd: (3)-->(4,5) + │ ├── key: (4) + │ ├── fd: (4)-->(5,7) │ ├── left-join (hash) - │ │ ├── columns: u:3!null v:4 k:5 i:6 + │ │ ├── columns: u:4!null v:5 k:7 i:8 │ │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) - │ │ ├── key: (3,5) - │ │ ├── fd: (3)-->(4), (5)-->(6) + │ │ ├── key: (4,7) + │ │ ├── fd: (4)-->(5), (7)-->(8) │ │ ├── scan uv - │ │ │ ├── columns: u:3!null v:4 - │ │ │ ├── key: (3) - │ │ │ └── fd: (3)-->(4) + │ │ │ ├── columns: u:4!null v:5 + │ │ │ ├── key: (4) + │ │ │ └── fd: (4)-->(5) │ │ ├── scan a - │ │ │ ├── columns: k:5!null i:6 - │ │ │ ├── key: (5) - │ │ │ └── fd: (5)-->(6) + │ │ │ ├── columns: k:7!null i:8 + │ │ │ ├── key: (7) + │ │ │ └── fd: (7)-->(8) │ │ └── filters - │ │ └── i:6 = u:3 [outer=(3,6), constraints=(/3: (/NULL - ]; /6: (/NULL - ]), fd=(3)==(6), (6)==(3)] + │ │ └── i:8 = u:4 [outer=(4,8), constraints=(/4: (/NULL - ]; /8: (/NULL - ]), fd=(4)==(8), (8)==(4)] │ └── aggregations - │ ├── const-agg [as=v:4, outer=(4)] - │ │ └── v:4 - │ └── const-agg [as=k:5, outer=(5)] - │ └── k:5 + │ ├── const-agg [as=v:5, outer=(5)] + │ │ └── v:5 + │ └── const-agg [as=k:7, outer=(7)] + │ └── k:7 └── filters - └── x:1 = k:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + └── x:1 = k:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # Don't extract equalities where one side is an expression with no outer cols # (#44746). This is a rare case where we can't constant fold because the @@ -3060,20 +3060,20 @@ norm expect-not=ExtractJoinEqualities SELECT * FROM xy FULL JOIN uv ON (substring('', ')') = '') = (u > 0) ---- full-join (cross) - ├── columns: x:1 y:2 u:3 v:4 + ├── columns: x:1 y:2 u:4 v:5 ├── immutable - ├── key: (1,3) - ├── fd: (1)-->(2), (3)-->(4) + ├── key: (1,4) + ├── fd: (1)-->(2), (4)-->(5) ├── scan xy │ ├── columns: x:1!null y:2 │ ├── key: (1) │ └── fd: (1)-->(2) ├── scan uv - │ ├── columns: u:3!null v:4 - │ ├── key: (3) - │ └── fd: (3)-->(4) + │ ├── columns: u:4!null v:5 + │ ├── key: (4) + │ └── fd: (4)-->(5) └── filters - └── (substring('', ')') = '') = (u:3 > 0) [outer=(3), immutable] + └── (substring('', ')') = '') = (u:4 > 0) [outer=(4), immutable] # -------------------------------------------------- # LeftAssociateJoinsLeft, LeftAssociateJoinsRight, @@ -3085,31 +3085,31 @@ norm expect=RightAssociateJoinsLeft SELECT * FROM xy, a, uv WHERE x=u AND y=k ---- inner-join (hash) - ├── columns: x:1!null y:2!null k:3!null i:4 f:5!null s:6 j:7 u:8!null v:9 + ├── columns: x:1!null y:2!null k:4!null i:5 f:6!null s:7 j:8 u:10!null v:11 ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (8) - ├── fd: (1)-->(2), (3)-->(4-7), (2)==(3), (3)==(2), (8)-->(9), (1)==(8), (8)==(1) + ├── key: (10) + ├── fd: (1)-->(2), (4)-->(5-8), (2)==(4), (4)==(2), (10)-->(11), (1)==(10), (10)==(1) ├── inner-join (hash) - │ ├── columns: x:1!null y:2!null k:3!null i:4 f:5!null s:6 j:7 + │ ├── columns: x:1!null y:2!null k:4!null i:5 f:6!null s:7 j:8 │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) │ ├── key: (1) - │ ├── fd: (1)-->(2), (3)-->(4-7), (2)==(3), (3)==(2) + │ ├── fd: (1)-->(2), (4)-->(5-8), (2)==(4), (4)==(2) │ ├── scan xy │ │ ├── columns: x:1!null y:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ ├── scan a - │ │ ├── columns: k:3!null i:4 f:5!null s:6 j:7 - │ │ ├── key: (3) - │ │ └── fd: (3)-->(4-7) + │ │ ├── columns: k:4!null i:5 f:6!null s:7 j:8 + │ │ ├── key: (4) + │ │ └── fd: (4)-->(5-8) │ └── filters - │ └── y:2 = k:3 [outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ]), fd=(2)==(3), (3)==(2)] + │ └── y:2 = k:4 [outer=(2,4), constraints=(/2: (/NULL - ]; /4: (/NULL - ]), fd=(2)==(4), (4)==(2)] ├── scan uv - │ ├── columns: u:8!null v:9 - │ ├── key: (8) - │ └── fd: (8)-->(9) + │ ├── columns: u:10!null v:11 + │ ├── key: (10) + │ └── fd: (10)-->(11) └── filters - └── x:1 = u:8 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] + └── x:1 = u:10 [outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] # LeftAssociateJoinsLeft case with one association operation. norm expect=LeftAssociateJoinsLeft format=hide-all diff --git a/pkg/sql/opt/norm/testdata/rules/limit b/pkg/sql/opt/norm/testdata/rules/limit index 07f2c7385d18..538282627794 100644 --- a/pkg/sql/opt/norm/testdata/rules/limit +++ b/pkg/sql/opt/norm/testdata/rules/limit @@ -162,11 +162,11 @@ norm expect=PushLimitIntoProject SELECT k, f*2.0 AS r FROM a LIMIT 5 ---- project - ├── columns: k:1!null r:6 + ├── columns: k:1!null r:7 ├── cardinality: [0 - 5] ├── immutable ├── key: (1) - ├── fd: (1)-->(6) + ├── fd: (1)-->(7) ├── limit │ ├── columns: k:1!null f:3 │ ├── cardinality: [0 - 5] @@ -179,17 +179,17 @@ project │ │ └── limit hint: 5.00 │ └── 5 └── projections - └── f:3 * 2.0 [as=r:6, outer=(3), immutable] + └── f:3 * 2.0 [as=r:7, outer=(3), immutable] norm expect=PushLimitIntoProject SELECT k, f*2.0 AS r FROM a ORDER BY k LIMIT 5 ---- project - ├── columns: k:1!null r:6 + ├── columns: k:1!null r:7 ├── cardinality: [0 - 5] ├── immutable ├── key: (1) - ├── fd: (1)-->(6) + ├── fd: (1)-->(7) ├── ordering: +1 ├── limit │ ├── columns: k:1!null f:3 @@ -206,7 +206,7 @@ project │ │ └── limit hint: 5.00 │ └── 5 └── projections - └── f:3 * 2.0 [as=r:6, outer=(3), immutable] + └── f:3 * 2.0 [as=r:7, outer=(3), immutable] # Don't push the limit through project when the ordering is on a # synthesized column. @@ -214,31 +214,31 @@ norm expect-not=PushLimitIntoProject SELECT k, f*2.0 AS r FROM a ORDER BY r LIMIT 5 ---- limit - ├── columns: k:1!null r:6 - ├── internal-ordering: +6 + ├── columns: k:1!null r:7 + ├── internal-ordering: +7 ├── cardinality: [0 - 5] ├── immutable ├── key: (1) - ├── fd: (1)-->(6) - ├── ordering: +6 + ├── fd: (1)-->(7) + ├── ordering: +7 ├── sort - │ ├── columns: k:1!null r:6 + │ ├── columns: k:1!null r:7 │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(6) - │ ├── ordering: +6 + │ ├── fd: (1)-->(7) + │ ├── ordering: +7 │ ├── limit hint: 5.00 │ └── project - │ ├── columns: r:6 k:1!null + │ ├── columns: r:7 k:1!null │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(6) + │ ├── fd: (1)-->(7) │ ├── scan a │ │ ├── columns: k:1!null f:3 │ │ ├── key: (1) │ │ └── fd: (1)-->(3) │ └── projections - │ └── f:3 * 2.0 [as=r:6, outer=(3), immutable] + │ └── f:3 * 2.0 [as=r:7, outer=(3), immutable] └── 5 @@ -247,7 +247,7 @@ norm SELECT f, f+1.1 AS r FROM (SELECT f, i FROM a GROUP BY f, i) a ORDER BY f LIMIT 5 ---- project - ├── columns: f:3 r:6 + ├── columns: f:3 r:7 ├── cardinality: [0 - 5] ├── immutable ├── ordering: +3 @@ -271,7 +271,7 @@ project │ │ └── columns: i:2 f:3 │ └── 5 └── projections - └── f:3 + 1.1 [as=r:6, outer=(3), immutable] + └── f:3 + 1.1 [as=r:7, outer=(3), immutable] # Don't push negative limit into Scan. norm @@ -297,10 +297,10 @@ norm expect=PushOffsetIntoProject SELECT k, f*2.0 AS r FROM a OFFSET 5 ---- project - ├── columns: k:1!null r:6 + ├── columns: k:1!null r:7 ├── immutable ├── key: (1) - ├── fd: (1)-->(6) + ├── fd: (1)-->(7) ├── offset │ ├── columns: k:1!null f:3 │ ├── key: (1) @@ -311,16 +311,16 @@ project │ │ └── fd: (1)-->(3) │ └── 5 └── projections - └── f:3 * 2.0 [as=r:6, outer=(3), immutable] + └── f:3 * 2.0 [as=r:7, outer=(3), immutable] norm expect=PushOffsetIntoProject SELECT k, f*2.0 AS r FROM a ORDER BY k OFFSET 5 ---- project - ├── columns: k:1!null r:6 + ├── columns: k:1!null r:7 ├── immutable ├── key: (1) - ├── fd: (1)-->(6) + ├── fd: (1)-->(7) ├── ordering: +1 ├── offset │ ├── columns: k:1!null f:3 @@ -335,7 +335,7 @@ project │ │ └── ordering: +1 │ └── 5 └── projections - └── f:3 * 2.0 [as=r:6, outer=(3), immutable] + └── f:3 * 2.0 [as=r:7, outer=(3), immutable] # Don't push the offset through project when the ordering is on a # synthesized column. @@ -343,29 +343,29 @@ norm expect-not=PushOffsetIntoProject SELECT k, f*2.0 AS r FROM a ORDER BY r OFFSET 5 ---- offset - ├── columns: k:1!null r:6 - ├── internal-ordering: +6 + ├── columns: k:1!null r:7 + ├── internal-ordering: +7 ├── immutable ├── key: (1) - ├── fd: (1)-->(6) - ├── ordering: +6 + ├── fd: (1)-->(7) + ├── ordering: +7 ├── sort - │ ├── columns: k:1!null r:6 + │ ├── columns: k:1!null r:7 │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(6) - │ ├── ordering: +6 + │ ├── fd: (1)-->(7) + │ ├── ordering: +7 │ └── project - │ ├── columns: r:6 k:1!null + │ ├── columns: r:7 k:1!null │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(6) + │ ├── fd: (1)-->(7) │ ├── scan a │ │ ├── columns: k:1!null f:3 │ │ ├── key: (1) │ │ └── fd: (1)-->(3) │ └── projections - │ └── f:3 * 2.0 [as=r:6, outer=(3), immutable] + │ └── f:3 * 2.0 [as=r:7, outer=(3), immutable] └── 5 # Detect PushOffsetIntoProject and FilterUnusedOffsetCols dependency cycle. @@ -373,7 +373,7 @@ norm SELECT f, f+1.1 AS r FROM (SELECT f, i FROM a GROUP BY f, i) a ORDER BY f OFFSET 5 ---- project - ├── columns: f:3 r:6 + ├── columns: f:3 r:7 ├── immutable ├── ordering: +3 ├── offset @@ -393,7 +393,7 @@ project │ │ └── columns: i:2 f:3 │ └── 5 └── projections - └── f:3 + 1.1 [as=r:6, outer=(3), immutable] + └── f:3 + 1.1 [as=r:7, outer=(3), immutable] # -------------------------------------------------- # PushLimitIntoProject + PushOffsetIntoProject @@ -402,11 +402,11 @@ norm expect=(PushLimitIntoProject,PushOffsetIntoProject) SELECT k, f*2.0 AS r FROM a OFFSET 5 LIMIT 10 ---- project - ├── columns: k:1!null r:6 + ├── columns: k:1!null r:7 ├── cardinality: [0 - 10] ├── immutable ├── key: (1) - ├── fd: (1)-->(6) + ├── fd: (1)-->(7) ├── offset │ ├── columns: k:1!null f:3 │ ├── cardinality: [0 - 10] @@ -425,13 +425,13 @@ project │ │ └── 15 │ └── 5 └── projections - └── f:3 * 2.0 [as=r:6, outer=(3), immutable] + └── f:3 * 2.0 [as=r:7, outer=(3), immutable] norm expect=(PushLimitIntoProject,PushOffsetIntoProject) SELECT f, f+1.1 AS r FROM (SELECT f, i FROM a GROUP BY f, i) a ORDER BY f OFFSET 5 LIMIT 10 ---- project - ├── columns: f:3 r:6 + ├── columns: f:3 r:7 ├── cardinality: [0 - 10] ├── immutable ├── ordering: +3 @@ -462,7 +462,7 @@ project │ │ └── 15 │ └── 5 └── projections - └── f:3 + 1.1 [as=r:6, outer=(3), immutable] + └── f:3 + 1.1 [as=r:7, outer=(3), immutable] # -------------------------------------------------- # PushLimitIntoOffset @@ -599,10 +599,10 @@ norm expect=PushLimitIntoOrdinality SELECT * FROM (SELECT * FROM a ORDER BY k) WITH ORDINALITY LIMIT 10 ---- ordinality - ├── columns: k:1!null i:2 f:3 s:4 j:5 ordinality:6!null + ├── columns: k:1!null i:2 f:3 s:4 j:5 ordinality:7!null ├── cardinality: [0 - 10] ├── key: (1) - ├── fd: (1)-->(2-6), (6)-->(1-5) + ├── fd: (1)-->(2-5,7), (7)-->(1-5) └── limit ├── columns: k:1!null i:2 f:3 s:4 j:5 ├── internal-ordering: +1 @@ -622,16 +622,16 @@ norm expect=PushLimitIntoOrdinality SELECT * FROM a WITH ORDINALITY ORDER BY k LIMIT 10 ---- sort - ├── columns: k:1!null i:2 f:3 s:4 j:5 ordinality:6!null + ├── columns: k:1!null i:2 f:3 s:4 j:5 ordinality:7!null ├── cardinality: [0 - 10] ├── key: (1) - ├── fd: (1)-->(2-6), (6)-->(1-5) + ├── fd: (1)-->(2-5,7), (7)-->(1-5) ├── ordering: +1 └── ordinality - ├── columns: k:1!null i:2 f:3 s:4 j:5 ordinality:6!null + ├── columns: k:1!null i:2 f:3 s:4 j:5 ordinality:7!null ├── cardinality: [0 - 10] ├── key: (1) - ├── fd: (1)-->(2-6), (6)-->(1-5) + ├── fd: (1)-->(2-5,7), (7)-->(1-5) └── limit ├── columns: k:1!null i:2 f:3 s:4 j:5 ├── internal-ordering: +1 @@ -653,10 +653,10 @@ norm expect=PushLimitIntoOrdinality SELECT * FROM (SELECT * FROM a WHERE i=f ORDER BY i, s) WITH ORDINALITY ORDER BY f LIMIT 10 ---- ordinality - ├── columns: k:1!null i:2!null f:3!null s:4 j:5 ordinality:6!null + ├── columns: k:1!null i:2!null f:3!null s:4 j:5 ordinality:7!null ├── cardinality: [0 - 10] ├── key: (1) - ├── fd: (1)-->(2-6), (2)==(3), (3)==(2), (6)-->(1-5) + ├── fd: (1)-->(2-5,7), (2)==(3), (3)==(2), (7)-->(1-5) ├── ordering: +(2|3) [actual: +2] └── limit ├── columns: k:1!null i:2!null f:3!null s:4 j:5 @@ -687,22 +687,22 @@ norm expect-not=PushLimitIntoOrdinality SELECT * FROM (SELECT * FROM a ORDER BY k) WITH ORDINALITY ORDER BY i LIMIT 10 ---- limit - ├── columns: k:1!null i:2 f:3 s:4 j:5 ordinality:6!null + ├── columns: k:1!null i:2 f:3 s:4 j:5 ordinality:7!null ├── internal-ordering: +2 ├── cardinality: [0 - 10] ├── key: (1) - ├── fd: (1)-->(2-6), (6)-->(1-5) + ├── fd: (1)-->(2-5,7), (7)-->(1-5) ├── ordering: +2 ├── sort - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 ordinality:6!null + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 ordinality:7!null │ ├── key: (1) - │ ├── fd: (1)-->(2-6), (6)-->(1-5) + │ ├── fd: (1)-->(2-5,7), (7)-->(1-5) │ ├── ordering: +2 │ ├── limit hint: 10.00 │ └── ordinality - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 ordinality:6!null + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 ordinality:7!null │ ├── key: (1) - │ ├── fd: (1)-->(2-6), (6)-->(1-5) + │ ├── fd: (1)-->(2-5,7), (7)-->(1-5) │ └── scan a │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ ├── key: (1) @@ -714,17 +714,17 @@ norm expect-not=PushLimitIntoOrdinality SELECT * FROM (SELECT * FROM a WITH ORDINALITY) ORDER BY ordinality LIMIT 10 ---- limit - ├── columns: k:1!null i:2 f:3 s:4 j:5 ordinality:6!null - ├── internal-ordering: +6 + ├── columns: k:1!null i:2 f:3 s:4 j:5 ordinality:7!null + ├── internal-ordering: +7 ├── cardinality: [0 - 10] ├── key: (1) - ├── fd: (1)-->(2-6), (6)-->(1-5) - ├── ordering: +6 + ├── fd: (1)-->(2-5,7), (7)-->(1-5) + ├── ordering: +7 ├── ordinality - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 ordinality:6!null + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 ordinality:7!null │ ├── key: (1) - │ ├── fd: (1)-->(2-6), (6)-->(1-5) - │ ├── ordering: +6 + │ ├── fd: (1)-->(2-5,7), (7)-->(1-5) + │ ├── ordering: +7 │ ├── limit hint: 10.00 │ └── scan a │ ├── columns: k:1!null i:2 f:3 s:4 j:5 @@ -742,11 +742,11 @@ norm expect=PushLimitIntoJoinLeft SELECT * FROM kvr_fk INNER JOIN uv ON r = u LIMIT 10 ---- inner-join (hash) - ├── columns: k:1!null v:2 r:3!null u:4!null v:5 + ├── columns: k:1!null v:2 r:3!null u:5!null v:6 ├── cardinality: [0 - 10] ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) ├── key: (1) - ├── fd: (1)-->(2,3), (4)-->(5), (3)==(4), (4)==(3) + ├── fd: (1)-->(2,3), (5)-->(6), (3)==(5), (5)==(3) ├── limit │ ├── columns: k:1!null kvr_fk.v:2 r:3!null │ ├── cardinality: [0 - 10] @@ -759,22 +759,22 @@ inner-join (hash) │ │ └── limit hint: 10.00 │ └── 10 ├── scan uv - │ ├── columns: u:4!null uv.v:5 - │ ├── key: (4) - │ └── fd: (4)-->(5) + │ ├── columns: u:5!null uv.v:6 + │ ├── key: (5) + │ └── fd: (5)-->(6) └── filters - └── r:3 = u:4 [outer=(3,4), constraints=(/3: (/NULL - ]; /4: (/NULL - ]), fd=(3)==(4), (4)==(3)] + └── r:3 = u:5 [outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ]), fd=(3)==(5), (5)==(3)] # LeftJoin case. norm expect=PushLimitIntoJoinLeft SELECT * FROM ab LEFT JOIN uv ON a = u LIMIT 10 ---- left-join (hash) - ├── columns: a:1!null b:2 u:3 v:4 + ├── columns: a:1!null b:2 u:4 v:5 ├── cardinality: [0 - 10] ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) ├── key: (1) - ├── fd: (1)-->(2-4), (3)-->(4) + ├── fd: (1)-->(2,4,5), (4)-->(5) ├── limit │ ├── columns: a:1!null b:2 │ ├── cardinality: [0 - 10] @@ -787,56 +787,56 @@ left-join (hash) │ │ └── limit hint: 10.00 │ └── 10 ├── scan uv - │ ├── columns: u:3!null v:4 - │ ├── key: (3) - │ └── fd: (3)-->(4) + │ ├── columns: u:4!null v:5 + │ ├── key: (4) + │ └── fd: (4)-->(5) └── filters - └── a:1 = u:3 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + └── a:1 = u:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] # InnerJoin case for PushLimitIntoJoinRight. norm expect=PushLimitIntoJoinRight SELECT * FROM uv INNER JOIN kvr_fk ON u = r LIMIT 10 ---- inner-join (hash) - ├── columns: u:1!null v:2 k:3!null v:4 r:5!null + ├── columns: u:1!null v:2 k:4!null v:5 r:6!null ├── cardinality: [0 - 10] ├── multiplicity: left-rows(zero-or-more), right-rows(exactly-one) - ├── key: (3) - ├── fd: (1)-->(2), (3)-->(4,5), (1)==(5), (5)==(1) + ├── key: (4) + ├── fd: (1)-->(2), (4)-->(5,6), (1)==(6), (6)==(1) ├── scan uv │ ├── columns: u:1!null uv.v:2 │ ├── key: (1) │ └── fd: (1)-->(2) ├── limit - │ ├── columns: k:3!null kvr_fk.v:4 r:5!null + │ ├── columns: k:4!null kvr_fk.v:5 r:6!null │ ├── cardinality: [0 - 10] - │ ├── key: (3) - │ ├── fd: (3)-->(4,5) + │ ├── key: (4) + │ ├── fd: (4)-->(5,6) │ ├── scan kvr_fk - │ │ ├── columns: k:3!null kvr_fk.v:4 r:5!null - │ │ ├── key: (3) - │ │ ├── fd: (3)-->(4,5) + │ │ ├── columns: k:4!null kvr_fk.v:5 r:6!null + │ │ ├── key: (4) + │ │ ├── fd: (4)-->(5,6) │ │ └── limit hint: 10.00 │ └── 10 └── filters - └── u:1 = r:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + └── u:1 = r:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] # Ordering can be pushed down. norm expect=PushLimitIntoJoinLeft SELECT * FROM ab LEFT JOIN uv ON a = u ORDER BY a LIMIT 10 ---- sort - ├── columns: a:1!null b:2 u:3 v:4 + ├── columns: a:1!null b:2 u:4 v:5 ├── cardinality: [0 - 10] ├── key: (1) - ├── fd: (1)-->(2-4), (3)-->(4) + ├── fd: (1)-->(2,4,5), (4)-->(5) ├── ordering: +1 └── left-join (hash) - ├── columns: a:1!null b:2 u:3 v:4 + ├── columns: a:1!null b:2 u:4 v:5 ├── cardinality: [0 - 10] ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) ├── key: (1) - ├── fd: (1)-->(2-4), (3)-->(4) + ├── fd: (1)-->(2,4,5), (4)-->(5) ├── limit │ ├── columns: a:1!null b:2 │ ├── internal-ordering: +1 @@ -851,27 +851,27 @@ sort │ │ └── limit hint: 10.00 │ └── 10 ├── scan uv - │ ├── columns: u:3!null v:4 - │ ├── key: (3) - │ └── fd: (3)-->(4) + │ ├── columns: u:4!null v:5 + │ ├── key: (4) + │ └── fd: (4)-->(5) └── filters - └── a:1 = u:3 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + └── a:1 = u:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] norm expect=PushLimitIntoJoinLeft SELECT * FROM ab LEFT JOIN uv ON a = u ORDER BY b LIMIT 10 ---- sort - ├── columns: a:1!null b:2 u:3 v:4 + ├── columns: a:1!null b:2 u:4 v:5 ├── cardinality: [0 - 10] ├── key: (1) - ├── fd: (1)-->(2-4), (3)-->(4) + ├── fd: (1)-->(2,4,5), (4)-->(5) ├── ordering: +2 └── left-join (hash) - ├── columns: a:1!null b:2 u:3 v:4 + ├── columns: a:1!null b:2 u:4 v:5 ├── cardinality: [0 - 10] ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) ├── key: (1) - ├── fd: (1)-->(2-4), (3)-->(4) + ├── fd: (1)-->(2,4,5), (4)-->(5) ├── limit │ ├── columns: a:1!null b:2 │ ├── internal-ordering: +2 @@ -890,11 +890,11 @@ sort │ │ └── fd: (1)-->(2) │ └── 10 ├── scan uv - │ ├── columns: u:3!null v:4 - │ ├── key: (3) - │ └── fd: (3)-->(4) + │ ├── columns: u:4!null v:5 + │ ├── key: (4) + │ └── fd: (4)-->(5) └── filters - └── a:1 = u:3 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + └── a:1 = u:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] # Ordering on u is not equivalent to ordering on a because of NULLs; it cannot # be pushed down. @@ -902,33 +902,33 @@ norm expect-not=PushLimitIntoJoinLeft SELECT * FROM ab LEFT JOIN uv ON a = u ORDER BY u LIMIT 10 ---- limit - ├── columns: a:1!null b:2 u:3 v:4 - ├── internal-ordering: +3 + ├── columns: a:1!null b:2 u:4 v:5 + ├── internal-ordering: +4 ├── cardinality: [0 - 10] ├── key: (1) - ├── fd: (1)-->(2-4), (3)-->(4) - ├── ordering: +3 + ├── fd: (1)-->(2,4,5), (4)-->(5) + ├── ordering: +4 ├── sort - │ ├── columns: a:1!null b:2 u:3 v:4 + │ ├── columns: a:1!null b:2 u:4 v:5 │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (3)-->(4) - │ ├── ordering: +3 + │ ├── fd: (1)-->(2,4,5), (4)-->(5) + │ ├── ordering: +4 │ ├── limit hint: 10.00 │ └── left-join (hash) - │ ├── columns: a:1!null b:2 u:3 v:4 + │ ├── columns: a:1!null b:2 u:4 v:5 │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (3)-->(4) + │ ├── fd: (1)-->(2,4,5), (4)-->(5) │ ├── scan ab │ │ ├── columns: a:1!null b:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ ├── scan uv - │ │ ├── columns: u:3!null v:4 - │ │ ├── key: (3) - │ │ └── fd: (3)-->(4) + │ │ ├── columns: u:4!null v:5 + │ │ ├── key: (4) + │ │ └── fd: (4)-->(5) │ └── filters - │ └── a:1 = u:3 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + │ └── a:1 = u:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] └── 10 # Ordering cannot be pushed down. @@ -936,98 +936,98 @@ norm expect-not=PushLimitIntoJoinLeft SELECT * FROM ab LEFT JOIN uv ON a = u ORDER BY v LIMIT 10 ---- limit - ├── columns: a:1!null b:2 u:3 v:4 - ├── internal-ordering: +4 + ├── columns: a:1!null b:2 u:4 v:5 + ├── internal-ordering: +5 ├── cardinality: [0 - 10] ├── key: (1) - ├── fd: (1)-->(2-4), (3)-->(4) - ├── ordering: +4 + ├── fd: (1)-->(2,4,5), (4)-->(5) + ├── ordering: +5 ├── sort - │ ├── columns: a:1!null b:2 u:3 v:4 + │ ├── columns: a:1!null b:2 u:4 v:5 │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (3)-->(4) - │ ├── ordering: +4 + │ ├── fd: (1)-->(2,4,5), (4)-->(5) + │ ├── ordering: +5 │ ├── limit hint: 10.00 │ └── left-join (hash) - │ ├── columns: a:1!null b:2 u:3 v:4 + │ ├── columns: a:1!null b:2 u:4 v:5 │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (3)-->(4) + │ ├── fd: (1)-->(2,4,5), (4)-->(5) │ ├── scan ab │ │ ├── columns: a:1!null b:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ ├── scan uv - │ │ ├── columns: u:3!null v:4 - │ │ ├── key: (3) - │ │ └── fd: (3)-->(4) + │ │ ├── columns: u:4!null v:5 + │ │ ├── key: (4) + │ │ └── fd: (4)-->(5) │ └── filters - │ └── a:1 = u:3 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + │ └── a:1 = u:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] └── 10 norm expect-not=PushLimitIntoJoinLeft SELECT * FROM ab LEFT JOIN uv ON b = v ORDER BY a, v LIMIT 10 ---- limit - ├── columns: a:1!null b:2 u:3 v:4 - ├── internal-ordering: +1,+4 + ├── columns: a:1!null b:2 u:4 v:5 + ├── internal-ordering: +1,+5 ├── cardinality: [0 - 10] - ├── key: (1,3) - ├── fd: (1)-->(2), (3)-->(4) - ├── ordering: +1,+4 + ├── key: (1,4) + ├── fd: (1)-->(2), (4)-->(5) + ├── ordering: +1,+5 ├── sort - │ ├── columns: a:1!null b:2 u:3 v:4 - │ ├── key: (1,3) - │ ├── fd: (1)-->(2), (3)-->(4) - │ ├── ordering: +1,+4 + │ ├── columns: a:1!null b:2 u:4 v:5 + │ ├── key: (1,4) + │ ├── fd: (1)-->(2), (4)-->(5) + │ ├── ordering: +1,+5 │ ├── limit hint: 10.00 │ └── left-join (hash) - │ ├── columns: a:1!null b:2 u:3 v:4 - │ ├── key: (1,3) - │ ├── fd: (1)-->(2), (3)-->(4) + │ ├── columns: a:1!null b:2 u:4 v:5 + │ ├── key: (1,4) + │ ├── fd: (1)-->(2), (4)-->(5) │ ├── scan ab │ │ ├── columns: a:1!null b:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ ├── scan uv - │ │ ├── columns: u:3!null v:4 - │ │ ├── key: (3) - │ │ └── fd: (3)-->(4) + │ │ ├── columns: u:4!null v:5 + │ │ ├── key: (4) + │ │ └── fd: (4)-->(5) │ └── filters - │ └── b:2 = v:4 [outer=(2,4), constraints=(/2: (/NULL - ]; /4: (/NULL - ]), fd=(2)==(4), (4)==(2)] + │ └── b:2 = v:5 [outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ]), fd=(2)==(5), (5)==(2)] └── 10 norm expect-not=PushLimitIntoJoinLeft SELECT * FROM ab LEFT JOIN uv ON a = u ORDER BY u, b LIMIT 10 ---- limit - ├── columns: a:1!null b:2 u:3 v:4 - ├── internal-ordering: +3,+2 + ├── columns: a:1!null b:2 u:4 v:5 + ├── internal-ordering: +4,+2 ├── cardinality: [0 - 10] ├── key: (1) - ├── fd: (1)-->(2-4), (3)-->(4) - ├── ordering: +3,+2 + ├── fd: (1)-->(2,4,5), (4)-->(5) + ├── ordering: +4,+2 ├── sort - │ ├── columns: a:1!null b:2 u:3 v:4 + │ ├── columns: a:1!null b:2 u:4 v:5 │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (3)-->(4) - │ ├── ordering: +3,+2 + │ ├── fd: (1)-->(2,4,5), (4)-->(5) + │ ├── ordering: +4,+2 │ ├── limit hint: 10.00 │ └── left-join (hash) - │ ├── columns: a:1!null b:2 u:3 v:4 + │ ├── columns: a:1!null b:2 u:4 v:5 │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (3)-->(4) + │ ├── fd: (1)-->(2,4,5), (4)-->(5) │ ├── scan ab │ │ ├── columns: a:1!null b:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ ├── scan uv - │ │ ├── columns: u:3!null v:4 - │ │ ├── key: (3) - │ │ └── fd: (3)-->(4) + │ │ ├── columns: u:4!null v:5 + │ │ ├── key: (4) + │ │ └── fd: (4)-->(5) │ └── filters - │ └── a:1 = u:3 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + │ └── a:1 = u:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] └── 10 # Rule should not fire if the input's cardinality is already less than the @@ -1036,11 +1036,11 @@ norm expect-not=PushLimitIntoJoinLeft SELECT * FROM (SELECT * FROM ab LIMIT 5) LEFT JOIN uv ON a = u LIMIT 10 ---- left-join (hash) - ├── columns: a:1!null b:2 u:3 v:4 + ├── columns: a:1!null b:2 u:4 v:5 ├── cardinality: [0 - 5] ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) ├── key: (1) - ├── fd: (1)-->(2-4), (3)-->(4) + ├── fd: (1)-->(2,4,5), (4)-->(5) ├── limit │ ├── columns: a:1!null b:2 │ ├── cardinality: [0 - 5] @@ -1053,22 +1053,22 @@ left-join (hash) │ │ └── limit hint: 5.00 │ └── 5 ├── scan uv - │ ├── columns: u:3!null v:4 - │ ├── key: (3) - │ └── fd: (3)-->(4) + │ ├── columns: u:4!null v:5 + │ ├── key: (4) + │ └── fd: (4)-->(5) └── filters - └── a:1 = u:3 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + └── a:1 = u:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] # Push the limit even if the input is already limited (but with a higher limit). norm expect=PushLimitIntoJoinLeft SELECT * FROM (SELECT * FROM ab LIMIT 20) LEFT JOIN uv ON a = u LIMIT 10 ---- left-join (hash) - ├── columns: a:1!null b:2 u:3 v:4 + ├── columns: a:1!null b:2 u:4 v:5 ├── cardinality: [0 - 10] ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) ├── key: (1) - ├── fd: (1)-->(2-4), (3)-->(4) + ├── fd: (1)-->(2,4,5), (4)-->(5) ├── limit │ ├── columns: a:1!null b:2 │ ├── cardinality: [0 - 10] @@ -1081,38 +1081,38 @@ left-join (hash) │ │ └── limit hint: 10.00 │ └── 10 ├── scan uv - │ ├── columns: u:3!null v:4 - │ ├── key: (3) - │ └── fd: (3)-->(4) + │ ├── columns: u:4!null v:5 + │ ├── key: (4) + │ └── fd: (4)-->(5) └── filters - └── a:1 = u:3 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + └── a:1 = u:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] # Don't push negative limits (or we would enter an infinite loop). norm expect-not=PushLimitIntoJoinLeft SELECT * FROM ab LEFT JOIN uv ON a = u LIMIT -1 ---- limit - ├── columns: a:1!null b:2 u:3 v:4 + ├── columns: a:1!null b:2 u:4 v:5 ├── cardinality: [0 - 0] ├── immutable ├── key: () - ├── fd: ()-->(1-4) + ├── fd: ()-->(1,2,4,5) ├── left-join (hash) - │ ├── columns: a:1!null b:2 u:3 v:4 + │ ├── columns: a:1!null b:2 u:4 v:5 │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (3)-->(4) + │ ├── fd: (1)-->(2,4,5), (4)-->(5) │ ├── limit hint: 1.00 │ ├── scan ab │ │ ├── columns: a:1!null b:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ ├── scan uv - │ │ ├── columns: u:3!null v:4 - │ │ ├── key: (3) - │ │ └── fd: (3)-->(4) + │ │ ├── columns: u:4!null v:5 + │ │ ├── key: (4) + │ │ └── fd: (4)-->(5) │ └── filters - │ └── a:1 = u:3 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + │ └── a:1 = u:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] └── -1 # Don't push limits into an inner join that may not preserve rows. @@ -1120,26 +1120,26 @@ norm expect-not=(PushLimitIntoJoinLeft,PushLimitIntoJoinRight) SELECT * FROM ab INNER JOIN uv ON a = u LIMIT 10 ---- limit - ├── columns: a:1!null b:2 u:3!null v:4 + ├── columns: a:1!null b:2 u:4!null v:5 ├── cardinality: [0 - 10] - ├── key: (3) - ├── fd: (1)-->(2), (3)-->(4), (1)==(3), (3)==(1) + ├── key: (4) + ├── fd: (1)-->(2), (4)-->(5), (1)==(4), (4)==(1) ├── inner-join (hash) - │ ├── columns: a:1!null b:2 u:3!null v:4 + │ ├── columns: a:1!null b:2 u:4!null v:5 │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - │ ├── key: (3) - │ ├── fd: (1)-->(2), (3)-->(4), (1)==(3), (3)==(1) + │ ├── key: (4) + │ ├── fd: (1)-->(2), (4)-->(5), (1)==(4), (4)==(1) │ ├── limit hint: 10.00 │ ├── scan ab │ │ ├── columns: a:1!null b:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ ├── scan uv - │ │ ├── columns: u:3!null v:4 - │ │ ├── key: (3) - │ │ └── fd: (3)-->(4) + │ │ ├── columns: u:4!null v:5 + │ │ ├── key: (4) + │ │ └── fd: (4)-->(5) │ └── filters - │ └── a:1 = u:3 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + │ └── a:1 = u:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] └── 10 # Don't push a limit into the right side of a LeftJoin. @@ -1147,15 +1147,15 @@ norm expect-not=PushLimitIntoJoinRight SELECT * FROM uv LEFT JOIN kvr_fk ON u = r LIMIT 10 ---- limit - ├── columns: u:1!null v:2 k:3 v:4 r:5 + ├── columns: u:1!null v:2 k:4 v:5 r:6 ├── cardinality: [0 - 10] - ├── key: (1,3) - ├── fd: (1)-->(2), (3)-->(4,5) + ├── key: (1,4) + ├── fd: (1)-->(2), (4)-->(5,6) ├── left-join (hash) - │ ├── columns: u:1!null uv.v:2 k:3 kvr_fk.v:4 r:5 + │ ├── columns: u:1!null uv.v:2 k:4 kvr_fk.v:5 r:6 │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) - │ ├── key: (1,3) - │ ├── fd: (1)-->(2), (3)-->(4,5) + │ ├── key: (1,4) + │ ├── fd: (1)-->(2), (4)-->(5,6) │ ├── limit hint: 10.00 │ ├── limit │ │ ├── columns: u:1!null uv.v:2 @@ -1169,11 +1169,11 @@ limit │ │ │ └── limit hint: 10.00 │ │ └── 10 │ ├── scan kvr_fk - │ │ ├── columns: k:3!null kvr_fk.v:4 r:5!null - │ │ ├── key: (3) - │ │ └── fd: (3)-->(4,5) + │ │ ├── columns: k:4!null kvr_fk.v:5 r:6!null + │ │ ├── key: (4) + │ │ └── fd: (4)-->(5,6) │ └── filters - │ └── u:1 = r:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + │ └── u:1 = r:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] └── 10 # Don't push a limit into either side of a FullJoin. @@ -1181,26 +1181,26 @@ norm expect-not=(PushLimitIntoJoinLeft,PushLimitIntoJoinRight) SELECT * FROM ab FULL JOIN uv ON a = u LIMIT 10 ---- limit - ├── columns: a:1 b:2 u:3 v:4 + ├── columns: a:1 b:2 u:4 v:5 ├── cardinality: [0 - 10] - ├── key: (1,3) - ├── fd: (1)-->(2), (3)-->(4) + ├── key: (1,4) + ├── fd: (1)-->(2), (4)-->(5) ├── full-join (hash) - │ ├── columns: a:1 b:2 u:3 v:4 + │ ├── columns: a:1 b:2 u:4 v:5 │ ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) - │ ├── key: (1,3) - │ ├── fd: (1)-->(2), (3)-->(4) + │ ├── key: (1,4) + │ ├── fd: (1)-->(2), (4)-->(5) │ ├── limit hint: 10.00 │ ├── scan ab │ │ ├── columns: a:1!null b:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ ├── scan uv - │ │ ├── columns: u:3!null v:4 - │ │ ├── key: (3) - │ │ └── fd: (3)-->(4) + │ │ ├── columns: u:4!null v:5 + │ │ ├── key: (4) + │ │ └── fd: (4)-->(5) │ └── filters - │ └── a:1 = u:3 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + │ └── a:1 = u:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] └── 10 # ---------- diff --git a/pkg/sql/opt/norm/testdata/rules/max1row b/pkg/sql/opt/norm/testdata/rules/max1row index 5ac7a887e52d..5f91dbbf6ba2 100644 --- a/pkg/sql/opt/norm/testdata/rules/max1row +++ b/pkg/sql/opt/norm/testdata/rules/max1row @@ -13,10 +13,10 @@ norm expect=EliminateMax1Row SELECT (SELECT i FROM a LIMIT 1) > 5 AS r ---- values - ├── columns: r:6 + ├── columns: r:7 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(6) + ├── fd: ()-->(7) └── tuple └── gt ├── subquery @@ -35,31 +35,31 @@ norm expect=EliminateMax1Row SELECT (SELECT count(*) FROM a) > 100 AS r ---- values - ├── columns: r:7 + ├── columns: r:8 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(7) + ├── fd: ()-->(8) └── tuple └── gt ├── subquery │ └── scalar-group-by - │ ├── columns: count_rows:6!null + │ ├── columns: count_rows:7!null │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(6) + │ ├── fd: ()-->(7) │ ├── scan a │ └── aggregations - │ └── count-rows [as=count_rows:6] + │ └── count-rows [as=count_rows:7] └── 100 norm expect=EliminateMax1Row SELECT (SELECT i FROM a LIMIT 0) > 5 AS r ---- values - ├── columns: r:6 + ├── columns: r:7 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(6) + ├── fd: ()-->(7) └── tuple └── gt ├── subquery @@ -75,10 +75,10 @@ norm expect-not=EliminateMax1Row SELECT (SELECT i FROM a) > 5 AS r ---- values - ├── columns: r:6 + ├── columns: r:7 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(6) + ├── fd: ()-->(7) └── tuple └── gt ├── subquery diff --git a/pkg/sql/opt/norm/testdata/rules/numeric b/pkg/sql/opt/norm/testdata/rules/numeric index c4a57aeca72b..18fef5e5011a 100644 --- a/pkg/sql/opt/norm/testdata/rules/numeric +++ b/pkg/sql/opt/norm/testdata/rules/numeric @@ -15,17 +15,17 @@ SELECT FROM a ---- project - ├── columns: r:6 s:7 t:8 u:9 v:10 w:11 + ├── columns: r:7 s:8 t:9 u:10 v:11 w:12 ├── immutable ├── scan a │ └── columns: i:2 f:3 d:4 └── projections - ├── i:2 + i:2 [as=r:6, outer=(2), immutable] - ├── i:2 + i:2 [as=s:7, outer=(2), immutable] - ├── f:3 + f:3 [as=t:8, outer=(3), immutable] - ├── f:3 + f:3 [as=u:9, outer=(3), immutable] - ├── d:4 + d:4 [as=v:10, outer=(4), immutable] - └── d:4 + d:4 [as=w:11, outer=(4), immutable] + ├── i:2 + i:2 [as=r:7, outer=(2), immutable] + ├── i:2 + i:2 [as=s:8, outer=(2), immutable] + ├── f:3 + f:3 [as=t:9, outer=(3), immutable] + ├── f:3 + f:3 [as=u:10, outer=(3), immutable] + ├── d:4 + d:4 [as=v:11, outer=(4), immutable] + └── d:4 + d:4 [as=w:12, outer=(4), immutable] # Regression test for #35113. @@ -33,23 +33,23 @@ norm expect=FoldPlusZero SELECT i + 0::decimal FROM a ---- project - ├── columns: "?column?":6 + ├── columns: "?column?":7 ├── immutable ├── scan a │ └── columns: i:2 └── projections - └── i:2::DECIMAL [as="?column?":6, outer=(2), immutable] + └── i:2::DECIMAL [as="?column?":7, outer=(2), immutable] norm expect=FoldZeroPlus SELECT 0::decimal + i FROM a ---- project - ├── columns: "?column?":6 + ├── columns: "?column?":7 ├── immutable ├── scan a │ └── columns: i:2 └── projections - └── i:2::DECIMAL [as="?column?":6, outer=(2), immutable] + └── i:2::DECIMAL [as="?column?":7, outer=(2), immutable] # -------------------------------------------------- # FoldMinusZero @@ -64,26 +64,26 @@ SELECT FROM a ---- project - ├── columns: r:6 s:7 t:8 + ├── columns: r:7 s:8 t:9 ├── immutable ├── scan a │ └── columns: i:2 f:3 d:4 └── projections - ├── i:2 + i:2 [as=r:6, outer=(2), immutable] - ├── f:3 + f:3 [as=s:7, outer=(3), immutable] - └── d:4 + d:4 [as=t:8, outer=(4), immutable] + ├── i:2 + i:2 [as=r:7, outer=(2), immutable] + ├── f:3 + f:3 [as=s:8, outer=(3), immutable] + └── d:4 + d:4 [as=t:9, outer=(4), immutable] # Regression test for #35113. norm expect=FoldMinusZero SELECT i - 0::decimal FROM a ---- project - ├── columns: "?column?":6 + ├── columns: "?column?":7 ├── immutable ├── scan a │ └── columns: i:2 └── projections - └── i:2::DECIMAL [as="?column?":6, outer=(2), immutable] + └── i:2::DECIMAL [as="?column?":7, outer=(2), immutable] # Regression test for #35612. norm expect-not=FoldMinusZero @@ -109,40 +109,40 @@ SELECT FROM a ---- project - ├── columns: r:6 s:7 t:8 u:9 v:10 w:11 + ├── columns: r:7 s:8 t:9 u:10 v:11 w:12 ├── immutable ├── scan a │ └── columns: i:2 f:3 d:4 └── projections - ├── i:2 + i:2 [as=r:6, outer=(2), immutable] - ├── i:2 + i:2 [as=s:7, outer=(2), immutable] - ├── f:3 + f:3 [as=t:8, outer=(3), immutable] - ├── f:3 + f:3 [as=u:9, outer=(3), immutable] - ├── d:4 + d:4 [as=v:10, outer=(4), immutable] - └── d:4 + d:4 [as=w:11, outer=(4), immutable] + ├── i:2 + i:2 [as=r:7, outer=(2), immutable] + ├── i:2 + i:2 [as=s:8, outer=(2), immutable] + ├── f:3 + f:3 [as=t:9, outer=(3), immutable] + ├── f:3 + f:3 [as=u:10, outer=(3), immutable] + ├── d:4 + d:4 [as=v:11, outer=(4), immutable] + └── d:4 + d:4 [as=w:12, outer=(4), immutable] # Regression test for #35113. norm expect=FoldMultOne SELECT i * 1::decimal FROM a ---- project - ├── columns: "?column?":6 + ├── columns: "?column?":7 ├── immutable ├── scan a │ └── columns: i:2 └── projections - └── i:2::DECIMAL [as="?column?":6, outer=(2), immutable] + └── i:2::DECIMAL [as="?column?":7, outer=(2), immutable] norm expect=FoldOneMult SELECT 1::decimal * i FROM a ---- project - ├── columns: "?column?":6 + ├── columns: "?column?":7 ├── immutable ├── scan a │ └── columns: i:2 └── projections - └── i:2::DECIMAL [as="?column?":6, outer=(2), immutable] + └── i:2::DECIMAL [as="?column?":7, outer=(2), immutable] # -------------------------------------------------- # FoldDivOne @@ -156,37 +156,37 @@ SELECT FROM a ---- project - ├── columns: r:6 s:7 t:8 + ├── columns: r:7 s:8 t:9 ├── immutable ├── scan a │ └── columns: i:2 f:3 d:4 └── projections - ├── i:2::DECIMAL [as=r:6, outer=(2), immutable] - ├── f:3 [as=s:7, outer=(3)] - └── d:4 [as=t:8, outer=(4)] + ├── i:2::DECIMAL [as=r:7, outer=(2), immutable] + ├── f:3 [as=s:8, outer=(3)] + └── d:4 [as=t:9, outer=(4)] # Regression test for #35113. norm expect=FoldDivOne SELECT i / 1::decimal FROM a ---- project - ├── columns: "?column?":6 + ├── columns: "?column?":7 ├── immutable ├── scan a │ └── columns: i:2 └── projections - └── i:2::DECIMAL [as="?column?":6, outer=(2), immutable] + └── i:2::DECIMAL [as="?column?":7, outer=(2), immutable] norm expect=FoldDivOne SELECT i / 1::int8 FROM a ---- project - ├── columns: "?column?":6 + ├── columns: "?column?":7 ├── immutable ├── scan a │ └── columns: i:2 └── projections - └── i:2::DECIMAL [as="?column?":6, outer=(2), immutable] + └── i:2::DECIMAL [as="?column?":7, outer=(2), immutable] # -------------------------------------------------- # InvertMinus @@ -199,14 +199,14 @@ SELECT FROM a ---- project - ├── columns: r:6 s:7 t:8 + ├── columns: r:7 s:8 t:9 ├── immutable ├── scan a │ └── columns: i:2 f:3 d:4 a.t:5 └── projections - ├── f:3 - f:3 [as=r:6, outer=(3), immutable] - ├── i:2 - d:4 [as=s:7, outer=(2,4), immutable] - └── a.t:5 - a.t:5 [as=t:8, outer=(5), immutable] + ├── f:3 - f:3 [as=r:7, outer=(3), immutable] + ├── i:2 - d:4 [as=s:8, outer=(2,4), immutable] + └── a.t:5 - a.t:5 [as=t:9, outer=(5), immutable] # -------------------------------------------------- # EliminateUnaryMinus @@ -215,8 +215,8 @@ norm expect=EliminateUnaryMinus SELECT -(-a.i::int) AS r FROM a ---- project - ├── columns: r:6 + ├── columns: r:7 ├── scan a │ └── columns: i:2 └── projections - └── i:2 [as=r:6, outer=(2)] + └── i:2 [as=r:7, outer=(2)] diff --git a/pkg/sql/opt/norm/testdata/rules/ordering b/pkg/sql/opt/norm/testdata/rules/ordering index 8ad1113dfab5..f63085dbc86f 100644 --- a/pkg/sql/opt/norm/testdata/rules/ordering +++ b/pkg/sql/opt/norm/testdata/rules/ordering @@ -113,10 +113,10 @@ norm SELECT array_agg(b), a, c FROM abcde GROUP BY b, a, c ORDER BY a, b, c ---- group-by - ├── columns: array_agg:6 a:1!null c:3 + ├── columns: array_agg:7 a:1!null c:3 ├── grouping columns: a:1!null ├── key: (1) - ├── fd: (1)-->(3,6) + ├── fd: (1)-->(3,7) ├── ordering: +1 ├── scan abcde │ ├── columns: a:1!null b:2 c:3 @@ -124,7 +124,7 @@ group-by │ ├── fd: (1)-->(2,3), (2,3)~~>(1) │ └── ordering: +1 └── aggregations - ├── array-agg [as=array_agg:6, outer=(2)] + ├── array-agg [as=array_agg:7, outer=(2)] │ └── b:2 └── const-agg [as=c:3, outer=(3)] └── c:3 @@ -134,18 +134,18 @@ norm expect=SimplifyGroupByOrdering SELECT array_agg(b) FROM (SELECT * FROM abcde ORDER BY a, b, c) ---- scalar-group-by - ├── columns: array_agg:6 + ├── columns: array_agg:7 ├── internal-ordering: +1 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(6) + ├── fd: ()-->(7) ├── scan abcde │ ├── columns: a:1!null b:2 │ ├── key: (1) │ ├── fd: (1)-->(2) │ └── ordering: +1 └── aggregations - └── array-agg [as=array_agg:6, outer=(2)] + └── array-agg [as=array_agg:7, outer=(2)] └── b:2 # DistinctOn case. @@ -180,9 +180,9 @@ norm expect=SimplifyOrdinalityOrdering SELECT * FROM (SELECT * FROM abcde WHERE b IS NOT NULL AND c IS NOT NULL ORDER BY c, d, b, e) WITH ORDINALITY ---- ordinality - ├── columns: a:1!null b:2!null c:3!null d:4 e:5 ordinality:6!null + ├── columns: a:1!null b:2!null c:3!null d:4 e:5 ordinality:7!null ├── key: (1) - ├── fd: (1)-->(2-6), (2,3)-->(1,4,5), (6)-->(1-5) + ├── fd: (1)-->(2-5,7), (2,3)-->(1,4,5), (7)-->(1-5) └── sort ├── columns: a:1!null b:2!null c:3!null d:4 e:5 ├── key: (1) @@ -208,24 +208,24 @@ norm expect=SimplifyExplainOrdering EXPLAIN SELECT b, b+1 AS plus, c FROM abcde ORDER BY b, plus, c ---- explain - ├── columns: tree:7 field:8 description:9 + ├── columns: tree:8 field:9 description:10 ├── immutable └── sort - ├── columns: b:2 plus:6 c:3 + ├── columns: b:2 plus:7 c:3 ├── immutable ├── lax-key: (2,3) - ├── fd: (2)-->(6) + ├── fd: (2)-->(7) ├── ordering: +2,+3 └── project - ├── columns: plus:6 b:2 c:3 + ├── columns: plus:7 b:2 c:3 ├── immutable ├── lax-key: (2,3) - ├── fd: (2)-->(6) + ├── fd: (2)-->(7) ├── scan abcde │ ├── columns: b:2 c:3 │ └── lax-key: (2,3) └── projections - └── b:2 + 1 [as=plus:6, outer=(2), immutable] + └── b:2 + 1 [as=plus:7, outer=(2), immutable] # Regression: Explain a statement having constant column, but with no ordering. norm diff --git a/pkg/sql/opt/norm/testdata/rules/project b/pkg/sql/opt/norm/testdata/rules/project index b317d6d35bdd..317daba03a6d 100644 --- a/pkg/sql/opt/norm/testdata/rules/project +++ b/pkg/sql/opt/norm/testdata/rules/project @@ -85,24 +85,24 @@ norm expect=EliminateJoinUnderProjectLeft SELECT 1+b.x FROM b LEFT JOIN a ON b.x = a.x ---- project - ├── columns: "?column?":8!null + ├── columns: "?column?":10!null ├── immutable ├── scan b │ ├── columns: b.x:1!null │ └── key: (1) └── projections - └── b.x:1 + 1 [as="?column?":8, outer=(1), immutable] + └── b.x:1 + 1 [as="?column?":10, outer=(1), immutable] # Case with no references to the left side. norm expect=EliminateJoinUnderProjectLeft SELECT 1 FROM b LEFT JOIN a ON b.x = a.x ---- project - ├── columns: "?column?":8!null - ├── fd: ()-->(8) + ├── columns: "?column?":10!null + ├── fd: ()-->(10) ├── scan b └── projections - └── 1 [as="?column?":8] + └── 1 [as="?column?":10] # SemiJoin case. norm expect=EliminateJoinUnderProjectLeft @@ -116,40 +116,40 @@ norm expect-not=EliminateJoinUnderProjectLeft SELECT 1 FROM b LEFT JOIN a ON True ---- project - ├── columns: "?column?":8!null - ├── fd: ()-->(8) + ├── columns: "?column?":10!null + ├── fd: ()-->(10) ├── left-join (cross) │ ├── scan b │ ├── scan a │ └── filters (true) └── projections - └── 1 [as="?column?":8] + └── 1 [as="?column?":10] # No-op case with a projection that references the right input. norm expect-not=EliminateJoinUnderProjectLeft SELECT b.x, b.z, 1+a.x FROM b LEFT JOIN a ON b.x = a.x ---- project - ├── columns: x:1!null z:2 "?column?":8 + ├── columns: x:1!null z:2 "?column?":10 ├── immutable ├── key: (1) - ├── fd: (1)-->(2,8) + ├── fd: (1)-->(2,10) ├── left-join (hash) - │ ├── columns: b.x:1!null z:2 a.x:4 + │ ├── columns: b.x:1!null z:2 a.x:5 │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ ├── key: (1) - │ ├── fd: (1)-->(2,4) + │ ├── fd: (1)-->(2,5) │ ├── scan b │ │ ├── columns: b.x:1!null z:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ ├── scan a - │ │ ├── columns: a.x:4!null - │ │ └── key: (4) + │ │ ├── columns: a.x:5!null + │ │ └── key: (5) │ └── filters - │ └── b.x:1 = a.x:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + │ └── b.x:1 = a.x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] └── projections - └── a.x:4 + 1 [as="?column?":8, outer=(4), immutable] + └── a.x:5 + 1 [as="?column?":10, outer=(5), immutable] # No-op case because r2 is nullable, and therefore rows may not match despite # the fact that it is a foreign key. @@ -161,19 +161,19 @@ project ├── key: (1) ├── fd: (1)-->(2) └── inner-join (hash) - ├── columns: k:1!null v:2 r2:5!null x:6!null + ├── columns: k:1!null v:2 r2:5!null x:7!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) ├── key: (1) - ├── fd: (1)-->(2,5), (5)==(6), (6)==(5) + ├── fd: (1)-->(2,5), (5)==(7), (7)==(5) ├── scan fks │ ├── columns: k:1!null v:2 r2:5 │ ├── key: (1) │ └── fd: (1)-->(2,5) ├── scan a - │ ├── columns: x:6!null - │ └── key: (6) + │ ├── columns: x:7!null + │ └── key: (7) └── filters - └── r2:5 = x:6 [outer=(5,6), constraints=(/5: (/NULL - ]; /6: (/NULL - ]), fd=(5)==(6), (6)==(5)] + └── r2:5 = x:7 [outer=(5,7), constraints=(/5: (/NULL - ]; /7: (/NULL - ]), fd=(5)==(7), (7)==(5)] # No-op case because r1 is not unique. # InnerJoin case with not-null foreign key. @@ -184,17 +184,17 @@ project ├── columns: x:1!null y:2 ├── fd: (1)-->(2) └── inner-join (hash) - ├── columns: x:1!null y:2 r1:8!null + ├── columns: x:1!null y:2 r1:9!null ├── multiplicity: left-rows(zero-or-more), right-rows(exactly-one) - ├── fd: (1)-->(2), (1)==(8), (8)==(1) + ├── fd: (1)-->(2), (1)==(9), (9)==(1) ├── scan a │ ├── columns: x:1!null y:2 │ ├── key: (1) │ └── fd: (1)-->(2) ├── scan fks - │ └── columns: r1:8!null + │ └── columns: r1:9!null └── filters - └── x:1 = r1:8 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] + └── x:1 = r1:9 [outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ]), fd=(1)==(9), (9)==(1)] # No-op case because r1 is not unique. # LeftJoin case with not-null foreign key. @@ -205,7 +205,7 @@ project ├── columns: x:1!null y:2 ├── fd: (1)-->(2) └── left-join (hash) - ├── columns: x:1!null y:2 r1:8 + ├── columns: x:1!null y:2 r1:9 ├── multiplicity: left-rows(one-or-more), right-rows(exactly-one) ├── fd: (1)-->(2) ├── scan a @@ -213,9 +213,9 @@ project │ ├── key: (1) │ └── fd: (1)-->(2) ├── scan fks - │ └── columns: r1:8!null + │ └── columns: r1:9!null └── filters - └── x:1 = r1:8 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] + └── x:1 = r1:9 [outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ]), fd=(1)==(9), (9)==(1)] # -------------------------------------------------- # EliminateJoinUnderProjectRight @@ -226,18 +226,18 @@ norm expect=EliminateJoinUnderProjectRight SELECT b1.x, b1.z FROM b INNER JOIN b AS b1 ON b.x = b1.x ---- scan b1 - ├── columns: x:4!null z:5 - ├── key: (4) - └── fd: (4)-->(5) + ├── columns: x:5!null z:6 + ├── key: (5) + └── fd: (5)-->(6) # InnerJoin case with not-null foreign key. norm expect=EliminateJoinUnderProjectRight SELECT k, v FROM a INNER JOIN fks ON r1 = x ---- scan fks - ├── columns: k:5!null v:6 - ├── key: (5) - └── fd: (5)-->(6) + ├── columns: k:6!null v:7 + ├── key: (6) + └── fd: (6)-->(7) # No-op case because columns from the right side of a LeftJoin are being # projected. @@ -245,18 +245,18 @@ norm expect-not=EliminateJoinUnderProjectRight SELECT b.x, b1.x FROM b LEFT JOIN b AS b1 ON b.x = b1.x ---- inner-join (hash) - ├── columns: x:1!null x:4!null + ├── columns: x:1!null x:5!null ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) - ├── key: (4) - ├── fd: (1)==(4), (4)==(1) + ├── key: (5) + ├── fd: (1)==(5), (5)==(1) ├── scan b │ ├── columns: b.x:1!null │ └── key: (1) ├── scan b1 - │ ├── columns: b1.x:4!null - │ └── key: (4) + │ ├── columns: b1.x:5!null + │ └── key: (5) └── filters - └── b.x:1 = b1.x:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + └── b.x:1 = b1.x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] # -------------------------------------------------- # EliminateProject @@ -294,15 +294,15 @@ norm expect-not=EliminateProject SELECT *, 1 r FROM a ---- project - ├── columns: x:1!null y:2 f:3 s:4 r:5!null + ├── columns: x:1!null y:2 f:3 s:4 r:6!null ├── key: (1) - ├── fd: ()-->(5), (1)-->(2-4) + ├── fd: ()-->(6), (1)-->(2-4) ├── scan a │ ├── columns: x:1!null y:2 f:3 s:4 │ ├── key: (1) │ └── fd: (1)-->(2-4) └── projections - └── 1 [as=r:5] + └── 1 [as=r:6] # -------------------------------------------------- # MergeProjects @@ -313,95 +313,95 @@ norm expect=MergeProjects SELECT y+1 AS r FROM (SELECT a.y FROM a, b WHERE a.x=b.x) a ---- project - ├── columns: r:8 + ├── columns: r:10 ├── immutable ├── inner-join (hash) - │ ├── columns: a.x:1!null y:2 b.x:5!null + │ ├── columns: a.x:1!null y:2 b.x:6!null │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - │ ├── key: (5) - │ ├── fd: (1)-->(2), (1)==(5), (5)==(1) + │ ├── key: (6) + │ ├── fd: (1)-->(2), (1)==(6), (6)==(1) │ ├── scan a │ │ ├── columns: a.x:1!null y:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ ├── scan b - │ │ ├── columns: b.x:5!null - │ │ └── key: (5) + │ │ ├── columns: b.x:6!null + │ │ └── key: (6) │ └── filters - │ └── a.x:1 = b.x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + │ └── a.x:1 = b.x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] └── projections - └── y:2 + 1 [as=r:8, outer=(2), immutable] + └── y:2 + 1 [as=r:10, outer=(2), immutable] # Outer and inner projections have synthesized columns. norm expect=MergeProjects SELECT y1, f+1 FROM (SELECT y+1 AS y1, f FROM a) ---- project - ├── columns: y1:5 "?column?":6 + ├── columns: y1:6 "?column?":7 ├── immutable ├── scan a │ └── columns: y:2 f:3 └── projections - ├── f:3 + 1.0 [as="?column?":6, outer=(3), immutable] - └── y:2 + 1 [as=y1:5, outer=(2), immutable] + ├── f:3 + 1.0 [as="?column?":7, outer=(3), immutable] + └── y:2 + 1 [as=y1:6, outer=(2), immutable] # Multiple synthesized columns in both outer and inner projections. norm expect=MergeProjects SELECT y1, f+1, x2, s||'foo' FROM (SELECT y+1 AS y1, f, s, x*2 AS x2 FROM a) ---- project - ├── columns: y1:5 "?column?":7 x2:6!null "?column?":8 + ├── columns: y1:6 "?column?":8 x2:7!null "?column?":9 ├── immutable ├── scan a │ ├── columns: x:1!null y:2 f:3 s:4 │ ├── key: (1) │ └── fd: (1)-->(2-4) └── projections - ├── f:3 + 1.0 [as="?column?":7, outer=(3), immutable] - ├── s:4 || 'foo' [as="?column?":8, outer=(4), immutable] - ├── y:2 + 1 [as=y1:5, outer=(2), immutable] - └── x:1 * 2 [as=x2:6, outer=(1), immutable] + ├── f:3 + 1.0 [as="?column?":8, outer=(3), immutable] + ├── s:4 || 'foo' [as="?column?":9, outer=(4), immutable] + ├── y:2 + 1 [as=y1:6, outer=(2), immutable] + └── x:1 * 2 [as=x2:7, outer=(1), immutable] # Outer project selects subset of inner columns. norm expect=MergeProjects SELECT y1 FROM (SELECT y+1 AS y1, f*2 AS f2 FROM a) ---- project - ├── columns: y1:5 + ├── columns: y1:6 ├── immutable ├── scan a │ └── columns: y:2 └── projections - └── y:2 + 1 [as=y1:5, outer=(2), immutable] + └── y:2 + 1 [as=y1:6, outer=(2), immutable] # Don't merge, since outer depends on inner. norm expect-not=MergeProjects SELECT y1*2, y1/2 FROM (SELECT y+1 AS y1 FROM a) ---- project - ├── columns: "?column?":6 "?column?":7 + ├── columns: "?column?":7 "?column?":8 ├── immutable ├── project - │ ├── columns: y1:5 + │ ├── columns: y1:6 │ ├── immutable │ ├── scan a │ │ └── columns: y:2 │ └── projections - │ └── y:2 + 1 [as=y1:5, outer=(2), immutable] + │ └── y:2 + 1 [as=y1:6, outer=(2), immutable] └── projections - ├── y1:5 * 2 [as="?column?":6, outer=(5), immutable] - └── y1:5 / 2 [as="?column?":7, outer=(5)] + ├── y1:6 * 2 [as="?column?":7, outer=(6), immutable] + └── y1:6 / 2 [as="?column?":8, outer=(6)] # Discard all inner columns. norm expect=MergeProjects SELECT 1 r FROM (SELECT y+1, x FROM a) a ---- project - ├── columns: r:6!null - ├── fd: ()-->(6) + ├── columns: r:7!null + ├── fd: ()-->(7) ├── scan a └── projections - └── 1 [as=r:6] + └── 1 [as=r:7] # -------------------------------------------------- # MergeProjectWithValues @@ -537,47 +537,47 @@ norm expect=FoldTupleAccessIntoValues SELECT (SELECT (tup).@1 * x FROM b) FROM (VALUES ((1,2)), ((3,4))) AS v(tup) ---- project - ├── columns: "?column?":6 + ├── columns: "?column?":7 ├── cardinality: [1 - ] ├── immutable ├── ensure-distinct-on - │ ├── columns: "?column?":5 rownum:9!null - │ ├── grouping columns: rownum:9!null + │ ├── columns: "?column?":6 rownum:10!null + │ ├── grouping columns: rownum:10!null │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── cardinality: [1 - ] │ ├── immutable - │ ├── key: (9) - │ ├── fd: (9)-->(5) + │ ├── key: (10) + │ ├── fd: (10)-->(6) │ ├── left-join-apply - │ │ ├── columns: "?column?":5 column1_1:7!null rownum:9!null + │ │ ├── columns: "?column?":6 column1_1:8!null rownum:10!null │ │ ├── cardinality: [2 - ] │ │ ├── immutable - │ │ ├── fd: (9)-->(7) + │ │ ├── fd: (10)-->(8) │ │ ├── ordinality - │ │ │ ├── columns: column1_1:7!null rownum:9!null + │ │ │ ├── columns: column1_1:8!null rownum:10!null │ │ │ ├── cardinality: [2 - 2] - │ │ │ ├── key: (9) - │ │ │ ├── fd: (9)-->(7) + │ │ │ ├── key: (10) + │ │ │ ├── fd: (10)-->(8) │ │ │ └── values - │ │ │ ├── columns: column1_1:7!null + │ │ │ ├── columns: column1_1:8!null │ │ │ ├── cardinality: [2 - 2] │ │ │ ├── (1,) │ │ │ └── (3,) │ │ ├── project - │ │ │ ├── columns: "?column?":5 - │ │ │ ├── outer: (7) + │ │ │ ├── columns: "?column?":6 + │ │ │ ├── outer: (8) │ │ │ ├── immutable │ │ │ ├── scan b │ │ │ │ ├── columns: x:2!null │ │ │ │ └── key: (2) │ │ │ └── projections - │ │ │ └── x:2 * column1_1:7 [as="?column?":5, outer=(2,7), immutable] + │ │ │ └── x:2 * column1_1:8 [as="?column?":6, outer=(2,8), immutable] │ │ └── filters (true) │ └── aggregations - │ └── const-agg [as="?column?":5, outer=(5)] - │ └── "?column?":5 + │ └── const-agg [as="?column?":6, outer=(6)] + │ └── "?column?":6 └── projections - └── "?column?":5 [as="?column?":6, outer=(5)] + └── "?column?":6 [as="?column?":7, outer=(6)] # Case where columns are unnested and then pruned away because the surrounding # project only references an outer column. @@ -707,31 +707,31 @@ norm expect-not=FoldTupleAccessIntoValues SELECT (tup).@1, (tup).@2 FROM (VALUES ((3,4)), ((SELECT (x, z) FROM b))) AS v(tup) ---- project - ├── columns: "?column?":6 "?column?":7 + ├── columns: "?column?":7 "?column?":8 ├── cardinality: [2 - 2] ├── values - │ ├── columns: column1:5 + │ ├── columns: column1:6 │ ├── cardinality: [2 - 2] │ ├── ((3, 4),) │ └── tuple │ └── subquery │ └── max1-row - │ ├── columns: "?column?":4 + │ ├── columns: "?column?":5 │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(4) + │ ├── fd: ()-->(5) │ └── project - │ ├── columns: "?column?":4 + │ ├── columns: "?column?":5 │ ├── scan b │ │ ├── columns: x:1!null z:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ └── projections - │ └── (x:1, z:2) [as="?column?":4, outer=(1,2)] + │ └── (x:1, z:2) [as="?column?":5, outer=(1,2)] └── projections - ├── (column1:5).@1 [as="?column?":6, outer=(5)] - └── (column1:5).@2 [as="?column?":7, outer=(5)] + ├── (column1:6).@1 [as="?column?":7, outer=(6)] + └── (column1:6).@2 [as="?column?":8, outer=(6)] # No-op case because the tuple itself is referenced rather than just its fields. norm expect-not=FoldTupleAccessIntoValues @@ -830,7 +830,7 @@ norm expect=PushColumnRemappingIntoValues WITH a AS (SELECT v FROM (VALUES (1), ((SELECT z FROM b WHERE z=1))) f(v)) SELECT v FROM a ---- values - ├── columns: v:5 + ├── columns: v:6 ├── cardinality: [2 - 2] ├── (1,) └── tuple @@ -997,13 +997,13 @@ norm expect-not=PushColumnRemappingIntoValues WITH t AS (SELECT * FROM a) SELECT x FROM t ---- project - ├── columns: x:5!null - ├── key: (5) + ├── columns: x:6!null + ├── key: (6) ├── scan a │ ├── columns: a.x:1!null │ └── key: (1) └── projections - └── a.x:1 [as=x:5, outer=(1)] + └── a.x:1 [as=x:6, outer=(1)] # No-op case with no projections on the Project surrounding the Values operator. # A Project with no projections is created when PruneUnionAllCols fires, and is @@ -1151,11 +1151,11 @@ INNER JOIN LATERAL ON True ---- project - ├── columns: x:1!null z:2 j:3 x:5 "?column?":6!null + ├── columns: x:1!null z:2 j:3 x:6 "?column?":7!null ├── immutable - ├── fd: (1)-->(2,3), (3)-->(5) + ├── fd: (1)-->(2,3), (3)-->(6) ├── inner-join (cross) - │ ├── columns: b.x:1!null z:2 j:3 "?column?":6!null + │ ├── columns: b.x:1!null z:2 j:3 "?column?":7!null │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-more) │ ├── fd: (1)-->(2,3) │ ├── scan b @@ -1163,13 +1163,13 @@ project │ │ ├── key: (1) │ │ └── fd: (1)-->(2,3) │ ├── values - │ │ ├── columns: "?column?":6!null + │ │ ├── columns: "?column?":7!null │ │ ├── cardinality: [2 - 2] │ │ ├── ('"zero"',) │ │ └── ('"one"',) │ └── filters (true) └── projections - └── j:3->'x' [as=x:5, outer=(3), immutable] + └── j:3->'x' [as=x:6, outer=(3), immutable] # Rule fires harmlessly when none of the JSON fields are accessed; the columns # are later pruned away by PruneValuesCols. @@ -1186,19 +1186,19 @@ SELECT FROM b ---- project - ├── columns: "?column?":7 + ├── columns: "?column?":8 ├── immutable ├── ensure-distinct-on - │ ├── columns: x:1!null "?column?":5 + │ ├── columns: x:1!null "?column?":6 │ ├── grouping columns: x:1!null │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(5) + │ ├── fd: (1)-->(6) │ ├── project - │ │ ├── columns: "?column?":5 x:1!null + │ │ ├── columns: "?column?":6 x:1!null │ │ ├── immutable - │ │ ├── fd: (1)-->(5) + │ │ ├── fd: (1)-->(6) │ │ ├── inner-join (cross) │ │ │ ├── columns: x:1!null j:3 │ │ │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-more) @@ -1213,12 +1213,12 @@ project │ │ │ │ └── () │ │ │ └── filters (true) │ │ └── projections - │ │ └── j:3->'x' [as="?column?":5, outer=(3), immutable] + │ │ └── j:3->'x' [as="?column?":6, outer=(3), immutable] │ └── aggregations - │ └── const-agg [as="?column?":5, outer=(5)] - │ └── "?column?":5 + │ └── const-agg [as="?column?":6, outer=(6)] + │ └── "?column?":6 └── projections - └── "?column?":5 [as="?column?":7, outer=(5)] + └── "?column?":6 [as="?column?":8, outer=(6)] # No-op case because the Values column contains strings, not JSON expressions. norm expect-not=FoldJSONAccessIntoValues @@ -1343,11 +1343,11 @@ FROM ) v(j) ---- project - ├── columns: x:5 + ├── columns: x:6 ├── cardinality: [2 - 2] ├── immutable ├── values - │ ├── columns: column1:4 + │ ├── columns: column1:5 │ ├── cardinality: [2 - 2] │ ├── ('{"x": "one"}',) │ └── tuple @@ -1361,7 +1361,7 @@ project │ └── scan b │ └── columns: j:3 └── projections - └── column1:4->'x' [as=x:5, outer=(4), immutable] + └── column1:5->'x' [as=x:6, outer=(5), immutable] # No-op case because the key being used to access JSON fields is not a constant # string. @@ -1378,20 +1378,20 @@ SELECT FROM a ---- project - ├── columns: "?column?":7 + ├── columns: "?column?":8 ├── immutable ├── ensure-distinct-on - │ ├── columns: x:1!null "?column?":6 + │ ├── columns: x:1!null "?column?":7 │ ├── grouping columns: x:1!null │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(6) + │ ├── fd: (1)-->(7) │ ├── project - │ │ ├── columns: "?column?":6 x:1!null + │ │ ├── columns: "?column?":7 x:1!null │ │ ├── immutable │ │ ├── inner-join (cross) - │ │ │ ├── columns: x:1!null s:4 column1:5!null + │ │ │ ├── columns: x:1!null s:4 column1:6!null │ │ │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-more) │ │ │ ├── fd: (1)-->(4) │ │ │ ├── scan a @@ -1399,18 +1399,18 @@ project │ │ │ │ ├── key: (1) │ │ │ │ └── fd: (1)-->(4) │ │ │ ├── values - │ │ │ │ ├── columns: column1:5!null + │ │ │ │ ├── columns: column1:6!null │ │ │ │ ├── cardinality: [2 - 2] │ │ │ │ ├── ('{"x": "one"}',) │ │ │ │ └── ('{"x": "two"}',) │ │ │ └── filters (true) │ │ └── projections - │ │ └── column1:5->s:4 [as="?column?":6, outer=(4,5), immutable] + │ │ └── column1:6->s:4 [as="?column?":7, outer=(4,6), immutable] │ └── aggregations - │ └── const-agg [as="?column?":6, outer=(6)] - │ └── "?column?":6 + │ └── const-agg [as="?column?":7, outer=(7)] + │ └── "?column?":7 └── projections - └── "?column?":6 [as="?column?":7, outer=(6)] + └── "?column?":7 [as="?column?":8, outer=(7)] # No-op case because one of the projections attempts to access a key that is not # in any of the rows. diff --git a/pkg/sql/opt/norm/testdata/rules/project_set b/pkg/sql/opt/norm/testdata/rules/project_set index d4c65e5956ed..f54be75e1213 100644 --- a/pkg/sql/opt/norm/testdata/rules/project_set +++ b/pkg/sql/opt/norm/testdata/rules/project_set @@ -78,16 +78,16 @@ norm expect=ConvertZipArraysToValues SELECT unnest(ARRAY[x,y]) FROM xy ---- project - ├── columns: unnest:4 + ├── columns: unnest:5 └── inner-join-apply - ├── columns: x:1!null y:2 unnest:4 + ├── columns: x:1!null y:2 unnest:5 ├── fd: (1)-->(2) ├── scan xy │ ├── columns: x:1!null y:2 │ ├── key: (1) │ └── fd: (1)-->(2) ├── values - │ ├── columns: unnest:4 + │ ├── columns: unnest:5 │ ├── outer: (1,2) │ ├── cardinality: [2 - 2] │ ├── (x:1,) @@ -113,7 +113,7 @@ semi-join-apply │ ├── key: (1) │ └── fd: (1)-->(2,3) ├── values - │ ├── columns: unnest:4 + │ ├── columns: unnest:5 │ ├── outer: (1) │ ├── cardinality: [6 - 6] │ ├── (NULL,) @@ -123,7 +123,7 @@ semi-join-apply │ ├── (5,) │ └── (x:1,) └── filters - └── unnest:4 = y:2 [outer=(2,4), constraints=(/2: (/NULL - ]; /4: (/NULL - ]), fd=(2)==(4), (4)==(2)] + └── unnest:5 = y:2 [outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ]), fd=(2)==(5), (5)==(2)] # Case with multiple arrays of different types and different sizes, including an # empty array. @@ -191,10 +191,10 @@ norm expect=ConvertZipArraysToValues SELECT unnest(ARRAY[x,y]), unnest(ARRAY[1,x*100]) FROM xy ---- project - ├── columns: unnest:4 unnest:5 + ├── columns: unnest:5 unnest:6 ├── immutable └── inner-join-apply - ├── columns: x:1!null y:2 unnest:4 unnest:5 + ├── columns: x:1!null y:2 unnest:5 unnest:6 ├── immutable ├── fd: (1)-->(2) ├── scan xy @@ -202,7 +202,7 @@ project │ ├── key: (1) │ └── fd: (1)-->(2) ├── values - │ ├── columns: unnest:4 unnest:5 + │ ├── columns: unnest:5 unnest:6 │ ├── outer: (1,2) │ ├── cardinality: [2 - 2] │ ├── immutable @@ -275,7 +275,7 @@ norm expect-not=ConvertZipArraysToValues SELECT unnest((SELECT array_agg(y) FROM xy)) ---- project-set - ├── columns: unnest:5 + ├── columns: unnest:6 ├── immutable ├── values │ ├── cardinality: [1 - 1] @@ -285,14 +285,14 @@ project-set └── function: unnest [immutable, subquery] └── subquery └── scalar-group-by - ├── columns: array_agg:4 + ├── columns: array_agg:5 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(4) + ├── fd: ()-->(5) ├── scan xy │ └── columns: y:2 └── aggregations - └── array-agg [as=array_agg:4, outer=(2)] + └── array-agg [as=array_agg:5, outer=(2)] └── y:2 # No-op case because a JSON column can only be determined at run-time. @@ -300,10 +300,10 @@ norm expect-not=ConvertZipArraysToValues SELECT json_array_elements(j) FROM xy ---- project - ├── columns: json_array_elements:4 + ├── columns: json_array_elements:5 ├── immutable └── project-set - ├── columns: j:3 json_array_elements:4 + ├── columns: j:3 json_array_elements:5 ├── immutable ├── scan xy │ └── columns: j:3 diff --git a/pkg/sql/opt/norm/testdata/rules/prune_cols b/pkg/sql/opt/norm/testdata/rules/prune_cols index c533872dcc3b..90fc4fce02b4 100644 --- a/pkg/sql/opt/norm/testdata/rules/prune_cols +++ b/pkg/sql/opt/norm/testdata/rules/prune_cols @@ -62,73 +62,73 @@ norm expect=PruneProjectCols SELECT k1*2 FROM (SELECT k+1 AS k1, i+1 FROM a) a ---- project - ├── columns: "?column?":7!null + ├── columns: "?column?":8!null ├── immutable ├── scan a │ ├── columns: k:1!null │ └── key: (1) └── projections - └── (k:1 + 1) * 2 [as="?column?":7, outer=(1), immutable] + └── (k:1 + 1) * 2 [as="?column?":8, outer=(1), immutable] # Use column values within computed column. norm expect=PruneProjectCols SELECT k+length(s) AS r FROM (SELECT i, k, s || 'foo' AS s FROM a) a ---- project - ├── columns: r:6 + ├── columns: r:7 ├── immutable ├── scan a │ ├── columns: k:1!null a.s:4 │ ├── key: (1) │ └── fd: (1)-->(4) └── projections - └── k:1 + length(a.s:4 || 'foo') [as=r:6, outer=(1,4), immutable] + └── k:1 + length(a.s:4 || 'foo') [as=r:7, outer=(1,4), immutable] # Discard non-computed columns and keep computed column. norm expect=PruneProjectCols SELECT l, l*2, k FROM (SELECT length(s) l, * FROM a) a ---- project - ├── columns: l:5 "?column?":6 k:1!null + ├── columns: l:6 "?column?":7 k:1!null ├── immutable ├── key: (1) - ├── fd: (1)-->(5), (5)-->(6) + ├── fd: (1)-->(6), (6)-->(7) ├── project - │ ├── columns: l:5 k:1!null + │ ├── columns: l:6 k:1!null │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(5) + │ ├── fd: (1)-->(6) │ ├── scan a │ │ ├── columns: k:1!null s:4 │ │ ├── key: (1) │ │ └── fd: (1)-->(4) │ └── projections - │ └── length(s:4) [as=l:5, outer=(4), immutable] + │ └── length(s:4) [as=l:6, outer=(4), immutable] └── projections - └── l:5 * 2 [as="?column?":6, outer=(5), immutable] + └── l:6 * 2 [as="?column?":7, outer=(6), immutable] # Compute column based on another computed column. norm expect=PruneProjectCols SELECT l*l AS r, k FROM (SELECT k, length(s) l, i FROM a) a ---- project - ├── columns: r:6 k:1!null + ├── columns: r:7 k:1!null ├── immutable ├── key: (1) - ├── fd: (1)-->(6) + ├── fd: (1)-->(7) ├── project - │ ├── columns: l:5 k:1!null + │ ├── columns: l:6 k:1!null │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(5) + │ ├── fd: (1)-->(6) │ ├── scan a │ │ ├── columns: k:1!null s:4 │ │ ├── key: (1) │ │ └── fd: (1)-->(4) │ └── projections - │ └── length(s:4) [as=l:5, outer=(4), immutable] + │ └── length(s:4) [as=l:6, outer=(4), immutable] └── projections - └── l:5 * l:5 [as=r:6, outer=(5), immutable] + └── l:6 * l:6 [as=r:7, outer=(6), immutable] # -------------------------------------------------- # PruneScanCols @@ -147,42 +147,42 @@ norm expect=PruneScanCols SELECT k, k+1 AS r, i+1 AS s FROM a ---- project - ├── columns: k:1!null r:5!null s:6 + ├── columns: k:1!null r:6!null s:7 ├── immutable ├── key: (1) - ├── fd: (1)-->(5,6) + ├── fd: (1)-->(6,7) ├── scan a │ ├── columns: k:1!null i:2 │ ├── key: (1) │ └── fd: (1)-->(2) └── projections - ├── k:1 + 1 [as=r:5, outer=(1), immutable] - └── i:2 + 1 [as=s:6, outer=(2), immutable] + ├── k:1 + 1 [as=r:6, outer=(1), immutable] + └── i:2 + 1 [as=s:7, outer=(2), immutable] # Use columns only in computed columns. norm expect=PruneScanCols SELECT k+i AS r FROM a ---- project - ├── columns: r:5 + ├── columns: r:6 ├── immutable ├── scan a │ ├── columns: k:1!null i:2 │ ├── key: (1) │ └── fd: (1)-->(2) └── projections - └── k:1 + i:2 [as=r:5, outer=(1,2), immutable] + └── k:1 + i:2 [as=r:6, outer=(1,2), immutable] # Use no scan columns. norm expect=PruneScanCols SELECT 1 r FROM a ---- project - ├── columns: r:5!null - ├── fd: ()-->(5) + ├── columns: r:6!null + ├── fd: ()-->(6) ├── scan a └── projections - └── 1 [as=r:5] + └── 1 [as=r:6] # -------------------------------------------------- # PruneSelectCols @@ -228,23 +228,23 @@ norm expect=PruneSelectCols SELECT 1 r FROM a WHERE $1<'2000-01-01T02:00:00'::timestamp ---- project - ├── columns: r:5!null + ├── columns: r:6!null ├── has-placeholder - ├── fd: ()-->(5) + ├── fd: ()-->(6) ├── select │ ├── has-placeholder │ ├── scan a │ └── filters │ └── $1 < '2000-01-01 02:00:00+00:00' └── projections - └── 1 [as=r:5] + └── 1 [as=r:6] # Select columns used in computed columns. norm expect=PruneSelectCols SELECT i-1 AS r, k*k AS t FROM a WHERE k+1<5 AND s||'o'='foo' ---- project - ├── columns: r:5 t:6!null + ├── columns: r:6 t:7!null ├── immutable ├── select │ ├── columns: k:1!null i:2 s:4 @@ -259,8 +259,8 @@ project │ ├── k:1 < 4 [outer=(1), constraints=(/1: (/NULL - /3]; tight)] │ └── (s:4 || 'o') = 'foo' [outer=(4), immutable] └── projections - ├── i:2 - 1 [as=r:5, outer=(2), immutable] - └── k:1 * k:1 [as=t:6, outer=(1), immutable] + ├── i:2 - 1 [as=r:6, outer=(2), immutable] + └── k:1 * k:1 [as=t:7, outer=(1), immutable] # Select nested in select. norm expect=PruneSelectCols @@ -273,16 +273,16 @@ project ├── key: () ├── fd: ()-->(2) └── select - ├── columns: i:2 f:5!null + ├── columns: i:2 f:6!null ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(2,5) + ├── fd: ()-->(2,6) ├── project - │ ├── columns: f:5 i:2 + │ ├── columns: f:6 i:2 │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(2,5) + │ ├── fd: ()-->(2,6) │ ├── select │ │ ├── columns: k:1!null i:2 a.f:3 │ │ ├── cardinality: [0 - 1] @@ -295,40 +295,40 @@ project │ │ └── filters │ │ └── k:1 = 5 [outer=(1), constraints=(/1: [/5 - /5]; tight), fd=()-->(1)] │ └── projections - │ └── a.f:3 / 2.0 [as=f:5, outer=(3)] + │ └── a.f:3 / 2.0 [as=f:6, outer=(3)] └── filters - └── f:5 = i:2::FLOAT8 [outer=(2,5), immutable, constraints=(/5: (/NULL - ])] + └── f:6 = i:2::FLOAT8 [outer=(2,6), immutable, constraints=(/6: (/NULL - ])] # Detect PruneSelectCols and PushSelectIntoProject dependency cycle. norm SELECT f, f+1.1 AS r FROM (SELECT f, k FROM a GROUP BY f, k HAVING sum(k)=100) a ---- project - ├── columns: f:3 r:6 + ├── columns: f:3 r:7 ├── immutable ├── select - │ ├── columns: k:1!null f:3 sum:5!null + │ ├── columns: k:1!null f:3 sum:6!null │ ├── immutable │ ├── key: (1) - │ ├── fd: ()-->(5), (1)-->(3) + │ ├── fd: ()-->(6), (1)-->(3) │ ├── group-by - │ │ ├── columns: k:1!null f:3 sum:5!null + │ │ ├── columns: k:1!null f:3 sum:6!null │ │ ├── grouping columns: k:1!null │ │ ├── key: (1) - │ │ ├── fd: (1)-->(3,5) + │ │ ├── fd: (1)-->(3,6) │ │ ├── scan a │ │ │ ├── columns: k:1!null f:3 │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(3) │ │ └── aggregations - │ │ ├── sum [as=sum:5, outer=(1)] + │ │ ├── sum [as=sum:6, outer=(1)] │ │ │ └── k:1 │ │ └── const-agg [as=f:3, outer=(3)] │ │ └── f:3 │ └── filters - │ └── sum:5 = 100 [outer=(5), immutable, constraints=(/5: [/100 - /100]; tight), fd=()-->(5)] + │ └── sum:6 = 100 [outer=(6), immutable, constraints=(/6: [/100 - /100]; tight), fd=()-->(6)] └── projections - └── f:3 + 1.1 [as=r:6, outer=(3), immutable] + └── f:3 + 1.1 [as=r:7, outer=(3), immutable] # -------------------------------------------------- # PruneLimitCols @@ -431,7 +431,7 @@ norm SELECT f, f*2.0 AS r FROM (SELECT f, s FROM a GROUP BY f, s LIMIT 5) a ---- project - ├── columns: f:3 r:5 + ├── columns: f:3 r:6 ├── cardinality: [0 - 5] ├── immutable ├── limit @@ -448,7 +448,7 @@ project │ │ └── limit hint: 6.02 │ └── 5 └── projections - └── f:3 * 2.0 [as=r:5, outer=(3), immutable] + └── f:3 * 2.0 [as=r:6, outer=(3), immutable] # -------------------------------------------------- # PruneOffsetCols @@ -544,23 +544,23 @@ FROM ) ---- offset - ├── columns: tree:12 columns:17 - ├── internal-ordering: +12 + ├── columns: tree:13 columns:18 + ├── internal-ordering: +13 ├── sort - │ ├── columns: tree:12 columns:17 - │ ├── ordering: +12 + │ ├── columns: tree:13 columns:18 + │ ├── ordering: +13 │ └── project - │ ├── columns: tree:12 columns:17 + │ ├── columns: tree:13 columns:18 │ ├── explain - │ │ ├── columns: tree:5 level:6 node_type:7 field:8 description:9 columns:10 ordering:11 + │ │ ├── columns: tree:6 level:7 node_type:8 field:9 description:10 columns:11 ordering:12 │ │ ├── mode: verbose │ │ └── scan a │ │ ├── columns: k:1!null i:2 f:3 s:4 │ │ ├── key: (1) │ │ └── fd: (1)-->(2-4) │ └── projections - │ ├── tree:5 [as=tree:12, outer=(5)] - │ └── columns:10 [as=columns:17, outer=(10)] + │ ├── tree:6 [as=tree:13, outer=(6)] + │ └── columns:11 [as=columns:18, outer=(11)] └── 1 # -------------------------------------------------- @@ -673,7 +673,7 @@ norm SELECT f, f*2.0 AS r FROM (SELECT f, s FROM a GROUP BY f, s OFFSET 5 LIMIT 5) a ---- project - ├── columns: f:3 r:5 + ├── columns: f:3 r:6 ├── cardinality: [0 - 5] ├── immutable ├── offset @@ -695,7 +695,7 @@ project │ │ └── 10 │ └── 5 └── projections - └── f:3 * 2.0 [as=r:5, outer=(3), immutable] + └── f:3 * 2.0 [as=r:6, outer=(3), immutable] # -------------------------------------------------- # PruneJoinLeftCols @@ -706,44 +706,44 @@ norm expect=PruneJoinLeftCols SELECT a.i, xy.* FROM a INNER JOIN xy ON a.k=xy.x ---- project - ├── columns: i:2 x:5!null y:6 - ├── key: (5) - ├── fd: (5)-->(2,6) + ├── columns: i:2 x:6!null y:7 + ├── key: (6) + ├── fd: (6)-->(2,7) └── inner-join (hash) - ├── columns: k:1!null i:2 x:5!null y:6 + ├── columns: k:1!null i:2 x:6!null y:7 ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (5) - ├── fd: (1)-->(2), (5)-->(6), (1)==(5), (5)==(1) + ├── key: (6) + ├── fd: (1)-->(2), (6)-->(7), (1)==(6), (6)==(1) ├── scan a │ ├── columns: k:1!null i:2 │ ├── key: (1) │ └── fd: (1)-->(2) ├── scan xy - │ ├── columns: x:5!null y:6 - │ ├── key: (5) - │ └── fd: (5)-->(6) + │ ├── columns: x:6!null y:7 + │ ├── key: (6) + │ └── fd: (6)-->(7) └── filters - └── k:1 = x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] # Columns used by both projection and on condition, left join. norm expect=PruneJoinLeftCols SELECT a.k, a.i, xy.* FROM a LEFT JOIN xy ON a.k=xy.x AND a.i<5 ---- left-join (hash) - ├── columns: k:1!null i:2 x:5 y:6 + ├── columns: k:1!null i:2 x:6 y:7 ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) ├── key: (1) - ├── fd: (1)-->(2,5,6), (5)-->(6) + ├── fd: (1)-->(2,6,7), (6)-->(7) ├── scan a │ ├── columns: k:1!null i:2 │ ├── key: (1) │ └── fd: (1)-->(2) ├── scan xy - │ ├── columns: x:5!null y:6 - │ ├── key: (5) - │ └── fd: (5)-->(6) + │ ├── columns: x:6!null y:7 + │ ├── key: (6) + │ └── fd: (6)-->(7) └── filters - ├── k:1 = x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + ├── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] └── i:2 < 5 [outer=(2), constraints=(/2: (/NULL - /4]; tight)] # Columns needed only by projection, full join. @@ -751,36 +751,36 @@ norm expect=PruneJoinLeftCols SELECT a.k+1 AS r, xy.* FROM a FULL JOIN xy ON True ---- project - ├── columns: r:7 x:5 y:6 + ├── columns: r:9 x:6 y:7 ├── immutable - ├── fd: (5)-->(6) + ├── fd: (6)-->(7) ├── full-join (cross) - │ ├── columns: k:1 x:5 y:6 - │ ├── key: (1,5) - │ ├── fd: (5)-->(6) + │ ├── columns: k:1 x:6 y:7 + │ ├── key: (1,6) + │ ├── fd: (6)-->(7) │ ├── scan a │ │ ├── columns: k:1!null │ │ └── key: (1) │ ├── scan xy - │ │ ├── columns: x:5!null y:6 - │ │ ├── key: (5) - │ │ └── fd: (5)-->(6) + │ │ ├── columns: x:6!null y:7 + │ │ ├── key: (6) + │ │ └── fd: (6)-->(7) │ └── filters (true) └── projections - └── k:1 + 1 [as=r:7, outer=(1), immutable] + └── k:1 + 1 [as=r:9, outer=(1), immutable] # No columns needed from left side of join. norm expect=PruneJoinLeftCols SELECT xy.* FROM a, xy ---- inner-join (cross) - ├── columns: x:5!null y:6 - ├── fd: (5)-->(6) + ├── columns: x:6!null y:7 + ├── fd: (6)-->(7) ├── scan a ├── scan xy - │ ├── columns: x:5!null y:6 - │ ├── key: (5) - │ └── fd: (5)-->(6) + │ ├── columns: x:6!null y:7 + │ ├── key: (6) + │ └── fd: (6)-->(7) └── filters (true) # Computed columns. @@ -788,20 +788,20 @@ norm expect=PruneJoinLeftCols SELECT a.k+1 AS r, a.i/2 AS s, xy.* FROM a INNER JOIN xy ON a.k*a.k=xy.x AND a.s||'o'='foo' ---- project - ├── columns: r:8!null s:9 x:5!null y:6 + ├── columns: r:10!null s:11 x:6!null y:7 ├── immutable - ├── fd: (5)-->(6) + ├── fd: (6)-->(7) ├── inner-join (hash) - │ ├── columns: k:1!null i:2 x:5!null y:6 column7:7!null + │ ├── columns: k:1!null i:2 x:6!null y:7 column9:9!null │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(2,7), (5)-->(6), (5)==(7), (7)==(5) + │ ├── fd: (1)-->(2,9), (6)-->(7), (6)==(9), (9)==(6) │ ├── project - │ │ ├── columns: column7:7!null k:1!null i:2 + │ │ ├── columns: column9:9!null k:1!null i:2 │ │ ├── immutable │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2,7) + │ │ ├── fd: (1)-->(2,9) │ │ ├── select │ │ │ ├── columns: k:1!null i:2 a.s:4 │ │ │ ├── immutable @@ -814,16 +814,16 @@ project │ │ │ └── filters │ │ │ └── (a.s:4 || 'o') = 'foo' [outer=(4), immutable] │ │ └── projections - │ │ └── k:1 * k:1 [as=column7:7, outer=(1), immutable] + │ │ └── k:1 * k:1 [as=column9:9, outer=(1), immutable] │ ├── scan xy - │ │ ├── columns: x:5!null y:6 - │ │ ├── key: (5) - │ │ └── fd: (5)-->(6) + │ │ ├── columns: x:6!null y:7 + │ │ ├── key: (6) + │ │ └── fd: (6)-->(7) │ └── filters - │ └── column7:7 = x:5 [outer=(5,7), constraints=(/5: (/NULL - ]; /7: (/NULL - ]), fd=(5)==(7), (7)==(5)] + │ └── column9:9 = x:6 [outer=(6,9), constraints=(/6: (/NULL - ]; /9: (/NULL - ]), fd=(6)==(9), (9)==(6)] └── projections - ├── k:1 + 1 [as=r:8, outer=(1), immutable] - └── i:2 / 2 [as=s:9, outer=(2)] + ├── k:1 + 1 [as=r:10, outer=(1), immutable] + └── i:2 / 2 [as=s:11, outer=(2)] # Join that is nested in another join. norm expect=PruneJoinLeftCols @@ -836,33 +836,33 @@ INNER JOIN xy ON a.i < xy.y ---- project - ├── columns: k:1!null x:7!null y:8!null - ├── key: (1,7) - ├── fd: (7)-->(8) + ├── columns: k:1!null x:9!null y:10!null + ├── key: (1,9) + ├── fd: (9)-->(10) └── inner-join (cross) - ├── columns: k:1!null i:2!null x:5!null x:7!null y:8!null - ├── key: (5,7) - ├── fd: (1)-->(2), (1)==(5), (5)==(1), (7)-->(8) + ├── columns: k:1!null i:2!null x:6!null x:9!null y:10!null + ├── key: (6,9) + ├── fd: (1)-->(2), (1)==(6), (6)==(1), (9)-->(10) ├── inner-join (hash) - │ ├── columns: k:1!null i:2 x:5!null + │ ├── columns: k:1!null i:2 x:6!null │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - │ ├── key: (5) - │ ├── fd: (1)-->(2), (1)==(5), (5)==(1) + │ ├── key: (6) + │ ├── fd: (1)-->(2), (1)==(6), (6)==(1) │ ├── scan a │ │ ├── columns: k:1!null i:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ ├── scan xy - │ │ ├── columns: x:5!null - │ │ └── key: (5) + │ │ ├── columns: x:6!null + │ │ └── key: (6) │ └── filters - │ └── k:1 = x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + │ └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] ├── scan xy - │ ├── columns: x:7!null y:8 - │ ├── key: (7) - │ └── fd: (7)-->(8) + │ ├── columns: x:9!null y:10 + │ ├── key: (9) + │ └── fd: (9)-->(10) └── filters - └── i:2 < y:8 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ])] + └── i:2 < y:10 [outer=(2,10), constraints=(/2: (/NULL - ]; /10: (/NULL - ])] # ApplyJoin operator. norm expect=PruneJoinLeftCols @@ -876,19 +876,19 @@ project ├── key: (1) ├── fd: (1)-->(2) └── select - ├── columns: k:1!null i:2 r:7!null + ├── columns: k:1!null i:2 r:9!null ├── immutable ├── key: (1) - ├── fd: ()-->(7), (1)-->(2) + ├── fd: ()-->(9), (1)-->(2) ├── ensure-distinct-on - │ ├── columns: k:1!null i:2 r:7 + │ ├── columns: k:1!null i:2 r:9 │ ├── grouping columns: k:1!null │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(2,7) + │ ├── fd: (1)-->(2,9) │ ├── left-join-apply - │ │ ├── columns: k:1!null i:2 y:6 r:7 + │ │ ├── columns: k:1!null i:2 y:7 r:9 │ │ ├── immutable │ │ ├── fd: (1)-->(2) │ │ ├── scan a @@ -896,23 +896,23 @@ project │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2) │ │ ├── project - │ │ │ ├── columns: r:7 y:6 + │ │ │ ├── columns: r:9 y:7 │ │ │ ├── outer: (1) │ │ │ ├── immutable - │ │ │ ├── fd: ()-->(7) + │ │ │ ├── fd: ()-->(9) │ │ │ ├── scan xy - │ │ │ │ └── columns: y:6 + │ │ │ │ └── columns: y:7 │ │ │ └── projections - │ │ │ └── k:1 + 1 [as=r:7, outer=(1), immutable] + │ │ │ └── k:1 + 1 [as=r:9, outer=(1), immutable] │ │ └── filters - │ │ └── y:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ │ └── y:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] │ └── aggregations │ ├── const-agg [as=i:2, outer=(2)] │ │ └── i:2 - │ └── const-agg [as=r:7, outer=(7)] - │ └── r:7 + │ └── const-agg [as=r:9, outer=(9)] + │ └── r:9 └── filters - └── r:7 = 1 [outer=(7), constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] + └── r:9 = 1 [outer=(9), constraints=(/9: [/1 - /1]; tight), fd=()-->(9)] # SemiJoin operator. norm expect=PruneJoinLeftCols @@ -937,15 +937,15 @@ project │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ ├── scan xy - │ │ ├── columns: x:7!null - │ │ └── key: (7) + │ │ ├── columns: x:9!null + │ │ └── key: (9) │ └── filters - │ └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + │ └── k:1 = x:9 [outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ]), fd=(1)==(9), (9)==(1)] ├── scan xy - │ ├── columns: x:5!null - │ └── key: (5) + │ ├── columns: x:6!null + │ └── key: (6) └── filters - └── k:1 = x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] # AntiJoin operator. norm expect=PruneJoinLeftCols @@ -970,15 +970,15 @@ project │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ ├── scan xy - │ │ ├── columns: x:7!null - │ │ └── key: (7) + │ │ ├── columns: x:9!null + │ │ └── key: (9) │ └── filters - │ └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + │ └── k:1 = x:9 [outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ]), fd=(1)==(9), (9)==(1)] ├── scan xy - │ ├── columns: x:5!null - │ └── key: (5) + │ ├── columns: x:6!null + │ └── key: (6) └── filters - └── k:1 = x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] # -------------------------------------------------- # PruneJoinRightCols @@ -989,73 +989,73 @@ norm expect=PruneJoinRightCols SELECT xy.*, a.i FROM xy INNER JOIN a ON xy.x=a.k ---- project - ├── columns: x:1!null y:2 i:4 + ├── columns: x:1!null y:2 i:5 ├── key: (1) - ├── fd: (1)-->(2,4) + ├── fd: (1)-->(2,5) └── inner-join (hash) - ├── columns: x:1!null y:2 k:3!null i:4 + ├── columns: x:1!null y:2 k:4!null i:5 ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (3) - ├── fd: (1)-->(2), (3)-->(4), (1)==(3), (3)==(1) + ├── key: (4) + ├── fd: (1)-->(2), (4)-->(5), (1)==(4), (4)==(1) ├── scan xy │ ├── columns: x:1!null y:2 │ ├── key: (1) │ └── fd: (1)-->(2) ├── scan a - │ ├── columns: k:3!null i:4 - │ ├── key: (3) - │ └── fd: (3)-->(4) + │ ├── columns: k:4!null i:5 + │ ├── key: (4) + │ └── fd: (4)-->(5) └── filters - └── x:1 = k:3 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + └── x:1 = k:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] # Columns used by both projection and on condition, left join. norm expect=PruneJoinRightCols SELECT xy.*, a.k, a.i FROM xy LEFT JOIN a ON xy.x=a.k AND a.i(2-4), (3)-->(4) + ├── fd: (1)-->(2,4,5), (4)-->(5) ├── scan xy │ ├── columns: x:1!null y:2 │ ├── key: (1) │ └── fd: (1)-->(2) ├── select - │ ├── columns: k:3!null i:4!null - │ ├── key: (3) - │ ├── fd: (3)-->(4) + │ ├── columns: k:4!null i:5!null + │ ├── key: (4) + │ ├── fd: (4)-->(5) │ ├── scan a - │ │ ├── columns: k:3!null i:4 - │ │ ├── key: (3) - │ │ └── fd: (3)-->(4) + │ │ ├── columns: k:4!null i:5 + │ │ ├── key: (4) + │ │ └── fd: (4)-->(5) │ └── filters - │ └── i:4 < k:3 [outer=(3,4), constraints=(/3: (/NULL - ]; /4: (/NULL - ])] + │ └── i:5 < k:4 [outer=(4,5), constraints=(/4: (/NULL - ]; /5: (/NULL - ])] └── filters - └── x:1 = k:3 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + └── x:1 = k:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] # Columns needed only by projection, full join. norm expect=PruneJoinRightCols SELECT xy.*, a.k+1 AS r FROM xy FULL JOIN a ON True ---- project - ├── columns: x:1 y:2 r:7 + ├── columns: x:1 y:2 r:9 ├── immutable ├── fd: (1)-->(2) ├── full-join (cross) - │ ├── columns: x:1 y:2 k:3 - │ ├── key: (1,3) + │ ├── columns: x:1 y:2 k:4 + │ ├── key: (1,4) │ ├── fd: (1)-->(2) │ ├── scan xy │ │ ├── columns: x:1!null y:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ ├── scan a - │ │ ├── columns: k:3!null - │ │ └── key: (3) + │ │ ├── columns: k:4!null + │ │ └── key: (4) │ └── filters (true) └── projections - └── k:3 + 1 [as=r:7, outer=(3), immutable] + └── k:4 + 1 [as=r:9, outer=(4), immutable] # No columns needed from right side of join. norm expect=PruneJoinRightCols @@ -1076,42 +1076,42 @@ norm expect=PruneJoinRightCols SELECT xy.*, a.k+1 AS r, a.i/2 AS s FROM xy INNER JOIN a ON xy.x=a.k*a.k AND a.s||'o'='foo' ---- project - ├── columns: x:1!null y:2 r:8!null s:9 + ├── columns: x:1!null y:2 r:10!null s:11 ├── immutable ├── fd: (1)-->(2) ├── inner-join (hash) - │ ├── columns: x:1!null y:2 k:3!null i:4 column7:7!null + │ ├── columns: x:1!null y:2 k:4!null i:5 column9:9!null │ ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) │ ├── immutable - │ ├── key: (3) - │ ├── fd: (1)-->(2), (3)-->(4,7), (1)==(7), (7)==(1) + │ ├── key: (4) + │ ├── fd: (1)-->(2), (4)-->(5,9), (1)==(9), (9)==(1) │ ├── scan xy │ │ ├── columns: x:1!null y:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ ├── project - │ │ ├── columns: column7:7!null k:3!null i:4 + │ │ ├── columns: column9:9!null k:4!null i:5 │ │ ├── immutable - │ │ ├── key: (3) - │ │ ├── fd: (3)-->(4,7) + │ │ ├── key: (4) + │ │ ├── fd: (4)-->(5,9) │ │ ├── select - │ │ │ ├── columns: k:3!null i:4 a.s:6 + │ │ │ ├── columns: k:4!null i:5 a.s:7 │ │ │ ├── immutable - │ │ │ ├── key: (3) - │ │ │ ├── fd: (3)-->(4,6) + │ │ │ ├── key: (4) + │ │ │ ├── fd: (4)-->(5,7) │ │ │ ├── scan a - │ │ │ │ ├── columns: k:3!null i:4 a.s:6 - │ │ │ │ ├── key: (3) - │ │ │ │ └── fd: (3)-->(4,6) + │ │ │ │ ├── columns: k:4!null i:5 a.s:7 + │ │ │ │ ├── key: (4) + │ │ │ │ └── fd: (4)-->(5,7) │ │ │ └── filters - │ │ │ └── (a.s:6 || 'o') = 'foo' [outer=(6), immutable] + │ │ │ └── (a.s:7 || 'o') = 'foo' [outer=(7), immutable] │ │ └── projections - │ │ └── k:3 * k:3 [as=column7:7, outer=(3), immutable] + │ │ └── k:4 * k:4 [as=column9:9, outer=(4), immutable] │ └── filters - │ └── x:1 = column7:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + │ └── x:1 = column9:9 [outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ]), fd=(1)==(9), (9)==(1)] └── projections - ├── k:3 + 1 [as=r:8, outer=(3), immutable] - └── i:4 / 2 [as=s:9, outer=(4)] + ├── k:4 + 1 [as=r:10, outer=(4), immutable] + └── i:5 / 2 [as=s:11, outer=(5)] # Join that is nested in another join. norm expect=PruneJoinRightCols @@ -1124,33 +1124,33 @@ INNER JOIN ON a.y < xy.y ---- project - ├── columns: k:3!null x:1!null y:2!null - ├── key: (1,3) + ├── columns: k:4!null x:1!null y:2!null + ├── key: (1,4) ├── fd: (1)-->(2) └── inner-join (cross) - ├── columns: x:1!null y:2!null k:3!null x:7!null y:8!null - ├── key: (1,7) - ├── fd: (1)-->(2), (7)-->(8), (3)==(7), (7)==(3) + ├── columns: x:1!null y:2!null k:4!null x:9!null y:10!null + ├── key: (1,9) + ├── fd: (1)-->(2), (9)-->(10), (4)==(9), (9)==(4) ├── scan xy │ ├── columns: x:1!null y:2 │ ├── key: (1) │ └── fd: (1)-->(2) ├── inner-join (hash) - │ ├── columns: k:3!null x:7!null y:8 + │ ├── columns: k:4!null x:9!null y:10 │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - │ ├── key: (7) - │ ├── fd: (7)-->(8), (3)==(7), (7)==(3) + │ ├── key: (9) + │ ├── fd: (9)-->(10), (4)==(9), (9)==(4) │ ├── scan a - │ │ ├── columns: k:3!null - │ │ └── key: (3) + │ │ ├── columns: k:4!null + │ │ └── key: (4) │ ├── scan xy - │ │ ├── columns: x:7!null y:8 - │ │ ├── key: (7) - │ │ └── fd: (7)-->(8) + │ │ ├── columns: x:9!null y:10 + │ │ ├── key: (9) + │ │ └── fd: (9)-->(10) │ └── filters - │ └── k:3 = x:7 [outer=(3,7), constraints=(/3: (/NULL - ]; /7: (/NULL - ]), fd=(3)==(7), (7)==(3)] + │ └── k:4 = x:9 [outer=(4,9), constraints=(/4: (/NULL - ]; /9: (/NULL - ]), fd=(4)==(9), (9)==(4)] └── filters - └── y:8 < y:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ])] + └── y:10 < y:2 [outer=(2,10), constraints=(/2: (/NULL - ]; /10: (/NULL - ])] # -------------------------------------------------- # PruneJoinLeftCols + PruneJoinRightCols @@ -1161,39 +1161,39 @@ norm expect=(PruneJoinLeftCols,PruneJoinRightCols) SELECT 1 r FROM a,xy ---- project - ├── columns: r:7!null - ├── fd: ()-->(7) + ├── columns: r:9!null + ├── fd: ()-->(9) ├── inner-join (cross) │ ├── scan a │ ├── scan xy │ └── filters (true) └── projections - └── 1 [as=r:7] + └── 1 [as=r:9] # Subset of columns needed by each side of join. norm expect=(PruneJoinLeftCols,PruneJoinRightCols) SELECT a.k, xy.x, a.k+xy.x AS r FROM a LEFT JOIN xy ON a.k=xy.x ---- project - ├── columns: k:1!null x:5 r:7 + ├── columns: k:1!null x:6 r:9 ├── immutable ├── key: (1) - ├── fd: (1)-->(5,7) + ├── fd: (1)-->(6,9) ├── left-join (hash) - │ ├── columns: k:1!null x:5 + │ ├── columns: k:1!null x:6 │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ ├── key: (1) - │ ├── fd: (1)-->(5) + │ ├── fd: (1)-->(6) │ ├── scan a │ │ ├── columns: k:1!null │ │ └── key: (1) │ ├── scan xy - │ │ ├── columns: x:5!null - │ │ └── key: (5) + │ │ ├── columns: x:6!null + │ │ └── key: (6) │ └── filters - │ └── k:1 = x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + │ └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] └── projections - └── k:1 + x:5 [as=r:7, outer=(1,5), immutable] + └── k:1 + x:6 [as=r:9, outer=(1,6), immutable] # -------------------------------------------------- # PruneAggCols @@ -1215,14 +1215,14 @@ norm expect=PruneAggCols SELECT s, sumi FROM (SELECT sum(i) sumi, s, min(s||'foo') FROM a GROUP BY s) a ---- group-by - ├── columns: s:4 sumi:5 + ├── columns: s:4 sumi:6 ├── grouping columns: s:4 ├── key: (4) - ├── fd: (4)-->(5) + ├── fd: (4)-->(6) ├── scan a │ └── columns: i:2 s:4 └── aggregations - └── sum [as=sum:5, outer=(2)] + └── sum [as=sum:6, outer=(2)] └── i:2 # No aggregates to discard. @@ -1230,8 +1230,8 @@ norm expect-not=PruneAggCols SELECT 1 r FROM (SELECT s FROM a GROUP BY s) a ---- project - ├── columns: r:5!null - ├── fd: ()-->(5) + ├── columns: r:6!null + ├── fd: ()-->(6) ├── distinct-on │ ├── columns: s:4 │ ├── grouping columns: s:4 @@ -1239,21 +1239,21 @@ project │ └── scan a │ └── columns: s:4 └── projections - └── 1 [as=r:5] + └── 1 [as=r:6] # Scalar GroupBy case. norm expect=PruneAggCols SELECT sumi FROM (SELECT sum(i) sumi, min(s||'foo') FROM a) a ---- scalar-group-by - ├── columns: sumi:5 + ├── columns: sumi:6 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(5) + ├── fd: ()-->(6) ├── scan a │ └── columns: i:2 └── aggregations - └── sum [as=sum:5, outer=(2)] + └── sum [as=sum:6, outer=(2)] └── i:2 # DistinctOn case. @@ -1278,55 +1278,55 @@ norm expect=PruneAggCols SELECT max((SELECT y FROM xy WHERE y=i)) FROM a ---- scalar-group-by - ├── columns: max:8 + ├── columns: max:10 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(8) + ├── fd: ()-->(10) ├── project - │ ├── columns: column7:7 + │ ├── columns: column9:9 │ ├── ensure-distinct-on - │ │ ├── columns: k:1!null y:6 + │ │ ├── columns: k:1!null y:7 │ │ ├── grouping columns: k:1!null │ │ ├── error: "more than one row returned by a subquery used as an expression" │ │ ├── key: (1) - │ │ ├── fd: (1)-->(6) + │ │ ├── fd: (1)-->(7) │ │ ├── left-join (hash) - │ │ │ ├── columns: k:1!null i:2 y:6 + │ │ │ ├── columns: k:1!null i:2 y:7 │ │ │ ├── fd: (1)-->(2) │ │ │ ├── scan a │ │ │ │ ├── columns: k:1!null i:2 │ │ │ │ ├── key: (1) │ │ │ │ └── fd: (1)-->(2) │ │ │ ├── scan xy - │ │ │ │ └── columns: y:6 + │ │ │ │ └── columns: y:7 │ │ │ └── filters - │ │ │ └── y:6 = i:2 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + │ │ │ └── y:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] │ │ └── aggregations - │ │ └── const-agg [as=y:6, outer=(6)] - │ │ └── y:6 + │ │ └── const-agg [as=y:7, outer=(7)] + │ │ └── y:7 │ └── projections - │ └── y:6 [as=column7:7, outer=(6)] + │ └── y:7 [as=column9:9, outer=(7)] └── aggregations - └── max [as=max:8, outer=(7)] - └── column7:7 + └── max [as=max:10, outer=(9)] + └── column9:9 # Columns used only by aggregation, no grouping columns. norm expect=PruneAggCols SELECT min(i), max(k), max(k) FROM a ORDER BY max(f) ---- scalar-group-by - ├── columns: min:5 max:6 max:6 + ├── columns: min:6 max:7 max:7 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(5,6) + ├── fd: ()-->(6,7) ├── scan a │ ├── columns: k:1!null i:2 │ ├── key: (1) │ └── fd: (1)-->(2) └── aggregations - ├── min [as=min:5, outer=(2)] + ├── min [as=min:6, outer=(2)] │ └── i:2 - └── max [as=max:6, outer=(1)] + └── max [as=max:7, outer=(1)] └── k:1 # -------------------------------------------------- @@ -1338,14 +1338,14 @@ norm expect=PruneGroupByCols SELECT s, sum(i) FROM a GROUP BY s, s||'foo' ---- group-by - ├── columns: s:4 sum:5 + ├── columns: s:4 sum:6 ├── grouping columns: s:4 ├── key: (4) - ├── fd: (4)-->(5) + ├── fd: (4)-->(6) ├── scan a │ └── columns: i:2 s:4 └── aggregations - └── sum [as=sum:5, outer=(2)] + └── sum [as=sum:6, outer=(2)] └── i:2 # Columns used by both grouping and aggregation should not be pruned. @@ -1353,29 +1353,29 @@ norm expect=PruneGroupByCols SELECT avg(s::int+i), s, i FROM a GROUP BY s, i, i+1 ---- group-by - ├── columns: avg:6 s:4 i:2 + ├── columns: avg:7 s:4 i:2 ├── grouping columns: i:2 s:4 ├── immutable ├── key: (2,4) - ├── fd: (2,4)-->(6) + ├── fd: (2,4)-->(7) ├── project - │ ├── columns: column5:5 i:2 s:4 + │ ├── columns: column6:6 i:2 s:4 │ ├── immutable - │ ├── fd: (2,4)-->(5) + │ ├── fd: (2,4)-->(6) │ ├── scan a │ │ └── columns: i:2 s:4 │ └── projections - │ └── i:2 + s:4::INT8 [as=column5:5, outer=(2,4), immutable] + │ └── i:2 + s:4::INT8 [as=column6:6, outer=(2,4), immutable] └── aggregations - └── avg [as=avg:6, outer=(5)] - └── column5:5 + └── avg [as=avg:7, outer=(6)] + └── column6:6 # Columns used only by groupings, no aggregation columns. norm expect=PruneGroupByCols SELECT s, i+1 AS r FROM a GROUP BY i, s, s||'foo' ---- project - ├── columns: s:4 r:6 + ├── columns: s:4 r:7 ├── immutable ├── distinct-on │ ├── columns: i:2 s:4 @@ -1384,32 +1384,32 @@ project │ └── scan a │ └── columns: i:2 s:4 └── projections - └── i:2 + 1 [as=r:6, outer=(2), immutable] + └── i:2 + 1 [as=r:7, outer=(2), immutable] # Groupby a groupby. norm expect=PruneGroupByCols SELECT min(sm), i FROM (SELECT s, i, sum(k) sm, avg(k) av FROM a GROUP BY i, s) a GROUP BY i, i+1 ---- group-by - ├── columns: min:7!null i:2 + ├── columns: min:8!null i:2 ├── grouping columns: i:2 ├── key: (2) - ├── fd: (2)-->(7) + ├── fd: (2)-->(8) ├── group-by - │ ├── columns: i:2 s:4 sum:5!null + │ ├── columns: i:2 s:4 sum:6!null │ ├── grouping columns: i:2 s:4 │ ├── key: (2,4) - │ ├── fd: (2,4)-->(5) + │ ├── fd: (2,4)-->(6) │ ├── scan a │ │ ├── columns: k:1!null i:2 s:4 │ │ ├── key: (1) │ │ └── fd: (1)-->(2,4) │ └── aggregations - │ └── sum [as=sum:5, outer=(1)] + │ └── sum [as=sum:6, outer=(1)] │ └── k:1 └── aggregations - └── min [as=min:7, outer=(5)] - └── sum:5 + └── min [as=min:8, outer=(6)] + └── sum:6 # Distinct (GroupBy operator with no aggregates). norm expect=PruneGroupByCols @@ -1431,21 +1431,21 @@ norm expect=PruneGroupByCols SELECT icnt FROM (SELECT count(i+1) AS icnt, count(k+1) FROM a); ---- scalar-group-by - ├── columns: icnt:6!null + ├── columns: icnt:7!null ├── cardinality: [1 - 1] ├── immutable ├── key: () - ├── fd: ()-->(6) + ├── fd: ()-->(7) ├── project - │ ├── columns: column5:5 + │ ├── columns: column6:6 │ ├── immutable │ ├── scan a │ │ └── columns: i:2 │ └── projections - │ └── i:2 + 1 [as=column5:5, outer=(2), immutable] + │ └── i:2 + 1 [as=column6:6, outer=(2), immutable] └── aggregations - └── count [as=count:6, outer=(5)] - └── column5:5 + └── count [as=count:7, outer=(6)] + └── column6:6 # -------------------------------------------------- # PruneValuesCols @@ -1504,14 +1504,14 @@ norm SELECT a.k, xy.y FROM a INNER JOIN xy ON a.k=xy.x WHERE a.i < 5 ---- project - ├── columns: k:1!null y:6 + ├── columns: k:1!null y:7 ├── key: (1) - ├── fd: (1)-->(6) + ├── fd: (1)-->(7) └── inner-join (hash) - ├── columns: k:1!null i:2!null x:5!null y:6 + ├── columns: k:1!null i:2!null x:6!null y:7 ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (5) - ├── fd: (1)-->(2), (5)-->(6), (1)==(5), (5)==(1) + ├── key: (6) + ├── fd: (1)-->(2), (6)-->(7), (1)==(6), (6)==(1) ├── select │ ├── columns: k:1!null i:2!null │ ├── key: (1) @@ -1523,11 +1523,11 @@ project │ └── filters │ └── i:2 < 5 [outer=(2), constraints=(/2: (/NULL - /4]; tight)] ├── scan xy - │ ├── columns: x:5!null y:6 - │ ├── key: (5) - │ └── fd: (5)-->(6) + │ ├── columns: x:6!null y:7 + │ ├── key: (6) + │ └── fd: (6)-->(7) └── filters - └── k:1 = x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] norm SELECT k FROM (SELECT k, min(s) FROM a GROUP BY k HAVING sum(i) > 5) @@ -1537,24 +1537,24 @@ project ├── immutable ├── key: (1) └── select - ├── columns: k:1!null sum:6!null + ├── columns: k:1!null sum:7!null ├── immutable ├── key: (1) - ├── fd: (1)-->(6) + ├── fd: (1)-->(7) ├── group-by - │ ├── columns: k:1!null sum:6 + │ ├── columns: k:1!null sum:7 │ ├── grouping columns: k:1!null │ ├── key: (1) - │ ├── fd: (1)-->(6) + │ ├── fd: (1)-->(7) │ ├── scan a │ │ ├── columns: k:1!null i:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ └── aggregations - │ └── sum [as=sum:6, outer=(2)] + │ └── sum [as=sum:7, outer=(2)] │ └── i:2 └── filters - └── sum:6 > 5 [outer=(6), immutable, constraints=(/6: (/5 - ]; tight)] + └── sum:7 > 5 [outer=(7), immutable, constraints=(/7: (/5 - ]; tight)] # -------------------------------------------------- # PruneOrdinalityCols @@ -1565,9 +1565,9 @@ SELECT i, s FROM a WITH ORDINALITY project ├── columns: i:2 s:4 └── ordinality - ├── columns: i:2 s:4 ordinality:5!null - ├── key: (5) - ├── fd: (5)-->(2,4) + ├── columns: i:2 s:4 ordinality:6!null + ├── key: (6) + ├── fd: (6)-->(2,4) └── scan a └── columns: i:2 s:4 @@ -1578,9 +1578,9 @@ SELECT i, s FROM (SELECT * FROM a ORDER BY f) WITH ORDINALITY project ├── columns: i:2 s:4 └── ordinality - ├── columns: i:2 f:3 s:4 ordinality:5!null - ├── key: (5) - ├── fd: (5)-->(2-4) + ├── columns: i:2 f:3 s:4 ordinality:6!null + ├── key: (6) + ├── fd: (6)-->(2-4) └── sort ├── columns: i:2 f:3 s:4 ├── ordering: +3 @@ -1594,7 +1594,7 @@ norm expect=PruneExplainCols EXPLAIN SELECT a FROM abcde WHERE b=1 AND c IS NOT NULL ORDER BY c, d ---- explain - ├── columns: tree:6 field:7 description:8 + ├── columns: tree:7 field:8 description:9 └── sort ├── columns: a:1!null [hidden: c:3!null] ├── key: (1) @@ -1623,11 +1623,11 @@ norm expect=PruneProjectSetCols SELECT a, b, generate_series(c, 10) FROM abcde ---- project - ├── columns: a:1!null b:2 generate_series:6 + ├── columns: a:1!null b:2 generate_series:7 ├── immutable ├── fd: (1)-->(2) └── project-set - ├── columns: a:1!null b:2 c:3 generate_series:6 + ├── columns: a:1!null b:2 c:3 generate_series:7 ├── immutable ├── fd: (1)-->(2,3), (2,3)~~>(1) ├── scan abcde @@ -1646,7 +1646,7 @@ distinct-on ├── immutable ├── key: (1) └── project-set - ├── columns: k:1!null i:2 s:4 generate_series:5 length:6 + ├── columns: k:1!null i:2 s:4 generate_series:6 length:7 ├── immutable ├── fd: (1)-->(2,4) ├── scan a @@ -1665,49 +1665,49 @@ norm expect=PruneWindowInputCols SELECT rank() OVER () FROM a ---- window partition=() - ├── columns: rank:5 + ├── columns: rank:6 ├── scan a └── windows - └── rank [as=rank:5] + └── rank [as=rank:6] norm expect=PruneWindowInputCols SELECT ntile(1) OVER () FROM a ---- project - ├── columns: ntile:5 + ├── columns: ntile:6 └── window partition=() - ├── columns: ntile:5 ntile_1_arg1:6!null - ├── fd: ()-->(6) + ├── columns: ntile:6 ntile_1_arg1:7!null + ├── fd: ()-->(7) ├── project - │ ├── columns: ntile_1_arg1:6!null - │ ├── fd: ()-->(6) + │ ├── columns: ntile_1_arg1:7!null + │ ├── fd: ()-->(7) │ ├── scan a │ └── projections - │ └── 1 [as=ntile_1_arg1:6] + │ └── 1 [as=ntile_1_arg1:7] └── windows - └── ntile [as=ntile:5, outer=(6)] - └── ntile_1_arg1:6 + └── ntile [as=ntile:6, outer=(7)] + └── ntile_1_arg1:7 norm expect=PruneWindowInputCols format=show-all SELECT ntile(i) OVER () FROM a ---- project - ├── columns: ntile:5(int) + ├── columns: ntile:6(int) ├── stats: [rows=1000] ├── cost: 1060.04 - ├── prune: (5) + ├── prune: (6) └── window partition=() - ├── columns: t.public.a.i:2(int) ntile:5(int) + ├── columns: t.public.a.i:2(int) ntile:6(int) ├── stats: [rows=1000] ├── cost: 1050.03 - ├── prune: (5) + ├── prune: (6) ├── scan t.public.a │ ├── columns: t.public.a.i:2(int) │ ├── stats: [rows=1000] │ ├── cost: 1050.02 │ └── prune: (2) └── windows - └── ntile [as=ntile:5, type=int, outer=(2)] + └── ntile [as=ntile:6, type=int, outer=(2)] └── variable: t.public.a.i:2 [type=int] # Ensure filter cols don't get pruned. @@ -1718,27 +1718,27 @@ SELECT FROM a ---- project - ├── columns: avg:5 avg:6 + ├── columns: avg:6 avg:7 └── window partition=() - ├── columns: i:2 avg:5 avg:6 avg_1_filter:7!null avg_2_filter:8!null - ├── fd: ()-->(7,8) + ├── columns: i:2 avg:6 avg:7 avg_1_filter:8!null avg_2_filter:9!null + ├── fd: ()-->(8,9) ├── project - │ ├── columns: avg_1_filter:7!null avg_2_filter:8!null i:2 - │ ├── fd: ()-->(7,8) + │ ├── columns: avg_1_filter:8!null avg_2_filter:9!null i:2 + │ ├── fd: ()-->(8,9) │ ├── scan a │ │ └── columns: i:2 │ └── projections - │ ├── true [as=avg_1_filter:7] - │ └── false [as=avg_2_filter:8] + │ ├── true [as=avg_1_filter:8] + │ └── false [as=avg_2_filter:9] └── windows - ├── agg-filter [as=avg:5, outer=(2,7)] + ├── agg-filter [as=avg:6, outer=(2,8)] │ ├── avg │ │ └── i:2 - │ └── avg_1_filter:7 - └── agg-filter [as=avg:6, outer=(2,8)] + │ └── avg_1_filter:8 + └── agg-filter [as=avg:7, outer=(2,9)] ├── avg │ └── i:2 - └── avg_2_filter:8 + └── avg_2_filter:9 # -------------------------------------------------- # PruneWindowOutputCols @@ -1748,52 +1748,52 @@ norm expect=PruneWindowOutputCols SELECT x FROM (SELECT ntile(1) OVER () AS x, ntile(2) OVER () FROM a) ---- project - ├── columns: x:5 + ├── columns: x:6 └── window partition=() - ├── columns: ntile:5 ntile_1_arg1:7!null - ├── fd: ()-->(7) + ├── columns: ntile:6 ntile_1_arg1:8!null + ├── fd: ()-->(8) ├── project - │ ├── columns: ntile_1_arg1:7!null - │ ├── fd: ()-->(7) + │ ├── columns: ntile_1_arg1:8!null + │ ├── fd: ()-->(8) │ ├── scan a │ └── projections - │ └── 1 [as=ntile_1_arg1:7] + │ └── 1 [as=ntile_1_arg1:8] └── windows - └── ntile [as=ntile:5, outer=(7)] - └── ntile_1_arg1:7 + └── ntile [as=ntile:6, outer=(8)] + └── ntile_1_arg1:8 norm expect=(PruneWindowOutputCols,EliminateWindow) SELECT 1 FROM (SELECT ntile(1) OVER () FROM a) ---- project - ├── columns: "?column?":7!null - ├── fd: ()-->(7) + ├── columns: "?column?":8!null + ├── fd: ()-->(8) ├── scan a └── projections - └── 1 [as="?column?":7] + └── 1 [as="?column?":8] norm expect=(PruneWindowOutputCols,EliminateWindow) SELECT 1 FROM (SELECT x FROM (SELECT ntile(1) OVER () AS x, ntile(2) OVER () FROM a)) ---- project - ├── columns: "?column?":9!null - ├── fd: ()-->(9) + ├── columns: "?column?":10!null + ├── fd: ()-->(10) ├── scan a └── projections - └── 1 [as="?column?":9] + └── 1 [as="?column?":10] norm expect-not=PruneWindowOutputCols SELECT round(avg(k) OVER (PARTITION BY f ORDER BY s)) FROM a ORDER BY 1 ---- sort - ├── columns: round:6 + ├── columns: round:7 ├── immutable - ├── ordering: +6 + ├── ordering: +7 └── project - ├── columns: round:6 + ├── columns: round:7 ├── immutable ├── window partition=(3) ordering=+4 opt(3) - │ ├── columns: k:1!null f:3 s:4 avg:5 + │ ├── columns: k:1!null f:3 s:4 avg:6 │ ├── key: (1) │ ├── fd: (1)-->(3,4) │ ├── scan a @@ -1801,31 +1801,31 @@ sort │ │ ├── key: (1) │ │ └── fd: (1)-->(3,4) │ └── windows - │ └── avg [as=avg:5, outer=(1)] + │ └── avg [as=avg:6, outer=(1)] │ └── k:1 └── projections - └── round(avg:5) [as=round:6, outer=(5), immutable] + └── round(avg:6) [as=round:7, outer=(6), immutable] norm expect=(PruneWindowInputCols,PruneWindowOutputCols) format=show-all SELECT x FROM (SELECT ntile(i) OVER () x, ntile(f::int) OVER () y FROM a) ---- project - ├── columns: x:5(int) + ├── columns: x:6(int) ├── stats: [rows=1000] ├── cost: 1060.04 - ├── prune: (5) + ├── prune: (6) └── window partition=() - ├── columns: t.public.a.i:2(int) ntile:5(int) + ├── columns: t.public.a.i:2(int) ntile:6(int) ├── stats: [rows=1000] ├── cost: 1050.03 - ├── prune: (5) + ├── prune: (6) ├── scan t.public.a │ ├── columns: t.public.a.i:2(int) │ ├── stats: [rows=1000] │ ├── cost: 1050.02 │ └── prune: (2) └── windows - └── ntile [as=ntile:5, type=int, outer=(2)] + └── ntile [as=ntile:6, type=int, outer=(2)] └── variable: t.public.a.i:2 [type=int] @@ -1839,12 +1839,12 @@ DELETE FROM a ---- delete a ├── columns: - ├── fetch columns: k:5 + ├── fetch columns: k:6 ├── cardinality: [0 - 0] ├── volatile, mutations └── scan a - ├── columns: k:5!null - └── key: (5) + ├── columns: k:6!null + └── key: (6) # Prune when computed ordering column is present. norm expect=(PruneMutationFetchCols,PruneMutationInputCols) @@ -1852,40 +1852,40 @@ DELETE FROM a WHERE i > 0 ORDER BY i*2 LIMIT 10 ---- delete a ├── columns: - ├── fetch columns: k:5 + ├── fetch columns: k:6 ├── cardinality: [0 - 0] ├── volatile, mutations └── limit - ├── columns: k:5!null column9:9!null - ├── internal-ordering: +9 + ├── columns: k:6!null column11:11!null + ├── internal-ordering: +11 ├── cardinality: [0 - 10] ├── immutable - ├── key: (5) - ├── fd: (5)-->(9) + ├── key: (6) + ├── fd: (6)-->(11) ├── sort - │ ├── columns: k:5!null column9:9!null + │ ├── columns: k:6!null column11:11!null │ ├── immutable - │ ├── key: (5) - │ ├── fd: (5)-->(9) - │ ├── ordering: +9 + │ ├── key: (6) + │ ├── fd: (6)-->(11) + │ ├── ordering: +11 │ ├── limit hint: 10.00 │ └── project - │ ├── columns: column9:9!null k:5!null + │ ├── columns: column11:11!null k:6!null │ ├── immutable - │ ├── key: (5) - │ ├── fd: (5)-->(9) + │ ├── key: (6) + │ ├── fd: (6)-->(11) │ ├── select - │ │ ├── columns: k:5!null i:6!null - │ │ ├── key: (5) - │ │ ├── fd: (5)-->(6) + │ │ ├── columns: k:6!null i:7!null + │ │ ├── key: (6) + │ │ ├── fd: (6)-->(7) │ │ ├── scan a - │ │ │ ├── columns: k:5!null i:6 - │ │ │ ├── key: (5) - │ │ │ └── fd: (5)-->(6) + │ │ │ ├── columns: k:6!null i:7 + │ │ │ ├── key: (6) + │ │ │ └── fd: (6)-->(7) │ │ └── filters - │ │ └── i:6 > 0 [outer=(6), constraints=(/6: [/1 - ]; tight)] + │ │ └── i:7 > 0 [outer=(7), constraints=(/7: [/1 - ]; tight)] │ └── projections - │ └── i:6 * 2 [as=column9:9, outer=(6), immutable] + │ └── i:7 * 2 [as=column11:11, outer=(7), immutable] └── 10 # Prune when a secondary index is present on the table. @@ -1894,19 +1894,19 @@ DELETE FROM abcde WHERE a > 0 ---- delete abcde ├── columns: - ├── fetch columns: a:6 b:7 c:8 + ├── fetch columns: a:7 b:8 c:9 ├── cardinality: [0 - 0] ├── volatile, mutations └── select - ├── columns: a:6!null b:7 c:8 - ├── key: (6) - ├── fd: (6)-->(7,8), (7,8)~~>(6) + ├── columns: a:7!null b:8 c:9 + ├── key: (7) + ├── fd: (7)-->(8,9), (8,9)~~>(7) ├── scan abcde - │ ├── columns: a:6!null b:7 c:8 - │ ├── key: (6) - │ └── fd: (6)-->(7,8), (7,8)~~>(6) + │ ├── columns: a:7!null b:8 c:9 + │ ├── key: (7) + │ └── fd: (7)-->(8,9), (8,9)~~>(7) └── filters - └── a:6 > 0 [outer=(6), constraints=(/6: [/1 - ]; tight)] + └── a:7 > 0 [outer=(7), constraints=(/7: [/1 - ]; tight)] # Prune when mutation columns/indexes exist. norm expect=(PruneMutationFetchCols,PruneMutationInputCols) @@ -1914,62 +1914,62 @@ DELETE FROM mutation ---- delete mutation ├── columns: - ├── fetch columns: a:6 b:7 d:9 e:10 + ├── fetch columns: a:7 b:8 d:10 e:11 ├── cardinality: [0 - 0] ├── volatile, mutations └── scan mutation - ├── columns: a:6!null b:7 d:9 e:10 - ├── key: (6) - └── fd: (6)-->(7,9,10) + ├── columns: a:7!null b:8 d:10 e:11 + ├── key: (7) + └── fd: (7)-->(8,10,11) norm expect=(PruneMutationFetchCols,PruneMutationInputCols) DELETE FROM a RETURNING k, s ---- delete a ├── columns: k:1!null s:4 - ├── fetch columns: k:5 s:8 + ├── fetch columns: k:6 s:9 ├── volatile, mutations ├── key: (1) ├── fd: (1)-->(4) └── scan a - ├── columns: k:5!null s:8 - ├── key: (5) - └── fd: (5)-->(8) + ├── columns: k:6!null s:9 + ├── key: (6) + └── fd: (6)-->(9) # Do not prune columns that are required for evaluating partial index # predicates. -norm expect-not=(PruneMutationFetchCols,PruneMutationInputCols) +norm expect-not=(PruneMutationFetchCols) UPDATE partial_indexes SET c = 'bar' RETURNING a ---- update partial_indexes ├── columns: a:1!null - ├── fetch columns: a:4 b:5 c:6 + ├── fetch columns: a:5 b:6 c:7 ├── update-mapping: - │ └── c_new:9 => c:3 - ├── partial index put columns: partial_index_put1:10 partial_index_put2:11 - ├── partial index del columns: partial_index_del1:7 partial_index_del2:8 + │ └── c_new:11 => c:3 + ├── partial index put columns: partial_index_put1:12 partial_index_put2:13 + ├── partial index del columns: partial_index_del1:9 partial_index_del2:10 ├── volatile, mutations ├── key: (1) └── project - ├── columns: partial_index_put1:10!null partial_index_put2:11 c_new:9!null partial_index_del1:7 partial_index_del2:8 a:4!null b:5 c:6 - ├── key: (4) - ├── fd: ()-->(9,10), (4)-->(5,6,8,11), (6)-->(7) + ├── columns: partial_index_put1:12!null partial_index_put2:13 c_new:11!null partial_index_del1:9 partial_index_del2:10 a:5!null b:6 c:7 + ├── key: (5) + ├── fd: ()-->(11,12), (5)-->(6,7,10,13), (7)-->(9) ├── scan partial_indexes - │ ├── columns: a:4!null b:5 c:6 + │ ├── columns: a:5!null b:6 c:7 │ ├── partial index predicates │ │ ├── secondary: filters - │ │ │ └── c:6 = 'foo' [outer=(6), constraints=(/6: [/'foo' - /'foo']; tight), fd=()-->(6)] + │ │ │ └── c:7 = 'foo' [outer=(7), constraints=(/7: [/'foo' - /'foo']; tight), fd=()-->(7)] │ │ └── secondary: filters - │ │ ├── a:4 > b:5 [outer=(4,5), constraints=(/4: (/NULL - ]; /5: (/NULL - ])] - │ │ └── c:6 = 'bar' [outer=(6), constraints=(/6: [/'bar' - /'bar']; tight), fd=()-->(6)] - │ ├── key: (4) - │ └── fd: (4)-->(5,6) + │ │ ├── a:5 > b:6 [outer=(5,6), constraints=(/5: (/NULL - ]; /6: (/NULL - ])] + │ │ └── c:7 = 'bar' [outer=(7), constraints=(/7: [/'bar' - /'bar']; tight), fd=()-->(7)] + │ ├── key: (5) + │ └── fd: (5)-->(6,7) └── projections - ├── false [as=partial_index_put1:10] - ├── a:4 > b:5 [as=partial_index_put2:11, outer=(4,5)] - ├── 'bar' [as=c_new:9] - ├── c:6 = 'foo' [as=partial_index_del1:7, outer=(6)] - └── (a:4 > b:5) AND (c:6 = 'bar') [as=partial_index_del2:8, outer=(4-6)] + ├── false [as=partial_index_put1:12] + ├── a:5 > b:6 [as=partial_index_put2:13, outer=(5,6)] + ├── 'bar' [as=c_new:11] + ├── c:7 = 'foo' [as=partial_index_del1:9, outer=(7)] + └── (a:5 > b:6) AND (c:7 = 'bar') [as=partial_index_del2:10, outer=(5-7)] # Prune secondary family column not needed for the update. norm expect=(PruneMutationFetchCols,PruneMutationInputCols) @@ -1977,50 +1977,50 @@ UPDATE family SET b=c WHERE a > 100 ---- update "family" ├── columns: - ├── fetch columns: a:6 b:7 + ├── fetch columns: a:7 b:8 ├── update-mapping: - │ └── c:8 => b:2 + │ └── c:9 => b:2 ├── cardinality: [0 - 0] ├── volatile, mutations └── select - ├── columns: a:6!null b:7 c:8 - ├── key: (6) - ├── fd: (6)-->(7,8) + ├── columns: a:7!null b:8 c:9 + ├── key: (7) + ├── fd: (7)-->(8,9) ├── scan "family" - │ ├── columns: a:6!null b:7 c:8 - │ ├── key: (6) - │ └── fd: (6)-->(7,8) + │ ├── columns: a:7!null b:8 c:9 + │ ├── key: (7) + │ └── fd: (7)-->(8,9) └── filters - └── a:6 > 100 [outer=(6), constraints=(/6: [/101 - ]; tight)] + └── a:7 > 100 [outer=(7), constraints=(/7: [/101 - ]; tight)] # Do not prune when key column is updated. -norm expect-not=(PruneMutationFetchCols,PruneMutationInputCols) +norm expect-not=(PruneMutationFetchCols) UPDATE family SET a=a+1 WHERE a > 100 ---- update "family" ├── columns: - ├── fetch columns: a:6 b:7 c:8 d:9 e:10 + ├── fetch columns: a:7 b:8 c:9 d:10 e:11 ├── update-mapping: - │ └── a_new:11 => a:1 + │ └── a_new:13 => a:1 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: a_new:11!null a:6!null b:7 c:8 d:9 e:10 + ├── columns: a_new:13!null a:7!null b:8 c:9 d:10 e:11 ├── immutable - ├── key: (6) - ├── fd: (6)-->(7-11) + ├── key: (7) + ├── fd: (7)-->(8-11,13) ├── select - │ ├── columns: a:6!null b:7 c:8 d:9 e:10 - │ ├── key: (6) - │ ├── fd: (6)-->(7-10) + │ ├── columns: a:7!null b:8 c:9 d:10 e:11 + │ ├── key: (7) + │ ├── fd: (7)-->(8-11) │ ├── scan "family" - │ │ ├── columns: a:6!null b:7 c:8 d:9 e:10 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7-10) + │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8-11) │ └── filters - │ └── a:6 > 100 [outer=(6), constraints=(/6: [/101 - ]; tight)] + │ └── a:7 > 100 [outer=(7), constraints=(/7: [/101 - ]; tight)] └── projections - └── a:6 + 1 [as=a_new:11, outer=(6), immutable] + └── a:7 + 1 [as=a_new:13, outer=(7), immutable] # Do not prune columns that must be returned. norm expect=(PruneMutationFetchCols, PruneMutationReturnCols) @@ -2031,23 +2031,23 @@ project ├── volatile, mutations └── update "family" ├── columns: a:1!null b:2 - ├── fetch columns: a:6 b:7 c:8 d:9 + ├── fetch columns: a:7 b:8 c:9 d:10 ├── update-mapping: - │ └── c_new:11 => c:3 + │ └── c_new:13 => c:3 ├── volatile, mutations ├── key: (1) ├── fd: (1)-->(2) └── project - ├── columns: c_new:11 a:6!null b:7 c:8 d:9 + ├── columns: c_new:13 a:7!null b:8 c:9 d:10 ├── immutable - ├── key: (6) - ├── fd: (6)-->(7-9), (8)-->(11) + ├── key: (7) + ├── fd: (7)-->(8-10), (9)-->(13) ├── scan "family" - │ ├── columns: a:6!null b:7 c:8 d:9 - │ ├── key: (6) - │ └── fd: (6)-->(7-9) + │ ├── columns: a:7!null b:8 c:9 d:10 + │ ├── key: (7) + │ └── fd: (7)-->(8-10) └── projections - └── c:8 + 1 [as=c_new:11, outer=(8), immutable] + └── c:9 + 1 [as=c_new:13, outer=(9), immutable] # Prune unused upsert columns. norm expect=PruneMutationInputCols @@ -2055,49 +2055,49 @@ INSERT INTO a (k, s) VALUES (1, 'foo') ON CONFLICT (k) DO UPDATE SET i=a.i+1 ---- upsert a ├── columns: - ├── canary column: 9 - ├── fetch columns: k:9 i:10 f:11 s:12 + ├── canary column: 10 + ├── fetch columns: k:10 i:11 f:12 s:13 ├── insert-mapping: - │ ├── column1:5 => k:1 - │ ├── column7:7 => i:2 - │ ├── column8:8 => f:3 - │ └── column2:6 => s:4 + │ ├── column1:6 => k:1 + │ ├── column8:8 => i:2 + │ ├── column9:9 => f:3 + │ └── column2:7 => s:4 ├── update-mapping: - │ └── upsert_i:15 => i:2 + │ └── upsert_i:17 => i:2 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: upsert_i:15 column1:5!null column2:6!null column7:7 column8:8 k:9 i:10 f:11 s:12 + ├── columns: upsert_i:17 column1:6!null column2:7!null column8:8 column9:9 k:10 i:11 f:12 s:13 ├── cardinality: [1 - 1] ├── immutable ├── key: () - ├── fd: ()-->(5-12,15) + ├── fd: ()-->(6-13,17) ├── left-join (cross) - │ ├── columns: column1:5!null column2:6!null column7:7 column8:8 k:9 i:10 f:11 s:12 + │ ├── columns: column1:6!null column2:7!null column8:8 column9:9 k:10 i:11 f:12 s:13 │ ├── cardinality: [1 - 1] │ ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) │ ├── key: () - │ ├── fd: ()-->(5-12) + │ ├── fd: ()-->(6-13) │ ├── values - │ │ ├── columns: column1:5!null column2:6!null column7:7 column8:8 + │ │ ├── columns: column1:6!null column2:7!null column8:8 column9:9 │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(5-8) + │ │ ├── fd: ()-->(6-9) │ │ └── (1, 'foo', NULL, NULL) │ ├── select - │ │ ├── columns: k:9!null i:10 f:11 s:12 + │ │ ├── columns: k:10!null i:11 f:12 s:13 │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(9-12) + │ │ ├── fd: ()-->(10-13) │ │ ├── scan a - │ │ │ ├── columns: k:9!null i:10 f:11 s:12 - │ │ │ ├── key: (9) - │ │ │ └── fd: (9)-->(10-12) + │ │ │ ├── columns: k:10!null i:11 f:12 s:13 + │ │ │ ├── key: (10) + │ │ │ └── fd: (10)-->(11-13) │ │ └── filters - │ │ └── k:9 = 1 [outer=(9), constraints=(/9: [/1 - /1]; tight), fd=()-->(9)] + │ │ └── k:10 = 1 [outer=(10), constraints=(/10: [/1 - /1]; tight), fd=()-->(10)] │ └── filters (true) └── projections - └── CASE WHEN k:9 IS NULL THEN column7:7 ELSE i:10 + 1 END [as=upsert_i:15, outer=(7,9,10), immutable] + └── CASE WHEN k:10 IS NULL THEN column8:8 ELSE i:11 + 1 END [as=upsert_i:17, outer=(8,10,11), immutable] # Prune update columns replaced by upsert columns. # TODO(andyk): Need to also prune output columns. @@ -2106,59 +2106,59 @@ INSERT INTO a (k, s) VALUES (1, 'foo') ON CONFLICT (k) DO UPDATE SET i=a.i+1 RET ---- upsert a ├── columns: k:1!null i:2 f:3 s:4 - ├── canary column: 9 - ├── fetch columns: k:9 i:10 f:11 s:12 + ├── canary column: 10 + ├── fetch columns: k:10 i:11 f:12 s:13 ├── insert-mapping: - │ ├── column1:5 => k:1 - │ ├── column7:7 => i:2 - │ ├── column8:8 => f:3 - │ └── column2:6 => s:4 + │ ├── column1:6 => k:1 + │ ├── column8:8 => i:2 + │ ├── column9:9 => f:3 + │ └── column2:7 => s:4 ├── update-mapping: - │ └── upsert_i:15 => i:2 + │ └── upsert_i:17 => i:2 ├── return-mapping: - │ ├── upsert_k:14 => k:1 - │ ├── upsert_i:15 => i:2 - │ ├── upsert_f:16 => f:3 - │ └── upsert_s:17 => s:4 + │ ├── upsert_k:16 => k:1 + │ ├── upsert_i:17 => i:2 + │ ├── upsert_f:18 => f:3 + │ └── upsert_s:19 => s:4 ├── cardinality: [1 - 1] ├── volatile, mutations ├── key: () ├── fd: ()-->(1-4) └── project - ├── columns: upsert_k:14 upsert_i:15 upsert_f:16 upsert_s:17 column1:5!null column2:6!null column7:7 column8:8 k:9 i:10 f:11 s:12 + ├── columns: upsert_k:16 upsert_i:17 upsert_f:18 upsert_s:19 column1:6!null column2:7!null column8:8 column9:9 k:10 i:11 f:12 s:13 ├── cardinality: [1 - 1] ├── immutable ├── key: () - ├── fd: ()-->(5-12,14-17) + ├── fd: ()-->(6-13,16-19) ├── left-join (cross) - │ ├── columns: column1:5!null column2:6!null column7:7 column8:8 k:9 i:10 f:11 s:12 + │ ├── columns: column1:6!null column2:7!null column8:8 column9:9 k:10 i:11 f:12 s:13 │ ├── cardinality: [1 - 1] │ ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) │ ├── key: () - │ ├── fd: ()-->(5-12) + │ ├── fd: ()-->(6-13) │ ├── values - │ │ ├── columns: column1:5!null column2:6!null column7:7 column8:8 + │ │ ├── columns: column1:6!null column2:7!null column8:8 column9:9 │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(5-8) + │ │ ├── fd: ()-->(6-9) │ │ └── (1, 'foo', NULL, NULL) │ ├── select - │ │ ├── columns: k:9!null i:10 f:11 s:12 + │ │ ├── columns: k:10!null i:11 f:12 s:13 │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(9-12) + │ │ ├── fd: ()-->(10-13) │ │ ├── scan a - │ │ │ ├── columns: k:9!null i:10 f:11 s:12 - │ │ │ ├── key: (9) - │ │ │ └── fd: (9)-->(10-12) + │ │ │ ├── columns: k:10!null i:11 f:12 s:13 + │ │ │ ├── key: (10) + │ │ │ └── fd: (10)-->(11-13) │ │ └── filters - │ │ └── k:9 = 1 [outer=(9), constraints=(/9: [/1 - /1]; tight), fd=()-->(9)] + │ │ └── k:10 = 1 [outer=(10), constraints=(/10: [/1 - /1]; tight), fd=()-->(10)] │ └── filters (true) └── projections - ├── CASE WHEN k:9 IS NULL THEN column1:5 ELSE k:9 END [as=upsert_k:14, outer=(5,9)] - ├── CASE WHEN k:9 IS NULL THEN column7:7 ELSE i:10 + 1 END [as=upsert_i:15, outer=(7,9,10), immutable] - ├── CASE WHEN k:9 IS NULL THEN column8:8 ELSE f:11 END [as=upsert_f:16, outer=(8,9,11)] - └── CASE WHEN k:9 IS NULL THEN column2:6 ELSE s:12 END [as=upsert_s:17, outer=(6,9,12)] + ├── CASE WHEN k:10 IS NULL THEN column1:6 ELSE k:10 END [as=upsert_k:16, outer=(6,10)] + ├── CASE WHEN k:10 IS NULL THEN column8:8 ELSE i:11 + 1 END [as=upsert_i:17, outer=(8,10,11), immutable] + ├── CASE WHEN k:10 IS NULL THEN column9:9 ELSE f:12 END [as=upsert_f:18, outer=(9,10,12)] + └── CASE WHEN k:10 IS NULL THEN column2:7 ELSE s:13 END [as=upsert_s:19, outer=(7,10,13)] # Prune column in column family that is not updated. norm expect=(PruneMutationFetchCols,PruneMutationInputCols) @@ -2166,41 +2166,41 @@ UPSERT INTO family (a, b) VALUES (1, 2) ---- upsert "family" ├── columns: - ├── canary column: 9 - ├── fetch columns: a:9 b:10 + ├── canary column: 10 + ├── fetch columns: a:10 b:11 ├── insert-mapping: - │ ├── column1:6 => a:1 - │ ├── column2:7 => b:2 - │ ├── column8:8 => c:3 - │ ├── column8:8 => d:4 - │ └── column8:8 => e:5 + │ ├── column1:7 => a:1 + │ ├── column2:8 => b:2 + │ ├── column9:9 => c:3 + │ ├── column9:9 => d:4 + │ └── column9:9 => e:5 ├── update-mapping: - │ └── column2:7 => b:2 + │ └── column2:8 => b:2 ├── cardinality: [0 - 0] ├── volatile, mutations └── left-join (cross) - ├── columns: column1:6!null column2:7!null column8:8 a:9 b:10 + ├── columns: column1:7!null column2:8!null column9:9 a:10 b:11 ├── cardinality: [1 - 1] ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) ├── key: () - ├── fd: ()-->(6-10) + ├── fd: ()-->(7-11) ├── values - │ ├── columns: column1:6!null column2:7!null column8:8 + │ ├── columns: column1:7!null column2:8!null column9:9 │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(6-8) + │ ├── fd: ()-->(7-9) │ └── (1, 2, NULL) ├── select - │ ├── columns: a:9!null b:10 + │ ├── columns: a:10!null b:11 │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(9,10) + │ ├── fd: ()-->(10,11) │ ├── scan "family" - │ │ ├── columns: a:9!null b:10 - │ │ ├── key: (9) - │ │ └── fd: (9)-->(10) + │ │ ├── columns: a:10!null b:11 + │ │ ├── key: (10) + │ │ └── fd: (10)-->(11) │ └── filters - │ └── a:9 = 1 [outer=(9), constraints=(/9: [/1 - /1]; tight), fd=()-->(9)] + │ └── a:10 = 1 [outer=(10), constraints=(/10: [/1 - /1]; tight), fd=()-->(10)] └── filters (true) norm @@ -2214,56 +2214,56 @@ project ├── fd: ()-->(5) └── upsert "family" ├── columns: a:1!null e:5 - ├── canary column: 11 - ├── fetch columns: a:11 c:13 d:14 e:15 + ├── canary column: 12 + ├── fetch columns: a:12 c:14 d:15 e:16 ├── insert-mapping: - │ ├── column1:6 => a:1 - │ ├── column2:7 => b:2 - │ ├── column3:8 => c:3 - │ ├── column4:9 => d:4 - │ └── column5:10 => e:5 + │ ├── column1:7 => a:1 + │ ├── column2:8 => b:2 + │ ├── column3:9 => c:3 + │ ├── column4:10 => d:4 + │ └── column5:11 => e:5 ├── update-mapping: - │ └── upsert_c:19 => c:3 + │ └── upsert_c:21 => c:3 ├── return-mapping: - │ ├── upsert_a:17 => a:1 - │ └── upsert_e:21 => e:5 + │ ├── upsert_a:19 => a:1 + │ └── upsert_e:23 => e:5 ├── cardinality: [1 - 1] ├── volatile, mutations ├── key: () ├── fd: ()-->(1,5) └── project - ├── columns: upsert_a:17 upsert_c:19!null upsert_e:21 column1:6!null column2:7!null column3:8!null column4:9!null column5:10!null a:11 c:13 d:14 e:15 + ├── columns: upsert_a:19 upsert_c:21!null upsert_e:23 column1:7!null column2:8!null column3:9!null column4:10!null column5:11!null a:12 c:14 d:15 e:16 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(6-11,13-15,17,19,21) + ├── fd: ()-->(7-12,14-16,19,21,23) ├── left-join (cross) - │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null column5:10!null a:11 c:13 d:14 e:15 + │ ├── columns: column1:7!null column2:8!null column3:9!null column4:10!null column5:11!null a:12 c:14 d:15 e:16 │ ├── cardinality: [1 - 1] │ ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) │ ├── key: () - │ ├── fd: ()-->(6-11,13-15) + │ ├── fd: ()-->(7-12,14-16) │ ├── values - │ │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null column5:10!null + │ │ ├── columns: column1:7!null column2:8!null column3:9!null column4:10!null column5:11!null │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(6-10) + │ │ ├── fd: ()-->(7-11) │ │ └── (1, 2, 3, 4, 5) │ ├── select - │ │ ├── columns: a:11!null c:13 d:14 e:15 + │ │ ├── columns: a:12!null c:14 d:15 e:16 │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(11,13-15) + │ │ ├── fd: ()-->(12,14-16) │ │ ├── scan "family" - │ │ │ ├── columns: a:11!null c:13 d:14 e:15 - │ │ │ ├── key: (11) - │ │ │ └── fd: (11)-->(13-15) + │ │ │ ├── columns: a:12!null c:14 d:15 e:16 + │ │ │ ├── key: (12) + │ │ │ └── fd: (12)-->(14-16) │ │ └── filters - │ │ └── a:11 = 1 [outer=(11), constraints=(/11: [/1 - /1]; tight), fd=()-->(11)] + │ │ └── a:12 = 1 [outer=(12), constraints=(/12: [/1 - /1]; tight), fd=()-->(12)] │ └── filters (true) └── projections - ├── CASE WHEN a:11 IS NULL THEN column1:6 ELSE a:11 END [as=upsert_a:17, outer=(6,11)] - ├── CASE WHEN a:11 IS NULL THEN column3:8 ELSE 10 END [as=upsert_c:19, outer=(8,11)] - └── CASE WHEN a:11 IS NULL THEN column5:10 ELSE e:15 END [as=upsert_e:21, outer=(10,11,15)] + ├── CASE WHEN a:12 IS NULL THEN column1:7 ELSE a:12 END [as=upsert_a:19, outer=(7,12)] + ├── CASE WHEN a:12 IS NULL THEN column3:9 ELSE 10 END [as=upsert_c:21, outer=(9,12)] + └── CASE WHEN a:12 IS NULL THEN column5:11 ELSE e:16 END [as=upsert_e:23, outer=(11,12,16)] # Do not prune column in same secondary family as updated column. But prune # non-key column in primary family. @@ -2272,49 +2272,49 @@ INSERT INTO family VALUES (1, 2, 3, 4) ON CONFLICT (a) DO UPDATE SET d=10 ---- upsert "family" ├── columns: - ├── canary column: 11 - ├── fetch columns: a:11 c:13 d:14 + ├── canary column: 12 + ├── fetch columns: a:12 c:14 d:15 ├── insert-mapping: - │ ├── column1:6 => a:1 - │ ├── column2:7 => b:2 - │ ├── column3:8 => c:3 - │ ├── column4:9 => d:4 - │ └── column10:10 => e:5 + │ ├── column1:7 => a:1 + │ ├── column2:8 => b:2 + │ ├── column3:9 => c:3 + │ ├── column4:10 => d:4 + │ └── column11:11 => e:5 ├── update-mapping: - │ └── upsert_d:20 => d:4 + │ └── upsert_d:22 => d:4 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: upsert_d:20!null column1:6!null column2:7!null column3:8!null column4:9!null column10:10 a:11 c:13 d:14 + ├── columns: upsert_d:22!null column1:7!null column2:8!null column3:9!null column4:10!null column11:11 a:12 c:14 d:15 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(6-11,13,14,20) + ├── fd: ()-->(7-12,14,15,22) ├── left-join (cross) - │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null column10:10 a:11 c:13 d:14 + │ ├── columns: column1:7!null column2:8!null column3:9!null column4:10!null column11:11 a:12 c:14 d:15 │ ├── cardinality: [1 - 1] │ ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) │ ├── key: () - │ ├── fd: ()-->(6-11,13,14) + │ ├── fd: ()-->(7-12,14,15) │ ├── values - │ │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null column10:10 + │ │ ├── columns: column1:7!null column2:8!null column3:9!null column4:10!null column11:11 │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(6-10) + │ │ ├── fd: ()-->(7-11) │ │ └── (1, 2, 3, 4, NULL) │ ├── select - │ │ ├── columns: a:11!null c:13 d:14 + │ │ ├── columns: a:12!null c:14 d:15 │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(11,13,14) + │ │ ├── fd: ()-->(12,14,15) │ │ ├── scan "family" - │ │ │ ├── columns: a:11!null c:13 d:14 - │ │ │ ├── key: (11) - │ │ │ └── fd: (11)-->(13,14) + │ │ │ ├── columns: a:12!null c:14 d:15 + │ │ │ ├── key: (12) + │ │ │ └── fd: (12)-->(14,15) │ │ └── filters - │ │ └── a:11 = 1 [outer=(11), constraints=(/11: [/1 - /1]; tight), fd=()-->(11)] + │ │ └── a:12 = 1 [outer=(12), constraints=(/12: [/1 - /1]; tight), fd=()-->(12)] │ └── filters (true) └── projections - └── CASE WHEN a:11 IS NULL THEN column4:9 ELSE 10 END [as=upsert_d:20, outer=(9,11)] + └── CASE WHEN a:12 IS NULL THEN column4:10 ELSE 10 END [as=upsert_d:22, outer=(10,12)] # Prune upsert columns when mutation columns/indexes exist. norm expect=(PruneMutationInputCols) @@ -2322,49 +2322,49 @@ INSERT INTO mutation VALUES (1, 2, 3) ON CONFLICT (a) DO UPDATE SET b=10 ---- upsert mutation ├── columns: - ├── canary column: 10 - ├── fetch columns: a:10 b:11 c:12 d:13 e:14 + ├── canary column: 11 + ├── fetch columns: a:11 b:12 c:13 d:14 e:15 ├── insert-mapping: - │ ├── column1:6 => a:1 - │ ├── column2:7 => b:2 - │ ├── column3:8 => c:3 - │ └── column9:9 => d:4 + │ ├── column1:7 => a:1 + │ ├── column2:8 => b:2 + │ ├── column3:9 => c:3 + │ └── column10:10 => d:4 ├── update-mapping: - │ ├── upsert_b:17 => b:2 - │ └── column9:9 => d:4 + │ ├── upsert_b:19 => b:2 + │ └── column10:10 => d:4 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: upsert_b:17!null column1:6!null column2:7!null column3:8!null column9:9 a:10 b:11 c:12 d:13 e:14 + ├── columns: upsert_b:19!null column1:7!null column2:8!null column3:9!null column10:10 a:11 b:12 c:13 d:14 e:15 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(6-14,17) + ├── fd: ()-->(7-15,19) ├── left-join (cross) - │ ├── columns: column1:6!null column2:7!null column3:8!null column9:9 a:10 b:11 c:12 d:13 e:14 + │ ├── columns: column1:7!null column2:8!null column3:9!null column10:10 a:11 b:12 c:13 d:14 e:15 │ ├── cardinality: [1 - 1] │ ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) │ ├── key: () - │ ├── fd: ()-->(6-14) + │ ├── fd: ()-->(7-15) │ ├── values - │ │ ├── columns: column1:6!null column2:7!null column3:8!null column9:9 + │ │ ├── columns: column1:7!null column2:8!null column3:9!null column10:10 │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(6-9) + │ │ ├── fd: ()-->(7-10) │ │ └── (1, 2, 3, NULL) │ ├── select - │ │ ├── columns: a:10!null b:11 c:12 d:13 e:14 + │ │ ├── columns: a:11!null b:12 c:13 d:14 e:15 │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(10-14) + │ │ ├── fd: ()-->(11-15) │ │ ├── scan mutation - │ │ │ ├── columns: a:10!null b:11 c:12 d:13 e:14 - │ │ │ ├── key: (10) - │ │ │ └── fd: (10)-->(11-14) + │ │ │ ├── columns: a:11!null b:12 c:13 d:14 e:15 + │ │ │ ├── key: (11) + │ │ │ └── fd: (11)-->(12-15) │ │ └── filters - │ │ └── a:10 = 1 [outer=(10), constraints=(/10: [/1 - /1]; tight), fd=()-->(10)] + │ │ └── a:11 = 1 [outer=(11), constraints=(/11: [/1 - /1]; tight), fd=()-->(11)] │ └── filters (true) └── projections - └── CASE WHEN a:10 IS NULL THEN column2:7 ELSE 10 END [as=upsert_b:17, outer=(7,10)] + └── CASE WHEN a:11 IS NULL THEN column2:8 ELSE 10 END [as=upsert_b:19, outer=(8,11)] # ------------------------------------------------------------------------------ # PruneMutationReturnCols @@ -2397,23 +2397,23 @@ project ├── volatile, mutations └── update returning_test ├── columns: a:1 b:2 c:3 d:4 e:5 f:6 g:7 rowid:8!null - ├── fetch columns: a:9 b:10 c:11 d:12 e:13 f:14 g:15 rowid:16 + ├── fetch columns: a:10 b:11 c:12 d:13 e:14 f:15 g:16 rowid:17 ├── update-mapping: - │ └── a_new:17 => a:1 + │ └── a_new:19 => a:1 ├── volatile, mutations ├── key: (8) ├── fd: (8)-->(1-7) └── project - ├── columns: a_new:17 a:9 b:10 c:11 d:12 e:13 f:14 g:15 rowid:16!null + ├── columns: a_new:19 a:10 b:11 c:12 d:13 e:14 f:15 g:16 rowid:17!null ├── immutable - ├── key: (16) - ├── fd: (16)-->(9-15), (9)~~>(10-16), (9)-->(17) + ├── key: (17) + ├── fd: (17)-->(10-16), (10)~~>(11-17), (10)-->(19) ├── scan returning_test - │ ├── columns: a:9 b:10 c:11 d:12 e:13 f:14 g:15 rowid:16!null - │ ├── key: (16) - │ └── fd: (16)-->(9-15), (9)~~>(10-16) + │ ├── columns: a:10 b:11 c:12 d:13 e:14 f:15 g:16 rowid:17!null + │ ├── key: (17) + │ └── fd: (17)-->(10-16), (10)~~>(11-17) └── projections - └── a:9 + 1 [as=a_new:17, outer=(9), immutable] + └── a:10 + 1 [as=a_new:19, outer=(10), immutable] # Fetch all the columns in the (d, e, f, g) family as d is being set. @@ -2427,23 +2427,23 @@ project ├── fd: (1)~~>(4) └── update returning_test ├── columns: a:1 d:4 rowid:8!null - ├── fetch columns: a:9 d:12 e:13 f:14 g:15 rowid:16 + ├── fetch columns: a:10 d:13 e:14 f:15 g:16 rowid:17 ├── update-mapping: - │ └── d_new:17 => d:4 + │ └── d_new:19 => d:4 ├── volatile, mutations ├── key: (8) ├── fd: (8)-->(1,4), (1)~~>(4,8) └── project - ├── columns: d_new:17 a:9 d:12 e:13 f:14 g:15 rowid:16!null + ├── columns: d_new:19 a:10 d:13 e:14 f:15 g:16 rowid:17!null ├── immutable - ├── key: (16) - ├── fd: (16)-->(9,12-15), (9)~~>(12-16), (9,12)-->(17) + ├── key: (17) + ├── fd: (17)-->(10,13-16), (10)~~>(13-17), (10,13)-->(19) ├── scan returning_test - │ ├── columns: a:9 d:12 e:13 f:14 g:15 rowid:16!null - │ ├── key: (16) - │ └── fd: (16)-->(9,12-15), (9)~~>(12-16) + │ ├── columns: a:10 d:13 e:14 f:15 g:16 rowid:17!null + │ ├── key: (17) + │ └── fd: (17)-->(10,13-16), (10)~~>(13-17) └── projections - └── a:9 + d:12 [as=d_new:17, outer=(9,12), immutable] + └── a:10 + d:13 [as=d_new:19, outer=(10,13), immutable] # Fetch only whats being updated (not the (d, e, f, g) family). norm @@ -2454,23 +2454,23 @@ project ├── volatile, mutations └── update returning_test ├── columns: a:1 rowid:8!null - ├── fetch columns: a:9 rowid:16 + ├── fetch columns: a:10 rowid:17 ├── update-mapping: - │ └── a_new:17 => a:1 + │ └── a_new:19 => a:1 ├── volatile, mutations ├── key: (8) ├── fd: (8)-->(1) └── project - ├── columns: a_new:17 a:9 rowid:16!null + ├── columns: a_new:19 a:10 rowid:17!null ├── immutable - ├── key: (16) - ├── fd: (16)-->(9,17), (9)~~>(16,17) + ├── key: (17) + ├── fd: (17)-->(10,19), (10)~~>(17,19) ├── scan returning_test - │ ├── columns: a:9 d:12 rowid:16!null - │ ├── key: (16) - │ └── fd: (16)-->(9,12), (9)~~>(12,16) + │ ├── columns: a:10 d:13 rowid:17!null + │ ├── key: (17) + │ └── fd: (17)-->(10,13), (10)~~>(13,17) └── projections - └── a:9 + d:12 [as=a_new:17, outer=(9,12), immutable] + └── a:10 + d:13 [as=a_new:19, outer=(10,13), immutable] # We only fetch the minimal set of columns which is (a, b, c, rowid). norm @@ -2483,24 +2483,24 @@ project ├── fd: (2)~~>(1,3) └── update returning_test ├── columns: a:1 b:2 c:3 rowid:8!null - ├── fetch columns: a:9 b:10 c:11 rowid:16 + ├── fetch columns: a:10 b:11 c:12 rowid:17 ├── update-mapping: - │ ├── a_new:17 => a:1 - │ └── a:9 => b:2 + │ ├── a_new:19 => a:1 + │ └── a:10 => b:2 ├── volatile, mutations ├── key: (8) ├── fd: (8)-->(1-3), (2)~~>(1,3,8) └── project - ├── columns: a_new:17 a:9 b:10 c:11 rowid:16!null + ├── columns: a_new:19 a:10 b:11 c:12 rowid:17!null ├── immutable - ├── key: (16) - ├── fd: (16)-->(9-11), (9)~~>(10,11,16), (9,10)-->(17) + ├── key: (17) + ├── fd: (17)-->(10-12), (10)~~>(11,12,17), (10,11)-->(19) ├── scan returning_test - │ ├── columns: a:9 b:10 c:11 rowid:16!null - │ ├── key: (16) - │ └── fd: (16)-->(9-11), (9)~~>(10,11,16) + │ ├── columns: a:10 b:11 c:12 rowid:17!null + │ ├── key: (17) + │ └── fd: (17)-->(10-12), (10)~~>(11,12,17) └── projections - └── a:9 + b:10 [as=a_new:17, outer=(9,10), immutable] + └── a:10 + b:11 [as=a_new:19, outer=(10,11), immutable] # We apply the PruneMutationReturnCols rule multiple times, to get @@ -2510,38 +2510,38 @@ norm SELECT a FROM [SELECT a, b FROM [UPDATE returning_test SET a = a + 1 RETURNING a, b, c]] ---- with &1 - ├── columns: a:21 + ├── columns: a:23 ├── volatile, mutations ├── project │ ├── columns: returning_test.a:1 returning_test.b:2 returning_test.c:3 │ ├── volatile, mutations │ └── update returning_test │ ├── columns: returning_test.a:1 returning_test.b:2 returning_test.c:3 rowid:8!null - │ ├── fetch columns: returning_test.a:9 returning_test.b:10 returning_test.c:11 rowid:16 + │ ├── fetch columns: returning_test.a:10 returning_test.b:11 returning_test.c:12 rowid:17 │ ├── update-mapping: - │ │ └── a_new:17 => returning_test.a:1 + │ │ └── a_new:19 => returning_test.a:1 │ ├── volatile, mutations │ ├── key: (8) │ ├── fd: (8)-->(1-3) │ └── project - │ ├── columns: a_new:17 returning_test.a:9 returning_test.b:10 returning_test.c:11 rowid:16!null + │ ├── columns: a_new:19 returning_test.a:10 returning_test.b:11 returning_test.c:12 rowid:17!null │ ├── immutable - │ ├── key: (16) - │ ├── fd: (16)-->(9-11), (9)~~>(10,11,16), (9)-->(17) + │ ├── key: (17) + │ ├── fd: (17)-->(10-12), (10)~~>(11,12,17), (10)-->(19) │ ├── scan returning_test - │ │ ├── columns: returning_test.a:9 returning_test.b:10 returning_test.c:11 rowid:16!null - │ │ ├── key: (16) - │ │ └── fd: (16)-->(9-11), (9)~~>(10,11,16) + │ │ ├── columns: returning_test.a:10 returning_test.b:11 returning_test.c:12 rowid:17!null + │ │ ├── key: (17) + │ │ └── fd: (17)-->(10-12), (10)~~>(11,12,17) │ └── projections - │ └── returning_test.a:9 + 1 [as=a_new:17, outer=(9), immutable] + │ └── returning_test.a:10 + 1 [as=a_new:19, outer=(10), immutable] └── project - ├── columns: a:21 + ├── columns: a:23 ├── with-scan &1 - │ ├── columns: a:18 + │ ├── columns: a:20 │ └── mapping: - │ └── returning_test.a:1 => a:18 + │ └── returning_test.a:1 => a:20 └── projections - └── a:18 [as=a:21, outer=(18)] + └── a:20 [as=a:23, outer=(20)] # We derive the prune cols for the mutation appropriately so we # can prune away columns even when the mutation is not under a @@ -2551,42 +2551,42 @@ norm SELECT a FROM [SELECT a, b FROM [UPDATE returning_test SET a = a + 1 RETURNING a, b, c] WHERE a > 1] ---- with &1 - ├── columns: a:21!null + ├── columns: a:23!null ├── volatile, mutations ├── project │ ├── columns: returning_test.a:1 returning_test.b:2 returning_test.c:3 │ ├── volatile, mutations │ └── update returning_test │ ├── columns: returning_test.a:1 returning_test.b:2 returning_test.c:3 rowid:8!null - │ ├── fetch columns: returning_test.a:9 returning_test.b:10 returning_test.c:11 rowid:16 + │ ├── fetch columns: returning_test.a:10 returning_test.b:11 returning_test.c:12 rowid:17 │ ├── update-mapping: - │ │ └── a_new:17 => returning_test.a:1 + │ │ └── a_new:19 => returning_test.a:1 │ ├── volatile, mutations │ ├── key: (8) │ ├── fd: (8)-->(1-3) │ └── project - │ ├── columns: a_new:17 returning_test.a:9 returning_test.b:10 returning_test.c:11 rowid:16!null + │ ├── columns: a_new:19 returning_test.a:10 returning_test.b:11 returning_test.c:12 rowid:17!null │ ├── immutable - │ ├── key: (16) - │ ├── fd: (16)-->(9-11), (9)~~>(10,11,16), (9)-->(17) + │ ├── key: (17) + │ ├── fd: (17)-->(10-12), (10)~~>(11,12,17), (10)-->(19) │ ├── scan returning_test - │ │ ├── columns: returning_test.a:9 returning_test.b:10 returning_test.c:11 rowid:16!null - │ │ ├── key: (16) - │ │ └── fd: (16)-->(9-11), (9)~~>(10,11,16) + │ │ ├── columns: returning_test.a:10 returning_test.b:11 returning_test.c:12 rowid:17!null + │ │ ├── key: (17) + │ │ └── fd: (17)-->(10-12), (10)~~>(11,12,17) │ └── projections - │ └── returning_test.a:9 + 1 [as=a_new:17, outer=(9), immutable] + │ └── returning_test.a:10 + 1 [as=a_new:19, outer=(10), immutable] └── project - ├── columns: a:21!null + ├── columns: a:23!null ├── select - │ ├── columns: a:18!null + │ ├── columns: a:20!null │ ├── with-scan &1 - │ │ ├── columns: a:18 + │ │ ├── columns: a:20 │ │ └── mapping: - │ │ └── returning_test.a:1 => a:18 + │ │ └── returning_test.a:1 => a:20 │ └── filters - │ └── a:18 > 1 [outer=(18), constraints=(/18: [/2 - ]; tight)] + │ └── a:20 > 1 [outer=(20), constraints=(/20: [/2 - ]; tight)] └── projections - └── a:18 [as=a:21, outer=(18)] + └── a:20 [as=a:23, outer=(20)] norm SELECT @@ -2597,59 +2597,59 @@ FROM AS y ON true ---- with &2 - ├── columns: a:9 b:10 a:31!null b:32 + ├── columns: a:10 b:11 a:34!null b:35 ├── volatile, mutations - ├── fd: (9)~~>(10) + ├── fd: (10)~~>(11) ├── project - │ ├── columns: returning_test.a:11 returning_test.b:12 returning_test.c:13 + │ ├── columns: returning_test.a:12 returning_test.b:13 returning_test.c:14 │ ├── volatile, mutations │ └── update returning_test - │ ├── columns: returning_test.a:11 returning_test.b:12 returning_test.c:13 rowid:18!null - │ ├── fetch columns: returning_test.a:19 returning_test.b:20 returning_test.c:21 rowid:26 + │ ├── columns: returning_test.a:12 returning_test.b:13 returning_test.c:14 rowid:19!null + │ ├── fetch columns: returning_test.a:21 returning_test.b:22 returning_test.c:23 rowid:28 │ ├── update-mapping: - │ │ └── a_new:27 => returning_test.a:11 + │ │ └── a_new:30 => returning_test.a:12 │ ├── volatile, mutations - │ ├── key: (18) - │ ├── fd: (18)-->(11-13) + │ ├── key: (19) + │ ├── fd: (19)-->(12-14) │ └── project - │ ├── columns: a_new:27 returning_test.a:19 returning_test.b:20 returning_test.c:21 rowid:26!null + │ ├── columns: a_new:30 returning_test.a:21 returning_test.b:22 returning_test.c:23 rowid:28!null │ ├── immutable - │ ├── key: (26) - │ ├── fd: (26)-->(19-21), (19)~~>(20,21,26), (19)-->(27) + │ ├── key: (28) + │ ├── fd: (28)-->(21-23), (21)~~>(22,23,28), (21)-->(30) │ ├── scan returning_test - │ │ ├── columns: returning_test.a:19 returning_test.b:20 returning_test.c:21 rowid:26!null - │ │ ├── key: (26) - │ │ └── fd: (26)-->(19-21), (19)~~>(20,21,26) + │ │ ├── columns: returning_test.a:21 returning_test.b:22 returning_test.c:23 rowid:28!null + │ │ ├── key: (28) + │ │ └── fd: (28)-->(21-23), (21)~~>(22,23,28) │ └── projections - │ └── returning_test.a:19 + 1 [as=a_new:27, outer=(19), immutable] + │ └── returning_test.a:21 + 1 [as=a_new:30, outer=(21), immutable] └── inner-join (cross) - ├── columns: a:9 b:10 a:31!null b:32 - ├── fd: (9)~~>(10) + ├── columns: a:10 b:11 a:34!null b:35 + ├── fd: (10)~~>(11) ├── project - │ ├── columns: a:9 b:10 - │ ├── lax-key: (9,10) - │ ├── fd: (9)~~>(10) + │ ├── columns: a:10 b:11 + │ ├── lax-key: (10,11) + │ ├── fd: (10)~~>(11) │ ├── scan returning_test │ │ ├── columns: returning_test.a:1 returning_test.b:2 │ │ ├── lax-key: (1,2) │ │ └── fd: (1)~~>(2) │ └── projections - │ ├── returning_test.a:1 [as=a:9, outer=(1)] - │ └── returning_test.b:2 [as=b:10, outer=(2)] + │ ├── returning_test.a:1 [as=a:10, outer=(1)] + │ └── returning_test.b:2 [as=b:11, outer=(2)] ├── project - │ ├── columns: a:31!null b:32 + │ ├── columns: a:34!null b:35 │ ├── select - │ │ ├── columns: a:28!null b:29 + │ │ ├── columns: a:31!null b:32 │ │ ├── with-scan &2 - │ │ │ ├── columns: a:28 b:29 + │ │ │ ├── columns: a:31 b:32 │ │ │ └── mapping: - │ │ │ ├── returning_test.a:11 => a:28 - │ │ │ └── returning_test.b:12 => b:29 + │ │ │ ├── returning_test.a:12 => a:31 + │ │ │ └── returning_test.b:13 => b:32 │ │ └── filters - │ │ └── a:28 > 1 [outer=(28), constraints=(/28: [/2 - ]; tight)] + │ │ └── a:31 > 1 [outer=(31), constraints=(/31: [/2 - ]; tight)] │ └── projections - │ ├── a:28 [as=a:31, outer=(28)] - │ └── b:29 [as=b:32, outer=(29)] + │ ├── a:31 [as=a:34, outer=(31)] + │ └── b:32 [as=b:35, outer=(32)] └── filters (true) # Check if the rule works as desired for other mutations. @@ -2664,65 +2664,65 @@ project ├── fd: ()-->(1-3) └── upsert returning_test ├── columns: a:1 b:2 c:3 rowid:8!null - ├── canary column: 21 - ├── fetch columns: a:14 b:15 c:16 rowid:21 + ├── canary column: 22 + ├── fetch columns: a:15 b:16 c:17 rowid:22 ├── insert-mapping: - │ ├── column1:9 => a:1 - │ ├── column2:10 => b:2 - │ ├── column3:11 => c:3 - │ ├── column12:12 => d:4 - │ ├── column12:12 => e:5 - │ ├── column12:12 => f:6 - │ ├── column12:12 => g:7 - │ └── column13:13 => rowid:8 + │ ├── column1:10 => a:1 + │ ├── column2:11 => b:2 + │ ├── column3:12 => c:3 + │ ├── column13:13 => d:4 + │ ├── column13:13 => e:5 + │ ├── column13:13 => f:6 + │ ├── column13:13 => g:7 + │ └── column14:14 => rowid:8 ├── update-mapping: - │ └── upsert_a:23 => a:1 + │ └── upsert_a:25 => a:1 ├── return-mapping: - │ ├── upsert_a:23 => a:1 - │ ├── upsert_b:24 => b:2 - │ ├── upsert_c:25 => c:3 - │ └── upsert_rowid:30 => rowid:8 + │ ├── upsert_a:25 => a:1 + │ ├── upsert_b:26 => b:2 + │ ├── upsert_c:27 => c:3 + │ └── upsert_rowid:32 => rowid:8 ├── cardinality: [1 - 1] ├── volatile, mutations ├── key: () ├── fd: ()-->(1-3,8) └── project - ├── columns: upsert_a:23 upsert_b:24 upsert_c:25 upsert_rowid:30 column1:9!null column2:10!null column3:11!null column12:12 column13:13 a:14 b:15 c:16 rowid:21 + ├── columns: upsert_a:25 upsert_b:26 upsert_c:27 upsert_rowid:32 column1:10!null column2:11!null column3:12!null column13:13 column14:14 a:15 b:16 c:17 rowid:22 ├── cardinality: [1 - 1] ├── volatile ├── key: () - ├── fd: ()-->(9-16,21,23-25,30) + ├── fd: ()-->(10-17,22,25-27,32) ├── left-join (cross) - │ ├── columns: column1:9!null column2:10!null column3:11!null column12:12 column13:13 a:14 b:15 c:16 rowid:21 + │ ├── columns: column1:10!null column2:11!null column3:12!null column13:13 column14:14 a:15 b:16 c:17 rowid:22 │ ├── cardinality: [1 - 1] │ ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) │ ├── volatile │ ├── key: () - │ ├── fd: ()-->(9-16,21) + │ ├── fd: ()-->(10-17,22) │ ├── values - │ │ ├── columns: column1:9!null column2:10!null column3:11!null column12:12 column13:13 + │ │ ├── columns: column1:10!null column2:11!null column3:12!null column13:13 column14:14 │ │ ├── cardinality: [1 - 1] │ │ ├── volatile │ │ ├── key: () - │ │ ├── fd: ()-->(9-13) + │ │ ├── fd: ()-->(10-14) │ │ └── (1, 2, 'c', CAST(NULL AS INT8), unique_rowid()) │ ├── select - │ │ ├── columns: a:14!null b:15 c:16 rowid:21!null + │ │ ├── columns: a:15!null b:16 c:17 rowid:22!null │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(14-16,21) + │ │ ├── fd: ()-->(15-17,22) │ │ ├── scan returning_test - │ │ │ ├── columns: a:14 b:15 c:16 rowid:21!null - │ │ │ ├── key: (21) - │ │ │ └── fd: (21)-->(14-16), (14)~~>(15,16,21) + │ │ │ ├── columns: a:15 b:16 c:17 rowid:22!null + │ │ │ ├── key: (22) + │ │ │ └── fd: (22)-->(15-17), (15)~~>(16,17,22) │ │ └── filters - │ │ └── a:14 = 1 [outer=(14), constraints=(/14: [/1 - /1]; tight), fd=()-->(14)] + │ │ └── a:15 = 1 [outer=(15), constraints=(/15: [/1 - /1]; tight), fd=()-->(15)] │ └── filters (true) └── projections - ├── CASE WHEN rowid:21 IS NULL THEN column1:9 ELSE column1:9 + a:14 END [as=upsert_a:23, outer=(9,14,21), immutable] - ├── CASE WHEN rowid:21 IS NULL THEN column2:10 ELSE b:15 END [as=upsert_b:24, outer=(10,15,21)] - ├── CASE WHEN rowid:21 IS NULL THEN column3:11 ELSE c:16 END [as=upsert_c:25, outer=(11,16,21)] - └── CASE WHEN rowid:21 IS NULL THEN column13:13 ELSE rowid:21 END [as=upsert_rowid:30, outer=(13,21)] + ├── CASE WHEN rowid:22 IS NULL THEN column1:10 ELSE column1:10 + a:15 END [as=upsert_a:25, outer=(10,15,22), immutable] + ├── CASE WHEN rowid:22 IS NULL THEN column2:11 ELSE b:16 END [as=upsert_b:26, outer=(11,16,22)] + ├── CASE WHEN rowid:22 IS NULL THEN column3:12 ELSE c:17 END [as=upsert_c:27, outer=(12,17,22)] + └── CASE WHEN rowid:22 IS NULL THEN column14:14 ELSE rowid:22 END [as=upsert_rowid:32, outer=(14,22)] norm DELETE FROM returning_test WHERE a < b + d RETURNING a, b, d @@ -2734,21 +2734,21 @@ project ├── fd: (1)-->(2,4) └── delete returning_test ├── columns: a:1!null b:2 d:4 rowid:8!null - ├── fetch columns: a:9 b:10 d:12 rowid:16 + ├── fetch columns: a:10 b:11 d:13 rowid:17 ├── volatile, mutations ├── key: (8) ├── fd: (8)-->(1,2,4), (1)-->(2,4,8) └── select - ├── columns: a:9!null b:10 d:12 rowid:16!null + ├── columns: a:10!null b:11 d:13 rowid:17!null ├── immutable - ├── key: (16) - ├── fd: (16)-->(9,10,12), (9)-->(10,12,16) + ├── key: (17) + ├── fd: (17)-->(10,11,13), (10)-->(11,13,17) ├── scan returning_test - │ ├── columns: a:9 b:10 d:12 rowid:16!null - │ ├── key: (16) - │ └── fd: (16)-->(9,10,12), (9)~~>(10,12,16) + │ ├── columns: a:10 b:11 d:13 rowid:17!null + │ ├── key: (17) + │ └── fd: (17)-->(10,11,13), (10)~~>(11,13,17) └── filters - └── a:9 < (b:10 + d:12) [outer=(9,10,12), immutable, constraints=(/9: (/NULL - ])] + └── a:10 < (b:11 + d:13) [outer=(10,11,13), immutable, constraints=(/10: (/NULL - ])] norm UPSERT INTO returning_test (a, b, c) VALUES (1, 2, 'c') RETURNING a, b, c, d @@ -2761,77 +2761,77 @@ project ├── fd: ()-->(1-4) └── upsert returning_test ├── columns: a:1!null b:2!null c:3!null d:4 rowid:8!null - ├── canary column: 21 - ├── fetch columns: a:14 b:15 c:16 d:17 rowid:21 + ├── canary column: 22 + ├── fetch columns: a:15 b:16 c:17 d:18 rowid:22 ├── insert-mapping: - │ ├── column1:9 => a:1 - │ ├── column2:10 => b:2 - │ ├── column3:11 => c:3 - │ ├── column12:12 => d:4 - │ ├── column12:12 => e:5 - │ ├── column12:12 => f:6 - │ ├── column12:12 => g:7 - │ └── column13:13 => rowid:8 + │ ├── column1:10 => a:1 + │ ├── column2:11 => b:2 + │ ├── column3:12 => c:3 + │ ├── column13:13 => d:4 + │ ├── column13:13 => e:5 + │ ├── column13:13 => f:6 + │ ├── column13:13 => g:7 + │ └── column14:14 => rowid:8 ├── update-mapping: - │ ├── column1:9 => a:1 - │ ├── column2:10 => b:2 - │ └── column3:11 => c:3 + │ ├── column1:10 => a:1 + │ ├── column2:11 => b:2 + │ └── column3:12 => c:3 ├── return-mapping: - │ ├── column1:9 => a:1 - │ ├── column2:10 => b:2 - │ ├── column3:11 => c:3 - │ ├── upsert_d:22 => d:4 - │ └── upsert_rowid:26 => rowid:8 + │ ├── column1:10 => a:1 + │ ├── column2:11 => b:2 + │ ├── column3:12 => c:3 + │ ├── upsert_d:24 => d:4 + │ └── upsert_rowid:28 => rowid:8 ├── cardinality: [1 - 1] ├── volatile, mutations ├── key: () ├── fd: ()-->(1-4,8) └── project - ├── columns: upsert_d:22 upsert_rowid:26 column1:9!null column2:10!null column3:11!null column12:12 column13:13 a:14 b:15 c:16 d:17 rowid:21 + ├── columns: upsert_d:24 upsert_rowid:28 column1:10!null column2:11!null column3:12!null column13:13 column14:14 a:15 b:16 c:17 d:18 rowid:22 ├── cardinality: [1 - 1] ├── volatile ├── key: () - ├── fd: ()-->(9-17,21,22,26) + ├── fd: ()-->(10-18,22,24,28) ├── left-join (hash) - │ ├── columns: column1:9!null column2:10!null column3:11!null column12:12 column13:13 a:14 b:15 c:16 d:17 rowid:21 + │ ├── columns: column1:10!null column2:11!null column3:12!null column13:13 column14:14 a:15 b:16 c:17 d:18 rowid:22 │ ├── cardinality: [1 - 1] │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ ├── volatile │ ├── key: () - │ ├── fd: ()-->(9-17,21) + │ ├── fd: ()-->(10-18,22) │ ├── ensure-upsert-distinct-on - │ │ ├── columns: column1:9!null column2:10!null column3:11!null column12:12 column13:13 - │ │ ├── grouping columns: column13:13 + │ │ ├── columns: column1:10!null column2:11!null column3:12!null column13:13 column14:14 + │ │ ├── grouping columns: column14:14 │ │ ├── error: "UPSERT or INSERT...ON CONFLICT command cannot affect row a second time" │ │ ├── cardinality: [1 - 1] │ │ ├── volatile │ │ ├── key: () - │ │ ├── fd: ()-->(9-13) + │ │ ├── fd: ()-->(10-14) │ │ ├── values - │ │ │ ├── columns: column1:9!null column2:10!null column3:11!null column12:12 column13:13 + │ │ │ ├── columns: column1:10!null column2:11!null column3:12!null column13:13 column14:14 │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── volatile │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(9-13) + │ │ │ ├── fd: ()-->(10-14) │ │ │ └── (1, 2, 'c', CAST(NULL AS INT8), unique_rowid()) │ │ └── aggregations - │ │ ├── first-agg [as=column1:9, outer=(9)] - │ │ │ └── column1:9 - │ │ ├── first-agg [as=column2:10, outer=(10)] - │ │ │ └── column2:10 - │ │ ├── first-agg [as=column3:11, outer=(11)] - │ │ │ └── column3:11 - │ │ └── first-agg [as=column12:12, outer=(12)] - │ │ └── column12:12 + │ │ ├── first-agg [as=column1:10, outer=(10)] + │ │ │ └── column1:10 + │ │ ├── first-agg [as=column2:11, outer=(11)] + │ │ │ └── column2:11 + │ │ ├── first-agg [as=column3:12, outer=(12)] + │ │ │ └── column3:12 + │ │ └── first-agg [as=column13:13, outer=(13)] + │ │ └── column13:13 │ ├── scan returning_test - │ │ ├── columns: a:14 b:15 c:16 d:17 rowid:21!null - │ │ ├── key: (21) - │ │ └── fd: (21)-->(14-17), (14)~~>(15-17,21) + │ │ ├── columns: a:15 b:16 c:17 d:18 rowid:22!null + │ │ ├── key: (22) + │ │ └── fd: (22)-->(15-18), (15)~~>(16-18,22) │ └── filters - │ └── column13:13 = rowid:21 [outer=(13,21), constraints=(/13: (/NULL - ]; /21: (/NULL - ]), fd=(13)==(21), (21)==(13)] + │ └── column14:14 = rowid:22 [outer=(14,22), constraints=(/14: (/NULL - ]; /22: (/NULL - ]), fd=(14)==(22), (22)==(14)] └── projections - ├── CASE WHEN rowid:21 IS NULL THEN column12:12 ELSE d:17 END [as=upsert_d:22, outer=(12,17,21)] - └── CASE WHEN rowid:21 IS NULL THEN column13:13 ELSE rowid:21 END [as=upsert_rowid:26, outer=(13,21)] + ├── CASE WHEN rowid:22 IS NULL THEN column13:13 ELSE d:18 END [as=upsert_d:24, outer=(13,18,22)] + └── CASE WHEN rowid:22 IS NULL THEN column14:14 ELSE rowid:22 END [as=upsert_rowid:28, outer=(14,22)] # Make sure the passthrough columns of an UPDATE ... FROM query are pruned. norm @@ -2846,29 +2846,29 @@ RETURNING abcde.a, family.b, family.c ---- update abcde - ├── columns: a:1!null b:12 c:13 - ├── fetch columns: abcde.a:6 abcde.b:7 abcde.c:8 abcde.d:9 abcde.e:10 + ├── columns: a:1!null b:14 c:15 + ├── fetch columns: abcde.a:7 abcde.b:8 abcde.c:9 abcde.d:10 abcde.e:11 ├── update-mapping: - │ ├── "family".b:12 => abcde.b:2 - │ └── "family".c:13 => abcde.c:3 + │ ├── "family".b:14 => abcde.b:2 + │ └── "family".c:15 => abcde.c:3 ├── volatile, mutations ├── key: (1) - ├── fd: (1)-->(12,13) + ├── fd: (1)-->(14,15) └── inner-join (hash) - ├── columns: abcde.a:6!null abcde.b:7 abcde.c:8 abcde.d:9 abcde.e:10 "family".a:11!null "family".b:12 "family".c:13 + ├── columns: abcde.a:7!null abcde.b:8 abcde.c:9 abcde.d:10 abcde.e:11 "family".a:13!null "family".b:14 "family".c:15 ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (11) - ├── fd: (6)-->(7-10), (7,8)~~>(6,9,10), (11)-->(12,13), (6)==(11), (11)==(6) + ├── key: (13) + ├── fd: (7)-->(8-11), (8,9)~~>(7,10,11), (13)-->(14,15), (7)==(13), (13)==(7) ├── scan abcde - │ ├── columns: abcde.a:6!null abcde.b:7 abcde.c:8 abcde.d:9 abcde.e:10 - │ ├── key: (6) - │ └── fd: (6)-->(7-10), (7,8)~~>(6,9,10) + │ ├── columns: abcde.a:7!null abcde.b:8 abcde.c:9 abcde.d:10 abcde.e:11 + │ ├── key: (7) + │ └── fd: (7)-->(8-11), (8,9)~~>(7,10,11) ├── scan "family" - │ ├── columns: "family".a:11!null "family".b:12 "family".c:13 - │ ├── key: (11) - │ └── fd: (11)-->(12,13) + │ ├── columns: "family".a:13!null "family".b:14 "family".c:15 + │ ├── key: (13) + │ └── fd: (13)-->(14,15) └── filters - └── abcde.a:6 = "family".a:11 [outer=(6,11), constraints=(/6: (/NULL - ]; /11: (/NULL - ]), fd=(6)==(11), (11)==(6)] + └── abcde.a:7 = "family".a:13 [outer=(7,13), constraints=(/7: (/NULL - ]; /13: (/NULL - ]), fd=(7)==(13), (13)==(7)] # -------------------------------------------------- # PruneSemiAntiJoinRightCols @@ -2887,10 +2887,10 @@ semi-join (hash) │ ├── key: (1) │ └── fd: (1)-->(2,3), (2,3)~~>(1) ├── scan "family" - │ ├── columns: "family".a:6!null - │ └── key: (6) + │ ├── columns: "family".a:7!null + │ └── key: (7) └── filters - └── abcde.a:1 = "family".a:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── abcde.a:1 = "family".a:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # We should see the `a`, `b` and `c` columns scanned for family. norm expect=PruneSemiAntiJoinRightCols @@ -2906,12 +2906,12 @@ semi-join (hash) │ ├── key: (1) │ └── fd: (1)-->(2,3), (2,3)~~>(1) ├── scan "family" - │ ├── columns: "family".a:6!null "family".b:7 "family".c:8 - │ ├── key: (6) - │ └── fd: (6)-->(7,8) + │ ├── columns: "family".a:7!null "family".b:8 "family".c:9 + │ ├── key: (7) + │ └── fd: (7)-->(8,9) └── filters - ├── abcde.a:1 = "family".a:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── abcde.b:2 > ("family".b:7 + "family".c:8) [outer=(2,7,8), immutable, constraints=(/2: (/NULL - ])] + ├── abcde.a:1 = "family".a:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + └── abcde.b:2 > ("family".b:8 + "family".c:9) [outer=(2,8,9), immutable, constraints=(/2: (/NULL - ])] norm expect=PruneSemiAntiJoinRightCols SELECT a, b, c FROM abcde WHERE NOT EXISTS (SELECT * FROM family WHERE abcde.a=family.a) @@ -2925,10 +2925,10 @@ anti-join (hash) │ ├── key: (1) │ └── fd: (1)-->(2,3), (2,3)~~>(1) ├── scan "family" - │ ├── columns: "family".a:6!null - │ └── key: (6) + │ ├── columns: "family".a:7!null + │ └── key: (7) └── filters - └── abcde.a:1 = "family".a:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── abcde.a:1 = "family".a:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # Test using multi-level nesting so we don't decorrelate the semi-join. norm expect=PruneSemiAntiJoinRightCols @@ -2955,19 +2955,19 @@ semi-join-apply │ ├── key: (1) │ └── fd: (1)-->(2,3), (2,3)~~>(1) ├── semi-join (cross) - │ ├── columns: "family".a:6!null + │ ├── columns: "family".a:7!null │ ├── outer: (1) - │ ├── key: (6) + │ ├── key: (7) │ ├── scan "family" - │ │ ├── columns: "family".a:6!null - │ │ └── key: (6) + │ │ ├── columns: "family".a:7!null + │ │ └── key: (7) │ ├── scan a - │ │ ├── columns: k:11!null - │ │ └── key: (11) + │ │ ├── columns: k:13!null + │ │ └── key: (13) │ └── filters - │ └── abcde.a:1 = k:11 [outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ]), fd=(1)==(11), (11)==(1)] + │ └── abcde.a:1 = k:13 [outer=(1,13), constraints=(/1: (/NULL - ]; /13: (/NULL - ]), fd=(1)==(13), (13)==(1)] └── filters - └── abcde.a:1 = "family".a:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── abcde.a:1 = "family".a:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # Test using multi-level nesting so we don't decorrelate the anti-join. norm expect=PruneSemiAntiJoinRightCols @@ -2994,44 +2994,44 @@ anti-join-apply │ ├── key: (1) │ └── fd: (1)-->(2,3), (2,3)~~>(1) ├── semi-join (cross) - │ ├── columns: "family".a:6!null + │ ├── columns: "family".a:7!null │ ├── outer: (1) - │ ├── key: (6) + │ ├── key: (7) │ ├── scan "family" - │ │ ├── columns: "family".a:6!null - │ │ └── key: (6) + │ │ ├── columns: "family".a:7!null + │ │ └── key: (7) │ ├── scan a - │ │ ├── columns: k:11!null - │ │ └── key: (11) + │ │ ├── columns: k:13!null + │ │ └── key: (13) │ └── filters - │ └── abcde.a:1 = k:11 [outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ]), fd=(1)==(11), (11)==(1)] + │ └── abcde.a:1 = k:13 [outer=(1,13), constraints=(/1: (/NULL - ]; /13: (/NULL - ]), fd=(1)==(13), (13)==(1)] └── filters - └── abcde.a:1 = "family".a:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── abcde.a:1 = "family".a:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] norm disable=InlineWith expect=PruneWithScanCols WITH foo AS (SELECT * FROM a) SELECT i FROM foo ---- with &1 (foo) - ├── columns: i:6 + ├── columns: i:7 ├── scan a │ ├── columns: a.k:1!null a.i:2 a.f:3 a.s:4 │ ├── key: (1) │ └── fd: (1)-->(2-4) └── with-scan &1 (foo) - ├── columns: i:6 + ├── columns: i:7 └── mapping: - └── a.i:2 => i:6 + └── a.i:2 => i:7 norm disable=InlineWith format=show-all expect=PruneWithCols WITH foo AS (SELECT * FROM a) SELECT i FROM (SELECT i, 1 AS y FROM foo) ORDER BY y ---- with &1 (foo) - ├── columns: i:6(int) + ├── columns: i:7(int) ├── stats: [rows=1000] ├── cost: 1080.04 - ├── prune: (6) + ├── prune: (7) ├── scan t.public.a │ ├── columns: t.public.a.k:1(int!null) t.public.a.i:2(int) t.public.a.f:3(float) t.public.a.s:4(string) │ ├── stats: [rows=1000] @@ -3041,12 +3041,12 @@ with &1 (foo) │ ├── prune: (1-4) │ └── interesting orderings: (+1) └── with-scan &1 (foo) - ├── columns: i:6(int) + ├── columns: i:7(int) ├── mapping: - │ └── t.public.a.i:2(int) => i:6(int) + │ └── t.public.a.i:2(int) => i:7(int) ├── stats: [rows=1000] ├── cost: 0.01 - ├── prune: (6) + ├── prune: (7) └── cte-uses └── &1: count=1 used-columns=(2) @@ -3062,15 +3062,15 @@ SELECT a FROM ( ) ---- union-all - ├── columns: a:8!null + ├── columns: a:10!null ├── left columns: abcde.a:1 - ├── right columns: x:6 + ├── right columns: x:7 ├── scan abcde │ ├── columns: abcde.a:1!null │ └── key: (1) └── scan xy - ├── columns: x:6!null - └── key: (6) + ├── columns: x:7!null + └── key: (7) norm expect=PruneUnionAllCols SELECT count(*) FROM ( @@ -3080,22 +3080,22 @@ SELECT count(*) FROM ( ) ---- scalar-group-by - ├── columns: count:10!null + ├── columns: count:12!null ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(10) + ├── fd: ()-->(12) ├── union-all │ ├── scan abcde │ └── scan xy └── aggregations - └── count-rows [as=count_rows:10] + └── count-rows [as=count_rows:12] norm expect=PruneUnionAllCols SELECT 1 FROM (SELECT a FROM abcde WHERE a > 3 UNION ALL SELECT a FROM abcde) ---- project - ├── columns: "?column?":12!null - ├── fd: ()-->(12) + ├── columns: "?column?":14!null + ├── fd: ()-->(14) ├── union-all │ ├── project │ │ └── select @@ -3108,31 +3108,31 @@ project │ │ └── abcde.a:1 > 3 [outer=(1), constraints=(/1: [/4 - ]; tight)] │ └── scan abcde └── projections - └── 1 [as="?column?":12] + └── 1 [as="?column?":14] norm expect=PruneUnionAllCols SELECT 1 FROM a INNER JOIN (SELECT a, b FROM abcde UNION ALL SELECT * from xy) AS b ON a.i=b.b ---- project - ├── columns: "?column?":14!null - ├── fd: ()-->(14) + ├── columns: "?column?":17!null + ├── fd: ()-->(17) ├── inner-join (hash) - │ ├── columns: i:2!null b:13!null - │ ├── fd: (2)==(13), (13)==(2) + │ ├── columns: i:2!null b:16!null + │ ├── fd: (2)==(16), (16)==(2) │ ├── scan a │ │ └── columns: i:2 │ ├── union-all - │ │ ├── columns: b:13 - │ │ ├── left columns: abcde.b:6 - │ │ ├── right columns: y:11 + │ │ ├── columns: b:16 + │ │ ├── left columns: abcde.b:7 + │ │ ├── right columns: y:13 │ │ ├── scan abcde - │ │ │ └── columns: abcde.b:6 + │ │ │ └── columns: abcde.b:7 │ │ └── scan xy - │ │ └── columns: y:11 + │ │ └── columns: y:13 │ └── filters - │ └── i:2 = b:13 [outer=(2,13), constraints=(/2: (/NULL - ]; /13: (/NULL - ]), fd=(2)==(13), (13)==(2)] + │ └── i:2 = b:16 [outer=(2,16), constraints=(/2: (/NULL - ]; /16: (/NULL - ]), fd=(2)==(16), (16)==(2)] └── projections - └── 1 [as="?column?":14] + └── 1 [as="?column?":17] # Test that even when one side of the UnionAll input has a greater # number of prunable columns than the other (neither the top-level @@ -3148,25 +3148,25 @@ SELECT 1 FROM ( ) ---- project - ├── columns: "?column?":10!null - ├── fd: ()-->(10) + ├── columns: "?column?":12!null + ├── fd: ()-->(12) ├── union-all │ ├── scan abcde │ └── project │ ├── cardinality: [0 - 1] │ ├── key: () │ └── select - │ ├── columns: x:6!null + │ ├── columns: x:7!null │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(6) + │ ├── fd: ()-->(7) │ ├── scan xy - │ │ ├── columns: x:6!null - │ │ └── key: (6) + │ │ ├── columns: x:7!null + │ │ └── key: (7) │ └── filters - │ └── x:6 = 1 [outer=(6), constraints=(/6: [/1 - /1]; tight), fd=()-->(6)] + │ └── x:7 = 1 [outer=(7), constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] └── projections - └── 1 [as="?column?":10] + └── 1 [as="?column?":12] # Regression test for #41772. @@ -3199,53 +3199,53 @@ WHERE b.u ---- project - ├── columns: "?column?":14 - ├── fd: ()-->(14) + ├── columns: "?column?":16 + ├── fd: ()-->(16) ├── inner-join (cross) - │ ├── columns: u:9!null + │ ├── columns: u:11!null │ ├── scan table41772 │ ├── union-all - │ │ ├── columns: u:9!null - │ │ ├── left columns: column1:5 - │ │ ├── right columns: column1:7 + │ │ ├── columns: u:11!null + │ │ ├── left columns: column1:7 + │ │ ├── right columns: column1:9 │ │ ├── cardinality: [0 - 3] │ │ ├── select - │ │ │ ├── columns: column1:5!null + │ │ │ ├── columns: column1:7!null │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(5) + │ │ │ ├── fd: ()-->(7) │ │ │ ├── values - │ │ │ │ ├── columns: column1:5 + │ │ │ │ ├── columns: column1:7 │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── key: () - │ │ │ │ ├── fd: ()-->(5) + │ │ │ │ ├── fd: ()-->(7) │ │ │ │ └── tuple │ │ │ │ └── subquery │ │ │ │ └── max1-row - │ │ │ │ ├── columns: bool:4!null + │ │ │ │ ├── columns: bool:6!null │ │ │ │ ├── error: "more than one row returned by a subquery used as an expression" │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ ├── key: () - │ │ │ │ ├── fd: ()-->(4) + │ │ │ │ ├── fd: ()-->(6) │ │ │ │ └── project - │ │ │ │ ├── columns: bool:4!null - │ │ │ │ ├── fd: ()-->(4) + │ │ │ │ ├── columns: bool:6!null + │ │ │ │ ├── fd: ()-->(6) │ │ │ │ ├── scan table41772 │ │ │ │ └── projections - │ │ │ │ └── true [as=bool:4] + │ │ │ │ └── true [as=bool:6] │ │ │ └── filters - │ │ │ └── column1:5 [outer=(5), constraints=(/5: [/true - /true]; tight), fd=()-->(5)] + │ │ │ └── column1:7 [outer=(7), constraints=(/7: [/true - /true]; tight), fd=()-->(7)] │ │ └── select - │ │ ├── columns: column1:7!null + │ │ ├── columns: column1:9!null │ │ ├── cardinality: [0 - 2] - │ │ ├── fd: ()-->(7) + │ │ ├── fd: ()-->(9) │ │ ├── values - │ │ │ ├── columns: column1:7 + │ │ │ ├── columns: column1:9 │ │ │ ├── cardinality: [2 - 2] │ │ │ ├── (NULL,) │ │ │ └── (false,) │ │ └── filters - │ │ └── column1:7 [outer=(7), constraints=(/7: [/true - /true]; tight), fd=()-->(7)] + │ │ └── column1:9 [outer=(9), constraints=(/9: [/true - /true]; tight), fd=()-->(9)] │ └── filters (true) └── projections - └── NULL [as="?column?":14] + └── NULL [as="?column?":16] diff --git a/pkg/sql/opt/norm/testdata/rules/reject_nulls b/pkg/sql/opt/norm/testdata/rules/reject_nulls index d538ab00cf3a..ce8ff7627235 100644 --- a/pkg/sql/opt/norm/testdata/rules/reject_nulls +++ b/pkg/sql/opt/norm/testdata/rules/reject_nulls @@ -18,36 +18,36 @@ norm expect=RejectNullsRightJoin SELECT * FROM a FULL JOIN xy ON true WHERE a.k IS NOT NULL ---- left-join (cross) - ├── columns: k:1!null i:2 f:3 s:4 x:5 y:6 - ├── key: (1,5) - ├── fd: (1)-->(2-4), (5)-->(6) + ├── columns: k:1!null i:2 f:3 s:4 x:6 y:7 + ├── key: (1,6) + ├── fd: (1)-->(2-4), (6)-->(7) ├── scan a │ ├── columns: k:1!null i:2 f:3 s:4 │ ├── key: (1) │ └── fd: (1)-->(2-4) ├── scan xy - │ ├── columns: x:5!null y:6 - │ ├── key: (5) - │ └── fd: (5)-->(6) + │ ├── columns: x:6!null y:7 + │ ├── key: (6) + │ └── fd: (6)-->(7) └── filters (true) norm expect=RejectNullsLeftJoin SELECT * FROM a FULL JOIN xy ON true WHERE xy.x > 5 ---- left-join (cross) - ├── columns: k:1 i:2 f:3 s:4 x:5!null y:6 - ├── key: (1,5) - ├── fd: (5)-->(6), (1)-->(2-4) + ├── columns: k:1 i:2 f:3 s:4 x:6!null y:7 + ├── key: (1,6) + ├── fd: (6)-->(7), (1)-->(2-4) ├── select - │ ├── columns: x:5!null y:6 - │ ├── key: (5) - │ ├── fd: (5)-->(6) + │ ├── columns: x:6!null y:7 + │ ├── key: (6) + │ ├── fd: (6)-->(7) │ ├── scan xy - │ │ ├── columns: x:5!null y:6 - │ │ ├── key: (5) - │ │ └── fd: (5)-->(6) + │ │ ├── columns: x:6!null y:7 + │ │ ├── key: (6) + │ │ └── fd: (6)-->(7) │ └── filters - │ └── x:5 > 5 [outer=(5), constraints=(/5: [/6 - ]; tight)] + │ └── x:6 > 5 [outer=(6), constraints=(/6: [/6 - ]; tight)] ├── scan a │ ├── columns: k:1!null i:2 f:3 s:4 │ ├── key: (1) @@ -62,48 +62,48 @@ INNER JOIN (SELECT * FROM a LEFT JOIN uv ON True) AS r ON l.u=1 AND r.v>2 ---- inner-join (cross) - ├── columns: k:1!null i:2 f:3 s:4 u:5!null v:6 k:7!null i:8 f:9 s:10 u:11!null v:12!null - ├── key: (1,7,11) - ├── fd: ()-->(5,6), (1)-->(2-4), (7)-->(8-10), (11)-->(12) + ├── columns: k:1!null i:2 f:3 s:4 u:6!null v:7 k:9!null i:10 f:11 s:12 u:14!null v:15!null + ├── key: (1,9,14) + ├── fd: ()-->(6,7), (1)-->(2-4), (9)-->(10-12), (14)-->(15) ├── inner-join (cross) - │ ├── columns: k:1!null i:2 f:3 s:4 u:5!null v:6 + │ ├── columns: k:1!null i:2 f:3 s:4 u:6!null v:7 │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) │ ├── key: (1) - │ ├── fd: ()-->(5,6), (1)-->(2-4) + │ ├── fd: ()-->(6,7), (1)-->(2-4) │ ├── scan a │ │ ├── columns: k:1!null i:2 f:3 s:4 │ │ ├── key: (1) │ │ └── fd: (1)-->(2-4) │ ├── select - │ │ ├── columns: u:5!null v:6 + │ │ ├── columns: u:6!null v:7 │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(5,6) + │ │ ├── fd: ()-->(6,7) │ │ ├── scan uv - │ │ │ ├── columns: u:5!null v:6 - │ │ │ ├── key: (5) - │ │ │ └── fd: (5)-->(6) + │ │ │ ├── columns: u:6!null v:7 + │ │ │ ├── key: (6) + │ │ │ └── fd: (6)-->(7) │ │ └── filters - │ │ └── u:5 = 1 [outer=(5), constraints=(/5: [/1 - /1]; tight), fd=()-->(5)] + │ │ └── u:6 = 1 [outer=(6), constraints=(/6: [/1 - /1]; tight), fd=()-->(6)] │ └── filters (true) ├── inner-join (cross) - │ ├── columns: k:7!null i:8 f:9 s:10 u:11!null v:12!null - │ ├── key: (7,11) - │ ├── fd: (7)-->(8-10), (11)-->(12) + │ ├── columns: k:9!null i:10 f:11 s:12 u:14!null v:15!null + │ ├── key: (9,14) + │ ├── fd: (9)-->(10-12), (14)-->(15) │ ├── scan a - │ │ ├── columns: k:7!null i:8 f:9 s:10 - │ │ ├── key: (7) - │ │ └── fd: (7)-->(8-10) + │ │ ├── columns: k:9!null i:10 f:11 s:12 + │ │ ├── key: (9) + │ │ └── fd: (9)-->(10-12) │ ├── select - │ │ ├── columns: u:11!null v:12!null - │ │ ├── key: (11) - │ │ ├── fd: (11)-->(12) + │ │ ├── columns: u:14!null v:15!null + │ │ ├── key: (14) + │ │ ├── fd: (14)-->(15) │ │ ├── scan uv - │ │ │ ├── columns: u:11!null v:12 - │ │ │ ├── key: (11) - │ │ │ └── fd: (11)-->(12) + │ │ │ ├── columns: u:14!null v:15 + │ │ │ ├── key: (14) + │ │ │ └── fd: (14)-->(15) │ │ └── filters - │ │ └── v:12 > 2 [outer=(12), constraints=(/12: [/3 - ]; tight)] + │ │ └── v:15 > 2 [outer=(15), constraints=(/15: [/3 - ]; tight)] │ └── filters (true) └── filters (true) @@ -112,39 +112,39 @@ norm expect=RejectNullsLeftJoin SELECT * FROM a LEFT JOIN xy ON true WHERE xy.x = a.k ---- inner-join (hash) - ├── columns: k:1!null i:2 f:3 s:4 x:5!null y:6 + ├── columns: k:1!null i:2 f:3 s:4 x:6!null y:7 ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (5) - ├── fd: (1)-->(2-4), (5)-->(6), (1)==(5), (5)==(1) + ├── key: (6) + ├── fd: (1)-->(2-4), (6)-->(7), (1)==(6), (6)==(1) ├── scan a │ ├── columns: k:1!null i:2 f:3 s:4 │ ├── key: (1) │ └── fd: (1)-->(2-4) ├── scan xy - │ ├── columns: x:5!null y:6 - │ ├── key: (5) - │ └── fd: (5)-->(6) + │ ├── columns: x:6!null y:7 + │ ├── key: (6) + │ └── fd: (6)-->(7) └── filters - └── x:5 = k:1 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + └── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] # Full-join operator. norm expect=RejectNullsLeftJoin SELECT * FROM a FULL JOIN xy ON true WHERE a.k IS NOT NULL AND xy.x > 5 ---- inner-join (cross) - ├── columns: k:1!null i:2 f:3 s:4 x:5!null y:6 - ├── key: (1,5) - ├── fd: (5)-->(6), (1)-->(2-4) + ├── columns: k:1!null i:2 f:3 s:4 x:6!null y:7 + ├── key: (1,6) + ├── fd: (6)-->(7), (1)-->(2-4) ├── select - │ ├── columns: x:5!null y:6 - │ ├── key: (5) - │ ├── fd: (5)-->(6) + │ ├── columns: x:6!null y:7 + │ ├── key: (6) + │ ├── fd: (6)-->(7) │ ├── scan xy - │ │ ├── columns: x:5!null y:6 - │ │ ├── key: (5) - │ │ └── fd: (5)-->(6) + │ │ ├── columns: x:6!null y:7 + │ │ ├── key: (6) + │ │ └── fd: (6)-->(7) │ └── filters - │ └── x:5 > 5 [outer=(5), constraints=(/5: [/6 - ]; tight)] + │ └── x:6 > 5 [outer=(6), constraints=(/6: [/6 - ]; tight)] ├── scan a │ ├── columns: k:1!null i:2 f:3 s:4 │ ├── key: (1) @@ -156,8 +156,8 @@ norm expect=RejectNullsLeftJoin SELECT * FROM a LEFT JOIN LATERAL (SELECT * FROM (VALUES (i), (i)) v(y)) ON y>10 WHERE i=y ---- inner-join-apply - ├── columns: k:1!null i:2!null f:3 s:4 y:5 - ├── fd: (1)-->(2-4), (2)==(5), (5)==(2) + ├── columns: k:1!null i:2!null f:3 s:4 y:6 + ├── fd: (1)-->(2-4), (2)==(6), (6)==(2) ├── select │ ├── columns: k:1!null i:2!null f:3 s:4 │ ├── key: (1) @@ -169,22 +169,22 @@ inner-join-apply │ └── filters │ └── i:2 > 10 [outer=(2), constraints=(/2: [/11 - ]; tight)] ├── values - │ ├── columns: column1:5 + │ ├── columns: column1:6 │ ├── outer: (2) │ ├── cardinality: [2 - 2] │ ├── (i:2,) │ └── (i:2,) └── filters - └── i:2 = column1:5 [outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ]), fd=(2)==(5), (5)==(2)] + └── i:2 = column1:6 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] # Full-join operator. norm expect=RejectNullsRightJoin SELECT * FROM a FULL JOIN xy ON true WHERE i IS NOT NULL ---- left-join (cross) - ├── columns: k:1!null i:2!null f:3 s:4 x:5 y:6 - ├── key: (1,5) - ├── fd: (1)-->(2-4), (5)-->(6) + ├── columns: k:1!null i:2!null f:3 s:4 x:6 y:7 + ├── key: (1,6) + ├── fd: (1)-->(2-4), (6)-->(7) ├── select │ ├── columns: k:1!null i:2!null f:3 s:4 │ ├── key: (1) @@ -196,9 +196,9 @@ left-join (cross) │ └── filters │ └── i:2 IS NOT NULL [outer=(2), constraints=(/2: (/NULL - ]; tight)] ├── scan xy - │ ├── columns: x:5!null y:6 - │ ├── key: (5) - │ └── fd: (5)-->(6) + │ ├── columns: x:6!null y:7 + │ ├── key: (6) + │ └── fd: (6)-->(7) └── filters (true) # ---------------------------------------------------------- @@ -215,32 +215,32 @@ GROUP BY k HAVING max(x)=1 ---- project - ├── columns: max:7!null - ├── fd: ()-->(7) + ├── columns: max:9!null + ├── fd: ()-->(9) └── select - ├── columns: k:1!null max:7!null + ├── columns: k:1!null max:9!null ├── key: (1) - ├── fd: ()-->(7) + ├── fd: ()-->(9) ├── group-by - │ ├── columns: k:1!null max:7!null + │ ├── columns: k:1!null max:9!null │ ├── grouping columns: k:1!null │ ├── key: (1) - │ ├── fd: (1)-->(7) + │ ├── fd: (1)-->(9) │ ├── inner-join (cross) - │ │ ├── columns: k:1!null x:5!null - │ │ ├── key: (1,5) + │ │ ├── columns: k:1!null x:6!null + │ │ ├── key: (1,6) │ │ ├── scan a │ │ │ ├── columns: k:1!null │ │ │ └── key: (1) │ │ ├── scan xy - │ │ │ ├── columns: x:5!null - │ │ │ └── key: (5) + │ │ │ ├── columns: x:6!null + │ │ │ └── key: (6) │ │ └── filters (true) │ └── aggregations - │ └── max [as=max:7, outer=(5)] - │ └── x:5 + │ └── max [as=max:9, outer=(6)] + │ └── x:6 └── filters - └── max:7 = 1 [outer=(7), constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] + └── max:9 = 1 [outer=(9), constraints=(/9: [/1 - /1]; tight), fd=()-->(9)] # Aggregate function with DISTINCT. norm expect=RejectNullsGroupBy @@ -252,39 +252,39 @@ GROUP BY k HAVING sum(DISTINCT y)=1 ---- project - ├── columns: sum:7!null max:8!null + ├── columns: sum:9!null max:10!null ├── immutable - ├── fd: ()-->(7) + ├── fd: ()-->(9) └── select - ├── columns: k:1!null sum:7!null max:8!null + ├── columns: k:1!null sum:9!null max:10!null ├── immutable ├── key: (1) - ├── fd: ()-->(7), (1)-->(8) + ├── fd: ()-->(9), (1)-->(10) ├── group-by - │ ├── columns: k:1!null sum:7!null max:8!null + │ ├── columns: k:1!null sum:9!null max:10!null │ ├── grouping columns: k:1!null │ ├── key: (1) - │ ├── fd: (1)-->(7,8) + │ ├── fd: (1)-->(9,10) │ ├── inner-join (cross) - │ │ ├── columns: k:1!null y:6!null + │ │ ├── columns: k:1!null y:7!null │ │ ├── scan a │ │ │ ├── columns: k:1!null │ │ │ └── key: (1) │ │ ├── select - │ │ │ ├── columns: y:6!null + │ │ │ ├── columns: y:7!null │ │ │ ├── scan xy - │ │ │ │ └── columns: y:6 + │ │ │ │ └── columns: y:7 │ │ │ └── filters - │ │ │ └── y:6 IS NOT NULL [outer=(6), constraints=(/6: (/NULL - ]; tight)] + │ │ │ └── y:7 IS NOT NULL [outer=(7), constraints=(/7: (/NULL - ]; tight)] │ │ └── filters (true) │ └── aggregations - │ ├── agg-distinct [as=sum:7, outer=(6)] + │ ├── agg-distinct [as=sum:9, outer=(7)] │ │ └── sum - │ │ └── y:6 - │ └── max [as=max:8, outer=(6)] - │ └── y:6 + │ │ └── y:7 + │ └── max [as=max:10, outer=(7)] + │ └── y:7 └── filters - └── sum:7 = 1 [outer=(7), immutable, constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] + └── sum:9 = 1 [outer=(9), immutable, constraints=(/9: [/1 - /1]; tight), fd=()-->(9)] # Single max aggregate function without grouping columns. norm expect=RejectNullsGroupBy @@ -295,27 +295,27 @@ ON True HAVING max(x)=1 ---- select - ├── columns: max:7!null + ├── columns: max:9!null ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(7) + ├── fd: ()-->(9) ├── scalar-group-by - │ ├── columns: max:7 + │ ├── columns: max:9 │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(7) + │ ├── fd: ()-->(9) │ ├── inner-join (cross) - │ │ ├── columns: x:5!null + │ │ ├── columns: x:6!null │ │ ├── scan a │ │ ├── scan xy - │ │ │ ├── columns: x:5!null - │ │ │ └── key: (5) + │ │ │ ├── columns: x:6!null + │ │ │ └── key: (6) │ │ └── filters (true) │ └── aggregations - │ └── max [as=max:7, outer=(5)] - │ └── x:5 + │ └── max [as=max:9, outer=(6)] + │ └── x:6 └── filters - └── max:7 = 1 [outer=(7), constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] + └── max:9 = 1 [outer=(9), constraints=(/9: [/1 - /1]; tight), fd=()-->(9)] # Multiple aggregate functions on same column. norm expect=RejectNullsGroupBy @@ -327,34 +327,34 @@ GROUP BY k HAVING min(x)=1 ---- project - ├── columns: min:7!null max:8!null - ├── fd: ()-->(7) + ├── columns: min:9!null max:10!null + ├── fd: ()-->(9) └── select - ├── columns: k:1!null min:7!null max:8!null + ├── columns: k:1!null min:9!null max:10!null ├── key: (1) - ├── fd: ()-->(7), (1)-->(8) + ├── fd: ()-->(9), (1)-->(10) ├── group-by - │ ├── columns: k:1!null min:7!null max:8!null + │ ├── columns: k:1!null min:9!null max:10!null │ ├── grouping columns: k:1!null │ ├── key: (1) - │ ├── fd: (1)-->(7,8) + │ ├── fd: (1)-->(9,10) │ ├── inner-join (cross) - │ │ ├── columns: k:1!null x:5!null - │ │ ├── key: (1,5) + │ │ ├── columns: k:1!null x:6!null + │ │ ├── key: (1,6) │ │ ├── scan a │ │ │ ├── columns: k:1!null │ │ │ └── key: (1) │ │ ├── scan xy - │ │ │ ├── columns: x:5!null - │ │ │ └── key: (5) + │ │ │ ├── columns: x:6!null + │ │ │ └── key: (6) │ │ └── filters (true) │ └── aggregations - │ ├── min [as=min:7, outer=(5)] - │ │ └── x:5 - │ └── max [as=max:8, outer=(5)] - │ └── x:5 + │ ├── min [as=min:9, outer=(6)] + │ │ └── x:6 + │ └── max [as=max:10, outer=(6)] + │ └── x:6 └── filters - └── min:7 = 1 [outer=(7), constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] + └── min:9 = 1 [outer=(9), constraints=(/9: [/1 - /1]; tight), fd=()-->(9)] # Multiple aggregate functions on same column, some with DISTINCT. norm expect=RejectNullsGroupBy @@ -366,37 +366,37 @@ GROUP BY k HAVING max(y)=1 ---- project - ├── columns: sum:7!null max:8!null - ├── fd: ()-->(8) + ├── columns: sum:9!null max:10!null + ├── fd: ()-->(10) └── select - ├── columns: k:1!null sum:7!null max:8!null + ├── columns: k:1!null sum:9!null max:10!null ├── key: (1) - ├── fd: ()-->(8), (1)-->(7) + ├── fd: ()-->(10), (1)-->(9) ├── group-by - │ ├── columns: k:1!null sum:7!null max:8!null + │ ├── columns: k:1!null sum:9!null max:10!null │ ├── grouping columns: k:1!null │ ├── key: (1) - │ ├── fd: (1)-->(7,8) + │ ├── fd: (1)-->(9,10) │ ├── inner-join (cross) - │ │ ├── columns: k:1!null y:6!null + │ │ ├── columns: k:1!null y:7!null │ │ ├── scan a │ │ │ ├── columns: k:1!null │ │ │ └── key: (1) │ │ ├── select - │ │ │ ├── columns: y:6!null + │ │ │ ├── columns: y:7!null │ │ │ ├── scan xy - │ │ │ │ └── columns: y:6 + │ │ │ │ └── columns: y:7 │ │ │ └── filters - │ │ │ └── y:6 IS NOT NULL [outer=(6), constraints=(/6: (/NULL - ]; tight)] + │ │ │ └── y:7 IS NOT NULL [outer=(7), constraints=(/7: (/NULL - ]; tight)] │ │ └── filters (true) │ └── aggregations - │ ├── agg-distinct [as=sum:7, outer=(6)] + │ ├── agg-distinct [as=sum:9, outer=(7)] │ │ └── sum - │ │ └── y:6 - │ └── max [as=max:8, outer=(6)] - │ └── y:6 + │ │ └── y:7 + │ └── max [as=max:10, outer=(7)] + │ └── y:7 └── filters - └── max:8 = 1 [outer=(8), constraints=(/8: [/1 - /1]; tight), fd=()-->(8)] + └── max:10 = 1 [outer=(10), constraints=(/10: [/1 - /1]; tight), fd=()-->(10)] # Ignore ConstAgg aggregates on other columns. @@ -423,37 +423,37 @@ exprnorm expect=RejectNullsGroupBy ) ---- select - ├── columns: u:3 v:4 [hidden: sum:5!null const:6] + ├── columns: u:4 v:5 [hidden: sum:7!null const:8] ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(5,6) + ├── fd: ()-->(7,8) ├── scalar-group-by - │ ├── columns: sum:5 const:6 + │ ├── columns: sum:7 const:8 │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(5,6) + │ ├── fd: ()-->(7,8) │ ├── inner-join (cross) - │ │ ├── columns: u:3!null v:4!null - │ │ ├── fd: (3)-->(4) + │ │ ├── columns: u:4!null v:5!null + │ │ ├── fd: (4)-->(5) │ │ ├── scan xy │ │ ├── select - │ │ │ ├── columns: u:3!null v:4!null - │ │ │ ├── key: (3) - │ │ │ ├── fd: (3)-->(4) + │ │ │ ├── columns: u:4!null v:5!null + │ │ │ ├── key: (4) + │ │ │ ├── fd: (4)-->(5) │ │ │ ├── scan uv - │ │ │ │ ├── columns: u:3!null v:4 - │ │ │ │ ├── key: (3) - │ │ │ │ └── fd: (3)-->(4) + │ │ │ │ ├── columns: u:4!null v:5 + │ │ │ │ ├── key: (4) + │ │ │ │ └── fd: (4)-->(5) │ │ │ └── filters - │ │ │ └── v:4 IS NOT NULL [outer=(4), constraints=(/4: (/NULL - ]; tight)] + │ │ │ └── v:5 IS NOT NULL [outer=(5), constraints=(/5: (/NULL - ]; tight)] │ │ └── filters (true) │ └── aggregations - │ ├── sum [as=sum:5, outer=(4)] - │ │ └── v:4 - │ └── const-agg [as=const:6, outer=(3)] - │ └── u:3 + │ ├── sum [as=sum:7, outer=(5)] + │ │ └── v:5 + │ └── const-agg [as=const:8, outer=(4)] + │ └── u:4 └── filters - └── sum:5 = 10 [outer=(5), constraints=(/5: [/10 - /10]; tight), fd=()-->(5)] + └── sum:7 = 10 [outer=(7), constraints=(/7: [/10 - /10]; tight), fd=()-->(7)] # Don't reject nulls when multiple columns are used. norm expect-not=RejectNullsGroupBy @@ -465,36 +465,36 @@ GROUP BY k HAVING min(x)=1 ---- project - ├── columns: min:7!null max:8 - ├── fd: ()-->(7) + ├── columns: min:9!null max:10 + ├── fd: ()-->(9) └── select - ├── columns: k:1!null min:7!null max:8 + ├── columns: k:1!null min:9!null max:10 ├── key: (1) - ├── fd: ()-->(7), (1)-->(8) + ├── fd: ()-->(9), (1)-->(10) ├── group-by - │ ├── columns: k:1!null min:7 max:8 + │ ├── columns: k:1!null min:9 max:10 │ ├── grouping columns: k:1!null │ ├── key: (1) - │ ├── fd: (1)-->(7,8) + │ ├── fd: (1)-->(9,10) │ ├── left-join (cross) - │ │ ├── columns: k:1!null x:5 y:6 - │ │ ├── key: (1,5) - │ │ ├── fd: (5)-->(6) + │ │ ├── columns: k:1!null x:6 y:7 + │ │ ├── key: (1,6) + │ │ ├── fd: (6)-->(7) │ │ ├── scan a │ │ │ ├── columns: k:1!null │ │ │ └── key: (1) │ │ ├── scan xy - │ │ │ ├── columns: x:5!null y:6 - │ │ │ ├── key: (5) - │ │ │ └── fd: (5)-->(6) + │ │ │ ├── columns: x:6!null y:7 + │ │ │ ├── key: (6) + │ │ │ └── fd: (6)-->(7) │ │ └── filters (true) │ └── aggregations - │ ├── min [as=min:7, outer=(5)] - │ │ └── x:5 - │ └── max [as=max:8, outer=(6)] - │ └── y:6 + │ ├── min [as=min:9, outer=(6)] + │ │ └── x:6 + │ └── max [as=max:10, outer=(7)] + │ └── y:7 └── filters - └── min:7 = 1 [outer=(7), constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] + └── min:9 = 1 [outer=(9), constraints=(/9: [/1 - /1]; tight), fd=()-->(9)] # Don't reject column when count function is used (it doesn't return nil when # input is empty). @@ -507,32 +507,32 @@ GROUP BY k HAVING count(x)=1 ---- project - ├── columns: count:7!null - ├── fd: ()-->(7) + ├── columns: count:9!null + ├── fd: ()-->(9) └── select - ├── columns: k:1!null count:7!null + ├── columns: k:1!null count:9!null ├── key: (1) - ├── fd: ()-->(7) + ├── fd: ()-->(9) ├── group-by - │ ├── columns: k:1!null count:7!null + │ ├── columns: k:1!null count:9!null │ ├── grouping columns: k:1!null │ ├── key: (1) - │ ├── fd: (1)-->(7) + │ ├── fd: (1)-->(9) │ ├── left-join (cross) - │ │ ├── columns: k:1!null x:5 - │ │ ├── key: (1,5) + │ │ ├── columns: k:1!null x:6 + │ │ ├── key: (1,6) │ │ ├── scan a │ │ │ ├── columns: k:1!null │ │ │ └── key: (1) │ │ ├── scan xy - │ │ │ ├── columns: x:5!null - │ │ │ └── key: (5) + │ │ │ ├── columns: x:6!null + │ │ │ └── key: (6) │ │ └── filters (true) │ └── aggregations - │ └── count [as=count:7, outer=(5)] - │ └── x:5 + │ └── count [as=count:9, outer=(6)] + │ └── x:6 └── filters - └── count:7 = 1 [outer=(7), constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] + └── count:9 = 1 [outer=(9), constraints=(/9: [/1 - /1]; tight), fd=()-->(9)] # ConstNotNullAgg rejects nulls (regression test for #28810). # TODO(andyk): Removal of filter pushdown into apply join inputs means that this @@ -543,51 +543,51 @@ norm SELECT 1 FROM a AS ref_0 LEFT JOIN a AS ref_1 ON EXISTS(SELECT 1 FROM a WHERE a.s = ref_0.s) ---- project - ├── columns: "?column?":17!null - ├── fd: ()-->(17) + ├── columns: "?column?":20!null + ├── fd: ()-->(20) ├── left-join-apply - │ ├── columns: ref_0.s:4 exists:16 + │ ├── columns: ref_0.s:4 exists:19 │ ├── scan ref_0 │ │ └── columns: ref_0.s:4 │ ├── project - │ │ ├── columns: exists:16!null + │ │ ├── columns: exists:19!null │ │ ├── outer: (4) │ │ ├── group-by - │ │ │ ├── columns: ref_1.k:5!null true_agg:15 - │ │ │ ├── grouping columns: ref_1.k:5!null + │ │ │ ├── columns: ref_1.k:6!null true_agg:18 + │ │ │ ├── grouping columns: ref_1.k:6!null │ │ │ ├── outer: (4) - │ │ │ ├── key: (5) - │ │ │ ├── fd: (5)-->(15) + │ │ │ ├── key: (6) + │ │ │ ├── fd: (6)-->(18) │ │ │ ├── left-join (cross) - │ │ │ │ ├── columns: ref_1.k:5!null true:14 + │ │ │ │ ├── columns: ref_1.k:6!null true:17 │ │ │ │ ├── outer: (4) │ │ │ │ ├── scan ref_1 - │ │ │ │ │ ├── columns: ref_1.k:5!null - │ │ │ │ │ └── key: (5) + │ │ │ │ │ ├── columns: ref_1.k:6!null + │ │ │ │ │ └── key: (6) │ │ │ │ ├── project - │ │ │ │ │ ├── columns: true:14!null + │ │ │ │ │ ├── columns: true:17!null │ │ │ │ │ ├── outer: (4) - │ │ │ │ │ ├── fd: ()-->(14) + │ │ │ │ │ ├── fd: ()-->(17) │ │ │ │ │ ├── select - │ │ │ │ │ │ ├── columns: a.s:12!null + │ │ │ │ │ │ ├── columns: a.s:14!null │ │ │ │ │ │ ├── outer: (4) - │ │ │ │ │ │ ├── fd: ()-->(12) + │ │ │ │ │ │ ├── fd: ()-->(14) │ │ │ │ │ │ ├── scan a - │ │ │ │ │ │ │ └── columns: a.s:12 + │ │ │ │ │ │ │ └── columns: a.s:14 │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── a.s:12 = ref_0.s:4 [outer=(4,12), constraints=(/4: (/NULL - ]; /12: (/NULL - ]), fd=(4)==(12), (12)==(4)] + │ │ │ │ │ │ └── a.s:14 = ref_0.s:4 [outer=(4,14), constraints=(/4: (/NULL - ]; /14: (/NULL - ]), fd=(4)==(14), (14)==(4)] │ │ │ │ │ └── projections - │ │ │ │ │ └── true [as=true:14] + │ │ │ │ │ └── true [as=true:17] │ │ │ │ └── filters (true) │ │ │ └── aggregations - │ │ │ └── const-not-null-agg [as=true_agg:15, outer=(14)] - │ │ │ └── true:14 + │ │ │ └── const-not-null-agg [as=true_agg:18, outer=(17)] + │ │ │ └── true:17 │ │ └── projections - │ │ └── true_agg:15 IS NOT NULL [as=exists:16, outer=(15)] + │ │ └── true_agg:18 IS NOT NULL [as=exists:19, outer=(18)] │ └── filters - │ └── exists:16 [outer=(16), constraints=(/16: [/true - /true]; tight), fd=()-->(16)] + │ └── exists:19 [outer=(19), constraints=(/19: [/true - /true]; tight), fd=()-->(19)] └── projections - └── 1 [as="?column?":17] + └── 1 [as="?column?":20] # Use with multi-argument aggregate function. norm expect=RejectNullsGroupBy @@ -599,43 +599,43 @@ GROUP BY k HAVING string_agg(s, ',')='foo' ---- project - ├── columns: string_agg:8!null - ├── fd: ()-->(8) + ├── columns: string_agg:10!null + ├── fd: ()-->(10) └── select - ├── columns: k:3!null string_agg:8!null - ├── key: (3) - ├── fd: ()-->(8) + ├── columns: k:4!null string_agg:10!null + ├── key: (4) + ├── fd: ()-->(10) ├── group-by - │ ├── columns: k:3!null string_agg:8!null - │ ├── grouping columns: k:3!null - │ ├── key: (3) - │ ├── fd: (3)-->(8) + │ ├── columns: k:4!null string_agg:10!null + │ ├── grouping columns: k:4!null + │ ├── key: (4) + │ ├── fd: (4)-->(10) │ ├── project - │ │ ├── columns: column7:7!null k:3!null s:6!null - │ │ ├── fd: ()-->(7), (3)-->(6) + │ │ ├── columns: column9:9!null k:4!null s:7!null + │ │ ├── fd: ()-->(9), (4)-->(7) │ │ ├── inner-join (cross) - │ │ │ ├── columns: k:3!null s:6!null - │ │ │ ├── fd: (3)-->(6) + │ │ │ ├── columns: k:4!null s:7!null + │ │ │ ├── fd: (4)-->(7) │ │ │ ├── scan xy │ │ │ ├── select - │ │ │ │ ├── columns: k:3!null s:6!null - │ │ │ │ ├── key: (3) - │ │ │ │ ├── fd: (3)-->(6) + │ │ │ │ ├── columns: k:4!null s:7!null + │ │ │ │ ├── key: (4) + │ │ │ │ ├── fd: (4)-->(7) │ │ │ │ ├── scan a - │ │ │ │ │ ├── columns: k:3!null s:6 - │ │ │ │ │ ├── key: (3) - │ │ │ │ │ └── fd: (3)-->(6) + │ │ │ │ │ ├── columns: k:4!null s:7 + │ │ │ │ │ ├── key: (4) + │ │ │ │ │ └── fd: (4)-->(7) │ │ │ │ └── filters - │ │ │ │ └── s:6 IS NOT NULL [outer=(6), constraints=(/6: (/NULL - ]; tight)] + │ │ │ │ └── s:7 IS NOT NULL [outer=(7), constraints=(/7: (/NULL - ]; tight)] │ │ │ └── filters (true) │ │ └── projections - │ │ └── ',' [as=column7:7] + │ │ └── ',' [as=column9:9] │ └── aggregations - │ └── string-agg [as=string_agg:8, outer=(6,7)] - │ ├── s:6 - │ └── column7:7 + │ └── string-agg [as=string_agg:10, outer=(7,9)] + │ ├── s:7 + │ └── column9:9 └── filters - └── string_agg:8 = 'foo' [outer=(8), constraints=(/8: [/'foo' - /'foo']; tight), fd=()-->(8)] + └── string_agg:10 = 'foo' [outer=(10), constraints=(/10: [/'foo' - /'foo']; tight), fd=()-->(10)] # Don't reject nulls when aggregate argument is a not a Project passthrough # column. @@ -648,42 +648,42 @@ GROUP BY k HAVING string_agg(s || 'bar', ',')='foo' ---- project - ├── columns: string_agg:9!null + ├── columns: string_agg:11!null ├── immutable - ├── fd: ()-->(9) + ├── fd: ()-->(11) └── select - ├── columns: k:3 string_agg:9!null + ├── columns: k:4 string_agg:11!null ├── immutable - ├── key: (3) - ├── fd: ()-->(9) + ├── key: (4) + ├── fd: ()-->(11) ├── group-by - │ ├── columns: k:3 string_agg:9 - │ ├── grouping columns: k:3 + │ ├── columns: k:4 string_agg:11 + │ ├── grouping columns: k:4 │ ├── immutable - │ ├── key: (3) - │ ├── fd: (3)-->(9) + │ ├── key: (4) + │ ├── fd: (4)-->(11) │ ├── project - │ │ ├── columns: column7:7 column8:8!null k:3 + │ │ ├── columns: column9:9 column10:10!null k:4 │ │ ├── immutable - │ │ ├── fd: ()-->(8), (3)-->(7) + │ │ ├── fd: ()-->(10), (4)-->(9) │ │ ├── left-join (cross) - │ │ │ ├── columns: k:3 s:6 - │ │ │ ├── fd: (3)-->(6) + │ │ │ ├── columns: k:4 s:7 + │ │ │ ├── fd: (4)-->(7) │ │ │ ├── scan xy │ │ │ ├── scan a - │ │ │ │ ├── columns: k:3!null s:6 - │ │ │ │ ├── key: (3) - │ │ │ │ └── fd: (3)-->(6) + │ │ │ │ ├── columns: k:4!null s:7 + │ │ │ │ ├── key: (4) + │ │ │ │ └── fd: (4)-->(7) │ │ │ └── filters (true) │ │ └── projections - │ │ ├── s:6 || 'bar' [as=column7:7, outer=(6), immutable] - │ │ └── ',' [as=column8:8] + │ │ ├── s:7 || 'bar' [as=column9:9, outer=(7), immutable] + │ │ └── ',' [as=column10:10] │ └── aggregations - │ └── string-agg [as=string_agg:9, outer=(7,8)] - │ ├── column7:7 - │ └── column8:8 + │ └── string-agg [as=string_agg:11, outer=(9,10)] + │ ├── column9:9 + │ └── column10:10 └── filters - └── string_agg:9 = 'foo' [outer=(9), constraints=(/9: [/'foo' - /'foo']; tight), fd=()-->(9)] + └── string_agg:11 = 'foo' [outer=(11), constraints=(/11: [/'foo' - /'foo']; tight), fd=()-->(11)] # Regression test: the not-null filter can't make it all the way down to the # join that requested it, so ensure that we don't endlessly try to introduce @@ -715,47 +715,47 @@ exprnorm ) ---- select - ├── columns: sum:6!null + ├── columns: sum:8!null ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(6) + ├── fd: ()-->(8) ├── scalar-group-by - │ ├── columns: sum:6 + │ ├── columns: sum:8 │ ├── cardinality: [1 - 1] │ ├── immutable │ ├── key: () - │ ├── fd: ()-->(6) + │ ├── fd: ()-->(8) │ ├── inner-join-apply - │ │ ├── columns: x:1!null u:3!null z:5 + │ │ ├── columns: x:1!null u:4!null z:7 │ │ ├── immutable - │ │ ├── key: (1,3) - │ │ ├── fd: (1,3)-->(5) + │ │ ├── key: (1,4) + │ │ ├── fd: (1,4)-->(7) │ │ ├── scan xy │ │ │ ├── columns: x:1!null │ │ │ └── key: (1) │ │ ├── left-join-apply - │ │ │ ├── columns: u:3!null z:5 + │ │ │ ├── columns: u:4!null z:7 │ │ │ ├── outer: (1) │ │ │ ├── immutable - │ │ │ ├── key: (3) - │ │ │ ├── fd: (3)-->(5) + │ │ │ ├── key: (4) + │ │ │ ├── fd: (4)-->(7) │ │ │ ├── scan uv - │ │ │ │ ├── columns: u:3!null - │ │ │ │ └── key: (3) + │ │ │ │ ├── columns: u:4!null + │ │ │ │ └── key: (4) │ │ │ ├── values - │ │ │ │ ├── columns: z:5 - │ │ │ │ ├── outer: (1,3) + │ │ │ │ ├── columns: z:7 + │ │ │ │ ├── outer: (1,4) │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── immutable │ │ │ │ ├── key: () - │ │ │ │ ├── fd: ()-->(5) - │ │ │ │ └── (x:1 + u:3,) + │ │ │ │ ├── fd: ()-->(7) + │ │ │ │ └── (x:1 + u:4,) │ │ │ └── filters │ │ │ └── x:1 = 3 [outer=(1), constraints=(/1: [/3 - /3]; tight), fd=()-->(1)] │ │ └── filters (true) │ └── aggregations - │ └── sum [as=sum:6, outer=(5)] - │ └── z:5 + │ └── sum [as=sum:8, outer=(7)] + │ └── z:7 └── filters - └── sum:6 = 10 [outer=(6), constraints=(/6: [/10 - /10]; tight), fd=()-->(6)] + └── sum:8 = 10 [outer=(8), constraints=(/8: [/10 - /10]; tight), fd=()-->(8)] diff --git a/pkg/sql/opt/norm/testdata/rules/scalar b/pkg/sql/opt/norm/testdata/rules/scalar index a4b3e83185b1..bb2ef64cb7c2 100644 --- a/pkg/sql/opt/norm/testdata/rules/scalar +++ b/pkg/sql/opt/norm/testdata/rules/scalar @@ -27,22 +27,22 @@ SELECT FROM a ---- project - ├── columns: r:7 s:8 t:9!null u:10!null v:11 w:12 x:13 y:14 z:15 + ├── columns: r:8 s:9 t:10!null u:11!null v:12 w:13 x:14 y:15 z:16 ├── immutable ├── scan a │ ├── columns: k:1!null i:2 │ ├── key: (1) │ └── fd: (1)-->(2) └── projections - ├── k:1 = (i:2 + 1) [as=r:7, outer=(1,2), immutable] - ├── i:2 != (2 - k:1) [as=s:8, outer=(1,2), immutable] - ├── k:1 IS NOT DISTINCT FROM (i:2 + 1) [as=t:9, outer=(1,2), immutable] - ├── k:1 IS DISTINCT FROM (i:2 - 1) [as=u:10, outer=(1,2), immutable] - ├── k:1 + (i:2 * 2) [as=v:11, outer=(1,2), immutable] - ├── k:1 * (i:2 + 2) [as=w:12, outer=(1,2), immutable] - ├── k:1 & (i:2 ^ 2) [as=x:13, outer=(1,2), immutable] - ├── k:1 | (i:2 ^ 2) [as=y:14, outer=(1,2), immutable] - └── k:1 # (i:2 * i:2) [as=z:15, outer=(1,2), immutable] + ├── k:1 = (i:2 + 1) [as=r:8, outer=(1,2), immutable] + ├── i:2 != (2 - k:1) [as=s:9, outer=(1,2), immutable] + ├── k:1 IS NOT DISTINCT FROM (i:2 + 1) [as=t:10, outer=(1,2), immutable] + ├── k:1 IS DISTINCT FROM (i:2 - 1) [as=u:11, outer=(1,2), immutable] + ├── k:1 + (i:2 * 2) [as=v:12, outer=(1,2), immutable] + ├── k:1 * (i:2 + 2) [as=w:13, outer=(1,2), immutable] + ├── k:1 & (i:2 ^ 2) [as=x:14, outer=(1,2), immutable] + ├── k:1 | (i:2 ^ 2) [as=y:15, outer=(1,2), immutable] + └── k:1 # (i:2 * i:2) [as=z:16, outer=(1,2), immutable] # -------------------------------------------------- # CommuteConst @@ -62,22 +62,22 @@ SELECT FROM a ---- project - ├── columns: r:7 s:8 t:9!null u:10!null v:11 w:12 x:13 y:14 z:15!null + ├── columns: r:8 s:9 t:10!null u:11!null v:12 w:13 x:14 y:15 z:16!null ├── immutable ├── scan a │ ├── columns: k:1!null i:2 f:3 │ ├── key: (1) │ └── fd: (1)-->(2,3) └── projections - ├── (i:2 + k:1) = 4 [as=r:7, outer=(1,2), immutable] - ├── (i:2 * 2) != 3 [as=s:8, outer=(2), immutable] - ├── (1 - k:1) IS NOT DISTINCT FROM 5 [as=t:9, outer=(1), immutable] - ├── k:1 IS DISTINCT FROM 11 [as=u:10, outer=(1)] - ├── f:3 + 1.0 [as=v:11, outer=(3), immutable] - ├── (i:2 * i:2) * 15 [as=w:12, outer=(2), immutable] - ├── (i:2 + i:2) & 10000 [as=x:13, outer=(2), immutable] - ├── (i:2 + i:2) | 4 [as=y:14, outer=(2), immutable] - └── (k:1 ^ 2) # -2 [as=z:15, outer=(1), immutable] + ├── (i:2 + k:1) = 4 [as=r:8, outer=(1,2), immutable] + ├── (i:2 * 2) != 3 [as=s:9, outer=(2), immutable] + ├── (1 - k:1) IS NOT DISTINCT FROM 5 [as=t:10, outer=(1), immutable] + ├── k:1 IS DISTINCT FROM 11 [as=u:11, outer=(1)] + ├── f:3 + 1.0 [as=v:12, outer=(3), immutable] + ├── (i:2 * i:2) * 15 [as=w:13, outer=(2), immutable] + ├── (i:2 + i:2) & 10000 [as=x:14, outer=(2), immutable] + ├── (i:2 + i:2) | 4 [as=y:15, outer=(2), immutable] + └── (k:1 ^ 2) # -2 [as=z:16, outer=(1), immutable] # -------------------------------------------------- # EliminateCoalesce @@ -86,21 +86,21 @@ norm expect=EliminateCoalesce SELECT COALESCE(i) FROM a ---- project - ├── columns: coalesce:7 + ├── columns: coalesce:8 ├── scan a │ └── columns: i:2 └── projections - └── i:2 [as=coalesce:7, outer=(2)] + └── i:2 [as=coalesce:8, outer=(2)] norm expect=EliminateCoalesce SELECT COALESCE(NULL) FROM a ---- project - ├── columns: coalesce:7 - ├── fd: ()-->(7) + ├── columns: coalesce:8 + ├── fd: ()-->(8) ├── scan a └── projections - └── NULL [as=coalesce:7] + └── NULL [as=coalesce:8] # -------------------------------------------------- # SimplifyCoalesce @@ -110,43 +110,43 @@ norm expect=SimplifyCoalesce SELECT COALESCE(NULL, 'foo', s) FROM a ---- project - ├── columns: coalesce:7!null - ├── fd: ()-->(7) + ├── columns: coalesce:8!null + ├── fd: ()-->(8) ├── scan a └── projections - └── 'foo' [as=coalesce:7] + └── 'foo' [as=coalesce:8] norm expect=SimplifyCoalesce SELECT COALESCE(NULL, NULL, s, s || 'foo') FROM a ---- project - ├── columns: coalesce:7 + ├── columns: coalesce:8 ├── immutable ├── scan a │ └── columns: s:4 └── projections - └── COALESCE(s:4, s:4 || 'foo') [as=coalesce:7, outer=(4), immutable] + └── COALESCE(s:4, s:4 || 'foo') [as=coalesce:8, outer=(4), immutable] # Trailing null can't be removed. norm SELECT COALESCE(i, NULL, NULL) FROM a ---- project - ├── columns: coalesce:7 + ├── columns: coalesce:8 ├── scan a │ └── columns: i:2 └── projections - └── COALESCE(i:2, CAST(NULL AS INT8), CAST(NULL AS INT8)) [as=coalesce:7, outer=(2)] + └── COALESCE(i:2, CAST(NULL AS INT8), CAST(NULL AS INT8)) [as=coalesce:8, outer=(2)] norm expect=SimplifyCoalesce SELECT COALESCE((1, 2, 3), (2, 3, 4)) FROM a ---- project - ├── columns: coalesce:7!null - ├── fd: ()-->(7) + ├── columns: coalesce:8!null + ├── fd: ()-->(8) ├── scan a └── projections - └── (1, 2, 3) [as=coalesce:7] + └── (1, 2, 3) [as=coalesce:8] # -------------------------------------------------- @@ -174,16 +174,16 @@ exprnorm expect=EliminateCast ) ---- project - ├── columns: c1:7 c2:8 c3:9!null c4:10 c5:11 - ├── fd: ()-->(9,10) + ├── columns: c1:8 c2:9 c3:10!null c4:11 c5:12 + ├── fd: ()-->(10,11) ├── scan a │ └── columns: i:2 s:4 arr:6 └── projections - ├── i:2 [as=c1:7, outer=(2)] - ├── arr:6 [as=c2:8, outer=(6)] - ├── '[1, 2]' [as=c3:9] - ├── CAST(NULL AS BIT) [as=c4:10] - └── s:4 [as=c5:11, outer=(4)] + ├── i:2 [as=c1:8, outer=(2)] + ├── arr:6 [as=c2:9, outer=(6)] + ├── '[1, 2]' [as=c3:10] + ├── CAST(NULL AS BIT) [as=c4:11] + └── s:4 [as=c5:12, outer=(4)] # Shouldn't eliminate these casts. norm expect-not=EliminateCast @@ -199,19 +199,19 @@ SELECT FROM a ---- project - ├── columns: i:7 arr:8 s:9 s:10 i:11 s:12 array:13 array:14 + ├── columns: i:8 arr:9 s:10 s:11 i:12 s:13 array:14 array:15 ├── stable ├── scan a │ └── columns: a.i:2 a.s:4 a.arr:6 └── projections - ├── a.i:2::FLOAT8 [as=i:7, outer=(2), immutable] - ├── a.arr:6::DECIMAL[] [as=arr:8, outer=(6), immutable] - ├── a.s:4::JSONB [as=s:9, outer=(4), immutable] - ├── a.s:4::VARCHAR(2) [as=s:10, outer=(4), immutable] - ├── a.i:2::INT2::INT8 [as=i:11, outer=(2), immutable] - ├── a.s:4::CHAR::VARCHAR [as=s:12, outer=(4), immutable] - ├── ARRAY[a.i:2, 2]::OIDVECTOR [as=array:13, outer=(2), stable] - └── ARRAY[a.i:2, 2]::INT2VECTOR [as=array:14, outer=(2), immutable] + ├── a.i:2::FLOAT8 [as=i:8, outer=(2), immutable] + ├── a.arr:6::DECIMAL[] [as=arr:9, outer=(6), immutable] + ├── a.s:4::JSONB [as=s:10, outer=(4), immutable] + ├── a.s:4::VARCHAR(2) [as=s:11, outer=(4), immutable] + ├── a.i:2::INT2::INT8 [as=i:12, outer=(2), immutable] + ├── a.s:4::CHAR::VARCHAR [as=s:13, outer=(4), immutable] + ├── ARRAY[a.i:2, 2]::OIDVECTOR [as=array:14, outer=(2), stable] + └── ARRAY[a.i:2, 2]::INT2VECTOR [as=array:15, outer=(2), immutable] # -------------------------------------------------- # NormalizeInConst @@ -220,34 +220,34 @@ norm expect=NormalizeInConst SELECT i IN (2, 1, 1, null, 3, 4.00, 4.0, null, 3.0) AS r FROM a ---- project - ├── columns: r:7 + ├── columns: r:8 ├── scan a │ └── columns: i:2 └── projections - └── i:2 IN (NULL, 1, 2, 3, 4) [as=r:7, outer=(2)] + └── i:2 IN (NULL, 1, 2, 3, 4) [as=r:8, outer=(2)] # Single value. norm expect-not=NormalizeInConst SELECT s NOT IN ('foo') AS r FROM a ---- project - ├── columns: r:7 + ├── columns: r:8 ├── scan a │ └── columns: s:4 └── projections - └── s:4 NOT IN ('foo',) [as=r:7, outer=(4)] + └── s:4 NOT IN ('foo',) [as=r:8, outer=(4)] # Don't sort, since the list is not constant. norm expect-not=NormalizeInConst SELECT s NOT IN ('foo', s || 'foo', 'bar', length(s)::string, NULL) AS r FROM a ---- project - ├── columns: r:7 + ├── columns: r:8 ├── immutable ├── scan a │ └── columns: s:4 └── projections - └── s:4 NOT IN ('foo', s:4 || 'foo', 'bar', length(s:4)::STRING, NULL) [as=r:7, outer=(4), immutable] + └── s:4 NOT IN ('foo', s:4 || 'foo', 'bar', length(s:4)::STRING, NULL) [as=r:8, outer=(4), immutable] # Regression test #36031. norm expect-not=NormalizeInConst @@ -302,14 +302,14 @@ select └── filters └── exists [subquery] └── limit - ├── columns: k:7!null i:8 + ├── columns: k:8!null i:9 ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(7,8) + ├── fd: ()-->(8,9) ├── scan a - │ ├── columns: k:7!null i:8 - │ ├── key: (7) - │ ├── fd: (7)-->(8) + │ ├── columns: k:8!null i:9 + │ ├── key: (8) + │ ├── fd: (8)-->(9) │ └── limit hint: 1.00 └── 1 @@ -332,18 +332,18 @@ select └── filters └── exists [subquery] └── scalar-group-by - ├── columns: max:13 + ├── columns: max:15 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(13) + ├── fd: ()-->(15) ├── values - │ ├── columns: s:10!null + │ ├── columns: s:11!null │ ├── cardinality: [0 - 0] │ ├── key: () - │ └── fd: ()-->(10) + │ └── fd: ()-->(11) └── aggregations - └── max [as=max:13, outer=(10)] - └── s:10 + └── max [as=max:15, outer=(11)] + └── s:11 norm expect=EliminateExistsGroupBy SELECT * FROM a WHERE EXISTS(SELECT DISTINCT s FROM a) @@ -359,12 +359,12 @@ select └── filters └── exists [subquery] └── limit - ├── columns: s:10 + ├── columns: s:11 ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(10) + ├── fd: ()-->(11) ├── scan a - │ ├── columns: s:10 + │ ├── columns: s:11 │ └── limit hint: 1.00 └── 1 @@ -382,12 +382,12 @@ select └── filters └── exists [subquery] └── limit - ├── columns: i:8 s:10 + ├── columns: i:9 s:11 ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(8,10) + ├── fd: ()-->(9,11) ├── scan a - │ ├── columns: i:8 s:10 + │ ├── columns: i:9 s:11 │ └── limit hint: 1.00 └── 1 @@ -406,30 +406,30 @@ select └── filters └── exists [subquery] └── limit - ├── columns: k:7!null xy.y:14 + ├── columns: k:8!null xy.y:16 ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(7,14) + ├── fd: ()-->(8,16) ├── ensure-distinct-on - │ ├── columns: k:7!null xy.y:14 - │ ├── grouping columns: k:7!null + │ ├── columns: k:8!null xy.y:16 + │ ├── grouping columns: k:8!null │ ├── error: "more than one row returned by a subquery used as an expression" - │ ├── key: (7) - │ ├── fd: (7)-->(14) + │ ├── key: (8) + │ ├── fd: (8)-->(16) │ ├── limit hint: 1.00 │ ├── left-join (hash) - │ │ ├── columns: k:7!null xy.y:14 + │ │ ├── columns: k:8!null xy.y:16 │ │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) │ │ ├── scan a - │ │ │ ├── columns: k:7!null - │ │ │ └── key: (7) + │ │ │ ├── columns: k:8!null + │ │ │ └── key: (8) │ │ ├── scan xy - │ │ │ └── columns: xy.y:14 + │ │ │ └── columns: xy.y:16 │ │ └── filters - │ │ └── xy.y:14 = k:7 [outer=(7,14), constraints=(/7: (/NULL - ]; /14: (/NULL - ]), fd=(7)==(14), (14)==(7)] + │ │ └── xy.y:16 = k:8 [outer=(8,16), constraints=(/8: (/NULL - ]; /16: (/NULL - ]), fd=(8)==(16), (16)==(8)] │ └── aggregations - │ └── const-agg [as=xy.y:14, outer=(14)] - │ └── xy.y:14 + │ └── const-agg [as=xy.y:16, outer=(16)] + │ └── xy.y:16 └── 1 # -------------------------------------------------- @@ -449,12 +449,12 @@ select └── filters └── exists [subquery] └── limit - ├── columns: i:8 s:10 + ├── columns: i:9 s:11 ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(8,10) + ├── fd: ()-->(9,11) ├── scan a - │ ├── columns: i:8 s:10 + │ ├── columns: i:9 s:11 │ └── limit hint: 1.00 └── 1 @@ -475,12 +475,12 @@ select └── filters └── exists [subquery] └── limit - ├── columns: i:8 + ├── columns: i:9 ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(8) + ├── fd: ()-->(9) ├── scan a - │ ├── columns: i:8 + │ ├── columns: i:9 │ └── limit hint: 1.00 └── 1 @@ -497,9 +497,9 @@ semi-join (hash) │ ├── key: (1) │ └── fd: (1)-->(2-6) ├── scan a2 - │ └── columns: a2.i:8 + │ └── columns: a2.i:9 └── filters - └── a1.i:2 = a2.i:8 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] + └── a1.i:2 = a2.i:9 [outer=(2,9), constraints=(/2: (/NULL - ]; /9: (/NULL - ]), fd=(2)==(9), (9)==(2)] # Don't introduce a limit when the subquery has one row. norm expect-not=IntroduceExistsLimit @@ -516,10 +516,10 @@ select └── filters └── exists [subquery] └── values - ├── columns: column1:7!null + ├── columns: column1:8!null ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(7) + ├── fd: ()-->(8) └── (1,) # -------------------------------------------------- @@ -537,9 +537,9 @@ semi-join (hash) │ ├── key: (1) │ └── fd: (1)-->(2-6) ├── scan a2 - │ └── columns: a2.i:8 + │ └── columns: a2.i:9 └── filters - └── a1.i:2 = a2.i:8 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] + └── a1.i:2 = a2.i:9 [outer=(2,9), constraints=(/2: (/NULL - ]; /9: (/NULL - ]), fd=(2)==(9), (9)==(2)] norm expect=EliminateExistsLimit SELECT * FROM a a1 WHERE NOT EXISTS(SELECT i FROM a a2 where a1.i = a2.i LIMIT 1) @@ -553,9 +553,9 @@ anti-join (hash) │ ├── key: (1) │ └── fd: (1)-->(2-6) ├── scan a2 - │ └── columns: a2.i:8 + │ └── columns: a2.i:9 └── filters - └── a1.i:2 = a2.i:8 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] + └── a1.i:2 = a2.i:9 [outer=(2,9), constraints=(/2: (/NULL - ]; /9: (/NULL - ]), fd=(2)==(9), (9)==(2)] # Don't eliminate a non-positive limit. norm expect-not=EliminateExistsLimit @@ -582,14 +582,14 @@ select └── filters └── exists [subquery] └── limit - ├── columns: k:7!null i:8 f:9 s:10 j:11 arr:12 + ├── columns: k:8!null i:9 f:10 s:11 j:12 arr:13 ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(7-12) + ├── fd: ()-->(8-13) ├── scan a - │ ├── columns: k:7!null i:8 f:9 s:10 j:11 arr:12 - │ ├── key: (7) - │ ├── fd: (7)-->(8-12) + │ ├── columns: k:8!null i:9 f:10 s:11 j:12 arr:13 + │ ├── key: (8) + │ ├── fd: (8)-->(9-13) │ └── limit hint: 1.00 └── 1 @@ -664,13 +664,13 @@ norm SELECT j->'a' = '["b"]'::JSON, j->'a' = '{"b": "c"}'::JSON FROM a ---- project - ├── columns: "?column?":7 "?column?":8 + ├── columns: "?column?":8 "?column?":9 ├── immutable ├── scan a │ └── columns: j:5 └── projections - ├── (j:5->'a') = '["b"]' [as="?column?":7, outer=(5), immutable] - └── (j:5->'a') = '{"b": "c"}' [as="?column?":8, outer=(5), immutable] + ├── (j:5->'a') = '["b"]' [as="?column?":8, outer=(5), immutable] + └── (j:5->'a') = '{"b": "c"}' [as="?column?":9, outer=(5), immutable] # -------------------------------------------------- # NormalizeJSONContains @@ -788,12 +788,12 @@ FROM a ---- project - ├── columns: case:7!null + ├── columns: case:8!null ├── scan a │ ├── columns: k:1!null │ └── key: (1) └── projections - └── CASE 1 WHEN k:1 THEN 'one' ELSE 'two' END [as=case:7, outer=(1)] + └── CASE 1 WHEN k:1 THEN 'one' ELSE 'two' END [as=case:8, outer=(1)] norm expect=SimplifyCaseWhenConstValue SELECT @@ -802,12 +802,12 @@ FROM a ---- project - ├── columns: case:7!null + ├── columns: case:8!null ├── scan a │ ├── columns: k:1!null │ └── key: (1) └── projections - └── CASE WHEN k:1 = 1 THEN 'one' ELSE 'two' END [as=case:7, outer=(1)] + └── CASE WHEN k:1 = 1 THEN 'one' ELSE 'two' END [as=case:8, outer=(1)] norm expect=SimplifyCaseWhenConstValue SELECT CASE 1 WHEN 2 THEN 'one' ELSE 'three' END @@ -831,12 +831,12 @@ FROM a ---- project - ├── columns: case:7!null + ├── columns: case:8!null ├── scan a │ ├── columns: k:1!null │ └── key: (1) └── projections - └── CASE 1 WHEN k:1 THEN 'two' ELSE 'three' END [as=case:7, outer=(1)] + └── CASE 1 WHEN k:1 THEN 'two' ELSE 'three' END [as=case:8, outer=(1)] norm expect=SimplifyCaseWhenConstValue SELECT @@ -1199,21 +1199,21 @@ select └── tuple ├── subquery │ └── max1-row - │ ├── columns: "?column?":13 + │ ├── columns: "?column?":15 │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── cardinality: [0 - 1] │ ├── immutable │ ├── key: () - │ ├── fd: ()-->(13) + │ ├── fd: ()-->(15) │ └── project - │ ├── columns: "?column?":13 + │ ├── columns: "?column?":15 │ ├── immutable │ ├── scan a - │ │ ├── columns: k:7!null i:8 - │ │ ├── key: (7) - │ │ └── fd: (7)-->(8) + │ │ ├── columns: k:8!null i:9 + │ │ ├── key: (8) + │ │ └── fd: (8)-->(9) │ └── projections - │ └── k:7 * i:8 [as="?column?":13, outer=(7,8), immutable] + │ └── k:8 * i:9 [as="?column?":15, outer=(8,9), immutable] ├── 2 └── 3 @@ -1249,33 +1249,33 @@ project ├── immutable ├── key: (1) └── semi-join (hash) - ├── columns: k:1!null column10:10 + ├── columns: k:1!null column11:11 ├── immutable ├── key: (1) - ├── fd: (1)-->(10) + ├── fd: (1)-->(11) ├── project - │ ├── columns: column10:10 k:1!null + │ ├── columns: column11:11 k:1!null │ ├── key: (1) - │ ├── fd: (1)-->(10) + │ ├── fd: (1)-->(11) │ ├── scan a │ │ ├── columns: k:1!null i:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ └── projections - │ └── (k:1, i:2) [as=column10:10, outer=(1,2)] + │ └── (k:1, i:2) [as=column11:11, outer=(1,2)] ├── project - │ ├── columns: column9:9!null + │ ├── columns: column10:10!null │ ├── cardinality: [3 - 3] │ ├── values - │ │ ├── columns: column1:7!null column2:8!null + │ │ ├── columns: column1:8!null column2:9!null │ │ ├── cardinality: [3 - 3] │ │ ├── (1, 1) │ │ ├── (2, 2) │ │ └── (3, 3) │ └── projections - │ └── (column2:8, column1:7) [as=column9:9, outer=(7,8)] + │ └── (column2:9, column1:8) [as=column10:10, outer=(8,9)] └── filters - └── column10:10 = column9:9 [outer=(9,10), immutable, constraints=(/9: (/NULL - ]; /10: (/NULL - ]), fd=(9)==(10), (10)==(9)] + └── column11:11 = column10:10 [outer=(10,11), immutable, constraints=(/10: (/NULL - ]; /11: (/NULL - ]), fd=(10)==(11), (11)==(10)] # -------------------------------------------------- # SimplifyEqualsAnyTuple @@ -1426,11 +1426,11 @@ norm expect-not=FoldCollate SELECT s COLLATE en_u_ks_level1 FROM a ---- project - ├── columns: s:7 + ├── columns: s:8 ├── scan a │ └── columns: a.s:4 └── projections - └── a.s:4 COLLATE en_u_ks_level1 [as=s:7, outer=(4)] + └── a.s:4 COLLATE en_u_ks_level1 [as=s:8, outer=(4)] # -------------------------------------------------- # NormalizeArrayFlattenToAgg @@ -1440,102 +1440,102 @@ norm expect=NormalizeArrayFlattenToAgg SELECT ARRAY(SELECT k FROM a WHERE a.k = b.k) FROM a AS b ---- project - ├── columns: array:14 + ├── columns: array:16 ├── group-by - │ ├── columns: b.k:1!null a.k:7!null array_agg:15!null + │ ├── columns: b.k:1!null a.k:8!null array_agg:17!null │ ├── grouping columns: b.k:1!null - │ ├── key: (7) - │ ├── fd: (1)==(7), (7)==(1), (7)-->(15), (1)-->(7,15) + │ ├── key: (8) + │ ├── fd: (1)==(8), (8)==(1), (8)-->(17), (1)-->(8,17) │ ├── inner-join (hash) - │ │ ├── columns: b.k:1!null a.k:7!null + │ │ ├── columns: b.k:1!null a.k:8!null │ │ ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) - │ │ ├── key: (7) - │ │ ├── fd: (1)==(7), (7)==(1) + │ │ ├── key: (8) + │ │ ├── fd: (1)==(8), (8)==(1) │ │ ├── scan b │ │ │ ├── columns: b.k:1!null │ │ │ └── key: (1) │ │ ├── scan a - │ │ │ ├── columns: a.k:7!null - │ │ │ └── key: (7) + │ │ │ ├── columns: a.k:8!null + │ │ │ └── key: (8) │ │ └── filters - │ │ └── a.k:7 = b.k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + │ │ └── a.k:8 = b.k:1 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] │ └── aggregations - │ ├── array-agg [as=array_agg:15, outer=(7)] - │ │ └── a.k:7 - │ └── any-not-null-agg [as=a.k:7, outer=(7)] - │ └── a.k:7 + │ ├── array-agg [as=array_agg:17, outer=(8)] + │ │ └── a.k:8 + │ └── any-not-null-agg [as=a.k:8, outer=(8)] + │ └── a.k:8 └── projections - └── COALESCE(CASE WHEN a.k:7 IS NOT NULL THEN array_agg:15 ELSE CAST(NULL AS INT8[]) END, ARRAY[]) [as=array:14, outer=(7,15)] + └── COALESCE(CASE WHEN a.k:8 IS NOT NULL THEN array_agg:17 ELSE CAST(NULL AS INT8[]) END, ARRAY[]) [as=array:16, outer=(8,17)] # Ensure ordering is maintained. norm expect=NormalizeArrayFlattenToAgg SELECT ARRAY(SELECT k FROM a WHERE a.i = b.i ORDER BY a.k) FROM a AS b ---- project - ├── columns: array:14 + ├── columns: array:16 ├── group-by - │ ├── columns: b.k:1!null a.k:7 array_agg:15 + │ ├── columns: b.k:1!null a.k:8 array_agg:17 │ ├── grouping columns: b.k:1!null - │ ├── internal-ordering: +7 opt(8) + │ ├── internal-ordering: +8 opt(9) │ ├── key: (1) - │ ├── fd: (1)-->(7,15) + │ ├── fd: (1)-->(8,17) │ ├── sort - │ │ ├── columns: b.k:1!null b.i:2 a.k:7 a.i:8 - │ │ ├── key: (1,7) - │ │ ├── fd: (1)-->(2), (7)-->(8) - │ │ ├── ordering: +7 opt(8) [actual: +7] + │ │ ├── columns: b.k:1!null b.i:2 a.k:8 a.i:9 + │ │ ├── key: (1,8) + │ │ ├── fd: (1)-->(2), (8)-->(9) + │ │ ├── ordering: +8 opt(9) [actual: +8] │ │ └── left-join (hash) - │ │ ├── columns: b.k:1!null b.i:2 a.k:7 a.i:8 - │ │ ├── key: (1,7) - │ │ ├── fd: (1)-->(2), (7)-->(8) + │ │ ├── columns: b.k:1!null b.i:2 a.k:8 a.i:9 + │ │ ├── key: (1,8) + │ │ ├── fd: (1)-->(2), (8)-->(9) │ │ ├── scan b │ │ │ ├── columns: b.k:1!null b.i:2 │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2) │ │ ├── scan a - │ │ │ ├── columns: a.k:7!null a.i:8 - │ │ │ ├── key: (7) - │ │ │ └── fd: (7)-->(8) + │ │ │ ├── columns: a.k:8!null a.i:9 + │ │ │ ├── key: (8) + │ │ │ └── fd: (8)-->(9) │ │ └── filters - │ │ └── a.i:8 = b.i:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] + │ │ └── a.i:9 = b.i:2 [outer=(2,9), constraints=(/2: (/NULL - ]; /9: (/NULL - ]), fd=(2)==(9), (9)==(2)] │ └── aggregations - │ ├── array-agg [as=array_agg:15, outer=(7)] - │ │ └── a.k:7 - │ └── any-not-null-agg [as=a.k:7, outer=(7)] - │ └── a.k:7 + │ ├── array-agg [as=array_agg:17, outer=(8)] + │ │ └── a.k:8 + │ └── any-not-null-agg [as=a.k:8, outer=(8)] + │ └── a.k:8 └── projections - └── COALESCE(CASE WHEN a.k:7 IS NOT NULL THEN array_agg:15 ELSE CAST(NULL AS INT8[]) END, ARRAY[]) [as=array:14, outer=(7,15)] + └── COALESCE(CASE WHEN a.k:8 IS NOT NULL THEN array_agg:17 ELSE CAST(NULL AS INT8[]) END, ARRAY[]) [as=array:16, outer=(8,17)] norm expect=NormalizeArrayFlattenToAgg SELECT ARRAY(SELECT generate_series(1, a.k) ORDER BY 1 DESC) FROM a ---- project - ├── columns: array:9 + ├── columns: array:10 ├── immutable ├── group-by - │ ├── columns: k:1!null canary:10 array_agg:11 + │ ├── columns: k:1!null canary:11 array_agg:12 │ ├── grouping columns: k:1!null - │ ├── internal-ordering: -7 + │ ├── internal-ordering: -8 │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(10,11) + │ ├── fd: (1)-->(11,12) │ ├── sort - │ │ ├── columns: k:1!null generate_series:7 canary:10 + │ │ ├── columns: k:1!null generate_series:8 canary:11 │ │ ├── immutable - │ │ ├── ordering: -7 + │ │ ├── ordering: -8 │ │ └── left-join-apply - │ │ ├── columns: k:1!null generate_series:7 canary:10 + │ │ ├── columns: k:1!null generate_series:8 canary:11 │ │ ├── immutable │ │ ├── scan a │ │ │ ├── columns: k:1!null │ │ │ └── key: (1) │ │ ├── project - │ │ │ ├── columns: canary:10!null generate_series:7 + │ │ │ ├── columns: canary:11!null generate_series:8 │ │ │ ├── outer: (1) │ │ │ ├── immutable - │ │ │ ├── fd: ()-->(10) + │ │ │ ├── fd: ()-->(11) │ │ │ ├── project-set - │ │ │ │ ├── columns: generate_series:7 + │ │ │ │ ├── columns: generate_series:8 │ │ │ │ ├── outer: (1) │ │ │ │ ├── immutable │ │ │ │ ├── values @@ -1545,143 +1545,143 @@ project │ │ │ │ └── zip │ │ │ │ └── generate_series(1, k:1) [outer=(1), immutable] │ │ │ └── projections - │ │ │ └── true [as=canary:10] + │ │ │ └── true [as=canary:11] │ │ └── filters (true) │ └── aggregations - │ ├── array-agg [as=array_agg:11, outer=(7)] - │ │ └── generate_series:7 - │ └── any-not-null-agg [as=canary:10, outer=(10)] - │ └── canary:10 + │ ├── array-agg [as=array_agg:12, outer=(8)] + │ │ └── generate_series:8 + │ └── any-not-null-agg [as=canary:11, outer=(11)] + │ └── canary:11 └── projections - └── COALESCE(CASE WHEN canary:10 IS NOT NULL THEN array_agg:11 ELSE CAST(NULL AS INT8[]) END, ARRAY[]) [as=array:9, outer=(10,11)] + └── COALESCE(CASE WHEN canary:11 IS NOT NULL THEN array_agg:12 ELSE CAST(NULL AS INT8[]) END, ARRAY[]) [as=array:10, outer=(11,12)] # Uncorrelated ArrayFlatten inside a correlated ArrayFlatten. norm expect=NormalizeArrayFlattenToAgg SELECT ARRAY(SELECT ARRAY(SELECT k FROM a)[1] FROM a as b WHERE b.k = c.k) FROM a AS c ---- project - ├── columns: array:21 + ├── columns: array:24 ├── group-by - │ ├── columns: c.k:1!null canary:22!null array_agg:23 + │ ├── columns: c.k:1!null canary:25!null array_agg:26 │ ├── grouping columns: c.k:1!null │ ├── key: (1) - │ ├── fd: ()-->(22), (1)-->(22,23) + │ ├── fd: ()-->(25), (1)-->(25,26) │ ├── inner-join (hash) - │ │ ├── columns: c.k:1!null b.k:7!null array:19 canary:22!null + │ │ ├── columns: c.k:1!null b.k:8!null array:22 canary:25!null │ │ ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) - │ │ ├── key: (7) - │ │ ├── fd: ()-->(19,22), (1)==(7), (7)==(1) + │ │ ├── key: (8) + │ │ ├── fd: ()-->(22,25), (1)==(8), (8)==(1) │ │ ├── scan c │ │ │ ├── columns: c.k:1!null │ │ │ └── key: (1) │ │ ├── project - │ │ │ ├── columns: canary:22!null array:19 b.k:7!null - │ │ │ ├── key: (7) - │ │ │ ├── fd: ()-->(19,22) + │ │ │ ├── columns: canary:25!null array:22 b.k:8!null + │ │ │ ├── key: (8) + │ │ │ ├── fd: ()-->(22,25) │ │ │ ├── scan b - │ │ │ │ ├── columns: b.k:7!null - │ │ │ │ └── key: (7) + │ │ │ │ ├── columns: b.k:8!null + │ │ │ │ └── key: (8) │ │ │ └── projections - │ │ │ ├── true [as=canary:22] - │ │ │ └── indirection [as=array:19, subquery] + │ │ │ ├── true [as=canary:25] + │ │ │ └── indirection [as=array:22, subquery] │ │ │ ├── array-flatten │ │ │ │ └── scan a - │ │ │ │ ├── columns: k:13!null - │ │ │ │ └── key: (13) + │ │ │ │ ├── columns: k:15!null + │ │ │ │ └── key: (15) │ │ │ └── 1 │ │ └── filters - │ │ └── b.k:7 = c.k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + │ │ └── b.k:8 = c.k:1 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] │ └── aggregations - │ ├── array-agg [as=array_agg:23, outer=(19)] - │ │ └── array:19 - │ └── any-not-null-agg [as=canary:22, outer=(22)] - │ └── canary:22 + │ ├── array-agg [as=array_agg:26, outer=(22)] + │ │ └── array:22 + │ └── any-not-null-agg [as=canary:25, outer=(25)] + │ └── canary:25 └── projections - └── COALESCE(CASE WHEN canary:22 IS NOT NULL THEN array_agg:23 ELSE CAST(NULL AS INT8[]) END, ARRAY[]) [as=array:21, outer=(22,23)] + └── COALESCE(CASE WHEN canary:25 IS NOT NULL THEN array_agg:26 ELSE CAST(NULL AS INT8[]) END, ARRAY[]) [as=array:24, outer=(25,26)] # Correlated ArrayFlatten inside another correlated ArrayFlatten. norm expect=NormalizeArrayFlattenToAgg SELECT ARRAY(SELECT ARRAY(SELECT k FROM a WHERE a.k = b.k)[1] FROM a as b WHERE b.k = c.k) FROM a AS c ---- project - ├── columns: array:23 + ├── columns: array:26 ├── group-by - │ ├── columns: c.k:1!null canary:24 array_agg:25 + │ ├── columns: c.k:1!null canary:27 array_agg:28 │ ├── grouping columns: c.k:1!null │ ├── key: (1) - │ ├── fd: (1)-->(24,25) + │ ├── fd: (1)-->(27,28) │ ├── left-join-apply - │ │ ├── columns: c.k:1!null array:20 canary:24 + │ │ ├── columns: c.k:1!null array:23 canary:27 │ │ ├── key: (1) - │ │ ├── fd: (1)-->(20,24) + │ │ ├── fd: (1)-->(23,27) │ │ ├── scan c │ │ │ ├── columns: c.k:1!null │ │ │ └── key: (1) │ │ ├── project - │ │ │ ├── columns: canary:24!null array:20 + │ │ │ ├── columns: canary:27!null array:23 │ │ │ ├── outer: (1) │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(20,24) + │ │ │ ├── fd: ()-->(23,27) │ │ │ ├── group-by - │ │ │ │ ├── columns: a.k:13!null array_agg:21!null + │ │ │ │ ├── columns: a.k:15!null array_agg:24!null │ │ │ │ ├── outer: (1) │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ ├── key: () - │ │ │ │ ├── fd: ()-->(13,21) + │ │ │ │ ├── fd: ()-->(15,24) │ │ │ │ ├── inner-join (hash) - │ │ │ │ │ ├── columns: b.k:7!null a.k:13!null + │ │ │ │ │ ├── columns: b.k:8!null a.k:15!null │ │ │ │ │ ├── outer: (1) │ │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ │ │ │ │ ├── key: () - │ │ │ │ │ ├── fd: ()-->(7,13) + │ │ │ │ │ ├── fd: ()-->(8,15) │ │ │ │ │ ├── select - │ │ │ │ │ │ ├── columns: b.k:7!null + │ │ │ │ │ │ ├── columns: b.k:8!null │ │ │ │ │ │ ├── outer: (1) │ │ │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ │ │ ├── key: () - │ │ │ │ │ │ ├── fd: ()-->(7) + │ │ │ │ │ │ ├── fd: ()-->(8) │ │ │ │ │ │ ├── scan b - │ │ │ │ │ │ │ ├── columns: b.k:7!null - │ │ │ │ │ │ │ └── key: (7) + │ │ │ │ │ │ │ ├── columns: b.k:8!null + │ │ │ │ │ │ │ └── key: (8) │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── b.k:7 = c.k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + │ │ │ │ │ │ └── b.k:8 = c.k:1 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] │ │ │ │ │ ├── scan a - │ │ │ │ │ │ ├── columns: a.k:13!null - │ │ │ │ │ │ └── key: (13) + │ │ │ │ │ │ ├── columns: a.k:15!null + │ │ │ │ │ │ └── key: (15) │ │ │ │ │ └── filters - │ │ │ │ │ └── a.k:13 = b.k:7 [outer=(7,13), constraints=(/7: (/NULL - ]; /13: (/NULL - ]), fd=(7)==(13), (13)==(7)] + │ │ │ │ │ └── a.k:15 = b.k:8 [outer=(8,15), constraints=(/8: (/NULL - ]; /15: (/NULL - ]), fd=(8)==(15), (15)==(8)] │ │ │ │ └── aggregations - │ │ │ │ ├── array-agg [as=array_agg:21, outer=(13)] - │ │ │ │ │ └── a.k:13 - │ │ │ │ └── any-not-null-agg [as=a.k:13, outer=(13)] - │ │ │ │ └── a.k:13 + │ │ │ │ ├── array-agg [as=array_agg:24, outer=(15)] + │ │ │ │ │ └── a.k:15 + │ │ │ │ └── any-not-null-agg [as=a.k:15, outer=(15)] + │ │ │ │ └── a.k:15 │ │ │ └── projections - │ │ │ ├── true [as=canary:24] - │ │ │ └── COALESCE(CASE WHEN a.k:13 IS NOT NULL THEN array_agg:21 ELSE CAST(NULL AS INT8[]) END, ARRAY[])[1] [as=array:20, outer=(13,21)] + │ │ │ ├── true [as=canary:27] + │ │ │ └── COALESCE(CASE WHEN a.k:15 IS NOT NULL THEN array_agg:24 ELSE CAST(NULL AS INT8[]) END, ARRAY[])[1] [as=array:23, outer=(15,24)] │ │ └── filters (true) │ └── aggregations - │ ├── array-agg [as=array_agg:25, outer=(20)] - │ │ └── array:20 - │ └── any-not-null-agg [as=canary:24, outer=(24)] - │ └── canary:24 + │ ├── array-agg [as=array_agg:28, outer=(23)] + │ │ └── array:23 + │ └── any-not-null-agg [as=canary:27, outer=(27)] + │ └── canary:27 └── projections - └── COALESCE(CASE WHEN canary:24 IS NOT NULL THEN array_agg:25 ELSE CAST(NULL AS INT8[]) END, ARRAY[]) [as=array:23, outer=(24,25)] + └── COALESCE(CASE WHEN canary:27 IS NOT NULL THEN array_agg:28 ELSE CAST(NULL AS INT8[]) END, ARRAY[]) [as=array:26, outer=(27,28)] # Shouldn't trigger if there's no correlation. norm expect-not=NormalizeArrayFlattenToAgg SELECT ARRAY(SELECT k FROM a) FROM a ---- project - ├── columns: array:13 - ├── fd: ()-->(13) + ├── columns: array:15 + ├── fd: ()-->(15) ├── scan a └── projections - └── array-flatten [as=array:13, subquery] + └── array-flatten [as=array:15, subquery] └── scan a - ├── columns: k:7!null - └── key: (7) + ├── columns: k:8!null + └── key: (8) exec-ddl CREATE TABLE pg_class ( @@ -1738,75 +1738,75 @@ SELECT ( FROM pg_class AS rel ---- project - ├── columns: array:66 + ├── columns: array:69 ├── inner-join-apply - │ ├── columns: rel.oid:1 array_agg:63 array:64 + │ ├── columns: rel.oid:1 array_agg:66 array:67 │ ├── scan rel │ │ └── columns: rel.oid:1 │ ├── inner-join-apply - │ │ ├── columns: array_agg:63 array:64 + │ │ ├── columns: array_agg:66 array:67 │ │ ├── outer: (1) │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(63,64) + │ │ ├── fd: ()-->(66,67) │ │ ├── project - │ │ │ ├── columns: array_agg:63 + │ │ │ ├── columns: array_agg:66 │ │ │ ├── outer: (1) │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(63) + │ │ │ ├── fd: ()-->(66) │ │ │ ├── group-by - │ │ │ │ ├── columns: inhrelid:30 array_agg:65 - │ │ │ │ ├── internal-ordering: +32 opt(30) + │ │ │ │ ├── columns: inhrelid:31 array_agg:68 + │ │ │ │ ├── internal-ordering: +33 opt(31) │ │ │ │ ├── outer: (1) │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ ├── key: () - │ │ │ │ ├── fd: ()-->(30,65) + │ │ │ │ ├── fd: ()-->(31,68) │ │ │ │ ├── sort - │ │ │ │ │ ├── columns: inhrelid:30 inhparent:31 inhseqno:32 c.oid:34 c.relname:35 + │ │ │ │ │ ├── columns: inhrelid:31 inhparent:32 inhseqno:33 c.oid:36 c.relname:37 │ │ │ │ │ ├── outer: (1) │ │ │ │ │ ├── cardinality: [1 - ] - │ │ │ │ │ ├── fd: (31)==(34), (34)==(31) - │ │ │ │ │ ├── ordering: +32 opt(30) [actual: +32] + │ │ │ │ │ ├── fd: (32)==(36), (36)==(32) + │ │ │ │ │ ├── ordering: +33 opt(31) [actual: +33] │ │ │ │ │ └── left-join (cross) - │ │ │ │ │ ├── columns: inhrelid:30 inhparent:31 inhseqno:32 c.oid:34 c.relname:35 + │ │ │ │ │ ├── columns: inhrelid:31 inhparent:32 inhseqno:33 c.oid:36 c.relname:37 │ │ │ │ │ ├── outer: (1) │ │ │ │ │ ├── cardinality: [1 - ] │ │ │ │ │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) - │ │ │ │ │ ├── fd: (31)==(34), (34)==(31) + │ │ │ │ │ ├── fd: (32)==(36), (36)==(32) │ │ │ │ │ ├── values │ │ │ │ │ │ ├── cardinality: [1 - 1] │ │ │ │ │ │ ├── key: () │ │ │ │ │ │ └── () │ │ │ │ │ ├── inner-join (hash) - │ │ │ │ │ │ ├── columns: inhrelid:30 inhparent:31!null inhseqno:32 c.oid:34!null c.relname:35!null - │ │ │ │ │ │ ├── fd: (31)==(34), (34)==(31) + │ │ │ │ │ │ ├── columns: inhrelid:31 inhparent:32!null inhseqno:33 c.oid:36!null c.relname:37!null + │ │ │ │ │ │ ├── fd: (32)==(36), (36)==(32) │ │ │ │ │ │ ├── scan i - │ │ │ │ │ │ │ └── columns: inhrelid:30 inhparent:31 inhseqno:32 + │ │ │ │ │ │ │ └── columns: inhrelid:31 inhparent:32 inhseqno:33 │ │ │ │ │ │ ├── scan c - │ │ │ │ │ │ │ └── columns: c.oid:34 c.relname:35!null + │ │ │ │ │ │ │ └── columns: c.oid:36 c.relname:37!null │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── c.oid:34 = inhparent:31 [outer=(31,34), constraints=(/31: (/NULL - ]; /34: (/NULL - ]), fd=(31)==(34), (34)==(31)] + │ │ │ │ │ │ └── c.oid:36 = inhparent:32 [outer=(32,36), constraints=(/32: (/NULL - ]; /36: (/NULL - ]), fd=(32)==(36), (36)==(32)] │ │ │ │ │ └── filters - │ │ │ │ │ └── inhrelid:30 = rel.oid:1 [outer=(1,30), constraints=(/1: (/NULL - ]; /30: (/NULL - ]), fd=(1)==(30), (30)==(1)] + │ │ │ │ │ └── inhrelid:31 = rel.oid:1 [outer=(1,31), constraints=(/1: (/NULL - ]; /31: (/NULL - ]), fd=(1)==(31), (31)==(1)] │ │ │ │ └── aggregations - │ │ │ │ ├── array-agg [as=array_agg:65, outer=(35)] - │ │ │ │ │ └── c.relname:35 - │ │ │ │ └── any-not-null-agg [as=inhrelid:30, outer=(30)] - │ │ │ │ └── inhrelid:30 + │ │ │ │ ├── array-agg [as=array_agg:68, outer=(37)] + │ │ │ │ │ └── c.relname:37 + │ │ │ │ └── any-not-null-agg [as=inhrelid:31, outer=(31)] + │ │ │ │ └── inhrelid:31 │ │ │ └── projections - │ │ │ └── CASE WHEN inhrelid:30 IS NOT NULL THEN array_agg:65 ELSE CAST(NULL AS NAME[]) END [as=array_agg:63, outer=(30,65)] + │ │ │ └── CASE WHEN inhrelid:31 IS NOT NULL THEN array_agg:68 ELSE CAST(NULL AS NAME[]) END [as=array_agg:66, outer=(31,68)] │ │ ├── values - │ │ │ ├── columns: array:64 - │ │ │ ├── outer: (63) + │ │ │ ├── columns: array:67 + │ │ │ ├── outer: (66) │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(64) - │ │ │ └── (COALESCE(array_agg:63, ARRAY[]),) + │ │ │ ├── fd: ()-->(67) + │ │ │ └── (COALESCE(array_agg:66, ARRAY[]),) │ │ └── filters (true) │ └── filters (true) └── projections - └── array:64 [as=array:66, outer=(64)] + └── array:67 [as=array:69, outer=(67)] # -------------------------------------------------- # SimplifySameVarEqualities @@ -1873,12 +1873,12 @@ norm expect=SimplifySameVarEqualities SELECT k = k FROM a ---- project - ├── columns: "?column?":7 + ├── columns: "?column?":8 ├── scan a │ ├── columns: k:1!null │ └── key: (1) └── projections - └── (k:1 IS DISTINCT FROM CAST(NULL AS INT8)) OR CAST(NULL AS BOOL) [as="?column?":7, outer=(1)] + └── (k:1 IS DISTINCT FROM CAST(NULL AS INT8)) OR CAST(NULL AS BOOL) [as="?column?":8, outer=(1)] # -------------------------------------------------- # SimplifySameVarInequalities @@ -1951,9 +1951,9 @@ norm expect=SimplifySameVarInequalities SELECT k != k FROM a ---- project - ├── columns: "?column?":7 + ├── columns: "?column?":8 ├── scan a │ ├── columns: k:1!null │ └── key: (1) └── projections - └── (k:1 IS NOT DISTINCT FROM CAST(NULL AS INT8)) AND CAST(NULL AS BOOL) [as="?column?":7, outer=(1)] + └── (k:1 IS NOT DISTINCT FROM CAST(NULL AS INT8)) AND CAST(NULL AS BOOL) [as="?column?":8, outer=(1)] diff --git a/pkg/sql/opt/norm/testdata/rules/select b/pkg/sql/opt/norm/testdata/rules/select index 073bdf2f0b01..38f4698c0614 100644 --- a/pkg/sql/opt/norm/testdata/rules/select +++ b/pkg/sql/opt/norm/testdata/rules/select @@ -41,53 +41,53 @@ norm expect=SimplifyJoinFilters SELECT * FROM a INNER JOIN xy ON NULL ---- values - ├── columns: k:1!null i:2!null f:3!null s:4!null j:5!null x:6!null y:7!null + ├── columns: k:1!null i:2!null f:3!null s:4!null j:5!null x:7!null y:8!null ├── cardinality: [0 - 0] ├── key: () - └── fd: ()-->(1-7) + └── fd: ()-->(1-5,7,8) norm expect=SimplifyJoinFilters SELECT * FROM a INNER JOIN xy ON x=1 OR NULL ---- inner-join (cross) - ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6!null y:7 + ├── columns: k:1!null i:2 f:3 s:4 j:5 x:7!null y:8 ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) ├── key: (1) - ├── fd: ()-->(6,7), (1)-->(2-5) + ├── fd: ()-->(7,8), (1)-->(2-5) ├── scan a │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── select - │ ├── columns: x:6!null y:7 + │ ├── columns: x:7!null y:8 │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(6,7) + │ ├── fd: ()-->(7,8) │ ├── scan xy - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ └── filters - │ └── x:6 = 1 [outer=(6), constraints=(/6: [/1 - /1]; tight), fd=()-->(6)] + │ └── x:7 = 1 [outer=(7), constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] └── filters (true) norm expect-not=SimplifyJoinFilters SELECT * FROM a INNER JOIN xy ON x=1 OR k=1 ---- inner-join (cross) - ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6!null y:7 - ├── key: (1,6) - ├── fd: (1)-->(2-5), (6)-->(7) + ├── columns: k:1!null i:2 f:3 s:4 j:5 x:7!null y:8 + ├── key: (1,7) + ├── fd: (1)-->(2-5), (7)-->(8) ├── scan a │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── scan xy - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + │ └── fd: (7)-->(8) └── filters - └── (x:6 = 1) OR (k:1 = 1) [outer=(1,6)] + └── (x:7 = 1) OR (k:1 = 1) [outer=(1,7)] norm expect=SimplifySelectFilters SELECT * FROM a WHERE i=1 AND Null @@ -149,10 +149,10 @@ norm expect=SimplifyJoinFilters SELECT * FROM a INNER JOIN xy ON (k=x AND i=y) AND true AND (f=3.5 AND s='foo') ---- inner-join (hash) - ├── columns: k:1!null i:2!null f:3!null s:4!null j:5 x:6!null y:7!null + ├── columns: k:1!null i:2!null f:3!null s:4!null j:5 x:7!null y:8!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (6) - ├── fd: ()-->(3,4), (1)-->(2,5), (6)-->(7), (1)==(6), (6)==(1), (2)==(7), (7)==(2) + ├── key: (7) + ├── fd: ()-->(3,4), (1)-->(2,5), (7)-->(8), (1)==(7), (7)==(1), (2)==(8), (8)==(2) ├── select │ ├── columns: k:1!null i:2 f:3!null s:4!null j:5 │ ├── key: (1) @@ -165,12 +165,12 @@ inner-join (hash) │ ├── f:3 = 3.5 [outer=(3), constraints=(/3: [/3.5 - /3.5]; tight), fd=()-->(3)] │ └── s:4 = 'foo' [outer=(4), constraints=(/4: [/'foo' - /'foo']; tight), fd=()-->(4)] ├── scan xy - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + │ └── fd: (7)-->(8) └── filters - ├── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── i:2 = y:7 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] + ├── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + └── i:2 = y:8 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] # -------------------------------------------------- # ConsolidateSelectFilters @@ -290,11 +290,11 @@ norm expect=ConsolidateSelectFilters disable=InlineConstVar SELECT * FROM (SELECT * FROM a WHERE k = 5) AS a, e WHERE a.k = e.k AND a.k > 1 AND e.k < 10 ---- inner-join (hash) - ├── columns: k:1!null i:2 f:3 s:4 j:5 k:6!null i:7 t:8 tz:9 d:10 + ├── columns: k:1!null i:2 f:3 s:4 j:5 k:7!null i:8 t:9 tz:10 d:11 ├── cardinality: [0 - 1] ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) ├── key: () - ├── fd: ()-->(1-10) + ├── fd: ()-->(1-5,7-11) ├── select │ ├── columns: a.k:1!null a.i:2 f:3 s:4 j:5 │ ├── cardinality: [0 - 1] @@ -307,27 +307,27 @@ inner-join (hash) │ └── filters │ └── ((a.k:1 = 5) AND (a.k:1 > 1)) AND (a.k:1 < 10) [outer=(1), constraints=(/1: [/5 - /5]; tight), fd=()-->(1)] ├── select - │ ├── columns: e.k:6!null e.i:7 t:8 tz:9 d:10 + │ ├── columns: e.k:7!null e.i:8 t:9 tz:10 d:11 │ ├── cardinality: [0 - 8] - │ ├── key: (6) - │ ├── fd: (6)-->(7-10) + │ ├── key: (7) + │ ├── fd: (7)-->(8-11) │ ├── scan e - │ │ ├── columns: e.k:6!null e.i:7 t:8 tz:9 d:10 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7-10) + │ │ ├── columns: e.k:7!null e.i:8 t:9 tz:10 d:11 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8-11) │ └── filters - │ └── (e.k:6 < 10) AND (e.k:6 > 1) [outer=(6), constraints=(/6: [/2 - /9]; tight)] + │ └── (e.k:7 < 10) AND (e.k:7 > 1) [outer=(7), constraints=(/7: [/2 - /9]; tight)] └── filters - └── a.k:1 = e.k:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── a.k:1 = e.k:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # The duplicate filter i >= 5 should be eliminated. norm expect=ConsolidateSelectFilters SELECT * FROM (SELECT * FROM a WHERE i >= 5 AND i < 10) AS a, xy WHERE i >= 5 ---- inner-join (cross) - ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:6!null y:7 - ├── key: (1,6) - ├── fd: (1)-->(2-5), (6)-->(7) + ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:7!null y:8 + ├── key: (1,7) + ├── fd: (1)-->(2-5), (7)-->(8) ├── select │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 │ ├── key: (1) @@ -339,18 +339,18 @@ inner-join (cross) │ └── filters │ └── (i:2 >= 5) AND (i:2 < 10) [outer=(2), constraints=(/2: [/5 - /9]; tight)] ├── scan xy - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + │ └── fd: (7)-->(8) └── filters (true) norm expect=ConsolidateSelectFilters SELECT * FROM (SELECT * FROM a WHERE i < 10 AND i >= 5) AS a, xy WHERE i >= 5 ---- inner-join (cross) - ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:6!null y:7 - ├── key: (1,6) - ├── fd: (1)-->(2-5), (6)-->(7) + ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:7!null y:8 + ├── key: (1,7) + ├── fd: (1)-->(2-5), (7)-->(8) ├── select │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 │ ├── key: (1) @@ -362,9 +362,9 @@ inner-join (cross) │ └── filters │ └── (i:2 < 10) AND (i:2 >= 5) [outer=(2), constraints=(/2: [/5 - /9]; tight)] ├── scan xy - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + │ └── fd: (7)-->(8) └── filters (true) norm expect=ConsolidateSelectFilters @@ -372,9 +372,9 @@ SELECT * FROM (SELECT * FROM a WHERE i < 10 AND i >= 5 AND i IN (0, 2, 4, 6, 8, WHERE i >= 5 AND i < 10 ---- inner-join (cross) - ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:6!null y:7 - ├── key: (1,6) - ├── fd: (1)-->(2-5), (6)-->(7) + ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:7!null y:8 + ├── key: (1,7) + ├── fd: (1)-->(2-5), (7)-->(8) ├── select │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 │ ├── key: (1) @@ -386,9 +386,9 @@ inner-join (cross) │ └── filters │ └── ((i:2 < 10) AND (i:2 >= 5)) AND (i:2 IN (0, 2, 4, 6, 8, 10, 12)) [outer=(2), constraints=(/2: [/6 - /6] [/8 - /8]; tight)] ├── scan xy - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + │ └── fd: (7)-->(8) └── filters (true) # -------------------------------------------------- @@ -467,9 +467,9 @@ norm expect=PushSelectIntoProject SELECT * FROM (SELECT i, i+1 AS r, f FROM a) a WHERE f=10.0 ---- project - ├── columns: i:2 r:6 f:3!null + ├── columns: i:2 r:7 f:3!null ├── immutable - ├── fd: ()-->(3), (2)-->(6) + ├── fd: ()-->(3), (2)-->(7) ├── select │ ├── columns: i:2 f:3!null │ ├── fd: ()-->(3) @@ -478,37 +478,37 @@ project │ └── filters │ └── f:3 = 10.0 [outer=(3), constraints=(/3: [/10.0 - /10.0]; tight), fd=()-->(3)] └── projections - └── i:2 + 1 [as=r:6, outer=(2), immutable] + └── i:2 + 1 [as=r:7, outer=(2), immutable] # Don't push down select if it depends on computed column that can't be inlined. norm expect-not=PushSelectIntoProject SELECT * FROM (SELECT i, i/2 div, f FROM a) a WHERE div=2 ---- select - ├── columns: i:2 div:6!null f:3 + ├── columns: i:2 div:7!null f:3 ├── immutable - ├── fd: ()-->(6) + ├── fd: ()-->(7) ├── project - │ ├── columns: div:6 i:2 f:3 - │ ├── fd: (2)-->(6) + │ ├── columns: div:7 i:2 f:3 + │ ├── fd: (2)-->(7) │ ├── scan a │ │ └── columns: i:2 f:3 │ └── projections - │ └── i:2 / 2 [as=div:6, outer=(2)] + │ └── i:2 / 2 [as=div:7, outer=(2)] └── filters - └── div:6 = 2 [outer=(6), immutable, constraints=(/6: [/2 - /2]; tight), fd=()-->(6)] + └── div:7 = 2 [outer=(7), immutable, constraints=(/7: [/2 - /2]; tight), fd=()-->(7)] # Push down some conjuncts, but not others. norm expect=PushSelectIntoProject SELECT * FROM (SELECT i, i/2 div, f FROM a) a WHERE 10.0=f AND 2=div AND i=1 ---- select - ├── columns: i:2!null div:6!null f:3!null + ├── columns: i:2!null div:7!null f:3!null ├── immutable - ├── fd: ()-->(2,3,6) + ├── fd: ()-->(2,3,7) ├── project - │ ├── columns: div:6!null i:2!null f:3!null - │ ├── fd: ()-->(2,3,6) + │ ├── columns: div:7!null i:2!null f:3!null + │ ├── fd: ()-->(2,3,7) │ ├── select │ │ ├── columns: i:2!null f:3!null │ │ ├── fd: ()-->(2,3) @@ -518,35 +518,35 @@ select │ │ ├── f:3 = 10.0 [outer=(3), constraints=(/3: [/10.0 - /10.0]; tight), fd=()-->(3)] │ │ └── i:2 = 1 [outer=(2), constraints=(/2: [/1 - /1]; tight), fd=()-->(2)] │ └── projections - │ └── i:2 / 2 [as=div:6, outer=(2)] + │ └── i:2 / 2 [as=div:7, outer=(2)] └── filters - └── div:6 = 2 [outer=(6), immutable, constraints=(/6: [/2 - /2]; tight), fd=()-->(6)] + └── div:7 = 2 [outer=(7), immutable, constraints=(/7: [/2 - /2]; tight), fd=()-->(7)] # Detect PushSelectIntoProject and FilterUnusedSelectCols dependency cycle. norm SELECT f, f+1.1 AS r FROM (SELECT f, i FROM a GROUP BY f, i HAVING sum(f)=10.0) a ---- project - ├── columns: f:3 r:7 + ├── columns: f:3 r:8 ├── immutable ├── select - │ ├── columns: i:2 f:3 sum:6!null + │ ├── columns: i:2 f:3 sum:7!null │ ├── key: (2,3) - │ ├── fd: ()-->(6) + │ ├── fd: ()-->(7) │ ├── group-by - │ │ ├── columns: i:2 f:3 sum:6 + │ │ ├── columns: i:2 f:3 sum:7 │ │ ├── grouping columns: i:2 f:3 │ │ ├── key: (2,3) - │ │ ├── fd: (2,3)-->(6) + │ │ ├── fd: (2,3)-->(7) │ │ ├── scan a │ │ │ └── columns: i:2 f:3 │ │ └── aggregations - │ │ └── sum [as=sum:6, outer=(3)] + │ │ └── sum [as=sum:7, outer=(3)] │ │ └── f:3 │ └── filters - │ └── sum:6 = 10.0 [outer=(6), constraints=(/6: [/10.0 - /10.0]; tight), fd=()-->(6)] + │ └── sum:7 = 10.0 [outer=(7), constraints=(/7: [/10.0 - /10.0]; tight), fd=()-->(7)] └── projections - └── f:3 + 1.1 [as=r:7, outer=(3), immutable] + └── f:3 + 1.1 [as=r:8, outer=(3), immutable] # -------------------------------------- # PushSelectCondLeftIntoJoinLeftAndRight @@ -557,14 +557,14 @@ norm expect=PushSelectCondLeftIntoJoinLeftAndRight SELECT * FROM a LEFT JOIN xy ON a.k=xy.x WHERE a.k > 5 AND (xy.x = 6 OR xy.x IS NULL) ---- select - ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6 y:7 + ├── columns: k:1!null i:2 f:3 s:4 j:5 x:7 y:8 ├── key: (1) - ├── fd: (1)-->(2-7), (6)-->(7) + ├── fd: (1)-->(2-5,7,8), (7)-->(8) ├── left-join (hash) - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6 y:7 + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:7 y:8 │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ ├── key: (1) - │ ├── fd: (1)-->(2-7), (6)-->(7) + │ ├── fd: (1)-->(2-5,7,8), (7)-->(8) │ ├── select │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ │ ├── key: (1) @@ -576,19 +576,19 @@ select │ │ └── filters │ │ └── k:1 > 5 [outer=(1), constraints=(/1: [/6 - ]; tight)] │ ├── select - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ ├── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ ├── fd: (7)-->(8) │ │ ├── scan xy - │ │ │ ├── columns: x:6!null y:7 - │ │ │ ├── key: (6) - │ │ │ └── fd: (6)-->(7) + │ │ │ ├── columns: x:7!null y:8 + │ │ │ ├── key: (7) + │ │ │ └── fd: (7)-->(8) │ │ └── filters - │ │ └── x:6 > 5 [outer=(6), constraints=(/6: [/6 - ]; tight)] + │ │ └── x:7 > 5 [outer=(7), constraints=(/7: [/6 - ]; tight)] │ └── filters - │ └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] └── filters - └── (x:6 = 6) OR (x:6 IS NULL) [outer=(6), constraints=(/6: [/NULL - /NULL] [/6 - /6]; tight)] + └── (x:7 = 6) OR (x:7 IS NULL) [outer=(7), constraints=(/7: [/NULL - /NULL] [/6 - /6]; tight)] norm expect=PushSelectCondLeftIntoJoinLeftAndRight SELECT * FROM a WHERE EXISTS (SELECT * FROM xy WHERE a.k=xy.x) AND a.k > 5 @@ -608,15 +608,15 @@ semi-join (hash) │ └── filters │ └── k:1 > 5 [outer=(1), constraints=(/1: [/6 - ]; tight)] ├── select - │ ├── columns: x:6!null - │ ├── key: (6) + │ ├── columns: x:7!null + │ ├── key: (7) │ ├── scan xy - │ │ ├── columns: x:6!null - │ │ └── key: (6) + │ │ ├── columns: x:7!null + │ │ └── key: (7) │ └── filters - │ └── x:6 > 5 [outer=(6), constraints=(/6: [/6 - ]; tight)] + │ └── x:7 > 5 [outer=(7), constraints=(/7: [/6 - ]; tight)] └── filters - └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] norm expect=PushSelectCondLeftIntoJoinLeftAndRight SELECT * FROM a WHERE NOT EXISTS (SELECT * FROM xy WHERE a.k=xy.x) AND a.k > 5 @@ -636,15 +636,15 @@ anti-join (hash) │ └── filters │ └── k:1 > 5 [outer=(1), constraints=(/1: [/6 - ]; tight)] ├── select - │ ├── columns: x:6!null - │ ├── key: (6) + │ ├── columns: x:7!null + │ ├── key: (7) │ ├── scan xy - │ │ ├── columns: x:6!null - │ │ └── key: (6) + │ │ ├── columns: x:7!null + │ │ └── key: (7) │ └── filters - │ └── x:6 > 5 [outer=(6), constraints=(/6: [/6 - ]; tight)] + │ └── x:7 > 5 [outer=(7), constraints=(/7: [/6 - ]; tight)] └── filters - └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # -------------------------------------------------- # PushSelectIntoJoinLeft @@ -653,10 +653,10 @@ norm expect=PushSelectIntoJoinLeft SELECT * FROM a LEFT JOIN xy ON a.k=xy.x WHERE a.f=1.1 ---- left-join (hash) - ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:6 y:7 + ├── columns: k:1!null i:2 f:3!null s:4 j:5 x:7 y:8 ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) ├── key: (1) - ├── fd: ()-->(3), (1)-->(2,4-7), (6)-->(7) + ├── fd: ()-->(3), (1)-->(2,4,5,7,8), (7)-->(8) ├── select │ ├── columns: k:1!null i:2 f:3!null s:4 j:5 │ ├── key: (1) @@ -668,25 +668,25 @@ left-join (hash) │ └── filters │ └── f:3 = 1.1 [outer=(3), constraints=(/3: [/1.1 - /1.1]; tight), fd=()-->(3)] ├── scan xy - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + │ └── fd: (7)-->(8) └── filters - └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] norm expect=PushSelectIntoJoinLeft SELECT * FROM a LEFT JOIN xy ON a.k=xy.x WHERE a.f=1.1 AND (a.i(3), (1)-->(2,4-7), (6)-->(7) + ├── fd: ()-->(3), (1)-->(2,4,5,7,8), (7)-->(8) ├── left-join (hash) - │ ├── columns: k:1!null i:2 f:3!null s:4!null j:5 x:6 y:7 + │ ├── columns: k:1!null i:2 f:3!null s:4!null j:5 x:7 y:8 │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ ├── key: (1) - │ ├── fd: ()-->(3), (1)-->(2,4-7), (6)-->(7) + │ ├── fd: ()-->(3), (1)-->(2,4,5,7,8), (7)-->(8) │ ├── select │ │ ├── columns: k:1!null i:2 f:3!null s:4!null j:5 │ │ ├── key: (1) @@ -699,23 +699,23 @@ select │ │ ├── f:3 = 1.1 [outer=(3), constraints=(/3: [/1.1 - /1.1]; tight), fd=()-->(3)] │ │ └── (s:4 = 'foo') OR (s:4 = 'bar') [outer=(4), constraints=(/4: [/'bar' - /'bar'] [/'foo' - /'foo']; tight)] │ ├── scan xy - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ └── filters - │ └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] └── filters - └── (i:2 < y:7) OR (y:7 IS NULL) [outer=(2,7)] + └── (i:2 < y:8) OR (y:8 IS NULL) [outer=(2,8)] # Pushdown constant condition. norm expect=PushSelectIntoJoinLeft SELECT * FROM a LEFT JOIN xy ON True WHERE a.i=100 AND $1>'2000-01-01T1:00:00' ---- left-join (cross) - ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:6 y:7 + ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:7 y:8 ├── has-placeholder - ├── key: (1,6) - ├── fd: ()-->(2), (1)-->(3-5), (6)-->(7) + ├── key: (1,7) + ├── fd: ()-->(2), (1)-->(3-5), (7)-->(8) ├── select │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 │ ├── has-placeholder @@ -729,14 +729,14 @@ left-join (cross) │ ├── $1 > '2000-01-01T1:00:00' │ └── i:2 = 100 [outer=(2), constraints=(/2: [/100 - /100]; tight), fd=()-->(2)] ├── select - │ ├── columns: x:6!null y:7 + │ ├── columns: x:7!null y:8 │ ├── has-placeholder - │ ├── key: (6) - │ ├── fd: (6)-->(7) + │ ├── key: (7) + │ ├── fd: (7)-->(8) │ ├── scan xy - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ └── filters │ └── $1 > '2000-01-01T1:00:00' └── filters (true) @@ -746,24 +746,24 @@ norm SELECT * FROM a RIGHT JOIN xy ON a.k=xy.x WHERE a.i=100 OR a.i IS NULL ---- select - ├── columns: k:1 i:2 f:3 s:4 j:5 x:6!null y:7 - ├── key: (6) - ├── fd: (6)-->(1-5,7), (1)-->(2-5) + ├── columns: k:1 i:2 f:3 s:4 j:5 x:7!null y:8 + ├── key: (7) + ├── fd: (7)-->(1-5,8), (1)-->(2-5) ├── left-join (hash) - │ ├── columns: k:1 i:2 f:3 s:4 j:5 x:6!null y:7 + │ ├── columns: k:1 i:2 f:3 s:4 j:5 x:7!null y:8 │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) - │ ├── key: (6) - │ ├── fd: (6)-->(1-5,7), (1)-->(2-5) + │ ├── key: (7) + │ ├── fd: (7)-->(1-5,8), (1)-->(2-5) │ ├── scan xy - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ ├── scan a │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ │ ├── key: (1) │ │ └── fd: (1)-->(2-5) │ └── filters - │ └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] └── filters └── (i:2 = 100) OR (i:2 IS NULL) [outer=(2), constraints=(/2: [/NULL - /NULL] [/100 - /100]; tight)] @@ -772,24 +772,24 @@ norm SELECT * FROM a FULL JOIN xy ON a.k=xy.x WHERE a.i=100 OR a.i IS NULL ---- select - ├── columns: k:1 i:2 f:3 s:4 j:5 x:6 y:7 - ├── key: (1,6) - ├── fd: (1)-->(2-5), (6)-->(7) + ├── columns: k:1 i:2 f:3 s:4 j:5 x:7 y:8 + ├── key: (1,7) + ├── fd: (1)-->(2-5), (7)-->(8) ├── full-join (hash) - │ ├── columns: k:1 i:2 f:3 s:4 j:5 x:6 y:7 + │ ├── columns: k:1 i:2 f:3 s:4 j:5 x:7 y:8 │ ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) - │ ├── key: (1,6) - │ ├── fd: (1)-->(2-5), (6)-->(7) + │ ├── key: (1,7) + │ ├── fd: (1)-->(2-5), (7)-->(8) │ ├── scan a │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ │ ├── key: (1) │ │ └── fd: (1)-->(2-5) │ ├── scan xy - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ └── filters - │ └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] └── filters └── (i:2 = 100) OR (i:2 IS NULL) [outer=(2), constraints=(/2: [/NULL - /NULL] [/100 - /100]; tight)] @@ -812,10 +812,10 @@ semi-join (hash) │ └── filters │ └── i:2 = 0 [outer=(2), constraints=(/2: [/0 - /0]; tight), fd=()-->(2)] ├── scan xy - │ ├── columns: x:6!null - │ └── key: (6) + │ ├── columns: x:7!null + │ └── key: (7) └── filters - └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # Push into anti-join. norm expect=PushSelectIntoJoinLeft @@ -836,62 +836,62 @@ anti-join (hash) │ └── filters │ └── i:2 = 0 [outer=(2), constraints=(/2: [/0 - /0]; tight), fd=()-->(2)] ├── scan xy - │ ├── columns: x:6!null - │ └── key: (6) + │ ├── columns: x:7!null + │ └── key: (7) └── filters - └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # Don't push down conditions in case of LEFT JOIN. norm SELECT * FROM xy LEFT JOIN a ON a.k=xy.x WHERE a.i=100 OR a.i IS NULL ---- select - ├── columns: x:1!null y:2 k:3 i:4 f:5 s:6 j:7 + ├── columns: x:1!null y:2 k:4 i:5 f:6 s:7 j:8 ├── key: (1) - ├── fd: (1)-->(2-7), (3)-->(4-7) + ├── fd: (1)-->(2,4-8), (4)-->(5-8) ├── left-join (hash) - │ ├── columns: x:1!null y:2 k:3 i:4 f:5 s:6 j:7 + │ ├── columns: x:1!null y:2 k:4 i:5 f:6 s:7 j:8 │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) │ ├── key: (1) - │ ├── fd: (1)-->(2-7), (3)-->(4-7) + │ ├── fd: (1)-->(2,4-8), (4)-->(5-8) │ ├── scan xy │ │ ├── columns: x:1!null y:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ ├── scan a - │ │ ├── columns: k:3!null i:4 f:5 s:6 j:7 - │ │ ├── key: (3) - │ │ └── fd: (3)-->(4-7) + │ │ ├── columns: k:4!null i:5 f:6 s:7 j:8 + │ │ ├── key: (4) + │ │ └── fd: (4)-->(5-8) │ └── filters - │ └── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + │ └── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] └── filters - └── (i:4 = 100) OR (i:4 IS NULL) [outer=(4), constraints=(/4: [/NULL - /NULL] [/100 - /100]; tight)] + └── (i:5 = 100) OR (i:5 IS NULL) [outer=(5), constraints=(/5: [/NULL - /NULL] [/100 - /100]; tight)] # Don't push down conditions in case of FULL JOIN. norm SELECT * FROM xy FULL JOIN a ON a.k=xy.x WHERE a.i=100 OR a.i IS NULL ---- select - ├── columns: x:1 y:2 k:3 i:4 f:5 s:6 j:7 - ├── key: (1,3) - ├── fd: (1)-->(2), (3)-->(4-7) + ├── columns: x:1 y:2 k:4 i:5 f:6 s:7 j:8 + ├── key: (1,4) + ├── fd: (1)-->(2), (4)-->(5-8) ├── full-join (hash) - │ ├── columns: x:1 y:2 k:3 i:4 f:5 s:6 j:7 + │ ├── columns: x:1 y:2 k:4 i:5 f:6 s:7 j:8 │ ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) - │ ├── key: (1,3) - │ ├── fd: (1)-->(2), (3)-->(4-7) + │ ├── key: (1,4) + │ ├── fd: (1)-->(2), (4)-->(5-8) │ ├── scan xy │ │ ├── columns: x:1!null y:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ ├── scan a - │ │ ├── columns: k:3!null i:4 f:5 s:6 j:7 - │ │ ├── key: (3) - │ │ └── fd: (3)-->(4-7) + │ │ ├── columns: k:4!null i:5 f:6 s:7 j:8 + │ │ ├── key: (4) + │ │ └── fd: (4)-->(5-8) │ └── filters - │ └── k:3 = x:1 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + │ └── k:4 = x:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] └── filters - └── (i:4 = 100) OR (i:4 IS NULL) [outer=(4), constraints=(/4: [/NULL - /NULL] [/100 - /100]; tight)] + └── (i:5 = 100) OR (i:5 IS NULL) [outer=(5), constraints=(/5: [/NULL - /NULL] [/100 - /100]; tight)] # -------------------------------------------------- # MergeSelectInnerJoin @@ -900,122 +900,122 @@ norm expect=MergeSelectInnerJoin SELECT * FROM a, xy WHERE a.k=xy.x AND (a.s='foo' OR xy.y<100) ---- inner-join (hash) - ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6!null y:7 + ├── columns: k:1!null i:2 f:3 s:4 j:5 x:7!null y:8 ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (6) - ├── fd: (1)-->(2-5), (6)-->(7), (1)==(6), (6)==(1) + ├── key: (7) + ├── fd: (1)-->(2-5), (7)-->(8), (1)==(7), (7)==(1) ├── scan a │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── scan xy - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + │ └── fd: (7)-->(8) └── filters - ├── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── (s:4 = 'foo') OR (y:7 < 100) [outer=(4,7)] + ├── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + └── (s:4 = 'foo') OR (y:8 < 100) [outer=(4,8)] norm expect=MergeSelectInnerJoin SELECT * FROM a INNER JOIN xy ON a.k=xy.x WHERE (a.s='foo' OR xy.y<100) ---- inner-join (hash) - ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6!null y:7 + ├── columns: k:1!null i:2 f:3 s:4 j:5 x:7!null y:8 ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── key: (6) - ├── fd: (1)-->(2-5), (6)-->(7), (1)==(6), (6)==(1) + ├── key: (7) + ├── fd: (1)-->(2-5), (7)-->(8), (1)==(7), (7)==(1) ├── scan a │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) ├── scan xy - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + │ └── fd: (7)-->(8) └── filters - ├── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - └── (s:4 = 'foo') OR (y:7 < 100) [outer=(4,7)] + ├── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + └── (s:4 = 'foo') OR (y:8 < 100) [outer=(4,8)] norm expect=MergeSelectInnerJoin SELECT * FROM a INNER JOIN xy ON a.k=xy.x WHERE False ---- values - ├── columns: k:1!null i:2!null f:3!null s:4!null j:5!null x:6!null y:7!null + ├── columns: k:1!null i:2!null f:3!null s:4!null j:5!null x:7!null y:8!null ├── cardinality: [0 - 0] ├── key: () - └── fd: ()-->(1-7) + └── fd: ()-->(1-5,7,8) # Don't merge with LEFT JOIN. norm expect-not=MergeSelectInnerJoin SELECT * FROM a LEFT JOIN xy ON True WHERE a.k=xy.x OR xy.x IS NULL ---- select - ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6 y:7 - ├── key: (1,6) - ├── fd: (1)-->(2-5), (6)-->(7) + ├── columns: k:1!null i:2 f:3 s:4 j:5 x:7 y:8 + ├── key: (1,7) + ├── fd: (1)-->(2-5), (7)-->(8) ├── left-join (cross) - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6 y:7 - │ ├── key: (1,6) - │ ├── fd: (1)-->(2-5), (6)-->(7) + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:7 y:8 + │ ├── key: (1,7) + │ ├── fd: (1)-->(2-5), (7)-->(8) │ ├── scan a │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ │ ├── key: (1) │ │ └── fd: (1)-->(2-5) │ ├── scan xy - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ └── filters (true) └── filters - └── (k:1 = x:6) OR (x:6 IS NULL) [outer=(1,6)] + └── (k:1 = x:7) OR (x:7 IS NULL) [outer=(1,7)] # Don't merge with RIGHT JOIN. norm expect-not=MergeSelectInnerJoin SELECT * FROM a RIGHT JOIN xy ON True WHERE a.k=xy.x OR a.k IS NULL ---- select - ├── columns: k:1 i:2 f:3 s:4 j:5 x:6!null y:7 - ├── key: (1,6) - ├── fd: (6)-->(7), (1)-->(2-5) + ├── columns: k:1 i:2 f:3 s:4 j:5 x:7!null y:8 + ├── key: (1,7) + ├── fd: (7)-->(8), (1)-->(2-5) ├── left-join (cross) - │ ├── columns: k:1 i:2 f:3 s:4 j:5 x:6!null y:7 - │ ├── key: (1,6) - │ ├── fd: (6)-->(7), (1)-->(2-5) + │ ├── columns: k:1 i:2 f:3 s:4 j:5 x:7!null y:8 + │ ├── key: (1,7) + │ ├── fd: (7)-->(8), (1)-->(2-5) │ ├── scan xy - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ ├── scan a │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ │ ├── key: (1) │ │ └── fd: (1)-->(2-5) │ └── filters (true) └── filters - └── (k:1 = x:6) OR (k:1 IS NULL) [outer=(1,6)] + └── (k:1 = x:7) OR (k:1 IS NULL) [outer=(1,7)] # Don't merge with FULL JOIN. norm expect-not=MergeSelectInnerJoin SELECT * FROM a FULL JOIN xy ON True WHERE a.k=xy.x OR a.k IS NULL OR xy.x IS NULL ---- select - ├── columns: k:1 i:2 f:3 s:4 j:5 x:6 y:7 - ├── key: (1,6) - ├── fd: (1)-->(2-5), (6)-->(7) + ├── columns: k:1 i:2 f:3 s:4 j:5 x:7 y:8 + ├── key: (1,7) + ├── fd: (1)-->(2-5), (7)-->(8) ├── full-join (cross) - │ ├── columns: k:1 i:2 f:3 s:4 j:5 x:6 y:7 - │ ├── key: (1,6) - │ ├── fd: (1)-->(2-5), (6)-->(7) + │ ├── columns: k:1 i:2 f:3 s:4 j:5 x:7 y:8 + │ ├── key: (1,7) + │ ├── fd: (1)-->(2-5), (7)-->(8) │ ├── scan a │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ │ ├── key: (1) │ │ └── fd: (1)-->(2-5) │ ├── scan xy - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ └── filters (true) └── filters - └── ((k:1 = x:6) OR (k:1 IS NULL)) OR (x:6 IS NULL) [outer=(1,6)] + └── ((k:1 = x:7) OR (k:1 IS NULL)) OR (x:7 IS NULL) [outer=(1,7)] # -------------------------------------------------- # PushSelectIntoJoinLeft + MergeSelectInnerJoin @@ -1024,10 +1024,10 @@ norm SELECT * FROM a INNER JOIN xy ON a.k=xy.x WHERE a.f=1.1 AND s='foo' AND xy.y=10 AND a.i(3,4,7), (1)-->(2,5), (1)==(6), (6)==(1) + ├── key: (7) + ├── fd: ()-->(3,4,8), (1)-->(2,5), (1)==(7), (7)==(1) ├── select │ ├── columns: k:1!null i:2!null f:3!null s:4!null j:5 │ ├── key: (1) @@ -1041,27 +1041,27 @@ inner-join (hash) │ ├── s:4 = 'foo' [outer=(4), constraints=(/4: [/'foo' - /'foo']; tight), fd=()-->(4)] │ └── i:2 < 10 [outer=(2), constraints=(/2: (/NULL - /9]; tight)] ├── select - │ ├── columns: x:6!null y:7!null - │ ├── key: (6) - │ ├── fd: ()-->(7) + │ ├── columns: x:7!null y:8!null + │ ├── key: (7) + │ ├── fd: ()-->(8) │ ├── scan xy - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ └── filters - │ └── y:7 = 10 [outer=(7), constraints=(/7: [/10 - /10]; tight), fd=()-->(7)] + │ └── y:8 = 10 [outer=(8), constraints=(/8: [/10 - /10]; tight), fd=()-->(8)] └── filters - └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] norm SELECT * FROM a, xy WHERE a.i=100 AND $1>'2000-01-01T1:00:00' AND xy.x=a.k ---- inner-join (hash) - ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:6!null y:7 + ├── columns: k:1!null i:2!null f:3 s:4 j:5 x:7!null y:8 ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) ├── has-placeholder - ├── key: (6) - ├── fd: ()-->(2), (1)-->(3-5), (6)-->(7), (1)==(6), (6)==(1) + ├── key: (7) + ├── fd: ()-->(2), (1)-->(3-5), (7)-->(8), (1)==(7), (7)==(1) ├── select │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 │ ├── has-placeholder @@ -1075,18 +1075,18 @@ inner-join (hash) │ ├── $1 > '2000-01-01T1:00:00' │ └── i:2 = 100 [outer=(2), constraints=(/2: [/100 - /100]; tight), fd=()-->(2)] ├── select - │ ├── columns: x:6!null y:7 + │ ├── columns: x:7!null y:8 │ ├── has-placeholder - │ ├── key: (6) - │ ├── fd: (6)-->(7) + │ ├── key: (7) + │ ├── fd: (7)-->(8) │ ├── scan xy - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ └── filters │ └── $1 > '2000-01-01T1:00:00' └── filters - └── x:6 = k:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── x:7 = k:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # -------------------------------------------------- # PushSelectIntoGroupBy @@ -1097,10 +1097,10 @@ norm expect=PushSelectIntoGroupBy SELECT * FROM (SELECT i, count(*) FROM a GROUP BY i) a WHERE i=1 ---- group-by - ├── columns: i:2!null count:6!null + ├── columns: i:2!null count:7!null ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(2,6) + ├── fd: ()-->(2,7) ├── select │ ├── columns: i:2!null │ ├── fd: ()-->(2) @@ -1109,7 +1109,7 @@ group-by │ └── filters │ └── i:2 = 1 [outer=(2), constraints=(/2: [/1 - /1]; tight), fd=()-->(2)] └── aggregations - ├── count-rows [as=count_rows:6] + ├── count-rows [as=count_rows:7] └── const-agg [as=i:2, outer=(2)] └── i:2 @@ -1138,14 +1138,14 @@ norm expect=PushSelectIntoGroupBy SELECT * FROM (SELECT k, i, max(s) m FROM a GROUP BY k, i) a WHERE i=k AND m='foo' ---- select - ├── columns: k:1!null i:2!null m:6!null + ├── columns: k:1!null i:2!null m:7!null ├── key: (1) - ├── fd: ()-->(6), (1)==(2), (2)==(1), (1)-->(2) + ├── fd: ()-->(7), (1)==(2), (2)==(1), (1)-->(2) ├── group-by - │ ├── columns: k:1!null i:2!null max:6 + │ ├── columns: k:1!null i:2!null max:7 │ ├── grouping columns: k:1!null │ ├── key: (1) - │ ├── fd: (1)==(2), (2)==(1), (1)-->(2,6) + │ ├── fd: (1)==(2), (2)==(1), (1)-->(2,7) │ ├── select │ │ ├── columns: k:1!null i:2!null s:4 │ │ ├── key: (1) @@ -1157,12 +1157,12 @@ select │ │ └── filters │ │ └── i:2 = k:1 [outer=(1,2), constraints=(/1: (/NULL - ]; /2: (/NULL - ]), fd=(1)==(2), (2)==(1)] │ └── aggregations - │ ├── max [as=max:6, outer=(4)] + │ ├── max [as=max:7, outer=(4)] │ │ └── s:4 │ └── const-agg [as=i:2, outer=(2)] │ └── i:2 └── filters - └── max:6 = 'foo' [outer=(6), constraints=(/6: [/'foo' - /'foo']; tight), fd=()-->(6)] + └── max:7 = 'foo' [outer=(7), constraints=(/7: [/'foo' - /'foo']; tight), fd=()-->(7)] # DistinctOn case. norm expect=PushSelectIntoGroupBy @@ -1188,14 +1188,14 @@ norm expect=PushSelectIntoGroupBy SELECT * FROM (SELECT DISTINCT ON (k, f, s) k, i, f, x FROM a JOIN xy ON i=y) WHERE k > f ---- distinct-on - ├── columns: k:1!null i:2!null f:3!null x:6!null + ├── columns: k:1!null i:2!null f:3!null x:7!null ├── grouping columns: k:1!null ├── key: (1) - ├── fd: (1)-->(2,3,6), (6)-->(2) + ├── fd: (1)-->(2,3,7), (7)-->(2) ├── inner-join (hash) - │ ├── columns: k:1!null i:2!null f:3!null x:6!null y:7!null - │ ├── key: (1,6) - │ ├── fd: (1)-->(2,3), (6)-->(7), (2)==(7), (7)==(2) + │ ├── columns: k:1!null i:2!null f:3!null x:7!null y:8!null + │ ├── key: (1,7) + │ ├── fd: (1)-->(2,3), (7)-->(8), (2)==(8), (8)==(2) │ ├── select │ │ ├── columns: k:1!null i:2 f:3!null │ │ ├── key: (1) @@ -1207,16 +1207,16 @@ distinct-on │ │ └── filters │ │ └── k:1 > f:3 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ])] │ ├── scan xy - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ └── filters - │ └── i:2 = y:7 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] + │ └── i:2 = y:8 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] └── aggregations ├── first-agg [as=i:2, outer=(2)] │ └── i:2 - ├── first-agg [as=x:6, outer=(6)] - │ └── x:6 + ├── first-agg [as=x:7, outer=(7)] + │ └── x:7 └── const-agg [as=f:3, outer=(3)] └── f:3 @@ -1225,22 +1225,22 @@ norm expect-not=PushSelectIntoGroupBy SELECT * FROM (SELECT count(*) c FROM a) a WHERE $1<'2000-01-01T10:00:00' AND c=0 ---- select - ├── columns: c:6!null + ├── columns: c:7!null ├── cardinality: [0 - 1] ├── has-placeholder ├── key: () - ├── fd: ()-->(6) + ├── fd: ()-->(7) ├── scalar-group-by - │ ├── columns: count_rows:6!null + │ ├── columns: count_rows:7!null │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(6) + │ ├── fd: ()-->(7) │ ├── scan a │ └── aggregations - │ └── count-rows [as=count_rows:6] + │ └── count-rows [as=count_rows:7] └── filters ├── $1 < '2000-01-01T10:00:00' - └── count_rows:6 = 0 [outer=(6), constraints=(/6: [/0 - /0]; tight), fd=()-->(6)] + └── count_rows:7 = 0 [outer=(7), constraints=(/7: [/0 - /0]; tight), fd=()-->(7)] # -------------------------------------------------- # RemoveNotNullCondition @@ -1373,7 +1373,7 @@ norm expect=PushSelectIntoProjectSet SELECT k, g FROM a, generate_series(0, a.k, 10) AS g WHERE k = 1 ---- project-set - ├── columns: k:1!null g:6 + ├── columns: k:1!null g:7 ├── immutable ├── fd: ()-->(1) ├── select @@ -1394,10 +1394,10 @@ norm expect-not=PushSelectIntoProjectSet SELECT k, g FROM a, generate_series(0, a.k, 10) AS g WHERE g > 1 ---- select - ├── columns: k:1!null g:6!null + ├── columns: k:1!null g:7!null ├── immutable ├── project-set - │ ├── columns: k:1!null generate_series:6 + │ ├── columns: k:1!null generate_series:7 │ ├── immutable │ ├── scan a │ │ ├── columns: k:1!null @@ -1405,18 +1405,18 @@ select │ └── zip │ └── generate_series(0, k:1, 10) [outer=(1), immutable] └── filters - └── generate_series:6 > 1 [outer=(6), constraints=(/6: [/2 - ]; tight)] + └── generate_series:7 > 1 [outer=(7), constraints=(/7: [/2 - ]; tight)] # Expect that only the applicable filters are pushed down into the project-set. norm expect=PushSelectIntoProjectSet SELECT k, g FROM a, generate_series(0, a.k, 10) AS g WHERE g > 1 AND k = 1 ---- select - ├── columns: k:1!null g:6!null + ├── columns: k:1!null g:7!null ├── immutable ├── fd: ()-->(1) ├── project-set - │ ├── columns: k:1!null generate_series:6 + │ ├── columns: k:1!null generate_series:7 │ ├── immutable │ ├── fd: ()-->(1) │ ├── select @@ -1432,7 +1432,7 @@ select │ └── zip │ └── generate_series(0, k:1, 10) [outer=(1), immutable] └── filters - └── generate_series:6 > 1 [outer=(6), constraints=(/6: [/2 - ]; tight)] + └── generate_series:7 > 1 [outer=(7), constraints=(/7: [/2 - ]; tight)] # -------------------------------------------------- @@ -1447,9 +1447,9 @@ SELECT k FROM WHERE k < 10 ---- union-all - ├── columns: k:11!null + ├── columns: k:13!null ├── left columns: b.k:1 - ├── right columns: b.k:6 + ├── right columns: b.k:7 ├── select │ ├── columns: b.k:1!null │ ├── key: (1) @@ -1459,13 +1459,13 @@ union-all │ └── filters │ └── b.k:1 < 10 [outer=(1), constraints=(/1: (/NULL - /9]; tight)] └── select - ├── columns: b.k:6!null - ├── key: (6) + ├── columns: b.k:7!null + ├── key: (7) ├── scan b - │ ├── columns: b.k:6!null - │ └── key: (6) + │ ├── columns: b.k:7!null + │ └── key: (7) └── filters - └── b.k:6 < 10 [outer=(6), constraints=(/6: (/NULL - /9]; tight)] + └── b.k:7 < 10 [outer=(7), constraints=(/7: (/NULL - /9]; tight)] norm expect=PushFilterIntoSetOp SELECT k FROM @@ -1475,10 +1475,10 @@ SELECT k FROM WHERE k < 10 AND k > 1 ---- union - ├── columns: k:11!null + ├── columns: k:13!null ├── left columns: b.k:1 - ├── right columns: a.i:7 - ├── key: (11) + ├── right columns: a.i:8 + ├── key: (13) ├── select │ ├── columns: b.k:1!null │ ├── cardinality: [0 - 8] @@ -1489,11 +1489,11 @@ union │ └── filters │ └── (b.k:1 < 10) AND (b.k:1 > 1) [outer=(1), constraints=(/1: [/2 - /9]; tight)] └── select - ├── columns: a.i:7!null + ├── columns: a.i:8!null ├── scan a - │ └── columns: a.i:7 + │ └── columns: a.i:8 └── filters - └── (a.i:7 < 10) AND (a.i:7 > 1) [outer=(7), constraints=(/7: [/2 - /9]; tight)] + └── (a.i:8 < 10) AND (a.i:8 > 1) [outer=(8), constraints=(/8: [/2 - /9]; tight)] norm expect=PushFilterIntoSetOp SELECT k FROM @@ -1505,7 +1505,7 @@ WHERE k < 10 AND k > 1 except ├── columns: k:1!null ├── left columns: b.k:1!null - ├── right columns: a.i:7 + ├── right columns: a.i:8 ├── cardinality: [0 - 8] ├── key: (1) ├── select @@ -1518,11 +1518,11 @@ except │ └── filters │ └── (b.k:1 < 10) AND (b.k:1 > 1) [outer=(1), constraints=(/1: [/2 - /9]; tight)] └── select - ├── columns: a.i:7!null + ├── columns: a.i:8!null ├── scan a - │ └── columns: a.i:7 + │ └── columns: a.i:8 └── filters - └── (a.i:7 < 10) AND (a.i:7 > 1) [outer=(7), constraints=(/7: [/2 - /9]; tight)] + └── (a.i:8 < 10) AND (a.i:8 > 1) [outer=(8), constraints=(/8: [/2 - /9]; tight)] norm expect=PushFilterIntoSetOp SELECT k FROM @@ -1534,7 +1534,7 @@ WHERE k < 10 AND k > 1 except-all ├── columns: k:1!null ├── left columns: b.k:1!null - ├── right columns: a.i:7 + ├── right columns: a.i:8 ├── cardinality: [0 - 8] ├── select │ ├── columns: b.k:1!null @@ -1546,11 +1546,11 @@ except-all │ └── filters │ └── (b.k:1 < 10) AND (b.k:1 > 1) [outer=(1), constraints=(/1: [/2 - /9]; tight)] └── select - ├── columns: a.i:7!null + ├── columns: a.i:8!null ├── scan a - │ └── columns: a.i:7 + │ └── columns: a.i:8 └── filters - └── (a.i:7 < 10) AND (a.i:7 > 1) [outer=(7), constraints=(/7: [/2 - /9]; tight)] + └── (a.i:8 < 10) AND (a.i:8 > 1) [outer=(8), constraints=(/8: [/2 - /9]; tight)] norm expect=PushFilterIntoSetOp SELECT k FROM @@ -1562,7 +1562,7 @@ WHERE k < 10 AND k > 1 intersect ├── columns: k:1!null ├── left columns: b.k:1!null - ├── right columns: a.i:7 + ├── right columns: a.i:8 ├── cardinality: [0 - 8] ├── key: (1) ├── select @@ -1575,11 +1575,11 @@ intersect │ └── filters │ └── (b.k:1 < 10) AND (b.k:1 > 1) [outer=(1), constraints=(/1: [/2 - /9]; tight)] └── select - ├── columns: a.i:7!null + ├── columns: a.i:8!null ├── scan a - │ └── columns: a.i:7 + │ └── columns: a.i:8 └── filters - └── (a.i:7 < 10) AND (a.i:7 > 1) [outer=(7), constraints=(/7: [/2 - /9]; tight)] + └── (a.i:8 < 10) AND (a.i:8 > 1) [outer=(8), constraints=(/8: [/2 - /9]; tight)] norm expect=PushFilterIntoSetOp SELECT k FROM @@ -1591,7 +1591,7 @@ WHERE k < 10 AND k > 1 intersect-all ├── columns: k:1!null ├── left columns: b.k:1!null - ├── right columns: a.i:7 + ├── right columns: a.i:8 ├── cardinality: [0 - 8] ├── select │ ├── columns: b.k:1!null @@ -1603,11 +1603,11 @@ intersect-all │ └── filters │ └── (b.k:1 < 10) AND (b.k:1 > 1) [outer=(1), constraints=(/1: [/2 - /9]; tight)] └── select - ├── columns: a.i:7!null + ├── columns: a.i:8!null ├── scan a - │ └── columns: a.i:7 + │ └── columns: a.i:8 └── filters - └── (a.i:7 < 10) AND (a.i:7 > 1) [outer=(7), constraints=(/7: [/2 - /9]; tight)] + └── (a.i:8 < 10) AND (a.i:8 > 1) [outer=(8), constraints=(/8: [/2 - /9]; tight)] norm expect=PushFilterIntoSetOp SELECT k FROM @@ -1617,11 +1617,11 @@ SELECT k FROM WHERE k < 10 AND k > 1 AND random() < 0.5 ---- union - ├── columns: k:11!null + ├── columns: k:13!null ├── left columns: b.k:1 - ├── right columns: a.i:7 + ├── right columns: a.i:8 ├── volatile - ├── key: (11) + ├── key: (13) ├── select │ ├── columns: b.k:1!null │ ├── cardinality: [0 - 8] @@ -1634,12 +1634,12 @@ union │ ├── (b.k:1 < 10) AND (b.k:1 > 1) [outer=(1), constraints=(/1: [/2 - /9]; tight)] │ └── random() < 0.5 [volatile] └── select - ├── columns: a.i:7!null + ├── columns: a.i:8!null ├── volatile ├── scan a - │ └── columns: a.i:7 + │ └── columns: a.i:8 └── filters - ├── (a.i:7 < 10) AND (a.i:7 > 1) [outer=(7), constraints=(/7: [/2 - /9]; tight)] + ├── (a.i:8 < 10) AND (a.i:8 > 1) [outer=(8), constraints=(/8: [/2 - /9]; tight)] └── random() < 0.5 [volatile] norm expect=PushFilterIntoSetOp @@ -1651,9 +1651,9 @@ SELECT * FROM SELECT * FROM a WHERE k=1) ---- union-all - ├── columns: k:11!null + ├── columns: k:13!null ├── left columns: b.k:1 - ├── right columns: b.k:6 + ├── right columns: b.k:7 ├── select │ ├── columns: b.k:1!null │ ├── key: (1) @@ -1663,35 +1663,35 @@ union-all │ └── filters │ └── exists [subquery] │ └── select - │ ├── columns: a.k:12!null a.i:13 a.f:14 a.s:15 a.j:16 + │ ├── columns: a.k:14!null a.i:15 a.f:16 a.s:17 a.j:18 │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(12-16) + │ ├── fd: ()-->(14-18) │ ├── scan a - │ │ ├── columns: a.k:12!null a.i:13 a.f:14 a.s:15 a.j:16 - │ │ ├── key: (12) - │ │ └── fd: (12)-->(13-16) + │ │ ├── columns: a.k:14!null a.i:15 a.f:16 a.s:17 a.j:18 + │ │ ├── key: (14) + │ │ └── fd: (14)-->(15-18) │ └── filters - │ └── a.k:12 = 1 [outer=(12), constraints=(/12: [/1 - /1]; tight), fd=()-->(12)] + │ └── a.k:14 = 1 [outer=(14), constraints=(/14: [/1 - /1]; tight), fd=()-->(14)] └── select - ├── columns: b.k:6!null - ├── key: (6) + ├── columns: b.k:7!null + ├── key: (7) ├── scan b - │ ├── columns: b.k:6!null - │ └── key: (6) + │ ├── columns: b.k:7!null + │ └── key: (7) └── filters └── exists [subquery] └── select - ├── columns: a.k:12!null a.i:13 a.f:14 a.s:15 a.j:16 + ├── columns: a.k:14!null a.i:15 a.f:16 a.s:17 a.j:18 ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(12-16) + ├── fd: ()-->(14-18) ├── scan a - │ ├── columns: a.k:12!null a.i:13 a.f:14 a.s:15 a.j:16 - │ ├── key: (12) - │ └── fd: (12)-->(13-16) + │ ├── columns: a.k:14!null a.i:15 a.f:16 a.s:17 a.j:18 + │ ├── key: (14) + │ └── fd: (14)-->(15-18) └── filters - └── a.k:12 = 1 [outer=(12), constraints=(/12: [/1 - /1]; tight), fd=()-->(12)] + └── a.k:14 = 1 [outer=(14), constraints=(/14: [/1 - /1]; tight), fd=()-->(14)] norm expect=PushFilterIntoSetOp SELECT * FROM @@ -1702,14 +1702,14 @@ WHERE EXISTS( SELECT * FROM a WHERE k=1) AND random() < 0.5 ---- union-all - ├── columns: k:23!null - ├── left columns: k:11 - ├── right columns: k:22 + ├── columns: k:27!null + ├── left columns: k:13 + ├── right columns: k:26 ├── volatile ├── union-all - │ ├── columns: k:11!null + │ ├── columns: k:13!null │ ├── left columns: b.k:1 - │ ├── right columns: b.k:6 + │ ├── right columns: b.k:7 │ ├── volatile │ ├── select │ │ ├── columns: b.k:1!null @@ -1721,84 +1721,84 @@ union-all │ │ └── filters │ │ ├── exists [subquery] │ │ │ └── select - │ │ │ ├── columns: a.k:24!null a.i:25 a.f:26 a.s:27 a.j:28 + │ │ │ ├── columns: a.k:28!null a.i:29 a.f:30 a.s:31 a.j:32 │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(24-28) + │ │ │ ├── fd: ()-->(28-32) │ │ │ ├── scan a - │ │ │ │ ├── columns: a.k:24!null a.i:25 a.f:26 a.s:27 a.j:28 - │ │ │ │ ├── key: (24) - │ │ │ │ └── fd: (24)-->(25-28) + │ │ │ │ ├── columns: a.k:28!null a.i:29 a.f:30 a.s:31 a.j:32 + │ │ │ │ ├── key: (28) + │ │ │ │ └── fd: (28)-->(29-32) │ │ │ └── filters - │ │ │ └── a.k:24 = 1 [outer=(24), constraints=(/24: [/1 - /1]; tight), fd=()-->(24)] + │ │ │ └── a.k:28 = 1 [outer=(28), constraints=(/28: [/1 - /1]; tight), fd=()-->(28)] │ │ └── random() < 0.5 [volatile] │ └── select - │ ├── columns: b.k:6!null + │ ├── columns: b.k:7!null │ ├── volatile - │ ├── key: (6) + │ ├── key: (7) │ ├── scan b - │ │ ├── columns: b.k:6!null - │ │ └── key: (6) + │ │ ├── columns: b.k:7!null + │ │ └── key: (7) │ └── filters │ ├── exists [subquery] │ │ └── select - │ │ ├── columns: a.k:24!null a.i:25 a.f:26 a.s:27 a.j:28 + │ │ ├── columns: a.k:28!null a.i:29 a.f:30 a.s:31 a.j:32 │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(24-28) + │ │ ├── fd: ()-->(28-32) │ │ ├── scan a - │ │ │ ├── columns: a.k:24!null a.i:25 a.f:26 a.s:27 a.j:28 - │ │ │ ├── key: (24) - │ │ │ └── fd: (24)-->(25-28) + │ │ │ ├── columns: a.k:28!null a.i:29 a.f:30 a.s:31 a.j:32 + │ │ │ ├── key: (28) + │ │ │ └── fd: (28)-->(29-32) │ │ └── filters - │ │ └── a.k:24 = 1 [outer=(24), constraints=(/24: [/1 - /1]; tight), fd=()-->(24)] + │ │ └── a.k:28 = 1 [outer=(28), constraints=(/28: [/1 - /1]; tight), fd=()-->(28)] │ └── random() < 0.5 [volatile] └── union-all - ├── columns: k:22!null - ├── left columns: b.k:12 - ├── right columns: b.k:17 + ├── columns: k:26!null + ├── left columns: b.k:14 + ├── right columns: b.k:20 ├── volatile ├── select - │ ├── columns: b.k:12!null + │ ├── columns: b.k:14!null │ ├── volatile - │ ├── key: (12) + │ ├── key: (14) │ ├── scan b - │ │ ├── columns: b.k:12!null - │ │ └── key: (12) + │ │ ├── columns: b.k:14!null + │ │ └── key: (14) │ └── filters │ ├── exists [subquery] │ │ └── select - │ │ ├── columns: a.k:24!null a.i:25 a.f:26 a.s:27 a.j:28 + │ │ ├── columns: a.k:28!null a.i:29 a.f:30 a.s:31 a.j:32 │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(24-28) + │ │ ├── fd: ()-->(28-32) │ │ ├── scan a - │ │ │ ├── columns: a.k:24!null a.i:25 a.f:26 a.s:27 a.j:28 - │ │ │ ├── key: (24) - │ │ │ └── fd: (24)-->(25-28) + │ │ │ ├── columns: a.k:28!null a.i:29 a.f:30 a.s:31 a.j:32 + │ │ │ ├── key: (28) + │ │ │ └── fd: (28)-->(29-32) │ │ └── filters - │ │ └── a.k:24 = 1 [outer=(24), constraints=(/24: [/1 - /1]; tight), fd=()-->(24)] + │ │ └── a.k:28 = 1 [outer=(28), constraints=(/28: [/1 - /1]; tight), fd=()-->(28)] │ └── random() < 0.5 [volatile] └── select - ├── columns: b.k:17!null + ├── columns: b.k:20!null ├── volatile - ├── key: (17) + ├── key: (20) ├── scan b - │ ├── columns: b.k:17!null - │ └── key: (17) + │ ├── columns: b.k:20!null + │ └── key: (20) └── filters ├── exists [subquery] │ └── select - │ ├── columns: a.k:24!null a.i:25 a.f:26 a.s:27 a.j:28 + │ ├── columns: a.k:28!null a.i:29 a.f:30 a.s:31 a.j:32 │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(24-28) + │ ├── fd: ()-->(28-32) │ ├── scan a - │ │ ├── columns: a.k:24!null a.i:25 a.f:26 a.s:27 a.j:28 - │ │ ├── key: (24) - │ │ └── fd: (24)-->(25-28) + │ │ ├── columns: a.k:28!null a.i:29 a.f:30 a.s:31 a.j:32 + │ │ ├── key: (28) + │ │ └── fd: (28)-->(29-32) │ └── filters - │ └── a.k:24 = 1 [outer=(24), constraints=(/24: [/1 - /1]; tight), fd=()-->(24)] + │ └── a.k:28 = 1 [outer=(28), constraints=(/28: [/1 - /1]; tight), fd=()-->(28)] └── random() < 0.5 [volatile] # No-op case because the filter references outer columns. @@ -1812,37 +1812,37 @@ SELECT FROM a ---- project - ├── columns: k:17 + ├── columns: k:20 ├── ensure-distinct-on - │ ├── columns: a.k:1!null k:16 + │ ├── columns: a.k:1!null k:19 │ ├── grouping columns: a.k:1!null │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── key: (1) - │ ├── fd: (1)-->(16) + │ ├── fd: (1)-->(19) │ ├── left-join (cross) - │ │ ├── columns: a.k:1!null a.i:2 k:16 + │ │ ├── columns: a.k:1!null a.i:2 k:19 │ │ ├── fd: (1)-->(2) │ │ ├── scan a │ │ │ ├── columns: a.k:1!null a.i:2 │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2) │ │ ├── union-all - │ │ │ ├── columns: k:16!null - │ │ │ ├── left columns: b.k:6 - │ │ │ ├── right columns: b.k:11 + │ │ │ ├── columns: k:19!null + │ │ │ ├── left columns: b.k:7 + │ │ │ ├── right columns: b.k:13 │ │ │ ├── scan b - │ │ │ │ ├── columns: b.k:6!null - │ │ │ │ └── key: (6) + │ │ │ │ ├── columns: b.k:7!null + │ │ │ │ └── key: (7) │ │ │ └── scan b - │ │ │ ├── columns: b.k:11!null - │ │ │ └── key: (11) + │ │ │ ├── columns: b.k:13!null + │ │ │ └── key: (13) │ │ └── filters - │ │ └── k:16 < a.i:2 [outer=(2,16), constraints=(/2: (/NULL - ]; /16: (/NULL - ])] + │ │ └── k:19 < a.i:2 [outer=(2,19), constraints=(/2: (/NULL - ]; /19: (/NULL - ])] │ └── aggregations - │ └── const-agg [as=k:16, outer=(16)] - │ └── k:16 + │ └── const-agg [as=k:19, outer=(19)] + │ └── k:19 └── projections - └── k:16 [as=k:17, outer=(16)] + └── k:19 [as=k:20, outer=(19)] norm SELECT * FROM ((values (1,2)) diff --git a/pkg/sql/opt/norm/testdata/rules/set b/pkg/sql/opt/norm/testdata/rules/set index 2d2832bc1921..c10870c8e713 100644 --- a/pkg/sql/opt/norm/testdata/rules/set +++ b/pkg/sql/opt/norm/testdata/rules/set @@ -17,13 +17,13 @@ SELECT k FROM (SELECT k FROM b WHERE k IN ()) ---- project - ├── columns: k:11!null - ├── key: (11) + ├── columns: k:13!null + ├── key: (13) ├── scan b │ ├── columns: b.k:1!null │ └── key: (1) └── projections - └── b.k:1 [as=k:11, outer=(1)] + └── b.k:1 [as=k:13, outer=(1)] # -------------------------------------------------- # EliminateUnionAllRight @@ -36,13 +36,13 @@ SELECT k FROM (SELECT k FROM b) ---- project - ├── columns: k:11!null - ├── key: (11) + ├── columns: k:13!null + ├── key: (13) ├── scan b - │ ├── columns: b.k:6!null - │ └── key: (6) + │ ├── columns: b.k:7!null + │ └── key: (7) └── projections - └── b.k:6 [as=k:11, outer=(6)] + └── b.k:7 [as=k:13, outer=(7)] norm SELECT k FROM @@ -51,7 +51,7 @@ SELECT k FROM (SELECT k FROM b WHERE i IN ()) ---- values - ├── columns: k:11!null + ├── columns: k:13!null ├── cardinality: [0 - 0] ├── key: () - └── fd: ()-->(11) + └── fd: ()-->(13) diff --git a/pkg/sql/opt/norm/testdata/rules/side_effects b/pkg/sql/opt/norm/testdata/rules/side_effects index 3fd63f7d1e22..b14898d3e412 100644 --- a/pkg/sql/opt/norm/testdata/rules/side_effects +++ b/pkg/sql/opt/norm/testdata/rules/side_effects @@ -17,38 +17,38 @@ norm SELECT * FROM a ORDER BY length('foo'), random()+1.0 ---- sort - ├── columns: k:1!null i:2 f:3 s:4 j:5 [hidden: column7:7] + ├── columns: k:1!null i:2 f:3 s:4 j:5 [hidden: column8:8] ├── volatile ├── key: (1) - ├── fd: (1)-->(2-5,7) - ├── ordering: +7 + ├── fd: (1)-->(2-5,8) + ├── ordering: +8 └── project - ├── columns: column7:7 k:1!null i:2 f:3 s:4 j:5 + ├── columns: column8:8 k:1!null i:2 f:3 s:4 j:5 ├── volatile ├── key: (1) - ├── fd: (1)-->(2-5,7) + ├── fd: (1)-->(2-5,8) ├── scan a │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) └── projections - └── random() + 1.0 [as=column7:7, volatile] + └── random() + 1.0 [as=column8:8, volatile] # Don't allow GROUP BY column to be eliminated if it has a side effect. norm SELECT avg(f) FROM a WHERE i=5 GROUP BY i+(random()*10)::int, i+1 ---- project - ├── columns: avg:6 + ├── columns: avg:7 ├── volatile └── group-by - ├── columns: avg:6 column7:7 - ├── grouping columns: column7:7 + ├── columns: avg:7 column8:8 + ├── grouping columns: column8:8 ├── volatile - ├── key: (7) - ├── fd: (7)-->(6) + ├── key: (8) + ├── fd: (8)-->(7) ├── project - │ ├── columns: column7:7 f:3 + │ ├── columns: column8:8 f:3 │ ├── volatile │ ├── select │ │ ├── columns: i:2!null f:3 @@ -58,9 +58,9 @@ project │ │ └── filters │ │ └── i:2 = 5 [outer=(2), constraints=(/2: [/5 - /5]; tight), fd=()-->(2)] │ └── projections - │ └── i:2 + (random() * 10.0)::INT8 [as=column7:7, outer=(2), volatile] + │ └── i:2 + (random() * 10.0)::INT8 [as=column8:8, outer=(2), volatile] └── aggregations - └── avg [as=avg:6, outer=(3)] + └── avg [as=avg:7, outer=(3)] └── f:3 # Allow elimination of side effecting expressions during column pruning. @@ -75,11 +75,11 @@ norm SELECT * FROM a INNER JOIN xy ON k=x WHERE k=random() ---- inner-join (hash) - ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6!null y:7 + ├── columns: k:1!null i:2 f:3 s:4 j:5 x:7!null y:8 ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) ├── volatile - ├── key: (6) - ├── fd: (1)-->(2-5), (6)-->(7), (1)==(6), (6)==(1) + ├── key: (7) + ├── fd: (1)-->(2-5), (7)-->(8), (1)==(7), (7)==(1) ├── select │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ ├── volatile @@ -92,39 +92,39 @@ inner-join (hash) │ └── filters │ └── k:1 = random() [outer=(1), volatile, constraints=(/1: (/NULL - ])] ├── select - │ ├── columns: x:6!null y:7 + │ ├── columns: x:7!null y:8 │ ├── volatile - │ ├── key: (6) - │ ├── fd: (6)-->(7) + │ ├── key: (7) + │ ├── fd: (7)-->(8) │ ├── scan xy - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ └── filters - │ └── x:6 = random() [outer=(6), volatile, constraints=(/6: (/NULL - ])] + │ └── x:7 = random() [outer=(7), volatile, constraints=(/7: (/NULL - ])] └── filters - └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── k:1 = x:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] # Decorrelate CASE WHEN branch if there are no side effects. norm SELECT CASE WHEN i<0 THEN (SELECT y FROM xy WHERE x=i LIMIT 1) ELSE 5 END FROM a ---- project - ├── columns: case:8 + ├── columns: case:10 ├── left-join (hash) - │ ├── columns: i:2 x:6 y:7 + │ ├── columns: i:2 x:7 y:8 │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ ├── fd: (6)-->(7) + │ ├── fd: (7)-->(8) │ ├── scan a │ │ └── columns: i:2 │ ├── scan xy - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ └── filters - │ └── x:6 = i:2 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + │ └── x:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] └── projections - └── CASE WHEN i:2 < 0 THEN y:7 ELSE 5 END [as=case:8, outer=(2,7)] + └── CASE WHEN i:2 < 0 THEN y:8 ELSE 5 END [as=case:10, outer=(2,8)] # Decorrelate CASE ELSE branch if there are no side effects. norm @@ -135,68 +135,68 @@ project ├── key: (1) ├── fd: (1)-->(2-5) └── select - ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6 y:7 + ├── columns: k:1!null i:2 f:3 s:4 j:5 x:7 y:8 ├── key: (1) - ├── fd: (1)-->(2-7), (6)-->(7) + ├── fd: (1)-->(2-5,7,8), (7)-->(8) ├── left-join (hash) - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6 y:7 + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:7 y:8 │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) │ ├── key: (1) - │ ├── fd: (1)-->(2-7), (6)-->(7) + │ ├── fd: (1)-->(2-5,7,8), (7)-->(8) │ ├── scan a │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ │ ├── key: (1) │ │ └── fd: (1)-->(2-5) │ ├── scan xy - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ └── filters - │ └── x:6 = i:2 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + │ └── x:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] └── filters - └── k:1 = CASE WHEN i:2 < 0 THEN 5 ELSE y:7 END [outer=(1,2,7), constraints=(/1: (/NULL - ])] + └── k:1 = CASE WHEN i:2 < 0 THEN 5 ELSE y:8 END [outer=(1,2,8), constraints=(/1: (/NULL - ])] # Don't decorrelate CASE WHEN branch if there are side effects. norm SELECT CASE WHEN i<0 THEN (SELECT y FROM xy WHERE x=i LIMIT (random()*10)::int) ELSE 5 END FROM a ---- project - ├── columns: case:8 + ├── columns: case:10 ├── volatile ├── scan a │ └── columns: i:2 └── projections - └── case [as=case:8, outer=(2), volatile, correlated-subquery] + └── case [as=case:10, outer=(2), volatile, correlated-subquery] ├── true ├── when │ ├── i:2 < 0 │ └── subquery │ └── project - │ ├── columns: y:7 + │ ├── columns: y:8 │ ├── outer: (2) │ ├── cardinality: [0 - 1] │ ├── volatile │ ├── key: () - │ ├── fd: ()-->(7) + │ ├── fd: ()-->(8) │ └── limit - │ ├── columns: x:6!null y:7 + │ ├── columns: x:7!null y:8 │ ├── outer: (2) │ ├── cardinality: [0 - 1] │ ├── volatile │ ├── key: () - │ ├── fd: ()-->(6,7) + │ ├── fd: ()-->(7,8) │ ├── select - │ │ ├── columns: x:6!null y:7 + │ │ ├── columns: x:7!null y:8 │ │ ├── outer: (2) │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(6,7) + │ │ ├── fd: ()-->(7,8) │ │ ├── scan xy - │ │ │ ├── columns: x:6!null y:7 - │ │ │ ├── key: (6) - │ │ │ └── fd: (6)-->(7) + │ │ │ ├── columns: x:7!null y:8 + │ │ │ ├── key: (7) + │ │ │ └── fd: (7)-->(8) │ │ └── filters - │ │ └── x:6 = i:2 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + │ │ └── x:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] │ └── (random() * 10.0)::INT8 └── 5 @@ -223,26 +223,26 @@ select │ └── 5 └── subquery └── project - ├── columns: y:7 + ├── columns: y:8 ├── outer: (2) ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(7) + ├── fd: ()-->(8) └── select - ├── columns: x:6!null y:7 + ├── columns: x:7!null y:8 ├── outer: (2) ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(6,7) + ├── fd: ()-->(7,8) ├── scan xy - │ ├── columns: x:6!null y:7 - │ ├── key: (6) - │ └── fd: (6)-->(7) + │ ├── columns: x:7!null y:8 + │ ├── key: (7) + │ └── fd: (7)-->(8) └── filters - ├── x:6 = i:2 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] - └── (5 / y:7) > 1 [outer=(7), immutable] + ├── x:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] + └── (5 / y:8) > 1 [outer=(8), immutable] # Don't decorrelate IFERROR branch if there are side effects @@ -266,28 +266,28 @@ select └── else └── subquery └── project - ├── columns: y:8 + ├── columns: y:10 ├── outer: (2) ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(8) + ├── fd: ()-->(10) ├── select - │ ├── columns: x:6!null xy.y:7 + │ ├── columns: x:7!null xy.y:8 │ ├── outer: (2) │ ├── cardinality: [0 - 1] │ ├── immutable │ ├── key: () - │ ├── fd: ()-->(6,7) + │ ├── fd: ()-->(7,8) │ ├── scan xy - │ │ ├── columns: x:6!null xy.y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null xy.y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ └── filters - │ ├── x:6 = i:2 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] - │ └── (5 / xy.y:7) > 1 [outer=(7), immutable] + │ ├── x:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] + │ └── (5 / xy.y:8) > 1 [outer=(8), immutable] └── projections - └── xy.y:7::DECIMAL [as=y:8, outer=(7), immutable] + └── xy.y:8::DECIMAL [as=y:10, outer=(8), immutable] # Decorrelate IFERROR branch if there are no side effects norm @@ -299,24 +299,24 @@ project ├── key: (1) ├── fd: (1)-->(2-5) └── select - ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6 y:7 + ├── columns: k:1!null i:2 f:3 s:4 j:5 x:7 y:8 ├── immutable ├── key: (1) - ├── fd: (1)-->(2-7), (6)-->(7) + ├── fd: (1)-->(2-5,7,8), (7)-->(8) ├── left-join (hash) - │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:6 y:7 + │ ├── columns: k:1!null i:2 f:3 s:4 j:5 x:7 y:8 │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) │ ├── key: (1) - │ ├── fd: (1)-->(2-7), (6)-->(7) + │ ├── fd: (1)-->(2-5,7,8), (7)-->(8) │ ├── scan a │ │ ├── columns: k:1!null i:2 f:3 s:4 j:5 │ │ ├── key: (1) │ │ └── fd: (1)-->(2-5) │ ├── scan xy - │ │ ├── columns: x:6!null y:7 - │ │ ├── key: (6) - │ │ └── fd: (6)-->(7) + │ │ ├── columns: x:7!null y:8 + │ │ ├── key: (7) + │ │ └── fd: (7)-->(8) │ └── filters - │ └── x:6 = i:2 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + │ └── x:7 = i:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] └── filters - └── k:1 = IFERROR((1 / 0)::INT8, y:7) [outer=(1,7), immutable, constraints=(/1: (/NULL - ])] + └── k:1 = IFERROR((1 / 0)::INT8, y:8) [outer=(1,8), immutable, constraints=(/1: (/NULL - ])] diff --git a/pkg/sql/opt/norm/testdata/rules/window b/pkg/sql/opt/norm/testdata/rules/window index 6814db68e2a3..6932dddbd5d8 100644 --- a/pkg/sql/opt/norm/testdata/rules/window +++ b/pkg/sql/opt/norm/testdata/rules/window @@ -10,27 +10,27 @@ norm expect=ReduceWindowPartitionCols SELECT rank() OVER (PARTITION BY k, i) FROM a ---- project - ├── columns: rank:6 + ├── columns: rank:7 └── window partition=(1) - ├── columns: k:1!null rank:6 + ├── columns: k:1!null rank:7 ├── key: (1) ├── scan a │ ├── columns: k:1!null │ └── key: (1) └── windows - └── rank [as=rank:6] + └── rank [as=rank:7] norm expect=ReduceWindowPartitionCols SELECT rank() OVER (PARTITION BY i, i+1) FROM a ---- project - ├── columns: rank:6 + ├── columns: rank:7 └── window partition=(2) - ├── columns: i:2 rank:6 + ├── columns: i:2 rank:7 ├── scan a │ └── columns: i:2 └── windows - └── rank [as=rank:6] + └── rank [as=rank:7] # -------------------------------------------------- # SimplifyWindowOrdering @@ -40,15 +40,15 @@ norm expect=SimplifyWindowOrdering SELECT rank() OVER (ORDER BY k, i) FROM a ---- project - ├── columns: rank:6 + ├── columns: rank:7 └── window partition=() ordering=+1 - ├── columns: k:1!null rank:6 + ├── columns: k:1!null rank:7 ├── key: (1) ├── scan a │ ├── columns: k:1!null │ └── key: (1) └── windows - └── rank [as=rank:6] + └── rank [as=rank:7] # We can simplify the ordering with the knowledge that within any partition # the set of partition cols is held constant. @@ -58,39 +58,39 @@ norm SELECT rank() OVER (PARTITION BY k ORDER BY i) FROM a ---- project - ├── columns: rank:6 + ├── columns: rank:7 └── window partition=(1) - ├── columns: k:1!null rank:6 + ├── columns: k:1!null rank:7 ├── key: (1) ├── scan a │ ├── columns: k:1!null │ └── key: (1) └── windows - └── rank [as=rank:6] + └── rank [as=rank:7] norm expect=SimplifyWindowOrdering SELECT rank() OVER (PARTITION BY i ORDER BY f, i+1) FROM a ---- project - ├── columns: rank:6 - └── window partition=(2) ordering=+3 opt(2,7) - ├── columns: i:2 f:3 rank:6 + ├── columns: rank:7 + └── window partition=(2) ordering=+3 opt(2,8) + ├── columns: i:2 f:3 rank:7 ├── scan a │ └── columns: i:2 f:3 └── windows - └── rank [as=rank:6] + └── rank [as=rank:7] norm expect=SimplifyWindowOrdering SELECT rank() OVER (PARTITION BY f ORDER BY i) FROM a ---- project - ├── columns: rank:6 + ├── columns: rank:7 └── window partition=(3) ordering=+2 opt(3) - ├── columns: i:2 f:3 rank:6 + ├── columns: i:2 f:3 rank:7 ├── scan a │ └── columns: i:2 f:3 └── windows - └── rank [as=rank:6] + └── rank [as=rank:7] # PushSelectIntoWindow @@ -98,7 +98,7 @@ norm expect=PushSelectIntoWindow SELECT * FROM (SELECT i, rank() OVER (PARTITION BY i) FROM a) WHERE i > 4 ---- window partition=(2) - ├── columns: i:2!null rank:6 + ├── columns: i:2!null rank:7 ├── select │ ├── columns: i:2!null │ ├── scan a @@ -106,17 +106,17 @@ window partition=(2) │ └── filters │ └── i:2 > 4 [outer=(2), constraints=(/2: [/5 - ]; tight)] └── windows - └── rank [as=rank:6] + └── rank [as=rank:7] # Only push down filters bound by the partition cols. norm expect=PushSelectIntoWindow SELECT * FROM (SELECT i, s, rank() OVER (PARTITION BY i) FROM a) WHERE i > 4 AND s = 'foo' ---- select - ├── columns: i:2!null s:4!null rank:6 + ├── columns: i:2!null s:4!null rank:7 ├── fd: ()-->(4) ├── window partition=(2) - │ ├── columns: i:2!null s:4 rank:6 + │ ├── columns: i:2!null s:4 rank:7 │ ├── select │ │ ├── columns: i:2!null s:4 │ │ ├── scan a @@ -124,7 +124,7 @@ select │ │ └── filters │ │ └── i:2 > 4 [outer=(2), constraints=(/2: [/5 - ]; tight)] │ └── windows - │ └── rank [as=rank:6] + │ └── rank [as=rank:7] └── filters └── s:4 = 'foo' [outer=(4), constraints=(/4: [/'foo' - /'foo']; tight), fd=()-->(4)] @@ -133,10 +133,10 @@ norm expect=PushSelectIntoWindow SELECT * FROM (SELECT i, s, f, rank() OVER (PARTITION BY i, f) FROM a) WHERE i > 4 AND f = 3.0 AND s = 'foo' ---- select - ├── columns: i:2!null s:4!null f:3!null rank:6 + ├── columns: i:2!null s:4!null f:3!null rank:7 ├── fd: ()-->(3,4) ├── window partition=(2) - │ ├── columns: i:2!null f:3!null s:4 rank:6 + │ ├── columns: i:2!null f:3!null s:4 rank:7 │ ├── fd: ()-->(3) │ ├── select │ │ ├── columns: i:2!null f:3!null s:4 @@ -147,7 +147,7 @@ select │ │ ├── i:2 > 4 [outer=(2), constraints=(/2: [/5 - ]; tight)] │ │ └── f:3 = 3.0 [outer=(3), constraints=(/3: [/3.0 - /3.0]; tight), fd=()-->(3)] │ └── windows - │ └── rank [as=rank:6] + │ └── rank [as=rank:7] └── filters └── s:4 = 'foo' [outer=(4), constraints=(/4: [/'foo' - /'foo']; tight), fd=()-->(4)] @@ -157,7 +157,7 @@ norm expect=PushSelectIntoWindow SELECT * FROM (SELECT i, s, f, rank() OVER (PARTITION BY i, f) FROM a) WHERE random() < 0.5 ---- window partition=(2,3) - ├── columns: i:2 s:4 f:3 rank:6 + ├── columns: i:2 s:4 f:3 rank:7 ├── volatile ├── select │ ├── columns: i:2 f:3 s:4 @@ -167,22 +167,22 @@ window partition=(2,3) │ └── filters │ └── random() < 0.5 [volatile] └── windows - └── rank [as=rank:6] + └── rank [as=rank:7] # Can't push down a filter on an ordering column. norm expect-not=PushSelectIntoWindow SELECT * FROM (SELECT f, rank() OVER (PARTITION BY i ORDER BY f) FROM a) WHERE f > 4 ---- project - ├── columns: f:3!null rank:6 + ├── columns: f:3!null rank:7 └── select - ├── columns: i:2 f:3!null rank:6 + ├── columns: i:2 f:3!null rank:7 ├── window partition=(2) ordering=+3 opt(2) - │ ├── columns: i:2 f:3 rank:6 + │ ├── columns: i:2 f:3 rank:7 │ ├── scan a │ │ └── columns: i:2 f:3 │ └── windows - │ └── rank [as=rank:6] + │ └── rank [as=rank:7] └── filters └── f:3 > 4.0 [outer=(3), constraints=(/3: [/4.000000000000001 - ]; tight)] @@ -191,17 +191,17 @@ norm expect-not=PushSelectIntoWindow SELECT * FROM (SELECT s, rank() OVER (PARTITION BY i ORDER BY f) FROM a) WHERE s = 'foo' ---- project - ├── columns: s:4!null rank:6 + ├── columns: s:4!null rank:7 ├── fd: ()-->(4) └── select - ├── columns: i:2 f:3 s:4!null rank:6 + ├── columns: i:2 f:3 s:4!null rank:7 ├── fd: ()-->(4) ├── window partition=(2) ordering=+3 opt(2) - │ ├── columns: i:2 f:3 s:4 rank:6 + │ ├── columns: i:2 f:3 s:4 rank:7 │ ├── scan a │ │ └── columns: i:2 f:3 s:4 │ └── windows - │ └── rank [as=rank:6] + │ └── rank [as=rank:7] └── filters └── s:4 = 'foo' [outer=(4), constraints=(/4: [/'foo' - /'foo']; tight), fd=()-->(4)] @@ -210,10 +210,10 @@ norm expect=PushSelectIntoWindow SELECT * FROM (SELECT i, rank() OVER (PARTITION BY k ORDER BY f) FROM a) WHERE i = 3 ---- project - ├── columns: i:2!null rank:6 + ├── columns: i:2!null rank:7 ├── fd: ()-->(2) └── window partition=(1) - ├── columns: k:1!null i:2!null rank:6 + ├── columns: k:1!null i:2!null rank:7 ├── key: (1) ├── fd: ()-->(2) ├── select @@ -227,16 +227,16 @@ project │ └── filters │ └── i:2 = 3 [outer=(2), constraints=(/2: [/3 - /3]; tight), fd=()-->(2)] └── windows - └── rank [as=rank:6] + └── rank [as=rank:7] norm expect=PushSelectIntoWindow SELECT * FROM (SELECT i, f, rank() OVER (PARTITION BY k ORDER BY f) FROM a) WHERE i*f::int = 3 ---- project - ├── columns: i:2 f:3 rank:6 + ├── columns: i:2 f:3 rank:7 ├── immutable └── window partition=(1) - ├── columns: k:1!null i:2 f:3 rank:6 + ├── columns: k:1!null i:2 f:3 rank:7 ├── immutable ├── key: (1) ├── fd: (1)-->(2,3) @@ -252,20 +252,20 @@ project │ └── filters │ └── (i:2 * f:3::INT8) = 3 [outer=(2,3), immutable] └── windows - └── rank [as=rank:6] + └── rank [as=rank:7] norm expect-not=PushSelectIntoWindow SELECT * FROM (SELECT i, f, rank() OVER (PARTITION BY k ORDER BY f) AS rnk FROM a) WHERE rnk = 3 ---- project - ├── columns: i:2 f:3 rnk:6!null - ├── fd: ()-->(6) + ├── columns: i:2 f:3 rnk:7!null + ├── fd: ()-->(7) └── select - ├── columns: k:1!null i:2 f:3 rank:6!null + ├── columns: k:1!null i:2 f:3 rank:7!null ├── key: (1) - ├── fd: ()-->(6), (1)-->(2,3) + ├── fd: ()-->(7), (1)-->(2,3) ├── window partition=(1) - │ ├── columns: k:1!null i:2 f:3 rank:6 + │ ├── columns: k:1!null i:2 f:3 rank:7 │ ├── key: (1) │ ├── fd: (1)-->(2,3) │ ├── scan a @@ -273,9 +273,9 @@ project │ │ ├── key: (1) │ │ └── fd: (1)-->(2,3) │ └── windows - │ └── rank [as=rank:6] + │ └── rank [as=rank:7] └── filters - └── rank:6 = 3 [outer=(6), constraints=(/6: [/3 - /3]; tight), fd=()-->(6)] + └── rank:7 = 3 [outer=(7), constraints=(/7: [/3 - /3]; tight), fd=()-->(7)] # -------------------------------------------------- # PushLimitIntoWindow @@ -285,7 +285,7 @@ norm SELECT rank() OVER () FROM a LIMIT 10 ---- window partition=() - ├── columns: rank:6 + ├── columns: rank:7 ├── cardinality: [0 - 10] ├── limit │ ├── cardinality: [0 - 10] @@ -293,16 +293,16 @@ window partition=() │ │ └── limit hint: 10.00 │ └── 10 └── windows - └── rank [as=rank:6] + └── rank [as=rank:7] norm SELECT rank() OVER (PARTITION BY i) FROM a LIMIT 10 ---- project - ├── columns: rank:6 + ├── columns: rank:7 ├── cardinality: [0 - 10] └── window partition=(2) - ├── columns: i:2 rank:6 + ├── columns: i:2 rank:7 ├── cardinality: [0 - 10] ├── limit │ ├── columns: i:2 @@ -316,16 +316,16 @@ project │ │ └── columns: i:2 │ └── 10 └── windows - └── rank [as=rank:6] + └── rank [as=rank:7] norm SELECT rank() OVER (PARTITION BY i ORDER BY f) FROM a LIMIT 10 ---- project - ├── columns: rank:6 + ├── columns: rank:7 ├── cardinality: [0 - 10] └── window partition=(2) ordering=+3 opt(2) - ├── columns: i:2 f:3 rank:6 + ├── columns: i:2 f:3 rank:7 ├── cardinality: [0 - 10] ├── limit │ ├── columns: i:2 f:3 @@ -339,7 +339,7 @@ project │ │ └── columns: i:2 f:3 │ └── 10 └── windows - └── rank [as=rank:6] + └── rank [as=rank:7] norm SELECT @@ -348,10 +348,10 @@ SELECT FROM a LIMIT 10 ---- project - ├── columns: rank:6 dense_rank:7 + ├── columns: rank:7 dense_rank:8 ├── cardinality: [0 - 10] └── window partition=(2) ordering=+3 opt(2) - ├── columns: i:2 f:3 rank:6 dense_rank:7 + ├── columns: i:2 f:3 rank:7 dense_rank:8 ├── cardinality: [0 - 10] ├── limit │ ├── columns: i:2 f:3 @@ -365,8 +365,8 @@ project │ │ └── columns: i:2 f:3 │ └── 10 └── windows - ├── rank [as=rank:6] - └── dense-rank [as=dense_rank:7] + ├── rank [as=rank:7] + └── dense-rank [as=dense_rank:8] # Can't push the limit down, because the window function used is not # "prefix-safe". @@ -374,21 +374,21 @@ norm expect-not=PushLimitIntoWindow SELECT avg(k) OVER () FROM a LIMIT 10 ---- project - ├── columns: avg:6 + ├── columns: avg:7 ├── cardinality: [0 - 10] └── limit - ├── columns: k:1!null avg:6 + ├── columns: k:1!null avg:7 ├── cardinality: [0 - 10] ├── key: (1) ├── window partition=() - │ ├── columns: k:1!null avg:6 + │ ├── columns: k:1!null avg:7 │ ├── key: (1) │ ├── limit hint: 10.00 │ ├── scan a │ │ ├── columns: k:1!null │ │ └── key: (1) │ └── windows - │ └── avg [as=avg:6, outer=(1)] + │ └── avg [as=avg:7, outer=(1)] │ └── k:1 └── 10 @@ -398,24 +398,24 @@ norm expect-not=PushLimitIntoWindow SELECT rank() OVER (ORDER BY i) FROM a ORDER BY f LIMIT 10 ---- project - ├── columns: rank:6 [hidden: f:3] + ├── columns: rank:7 [hidden: f:3] ├── cardinality: [0 - 10] ├── ordering: +3 └── limit - ├── columns: i:2 f:3 rank:6 + ├── columns: i:2 f:3 rank:7 ├── internal-ordering: +3 ├── cardinality: [0 - 10] ├── ordering: +3 ├── sort - │ ├── columns: i:2 f:3 rank:6 + │ ├── columns: i:2 f:3 rank:7 │ ├── ordering: +3 │ ├── limit hint: 10.00 │ └── window partition=() ordering=+2 - │ ├── columns: i:2 f:3 rank:6 + │ ├── columns: i:2 f:3 rank:7 │ ├── scan a │ │ └── columns: i:2 f:3 │ └── windows - │ └── rank [as=rank:6] + │ └── rank [as=rank:7] └── 10 # The limit should become stronger as it gets pushed down (going from +f to @@ -425,14 +425,14 @@ norm SELECT rank() OVER (ORDER BY f, i) FROM a ORDER BY f LIMIT 10 ---- sort - ├── columns: rank:6 [hidden: f:3] + ├── columns: rank:7 [hidden: f:3] ├── cardinality: [0 - 10] ├── ordering: +3 └── project - ├── columns: f:3 rank:6 + ├── columns: f:3 rank:7 ├── cardinality: [0 - 10] └── window partition=() ordering=+3,+2 - ├── columns: i:2 f:3 rank:6 + ├── columns: i:2 f:3 rank:7 ├── cardinality: [0 - 10] ├── limit │ ├── columns: i:2 f:3 @@ -446,7 +446,7 @@ sort │ │ └── columns: i:2 f:3 │ └── 10 └── windows - └── rank [as=rank:6] + └── rank [as=rank:7] # Here we would only be able to push below the rank(), and not the avg(k). This # is not profitable because we still have to do the partitioning and ordering @@ -462,15 +462,15 @@ LIMIT 10 ---- project - ├── columns: rank:6 avg:7 + ├── columns: rank:7 avg:8 ├── cardinality: [0 - 10] └── limit - ├── columns: k:1!null i:2 f:3 rank:6 avg:7 + ├── columns: k:1!null i:2 f:3 rank:7 avg:8 ├── cardinality: [0 - 10] ├── key: (1) ├── fd: (1)-->(2,3) ├── window partition=(2) ordering=+3 opt(2) - │ ├── columns: k:1!null i:2 f:3 rank:6 avg:7 + │ ├── columns: k:1!null i:2 f:3 rank:7 avg:8 │ ├── key: (1) │ ├── fd: (1)-->(2,3) │ ├── limit hint: 10.00 @@ -479,8 +479,8 @@ project │ │ ├── key: (1) │ │ └── fd: (1)-->(2,3) │ └── windows - │ ├── rank [as=rank:6] - │ └── avg [as=avg:7, outer=(1)] + │ ├── rank [as=rank:7] + │ └── avg [as=avg:8, outer=(1)] │ └── k:1 └── 10 @@ -492,20 +492,20 @@ norm expect-not=PushLimitIntoWindow SELECT *, rank() OVER (PARTITION BY z ORDER BY y) FROM wxyz ORDER BY y LIMIT 2 ---- limit - ├── columns: w:1!null x:2 y:3 z:4 rank:5 + ├── columns: w:1!null x:2 y:3 z:4 rank:6 ├── internal-ordering: +3 ├── cardinality: [0 - 2] ├── key: (1) ├── fd: (1)-->(2-4) ├── ordering: +3 ├── sort - │ ├── columns: w:1!null x:2 y:3 z:4 rank:5 + │ ├── columns: w:1!null x:2 y:3 z:4 rank:6 │ ├── key: (1) │ ├── fd: (1)-->(2-4) │ ├── ordering: +3 │ ├── limit hint: 2.00 │ └── window partition=(4) ordering=+3 opt(4) - │ ├── columns: w:1!null x:2 y:3 z:4 rank:5 + │ ├── columns: w:1!null x:2 y:3 z:4 rank:6 │ ├── key: (1) │ ├── fd: (1)-->(2-4) │ ├── scan wxyz @@ -513,20 +513,20 @@ limit │ │ ├── key: (1) │ │ └── fd: (1)-->(2-4) │ └── windows - │ └── rank [as=rank:5] + │ └── rank [as=rank:6] └── 2 norm expect=PushLimitIntoWindow SELECT *, rank() OVER (PARTITION BY w ORDER BY y) FROM wxyz ORDER BY y LIMIT 2 ---- sort - ├── columns: w:1!null x:2 y:3 z:4 rank:5 + ├── columns: w:1!null x:2 y:3 z:4 rank:6 ├── cardinality: [0 - 2] ├── key: (1) ├── fd: (1)-->(2-4) ├── ordering: +3 └── window partition=(1) - ├── columns: w:1!null x:2 y:3 z:4 rank:5 + ├── columns: w:1!null x:2 y:3 z:4 rank:6 ├── cardinality: [0 - 2] ├── key: (1) ├── fd: (1)-->(2-4) @@ -548,19 +548,19 @@ sort │ │ └── fd: (1)-->(2-4) │ └── 2 └── windows - └── rank [as=rank:5] + └── rank [as=rank:6] norm expect=PushLimitIntoWindow SELECT *, rank() OVER (PARTITION BY w ORDER BY y) FROM wxyz ORDER BY w, y LIMIT 2 ---- sort - ├── columns: w:1!null x:2 y:3 z:4 rank:5 + ├── columns: w:1!null x:2 y:3 z:4 rank:6 ├── cardinality: [0 - 2] ├── key: (1) ├── fd: (1)-->(2-4) ├── ordering: +1 └── window partition=(1) - ├── columns: w:1!null x:2 y:3 z:4 rank:5 + ├── columns: w:1!null x:2 y:3 z:4 rank:6 ├── cardinality: [0 - 2] ├── key: (1) ├── fd: (1)-->(2-4) @@ -578,19 +578,19 @@ sort │ │ └── limit hint: 2.00 │ └── 2 └── windows - └── rank [as=rank:5] + └── rank [as=rank:6] norm expect=PushLimitIntoWindow SELECT *, rank() OVER (PARTITION BY w ORDER BY y) FROM wxyz ORDER BY w LIMIT 2 ---- sort - ├── columns: w:1!null x:2 y:3 z:4 rank:5 + ├── columns: w:1!null x:2 y:3 z:4 rank:6 ├── cardinality: [0 - 2] ├── key: (1) ├── fd: (1)-->(2-4) ├── ordering: +1 └── window partition=(1) - ├── columns: w:1!null x:2 y:3 z:4 rank:5 + ├── columns: w:1!null x:2 y:3 z:4 rank:6 ├── cardinality: [0 - 2] ├── key: (1) ├── fd: (1)-->(2-4) @@ -608,19 +608,19 @@ sort │ │ └── limit hint: 2.00 │ └── 2 └── windows - └── rank [as=rank:5] + └── rank [as=rank:6] norm expect=PushLimitIntoWindow SELECT *, rank() OVER (PARTITION BY w ORDER BY y) FROM wxyz ORDER BY y, w LIMIT 2 ---- sort - ├── columns: w:1!null x:2 y:3 z:4 rank:5 + ├── columns: w:1!null x:2 y:3 z:4 rank:6 ├── cardinality: [0 - 2] ├── key: (1) ├── fd: (1)-->(2-4) ├── ordering: +3,+1 └── window partition=(1) - ├── columns: w:1!null x:2 y:3 z:4 rank:5 + ├── columns: w:1!null x:2 y:3 z:4 rank:6 ├── cardinality: [0 - 2] ├── key: (1) ├── fd: (1)-->(2-4) @@ -642,19 +642,19 @@ sort │ │ └── fd: (1)-->(2-4) │ └── 2 └── windows - └── rank [as=rank:5] + └── rank [as=rank:6] norm expect=PushLimitIntoWindow SELECT *, rank() OVER (PARTITION BY w, z ORDER BY y) FROM wxyz ORDER BY w, z LIMIT 2 ---- sort - ├── columns: w:1!null x:2 y:3 z:4 rank:5 + ├── columns: w:1!null x:2 y:3 z:4 rank:6 ├── cardinality: [0 - 2] ├── key: (1) ├── fd: (1)-->(2-4) ├── ordering: +1 └── window partition=(1) - ├── columns: w:1!null x:2 y:3 z:4 rank:5 + ├── columns: w:1!null x:2 y:3 z:4 rank:6 ├── cardinality: [0 - 2] ├── key: (1) ├── fd: (1)-->(2-4) @@ -672,19 +672,19 @@ sort │ │ └── limit hint: 2.00 │ └── 2 └── windows - └── rank [as=rank:5] + └── rank [as=rank:6] norm SELECT *, rank() OVER (PARTITION BY x, z ORDER BY y) FROM wxyz ORDER BY z, x LIMIT 2 ---- sort - ├── columns: w:1!null x:2 y:3 z:4 rank:5 + ├── columns: w:1!null x:2 y:3 z:4 rank:6 ├── cardinality: [0 - 2] ├── key: (1) ├── fd: (1)-->(2-4) ├── ordering: +4,+2 └── window partition=(2,4) ordering=+3 opt(2,4) - ├── columns: w:1!null x:2 y:3 z:4 rank:5 + ├── columns: w:1!null x:2 y:3 z:4 rank:6 ├── cardinality: [0 - 2] ├── key: (1) ├── fd: (1)-->(2-4) @@ -706,19 +706,19 @@ sort │ │ └── fd: (1)-->(2-4) │ └── 2 └── windows - └── rank [as=rank:5] + └── rank [as=rank:6] norm expect=PushLimitIntoWindow SELECT *, rank() OVER (PARTITION BY z ORDER BY y) FROM wxyz ORDER BY z, y LIMIT 2 ---- sort - ├── columns: w:1!null x:2 y:3 z:4 rank:5 + ├── columns: w:1!null x:2 y:3 z:4 rank:6 ├── cardinality: [0 - 2] ├── key: (1) ├── fd: (1)-->(2-4) ├── ordering: +4,+3 └── window partition=(4) ordering=+3 opt(4) - ├── columns: w:1!null x:2 y:3 z:4 rank:5 + ├── columns: w:1!null x:2 y:3 z:4 rank:6 ├── cardinality: [0 - 2] ├── key: (1) ├── fd: (1)-->(2-4) @@ -740,26 +740,26 @@ sort │ │ └── fd: (1)-->(2-4) │ └── 2 └── windows - └── rank [as=rank:5] + └── rank [as=rank:6] norm expect-not=PushLimitIntoWindow SELECT *, rank() OVER (PARTITION BY z ORDER BY y) FROM wxyz ORDER BY y LIMIT 2 ---- limit - ├── columns: w:1!null x:2 y:3 z:4 rank:5 + ├── columns: w:1!null x:2 y:3 z:4 rank:6 ├── internal-ordering: +3 ├── cardinality: [0 - 2] ├── key: (1) ├── fd: (1)-->(2-4) ├── ordering: +3 ├── sort - │ ├── columns: w:1!null x:2 y:3 z:4 rank:5 + │ ├── columns: w:1!null x:2 y:3 z:4 rank:6 │ ├── key: (1) │ ├── fd: (1)-->(2-4) │ ├── ordering: +3 │ ├── limit hint: 2.00 │ └── window partition=(4) ordering=+3 opt(4) - │ ├── columns: w:1!null x:2 y:3 z:4 rank:5 + │ ├── columns: w:1!null x:2 y:3 z:4 rank:6 │ ├── key: (1) │ ├── fd: (1)-->(2-4) │ ├── scan wxyz @@ -767,20 +767,20 @@ limit │ │ ├── key: (1) │ │ └── fd: (1)-->(2-4) │ └── windows - │ └── rank [as=rank:5] + │ └── rank [as=rank:6] └── 2 norm expect=PushLimitIntoWindow SELECT *, rank() OVER (PARTITION BY w, z ORDER BY y) FROM wxyz ORDER BY w, z, y LIMIT 2 ---- sort - ├── columns: w:1!null x:2 y:3 z:4 rank:5 + ├── columns: w:1!null x:2 y:3 z:4 rank:6 ├── cardinality: [0 - 2] ├── key: (1) ├── fd: (1)-->(2-4) ├── ordering: +1 └── window partition=(1) - ├── columns: w:1!null x:2 y:3 z:4 rank:5 + ├── columns: w:1!null x:2 y:3 z:4 rank:6 ├── cardinality: [0 - 2] ├── key: (1) ├── fd: (1)-->(2-4) @@ -798,19 +798,19 @@ sort │ │ └── limit hint: 2.00 │ └── 2 └── windows - └── rank [as=rank:5] + └── rank [as=rank:6] norm SELECT *, rank() OVER (PARTITION BY w, z ORDER BY y) FROM wxyz ORDER BY z, w, y LIMIT 2 ---- sort - ├── columns: w:1!null x:2 y:3 z:4 rank:5 + ├── columns: w:1!null x:2 y:3 z:4 rank:6 ├── cardinality: [0 - 2] ├── key: (1) ├── fd: (1)-->(2-4) ├── ordering: +4,+1 └── window partition=(1) - ├── columns: w:1!null x:2 y:3 z:4 rank:5 + ├── columns: w:1!null x:2 y:3 z:4 rank:6 ├── cardinality: [0 - 2] ├── key: (1) ├── fd: (1)-->(2-4) @@ -832,19 +832,19 @@ sort │ │ └── fd: (1)-->(2-4) │ └── 2 └── windows - └── rank [as=rank:5] + └── rank [as=rank:6] norm SELECT *, rank() OVER (PARTITION BY w ORDER BY y) FROM wxyz ORDER BY z LIMIT 2 ---- sort - ├── columns: w:1!null x:2 y:3 z:4 rank:5 + ├── columns: w:1!null x:2 y:3 z:4 rank:6 ├── cardinality: [0 - 2] ├── key: (1) ├── fd: (1)-->(2-4) ├── ordering: +4 └── window partition=(1) - ├── columns: w:1!null x:2 y:3 z:4 rank:5 + ├── columns: w:1!null x:2 y:3 z:4 rank:6 ├── cardinality: [0 - 2] ├── key: (1) ├── fd: (1)-->(2-4) @@ -866,4 +866,4 @@ sort │ │ └── fd: (1)-->(2-4) │ └── 2 └── windows - └── rank [as=rank:5] + └── rank [as=rank:6] diff --git a/pkg/sql/opt/norm/testdata/rules/with b/pkg/sql/opt/norm/testdata/rules/with index a64b7b2655b5..7172022cdb46 100644 --- a/pkg/sql/opt/norm/testdata/rules/with +++ b/pkg/sql/opt/norm/testdata/rules/with @@ -408,60 +408,60 @@ anti-join-apply │ ├── key: (2) │ └── fd: (2)-->(3-6) ├── values - │ ├── columns: column1:8 + │ ├── columns: column1:9 │ ├── outer: (2) │ ├── cardinality: [2 - 2] │ ├── (k:2,) │ └── tuple │ └── subquery │ └── values - │ ├── columns: column1:7!null + │ ├── columns: column1:8!null │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(7) + │ ├── fd: ()-->(8) │ └── (1,) └── filters - └── column1:8 = k:2 [outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] + └── column1:9 = k:2 [outer=(2,9), constraints=(/2: (/NULL - ]; /9: (/NULL - ]), fd=(2)==(9), (9)==(2)] # Don't inline side-effecting expressions. norm WITH foo AS (INSERT INTO a VALUES (1) RETURNING *) SELECT * FROM foo ---- with &1 (foo) - ├── columns: k:11!null i:12 f:13 s:14 j:15 + ├── columns: k:12!null i:13 f:14 s:15 j:16 ├── cardinality: [1 - 1] ├── volatile, mutations ├── key: () - ├── fd: ()-->(11-15) + ├── fd: ()-->(12-16) ├── insert a │ ├── columns: a.k:1!null a.i:2 a.f:3 a.s:4 a.j:5 │ ├── insert-mapping: - │ │ ├── column1:6 => a.k:1 - │ │ ├── column7:7 => a.i:2 - │ │ ├── column8:8 => a.f:3 - │ │ ├── column9:9 => a.s:4 - │ │ └── column10:10 => a.j:5 + │ │ ├── column1:7 => a.k:1 + │ │ ├── column8:8 => a.i:2 + │ │ ├── column9:9 => a.f:3 + │ │ ├── column10:10 => a.s:4 + │ │ └── column11:11 => a.j:5 │ ├── cardinality: [1 - 1] │ ├── volatile, mutations │ ├── key: () │ ├── fd: ()-->(1-5) │ └── values - │ ├── columns: column1:6!null column7:7 column8:8 column9:9 column10:10 + │ ├── columns: column1:7!null column8:8 column9:9 column10:10 column11:11 │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(6-10) + │ ├── fd: ()-->(7-11) │ └── (1, NULL, NULL, NULL, NULL) └── with-scan &1 (foo) - ├── columns: k:11!null i:12 f:13 s:14 j:15 + ├── columns: k:12!null i:13 f:14 s:15 j:16 ├── mapping: - │ ├── a.k:1 => k:11 - │ ├── a.i:2 => i:12 - │ ├── a.f:3 => f:13 - │ ├── a.s:4 => s:14 - │ └── a.j:5 => j:15 + │ ├── a.k:1 => k:12 + │ ├── a.i:2 => i:13 + │ ├── a.f:3 => f:14 + │ ├── a.s:4 => s:15 + │ └── a.j:5 => j:16 ├── cardinality: [1 - 1] ├── key: () - └── fd: ()-->(11-15) + └── fd: ()-->(12-16) norm expect-not=InlineWith WITH foo AS (SELECT crdb_internal.notice('foo')) SELECT * FROM foo @@ -530,23 +530,23 @@ norm WITH cte AS (SELECT * FROM a) (SELECT 1 FROM cte LIMIT 9) UNION (SELECT 1 FROM cte LIMIT 10) ---- with &1 (cte) - ├── columns: "?column?":18!null + ├── columns: "?column?":19!null ├── cardinality: [0 - 19] - ├── key: (18) + ├── key: (19) ├── scan a │ ├── columns: a.k:1!null a.i:2 a.f:3 a.s:4 a.j:5 │ ├── key: (1) │ └── fd: (1)-->(2-5) └── union - ├── columns: "?column?":18!null - ├── left columns: "?column?":11 - ├── right columns: "?column?":17 + ├── columns: "?column?":19!null + ├── left columns: "?column?":12 + ├── right columns: "?column?":18 ├── cardinality: [0 - 19] - ├── key: (18) + ├── key: (19) ├── project - │ ├── columns: "?column?":11!null + │ ├── columns: "?column?":12!null │ ├── cardinality: [0 - 9] - │ ├── fd: ()-->(11) + │ ├── fd: ()-->(12) │ ├── limit │ │ ├── cardinality: [0 - 9] │ │ ├── with-scan &1 (cte) @@ -554,11 +554,11 @@ with &1 (cte) │ │ │ └── limit hint: 9.00 │ │ └── 9 │ └── projections - │ └── 1 [as="?column?":11] + │ └── 1 [as="?column?":12] └── project - ├── columns: "?column?":17!null + ├── columns: "?column?":18!null ├── cardinality: [0 - 10] - ├── fd: ()-->(17) + ├── fd: ()-->(18) ├── limit │ ├── cardinality: [0 - 10] │ ├── with-scan &1 (cte) @@ -566,7 +566,7 @@ with &1 (cte) │ │ └── limit hint: 10.00 │ └── 10 └── projections - └── 1 [as="?column?":17] + └── 1 [as="?column?":18] # Check cte-uses when used with mutations (for FK checks). exec-ddl @@ -581,17 +581,17 @@ norm format=show-all WITH cte AS (INSERT INTO child VALUES (1, 1) RETURNING c) SELECT c FROM cte UNION SELECT c+1 FROM cte ---- with &2 (cte) - ├── columns: c:10(int!null) + ├── columns: c:12(int!null) ├── cardinality: [1 - 2] ├── volatile, mutations - ├── stats: [rows=2, distinct(10)=2, null(10)=0] + ├── stats: [rows=2, distinct(12)=2, null(12)=0] ├── cost: 1037.7125 - ├── key: (10) + ├── key: (12) ├── insert t.public.child │ ├── columns: t.public.child.c:1(int!null) │ ├── insert-mapping: - │ │ ├── column1:3 => t.public.child.c:1 - │ │ └── column2:4 => t.public.child.p:2 + │ │ ├── column1:4 => t.public.child.c:1 + │ │ └── column2:5 => t.public.child.p:2 │ ├── input binding: &1 │ ├── cardinality: [1 - 1] │ ├── volatile, mutations @@ -600,92 +600,92 @@ with &2 (cte) │ ├── key: () │ ├── fd: ()-->(1) │ ├── values - │ │ ├── columns: column1:3(int!null) column2:4(int!null) + │ │ ├── columns: column1:4(int!null) column2:5(int!null) │ │ ├── cardinality: [1 - 1] │ │ ├── stats: [rows=1] │ │ ├── cost: 0.02 │ │ ├── key: () - │ │ ├── fd: ()-->(3,4) - │ │ ├── prune: (3,4) + │ │ ├── fd: ()-->(4,5) + │ │ ├── prune: (4,5) │ │ └── tuple [type=tuple{int, int}] │ │ ├── const: 1 [type=int] │ │ └── const: 1 [type=int] │ └── f-k-checks │ └── f-k-checks-item: child(p) -> parent(p) │ └── anti-join (hash) - │ ├── columns: column2:5(int!null) + │ ├── columns: column2:6(int!null) │ ├── cardinality: [0 - 1] │ ├── stats: [rows=1e-10] │ ├── cost: 1037.5725 │ ├── key: () - │ ├── fd: ()-->(5) + │ ├── fd: ()-->(6) │ ├── cte-uses - │ │ └── &1: count=1 used-columns=(4) + │ │ └── &1: count=1 used-columns=(5) │ ├── with-scan &1 - │ │ ├── columns: column2:5(int!null) + │ │ ├── columns: column2:6(int!null) │ │ ├── mapping: - │ │ │ └── column2:4(int) => column2:5(int) + │ │ │ └── column2:5(int) => column2:6(int) │ │ ├── cardinality: [1 - 1] - │ │ ├── stats: [rows=1, distinct(5)=1, null(5)=0] + │ │ ├── stats: [rows=1, distinct(6)=1, null(6)=0] │ │ ├── cost: 0.01 │ │ ├── key: () - │ │ ├── fd: ()-->(5) - │ │ ├── prune: (5) + │ │ ├── fd: ()-->(6) + │ │ ├── prune: (6) │ │ └── cte-uses - │ │ └── &1: count=1 used-columns=(4) + │ │ └── &1: count=1 used-columns=(5) │ ├── scan t.public.parent - │ │ ├── columns: t.public.parent.p:6(int!null) - │ │ ├── stats: [rows=1000, distinct(6)=1000, null(6)=0] + │ │ ├── columns: t.public.parent.p:7(int!null) + │ │ ├── stats: [rows=1000, distinct(7)=1000, null(7)=0] │ │ ├── cost: 1020.02 - │ │ ├── key: (6) - │ │ ├── prune: (6) - │ │ ├── interesting orderings: (+6) - │ │ └── unfiltered-cols: (6) + │ │ ├── key: (7) + │ │ ├── prune: (7) + │ │ ├── interesting orderings: (+7) + │ │ └── unfiltered-cols: (7,8) │ └── filters - │ └── eq [type=bool, outer=(5,6), constraints=(/5: (/NULL - ]; /6: (/NULL - ]), fd=(5)==(6), (6)==(5)] - │ ├── variable: column2:5 [type=int] - │ └── variable: t.public.parent.p:6 [type=int] + │ └── eq [type=bool, outer=(6,7), constraints=(/6: (/NULL - ]; /7: (/NULL - ]), fd=(6)==(7), (7)==(6)] + │ ├── variable: column2:6 [type=int] + │ └── variable: t.public.parent.p:7 [type=int] └── union - ├── columns: c:10(int!null) - ├── left columns: c:7(int) - ├── right columns: "?column?":9(int) + ├── columns: c:12(int!null) + ├── left columns: c:9(int) + ├── right columns: "?column?":11(int) ├── cardinality: [1 - 2] ├── immutable - ├── stats: [rows=2, distinct(10)=2, null(10)=0] + ├── stats: [rows=2, distinct(12)=2, null(12)=0] ├── cost: 0.1 - ├── key: (10) + ├── key: (12) ├── with-scan &2 (cte) - │ ├── columns: c:7(int!null) + │ ├── columns: c:9(int!null) │ ├── mapping: - │ │ └── t.public.child.c:1(int) => c:7(int) + │ │ └── t.public.child.c:1(int) => c:9(int) │ ├── cardinality: [1 - 1] - │ ├── stats: [rows=1, distinct(7)=1, null(7)=0] + │ ├── stats: [rows=1, distinct(9)=1, null(9)=0] │ ├── cost: 0.01 │ ├── key: () - │ ├── fd: ()-->(7) - │ └── prune: (7) + │ ├── fd: ()-->(9) + │ └── prune: (9) └── project - ├── columns: "?column?":9(int!null) + ├── columns: "?column?":11(int!null) ├── cardinality: [1 - 1] ├── immutable - ├── stats: [rows=1, distinct(9)=1, null(9)=0] + ├── stats: [rows=1, distinct(11)=1, null(11)=0] ├── cost: 0.04 ├── key: () - ├── fd: ()-->(9) - ├── prune: (9) + ├── fd: ()-->(11) + ├── prune: (11) ├── with-scan &2 (cte) - │ ├── columns: c:8(int!null) + │ ├── columns: c:10(int!null) │ ├── mapping: - │ │ └── t.public.child.c:1(int) => c:8(int) + │ │ └── t.public.child.c:1(int) => c:10(int) │ ├── cardinality: [1 - 1] - │ ├── stats: [rows=1, distinct(8)=1, null(8)=0] + │ ├── stats: [rows=1, distinct(10)=1, null(10)=0] │ ├── cost: 0.01 │ ├── key: () - │ ├── fd: ()-->(8) - │ └── prune: (8) + │ ├── fd: ()-->(10) + │ └── prune: (10) └── projections - └── plus [as="?column?":9, type=int, outer=(8), immutable] - ├── variable: c:8 [type=int] + └── plus [as="?column?":11, type=int, outer=(10), immutable] + ├── variable: c:10 [type=int] └── const: 1 [type=int] # Original CTE is inlined, adding "NOT MATERIALIZED" should not change the behavior. diff --git a/pkg/sql/opt/norm/testdata/rules/zero_cardinality b/pkg/sql/opt/norm/testdata/rules/zero_cardinality index f1b71c858134..c54592e8f0b5 100644 --- a/pkg/sql/opt/norm/testdata/rules/zero_cardinality +++ b/pkg/sql/opt/norm/testdata/rules/zero_cardinality @@ -28,10 +28,10 @@ norm expect=SimplifyZeroCardinalityGroup SELECT * FROM b INNER JOIN b b2 ON False ---- values - ├── columns: k:1!null i:2!null f:3!null s:4!null j:5!null k:6!null i:7!null f:8!null s:9!null j:10!null + ├── columns: k:1!null i:2!null f:3!null s:4!null j:5!null k:7!null i:8!null f:9!null s:10!null j:11!null ├── cardinality: [0 - 0] ├── key: () - └── fd: ()-->(1-10) + └── fd: ()-->(1-5,7-11) norm expect=SimplifyZeroCardinalityGroup SELECT * FROM b LIMIT 0 @@ -73,15 +73,15 @@ norm SELECT * FROM (SELECT CASE WHEN k < 0 THEN 3 / 0 ELSE 3 END FROM b) WHERE false ---- project - ├── columns: case:6!null + ├── columns: case:7!null ├── cardinality: [0 - 0] ├── immutable ├── key: () - ├── fd: ()-->(6) + ├── fd: ()-->(7) ├── values │ ├── columns: k:1!null │ ├── cardinality: [0 - 0] │ ├── key: () │ └── fd: ()-->(1) └── projections - └── CASE WHEN k:1 < 0 THEN 3 / 0 ELSE 3 END [as=case:6, outer=(1), immutable] + └── CASE WHEN k:1 < 0 THEN 3 / 0 ELSE 3 END [as=case:7, outer=(1), immutable] diff --git a/pkg/sql/opt/optbuilder/fk_cascade.go b/pkg/sql/opt/optbuilder/fk_cascade.go index 001a2c553744..e4f581329445 100644 --- a/pkg/sql/opt/optbuilder/fk_cascade.go +++ b/pkg/sql/opt/optbuilder/fk_cascade.go @@ -113,7 +113,10 @@ func (cb *onDeleteCascadeBuilder) Build( // Set list of columns that will be fetched by the input expression. for i := range mb.outScope.cols { - mb.fetchColIDs[i] = mb.outScope.cols[i].id + // Ensure that we don't add system columns to the fetch columns. + if !mb.outScope.cols[i].system { + mb.fetchColIDs[i] = mb.outScope.cols[i].id + } } mb.buildDelete(nil /* returning */) return mb.outScope.expr, nil @@ -223,7 +226,10 @@ func (cb *onDeleteSetBuilder) Build( // Set list of columns that will be fetched by the input expression. for i := range mb.outScope.cols { - mb.fetchColIDs[i] = mb.outScope.cols[i].id + // Ensure that we don't add system columns to the fetch columns. + if !mb.outScope.cols[i].system { + mb.fetchColIDs[i] = mb.outScope.cols[i].id + } } // Add target columns. numFKCols := fk.ColumnCount() @@ -445,7 +451,10 @@ func (cb *onUpdateCascadeBuilder) Build( // Set list of columns that will be fetched by the input expression. for i := range tableScopeCols { - mb.fetchColIDs[i] = tableScopeCols[i].id + // Ensure that we don't add system columns to the fetch columns. + if !tableScopeCols[i].system { + mb.fetchColIDs[i] = tableScopeCols[i].id + } } // Add target columns. for i := 0; i < numFKCols; i++ { diff --git a/pkg/sql/opt/optbuilder/insert.go b/pkg/sql/opt/optbuilder/insert.go index f7f46977218b..1de1d7931bb3 100644 --- a/pkg/sql/opt/optbuilder/insert.go +++ b/pkg/sql/opt/optbuilder/insert.go @@ -371,6 +371,10 @@ func (mb *mutationBuilder) needExistingRows() bool { // #1: Don't consider key columns. continue } + if cat.IsSystemColumn(mb.tab, i) { + // #2: Don't consider system columns. + continue + } insertColID := mb.insertColIDs[i] if insertColID == 0 { // #2: Non-key column does not have insert value specified. @@ -513,8 +517,8 @@ func (mb *mutationBuilder) addTargetTableColsForInsert(maxCols int) { // the SQL user. numCols := 0 for i, n := 0, mb.tab.ColumnCount(); i < n && numCols < maxCols; i++ { - // Skip mutation or hidden columns. - if cat.IsMutationColumn(mb.tab, i) || mb.tab.Column(i).IsHidden() { + // Skip mutation, hidden or system columns. + if mb.tab.ColumnKind(i) != cat.Ordinary || mb.tab.Column(i).IsHidden() { continue } @@ -561,7 +565,7 @@ func (mb *mutationBuilder) buildInputForInsert(inScope *scope, inputRows *tree.S } else { desiredTypes = make([]*types.T, 0, mb.tab.ColumnCount()) for i, n := 0, mb.tab.ColumnCount(); i < n; i++ { - if !cat.IsMutationColumn(mb.tab, i) { + if !cat.IsMutationColumn(mb.tab, i) && !cat.IsSystemColumn(mb.tab, i) { tabCol := mb.tab.Column(i) if !tabCol.IsHidden() { desiredTypes = append(desiredTypes, tabCol.DatumType()) @@ -823,7 +827,10 @@ func (mb *mutationBuilder) buildInputForUpsert( // Set fetchColIDs to reference the columns created for the fetch values. for i := range fetchScope.cols { - mb.fetchColIDs[i] = fetchScope.cols[i].id + // Ensure that we don't add system columns to the fetch columns. + if !fetchScope.cols[i].system { + mb.fetchColIDs[i] = fetchScope.cols[i].id + } } // Add the fetch columns to the current scope. It's OK to modify the current @@ -910,9 +917,9 @@ func (mb *mutationBuilder) setUpsertCols(insertCols tree.NameList) { copy(mb.updateColIDs, mb.insertColIDs) } - // Never update mutation columns. + // Never update mutation or system columns. for i, n := 0, mb.tab.ColumnCount(); i < n; i++ { - if cat.IsMutationColumn(mb.tab, i) { + if cat.IsMutationColumn(mb.tab, i) || cat.IsSystemColumn(mb.tab, i) { mb.updateColIDs[i] = 0 } } @@ -987,6 +994,11 @@ func (mb *mutationBuilder) projectUpsertColumns() { continue } + // Skip system columns. + if cat.IsSystemColumn(mb.tab, i) { + continue + } + // Generate CASE that toggles between insert and update column. caseExpr := mb.b.factory.ConstructCase( memo.TrueSingleton, @@ -1050,15 +1062,15 @@ func (mb *mutationBuilder) ensureUniqueConflictCols(conflictOrds util.FastIntSet } // mapPublicColumnNamesToOrdinals returns the set of ordinal positions within -// the target table that correspond to the given names. Mutation columns are -// ignored. +// the target table that correspond to the given names. Mutation and system +// columns are ignored. func (mb *mutationBuilder) mapPublicColumnNamesToOrdinals(names tree.NameList) util.FastIntSet { var ords util.FastIntSet for _, name := range names { found := false for i, n := 0, mb.tab.ColumnCount(); i < n; i++ { tabCol := mb.tab.Column(i) - if tabCol.ColName() == name && !cat.IsMutationColumn(mb.tab, i) { + if tabCol.ColName() == name && !cat.IsMutationColumn(mb.tab, i) && !cat.IsSystemColumn(mb.tab, i) { ords.Add(i) found = true break diff --git a/pkg/sql/opt/optbuilder/mutation_builder.go b/pkg/sql/opt/optbuilder/mutation_builder.go index 183b9594172d..bb624fd56a16 100644 --- a/pkg/sql/opt/optbuilder/mutation_builder.go +++ b/pkg/sql/opt/optbuilder/mutation_builder.go @@ -241,7 +241,10 @@ func (mb *mutationBuilder) buildInputForUpdate( // Set list of columns that will be fetched by the input expression. for i := range mb.outScope.cols { - mb.fetchColIDs[i] = mb.outScope.cols[i].id + // Ensure that we don't add system columns to the fetch columns. + if !mb.outScope.cols[i].system { + mb.fetchColIDs[i] = mb.outScope.cols[i].id + } } // If there is a FROM clause present, we must join all the tables @@ -365,7 +368,10 @@ func (mb *mutationBuilder) buildInputForDelete( // Set list of columns that will be fetched by the input expression. for i := range mb.outScope.cols { - mb.fetchColIDs[i] = mb.outScope.cols[i].id + // Ensure that we don't add system columns to the fetch columns. + if !mb.outScope.cols[i].system { + mb.fetchColIDs[i] = mb.outScope.cols[i].id + } } } @@ -376,6 +382,10 @@ func (mb *mutationBuilder) addTargetColsByName(names tree.NameList) { // Determine the ordinal position of the named column in the table and // add it as a target column. if ord := findPublicTableColumnByName(mb.tab, name); ord != -1 { + // System columns are invalid target columns. + if cat.IsSystemColumn(mb.tab, ord) { + panic(pgerror.Newf(pgcode.InvalidColumnReference, "cannot modify system column %q", name)) + } mb.addTargetCol(ord) continue } @@ -532,6 +542,11 @@ func (mb *mutationBuilder) addSynthesizedCols(colIDs opt.ColList, addCol func(co continue } + // Skip system columns. + if cat.IsSystemColumn(mb.tab, i) { + continue + } + // Invoke addCol to determine whether column should be added. if !addCol(i) { continue @@ -818,8 +833,8 @@ func (mb *mutationBuilder) makeMutationPrivate(needResults bool) *memo.MutationP if needResults { private.ReturnCols = make(opt.ColList, mb.tab.ColumnCount()) for i, n := 0, mb.tab.ColumnCount(); i < n; i++ { - if cat.IsMutationColumn(mb.tab, i) { - // Only non-mutation columns are output columns. + if mb.tab.ColumnKind(i) != cat.Ordinary { + // Only non-mutation and non-system columns are output columns. continue } retColID := mb.mapToReturnColID(i) @@ -891,7 +906,7 @@ func (mb *mutationBuilder) buildReturning(returning tree.ReturningExprs) { // inScope := mb.outScope.replace() inScope.expr = mb.outScope.expr - inScope.appendColumnsFromTable(mb.md.TableMeta(mb.tabID), &mb.alias) + inScope.appendOrdinaryColumnsFromTable(mb.md.TableMeta(mb.tabID), &mb.alias) // extraAccessibleCols contains all the columns that the RETURNING // clause can refer to in addition to the table columns. This is useful for diff --git a/pkg/sql/opt/optbuilder/scope.go b/pkg/sql/opt/optbuilder/scope.go index 70bdde2feaa8..be5c703e27ae 100644 --- a/pkg/sql/opt/optbuilder/scope.go +++ b/pkg/sql/opt/optbuilder/scope.go @@ -219,15 +219,15 @@ func (s *scope) appendColumnsFromScope(src *scope) { } } -// appendColumnsFromTable adds all non-mutation columns from the given table -// metadata to this scope. -func (s *scope) appendColumnsFromTable(tabMeta *opt.TableMeta, alias *tree.TableName) { +// appendOrdinaryColumnsFromTable adds all non-mutation and non-system columns from the +// given table metadata to this scope. +func (s *scope) appendOrdinaryColumnsFromTable(tabMeta *opt.TableMeta, alias *tree.TableName) { tab := tabMeta.Table if s.cols == nil { s.cols = make([]scopeColumn, 0, tab.ColumnCount()) } for i, n := 0, tab.ColumnCount(); i < n; i++ { - if cat.IsMutationColumn(tab, i) { + if tab.ColumnKind(i) != cat.Ordinary { continue } tabCol := tab.Column(i) diff --git a/pkg/sql/opt/optbuilder/scope_column.go b/pkg/sql/opt/optbuilder/scope_column.go index b69193dac580..b48d251f703a 100644 --- a/pkg/sql/opt/optbuilder/scope_column.go +++ b/pkg/sql/opt/optbuilder/scope_column.go @@ -43,6 +43,10 @@ type scopeColumn struct { // to the table. It should not be visible to variable references. mutation bool + // system is true if the column is an implicit system column. It should not + // be included in mutations. + system bool + // descending indicates whether this column is sorted in descending order. // This field is only used for ordering columns. descending bool diff --git a/pkg/sql/opt/optbuilder/select.go b/pkg/sql/opt/optbuilder/select.go index b57b8f4ea822..3ba36ffd9b95 100644 --- a/pkg/sql/opt/optbuilder/select.go +++ b/pkg/sql/opt/optbuilder/select.go @@ -457,6 +457,7 @@ func (b *Builder) buildScan( typ: col.DatumType(), hidden: col.IsHidden() || isMutation, mutation: isMutation, + system: cat.IsSystemColumn(tab, ord), }) } @@ -574,7 +575,7 @@ func (b *Builder) addCheckConstraintsForTable(tabMeta *opt.TableMeta) { // Create a scope that can be used for building the scalar expressions. tableScope := b.allocScope() - tableScope.appendColumnsFromTable(tabMeta, &tabMeta.Alias) + tableScope.appendOrdinaryColumnsFromTable(tabMeta, &tabMeta.Alias) // Find the non-nullable table columns. Mutation columns can be NULL during // backfill, so they should be excluded. @@ -645,7 +646,7 @@ func (b *Builder) addComputedColsForTable(tabMeta *opt.TableMeta) { if tableScope == nil { tableScope = b.allocScope() - tableScope.appendColumnsFromTable(tabMeta, &tabMeta.Alias) + tableScope.appendOrdinaryColumnsFromTable(tabMeta, &tabMeta.Alias) } if texpr := tableScope.resolveAndRequireType(expr, types.Any); texpr != nil { @@ -691,7 +692,7 @@ func (b *Builder) addPartialIndexPredicatesForTable(tabMeta *opt.TableMeta) { // Create a scope that can be used for building the scalar expressions. tableScope := b.allocScope() - tableScope.appendColumnsFromTable(tabMeta, &tabMeta.Alias) + tableScope.appendOrdinaryColumnsFromTable(tabMeta, &tabMeta.Alias) // Skip to the first partial index we found above. for ; indexOrd < numIndexes; indexOrd++ { diff --git a/pkg/sql/opt/optbuilder/testdata/aggregate b/pkg/sql/opt/optbuilder/testdata/aggregate index f61747bdd773..fc0c21d55792 100644 --- a/pkg/sql/opt/optbuilder/testdata/aggregate +++ b/pkg/sql/opt/optbuilder/testdata/aggregate @@ -24,75 +24,75 @@ SELECT min(1), max(1), count(1), sum_int(1), avg(1), sum(1), stddev(1), variance(1), bool_and(true), bool_or(false), xor_agg(b'\x01') FROM kv ---- scalar-group-by - ├── columns: min:6 max:7 count:8!null sum_int:9 avg:10 sum:11 stddev:12 variance:13 bool_and:15 bool_or:17 xor_agg:19 + ├── columns: min:7 max:8 count:9!null sum_int:10 avg:11 sum:12 stddev:13 variance:14 bool_and:16 bool_or:18 xor_agg:20 ├── project - │ ├── columns: column5:5!null column14:14!null column16:16!null column18:18!null + │ ├── columns: column6:6!null column15:15!null column17:17!null column19:19!null │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── projections - │ ├── 1 [as=column5:5] - │ ├── true [as=column14:14] - │ ├── false [as=column16:16] - │ └── '\x01' [as=column18:18] + │ ├── 1 [as=column6:6] + │ ├── true [as=column15:15] + │ ├── false [as=column17:17] + │ └── '\x01' [as=column19:19] └── aggregations - ├── min [as=min:6] - │ └── column5:5 - ├── max [as=max:7] - │ └── column5:5 - ├── count [as=count:8] - │ └── column5:5 - ├── sum-int [as=sum_int:9] - │ └── column5:5 - ├── avg [as=avg:10] - │ └── column5:5 - ├── sum [as=sum:11] - │ └── column5:5 - ├── std-dev [as=stddev:12] - │ └── column5:5 - ├── variance [as=variance:13] - │ └── column5:5 - ├── bool-and [as=bool_and:15] - │ └── column14:14 - ├── bool-or [as=bool_or:17] - │ └── column16:16 - └── xor-agg [as=xor_agg:19] - └── column18:18 + ├── min [as=min:7] + │ └── column6:6 + ├── max [as=max:8] + │ └── column6:6 + ├── count [as=count:9] + │ └── column6:6 + ├── sum-int [as=sum_int:10] + │ └── column6:6 + ├── avg [as=avg:11] + │ └── column6:6 + ├── sum [as=sum:12] + │ └── column6:6 + ├── std-dev [as=stddev:13] + │ └── column6:6 + ├── variance [as=variance:14] + │ └── column6:6 + ├── bool-and [as=bool_and:16] + │ └── column15:15 + ├── bool-or [as=bool_or:18] + │ └── column17:17 + └── xor-agg [as=xor_agg:20] + └── column19:19 build SELECT min(v), max(v), count(v), sum_int(1), avg(v), sum(v), stddev(v), variance(v), bool_and(v = 1), bool_and(v = 1), xor_agg(s::bytes) FROM kv ---- scalar-group-by - ├── columns: min:5 max:6 count:7!null sum_int:9 avg:10 sum:11 stddev:12 variance:13 bool_and:15 bool_and:15 xor_agg:17 + ├── columns: min:6 max:7 count:8!null sum_int:10 avg:11 sum:12 stddev:13 variance:14 bool_and:16 bool_and:16 xor_agg:18 ├── project - │ ├── columns: column8:8!null column14:14 column16:16 v:2 + │ ├── columns: column9:9!null column15:15 column17:17 v:2 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── projections - │ ├── 1 [as=column8:8] - │ ├── v:2 = 1 [as=column14:14] - │ └── s:4::BYTES [as=column16:16] + │ ├── 1 [as=column9:9] + │ ├── v:2 = 1 [as=column15:15] + │ └── s:4::BYTES [as=column17:17] └── aggregations - ├── min [as=min:5] + ├── min [as=min:6] │ └── v:2 - ├── max [as=max:6] + ├── max [as=max:7] │ └── v:2 - ├── count [as=count:7] + ├── count [as=count:8] │ └── v:2 - ├── sum-int [as=sum_int:9] - │ └── column8:8 - ├── avg [as=avg:10] + ├── sum-int [as=sum_int:10] + │ └── column9:9 + ├── avg [as=avg:11] │ └── v:2 - ├── sum [as=sum:11] + ├── sum [as=sum:12] │ └── v:2 - ├── std-dev [as=stddev:12] + ├── std-dev [as=stddev:13] │ └── v:2 - ├── variance [as=variance:13] + ├── variance [as=variance:14] │ └── v:2 - ├── bool-and [as=bool_and:15] - │ └── column14:14 - └── xor-agg [as=xor_agg:17] - └── column16:16 + ├── bool-and [as=bool_and:16] + │ └── column15:15 + └── xor-agg [as=xor_agg:18] + └── column17:17 build SELECT min(1, 2) @@ -147,28 +147,28 @@ build SELECT array_agg(1) FROM kv ---- scalar-group-by - ├── columns: array_agg:6 + ├── columns: array_agg:7 ├── project - │ ├── columns: column5:5!null + │ ├── columns: column6:6!null │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── projections - │ └── 1 [as=column5:5] + │ └── 1 [as=column6:6] └── aggregations - └── array-agg [as=array_agg:6] - └── column5:5 + └── array-agg [as=array_agg:7] + └── column6:6 build SELECT json_agg(v) FROM kv ---- scalar-group-by - ├── columns: json_agg:5 + ├── columns: json_agg:6 ├── project │ ├── columns: v:2 │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - └── json-agg [as=json_agg:5] + └── json-agg [as=json_agg:6] └── v:2 build @@ -191,16 +191,16 @@ build SELECT 1 r FROM kv GROUP BY v ---- project - ├── columns: r:5!null + ├── columns: r:6!null ├── group-by │ ├── columns: v:2 │ ├── grouping columns: v:2 │ └── project │ ├── columns: v:2 │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── projections - └── 1 [as=r:5] + └── 1 [as=r:6] # This should ideally return {NULL}, but this is a pathological case, and # Postgres has the same behavior, so it's sufficient for now. @@ -247,28 +247,28 @@ build SELECT (SELECT COALESCE(max(1), 0) FROM kv) ---- project - ├── columns: coalesce:8 + ├── columns: coalesce:9 ├── values │ └── () └── projections - └── subquery [as=coalesce:8] + └── subquery [as=coalesce:9] └── max1-row - ├── columns: coalesce:7 + ├── columns: coalesce:8 └── project - ├── columns: coalesce:7 + ├── columns: coalesce:8 ├── scalar-group-by - │ ├── columns: max:6 + │ ├── columns: max:7 │ ├── project - │ │ ├── columns: column5:5!null + │ │ ├── columns: column6:6!null │ │ ├── scan kv - │ │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ │ └── projections - │ │ └── 1 [as=column5:5] + │ │ └── 1 [as=column6:6] │ └── aggregations - │ └── max [as=max:6] - │ └── column5:5 + │ └── max [as=max:7] + │ └── column6:6 └── projections - └── COALESCE(max:6, 0) [as=coalesce:7] + └── COALESCE(max:7, 0) [as=coalesce:8] build SELECT count(*), k FROM kv @@ -284,28 +284,28 @@ build SELECT count(*), k FROM kv GROUP BY k ---- group-by - ├── columns: count:5!null k:1!null + ├── columns: count:6!null k:1!null ├── grouping columns: k:1!null ├── project │ ├── columns: k:1!null │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - └── count-rows [as=count_rows:5] + └── count-rows [as=count_rows:6] # GROUP BY specified using column index works. build SELECT count(*), k FROM kv GROUP BY 2 ---- group-by - ├── columns: count:5!null k:1!null + ├── columns: count:6!null k:1!null ├── grouping columns: k:1!null ├── project │ ├── columns: k:1!null │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - └── count-rows [as=count_rows:5] + └── count-rows [as=count_rows:6] build SELECT * FROM kv GROUP BY v, count(w) @@ -352,150 +352,150 @@ build SELECT count(*), kv.s FROM kv GROUP BY s ---- group-by - ├── columns: count:5!null s:4 + ├── columns: count:6!null s:4 ├── grouping columns: s:4 ├── project │ ├── columns: s:4 │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - └── count-rows [as=count_rows:5] + └── count-rows [as=count_rows:6] build SELECT count(*), s FROM kv GROUP BY kv.s ---- group-by - ├── columns: count:5!null s:4 + ├── columns: count:6!null s:4 ├── grouping columns: s:4 ├── project │ ├── columns: s:4 │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - └── count-rows [as=count_rows:5] + └── count-rows [as=count_rows:6] build SELECT count(*), kv.s FROM kv GROUP BY kv.s ---- group-by - ├── columns: count:5!null s:4 + ├── columns: count:6!null s:4 ├── grouping columns: s:4 ├── project │ ├── columns: s:4 │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - └── count-rows [as=count_rows:5] + └── count-rows [as=count_rows:6] build SELECT count(*), s FROM kv GROUP BY s ---- group-by - ├── columns: count:5!null s:4 + ├── columns: count:6!null s:4 ├── grouping columns: s:4 ├── project │ ├── columns: s:4 │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - └── count-rows [as=count_rows:5] + └── count-rows [as=count_rows:6] # Grouping by more than one column works. build SELECT v, count(*), w FROM kv GROUP BY v, w ---- group-by - ├── columns: v:2 count:5!null w:3 + ├── columns: v:2 count:6!null w:3 ├── grouping columns: v:2 w:3 ├── project │ ├── columns: v:2 w:3 │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - └── count-rows [as=count_rows:5] + └── count-rows [as=count_rows:6] # Grouping by more than one column using column numbers works. build SELECT v, count(*), w FROM kv GROUP BY 1, 3 ---- group-by - ├── columns: v:2 count:5!null w:3 + ├── columns: v:2 count:6!null w:3 ├── grouping columns: v:2 w:3 ├── project │ ├── columns: v:2 w:3 │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - └── count-rows [as=count_rows:5] + └── count-rows [as=count_rows:6] # Selecting and grouping on a function expression works. build SELECT count(*), upper(s) FROM kv GROUP BY upper(s) ---- group-by - ├── columns: count:5!null upper:6 - ├── grouping columns: column6:6 + ├── columns: count:6!null upper:7 + ├── grouping columns: column7:7 ├── project - │ ├── columns: column6:6 + │ ├── columns: column7:7 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── projections - │ └── upper(s:4) [as=column6:6] + │ └── upper(s:4) [as=column7:7] └── aggregations - └── count-rows [as=count_rows:5] + └── count-rows [as=count_rows:6] # Selecting and grouping on a constant works. build SELECT count(*) FROM kv GROUP BY 1+2 ---- project - ├── columns: count:5!null + ├── columns: count:6!null └── group-by - ├── columns: count_rows:5!null column6:6!null - ├── grouping columns: column6:6!null + ├── columns: count_rows:6!null column7:7!null + ├── grouping columns: column7:7!null ├── project - │ ├── columns: column6:6!null + │ ├── columns: column7:7!null │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── projections - │ └── 1 + 2 [as=column6:6] + │ └── 1 + 2 [as=column7:7] └── aggregations - └── count-rows [as=count_rows:5] + └── count-rows [as=count_rows:6] build SELECT count(*) FROM kv GROUP BY length('abc') ---- project - ├── columns: count:5!null + ├── columns: count:6!null └── group-by - ├── columns: count_rows:5!null column6:6 - ├── grouping columns: column6:6 + ├── columns: count_rows:6!null column7:7 + ├── grouping columns: column7:7 ├── project - │ ├── columns: column6:6 + │ ├── columns: column7:7 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── projections - │ └── length('abc') [as=column6:6] + │ └── length('abc') [as=column7:7] └── aggregations - └── count-rows [as=count_rows:5] + └── count-rows [as=count_rows:6] # Selecting a function of something which is grouped works. build SELECT count(*), upper(s) FROM kv GROUP BY s ---- project - ├── columns: count:5!null upper:6 + ├── columns: count:6!null upper:7 ├── group-by - │ ├── columns: s:4 count_rows:5!null + │ ├── columns: s:4 count_rows:6!null │ ├── grouping columns: s:4 │ ├── project │ │ ├── columns: s:4 │ │ └── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── aggregations - │ └── count-rows [as=count_rows:5] + │ └── count-rows [as=count_rows:6] └── projections - └── upper(s:4) [as=upper:6] + └── upper(s:4) [as=upper:7] # Selecting a value that is not grouped, even if a function of it it, does not work. build @@ -508,16 +508,16 @@ build SELECT count(*), k+v AS r FROM kv GROUP BY k+v ---- group-by - ├── columns: count:5!null r:6 - ├── grouping columns: column6:6 + ├── columns: count:6!null r:7 + ├── grouping columns: column7:7 ├── project - │ ├── columns: column6:6 + │ ├── columns: column7:7 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── projections - │ └── k:1 + v:2 [as=column6:6] + │ └── k:1 + v:2 [as=column7:7] └── aggregations - └── count-rows [as=count_rows:5] + └── count-rows [as=count_rows:6] # Selecting a more complex expression, made up of things which are each grouped, works. @@ -525,18 +525,18 @@ build SELECT count(*), k+v AS r FROM kv GROUP BY k, v ---- project - ├── columns: count:5!null r:6 + ├── columns: count:6!null r:7 ├── group-by - │ ├── columns: k:1!null v:2 count_rows:5!null + │ ├── columns: k:1!null v:2 count_rows:6!null │ ├── grouping columns: k:1!null v:2 │ ├── project │ │ ├── columns: k:1!null v:2 │ │ └── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── aggregations - │ └── count-rows [as=count_rows:5] + │ └── count-rows [as=count_rows:6] └── projections - └── k:1 + v:2 [as=r:6] + └── k:1 + v:2 [as=r:7] build SELECT count(*), k+v FROM kv GROUP BY v @@ -563,16 +563,16 @@ build SELECT count(kv.k) AS count_1, kv.v + kv.w AS lx FROM kv GROUP BY kv.v + kv.w ---- group-by - ├── columns: count_1:5!null lx:6 - ├── grouping columns: column6:6 + ├── columns: count_1:6!null lx:7 + ├── grouping columns: column7:7 ├── project - │ ├── columns: column6:6 k:1!null + │ ├── columns: column7:7 k:1!null │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── projections - │ └── v:2 + w:3 [as=column6:6] + │ └── v:2 + w:3 [as=column7:7] └── aggregations - └── count [as=count:5] + └── count [as=count:6] └── k:1 build @@ -589,13 +589,13 @@ build SELECT count(k) from kv ---- scalar-group-by - ├── columns: count:5!null + ├── columns: count:6!null ├── project │ ├── columns: k:1!null │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - └── count [as=count:5] + └── count [as=count:6] └── k:1 build @@ -617,16 +617,16 @@ build SELECT count(1) from kv ---- scalar-group-by - ├── columns: count:6!null + ├── columns: count:7!null ├── project - │ ├── columns: column5:5!null + │ ├── columns: column6:6!null │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── projections - │ └── 1 [as=column5:5] + │ └── 1 [as=column6:6] └── aggregations - └── count [as=count:6] - └── column5:5 + └── count [as=count:7] + └── column6:6 build SELECT count(k, v) FROM kv @@ -637,153 +637,153 @@ build SELECT v, count(k) FROM kv GROUP BY v ORDER BY v ---- sort - ├── columns: v:2 count:5!null + ├── columns: v:2 count:6!null ├── ordering: +2 └── group-by - ├── columns: v:2 count:5!null + ├── columns: v:2 count:6!null ├── grouping columns: v:2 ├── project │ ├── columns: k:1!null v:2 │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - └── count [as=count:5] + └── count [as=count:6] └── k:1 build SELECT v, count(k) FROM kv GROUP BY v ORDER BY v DESC ---- sort - ├── columns: v:2 count:5!null + ├── columns: v:2 count:6!null ├── ordering: -2 └── group-by - ├── columns: v:2 count:5!null + ├── columns: v:2 count:6!null ├── grouping columns: v:2 ├── project │ ├── columns: k:1!null v:2 │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - └── count [as=count:5] + └── count [as=count:6] └── k:1 build SELECT v, count(k) FROM kv GROUP BY v ORDER BY count(k) DESC ---- sort - ├── columns: v:2 count:5!null - ├── ordering: -5 + ├── columns: v:2 count:6!null + ├── ordering: -6 └── group-by - ├── columns: v:2 count:5!null + ├── columns: v:2 count:6!null ├── grouping columns: v:2 ├── project │ ├── columns: k:1!null v:2 │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - └── count [as=count:5] + └── count [as=count:6] └── k:1 build SELECT v, count(k) FROM kv GROUP BY v ORDER BY v-count(k) ---- sort - ├── columns: v:2 count:5!null [hidden: column6:6] - ├── ordering: +6 + ├── columns: v:2 count:6!null [hidden: column7:7] + ├── ordering: +7 └── project - ├── columns: column6:6 v:2 count:5!null + ├── columns: column7:7 v:2 count:6!null ├── group-by - │ ├── columns: v:2 count:5!null + │ ├── columns: v:2 count:6!null │ ├── grouping columns: v:2 │ ├── project │ │ ├── columns: k:1!null v:2 │ │ └── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── aggregations - │ └── count [as=count:5] + │ └── count [as=count:6] │ └── k:1 └── projections - └── v:2 - count:5 [as=column6:6] + └── v:2 - count:6 [as=column7:7] build SELECT v FROM kv GROUP BY v ORDER BY sum(k) ---- sort - ├── columns: v:2 [hidden: sum:5!null] - ├── ordering: +5 + ├── columns: v:2 [hidden: sum:6!null] + ├── ordering: +6 └── group-by - ├── columns: v:2 sum:5!null + ├── columns: v:2 sum:6!null ├── grouping columns: v:2 ├── project │ ├── columns: k:1!null v:2 │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - └── sum [as=sum:5] + └── sum [as=sum:6] └── k:1 build SELECT v, count(k) FROM kv GROUP BY v ORDER BY 1 DESC ---- sort - ├── columns: v:2 count:5!null + ├── columns: v:2 count:6!null ├── ordering: -2 └── group-by - ├── columns: v:2 count:5!null + ├── columns: v:2 count:6!null ├── grouping columns: v:2 ├── project │ ├── columns: k:1!null v:2 │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - └── count [as=count:5] + └── count [as=count:6] └── k:1 build SELECT count(*), count(k), count(kv.v) FROM kv ---- scalar-group-by - ├── columns: count:5!null count:6!null count:7!null + ├── columns: count:6!null count:7!null count:8!null ├── project │ ├── columns: k:1!null v:2 │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - ├── count-rows [as=count_rows:5] - ├── count [as=count:6] + ├── count-rows [as=count_rows:6] + ├── count [as=count:7] │ └── k:1 - └── count [as=count:7] + └── count [as=count:8] └── v:2 build SELECT count(kv.*) FROM kv ---- scalar-group-by - ├── columns: count:6!null + ├── columns: count:7!null ├── project - │ ├── columns: column5:5 + │ ├── columns: column6:6 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── projections - │ └── ((k:1, v:2, w:3, s:4) AS k, v, w, s) [as=column5:5] + │ └── ((k:1, v:2, w:3, s:4) AS k, v, w, s) [as=column6:6] └── aggregations - └── count [as=count:6] - └── column5:5 + └── count [as=count:7] + └── column6:6 build SELECT count(DISTINCT k), count(DISTINCT v), count(DISTINCT (v)) FROM kv ---- scalar-group-by - ├── columns: count:5!null count:6!null count:6!null + ├── columns: count:6!null count:7!null count:7!null ├── project │ ├── columns: k:1!null v:2 │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - ├── agg-distinct [as=count:5] + ├── agg-distinct [as=count:6] │ └── count │ └── k:1 - └── agg-distinct [as=count:6] + └── agg-distinct [as=count:7] └── count └── v:2 @@ -791,19 +791,19 @@ build SELECT upper(s), count(DISTINCT k), count(DISTINCT v), count(DISTINCT (v)) FROM kv GROUP BY upper(s) ---- group-by - ├── columns: upper:7 count:5!null count:6!null count:6!null - ├── grouping columns: column7:7 + ├── columns: upper:8 count:6!null count:7!null count:7!null + ├── grouping columns: column8:8 ├── project - │ ├── columns: column7:7 k:1!null v:2 + │ ├── columns: column8:8 k:1!null v:2 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── projections - │ └── upper(s:4) [as=column7:7] + │ └── upper(s:4) [as=column8:8] └── aggregations - ├── agg-distinct [as=count:5] + ├── agg-distinct [as=count:6] │ └── count │ └── k:1 - └── agg-distinct [as=count:6] + └── agg-distinct [as=count:7] └── count └── v:2 @@ -811,120 +811,120 @@ build SELECT count((k, v)) FROM kv ---- scalar-group-by - ├── columns: count:6!null + ├── columns: count:7!null ├── project - │ ├── columns: column5:5 + │ ├── columns: column6:6 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── projections - │ └── (k:1, v:2) [as=column5:5] + │ └── (k:1, v:2) [as=column6:6] └── aggregations - └── count [as=count:6] - └── column5:5 + └── count [as=count:7] + └── column6:6 build SELECT count(DISTINCT (k, v)) FROM kv ---- scalar-group-by - ├── columns: count:6!null + ├── columns: count:7!null ├── project - │ ├── columns: column5:5 + │ ├── columns: column6:6 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── projections - │ └── (k:1, v:2) [as=column5:5] + │ └── (k:1, v:2) [as=column6:6] └── aggregations - └── agg-distinct [as=count:6] + └── agg-distinct [as=count:7] └── count - └── column5:5 + └── column6:6 build SELECT count(DISTINCT (k, (v))) FROM kv ---- scalar-group-by - ├── columns: count:6!null + ├── columns: count:7!null ├── project - │ ├── columns: column5:5 + │ ├── columns: column6:6 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── projections - │ └── (k:1, v:2) [as=column5:5] + │ └── (k:1, v:2) [as=column6:6] └── aggregations - └── agg-distinct [as=count:6] + └── agg-distinct [as=count:7] └── count - └── column5:5 + └── column6:6 build SELECT count(*) FROM kv a, kv b ---- scalar-group-by - ├── columns: count:9!null + ├── columns: count:11!null ├── project │ └── inner-join (cross) - │ ├── columns: a.k:1!null a.v:2 a.w:3 a.s:4 b.k:5!null b.v:6 b.w:7 b.s:8 + │ ├── columns: a.k:1!null a.v:2 a.w:3 a.s:4 a.crdb_internal_mvcc_timestamp:5 b.k:6!null b.v:7 b.w:8 b.s:9 b.crdb_internal_mvcc_timestamp:10 │ ├── scan a - │ │ └── columns: a.k:1!null a.v:2 a.w:3 a.s:4 + │ │ └── columns: a.k:1!null a.v:2 a.w:3 a.s:4 a.crdb_internal_mvcc_timestamp:5 │ ├── scan b - │ │ └── columns: b.k:5!null b.v:6 b.w:7 b.s:8 + │ │ └── columns: b.k:6!null b.v:7 b.w:8 b.s:9 b.crdb_internal_mvcc_timestamp:10 │ └── filters (true) └── aggregations - └── count-rows [as=count_rows:9] + └── count-rows [as=count_rows:11] build SELECT count((k, v)) FROM kv LIMIT 1 ---- limit - ├── columns: count:6!null + ├── columns: count:7!null ├── scalar-group-by - │ ├── columns: count:6!null + │ ├── columns: count:7!null │ ├── limit hint: 1.00 │ ├── project - │ │ ├── columns: column5:5 + │ │ ├── columns: column6:6 │ │ ├── scan kv - │ │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ │ └── projections - │ │ └── (k:1, v:2) [as=column5:5] + │ │ └── (k:1, v:2) [as=column6:6] │ └── aggregations - │ └── count [as=count:6] - │ └── column5:5 + │ └── count [as=count:7] + │ └── column6:6 └── 1 build SELECT count((k, v)) FROM kv OFFSET 1 ---- offset - ├── columns: count:6!null + ├── columns: count:7!null ├── scalar-group-by - │ ├── columns: count:6!null + │ ├── columns: count:7!null │ ├── project - │ │ ├── columns: column5:5 + │ │ ├── columns: column6:6 │ │ ├── scan kv - │ │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ │ └── projections - │ │ └── (k:1, v:2) [as=column5:5] + │ │ └── (k:1, v:2) [as=column6:6] │ └── aggregations - │ └── count [as=count:6] - │ └── column5:5 + │ └── count [as=count:7] + │ └── column6:6 └── 1 build SELECT count(k)+count(kv.v) AS r FROM kv ---- project - ├── columns: r:7!null + ├── columns: r:8!null ├── scalar-group-by - │ ├── columns: count:5!null count:6!null + │ ├── columns: count:6!null count:7!null │ ├── project │ │ ├── columns: k:1!null v:2 │ │ └── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── aggregations - │ ├── count [as=count:5] + │ ├── count [as=count:6] │ │ └── k:1 - │ └── count [as=count:6] + │ └── count [as=count:7] │ └── v:2 └── projections - └── count:5 + count:6 [as=r:7] + └── count:6 + count:7 [as=r:8] build SELECT count(NULL::int), count((NULL, NULL)) @@ -948,67 +948,67 @@ build SELECT min(k), max(k), min(v), max(v) FROM kv ---- scalar-group-by - ├── columns: min:5 max:6 min:7 max:8 + ├── columns: min:6 max:7 min:8 max:9 ├── project │ ├── columns: k:1!null v:2 │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - ├── min [as=min:5] + ├── min [as=min:6] │ └── k:1 - ├── max [as=max:6] + ├── max [as=max:7] │ └── k:1 - ├── min [as=min:7] + ├── min [as=min:8] │ └── v:2 - └── max [as=max:8] + └── max [as=max:9] └── v:2 build SELECT min(k), max(k), min(v), max(v) FROM kv WHERE k > 8 ---- scalar-group-by - ├── columns: min:5 max:6 min:7 max:8 + ├── columns: min:6 max:7 min:8 max:9 ├── project │ ├── columns: k:1!null v:2 │ └── select - │ ├── columns: k:1!null v:2 w:3 s:4 + │ ├── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── filters │ └── k:1 > 8 └── aggregations - ├── min [as=min:5] + ├── min [as=min:6] │ └── k:1 - ├── max [as=max:6] + ├── max [as=max:7] │ └── k:1 - ├── min [as=min:7] + ├── min [as=min:8] │ └── v:2 - └── max [as=max:8] + └── max [as=max:9] └── v:2 build SELECT array_agg(k), array_agg(s) FROM (SELECT k, s FROM kv ORDER BY k) ---- scalar-group-by - ├── columns: array_agg:5 array_agg:6 + ├── columns: array_agg:6 array_agg:7 ├── internal-ordering: +1 ├── project │ ├── columns: k:1!null s:4 │ ├── ordering: +1 │ └── scan kv - │ ├── columns: k:1!null v:2 w:3 s:4 + │ ├── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── ordering: +1 └── aggregations - ├── array-agg [as=array_agg:5] + ├── array-agg [as=array_agg:6] │ └── k:1 - └── array-agg [as=array_agg:6] + └── array-agg [as=array_agg:7] └── s:4 build SELECT array_agg(k) FROM (SELECT k FROM kv ORDER BY s) ---- scalar-group-by - ├── columns: array_agg:5 + ├── columns: array_agg:6 ├── internal-ordering: +4 ├── sort │ ├── columns: k:1!null s:4 @@ -1016,24 +1016,24 @@ scalar-group-by │ └── project │ ├── columns: k:1!null s:4 │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - └── array-agg [as=array_agg:5] + └── array-agg [as=array_agg:6] └── k:1 build SELECT max(k) FROM (SELECT k FROM kv ORDER BY s) ---- scalar-group-by - ├── columns: max:5 + ├── columns: max:6 ├── project │ ├── columns: k:1!null │ └── project │ ├── columns: k:1!null s:4 │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - └── max [as=max:5] + └── max [as=max:6] └── k:1 @@ -1041,9 +1041,9 @@ build SELECT array_agg(k) || 1 FROM (SELECT k FROM kv ORDER BY s) ---- project - ├── columns: "?column?":6 + ├── columns: "?column?":7 ├── scalar-group-by - │ ├── columns: array_agg:5 + │ ├── columns: array_agg:6 │ ├── internal-ordering: +4 │ ├── sort │ │ ├── columns: k:1!null s:4 @@ -1051,91 +1051,91 @@ project │ │ └── project │ │ ├── columns: k:1!null s:4 │ │ └── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── aggregations - │ └── array-agg [as=array_agg:5] + │ └── array-agg [as=array_agg:6] │ └── k:1 └── projections - └── array_agg:5 || 1 [as="?column?":6] + └── array_agg:6 || 1 [as="?column?":7] build SELECT array_agg(s) FROM kv WHERE s IS NULL ---- scalar-group-by - ├── columns: array_agg:5 + ├── columns: array_agg:6 ├── project │ ├── columns: s:4 │ └── select - │ ├── columns: k:1!null v:2 w:3 s:4 + │ ├── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── filters │ └── s:4 IS NULL └── aggregations - └── array-agg [as=array_agg:5] + └── array-agg [as=array_agg:6] └── s:4 build SELECT avg(k), avg(v), sum(k), sum(v) FROM kv ---- scalar-group-by - ├── columns: avg:5 avg:6 sum:7 sum:8 + ├── columns: avg:6 avg:7 sum:8 sum:9 ├── project │ ├── columns: k:1!null v:2 │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - ├── avg [as=avg:5] - │ └── k:1 ├── avg [as=avg:6] + │ └── k:1 + ├── avg [as=avg:7] │ └── v:2 - ├── sum [as=sum:7] + ├── sum [as=sum:8] │ └── k:1 - └── sum [as=sum:8] + └── sum [as=sum:9] └── v:2 build SELECT avg(k::decimal), avg(v::decimal), sum(k::decimal), sum(v::decimal) FROM kv ---- scalar-group-by - ├── columns: avg:6 avg:8 sum:9 sum:10 + ├── columns: avg:7 avg:9 sum:10 sum:11 ├── project - │ ├── columns: column5:5!null column7:7 + │ ├── columns: column6:6!null column8:8 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── projections - │ ├── k:1::DECIMAL [as=column5:5] - │ └── v:2::DECIMAL [as=column7:7] + │ ├── k:1::DECIMAL [as=column6:6] + │ └── v:2::DECIMAL [as=column8:8] └── aggregations - ├── avg [as=avg:6] - │ └── column5:5 - ├── avg [as=avg:8] - │ └── column7:7 - ├── sum [as=sum:9] - │ └── column5:5 - └── sum [as=sum:10] - └── column7:7 + ├── avg [as=avg:7] + │ └── column6:6 + ├── avg [as=avg:9] + │ └── column8:8 + ├── sum [as=sum:10] + │ └── column6:6 + └── sum [as=sum:11] + └── column8:8 build SELECT avg(DISTINCT k), avg(DISTINCT v), sum(DISTINCT k), sum(DISTINCT v) FROM kv ---- scalar-group-by - ├── columns: avg:5 avg:6 sum:7 sum:8 + ├── columns: avg:6 avg:7 sum:8 sum:9 ├── project │ ├── columns: k:1!null v:2 │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - ├── agg-distinct [as=avg:5] + ├── agg-distinct [as=avg:6] │ └── avg │ └── k:1 - ├── agg-distinct [as=avg:6] + ├── agg-distinct [as=avg:7] │ └── avg │ └── v:2 - ├── agg-distinct [as=sum:7] + ├── agg-distinct [as=sum:8] │ └── sum │ └── k:1 - └── agg-distinct [as=sum:8] + └── agg-distinct [as=sum:9] └── sum └── v:2 @@ -1143,43 +1143,43 @@ build SELECT avg(k) * 2.0 + max(v)::DECIMAL AS r FROM kv ---- project - ├── columns: r:7 + ├── columns: r:8 ├── scalar-group-by - │ ├── columns: avg:5 max:6 + │ ├── columns: avg:6 max:7 │ ├── project │ │ ├── columns: k:1!null v:2 │ │ └── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── aggregations - │ ├── avg [as=avg:5] + │ ├── avg [as=avg:6] │ │ └── k:1 - │ └── max [as=max:6] + │ └── max [as=max:7] │ └── v:2 └── projections - └── (avg:5 * 2.0) + max:6::DECIMAL [as=r:7] + └── (avg:6 * 2.0) + max:7::DECIMAL [as=r:8] build SELECT avg(k) * 2.0 + max(v)::DECIMAL AS r FROM kv WHERE w*2 = k ---- project - ├── columns: r:7 + ├── columns: r:8 ├── scalar-group-by - │ ├── columns: avg:5 max:6 + │ ├── columns: avg:6 max:7 │ ├── project │ │ ├── columns: k:1!null v:2 │ │ └── select - │ │ ├── columns: k:1!null v:2 w:3 s:4 + │ │ ├── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ │ ├── scan kv - │ │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ │ └── filters │ │ └── (w:3 * 2) = k:1 │ └── aggregations - │ ├── avg [as=avg:5] + │ ├── avg [as=avg:6] │ │ └── k:1 - │ └── max [as=max:6] + │ └── max [as=max:7] │ └── v:2 └── projections - └── (avg:5 * 2.0) + max:6::DECIMAL [as=r:7] + └── (avg:6 * 2.0) + max:7::DECIMAL [as=r:8] exec-ddl CREATE TABLE abc ( @@ -1194,53 +1194,57 @@ build SELECT min(a), min(b), min(c), min(d) FROM abc ---- scalar-group-by - ├── columns: min:5 min:6 min:7 min:8 - ├── scan abc - │ └── columns: a:1!null b:2 c:3 d:4 + ├── columns: min:6 min:7 min:8 min:9 + ├── project + │ ├── columns: a:1!null b:2 c:3 d:4 + │ └── scan abc + │ └── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 └── aggregations - ├── min [as=min:5] - │ └── a:1 ├── min [as=min:6] - │ └── b:2 + │ └── a:1 ├── min [as=min:7] + │ └── b:2 + ├── min [as=min:8] │ └── c:3 - └── min [as=min:8] + └── min [as=min:9] └── d:4 build SELECT max(a), max(b), max(c), max(d) FROM abc ---- scalar-group-by - ├── columns: max:5 max:6 max:7 max:8 - ├── scan abc - │ └── columns: a:1!null b:2 c:3 d:4 + ├── columns: max:6 max:7 max:8 max:9 + ├── project + │ ├── columns: a:1!null b:2 c:3 d:4 + │ └── scan abc + │ └── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 └── aggregations - ├── max [as=max:5] - │ └── a:1 ├── max [as=max:6] - │ └── b:2 + │ └── a:1 ├── max [as=max:7] + │ └── b:2 + ├── max [as=max:8] │ └── c:3 - └── max [as=max:8] + └── max [as=max:9] └── d:4 build SELECT avg(b), sum(b), avg(d), sum(d) FROM abc ---- scalar-group-by - ├── columns: avg:5 sum:6 avg:7 sum:8 + ├── columns: avg:6 sum:7 avg:8 sum:9 ├── project │ ├── columns: b:2 d:4 │ └── scan abc - │ └── columns: a:1!null b:2 c:3 d:4 + │ └── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 └── aggregations - ├── avg [as=avg:5] + ├── avg [as=avg:6] │ └── b:2 - ├── sum [as=sum:6] + ├── sum [as=sum:7] │ └── b:2 - ├── avg [as=avg:7] + ├── avg [as=avg:8] │ └── d:4 - └── sum [as=sum:8] + └── sum [as=sum:9] └── d:4 # Verify summing of intervals @@ -1254,11 +1258,13 @@ build SELECT sum(a) FROM intervals ---- scalar-group-by - ├── columns: sum:2 - ├── scan intervals - │ └── columns: a:1!null + ├── columns: sum:3 + ├── project + │ ├── columns: a:1!null + │ └── scan intervals + │ └── columns: a:1!null crdb_internal_mvcc_timestamp:2 └── aggregations - └── sum [as=sum:2] + └── sum [as=sum:3] └── a:1 build @@ -1308,111 +1314,111 @@ build SELECT min(x) FROM xyz ---- scalar-group-by - ├── columns: min:4 + ├── columns: min:5 ├── project │ ├── columns: x:1!null │ └── scan xyz - │ └── columns: x:1!null y:2 z:3 + │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 └── aggregations - └── min [as=min:4] + └── min [as=min:5] └── x:1 build SELECT min(x) FROM xyz WHERE x in (0, 4, 7) ---- scalar-group-by - ├── columns: min:4 + ├── columns: min:5 ├── project │ ├── columns: x:1!null │ └── select - │ ├── columns: x:1!null y:2 z:3 + │ ├── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 │ ├── scan xyz - │ │ └── columns: x:1!null y:2 z:3 + │ │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 │ └── filters │ └── x:1 IN (0, 4, 7) └── aggregations - └── min [as=min:4] + └── min [as=min:5] └── x:1 build SELECT max(x) FROM xyz ---- scalar-group-by - ├── columns: max:4 + ├── columns: max:5 ├── project │ ├── columns: x:1!null │ └── scan xyz - │ └── columns: x:1!null y:2 z:3 + │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 └── aggregations - └── max [as=max:4] + └── max [as=max:5] └── x:1 build SELECT max(y) FROM xyz WHERE x = 1 ---- scalar-group-by - ├── columns: max:4 + ├── columns: max:5 ├── project │ ├── columns: y:2 │ └── select - │ ├── columns: x:1!null y:2 z:3 + │ ├── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 │ ├── scan xyz - │ │ └── columns: x:1!null y:2 z:3 + │ │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 │ └── filters │ └── x:1 = 1 └── aggregations - └── max [as=max:4] + └── max [as=max:5] └── y:2 build SELECT min(y) FROM xyz WHERE x = 7 ---- scalar-group-by - ├── columns: min:4 + ├── columns: min:5 ├── project │ ├── columns: y:2 │ └── select - │ ├── columns: x:1!null y:2 z:3 + │ ├── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 │ ├── scan xyz - │ │ └── columns: x:1!null y:2 z:3 + │ │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 │ └── filters │ └── x:1 = 7 └── aggregations - └── min [as=min:4] + └── min [as=min:5] └── y:2 build SELECT min(x) FROM xyz WHERE (y, z) = (2, 3.0) ---- scalar-group-by - ├── columns: min:4 + ├── columns: min:5 ├── project │ ├── columns: x:1!null │ └── select - │ ├── columns: x:1!null y:2 z:3 + │ ├── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 │ ├── scan xyz - │ │ └── columns: x:1!null y:2 z:3 + │ │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 │ └── filters │ └── (y:2, z:3) = (2, 3.0) └── aggregations - └── min [as=min:4] + └── min [as=min:5] └── x:1 build SELECT max(x) FROM xyz WHERE (z, y) = (3.0, 2) ---- scalar-group-by - ├── columns: max:4 + ├── columns: max:5 ├── project │ ├── columns: x:1!null │ └── select - │ ├── columns: x:1!null y:2 z:3 + │ ├── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 │ ├── scan xyz - │ │ └── columns: x:1!null y:2 z:3 + │ │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 │ └── filters │ └── (z:3, y:2) = (3.0, 2) └── aggregations - └── max [as=max:4] + └── max [as=max:5] └── x:1 @@ -1422,80 +1428,80 @@ build SELECT variance(x), variance(y::decimal), round(variance(z), 14) FROM xyz ---- project - ├── columns: variance:4 variance:6 round:8 + ├── columns: variance:5 variance:7 round:9 ├── scalar-group-by - │ ├── columns: variance:4 variance:6 variance:7 + │ ├── columns: variance:5 variance:7 variance:8 │ ├── project - │ │ ├── columns: column5:5 x:1!null z:3 + │ │ ├── columns: column6:6 x:1!null z:3 │ │ ├── scan xyz - │ │ │ └── columns: x:1!null y:2 z:3 + │ │ │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 │ │ └── projections - │ │ └── y:2::DECIMAL [as=column5:5] + │ │ └── y:2::DECIMAL [as=column6:6] │ └── aggregations - │ ├── variance [as=variance:4] + │ ├── variance [as=variance:5] │ │ └── x:1 - │ ├── variance [as=variance:6] - │ │ └── column5:5 - │ └── variance [as=variance:7] + │ ├── variance [as=variance:7] + │ │ └── column6:6 + │ └── variance [as=variance:8] │ └── z:3 └── projections - └── round(variance:7, 14) [as=round:8] + └── round(variance:8, 14) [as=round:9] build SELECT variance(x) FROM xyz WHERE x = 10 ---- scalar-group-by - ├── columns: variance:4 + ├── columns: variance:5 ├── project │ ├── columns: x:1!null │ └── select - │ ├── columns: x:1!null y:2 z:3 + │ ├── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 │ ├── scan xyz - │ │ └── columns: x:1!null y:2 z:3 + │ │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 │ └── filters │ └── x:1 = 10 └── aggregations - └── variance [as=variance:4] + └── variance [as=variance:5] └── x:1 build SELECT stddev(x), stddev(y::decimal), round(stddev(z), 14) FROM xyz ---- project - ├── columns: stddev:4 stddev:6 round:8 + ├── columns: stddev:5 stddev:7 round:9 ├── scalar-group-by - │ ├── columns: stddev:4 stddev:6 stddev:7 + │ ├── columns: stddev:5 stddev:7 stddev:8 │ ├── project - │ │ ├── columns: column5:5 x:1!null z:3 + │ │ ├── columns: column6:6 x:1!null z:3 │ │ ├── scan xyz - │ │ │ └── columns: x:1!null y:2 z:3 + │ │ │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 │ │ └── projections - │ │ └── y:2::DECIMAL [as=column5:5] + │ │ └── y:2::DECIMAL [as=column6:6] │ └── aggregations - │ ├── std-dev [as=stddev:4] + │ ├── std-dev [as=stddev:5] │ │ └── x:1 - │ ├── std-dev [as=stddev:6] - │ │ └── column5:5 - │ └── std-dev [as=stddev:7] + │ ├── std-dev [as=stddev:7] + │ │ └── column6:6 + │ └── std-dev [as=stddev:8] │ └── z:3 └── projections - └── round(stddev:7, 14) [as=round:8] + └── round(stddev:8, 14) [as=round:9] build SELECT stddev(x) FROM xyz WHERE x = 1 ---- scalar-group-by - ├── columns: stddev:4 + ├── columns: stddev:5 ├── project │ ├── columns: x:1!null │ └── select - │ ├── columns: x:1!null y:2 z:3 + │ ├── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 │ ├── scan xyz - │ │ └── columns: x:1!null y:2 z:3 + │ │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 │ └── filters │ └── x:1 = 1 └── aggregations - └── std-dev [as=stddev:4] + └── std-dev [as=stddev:5] └── x:1 build @@ -1613,58 +1619,58 @@ build SELECT x > (SELECT avg(0)) AS r FROM xyz LIMIT 1 ---- limit - ├── columns: r:6 + ├── columns: r:7 ├── project - │ ├── columns: r:6 + │ ├── columns: r:7 │ ├── limit hint: 1.00 │ ├── scan xyz - │ │ ├── columns: x:1!null y:2 z:3 + │ │ ├── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 │ │ └── limit hint: 1.00 │ └── projections - │ └── gt [as=r:6] + │ └── gt [as=r:7] │ ├── x:1 │ └── subquery │ └── max1-row - │ ├── columns: avg:5 + │ ├── columns: avg:6 │ └── scalar-group-by - │ ├── columns: avg:5 + │ ├── columns: avg:6 │ ├── project - │ │ ├── columns: column4:4!null + │ │ ├── columns: column5:5!null │ │ ├── values │ │ │ └── () │ │ └── projections - │ │ └── 0 [as=column4:4] + │ │ └── 0 [as=column5:5] │ └── aggregations - │ └── avg [as=avg:5] - │ └── column4:4 + │ └── avg [as=avg:6] + │ └── column5:5 └── 1 build SELECT x > (SELECT avg(y) FROM xyz) AS r FROM xyz LIMIT 1 ---- limit - ├── columns: r:8 + ├── columns: r:10 ├── project - │ ├── columns: r:8 + │ ├── columns: r:10 │ ├── limit hint: 1.00 │ ├── scan xyz - │ │ ├── columns: x:1!null y:2 z:3 + │ │ ├── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 │ │ └── limit hint: 1.00 │ └── projections - │ └── gt [as=r:8] + │ └── gt [as=r:10] │ ├── x:1 │ └── subquery │ └── max1-row - │ ├── columns: avg:7 + │ ├── columns: avg:9 │ └── scalar-group-by - │ ├── columns: avg:7 + │ ├── columns: avg:9 │ ├── project - │ │ ├── columns: y:5 + │ │ ├── columns: y:6 │ │ └── scan xyz - │ │ └── columns: x:4!null y:5 z:6 + │ │ └── columns: x:5!null y:6 z:7 crdb_internal_mvcc_timestamp:8 │ └── aggregations - │ └── avg [as=avg:7] - │ └── y:5 + │ └── avg [as=avg:9] + │ └── y:6 └── 1 exec-ddl @@ -1675,15 +1681,15 @@ build SELECT bool_and(b), bool_or(b) FROM bools ---- scalar-group-by - ├── columns: bool_and:3 bool_or:4 + ├── columns: bool_and:4 bool_or:5 ├── project │ ├── columns: b:1 │ └── scan bools - │ └── columns: b:1 rowid:2!null + │ └── columns: b:1 rowid:2!null crdb_internal_mvcc_timestamp:3 └── aggregations - ├── bool-and [as=bool_and:3] + ├── bool-and [as=bool_and:4] │ └── b:1 - └── bool-or [as=bool_or:4] + └── bool-or [as=bool_or:5] └── b:1 @@ -1692,14 +1698,16 @@ build SELECT 1 r FROM kv GROUP BY kv.*; ---- project - ├── columns: r:5!null + ├── columns: r:6!null ├── group-by │ ├── columns: k:1!null v:2 w:3 s:4 │ ├── grouping columns: k:1!null v:2 w:3 s:4 - │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── project + │ ├── columns: k:1!null v:2 w:3 s:4 + │ └── scan kv + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── projections - └── 1 [as=r:5] + └── 1 [as=r:6] exec-ddl CREATE TABLE xor_bytes (a bytes, b int, c int) @@ -1709,43 +1717,43 @@ build SELECT to_hex(xor_agg(a)), xor_agg(c) FROM xor_bytes ---- project - ├── columns: to_hex:7 xor_agg:6 + ├── columns: to_hex:8 xor_agg:7 ├── scalar-group-by - │ ├── columns: xor_agg:5 xor_agg:6 + │ ├── columns: xor_agg:6 xor_agg:7 │ ├── project │ │ ├── columns: a:1 c:3 │ │ └── scan xor_bytes - │ │ └── columns: a:1 b:2 c:3 rowid:4!null + │ │ └── columns: a:1 b:2 c:3 rowid:4!null crdb_internal_mvcc_timestamp:5 │ └── aggregations - │ ├── xor-agg [as=xor_agg:5] + │ ├── xor-agg [as=xor_agg:6] │ │ └── a:1 - │ └── xor-agg [as=xor_agg:6] + │ └── xor-agg [as=xor_agg:7] │ └── c:3 └── projections - └── to_hex(xor_agg:5) [as=to_hex:7] + └── to_hex(xor_agg:6) [as=to_hex:8] build SELECT to_hex(xor_agg(a)), b, xor_agg(c) FROM xor_bytes GROUP BY b ORDER BY b ---- sort - ├── columns: to_hex:7 b:2 xor_agg:6 + ├── columns: to_hex:8 b:2 xor_agg:7 ├── ordering: +2 └── project - ├── columns: to_hex:7 b:2 xor_agg:6 + ├── columns: to_hex:8 b:2 xor_agg:7 ├── group-by - │ ├── columns: b:2 xor_agg:5 xor_agg:6 + │ ├── columns: b:2 xor_agg:6 xor_agg:7 │ ├── grouping columns: b:2 │ ├── project │ │ ├── columns: a:1 b:2 c:3 │ │ └── scan xor_bytes - │ │ └── columns: a:1 b:2 c:3 rowid:4!null + │ │ └── columns: a:1 b:2 c:3 rowid:4!null crdb_internal_mvcc_timestamp:5 │ └── aggregations - │ ├── xor-agg [as=xor_agg:5] + │ ├── xor-agg [as=xor_agg:6] │ │ └── a:1 - │ └── xor-agg [as=xor_agg:6] + │ └── xor-agg [as=xor_agg:7] │ └── c:3 └── projections - └── to_hex(xor_agg:5) [as=to_hex:7] + └── to_hex(xor_agg:6) [as=to_hex:8] # At execution time, this query will cause the error: # "arguments to xor must all be the same length" @@ -1783,48 +1791,48 @@ build SELECT concat_agg(s) FROM (SELECT s FROM kv ORDER BY k) ---- scalar-group-by - ├── columns: concat_agg:5 + ├── columns: concat_agg:6 ├── internal-ordering: +1 ├── project │ ├── columns: k:1!null s:4 │ ├── ordering: +1 │ └── scan kv - │ ├── columns: k:1!null v:2 w:3 s:4 + │ ├── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── ordering: +1 └── aggregations - └── concat-agg [as=concat_agg:5] + └── concat-agg [as=concat_agg:6] └── s:4 build SELECT json_agg(s) FROM (SELECT s FROM kv ORDER BY k) ---- scalar-group-by - ├── columns: json_agg:5 + ├── columns: json_agg:6 ├── internal-ordering: +1 ├── project │ ├── columns: k:1!null s:4 │ ├── ordering: +1 │ └── scan kv - │ ├── columns: k:1!null v:2 w:3 s:4 + │ ├── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── ordering: +1 └── aggregations - └── json-agg [as=json_agg:5] + └── json-agg [as=json_agg:6] └── s:4 build SELECT jsonb_agg(s) FROM (SELECT s FROM kv ORDER BY k) ---- scalar-group-by - ├── columns: jsonb_agg:5 + ├── columns: jsonb_agg:6 ├── internal-ordering: +1 ├── project │ ├── columns: k:1!null s:4 │ ├── ordering: +1 │ └── scan kv - │ ├── columns: k:1!null v:2 w:3 s:4 + │ ├── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── ordering: +1 └── aggregations - └── jsonb-agg [as=jsonb_agg:5] + └── jsonb-agg [as=jsonb_agg:6] └── s:4 exec-ddl @@ -1845,176 +1853,178 @@ build SELECT (b, a) AS r FROM ab GROUP BY (b, a) ---- project - ├── columns: r:3 + ├── columns: r:4 ├── group-by │ ├── columns: a:1!null b:2 │ ├── grouping columns: a:1!null b:2 - │ └── scan ab - │ └── columns: a:1!null b:2 + │ └── project + │ ├── columns: a:1!null b:2 + │ └── scan ab + │ └── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 └── projections - └── (b:2, a:1) [as=r:3] + └── (b:2, a:1) [as=r:4] build SELECT min(y), (b, a) AS r FROM ab, xy GROUP BY (x, (a, b)) ---- project - ├── columns: min:6 r:7 + ├── columns: min:8 r:9 ├── group-by - │ ├── columns: a:1!null b:2 x:3 min:6 - │ ├── grouping columns: a:1!null b:2 x:3 + │ ├── columns: a:1!null b:2 x:4 min:8 + │ ├── grouping columns: a:1!null b:2 x:4 │ ├── project - │ │ ├── columns: a:1!null b:2 x:3 y:4 + │ │ ├── columns: a:1!null b:2 x:4 y:5 │ │ └── inner-join (cross) - │ │ ├── columns: a:1!null b:2 x:3 y:4 rowid:5!null + │ │ ├── columns: a:1!null b:2 ab.crdb_internal_mvcc_timestamp:3 x:4 y:5 rowid:6!null xy.crdb_internal_mvcc_timestamp:7 │ │ ├── scan ab - │ │ │ └── columns: a:1!null b:2 + │ │ │ └── columns: a:1!null b:2 ab.crdb_internal_mvcc_timestamp:3 │ │ ├── scan xy - │ │ │ └── columns: x:3 y:4 rowid:5!null + │ │ │ └── columns: x:4 y:5 rowid:6!null xy.crdb_internal_mvcc_timestamp:7 │ │ └── filters (true) │ └── aggregations - │ └── min [as=min:6] - │ └── y:4 + │ └── min [as=min:8] + │ └── y:5 └── projections - └── (b:2, a:1) [as=r:7] + └── (b:2, a:1) [as=r:9] build SELECT v, count(k) FROM kv GROUP BY v ORDER BY count(k) ---- sort - ├── columns: v:2 count:5!null - ├── ordering: +5 + ├── columns: v:2 count:6!null + ├── ordering: +6 └── group-by - ├── columns: v:2 count:5!null + ├── columns: v:2 count:6!null ├── grouping columns: v:2 ├── project │ ├── columns: k:1!null v:2 │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - └── count [as=count:5] + └── count [as=count:6] └── k:1 build SELECT v, count(*) FROM kv GROUP BY v ORDER BY count(*) ---- sort - ├── columns: v:2 count:5!null - ├── ordering: +5 + ├── columns: v:2 count:6!null + ├── ordering: +6 └── group-by - ├── columns: v:2 count_rows:5!null + ├── columns: v:2 count_rows:6!null ├── grouping columns: v:2 ├── project │ ├── columns: v:2 │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - └── count-rows [as=count_rows:5] + └── count-rows [as=count_rows:6] build SELECT v, count(1) FROM kv GROUP BY v ORDER BY count(1) ---- sort - ├── columns: v:2 count:6!null - ├── ordering: +6 + ├── columns: v:2 count:7!null + ├── ordering: +7 └── group-by - ├── columns: v:2 count:6!null + ├── columns: v:2 count:7!null ├── grouping columns: v:2 ├── project - │ ├── columns: column5:5!null v:2 + │ ├── columns: column6:6!null v:2 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── projections - │ └── 1 [as=column5:5] + │ └── 1 [as=column6:6] └── aggregations - └── count [as=count:6] - └── column5:5 + └── count [as=count:7] + └── column6:6 build SELECT (k+v)/(v+w) AS r FROM kv GROUP BY k+v, v+w; ---- project - ├── columns: r:7 + ├── columns: r:8 ├── group-by - │ ├── columns: column5:5 column6:6 - │ ├── grouping columns: column5:5 column6:6 + │ ├── columns: column6:6 column7:7 + │ ├── grouping columns: column6:6 column7:7 │ └── project - │ ├── columns: column5:5 column6:6 + │ ├── columns: column6:6 column7:7 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── projections - │ ├── k:1 + v:2 [as=column5:5] - │ └── v:2 + w:3 [as=column6:6] + │ ├── k:1 + v:2 [as=column6:6] + │ └── v:2 + w:3 [as=column7:7] └── projections - └── column5:5 / column6:6 [as=r:7] + └── column6:6 / column7:7 [as=r:8] # Check that everything still works with differently qualified names build fully-qualify-names SELECT sum(t.kv.w), t.kv.v FROM t.kv GROUP BY v, kv.k * w ---- project - ├── columns: sum:5 v:2 + ├── columns: sum:6 v:2 └── group-by - ├── columns: t.public.kv.v:2 sum:5 column6:6 - ├── grouping columns: t.public.kv.v:2 column6:6 + ├── columns: t.public.kv.v:2 sum:6 column7:7 + ├── grouping columns: t.public.kv.v:2 column7:7 ├── project - │ ├── columns: column6:6 t.public.kv.v:2 t.public.kv.w:3 + │ ├── columns: column7:7 t.public.kv.v:2 t.public.kv.w:3 │ ├── scan t.public.kv - │ │ └── columns: t.public.kv.k:1!null t.public.kv.v:2 t.public.kv.w:3 t.public.kv.s:4 + │ │ └── columns: t.public.kv.k:1!null t.public.kv.v:2 t.public.kv.w:3 t.public.kv.s:4 t.public.kv.crdb_internal_mvcc_timestamp:5 │ └── projections - │ └── t.public.kv.k:1 * t.public.kv.w:3 [as=column6:6] + │ └── t.public.kv.k:1 * t.public.kv.w:3 [as=column7:7] └── aggregations - └── sum [as=sum:5] + └── sum [as=sum:6] └── t.public.kv.w:3 build fully-qualify-names SELECT sum(t.kv.w), lower(s), t.kv.v + k * t.kv.w AS r, t.kv.v FROM t.kv GROUP BY v, lower(kv.s), kv.k * w ---- project - ├── columns: sum:5 lower:6 r:8 v:2 + ├── columns: sum:6 lower:7 r:9 v:2 ├── group-by - │ ├── columns: t.public.kv.v:2 sum:5 column6:6 column7:7 - │ ├── grouping columns: t.public.kv.v:2 column6:6 column7:7 + │ ├── columns: t.public.kv.v:2 sum:6 column7:7 column8:8 + │ ├── grouping columns: t.public.kv.v:2 column7:7 column8:8 │ ├── project - │ │ ├── columns: column6:6 column7:7 t.public.kv.v:2 t.public.kv.w:3 + │ │ ├── columns: column7:7 column8:8 t.public.kv.v:2 t.public.kv.w:3 │ │ ├── scan t.public.kv - │ │ │ └── columns: t.public.kv.k:1!null t.public.kv.v:2 t.public.kv.w:3 t.public.kv.s:4 + │ │ │ └── columns: t.public.kv.k:1!null t.public.kv.v:2 t.public.kv.w:3 t.public.kv.s:4 t.public.kv.crdb_internal_mvcc_timestamp:5 │ │ └── projections - │ │ ├── lower(t.public.kv.s:4) [as=column6:6] - │ │ └── t.public.kv.k:1 * t.public.kv.w:3 [as=column7:7] + │ │ ├── lower(t.public.kv.s:4) [as=column7:7] + │ │ └── t.public.kv.k:1 * t.public.kv.w:3 [as=column8:8] │ └── aggregations - │ └── sum [as=sum:5] + │ └── sum [as=sum:6] │ └── t.public.kv.w:3 └── projections - └── t.public.kv.v:2 + column7:7 [as=r:8] + └── t.public.kv.v:2 + column8:8 [as=r:9] # Check all the different types of scalar expressions as group by columns build SELECT b1.b AND abc.c AND b2.b AS r FROM bools b1, bools b2, abc GROUP BY b1.b AND abc.c, b2.b ---- project - ├── columns: r:10 + ├── columns: r:13 ├── group-by - │ ├── columns: b2.b:3 column9:9 - │ ├── grouping columns: b2.b:3 column9:9 + │ ├── columns: b2.b:4 column12:12 + │ ├── grouping columns: b2.b:4 column12:12 │ └── project - │ ├── columns: column9:9 b2.b:3 + │ ├── columns: column12:12 b2.b:4 │ ├── inner-join (cross) - │ │ ├── columns: b1.b:1 b1.rowid:2!null b2.b:3 b2.rowid:4!null a:5!null abc.b:6 c:7 d:8 + │ │ ├── columns: b1.b:1 b1.rowid:2!null b1.crdb_internal_mvcc_timestamp:3 b2.b:4 b2.rowid:5!null b2.crdb_internal_mvcc_timestamp:6 a:7!null abc.b:8 c:9 d:10 abc.crdb_internal_mvcc_timestamp:11 │ │ ├── scan b1 - │ │ │ └── columns: b1.b:1 b1.rowid:2!null + │ │ │ └── columns: b1.b:1 b1.rowid:2!null b1.crdb_internal_mvcc_timestamp:3 │ │ ├── inner-join (cross) - │ │ │ ├── columns: b2.b:3 b2.rowid:4!null a:5!null abc.b:6 c:7 d:8 + │ │ │ ├── columns: b2.b:4 b2.rowid:5!null b2.crdb_internal_mvcc_timestamp:6 a:7!null abc.b:8 c:9 d:10 abc.crdb_internal_mvcc_timestamp:11 │ │ │ ├── scan b2 - │ │ │ │ └── columns: b2.b:3 b2.rowid:4!null + │ │ │ │ └── columns: b2.b:4 b2.rowid:5!null b2.crdb_internal_mvcc_timestamp:6 │ │ │ ├── scan abc - │ │ │ │ └── columns: a:5!null abc.b:6 c:7 d:8 + │ │ │ │ └── columns: a:7!null abc.b:8 c:9 d:10 abc.crdb_internal_mvcc_timestamp:11 │ │ │ └── filters (true) │ │ └── filters (true) │ └── projections - │ └── b1.b:1 AND c:7 [as=column9:9] + │ └── b1.b:1 AND c:9 [as=column12:12] └── projections - └── column9:9 AND b2.b:3 [as=r:10] + └── column12:12 AND b2.b:4 [as=r:13] build SELECT b1.b AND abc.c AND abc.c FROM bools b1, bools b2, abc GROUP BY b1.b AND abc.c, b2.b @@ -2025,28 +2035,28 @@ build SELECT b1.b OR abc.c OR b2.b AS r FROM bools b1, bools b2, abc GROUP BY b1.b OR abc.c, b2.b ---- project - ├── columns: r:10 + ├── columns: r:13 ├── group-by - │ ├── columns: b2.b:3 column9:9 - │ ├── grouping columns: b2.b:3 column9:9 + │ ├── columns: b2.b:4 column12:12 + │ ├── grouping columns: b2.b:4 column12:12 │ └── project - │ ├── columns: column9:9 b2.b:3 + │ ├── columns: column12:12 b2.b:4 │ ├── inner-join (cross) - │ │ ├── columns: b1.b:1 b1.rowid:2!null b2.b:3 b2.rowid:4!null a:5!null abc.b:6 c:7 d:8 + │ │ ├── columns: b1.b:1 b1.rowid:2!null b1.crdb_internal_mvcc_timestamp:3 b2.b:4 b2.rowid:5!null b2.crdb_internal_mvcc_timestamp:6 a:7!null abc.b:8 c:9 d:10 abc.crdb_internal_mvcc_timestamp:11 │ │ ├── scan b1 - │ │ │ └── columns: b1.b:1 b1.rowid:2!null + │ │ │ └── columns: b1.b:1 b1.rowid:2!null b1.crdb_internal_mvcc_timestamp:3 │ │ ├── inner-join (cross) - │ │ │ ├── columns: b2.b:3 b2.rowid:4!null a:5!null abc.b:6 c:7 d:8 + │ │ │ ├── columns: b2.b:4 b2.rowid:5!null b2.crdb_internal_mvcc_timestamp:6 a:7!null abc.b:8 c:9 d:10 abc.crdb_internal_mvcc_timestamp:11 │ │ │ ├── scan b2 - │ │ │ │ └── columns: b2.b:3 b2.rowid:4!null + │ │ │ │ └── columns: b2.b:4 b2.rowid:5!null b2.crdb_internal_mvcc_timestamp:6 │ │ │ ├── scan abc - │ │ │ │ └── columns: a:5!null abc.b:6 c:7 d:8 + │ │ │ │ └── columns: a:7!null abc.b:8 c:9 d:10 abc.crdb_internal_mvcc_timestamp:11 │ │ │ └── filters (true) │ │ └── filters (true) │ └── projections - │ └── b1.b:1 OR c:7 [as=column9:9] + │ └── b1.b:1 OR c:9 [as=column12:12] └── projections - └── column9:9 OR b2.b:3 [as=r:10] + └── column12:12 OR b2.b:4 [as=r:13] build SELECT b1.b OR abc.c OR abc.c FROM bools b1, bools b2, abc GROUP BY b1.b OR abc.c, b2.b @@ -2057,40 +2067,40 @@ build SELECT k % w % v AS r FROM kv GROUP BY k % w, v ---- project - ├── columns: r:6 + ├── columns: r:7 ├── group-by - │ ├── columns: v:2 column5:5 - │ ├── grouping columns: v:2 column5:5 + │ ├── columns: v:2 column6:6 + │ ├── grouping columns: v:2 column6:6 │ └── project - │ ├── columns: column5:5 v:2 + │ ├── columns: column6:6 v:2 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── projections - │ └── k:1 % w:3 [as=column5:5] + │ └── k:1 % w:3 [as=column6:6] └── projections - └── column5:5 % v:2 [as=r:6] + └── column6:6 % v:2 [as=r:7] build SELECT concat(concat(s, a), a) FROM kv, abc GROUP BY concat(s, a), a ---- project - ├── columns: concat:10 + ├── columns: concat:12 ├── group-by - │ ├── columns: a:5!null column9:9 - │ ├── grouping columns: a:5!null column9:9 + │ ├── columns: a:6!null column11:11 + │ ├── grouping columns: a:6!null column11:11 │ └── project - │ ├── columns: column9:9 a:5!null + │ ├── columns: column11:11 a:6!null │ ├── inner-join (cross) - │ │ ├── columns: k:1!null v:2 w:3 s:4 a:5!null b:6 c:7 d:8 + │ │ ├── columns: k:1!null v:2 w:3 s:4 kv.crdb_internal_mvcc_timestamp:5 a:6!null b:7 c:8 d:9 abc.crdb_internal_mvcc_timestamp:10 │ │ ├── scan kv - │ │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ │ └── columns: k:1!null v:2 w:3 s:4 kv.crdb_internal_mvcc_timestamp:5 │ │ ├── scan abc - │ │ │ └── columns: a:5!null b:6 c:7 d:8 + │ │ │ └── columns: a:6!null b:7 c:8 d:9 abc.crdb_internal_mvcc_timestamp:10 │ │ └── filters (true) │ └── projections - │ └── concat(s:4, a:5) [as=column9:9] + │ └── concat(s:4, a:6) [as=column11:11] └── projections - └── concat(column9:9, a:5) [as=concat:10] + └── concat(column11:11, a:6) [as=concat:12] build SELECT concat(concat(s, a), s) FROM kv, abc GROUP BY concat(s, a), a @@ -2101,18 +2111,18 @@ build SELECT k < w AND v != 5 AS r FROM kv GROUP BY k < w, v ---- project - ├── columns: r:6 + ├── columns: r:7 ├── group-by - │ ├── columns: v:2 column5:5 - │ ├── grouping columns: v:2 column5:5 + │ ├── columns: v:2 column6:6 + │ ├── grouping columns: v:2 column6:6 │ └── project - │ ├── columns: column5:5 v:2 + │ ├── columns: column6:6 v:2 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── projections - │ └── k:1 < w:3 [as=column5:5] + │ └── k:1 < w:3 [as=column6:6] └── projections - └── column5:5 AND (v:2 != 5) [as=r:6] + └── column6:6 AND (v:2 != 5) [as=r:7] build SELECT k < w AND k < v FROM kv GROUP BY k < w, v @@ -2127,23 +2137,23 @@ build SELECT a.bar @> b.baz AND b.baz @> b.baz AS r FROM foo AS a, foo AS b GROUP BY a.bar @> b.baz, b.baz ---- project - ├── columns: r:8 + ├── columns: r:10 ├── group-by - │ ├── columns: b.baz:5 column7:7 - │ ├── grouping columns: b.baz:5 column7:7 + │ ├── columns: b.baz:6 column9:9 + │ ├── grouping columns: b.baz:6 column9:9 │ └── project - │ ├── columns: column7:7 b.baz:5 + │ ├── columns: column9:9 b.baz:6 │ ├── inner-join (cross) - │ │ ├── columns: a.bar:1 a.baz:2 a.rowid:3!null b.bar:4 b.baz:5 b.rowid:6!null + │ │ ├── columns: a.bar:1 a.baz:2 a.rowid:3!null a.crdb_internal_mvcc_timestamp:4 b.bar:5 b.baz:6 b.rowid:7!null b.crdb_internal_mvcc_timestamp:8 │ │ ├── scan a - │ │ │ └── columns: a.bar:1 a.baz:2 a.rowid:3!null + │ │ │ └── columns: a.bar:1 a.baz:2 a.rowid:3!null a.crdb_internal_mvcc_timestamp:4 │ │ ├── scan b - │ │ │ └── columns: b.bar:4 b.baz:5 b.rowid:6!null + │ │ │ └── columns: b.bar:5 b.baz:6 b.rowid:7!null b.crdb_internal_mvcc_timestamp:8 │ │ └── filters (true) │ └── projections - │ └── a.bar:1 @> b.baz:5 [as=column7:7] + │ └── a.bar:1 @> b.baz:6 [as=column9:9] └── projections - └── column7:7 AND (b.baz:5 @> b.baz:5) [as=r:8] + └── column9:9 AND (b.baz:6 @> b.baz:6) [as=r:10] build SELECT a.bar @> b.baz AND b.baz @> b.baz FROM foo AS a, foo AS b GROUP BY b.baz <@ a.bar, b.baz @@ -2154,23 +2164,23 @@ build SELECT b.baz <@ a.bar AND b.baz <@ b.baz AS r FROM foo AS a, foo AS b GROUP BY b.baz <@ a.bar, b.baz ---- project - ├── columns: r:8 + ├── columns: r:10 ├── group-by - │ ├── columns: b.baz:5 column7:7 - │ ├── grouping columns: b.baz:5 column7:7 + │ ├── columns: b.baz:6 column9:9 + │ ├── grouping columns: b.baz:6 column9:9 │ └── project - │ ├── columns: column7:7 b.baz:5 + │ ├── columns: column9:9 b.baz:6 │ ├── inner-join (cross) - │ │ ├── columns: a.bar:1 a.baz:2 a.rowid:3!null b.bar:4 b.baz:5 b.rowid:6!null + │ │ ├── columns: a.bar:1 a.baz:2 a.rowid:3!null a.crdb_internal_mvcc_timestamp:4 b.bar:5 b.baz:6 b.rowid:7!null b.crdb_internal_mvcc_timestamp:8 │ │ ├── scan a - │ │ │ └── columns: a.bar:1 a.baz:2 a.rowid:3!null + │ │ │ └── columns: a.bar:1 a.baz:2 a.rowid:3!null a.crdb_internal_mvcc_timestamp:4 │ │ ├── scan b - │ │ │ └── columns: b.bar:4 b.baz:5 b.rowid:6!null + │ │ │ └── columns: b.bar:5 b.baz:6 b.rowid:7!null b.crdb_internal_mvcc_timestamp:8 │ │ └── filters (true) │ └── projections - │ └── a.bar:1 @> b.baz:5 [as=column7:7] + │ └── a.bar:1 @> b.baz:6 [as=column9:9] └── projections - └── column7:7 AND (b.baz:5 @> b.baz:5) [as=r:8] + └── column9:9 AND (b.baz:6 @> b.baz:6) [as=r:10] exec-ddl CREATE TABLE times (t time PRIMARY KEY) @@ -2181,24 +2191,24 @@ SELECT date_trunc('second', a.t) - date_trunc('minute', b.t) AS r FROM times a, GROUP BY date_trunc('second', a.t), date_trunc('minute', b.t) ---- project - ├── columns: r:5 + ├── columns: r:7 ├── group-by - │ ├── columns: column3:3 column4:4 - │ ├── grouping columns: column3:3 column4:4 + │ ├── columns: column5:5 column6:6 + │ ├── grouping columns: column5:5 column6:6 │ └── project - │ ├── columns: column3:3 column4:4 + │ ├── columns: column5:5 column6:6 │ ├── inner-join (cross) - │ │ ├── columns: a.t:1!null b.t:2!null + │ │ ├── columns: a.t:1!null a.crdb_internal_mvcc_timestamp:2 b.t:3!null b.crdb_internal_mvcc_timestamp:4 │ │ ├── scan a - │ │ │ └── columns: a.t:1!null + │ │ │ └── columns: a.t:1!null a.crdb_internal_mvcc_timestamp:2 │ │ ├── scan b - │ │ │ └── columns: b.t:2!null + │ │ │ └── columns: b.t:3!null b.crdb_internal_mvcc_timestamp:4 │ │ └── filters (true) │ └── projections - │ ├── date_trunc('second', a.t:1) [as=column3:3] - │ └── date_trunc('minute', b.t:2) [as=column4:4] + │ ├── date_trunc('second', a.t:1) [as=column5:5] + │ └── date_trunc('minute', b.t:3) [as=column6:6] └── projections - └── column3:3 - column4:4 [as=r:5] + └── column5:5 - column6:6 [as=r:7] build SELECT date_trunc('second', a.t) - date_trunc('second', b.t) FROM times a, times b @@ -2210,14 +2220,14 @@ build SELECT NOT b AS r FROM bools GROUP BY NOT b ---- group-by - ├── columns: r:3 - ├── grouping columns: column3:3 + ├── columns: r:4 + ├── grouping columns: column4:4 └── project - ├── columns: column3:3 + ├── columns: column4:4 ├── scan bools - │ └── columns: b:1 rowid:2!null + │ └── columns: b:1 rowid:2!null crdb_internal_mvcc_timestamp:3 └── projections - └── NOT b:1 [as=column3:3] + └── NOT b:1 [as=column4:4] build SELECT b FROM bools GROUP BY NOT b @@ -2228,153 +2238,153 @@ build SELECT NOT b AS r FROM bools GROUP BY b ---- project - ├── columns: r:3 + ├── columns: r:4 ├── group-by │ ├── columns: b:1 │ ├── grouping columns: b:1 │ └── project │ ├── columns: b:1 │ └── scan bools - │ └── columns: b:1 rowid:2!null + │ └── columns: b:1 rowid:2!null crdb_internal_mvcc_timestamp:3 └── projections - └── NOT b:1 [as=r:3] + └── NOT b:1 [as=r:4] build SELECT +k * (-w) AS r FROM kv GROUP BY +k, -w ---- project - ├── columns: r:6 + ├── columns: r:7 ├── group-by - │ ├── columns: k:1!null column5:5 - │ ├── grouping columns: k:1!null column5:5 + │ ├── columns: k:1!null column6:6 + │ ├── grouping columns: k:1!null column6:6 │ └── project - │ ├── columns: column5:5 k:1!null + │ ├── columns: column6:6 k:1!null │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── projections - │ └── -w:3 [as=column5:5] + │ └── -w:3 [as=column6:6] └── projections - └── k:1 * column5:5 [as=r:6] + └── k:1 * column6:6 [as=r:7] build SELECT k * (-w) FROM kv GROUP BY +k, -w ---- project - ├── columns: "?column?":6 + ├── columns: "?column?":7 ├── group-by - │ ├── columns: k:1!null column5:5 - │ ├── grouping columns: k:1!null column5:5 + │ ├── columns: k:1!null column6:6 + │ ├── grouping columns: k:1!null column6:6 │ └── project - │ ├── columns: column5:5 k:1!null + │ ├── columns: column6:6 k:1!null │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── projections - │ └── -w:3 [as=column5:5] + │ └── -w:3 [as=column6:6] └── projections - └── k:1 * column5:5 [as="?column?":6] + └── k:1 * column6:6 [as="?column?":7] build SELECT +k * (-w) AS r FROM kv GROUP BY k, w ---- project - ├── columns: r:5 + ├── columns: r:6 ├── group-by │ ├── columns: k:1!null w:3 │ ├── grouping columns: k:1!null w:3 │ └── project │ ├── columns: k:1!null w:3 │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── projections - └── k:1 * (-w:3) [as=r:5] + └── k:1 * (-w:3) [as=r:6] build SELECT 1 + min(v*2) AS r FROM kv GROUP BY k+3 ---- project - ├── columns: r:8 + ├── columns: r:9 ├── group-by - │ ├── columns: min:6 column7:7!null - │ ├── grouping columns: column7:7!null + │ ├── columns: min:7 column8:8!null + │ ├── grouping columns: column8:8!null │ ├── project - │ │ ├── columns: column5:5 column7:7!null + │ │ ├── columns: column6:6 column8:8!null │ │ ├── scan kv - │ │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ │ └── projections - │ │ ├── v:2 * 2 [as=column5:5] - │ │ └── k:1 + 3 [as=column7:7] + │ │ ├── v:2 * 2 [as=column6:6] + │ │ └── k:1 + 3 [as=column8:8] │ └── aggregations - │ └── min [as=min:6] - │ └── column5:5 + │ └── min [as=min:7] + │ └── column6:6 └── projections - └── 1 + min:6 [as=r:8] + └── 1 + min:7 [as=r:9] build SELECT count(*) FROM kv GROUP BY k, k ---- project - ├── columns: count:5!null + ├── columns: count:6!null └── group-by - ├── columns: k:1!null count_rows:5!null + ├── columns: k:1!null count_rows:6!null ├── grouping columns: k:1!null ├── project │ ├── columns: k:1!null │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - └── count-rows [as=count_rows:5] + └── count-rows [as=count_rows:6] build SELECT count(upper(s)) FROM kv GROUP BY upper(s) ---- project - ├── columns: count:6!null + ├── columns: count:7!null └── group-by - ├── columns: column5:5 count:6!null - ├── grouping columns: column5:5 + ├── columns: column6:6 count:7!null + ├── grouping columns: column6:6 ├── project - │ ├── columns: column5:5 + │ ├── columns: column6:6 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── projections - │ └── upper(s:4) [as=column5:5] + │ └── upper(s:4) [as=column6:6] └── aggregations - └── count [as=count:6] - └── column5:5 + └── count [as=count:7] + └── column6:6 build SELECT sum(abc.d) FROM kv JOIN abc ON kv.k >= abc.d GROUP BY kv.* ---- project - ├── columns: sum:9!null + ├── columns: sum:11!null └── group-by - ├── columns: k:1!null v:2 w:3 s:4 sum:9!null + ├── columns: k:1!null v:2 w:3 s:4 sum:11!null ├── grouping columns: k:1!null v:2 w:3 s:4 ├── project - │ ├── columns: k:1!null v:2 w:3 s:4 d:8!null + │ ├── columns: k:1!null v:2 w:3 s:4 d:9!null │ └── inner-join (cross) - │ ├── columns: k:1!null v:2 w:3 s:4 a:5!null b:6 c:7 d:8!null + │ ├── columns: k:1!null v:2 w:3 s:4 kv.crdb_internal_mvcc_timestamp:5 a:6!null b:7 c:8 d:9!null abc.crdb_internal_mvcc_timestamp:10 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 kv.crdb_internal_mvcc_timestamp:5 │ ├── scan abc - │ │ └── columns: a:5!null b:6 c:7 d:8 + │ │ └── columns: a:6!null b:7 c:8 d:9 abc.crdb_internal_mvcc_timestamp:10 │ └── filters - │ └── k:1 >= d:8 + │ └── k:1 >= d:9 └── aggregations - └── sum [as=sum:9] - └── d:8 + └── sum [as=sum:11] + └── d:9 build SELECT sum(DISTINCT abc.d) FROM abc ---- scalar-group-by - ├── columns: sum:5 + ├── columns: sum:6 ├── project │ ├── columns: d:4 │ └── scan abc - │ └── columns: a:1!null b:2 c:3 d:4 + │ └── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 └── aggregations - └── agg-distinct [as=sum:5] + └── agg-distinct [as=sum:6] └── sum └── d:4 @@ -2384,18 +2394,18 @@ build SELECT sum(abc.d) FILTER (WHERE abc.d > 0) FROM abc ---- scalar-group-by - ├── columns: sum:6 + ├── columns: sum:7 ├── project - │ ├── columns: column5:5 d:4 + │ ├── columns: column6:6 d:4 │ ├── scan abc - │ │ └── columns: a:1!null b:2 c:3 d:4 + │ │ └── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 │ └── projections - │ └── d:4 > 0 [as=column5:5] + │ └── d:4 > 0 [as=column6:6] └── aggregations - └── agg-filter [as=sum:6] + └── agg-filter [as=sum:7] ├── sum │ └── d:4 - └── column5:5 + └── column6:6 # Ensure aggregates with FILTER coexist properly with non-FILTER aggregates. build @@ -2406,26 +2416,26 @@ SELECT FROM xyz ---- scalar-group-by - ├── columns: sum:5 avg:6 avg:7 + ├── columns: sum:6 avg:7 avg:8 ├── project - │ ├── columns: column4:4 x:1!null z:3 + │ ├── columns: column5:5 x:1!null z:3 │ ├── scan xyz - │ │ └── columns: x:1!null y:2 z:3 + │ │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 │ └── projections - │ └── y:2 > 0 [as=column4:4] + │ └── y:2 > 0 [as=column5:5] └── aggregations - ├── agg-filter [as=sum:5] + ├── agg-filter [as=sum:6] │ ├── sum │ │ └── x:1 - │ └── column4:4 - ├── agg-distinct [as=avg:6] + │ └── column5:5 + ├── agg-distinct [as=avg:7] │ └── avg │ └── z:3 - └── agg-filter [as=avg:7] + └── agg-filter [as=avg:8] ├── agg-distinct │ └── avg │ └── z:3 - └── column4:4 + └── column5:5 # Ensure aggregates involving FILTER are deduplicated. build @@ -2437,22 +2447,22 @@ SELECT FROM xyz ---- scalar-group-by - ├── columns: avg:4 avg:4 avg:6 avg:6 + ├── columns: avg:5 avg:5 avg:7 avg:7 ├── project - │ ├── columns: column5:5 x:1!null + │ ├── columns: column6:6 x:1!null │ ├── scan xyz - │ │ └── columns: x:1!null y:2 z:3 + │ │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 │ └── projections - │ └── y:2 > 0 [as=column5:5] + │ └── y:2 > 0 [as=column6:6] └── aggregations - ├── agg-distinct [as=avg:4] + ├── agg-distinct [as=avg:5] │ └── avg │ └── x:1 - └── agg-filter [as=avg:6] + └── agg-filter [as=avg:7] ├── agg-distinct │ └── avg │ └── x:1 - └── column5:5 + └── column6:6 build SELECT @@ -2460,40 +2470,40 @@ SELECT FROM xyz ---- scalar-group-by - ├── columns: string_agg:7 + ├── columns: string_agg:8 ├── project - │ ├── columns: column4:4!null column5:5!null column6:6 + │ ├── columns: column5:5!null column6:6!null column7:7 │ ├── scan xyz - │ │ └── columns: x:1!null y:2 z:3 + │ │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 │ └── projections - │ ├── x:1::STRING [as=column4:4] - │ ├── 'foo' [as=column5:5] - │ └── y:2 > 0 [as=column6:6] + │ ├── x:1::STRING [as=column5:5] + │ ├── 'foo' [as=column6:6] + │ └── y:2 > 0 [as=column7:7] └── aggregations - └── agg-filter [as=string_agg:7] + └── agg-filter [as=string_agg:8] ├── string-agg - │ ├── column4:4 - │ └── column5:5 - └── column6:6 + │ ├── column5:5 + │ └── column6:6 + └── column7:7 build SELECT y, count(*) FILTER (WHERE x > 5) FROM xyz GROUP BY y ---- group-by - ├── columns: y:2 count:6!null + ├── columns: y:2 count:7!null ├── grouping columns: y:2 ├── project - │ ├── columns: column4:4!null column5:5!null y:2 + │ ├── columns: column5:5!null column6:6!null y:2 │ ├── scan xyz - │ │ └── columns: x:1!null y:2 z:3 + │ │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 │ └── projections - │ ├── true [as=column4:4] - │ └── x:1 > 5 [as=column5:5] + │ ├── true [as=column5:5] + │ └── x:1 > 5 [as=column6:6] └── aggregations - └── agg-filter [as=count:6] + └── agg-filter [as=count:7] ├── count - │ └── column4:4 - └── column5:5 + │ └── column5:5 + └── column6:6 build SELECT y, count(*) FILTER (WHERE count(*) > 5) FROM xyz GROUP BY y @@ -2506,98 +2516,98 @@ build SELECT max(k) AS mk FROM kv GROUP BY v ORDER BY mk ---- sort - ├── columns: mk:5!null - ├── ordering: +5 + ├── columns: mk:6!null + ├── ordering: +6 └── project - ├── columns: max:5!null + ├── columns: max:6!null └── group-by - ├── columns: v:2 max:5!null + ├── columns: v:2 max:6!null ├── grouping columns: v:2 ├── project │ ├── columns: k:1!null v:2 │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - └── max [as=max:5] + └── max [as=max:6] └── k:1 build SELECT max(k) AS mk FROM kv GROUP BY v ORDER BY max(k) ---- sort - ├── columns: mk:5!null - ├── ordering: +5 + ├── columns: mk:6!null + ├── ordering: +6 └── project - ├── columns: max:5!null + ├── columns: max:6!null └── group-by - ├── columns: v:2 max:5!null + ├── columns: v:2 max:6!null ├── grouping columns: v:2 ├── project │ ├── columns: k:1!null v:2 │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - └── max [as=max:5] + └── max [as=max:6] └── k:1 build SELECT max(k) AS mk1, max(k) AS mk2 FROM kv GROUP BY v ORDER BY mk1 ---- sort - ├── columns: mk1:5!null mk2:5!null - ├── ordering: +5 + ├── columns: mk1:6!null mk2:6!null + ├── ordering: +6 └── project - ├── columns: max:5!null + ├── columns: max:6!null └── group-by - ├── columns: v:2 max:5!null + ├── columns: v:2 max:6!null ├── grouping columns: v:2 ├── project │ ├── columns: k:1!null v:2 │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - └── max [as=max:5] + └── max [as=max:6] └── k:1 build SELECT max(k) AS mk1, max(k) AS mk2 FROM kv GROUP BY v ORDER BY mk2 ---- sort - ├── columns: mk1:5!null mk2:5!null - ├── ordering: +5 + ├── columns: mk1:6!null mk2:6!null + ├── ordering: +6 └── project - ├── columns: max:5!null + ├── columns: max:6!null └── group-by - ├── columns: v:2 max:5!null + ├── columns: v:2 max:6!null ├── grouping columns: v:2 ├── project │ ├── columns: k:1!null v:2 │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - └── max [as=max:5] + └── max [as=max:6] └── k:1 build SELECT max(k) AS mk1, max(k)/5 AS mk2 FROM kv GROUP BY v ORDER BY mk2 ---- sort - ├── columns: mk1:5!null mk2:6!null - ├── ordering: +6 + ├── columns: mk1:6!null mk2:7!null + ├── ordering: +7 └── project - ├── columns: mk2:6!null max:5!null + ├── columns: mk2:7!null max:6!null ├── group-by - │ ├── columns: v:2 max:5!null + │ ├── columns: v:2 max:6!null │ ├── grouping columns: v:2 │ ├── project │ │ ├── columns: k:1!null v:2 │ │ └── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── aggregations - │ └── max [as=max:5] + │ └── max [as=max:6] │ └── k:1 └── projections - └── max:5 / 5 [as=mk2:6] + └── max:6 / 5 [as=mk2:7] # Grouping columns cannot be reused inside an aggregate input expression # because the aggregate input expressions and grouping expressions are @@ -2606,43 +2616,43 @@ build SELECT max((k+v)/(k-v)) AS r, (k+v)*(k-v) AS s FROM kv GROUP BY k+v, k-v ---- project - ├── columns: r:6 s:9 + ├── columns: r:7 s:10 ├── group-by - │ ├── columns: max:6 column7:7 column8:8 - │ ├── grouping columns: column7:7 column8:8 + │ ├── columns: max:7 column8:8 column9:9 + │ ├── grouping columns: column8:8 column9:9 │ ├── project - │ │ ├── columns: column5:5 column7:7 column8:8 + │ │ ├── columns: column6:6 column8:8 column9:9 │ │ ├── scan kv - │ │ │ └── columns: k:1!null v:2 w:3 kv.s:4 + │ │ │ └── columns: k:1!null v:2 w:3 kv.s:4 crdb_internal_mvcc_timestamp:5 │ │ └── projections - │ │ ├── (k:1 + v:2) / (k:1 - v:2) [as=column5:5] - │ │ ├── k:1 + v:2 [as=column7:7] - │ │ └── k:1 - v:2 [as=column8:8] + │ │ ├── (k:1 + v:2) / (k:1 - v:2) [as=column6:6] + │ │ ├── k:1 + v:2 [as=column8:8] + │ │ └── k:1 - v:2 [as=column9:9] │ └── aggregations - │ └── max [as=max:6] - │ └── column5:5 + │ └── max [as=max:7] + │ └── column6:6 └── projections - └── column7:7 * column8:8 [as=s:9] + └── column8:8 * column9:9 [as=s:10] build SELECT max((k+v)/(k-v)) AS r, (k+v)*(k-v) AS s FROM kv GROUP BY k+v, (k+v)/(k-v), (k+v)*(k-v) ---- project - ├── columns: r:6 s:8 + ├── columns: r:7 s:9 └── group-by - ├── columns: column5:5 max:6 column7:7 column8:8 - ├── grouping columns: column5:5 column7:7 column8:8 + ├── columns: column6:6 max:7 column8:8 column9:9 + ├── grouping columns: column6:6 column8:8 column9:9 ├── project - │ ├── columns: column5:5 column7:7 column8:8 + │ ├── columns: column6:6 column8:8 column9:9 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── projections - │ ├── (k:1 + v:2) / (k:1 - v:2) [as=column5:5] - │ ├── k:1 + v:2 [as=column7:7] - │ └── (k:1 + v:2) * (k:1 - v:2) [as=column8:8] + │ ├── (k:1 + v:2) / (k:1 - v:2) [as=column6:6] + │ ├── k:1 + v:2 [as=column8:8] + │ └── (k:1 + v:2) * (k:1 - v:2) [as=column9:9] └── aggregations - └── max [as=max:6] - └── column5:5 + └── max [as=max:7] + └── column6:6 # Tests for corr. @@ -2650,13 +2660,13 @@ build SELECT corr(k, v) FROM kv ---- scalar-group-by - ├── columns: corr:5 + ├── columns: corr:6 ├── project │ ├── columns: k:1!null v:2 │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - └── corr [as=corr:5] + └── corr [as=corr:6] ├── k:1 └── v:2 @@ -2674,13 +2684,13 @@ build SELECT corr(DISTINCT k, v) FROM kv ---- scalar-group-by - ├── columns: corr:5 + ├── columns: corr:6 ├── project │ ├── columns: k:1!null v:2 │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - └── agg-distinct [as=corr:5] + └── agg-distinct [as=corr:6] └── corr ├── k:1 └── v:2 @@ -2689,18 +2699,18 @@ build SELECT max(s), corr(k, v), min(s) FROM kv ---- scalar-group-by - ├── columns: max:5 corr:6 min:7 + ├── columns: max:6 corr:7 min:8 ├── project │ ├── columns: k:1!null v:2 s:4 │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - ├── max [as=max:5] + ├── max [as=max:6] │ └── s:4 - ├── corr [as=corr:6] + ├── corr [as=corr:7] │ ├── k:1 │ └── v:2 - └── min [as=min:7] + └── min [as=min:8] └── s:4 build @@ -2721,17 +2731,17 @@ build SELECT string_agg(s, 'separator') FROM kv ---- scalar-group-by - ├── columns: string_agg:6 + ├── columns: string_agg:7 ├── project - │ ├── columns: column5:5!null s:4 + │ ├── columns: column6:6!null s:4 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── projections - │ └── 'separator' [as=column5:5] + │ └── 'separator' [as=column6:6] └── aggregations - └── string-agg [as=string_agg:6] + └── string-agg [as=string_agg:7] ├── s:4 - └── column5:5 + └── column6:6 build SELECT string_agg(s) FROM kv @@ -2747,41 +2757,41 @@ build SELECT string_agg(DISTINCT s, 'separator') FROM kv ---- scalar-group-by - ├── columns: string_agg:6 + ├── columns: string_agg:7 ├── project - │ ├── columns: column5:5!null s:4 + │ ├── columns: column6:6!null s:4 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── projections - │ └── 'separator' [as=column5:5] + │ └── 'separator' [as=column6:6] └── aggregations - └── agg-distinct [as=string_agg:6] + └── agg-distinct [as=string_agg:7] └── string-agg ├── s:4 - └── column5:5 + └── column6:6 build SELECT max(s), string_agg(s, 'sep1'), string_agg(s, 'sep2'), min(s) FROM kv ---- scalar-group-by - ├── columns: max:5 string_agg:7 string_agg:9 min:10 + ├── columns: max:6 string_agg:8 string_agg:10 min:11 ├── project - │ ├── columns: column6:6!null column8:8!null s:4 + │ ├── columns: column7:7!null column9:9!null s:4 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── projections - │ ├── 'sep1' [as=column6:6] - │ └── 'sep2' [as=column8:8] + │ ├── 'sep1' [as=column7:7] + │ └── 'sep2' [as=column9:9] └── aggregations - ├── max [as=max:5] + ├── max [as=max:6] │ └── s:4 - ├── string-agg [as=string_agg:7] + ├── string-agg [as=string_agg:8] │ ├── s:4 - │ └── column6:6 - ├── string-agg [as=string_agg:9] + │ └── column7:7 + ├── string-agg [as=string_agg:10] │ ├── s:4 - │ └── column8:8 - └── min [as=min:10] + │ └── column9:9 + └── min [as=min:11] └── s:4 # The separator must be constant, but it need not be a literal - any constant (as @@ -2790,45 +2800,45 @@ build SELECT string_agg(s, 'abc' || 'xyz') FROM kv ---- scalar-group-by - ├── columns: string_agg:6 + ├── columns: string_agg:7 ├── project - │ ├── columns: column5:5!null s:4 + │ ├── columns: column6:6!null s:4 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── projections - │ └── 'abc' || 'xyz' [as=column5:5] + │ └── 'abc' || 'xyz' [as=column6:6] └── aggregations - └── string-agg [as=string_agg:6] + └── string-agg [as=string_agg:7] ├── s:4 - └── column5:5 + └── column6:6 build SELECT string_agg(s, NULL) FROM kv ---- scalar-group-by - ├── columns: string_agg:6 + ├── columns: string_agg:7 ├── project - │ ├── columns: column5:5 s:4 + │ ├── columns: column6:6 s:4 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── projections - │ └── NULL [as=column5:5] + │ └── NULL [as=column6:6] └── aggregations - └── string-agg [as=string_agg:6] + └── string-agg [as=string_agg:7] ├── s:4 - └── column5:5 + └── column6:6 build SELECT string_agg(s, s) FROM kv ---- scalar-group-by - ├── columns: string_agg:5 + ├── columns: string_agg:6 ├── project │ ├── columns: s:4 │ └── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 └── aggregations - └── string-agg [as=string_agg:5] + └── string-agg [as=string_agg:6] ├── s:4 └── s:4 @@ -2837,56 +2847,59 @@ build SELECT 123 r FROM kv ORDER BY max(v) ---- project - ├── columns: r:6!null [hidden: max:5] - ├── ordering: +5 + ├── columns: r:7!null [hidden: max:6] + ├── ordering: +6 ├── scalar-group-by - │ ├── columns: max:5 + │ ├── columns: max:6 │ ├── project │ │ ├── columns: v:2 │ │ └── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── aggregations - │ └── max [as=max:5] + │ └── max [as=max:6] │ └── v:2 └── projections - └── 123 [as=r:6] + └── 123 [as=r:7] # Check that ordering columns are projected correctly. build SELECT array_agg(y) FROM (SELECT * FROM xyz ORDER BY x+y) ---- scalar-group-by - ├── columns: array_agg:5 - ├── internal-ordering: +4 + ├── columns: array_agg:6 + ├── internal-ordering: +5 ├── sort - │ ├── columns: y:2 column4:4 - │ ├── ordering: +4 + │ ├── columns: y:2 column5:5 + │ ├── ordering: +5 │ └── project - │ ├── columns: y:2 column4:4 + │ ├── columns: y:2 column5:5 │ └── project - │ ├── columns: column4:4 x:1!null y:2 z:3 + │ ├── columns: column5:5 x:1!null y:2 z:3 │ ├── scan xyz - │ │ └── columns: x:1!null y:2 z:3 + │ │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 │ └── projections - │ └── x:1 + y:2 [as=column4:4] + │ └── x:1 + y:2 [as=column5:5] └── aggregations - └── array-agg [as=array_agg:5] + └── array-agg [as=array_agg:6] └── y:2 build SELECT array_agg(y) FROM (SELECT * FROM xyz ORDER BY x DESC) ---- scalar-group-by - ├── columns: array_agg:4 + ├── columns: array_agg:5 ├── internal-ordering: -1 ├── project │ ├── columns: x:1!null y:2 │ ├── ordering: -1 - │ └── scan xyz,rev + │ └── project │ ├── columns: x:1!null y:2 z:3 - │ └── ordering: -1 + │ ├── ordering: -1 + │ └── scan xyz,rev + │ ├── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 + │ └── ordering: -1 └── aggregations - └── array-agg [as=array_agg:4] + └── array-agg [as=array_agg:5] └── y:2 # Regression test for #30166. @@ -2924,25 +2937,25 @@ group-by ├── columns: firstcol:1(int!null) secondcol:2(int) thirdcol:2(int) ├── grouping columns: t.public.xyz.x:1(int!null) t.public.xyz.y:2(int) ├── stats: [rows=1000, distinct(1,2)=1000, null(1,2)=0] - ├── cost: 1110.05 + ├── cost: 1120.05 ├── key: (1) ├── fd: (1)-->(2) ├── interesting orderings: (+1,+2) └── project ├── columns: t.public.xyz.x:1(int!null) t.public.xyz.y:2(int) ├── stats: [rows=1000, distinct(1,2)=1000, null(1,2)=0] - ├── cost: 1070.03 + ├── cost: 1080.03 ├── key: (1) ├── fd: (1)-->(2) ├── prune: (1,2) ├── interesting orderings: (+1,+2) └── scan t.public.xyz - ├── columns: t.public.xyz.x:1(int!null) t.public.xyz.y:2(int) t.public.xyz.z:3(float) + ├── columns: t.public.xyz.x:1(int!null) t.public.xyz.y:2(int) t.public.xyz.z:3(float) t.public.xyz.crdb_internal_mvcc_timestamp:4(decimal) ├── stats: [rows=1000, distinct(1,2)=1000, null(1,2)=0] - ├── cost: 1060.02 + ├── cost: 1070.02 ├── key: (1) - ├── fd: (1)-->(2,3) - ├── prune: (1-3) + ├── fd: (1)-->(2-4) + ├── prune: (1-4) └── interesting orderings: (+1,+2) (+3,+2,+1) # Testing ordered aggregations. @@ -2954,30 +2967,30 @@ build SELECT array_agg(col1 ORDER BY col1) FROM tab ---- scalar-group-by - ├── columns: array_agg:5 + ├── columns: array_agg:6 ├── window partition=() ordering=+1 - │ ├── columns: col1:1!null col2:2!null col3:3 rowid:4!null array_agg:5 + │ ├── columns: col1:1!null col2:2!null col3:3 rowid:4!null crdb_internal_mvcc_timestamp:5 array_agg:6 │ ├── scan tab - │ │ └── columns: col1:1!null col2:2!null col3:3 rowid:4!null + │ │ └── columns: col1:1!null col2:2!null col3:3 rowid:4!null crdb_internal_mvcc_timestamp:5 │ └── windows - │ └── array-agg [as=array_agg:5, frame="range from unbounded to unbounded"] + │ └── array-agg [as=array_agg:6, frame="range from unbounded to unbounded"] │ └── col1:1 └── aggregations - └── const-agg [as=array_agg:5] - └── array_agg:5 + └── const-agg [as=array_agg:6] + └── array_agg:6 # Ignore aggregate orderings for non commutative aggregates. build SELECT count(col1 ORDER BY col2) FROM tab ---- scalar-group-by - ├── columns: count:5!null + ├── columns: count:6!null ├── project │ ├── columns: col1:1!null col2:2!null │ └── scan tab - │ └── columns: col1:1!null col2:2!null col3:3 rowid:4!null + │ └── columns: col1:1!null col2:2!null col3:3 rowid:4!null crdb_internal_mvcc_timestamp:5 └── aggregations - └── count [as=count:5] + └── count [as=count:6] └── col1:1 # Multiple ordered aggregations. @@ -2985,137 +2998,137 @@ build SELECT array_agg(col1 ORDER BY col1), array_agg(col1 ORDER BY col2) from tab ---- scalar-group-by - ├── columns: array_agg:5 array_agg:6 + ├── columns: array_agg:6 array_agg:7 ├── window partition=() ordering=+2 - │ ├── columns: col1:1!null col2:2!null col3:3 rowid:4!null array_agg:5 array_agg:6 + │ ├── columns: col1:1!null col2:2!null col3:3 rowid:4!null crdb_internal_mvcc_timestamp:5 array_agg:6 array_agg:7 │ ├── window partition=() ordering=+1 - │ │ ├── columns: col1:1!null col2:2!null col3:3 rowid:4!null array_agg:5 + │ │ ├── columns: col1:1!null col2:2!null col3:3 rowid:4!null crdb_internal_mvcc_timestamp:5 array_agg:6 │ │ ├── scan tab - │ │ │ └── columns: col1:1!null col2:2!null col3:3 rowid:4!null + │ │ │ └── columns: col1:1!null col2:2!null col3:3 rowid:4!null crdb_internal_mvcc_timestamp:5 │ │ └── windows - │ │ └── array-agg [as=array_agg:5, frame="range from unbounded to unbounded"] + │ │ └── array-agg [as=array_agg:6, frame="range from unbounded to unbounded"] │ │ └── col1:1 │ └── windows - │ └── array-agg [as=array_agg:6, frame="range from unbounded to unbounded"] + │ └── array-agg [as=array_agg:7, frame="range from unbounded to unbounded"] │ └── col1:1 └── aggregations - ├── const-agg [as=array_agg:5] - │ └── array_agg:5 - └── const-agg [as=array_agg:6] - └── array_agg:6 + ├── const-agg [as=array_agg:6] + │ └── array_agg:6 + └── const-agg [as=array_agg:7] + └── array_agg:7 build SELECT concat_agg(col3 ORDER BY col1), array_agg(col1) FROM tab ---- scalar-group-by - ├── columns: concat_agg:5 array_agg:6 + ├── columns: concat_agg:6 array_agg:7 ├── window partition=() - │ ├── columns: col1:1!null col2:2!null col3:3 rowid:4!null concat_agg:5 array_agg:6 + │ ├── columns: col1:1!null col2:2!null col3:3 rowid:4!null crdb_internal_mvcc_timestamp:5 concat_agg:6 array_agg:7 │ ├── window partition=() ordering=+1 - │ │ ├── columns: col1:1!null col2:2!null col3:3 rowid:4!null concat_agg:5 + │ │ ├── columns: col1:1!null col2:2!null col3:3 rowid:4!null crdb_internal_mvcc_timestamp:5 concat_agg:6 │ │ ├── scan tab - │ │ │ └── columns: col1:1!null col2:2!null col3:3 rowid:4!null + │ │ │ └── columns: col1:1!null col2:2!null col3:3 rowid:4!null crdb_internal_mvcc_timestamp:5 │ │ └── windows - │ │ └── concat-agg [as=concat_agg:5, frame="range from unbounded to unbounded"] + │ │ └── concat-agg [as=concat_agg:6, frame="range from unbounded to unbounded"] │ │ └── col3:3 │ └── windows - │ └── array-agg [as=array_agg:6, frame="range from unbounded to unbounded"] + │ └── array-agg [as=array_agg:7, frame="range from unbounded to unbounded"] │ └── col1:1 └── aggregations - ├── const-agg [as=concat_agg:5] - │ └── concat_agg:5 - └── const-agg [as=array_agg:6] - └── array_agg:6 + ├── const-agg [as=concat_agg:6] + │ └── concat_agg:6 + └── const-agg [as=array_agg:7] + └── array_agg:7 build SELECT concat_agg(col3 ORDER BY col1), sum(col1 ORDER BY col2) FROM tab ---- scalar-group-by - ├── columns: concat_agg:5 sum:6 + ├── columns: concat_agg:6 sum:7 ├── window partition=() - │ ├── columns: col1:1!null col2:2!null col3:3 rowid:4!null concat_agg:5 sum:6 + │ ├── columns: col1:1!null col2:2!null col3:3 rowid:4!null crdb_internal_mvcc_timestamp:5 concat_agg:6 sum:7 │ ├── window partition=() ordering=+1 - │ │ ├── columns: col1:1!null col2:2!null col3:3 rowid:4!null concat_agg:5 + │ │ ├── columns: col1:1!null col2:2!null col3:3 rowid:4!null crdb_internal_mvcc_timestamp:5 concat_agg:6 │ │ ├── scan tab - │ │ │ └── columns: col1:1!null col2:2!null col3:3 rowid:4!null + │ │ │ └── columns: col1:1!null col2:2!null col3:3 rowid:4!null crdb_internal_mvcc_timestamp:5 │ │ └── windows - │ │ └── concat-agg [as=concat_agg:5, frame="range from unbounded to unbounded"] + │ │ └── concat-agg [as=concat_agg:6, frame="range from unbounded to unbounded"] │ │ └── col3:3 │ └── windows - │ └── sum [as=sum:6, frame="range from unbounded to unbounded"] + │ └── sum [as=sum:7, frame="range from unbounded to unbounded"] │ └── col1:1 └── aggregations - ├── const-agg [as=concat_agg:5] - │ └── concat_agg:5 - └── const-agg [as=sum:6] - └── sum:6 + ├── const-agg [as=concat_agg:6] + │ └── concat_agg:6 + └── const-agg [as=sum:7] + └── sum:7 build SELECT array_agg(col1 ORDER BY col1) FROM tab GROUP BY col2 ---- project - ├── columns: array_agg:5 + ├── columns: array_agg:6 └── group-by - ├── columns: col2:2!null array_agg:5 + ├── columns: col2:2!null array_agg:6 ├── grouping columns: col2:2!null ├── window partition=(2) ordering=+1 - │ ├── columns: col1:1!null col2:2!null col3:3 rowid:4!null array_agg:5 + │ ├── columns: col1:1!null col2:2!null col3:3 rowid:4!null crdb_internal_mvcc_timestamp:5 array_agg:6 │ ├── scan tab - │ │ └── columns: col1:1!null col2:2!null col3:3 rowid:4!null + │ │ └── columns: col1:1!null col2:2!null col3:3 rowid:4!null crdb_internal_mvcc_timestamp:5 │ └── windows - │ └── array-agg [as=array_agg:5, frame="range from unbounded to unbounded"] + │ └── array-agg [as=array_agg:6, frame="range from unbounded to unbounded"] │ └── col1:1 └── aggregations - └── const-agg [as=array_agg:5] - └── array_agg:5 + └── const-agg [as=array_agg:6] + └── array_agg:6 build SELECT array_agg(col1 ORDER BY col1), array_agg(col3 ORDER BY col1) FROM tab GROUP BY col2 ---- project - ├── columns: array_agg:5 array_agg:6 + ├── columns: array_agg:6 array_agg:7 └── group-by - ├── columns: col2:2!null array_agg:5 array_agg:6 + ├── columns: col2:2!null array_agg:6 array_agg:7 ├── grouping columns: col2:2!null ├── window partition=(2) ordering=+1 - │ ├── columns: col1:1!null col2:2!null col3:3 rowid:4!null array_agg:5 array_agg:6 + │ ├── columns: col1:1!null col2:2!null col3:3 rowid:4!null crdb_internal_mvcc_timestamp:5 array_agg:6 array_agg:7 │ ├── scan tab - │ │ └── columns: col1:1!null col2:2!null col3:3 rowid:4!null + │ │ └── columns: col1:1!null col2:2!null col3:3 rowid:4!null crdb_internal_mvcc_timestamp:5 │ └── windows - │ ├── array-agg [as=array_agg:5, frame="range from unbounded to unbounded"] + │ ├── array-agg [as=array_agg:6, frame="range from unbounded to unbounded"] │ │ └── col1:1 - │ └── array-agg [as=array_agg:6, frame="range from unbounded to unbounded"] + │ └── array-agg [as=array_agg:7, frame="range from unbounded to unbounded"] │ └── col3:3 └── aggregations - ├── const-agg [as=array_agg:5] - │ └── array_agg:5 - └── const-agg [as=array_agg:6] - └── array_agg:6 + ├── const-agg [as=array_agg:6] + │ └── array_agg:6 + └── const-agg [as=array_agg:7] + └── array_agg:7 build SELECT array_agg(col1 ORDER BY col1), array_agg(col3 ORDER BY col1) FROM tab GROUP BY col2 HAVING col2 > 1 ---- project - ├── columns: array_agg:5 array_agg:6 + ├── columns: array_agg:6 array_agg:7 └── select - ├── columns: col2:2!null array_agg:5 array_agg:6 + ├── columns: col2:2!null array_agg:6 array_agg:7 ├── group-by - │ ├── columns: col2:2!null array_agg:5 array_agg:6 + │ ├── columns: col2:2!null array_agg:6 array_agg:7 │ ├── grouping columns: col2:2!null │ ├── window partition=(2) ordering=+1 - │ │ ├── columns: col1:1!null col2:2!null col3:3 rowid:4!null array_agg:5 array_agg:6 + │ │ ├── columns: col1:1!null col2:2!null col3:3 rowid:4!null crdb_internal_mvcc_timestamp:5 array_agg:6 array_agg:7 │ │ ├── scan tab - │ │ │ └── columns: col1:1!null col2:2!null col3:3 rowid:4!null + │ │ │ └── columns: col1:1!null col2:2!null col3:3 rowid:4!null crdb_internal_mvcc_timestamp:5 │ │ └── windows - │ │ ├── array-agg [as=array_agg:5, frame="range from unbounded to unbounded"] + │ │ ├── array-agg [as=array_agg:6, frame="range from unbounded to unbounded"] │ │ │ └── col1:1 - │ │ └── array-agg [as=array_agg:6, frame="range from unbounded to unbounded"] + │ │ └── array-agg [as=array_agg:7, frame="range from unbounded to unbounded"] │ │ └── col3:3 │ └── aggregations - │ ├── const-agg [as=array_agg:5] - │ │ └── array_agg:5 - │ └── const-agg [as=array_agg:6] - │ └── array_agg:6 + │ ├── const-agg [as=array_agg:6] + │ │ └── array_agg:6 + │ └── const-agg [as=array_agg:7] + │ └── array_agg:7 └── filters └── col2:2 > 1 @@ -3124,138 +3137,138 @@ build SELECT count(DISTINCT col1), count(*), array_agg(col1 ORDER BY col2) FROM tab ---- project - ├── columns: count:5 count:6 array_agg:7 + ├── columns: count:6 count:7 array_agg:8 └── project - ├── columns: count:5 count_rows:6 col1:1 col2:2 col3:3 rowid:4 array_agg:7 + ├── columns: count:6 count_rows:7 col1:1 col2:2 col3:3 rowid:4 crdb_internal_mvcc_timestamp:5 array_agg:8 ├── scalar-group-by - │ ├── columns: array_agg:7 count:8 count_rows:9 + │ ├── columns: array_agg:8 count:9 count_rows:10 │ ├── window partition=() ordering=+2 - │ │ ├── columns: col1:1!null col2:2!null col3:3 rowid:4!null count:5 count_rows:6 array_agg:7 + │ │ ├── columns: col1:1!null col2:2!null col3:3 rowid:4!null crdb_internal_mvcc_timestamp:5 count:6 count_rows:7 array_agg:8 │ │ ├── window partition=() - │ │ │ ├── columns: col1:1!null col2:2!null col3:3 rowid:4!null count:5 count_rows:6 + │ │ │ ├── columns: col1:1!null col2:2!null col3:3 rowid:4!null crdb_internal_mvcc_timestamp:5 count:6 count_rows:7 │ │ │ ├── scan tab - │ │ │ │ └── columns: col1:1!null col2:2!null col3:3 rowid:4!null + │ │ │ │ └── columns: col1:1!null col2:2!null col3:3 rowid:4!null crdb_internal_mvcc_timestamp:5 │ │ │ └── windows - │ │ │ ├── count [as=count:5, frame="range from unbounded to unbounded"] + │ │ │ ├── count [as=count:6, frame="range from unbounded to unbounded"] │ │ │ │ └── col1:1 - │ │ │ └── count-rows [as=count_rows:6, frame="range from unbounded to unbounded"] + │ │ │ └── count-rows [as=count_rows:7, frame="range from unbounded to unbounded"] │ │ └── windows - │ │ └── array-agg [as=array_agg:7, frame="range from unbounded to unbounded"] + │ │ └── array-agg [as=array_agg:8, frame="range from unbounded to unbounded"] │ │ └── col1:1 │ └── aggregations - │ ├── const-agg [as=count:8] - │ │ └── count:5 - │ ├── const-agg [as=count_rows:9] - │ │ └── count_rows:6 - │ └── const-agg [as=array_agg:7] - │ └── array_agg:7 + │ ├── const-agg [as=count:9] + │ │ └── count:6 + │ ├── const-agg [as=count_rows:10] + │ │ └── count_rows:7 + │ └── const-agg [as=array_agg:8] + │ └── array_agg:8 └── projections - ├── CASE WHEN count:8 IS NULL THEN 0 ELSE count:8 END [as=count:5] - └── CASE WHEN count_rows:9 IS NULL THEN 0 ELSE count_rows:9 END [as=count_rows:6] + ├── CASE WHEN count:9 IS NULL THEN 0 ELSE count:9 END [as=count:6] + └── CASE WHEN count_rows:10 IS NULL THEN 0 ELSE count_rows:10 END [as=count_rows:7] # Testing aggregations as window when group by has a projection. build SELECT array_agg(col1 ORDER BY col1) FROM tab GROUP BY upper(col3) ---- project - ├── columns: array_agg:5 + ├── columns: array_agg:6 └── group-by - ├── columns: array_agg:5 column6:6 - ├── grouping columns: column6:6 - ├── window partition=(6) ordering=+1 - │ ├── columns: col1:1!null col2:2!null col3:3 rowid:4!null array_agg:5 column6:6 + ├── columns: array_agg:6 column7:7 + ├── grouping columns: column7:7 + ├── window partition=(7) ordering=+1 + │ ├── columns: col1:1!null col2:2!null col3:3 rowid:4!null crdb_internal_mvcc_timestamp:5 array_agg:6 column7:7 │ ├── project - │ │ ├── columns: column6:6 col1:1!null col2:2!null col3:3 rowid:4!null + │ │ ├── columns: column7:7 col1:1!null col2:2!null col3:3 rowid:4!null crdb_internal_mvcc_timestamp:5 │ │ ├── scan tab - │ │ │ └── columns: col1:1!null col2:2!null col3:3 rowid:4!null + │ │ │ └── columns: col1:1!null col2:2!null col3:3 rowid:4!null crdb_internal_mvcc_timestamp:5 │ │ └── projections - │ │ └── upper(col3:3) [as=column6:6] + │ │ └── upper(col3:3) [as=column7:7] │ └── windows - │ └── array-agg [as=array_agg:5, frame="range from unbounded to unbounded"] + │ └── array-agg [as=array_agg:6, frame="range from unbounded to unbounded"] │ └── col1:1 └── aggregations - └── const-agg [as=array_agg:5] - └── array_agg:5 + └── const-agg [as=array_agg:6] + └── array_agg:6 build SELECT array_agg(col1 ORDER BY col1), upper(col3) FROM tab GROUP BY upper(col3) ---- group-by - ├── columns: array_agg:5 upper:6 - ├── grouping columns: column6:6 - ├── window partition=(6) ordering=+1 - │ ├── columns: col1:1!null col2:2!null col3:3 rowid:4!null array_agg:5 column6:6 + ├── columns: array_agg:6 upper:7 + ├── grouping columns: column7:7 + ├── window partition=(7) ordering=+1 + │ ├── columns: col1:1!null col2:2!null col3:3 rowid:4!null crdb_internal_mvcc_timestamp:5 array_agg:6 column7:7 │ ├── project - │ │ ├── columns: column6:6 col1:1!null col2:2!null col3:3 rowid:4!null + │ │ ├── columns: column7:7 col1:1!null col2:2!null col3:3 rowid:4!null crdb_internal_mvcc_timestamp:5 │ │ ├── scan tab - │ │ │ └── columns: col1:1!null col2:2!null col3:3 rowid:4!null + │ │ │ └── columns: col1:1!null col2:2!null col3:3 rowid:4!null crdb_internal_mvcc_timestamp:5 │ │ └── projections - │ │ └── upper(col3:3) [as=column6:6] + │ │ └── upper(col3:3) [as=column7:7] │ └── windows - │ └── array-agg [as=array_agg:5, frame="range from unbounded to unbounded"] + │ └── array-agg [as=array_agg:6, frame="range from unbounded to unbounded"] │ └── col1:1 └── aggregations - └── const-agg [as=array_agg:5] - └── array_agg:5 + └── const-agg [as=array_agg:6] + └── array_agg:6 build SELECT array_agg(lower(col3)) FROM tab GROUP BY upper(col3) ---- project - ├── columns: array_agg:6 + ├── columns: array_agg:7 └── group-by - ├── columns: array_agg:6 column7:7 - ├── grouping columns: column7:7 + ├── columns: array_agg:7 column8:8 + ├── grouping columns: column8:8 ├── project - │ ├── columns: column5:5 column7:7 + │ ├── columns: column6:6 column8:8 │ ├── scan tab - │ │ └── columns: col1:1!null col2:2!null col3:3 rowid:4!null + │ │ └── columns: col1:1!null col2:2!null col3:3 rowid:4!null crdb_internal_mvcc_timestamp:5 │ └── projections - │ ├── lower(col3:3) [as=column5:5] - │ └── upper(col3:3) [as=column7:7] + │ ├── lower(col3:3) [as=column6:6] + │ └── upper(col3:3) [as=column8:8] └── aggregations - └── array-agg [as=array_agg:6] - └── column5:5 + └── array-agg [as=array_agg:7] + └── column6:6 build SELECT array_agg(v+w ORDER BY w) FROM kv ---- scalar-group-by - ├── columns: array_agg:6 + ├── columns: array_agg:7 ├── window partition=() ordering=+3 - │ ├── columns: k:1!null v:2 w:3 s:4 column5:5 array_agg:6 + │ ├── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 column6:6 array_agg:7 │ ├── project - │ │ ├── columns: column5:5 k:1!null v:2 w:3 s:4 + │ │ ├── columns: column6:6 k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ │ ├── scan kv - │ │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ │ └── projections - │ │ └── v:2 + w:3 [as=column5:5] + │ │ └── v:2 + w:3 [as=column6:6] │ └── windows - │ └── array-agg [as=array_agg:6, frame="range from unbounded to unbounded"] - │ └── column5:5 + │ └── array-agg [as=array_agg:7, frame="range from unbounded to unbounded"] + │ └── column6:6 └── aggregations - └── const-agg [as=array_agg:6] - └── array_agg:6 + └── const-agg [as=array_agg:7] + └── array_agg:7 build SELECT array_agg(v ORDER BY v+w) FROM kv ---- scalar-group-by - ├── columns: array_agg:6 - ├── window partition=() ordering=+5 - │ ├── columns: k:1!null v:2 w:3 s:4 column5:5 array_agg:6 + ├── columns: array_agg:7 + ├── window partition=() ordering=+6 + │ ├── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 column6:6 array_agg:7 │ ├── project - │ │ ├── columns: column5:5 k:1!null v:2 w:3 s:4 + │ │ ├── columns: column6:6 k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ │ ├── scan kv - │ │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ │ └── projections - │ │ └── v:2 + w:3 [as=column5:5] + │ │ └── v:2 + w:3 [as=column6:6] │ └── windows - │ └── array-agg [as=array_agg:6, frame="range from unbounded to unbounded"] + │ └── array-agg [as=array_agg:7, frame="range from unbounded to unbounded"] │ └── v:2 └── aggregations - └── const-agg [as=array_agg:6] - └── array_agg:6 + └── const-agg [as=array_agg:7] + └── array_agg:7 build SELECT array_agg(a ORDER BY b) FROM (SELECT 1 AS a, 2 AS b) @@ -3296,7 +3309,7 @@ project └── project ├── columns: k:1!null v:2 └── scan kv - └── columns: k:1!null v:2 w:3 s:4 + └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 # This should be equivalent to the query above. build @@ -3310,41 +3323,41 @@ project └── project ├── columns: k:1!null v:2 └── scan kv - └── columns: k:1!null v:2 w:3 s:4 + └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 build SELECT count(*), k+v FROM kv GROUP BY k ---- project - ├── columns: count:5!null "?column?":6 + ├── columns: count:6!null "?column?":7 ├── group-by - │ ├── columns: k:1!null v:2 count_rows:5!null + │ ├── columns: k:1!null v:2 count_rows:6!null │ ├── grouping columns: k:1!null v:2 │ ├── project │ │ ├── columns: k:1!null v:2 │ │ └── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── aggregations - │ └── count-rows [as=count_rows:5] + │ └── count-rows [as=count_rows:6] └── projections - └── k:1 + v:2 [as="?column?":6] + └── k:1 + v:2 [as="?column?":7] build SELECT count(*) FROM kv GROUP BY k HAVING v=1 ---- project - ├── columns: count:5!null + ├── columns: count:6!null └── select - ├── columns: k:1!null v:2!null count_rows:5!null + ├── columns: k:1!null v:2!null count_rows:6!null ├── group-by - │ ├── columns: k:1!null v:2 count_rows:5!null + │ ├── columns: k:1!null v:2 count_rows:6!null │ ├── grouping columns: k:1!null v:2 │ ├── project │ │ ├── columns: k:1!null v:2 │ │ └── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── aggregations - │ └── count-rows [as=count_rows:5] + │ └── count-rows [as=count_rows:6] └── filters └── v:2 = 1 @@ -3352,20 +3365,20 @@ build SELECT k, v, count(*) FROM kv JOIN ab ON a=k GROUP BY k ---- group-by - ├── columns: k:1!null v:2 count:7!null + ├── columns: k:1!null v:2 count:9!null ├── grouping columns: k:1!null v:2 ├── project │ ├── columns: k:1!null v:2 │ └── inner-join (hash) - │ ├── columns: k:1!null v:2 w:3 s:4 a:5!null b:6 + │ ├── columns: k:1!null v:2 w:3 s:4 kv.crdb_internal_mvcc_timestamp:5 a:6!null b:7 ab.crdb_internal_mvcc_timestamp:8 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 kv.crdb_internal_mvcc_timestamp:5 │ ├── scan ab - │ │ └── columns: a:5!null b:6 + │ │ └── columns: a:6!null b:7 ab.crdb_internal_mvcc_timestamp:8 │ └── filters - │ └── a:5 = k:1 + │ └── a:6 = k:1 └── aggregations - └── count-rows [as=count_rows:7] + └── count-rows [as=count_rows:9] # Not allowed when grouping on a subset of the PK. build @@ -3381,8 +3394,10 @@ project └── group-by ├── columns: a:1!null b:2!null x:3 y:4 ├── grouping columns: a:1!null b:2!null x:3 y:4 - └── scan abxy - └── columns: a:1!null b:2!null x:3 y:4 + └── project + ├── columns: a:1!null b:2!null x:3 y:4 + └── scan abxy + └── columns: a:1!null b:2!null x:3 y:4 crdb_internal_mvcc_timestamp:5 # The following two should be equivalent to the one above. build @@ -3393,8 +3408,10 @@ project └── group-by ├── columns: a:1!null b:2!null x:3 y:4 ├── grouping columns: a:1!null b:2!null x:3 y:4 - └── scan abxy - └── columns: a:1!null b:2!null x:3 y:4 + └── project + ├── columns: a:1!null b:2!null x:3 y:4 + └── scan abxy + └── columns: a:1!null b:2!null x:3 y:4 crdb_internal_mvcc_timestamp:5 build SELECT x, y FROM abxy GROUP BY x, y, a, b @@ -3404,8 +3421,10 @@ project └── group-by ├── columns: a:1!null b:2!null x:3 y:4 ├── grouping columns: a:1!null b:2!null x:3 y:4 - └── scan abxy - └── columns: a:1!null b:2!null x:3 y:4 + └── project + ├── columns: a:1!null b:2!null x:3 y:4 + └── scan abxy + └── columns: a:1!null b:2!null x:3 y:4 crdb_internal_mvcc_timestamp:5 build SELECT x, y FROM abxy NATURAL JOIN ab GROUP BY a, b @@ -3418,14 +3437,14 @@ project └── project ├── columns: abxy.a:1!null abxy.b:2!null x:3 y:4 └── inner-join (hash) - ├── columns: abxy.a:1!null abxy.b:2!null x:3 y:4 ab.a:5!null ab.b:6!null + ├── columns: abxy.a:1!null abxy.b:2!null x:3 y:4 abxy.crdb_internal_mvcc_timestamp:5 ab.a:6!null ab.b:7!null ab.crdb_internal_mvcc_timestamp:8 ├── scan abxy - │ └── columns: abxy.a:1!null abxy.b:2!null x:3 y:4 + │ └── columns: abxy.a:1!null abxy.b:2!null x:3 y:4 abxy.crdb_internal_mvcc_timestamp:5 ├── scan ab - │ └── columns: ab.a:5!null ab.b:6 + │ └── columns: ab.a:6!null ab.b:7 ab.crdb_internal_mvcc_timestamp:8 └── filters - ├── abxy.a:1 = ab.a:5 - └── abxy.b:2 = ab.b:6 + ├── abxy.a:1 = ab.a:6 + └── abxy.b:2 = ab.b:7 # Should be equivalent to the one above. build @@ -3439,28 +3458,30 @@ project └── project ├── columns: abxy.a:1!null abxy.b:2!null x:3 y:4 └── inner-join (hash) - ├── columns: abxy.a:1!null abxy.b:2!null x:3 y:4 ab.a:5!null ab.b:6!null + ├── columns: abxy.a:1!null abxy.b:2!null x:3 y:4 abxy.crdb_internal_mvcc_timestamp:5 ab.a:6!null ab.b:7!null ab.crdb_internal_mvcc_timestamp:8 ├── scan abxy - │ └── columns: abxy.a:1!null abxy.b:2!null x:3 y:4 + │ └── columns: abxy.a:1!null abxy.b:2!null x:3 y:4 abxy.crdb_internal_mvcc_timestamp:5 ├── scan ab - │ └── columns: ab.a:5!null ab.b:6 + │ └── columns: ab.a:6!null ab.b:7 ab.crdb_internal_mvcc_timestamp:8 └── filters - ├── abxy.a:1 = ab.a:5 - └── abxy.b:2 = ab.b:6 + ├── abxy.a:1 = ab.a:6 + └── abxy.b:2 = ab.b:7 build SELECT abxy.*, ab.* FROM abxy, ab GROUP BY abxy.a, abxy.b, ab.a ---- group-by - ├── columns: a:1!null b:2!null x:3 y:4 a:5!null b:6 - ├── grouping columns: abxy.a:1!null abxy.b:2!null x:3 y:4 ab.a:5!null ab.b:6 - └── inner-join (cross) - ├── columns: abxy.a:1!null abxy.b:2!null x:3 y:4 ab.a:5!null ab.b:6 - ├── scan abxy - │ └── columns: abxy.a:1!null abxy.b:2!null x:3 y:4 - ├── scan ab - │ └── columns: ab.a:5!null ab.b:6 - └── filters (true) + ├── columns: a:1!null b:2!null x:3 y:4 a:6!null b:7 + ├── grouping columns: abxy.a:1!null abxy.b:2!null x:3 y:4 ab.a:6!null ab.b:7 + └── project + ├── columns: abxy.a:1!null abxy.b:2!null x:3 y:4 ab.a:6!null ab.b:7 + └── inner-join (cross) + ├── columns: abxy.a:1!null abxy.b:2!null x:3 y:4 abxy.crdb_internal_mvcc_timestamp:5 ab.a:6!null ab.b:7 ab.crdb_internal_mvcc_timestamp:8 + ├── scan abxy + │ └── columns: abxy.a:1!null abxy.b:2!null x:3 y:4 abxy.crdb_internal_mvcc_timestamp:5 + ├── scan ab + │ └── columns: ab.a:6!null ab.b:7 ab.crdb_internal_mvcc_timestamp:8 + └── filters (true) # Not allowed with UNION. build @@ -3480,17 +3501,17 @@ project └── except ├── columns: abxy.a:1!null abxy.b:2 x:3 ├── left columns: abxy.a:1!null abxy.b:2 x:3 - ├── right columns: ab.a:5 ab.b:6 "?column?":7 + ├── right columns: ab.a:6 ab.b:7 "?column?":9 ├── project │ ├── columns: abxy.a:1!null abxy.b:2!null x:3 │ └── scan abxy - │ └── columns: abxy.a:1!null abxy.b:2!null x:3 y:4 + │ └── columns: abxy.a:1!null abxy.b:2!null x:3 y:4 abxy.crdb_internal_mvcc_timestamp:5 └── project - ├── columns: "?column?":7!null ab.a:5!null ab.b:6 + ├── columns: "?column?":9!null ab.a:6!null ab.b:7 ├── scan ab - │ └── columns: ab.a:5!null ab.b:6 + │ └── columns: ab.a:6!null ab.b:7 ab.crdb_internal_mvcc_timestamp:8 └── projections - └── 1 [as="?column?":7] + └── 1 [as="?column?":9] # Allowed even with outer joins. It's a little subtle why this is correct: the # PK columns are also non-nullable so any "outer" rows are never in the same @@ -3506,13 +3527,13 @@ project └── project ├── columns: k:1 v:2 w:3 └── full-join (hash) - ├── columns: k:1 v:2 w:3 s:4 a:5 b:6 + ├── columns: k:1 v:2 w:3 s:4 kv.crdb_internal_mvcc_timestamp:5 a:6 b:7 ab.crdb_internal_mvcc_timestamp:8 ├── scan kv - │ └── columns: k:1!null v:2 w:3 s:4 + │ └── columns: k:1!null v:2 w:3 s:4 kv.crdb_internal_mvcc_timestamp:5 ├── scan ab - │ └── columns: a:5!null b:6 + │ └── columns: a:6!null b:7 ab.crdb_internal_mvcc_timestamp:8 └── filters - └── k:1 = a:5 + └── k:1 = a:6 # Verify that we handle tables with no primary index (#44659). build @@ -3525,30 +3546,30 @@ build SELECT x + 1 AS z FROM abxy GROUP BY z ---- group-by - ├── columns: z:5 - ├── grouping columns: z:5 + ├── columns: z:6 + ├── grouping columns: z:6 └── project - ├── columns: z:5 + ├── columns: z:6 ├── scan abxy - │ └── columns: a:1!null b:2!null x:3 y:4 + │ └── columns: a:1!null b:2!null x:3 y:4 crdb_internal_mvcc_timestamp:5 └── projections - └── x:3 + 1 [as=z:5] + └── x:3 + 1 [as=z:6] # The FROM column has precedence, we should be grouping by abxy.x, not by x%10. build SELECT (x % 10) AS x FROM abxy GROUP BY x ---- project - ├── columns: x:5 + ├── columns: x:6 ├── group-by │ ├── columns: abxy.x:3 │ ├── grouping columns: abxy.x:3 │ └── project │ ├── columns: abxy.x:3 │ └── scan abxy - │ └── columns: a:1!null b:2!null abxy.x:3 y:4 + │ └── columns: a:1!null b:2!null abxy.x:3 y:4 crdb_internal_mvcc_timestamp:5 └── projections - └── abxy.x:3 % 10 [as=x:5] + └── abxy.x:3 % 10 [as=x:6] # But aliases have precedence over columns from higher scopes. Here we are # grouping by v, not by the outer x. @@ -3556,20 +3577,20 @@ build SELECT x, (SELECT v AS x FROM kv GROUP BY x) FROM abxy ---- project - ├── columns: x:3 x:9 + ├── columns: x:3 x:11 ├── scan abxy - │ └── columns: a:1!null b:2!null abxy.x:3 y:4 + │ └── columns: a:1!null b:2!null abxy.x:3 y:4 abxy.crdb_internal_mvcc_timestamp:5 └── projections - └── subquery [as=x:9] + └── subquery [as=x:11] └── max1-row - ├── columns: v:6 + ├── columns: v:7 └── group-by - ├── columns: v:6 - ├── grouping columns: v:6 + ├── columns: v:7 + ├── grouping columns: v:7 └── project - ├── columns: v:6 + ├── columns: v:7 └── scan kv - └── columns: k:5!null v:6 w:7 s:8 + └── columns: k:6!null v:7 w:8 s:9 kv.crdb_internal_mvcc_timestamp:10 build SELECT sum(x) AS u FROM abxy GROUP BY u @@ -3581,14 +3602,14 @@ build SELECT x + 1 FROM abxy GROUP BY "?column?" ---- group-by - ├── columns: "?column?":5 - ├── grouping columns: "?column?":5 + ├── columns: "?column?":6 + ├── grouping columns: "?column?":6 └── project - ├── columns: "?column?":5 + ├── columns: "?column?":6 ├── scan abxy - │ └── columns: a:1!null b:2!null x:3 y:4 + │ └── columns: a:1!null b:2!null x:3 y:4 crdb_internal_mvcc_timestamp:5 └── projections - └── x:3 + 1 [as="?column?":5] + └── x:3 + 1 [as="?column?":6] build SELECT sum(x) FROM abxy GROUP BY sum @@ -3613,14 +3634,14 @@ build SELECT (x + 1) AS u, (x + 1) AS u FROM abxy GROUP BY u ---- group-by - ├── columns: u:5 u:5 - ├── grouping columns: u:5 + ├── columns: u:6 u:6 + ├── grouping columns: u:6 └── project - ├── columns: u:5 + ├── columns: u:6 ├── scan abxy - │ └── columns: a:1!null b:2!null x:3 y:4 + │ └── columns: a:1!null b:2!null x:3 y:4 crdb_internal_mvcc_timestamp:5 └── projections - └── x:3 + 1 [as=u:5] + └── x:3 + 1 [as=u:6] build SELECT (x + 1) AS u, (x + 1) AS u, (y + 1) AS u FROM abxy GROUP BY u @@ -3632,22 +3653,22 @@ build SELECT sum(x + 1) AS x, sum(y + 1) AS x FROM abxy GROUP BY x ---- project - ├── columns: x:6 x:8 + ├── columns: x:7 x:9 └── group-by - ├── columns: x:3 sum:6 sum:8 + ├── columns: x:3 sum:7 sum:9 ├── grouping columns: x:3 ├── project - │ ├── columns: column5:5 column7:7 x:3 + │ ├── columns: column6:6 column8:8 x:3 │ ├── scan abxy - │ │ └── columns: a:1!null b:2!null x:3 y:4 + │ │ └── columns: a:1!null b:2!null x:3 y:4 crdb_internal_mvcc_timestamp:5 │ └── projections - │ ├── x:3 + 1 [as=column5:5] - │ └── y:4 + 1 [as=column7:7] + │ ├── x:3 + 1 [as=column6:6] + │ └── y:4 + 1 [as=column8:8] └── aggregations - ├── sum [as=sum:6] - │ └── column5:5 - └── sum [as=sum:8] - └── column7:7 + ├── sum [as=sum:7] + │ └── column6:6 + └── sum [as=sum:9] + └── column8:8 # Regression test for #44724. build @@ -3668,31 +3689,31 @@ GROUP BY y HAVING EXISTS(SELECT 1 FROM abxy AS t2 WHERE sum(t.x) = 1) ---- project - ├── columns: sum:5 + ├── columns: sum:6 └── select - ├── columns: t.y:4 sum:5 + ├── columns: t.y:4 sum:6 ├── group-by - │ ├── columns: t.y:4 sum:5 + │ ├── columns: t.y:4 sum:6 │ ├── grouping columns: t.y:4 │ ├── project │ │ ├── columns: t.x:3 t.y:4 │ │ └── scan t - │ │ └── columns: t.a:1!null t.b:2!null t.x:3 t.y:4 + │ │ └── columns: t.a:1!null t.b:2!null t.x:3 t.y:4 t.crdb_internal_mvcc_timestamp:5 │ └── aggregations - │ └── sum [as=sum:5] + │ └── sum [as=sum:6] │ └── t.x:3 └── filters └── exists └── project - ├── columns: "?column?":11!null + ├── columns: "?column?":13!null ├── select - │ ├── columns: t2.a:6!null t2.b:7!null t2.x:8 t2.y:9 + │ ├── columns: t2.a:7!null t2.b:8!null t2.x:9 t2.y:10 t2.crdb_internal_mvcc_timestamp:11 │ ├── scan t2 - │ │ └── columns: t2.a:6!null t2.b:7!null t2.x:8 t2.y:9 + │ │ └── columns: t2.a:7!null t2.b:8!null t2.x:9 t2.y:10 t2.crdb_internal_mvcc_timestamp:11 │ └── filters - │ └── sum:5 = 1 + │ └── sum:6 = 1 └── projections - └── 1 [as="?column?":11] + └── 1 [as="?column?":13] exec-ddl CREATE TABLE onek ( @@ -3725,30 +3746,30 @@ GROUP BY ten ) ---- select - ├── columns: ten:5 sum:18 + ├── columns: ten:5 sum:19 ├── group-by - │ ├── columns: a.ten:5 sum:18 + │ ├── columns: a.ten:5 sum:19 │ ├── grouping columns: a.ten:5 │ ├── project │ │ ├── columns: a.four:4 a.ten:5 │ │ └── scan a - │ │ └── columns: a.unique1:1 a.unique2:2 a.two:3 a.four:4 a.ten:5 a.twenty:6 a.hundred:7 a.thousand:8 a.twothousand:9 a.fivethous:10 a.tenthous:11 a.odd:12 a.even:13 a.stringu1:14 a.stringu2:15 a.string4:16 a.rowid:17!null + │ │ └── columns: a.unique1:1 a.unique2:2 a.two:3 a.four:4 a.ten:5 a.twenty:6 a.hundred:7 a.thousand:8 a.twothousand:9 a.fivethous:10 a.tenthous:11 a.odd:12 a.even:13 a.stringu1:14 a.stringu2:15 a.string4:16 a.rowid:17!null a.crdb_internal_mvcc_timestamp:18 │ └── aggregations - │ └── agg-distinct [as=sum:18] + │ └── agg-distinct [as=sum:19] │ └── sum │ └── a.four:4 └── filters └── exists └── project - ├── columns: "?column?":37!null + ├── columns: "?column?":39!null ├── select - │ ├── columns: b.unique1:19 b.unique2:20 b.two:21 b.four:22!null b.ten:23 b.twenty:24 b.hundred:25 b.thousand:26 b.twothousand:27 b.fivethous:28 b.tenthous:29 b.odd:30 b.even:31 b.stringu1:32 b.stringu2:33 b.string4:34 b.rowid:35!null + │ ├── columns: b.unique1:20 b.unique2:21 b.two:22 b.four:23!null b.ten:24 b.twenty:25 b.hundred:26 b.thousand:27 b.twothousand:28 b.fivethous:29 b.tenthous:30 b.odd:31 b.even:32 b.stringu1:33 b.stringu2:34 b.string4:35 b.rowid:36!null b.crdb_internal_mvcc_timestamp:37 │ ├── scan b - │ │ └── columns: b.unique1:19 b.unique2:20 b.two:21 b.four:22 b.ten:23 b.twenty:24 b.hundred:25 b.thousand:26 b.twothousand:27 b.fivethous:28 b.tenthous:29 b.odd:30 b.even:31 b.stringu1:32 b.stringu2:33 b.string4:34 b.rowid:35!null + │ │ └── columns: b.unique1:20 b.unique2:21 b.two:22 b.four:23 b.ten:24 b.twenty:25 b.hundred:26 b.thousand:27 b.twothousand:28 b.fivethous:29 b.tenthous:30 b.odd:31 b.even:32 b.stringu1:33 b.stringu2:34 b.string4:35 b.rowid:36!null b.crdb_internal_mvcc_timestamp:37 │ └── filters - │ └── sum:18 = b.four:22 + │ └── sum:19 = b.four:23 └── projections - └── 1 [as="?column?":37] + └── 1 [as="?column?":39] build SELECT ten, sum(DISTINCT four) @@ -3770,32 +3791,32 @@ SELECT ( FROM abxy AS t1 ---- project - ├── columns: a:15 + ├── columns: a:18 ├── scalar-group-by - │ ├── columns: sum:14 + │ ├── columns: sum:17 │ ├── project - │ │ ├── columns: x:13 + │ │ ├── columns: x:16 │ │ ├── scan t1 - │ │ │ └── columns: t1.a:1!null t1.b:2!null t1.x:3 t1.y:4 + │ │ │ └── columns: t1.a:1!null t1.b:2!null t1.x:3 t1.y:4 t1.crdb_internal_mvcc_timestamp:5 │ │ └── projections - │ │ └── t1.x:3 [as=x:13] + │ │ └── t1.x:3 [as=x:16] │ └── aggregations - │ └── sum [as=sum:14] - │ └── x:13 + │ └── sum [as=sum:17] + │ └── x:16 └── projections - └── subquery [as=a:15] + └── subquery [as=a:18] └── max1-row - ├── columns: t2.a:5!null + ├── columns: t2.a:6!null └── project - ├── columns: t2.a:5!null + ├── columns: t2.a:6!null └── inner-join (cross) - ├── columns: t2.a:5!null t2.b:6!null t2.x:7 t2.y:8 t3.a:9!null t3.b:10!null t3.x:11!null t3.y:12 + ├── columns: t2.a:6!null t2.b:7!null t2.x:8 t2.y:9 t2.crdb_internal_mvcc_timestamp:10 t3.a:11!null t3.b:12!null t3.x:13!null t3.y:14 t3.crdb_internal_mvcc_timestamp:15 ├── scan t2 - │ └── columns: t2.a:5!null t2.b:6!null t2.x:7 t2.y:8 + │ └── columns: t2.a:6!null t2.b:7!null t2.x:8 t2.y:9 t2.crdb_internal_mvcc_timestamp:10 ├── scan t3 - │ └── columns: t3.a:9!null t3.b:10!null t3.x:11 t3.y:12 + │ └── columns: t3.a:11!null t3.b:12!null t3.x:13 t3.y:14 t3.crdb_internal_mvcc_timestamp:15 └── filters - └── sum:14 = t3.x:11 + └── sum:17 = t3.x:13 # Regression test for #45631. build @@ -3805,7 +3826,7 @@ FROM ROWS FROM (unnest(ARRAY[1, 2])) ---- project - ├── columns: "?column?":7 + ├── columns: "?column?":8 ├── project-set │ ├── columns: unnest:1 │ ├── values @@ -3813,29 +3834,29 @@ project │ └── zip │ └── unnest(ARRAY[1,2]) └── projections - └── subquery [as="?column?":7] + └── subquery [as="?column?":8] └── max1-row - ├── columns: "?column?":6 + ├── columns: "?column?":7 └── project - ├── columns: "?column?":6 + ├── columns: "?column?":7 ├── group-by - │ ├── columns: max:4 unnest:5 - │ ├── grouping columns: unnest:5 + │ ├── columns: max:5 unnest:6 + │ ├── grouping columns: unnest:6 │ ├── project - │ │ ├── columns: unnest:5 b:3 + │ │ ├── columns: unnest:6 b:3 │ │ ├── select - │ │ │ ├── columns: a:2!null b:3 + │ │ │ ├── columns: a:2!null b:3 crdb_internal_mvcc_timestamp:4 │ │ │ ├── scan ab - │ │ │ │ └── columns: a:2!null b:3 + │ │ │ │ └── columns: a:2!null b:3 crdb_internal_mvcc_timestamp:4 │ │ │ └── filters │ │ │ └── a:2 = unnest:1 │ │ └── projections - │ │ └── unnest:1 [as=unnest:5] + │ │ └── unnest:1 [as=unnest:6] │ └── aggregations - │ └── max [as=max:4] + │ └── max [as=max:5] │ └── b:3 └── projections - └── (max:4, unnest:1) [as="?column?":6] + └── (max:5, unnest:1) [as="?column?":7] # Regression test for #46196. Don't eliminate the scalar group by, and # default to type string. @@ -3860,85 +3881,85 @@ build SELECT percentile_disc(0.95) WITHIN GROUP (ORDER BY b) FROM abc; ---- scalar-group-by - ├── columns: percentile_disc:6 + ├── columns: percentile_disc:7 ├── window partition=() ordering=+2 - │ ├── columns: a:1!null b:2 c:3 d:4 column5:5!null percentile_disc:6 + │ ├── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 column6:6!null percentile_disc:7 │ ├── project - │ │ ├── columns: column5:5!null a:1!null b:2 c:3 d:4 + │ │ ├── columns: column6:6!null a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 │ │ ├── scan abc - │ │ │ └── columns: a:1!null b:2 c:3 d:4 + │ │ │ └── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 │ │ └── projections - │ │ └── 0.95 [as=column5:5] + │ │ └── 0.95 [as=column6:6] │ └── windows - │ └── percentile-disc [as=percentile_disc:6, frame="range from unbounded to unbounded"] - │ ├── column5:5 + │ └── percentile-disc [as=percentile_disc:7, frame="range from unbounded to unbounded"] + │ ├── column6:6 │ └── b:2 └── aggregations - └── const-agg [as=percentile_disc:6] - └── percentile_disc:6 + └── const-agg [as=percentile_disc:7] + └── percentile_disc:7 build SELECT percentile_cont(0.95) WITHIN GROUP (ORDER BY b) FROM abc; ---- scalar-group-by - ├── columns: percentile_cont:6 + ├── columns: percentile_cont:7 ├── window partition=() ordering=+2 - │ ├── columns: a:1!null b:2 c:3 d:4 column5:5!null percentile_cont:6 + │ ├── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 column6:6!null percentile_cont:7 │ ├── project - │ │ ├── columns: column5:5!null a:1!null b:2 c:3 d:4 + │ │ ├── columns: column6:6!null a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 │ │ ├── scan abc - │ │ │ └── columns: a:1!null b:2 c:3 d:4 + │ │ │ └── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 │ │ └── projections - │ │ └── 0.95 [as=column5:5] + │ │ └── 0.95 [as=column6:6] │ └── windows - │ └── percentile-cont [as=percentile_cont:6, frame="range from unbounded to unbounded"] - │ ├── column5:5 + │ └── percentile-cont [as=percentile_cont:7, frame="range from unbounded to unbounded"] + │ ├── column6:6 │ └── b:2 └── aggregations - └── const-agg [as=percentile_cont:6] - └── percentile_cont:6 + └── const-agg [as=percentile_cont:7] + └── percentile_cont:7 build SELECT percentile_disc(ARRAY[0.90, 0.95]::float[]) WITHIN GROUP (ORDER BY b) FROM abc; ---- scalar-group-by - ├── columns: percentile_disc:6 + ├── columns: percentile_disc:7 ├── window partition=() ordering=+2 - │ ├── columns: a:1!null b:2 c:3 d:4 column5:5!null percentile_disc:6 + │ ├── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 column6:6!null percentile_disc:7 │ ├── project - │ │ ├── columns: column5:5!null a:1!null b:2 c:3 d:4 + │ │ ├── columns: column6:6!null a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 │ │ ├── scan abc - │ │ │ └── columns: a:1!null b:2 c:3 d:4 + │ │ │ └── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 │ │ └── projections - │ │ └── ARRAY[0.90,0.95]::FLOAT8[] [as=column5:5] + │ │ └── ARRAY[0.90,0.95]::FLOAT8[] [as=column6:6] │ └── windows - │ └── percentile-disc [as=percentile_disc:6, frame="range from unbounded to unbounded"] - │ ├── column5:5 + │ └── percentile-disc [as=percentile_disc:7, frame="range from unbounded to unbounded"] + │ ├── column6:6 │ └── b:2 └── aggregations - └── const-agg [as=percentile_disc:6] - └── percentile_disc:6 + └── const-agg [as=percentile_disc:7] + └── percentile_disc:7 build SELECT percentile_cont(ARRAY[0.90, 0.95]::float[]) WITHIN GROUP (ORDER BY b) FROM abc; ---- scalar-group-by - ├── columns: percentile_cont:6 + ├── columns: percentile_cont:7 ├── window partition=() ordering=+2 - │ ├── columns: a:1!null b:2 c:3 d:4 column5:5!null percentile_cont:6 + │ ├── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 column6:6!null percentile_cont:7 │ ├── project - │ │ ├── columns: column5:5!null a:1!null b:2 c:3 d:4 + │ │ ├── columns: column6:6!null a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 │ │ ├── scan abc - │ │ │ └── columns: a:1!null b:2 c:3 d:4 + │ │ │ └── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 │ │ └── projections - │ │ └── ARRAY[0.90,0.95]::FLOAT8[] [as=column5:5] + │ │ └── ARRAY[0.90,0.95]::FLOAT8[] [as=column6:6] │ └── windows - │ └── percentile-cont [as=percentile_cont:6, frame="range from unbounded to unbounded"] - │ ├── column5:5 + │ └── percentile-cont [as=percentile_cont:7, frame="range from unbounded to unbounded"] + │ ├── column6:6 │ └── b:2 └── aggregations - └── const-agg [as=percentile_cont:6] - └── percentile_cont:6 + └── const-agg [as=percentile_cont:7] + └── percentile_cont:7 # Regression test for #46914. exec-ddl @@ -3951,16 +3972,16 @@ SELECT * FROM t0 GROUP BY t0.c0 HAVING min((CASE WHEN false THEN NULL END):::FLO project ├── columns: c0:1(int) └── select - ├── columns: c0:1(int) min:4(float!null) + ├── columns: c0:1(int) min:5(float!null) ├── group-by - │ ├── columns: c0:1(int) min:4(float) + │ ├── columns: c0:1(int) min:5(float) │ ├── grouping columns: c0:1(int) │ ├── project - │ │ ├── columns: column3:3(float) c0:1(int) + │ │ ├── columns: column4:4(float) c0:1(int) │ │ ├── scan t0 - │ │ │ └── columns: c0:1(int) rowid:2(int!null) + │ │ │ └── columns: c0:1(int) rowid:2(int!null) crdb_internal_mvcc_timestamp:3(decimal) │ │ └── projections - │ │ └── case [as=column3:3, type=float] + │ │ └── case [as=column4:4, type=float] │ │ ├── true [type=bool] │ │ ├── when [type=float] │ │ │ ├── false [type=bool] @@ -3968,9 +3989,9 @@ project │ │ │ └── null [type=unknown] │ │ └── null [type=float] │ └── aggregations - │ └── min [as=min:4, type=float] - │ └── variable: column3:3 [type=float] + │ └── min [as=min:5, type=float] + │ └── variable: column4:4 [type=float] └── filters └── ne [type=bool] - ├── variable: min:4 [type=float] + ├── variable: min:5 [type=float] └── const: NaN [type=float] diff --git a/pkg/sql/opt/optbuilder/testdata/alter_table b/pkg/sql/opt/optbuilder/testdata/alter_table index 92b8ddd6f591..e87a2ed4d776 100644 --- a/pkg/sql/opt/optbuilder/testdata/alter_table +++ b/pkg/sql/opt/optbuilder/testdata/alter_table @@ -61,12 +61,12 @@ build ALTER INDEX abc@bc SPLIT AT SELECT b FROM abc ORDER BY a ---- alter-table-split abc@bc - ├── columns: key:4 pretty:5 split_enforced_until:6 + ├── columns: key:5 pretty:6 split_enforced_until:7 ├── project │ ├── columns: b:2 [hidden: a:1!null] │ ├── ordering: +1 │ └── scan abc - │ ├── columns: a:1!null b:2 c:3 + │ ├── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 │ └── ordering: +1 └── CAST(NULL AS STRING) @@ -77,7 +77,7 @@ ALTER TABLE abc UNSPLIT AT VALUES (1), (2) alter-table-unsplit abc ├── columns: key:1 pretty:2 └── values - ├── columns: column1:6!null + ├── columns: column1:7!null ├── (1,) └── (2,) @@ -104,7 +104,7 @@ ALTER INDEX abc@bc UNSPLIT AT VALUES (1, 'foo'), (2, 'bar') alter-table-unsplit abc@bc ├── columns: key:1 pretty:2 └── values - ├── columns: column1:6!null column2:7!null + ├── columns: column1:7!null column2:8!null ├── (1, 'foo') └── (2, 'bar') @@ -119,11 +119,11 @@ ALTER INDEX abc@bc UNSPLIT AT SELECT b FROM abc ORDER BY a alter-table-unsplit abc@bc ├── columns: key:1 pretty:2 └── project - ├── columns: b:7 [hidden: a:6!null] - ├── ordering: +6 + ├── columns: b:8 [hidden: a:7!null] + ├── ordering: +7 └── scan abc - ├── columns: a:6!null b:7 c:8 - └── ordering: +6 + ├── columns: a:7!null b:8 c:9 crdb_internal_mvcc_timestamp:10 + └── ordering: +7 # Tests for ALTER TABLE EXPERIMENTAL_RELOCATE. build diff --git a/pkg/sql/opt/optbuilder/testdata/create_view b/pkg/sql/opt/optbuilder/testdata/create_view index 79b0ec605c25..e099a384e643 100644 --- a/pkg/sql/opt/optbuilder/testdata/create_view +++ b/pkg/sql/opt/optbuilder/testdata/create_view @@ -66,7 +66,7 @@ CREATE VIEW v1 AS SELECT av.a, ab.a FROM av, ab ---- create-view t.public.v1 ├── SELECT av.a, ab.a FROM t.public.av, t.public.ab - ├── columns: a:1 a:3 + ├── columns: a:1 a:4 └── dependencies ├── av └── ab [columns: a] @@ -77,7 +77,7 @@ CREATE VIEW v1 AS SELECT a, table_schema FROM ab, information_schema.columns ---- create-view t.public.v1 ├── SELECT a, table_schema FROM t.public.ab, "".information_schema.columns - ├── columns: a:1 table_schema:5 + ├── columns: a:1 table_schema:6 └── dependencies └── ab [columns: a] @@ -146,7 +146,7 @@ WITH cd AS (SELECT a, b FROM ab) ---- create-view t.public.v8 ├── WITH cd AS (SELECT a, b FROM t.public.ab) SELECT a + b FROM cd - ├── columns: "?column?":5 + ├── columns: "?column?":6 └── dependencies └── ab [columns: a b] @@ -184,7 +184,7 @@ CREATE VIEW v10 as SELECT 1 FROM ab JOIN cd ON ab.a = cd.c ---- create-view t.public.v10 ├── SELECT 1 FROM t.public.ab JOIN t.public.cd ON ab.a = cd.c - ├── columns: "?column?":5 + ├── columns: "?column?":7 └── dependencies ├── ab [columns: a] └── cd [columns: c] @@ -199,7 +199,7 @@ CREATE VIEW v11 as SELECT 1 FROM ab NATURAL JOIN ac ---- create-view t.public.v11 ├── SELECT 1 FROM t.public.ab NATURAL JOIN t.public.ac - ├── columns: "?column?":6 + ├── columns: "?column?":8 └── dependencies ├── ab [columns: a] └── ac [columns: a] @@ -210,7 +210,7 @@ CREATE VIEW v12 as SELECT 1 FROM ab JOIN ac USING (a) ---- create-view t.public.v12 ├── SELECT 1 FROM t.public.ab JOIN t.public.ac USING (a) - ├── columns: "?column?":6 + ├── columns: "?column?":8 └── dependencies ├── ab [columns: a] └── ac [columns: a] @@ -231,7 +231,7 @@ CREATE VIEW v14 AS SELECT sum(a) FROM ab; ---- create-view t.public.v14 ├── SELECT sum(a) FROM t.public.ab - ├── columns: sum:3 + ├── columns: sum:4 └── dependencies └── ab [columns: a] @@ -241,7 +241,7 @@ CREATE VIEW v15 AS SELECT sum(a) OVER (PARTITION by b) FROM ab; ---- create-view t.public.v15 ├── SELECT sum(a) OVER (PARTITION BY b) FROM t.public.ab - ├── columns: sum:3 + ├── columns: sum:4 └── dependencies └── ab [columns: a b] @@ -275,7 +275,7 @@ CREATE VIEW v17 AS SELECT percentile_cont(0.50) WITHIN GROUP (ORDER BY f) FROM t ---- create-view t.public.v17 ├── SELECT percentile_cont(0.50) WITHIN GROUP (ORDER BY f) FROM t.public.tf - ├── columns: percentile_cont:4 + ├── columns: percentile_cont:5 └── dependencies └── tf [columns: f] @@ -285,7 +285,7 @@ CREATE VIEW v18 AS SELECT ab.a, ab2.b FROM ab, ab as ab2 ---- create-view t.public.v18 ├── SELECT ab.a, ab2.b FROM t.public.ab, t.public.ab AS ab2 - ├── columns: a:1 b:4 + ├── columns: a:1 b:5 └── dependencies ├── ab [columns: a] └── ab [columns: b] @@ -295,7 +295,7 @@ CREATE VIEW v19 AS SELECT 1 FROM (SELECT a FROM ab) t1 JOIN (SELECT b FROM AB) t ---- create-view t.public.v19 ├── SELECT 1 FROM (SELECT a FROM t.public.ab) AS t1 JOIN (SELECT b FROM t.public.ab) AS t2 ON t1.a = t2.b - ├── columns: "?column?":5 + ├── columns: "?column?":7 └── dependencies ├── ab [columns: a] └── ab [columns: b] @@ -306,7 +306,7 @@ CREATE VIEW v20 AS SELECT a + b FROM ab ---- create-view t.public.v20 ├── SELECT a + b FROM t.public.ab - ├── columns: "?column?":3 + ├── columns: "?column?":4 └── dependencies └── ab [columns: a b] @@ -320,7 +320,7 @@ CREATE VIEW v21 AS SELECT sum(a) OVER (PARTITION BY b ORDER BY c) FROM abc ---- create-view t.public.v21 ├── SELECT sum(a) OVER (PARTITION BY b ORDER BY c) FROM t.public.abc - ├── columns: sum:5 + ├── columns: sum:6 └── dependencies └── abc [columns: a b c] diff --git a/pkg/sql/opt/optbuilder/testdata/delete b/pkg/sql/opt/optbuilder/testdata/delete index 78893b65f807..1d50446a802f 100644 --- a/pkg/sql/opt/optbuilder/testdata/delete +++ b/pkg/sql/opt/optbuilder/testdata/delete @@ -42,14 +42,14 @@ DELETE FROM abcde ---- delete abcde ├── columns: - ├── fetch columns: a:7 b:8 c:9 d:10 e:11 rowid:12 + ├── fetch columns: a:8 b:9 c:10 d:11 e:12 rowid:13 └── scan abcde - ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 └── computed column expressions - ├── d:10 - │ └── (b:8 + c:9) + 1 - └── e:11 - └── a:7 + ├── d:11 + │ └── (b:9 + c:10) + 1 + └── e:12 + └── a:8 # Use WHERE, ORDER BY, LIMIT. build @@ -57,25 +57,25 @@ DELETE FROM abcde WHERE a>0 ORDER BY a LIMIT 10 ---- delete abcde ├── columns: - ├── fetch columns: a:7 b:8 c:9 d:10 e:11 rowid:12 + ├── fetch columns: a:8 b:9 c:10 d:11 e:12 rowid:13 └── limit - ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null - ├── internal-ordering: +7 + ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 + ├── internal-ordering: +8 ├── sort - │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null - │ ├── ordering: +7 + │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 + │ ├── ordering: +8 │ ├── limit hint: 10.00 │ └── select - │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ ├── scan abcde - │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ └── computed column expressions - │ │ ├── d:10 - │ │ │ └── (b:8 + c:9) + 1 - │ │ └── e:11 - │ │ └── a:7 + │ │ ├── d:11 + │ │ │ └── (b:9 + c:10) + 1 + │ │ └── e:12 + │ │ └── a:8 │ └── filters - │ └── a:7 > 0 + │ └── a:8 > 0 └── 10 # Use aliased table name. @@ -84,25 +84,25 @@ DELETE FROM abcde AS foo WHERE foo.a>0 ORDER BY foo.a LIMIT 10 ---- delete foo ├── columns: - ├── fetch columns: a:7 b:8 c:9 d:10 e:11 rowid:12 + ├── fetch columns: a:8 b:9 c:10 d:11 e:12 rowid:13 └── limit - ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null - ├── internal-ordering: +7 + ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 + ├── internal-ordering: +8 ├── sort - │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null - │ ├── ordering: +7 + │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 + │ ├── ordering: +8 │ ├── limit hint: 10.00 │ └── select - │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ ├── scan foo - │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ └── computed column expressions - │ │ ├── d:10 - │ │ │ └── (b:8 + c:9) + 1 - │ │ └── e:11 - │ │ └── a:7 + │ │ ├── d:11 + │ │ │ └── (b:9 + c:10) + 1 + │ │ └── e:12 + │ │ └── a:8 │ └── filters - │ └── a:7 > 0 + │ └── a:8 > 0 └── 10 # DELETE with index hints. @@ -121,9 +121,9 @@ DELETE FROM xyzw@primary ---- delete xyzw ├── columns: - ├── fetch columns: x:5 y:6 z:7 w:8 + ├── fetch columns: x:6 y:7 z:8 w:9 └── scan xyzw - ├── columns: x:5!null y:6 z:7 w:8 + ├── columns: x:6!null y:7 z:8 w:9 crdb_internal_mvcc_timestamp:10 └── flags: force-index=primary build @@ -131,9 +131,9 @@ DELETE FROM xyzw@foo ---- delete xyzw ├── columns: - ├── fetch columns: x:5 y:6 z:7 w:8 + ├── fetch columns: x:6 y:7 z:8 w:9 └── scan xyzw - ├── columns: x:5!null y:6 z:7 w:8 + ├── columns: x:6!null y:7 z:8 w:9 crdb_internal_mvcc_timestamp:10 └── flags: force-index=foo build @@ -141,9 +141,9 @@ DELETE FROM xyzw@{FORCE_INDEX=foo,ASC} ---- delete xyzw ├── columns: - ├── fetch columns: x:5 y:6 z:7 w:8 + ├── fetch columns: x:6 y:7 z:8 w:9 └── scan xyzw - ├── columns: x:5!null y:6 z:7 w:8 + ├── columns: x:6!null y:7 z:8 w:9 crdb_internal_mvcc_timestamp:10 └── flags: force-index=foo,fwd build @@ -151,9 +151,9 @@ DELETE FROM xyzw@{FORCE_INDEX=foo,DESC} ---- delete xyzw ├── columns: - ├── fetch columns: x:5 y:6 z:7 w:8 + ├── fetch columns: x:6 y:7 z:8 w:9 └── scan xyzw,rev - ├── columns: x:5!null y:6 z:7 w:8 + ├── columns: x:6!null y:7 z:8 w:9 crdb_internal_mvcc_timestamp:10 └── flags: force-index=foo,rev build @@ -161,9 +161,9 @@ DELETE FROM xyzw@{NO_INDEX_JOIN} ---- delete xyzw ├── columns: - ├── fetch columns: x:5 y:6 z:7 w:8 + ├── fetch columns: x:6 y:7 z:8 w:9 └── scan xyzw - ├── columns: x:5!null y:6 z:7 w:8 + ├── columns: x:6!null y:7 z:8 w:9 crdb_internal_mvcc_timestamp:10 └── flags: no-index-join build @@ -177,24 +177,24 @@ DELETE FROM xyz WHERE x=$1 ORDER BY y+$2 DESC LIMIT 2 ---- delete xyz ├── columns: - ├── fetch columns: x:4 y:5 z:6 + ├── fetch columns: x:5 y:6 z:7 └── limit - ├── columns: x:4!null y:5 z:6 column7:7 - ├── internal-ordering: -7 + ├── columns: x:5!null y:6 z:7 crdb_internal_mvcc_timestamp:8 column9:9 + ├── internal-ordering: -9 ├── sort - │ ├── columns: x:4!null y:5 z:6 column7:7 - │ ├── ordering: -7 + │ ├── columns: x:5!null y:6 z:7 crdb_internal_mvcc_timestamp:8 column9:9 + │ ├── ordering: -9 │ ├── limit hint: 2.00 │ └── project - │ ├── columns: column7:7 x:4!null y:5 z:6 + │ ├── columns: column9:9 x:5!null y:6 z:7 crdb_internal_mvcc_timestamp:8 │ ├── select - │ │ ├── columns: x:4!null y:5 z:6 + │ │ ├── columns: x:5!null y:6 z:7 crdb_internal_mvcc_timestamp:8 │ │ ├── scan xyz - │ │ │ └── columns: x:4!null y:5 z:6 + │ │ │ └── columns: x:5!null y:6 z:7 crdb_internal_mvcc_timestamp:8 │ │ └── filters - │ │ └── x:4 = $1 + │ │ └── x:5 = $1 │ └── projections - │ └── y:5 + $2 [as=column7:7] + │ └── y:6 + $2 [as=column9:9] └── 2 @@ -206,25 +206,25 @@ with &1 (cte) ├── project │ ├── columns: xyz.x:1!null │ └── scan xyz - │ └── columns: xyz.x:1!null y:2 z:3 + │ └── columns: xyz.x:1!null y:2 z:3 xyz.crdb_internal_mvcc_timestamp:4 └── delete abcde ├── columns: - ├── fetch columns: a:10 b:11 c:12 d:13 e:14 rowid:15 + ├── fetch columns: a:12 b:13 c:14 d:15 e:16 rowid:17 └── select - ├── columns: a:10!null b:11 c:12 d:13 e:14 rowid:15!null + ├── columns: a:12!null b:13 c:14 d:15 e:16 rowid:17!null abcde.crdb_internal_mvcc_timestamp:18 ├── scan abcde - │ ├── columns: a:10!null b:11 c:12 d:13 e:14 rowid:15!null + │ ├── columns: a:12!null b:13 c:14 d:15 e:16 rowid:17!null abcde.crdb_internal_mvcc_timestamp:18 │ └── computed column expressions - │ ├── d:13 - │ │ └── (b:11 + c:12) + 1 - │ └── e:14 - │ └── a:10 + │ ├── d:15 + │ │ └── (b:13 + c:14) + 1 + │ └── e:16 + │ └── a:12 └── filters └── exists └── with-scan &1 (cte) - ├── columns: x:16!null + ├── columns: x:19!null └── mapping: - └── xyz.x:1 => x:16 + └── xyz.x:1 => x:19 # Unknown target table. build @@ -247,43 +247,43 @@ with &1 │ ├── columns: abcde.a:1!null abcde.b:2!null abcde.c:3 abcde.d:4 abcde.e:5 │ └── delete abcde │ ├── columns: abcde.a:1!null abcde.b:2!null abcde.c:3 abcde.d:4 abcde.e:5 rowid:6!null - │ ├── fetch columns: abcde.a:7 abcde.b:8 abcde.c:9 abcde.d:10 abcde.e:11 rowid:12 + │ ├── fetch columns: abcde.a:8 abcde.b:9 abcde.c:10 abcde.d:11 abcde.e:12 rowid:13 │ └── select - │ ├── columns: abcde.a:7!null abcde.b:8!null abcde.c:9 abcde.d:10 abcde.e:11 rowid:12!null + │ ├── columns: abcde.a:8!null abcde.b:9!null abcde.c:10 abcde.d:11 abcde.e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ ├── scan abcde - │ │ ├── columns: abcde.a:7!null abcde.b:8 abcde.c:9 abcde.d:10 abcde.e:11 rowid:12!null + │ │ ├── columns: abcde.a:8!null abcde.b:9 abcde.c:10 abcde.d:11 abcde.e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ └── computed column expressions - │ │ ├── abcde.d:10 - │ │ │ └── (abcde.b:8 + abcde.c:9) + 1 - │ │ └── abcde.e:11 - │ │ └── abcde.a:7 + │ │ ├── abcde.d:11 + │ │ │ └── (abcde.b:9 + abcde.c:10) + 1 + │ │ └── abcde.e:12 + │ │ └── abcde.a:8 │ └── filters - │ └── abcde.a:7 = abcde.b:8 + │ └── abcde.a:8 = abcde.b:9 └── with &2 (cte) ├── project - │ ├── columns: b:14!null + │ ├── columns: b:16!null │ └── with-scan &1 - │ ├── columns: a:13!null b:14!null c:15 d:16 e:17 + │ ├── columns: a:15!null b:16!null c:17 d:18 e:19 │ └── mapping: - │ ├── abcde.a:1 => a:13 - │ ├── abcde.b:2 => b:14 - │ ├── abcde.c:3 => c:15 - │ ├── abcde.d:4 => d:16 - │ └── abcde.e:5 => e:17 + │ ├── abcde.a:1 => a:15 + │ ├── abcde.b:2 => b:16 + │ ├── abcde.c:3 => c:17 + │ ├── abcde.d:4 => d:18 + │ └── abcde.e:5 => e:19 └── delete abcde ├── columns: - ├── fetch columns: abcde.a:24 abcde.b:25 abcde.c:26 abcde.d:27 abcde.e:28 rowid:29 + ├── fetch columns: abcde.a:27 abcde.b:28 abcde.c:29 abcde.d:30 abcde.e:31 rowid:32 └── select - ├── columns: abcde.a:24!null abcde.b:25!null abcde.c:26 abcde.d:27 abcde.e:28 rowid:29!null + ├── columns: abcde.a:27!null abcde.b:28!null abcde.c:29 abcde.d:30 abcde.e:31 rowid:32!null crdb_internal_mvcc_timestamp:33 ├── scan abcde - │ ├── columns: abcde.a:24!null abcde.b:25 abcde.c:26 abcde.d:27 abcde.e:28 rowid:29!null + │ ├── columns: abcde.a:27!null abcde.b:28 abcde.c:29 abcde.d:30 abcde.e:31 rowid:32!null crdb_internal_mvcc_timestamp:33 │ └── computed column expressions - │ ├── abcde.d:27 - │ │ └── (abcde.b:25 + abcde.c:26) + 1 - │ └── abcde.e:28 - │ └── abcde.a:24 + │ ├── abcde.d:30 + │ │ └── (abcde.b:28 + abcde.c:29) + 1 + │ └── abcde.e:31 + │ └── abcde.a:27 └── filters - └── abcde.a:24 = abcde.b:25 + └── abcde.a:27 = abcde.b:28 # With alias, original table name should be inaccessible. build @@ -309,83 +309,83 @@ project ├── columns: a:1!null b:2 c:3 d:4 e:5 └── delete abcde ├── columns: a:1!null b:2 c:3 d:4 e:5 rowid:6!null - ├── fetch columns: a:7 b:8 c:9 d:10 e:11 rowid:12 + ├── fetch columns: a:8 b:9 c:10 d:11 e:12 rowid:13 └── select - ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 ├── scan abcde - │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ └── computed column expressions - │ ├── d:10 - │ │ └── (b:8 + c:9) + 1 - │ └── e:11 - │ └── a:7 + │ ├── d:11 + │ │ └── (b:9 + c:10) + 1 + │ └── e:12 + │ └── a:8 └── filters - └── a:7 = 1 + └── a:8 = 1 # Return values from aliased table. build DELETE FROM abcde AS foo WHERE a=1 RETURNING foo.a+1, foo.b * foo.d ---- project - ├── columns: "?column?":13!null "?column?":14 + ├── columns: "?column?":15!null "?column?":16 ├── delete foo │ ├── columns: a:1!null b:2 c:3 d:4 e:5 rowid:6!null - │ ├── fetch columns: a:7 b:8 c:9 d:10 e:11 rowid:12 + │ ├── fetch columns: a:8 b:9 c:10 d:11 e:12 rowid:13 │ └── select - │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ ├── scan foo - │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ └── computed column expressions - │ │ ├── d:10 - │ │ │ └── (b:8 + c:9) + 1 - │ │ └── e:11 - │ │ └── a:7 + │ │ ├── d:11 + │ │ │ └── (b:9 + c:10) + 1 + │ │ └── e:12 + │ │ └── a:8 │ └── filters - │ └── a:7 = 1 + │ └── a:8 = 1 └── projections - ├── a:1 + 1 [as="?column?":13] - └── b:2 * d:4 [as="?column?":14] + ├── a:1 + 1 [as="?column?":15] + └── b:2 * d:4 [as="?column?":16] # Use returning DELETE as a FROM expression. build SELECT a, d FROM [DELETE FROM abcde WHERE a>0 ORDER BY b LIMIT 10 RETURNING *] ---- with &1 - ├── columns: a:13!null d:16 + ├── columns: a:15!null d:18 ├── project │ ├── columns: abcde.a:1!null abcde.b:2 abcde.c:3 abcde.d:4 abcde.e:5 │ └── delete abcde │ ├── columns: abcde.a:1!null abcde.b:2 abcde.c:3 abcde.d:4 abcde.e:5 rowid:6!null - │ ├── fetch columns: abcde.a:7 abcde.b:8 abcde.c:9 abcde.d:10 abcde.e:11 rowid:12 + │ ├── fetch columns: abcde.a:8 abcde.b:9 abcde.c:10 abcde.d:11 abcde.e:12 rowid:13 │ └── limit - │ ├── columns: abcde.a:7!null abcde.b:8 abcde.c:9 abcde.d:10 abcde.e:11 rowid:12!null - │ ├── internal-ordering: +8 + │ ├── columns: abcde.a:8!null abcde.b:9 abcde.c:10 abcde.d:11 abcde.e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 + │ ├── internal-ordering: +9 │ ├── sort - │ │ ├── columns: abcde.a:7!null abcde.b:8 abcde.c:9 abcde.d:10 abcde.e:11 rowid:12!null - │ │ ├── ordering: +8 + │ │ ├── columns: abcde.a:8!null abcde.b:9 abcde.c:10 abcde.d:11 abcde.e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 + │ │ ├── ordering: +9 │ │ ├── limit hint: 10.00 │ │ └── select - │ │ ├── columns: abcde.a:7!null abcde.b:8 abcde.c:9 abcde.d:10 abcde.e:11 rowid:12!null + │ │ ├── columns: abcde.a:8!null abcde.b:9 abcde.c:10 abcde.d:11 abcde.e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ ├── scan abcde - │ │ │ ├── columns: abcde.a:7!null abcde.b:8 abcde.c:9 abcde.d:10 abcde.e:11 rowid:12!null + │ │ │ ├── columns: abcde.a:8!null abcde.b:9 abcde.c:10 abcde.d:11 abcde.e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ │ └── computed column expressions - │ │ │ ├── abcde.d:10 - │ │ │ │ └── (abcde.b:8 + abcde.c:9) + 1 - │ │ │ └── abcde.e:11 - │ │ │ └── abcde.a:7 + │ │ │ ├── abcde.d:11 + │ │ │ │ └── (abcde.b:9 + abcde.c:10) + 1 + │ │ │ └── abcde.e:12 + │ │ │ └── abcde.a:8 │ │ └── filters - │ │ └── abcde.a:7 > 0 + │ │ └── abcde.a:8 > 0 │ └── 10 └── project - ├── columns: a:13!null d:16 + ├── columns: a:15!null d:18 └── with-scan &1 - ├── columns: a:13!null b:14 c:15 d:16 e:17 + ├── columns: a:15!null b:16 c:17 d:18 e:19 └── mapping: - ├── abcde.a:1 => a:13 - ├── abcde.b:2 => b:14 - ├── abcde.c:3 => c:15 - ├── abcde.d:4 => d:16 - └── abcde.e:5 => e:17 + ├── abcde.a:1 => a:15 + ├── abcde.b:2 => b:16 + ├── abcde.c:3 => c:17 + ├── abcde.d:4 => d:18 + └── abcde.e:5 => e:19 # ------------------------------------------------------------------------------ # Tests with mutations. @@ -397,13 +397,13 @@ DELETE FROM mutation WHERE m=1 ---- delete mutation ├── columns: - ├── fetch columns: m:5 n:6 o:7 p:8 + ├── fetch columns: m:6 n:7 o:8 p:9 └── select - ├── columns: m:5!null n:6 o:7 p:8 + ├── columns: m:6!null n:7 o:8 p:9 crdb_internal_mvcc_timestamp:10 ├── scan mutation - │ └── columns: m:5!null n:6 o:7 p:8 + │ └── columns: m:6!null n:7 o:8 p:9 crdb_internal_mvcc_timestamp:10 └── filters - └── m:5 = 1 + └── m:6 = 1 # With RETURNING clause. build @@ -411,13 +411,13 @@ DELETE FROM mutation WHERE m=1 RETURNING * ---- delete mutation ├── columns: m:1!null n:2 - ├── fetch columns: m:5 n:6 o:7 p:8 + ├── fetch columns: m:6 n:7 o:8 p:9 └── select - ├── columns: m:5!null n:6 o:7 p:8 + ├── columns: m:6!null n:7 o:8 p:9 crdb_internal_mvcc_timestamp:10 ├── scan mutation - │ └── columns: m:5!null n:6 o:7 p:8 + │ └── columns: m:6!null n:7 o:8 p:9 crdb_internal_mvcc_timestamp:10 └── filters - └── m:5 = 1 + └── m:6 = 1 # Try to return a mutation column. @@ -460,20 +460,20 @@ DELETE FROM partial_indexes ---- delete partial_indexes ├── columns: - ├── fetch columns: a:4 b:5 c:6 - ├── partial index del columns: partial_index_del1:7 partial_index_del2:8 partial_index_del3:9 partial_index_del4:10 + ├── fetch columns: a:5 b:6 c:7 + ├── partial index del columns: partial_index_del1:9 partial_index_del2:10 partial_index_del3:11 partial_index_del4:12 └── project - ├── columns: partial_index_del1:7 partial_index_del2:8 partial_index_del3:9 partial_index_del4:10 a:4!null b:5 c:6 + ├── columns: partial_index_del1:9 partial_index_del2:10 partial_index_del3:11 partial_index_del4:12 a:5!null b:6 c:7 crdb_internal_mvcc_timestamp:8 ├── scan partial_indexes - │ ├── columns: a:4!null b:5 c:6 + │ ├── columns: a:5!null b:6 c:7 crdb_internal_mvcc_timestamp:8 │ └── partial index predicates │ ├── secondary: filters - │ │ └── c:6 = 'foo' + │ │ └── c:7 = 'foo' │ └── secondary: filters - │ ├── a:4 > b:5 - │ └── c:6 = 'bar' + │ ├── a:5 > b:6 + │ └── c:7 = 'bar' └── projections - ├── c:6 = 'foo' [as=partial_index_del1:7] - ├── (a:4 > b:5) AND (c:6 = 'bar') [as=partial_index_del2:8] - ├── c:6 = 'delete-only' [as=partial_index_del3:9] - └── c:6 = 'write-only' [as=partial_index_del4:10] + ├── c:7 = 'foo' [as=partial_index_del1:9] + ├── (a:5 > b:6) AND (c:7 = 'bar') [as=partial_index_del2:10] + ├── c:7 = 'delete-only' [as=partial_index_del3:11] + └── c:7 = 'write-only' [as=partial_index_del4:12] diff --git a/pkg/sql/opt/optbuilder/testdata/distinct b/pkg/sql/opt/optbuilder/testdata/distinct index b9517523e278..53eb4fd2ce72 100644 --- a/pkg/sql/opt/optbuilder/testdata/distinct +++ b/pkg/sql/opt/optbuilder/testdata/distinct @@ -19,7 +19,7 @@ SELECT y, z FROM xyz project ├── columns: y:2 z:3 └── scan xyz - └── columns: x:1!null y:2 z:3 + └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 build SELECT DISTINCT y, z FROM xyz @@ -30,7 +30,7 @@ distinct-on └── project ├── columns: y:2 z:3 └── scan xyz - └── columns: x:1!null y:2 z:3 + └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 build SELECT y FROM (SELECT DISTINCT y, z FROM xyz) @@ -43,7 +43,7 @@ project └── project ├── columns: y:2 z:3 └── scan xyz - └── columns: x:1!null y:2 z:3 + └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 build SELECT DISTINCT y, z FROM xyz ORDER BY z @@ -58,7 +58,7 @@ distinct-on └── project ├── columns: y:2 z:3 └── scan xyz - └── columns: x:1!null y:2 z:3 + └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 build SELECT DISTINCT y, z FROM xyz ORDER BY y @@ -73,7 +73,7 @@ distinct-on └── project ├── columns: y:2 z:3 └── scan xyz - └── columns: x:1!null y:2 z:3 + └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 build SELECT DISTINCT y, z FROM xyz ORDER BY y, z @@ -88,41 +88,41 @@ distinct-on └── project ├── columns: y:2 z:3 └── scan xyz - └── columns: x:1!null y:2 z:3 + └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 build SELECT DISTINCT y + x AS r FROM xyz ORDER by (y + x) ---- distinct-on - ├── columns: r:4 - ├── grouping columns: r:4 - ├── ordering: +4 + ├── columns: r:5 + ├── grouping columns: r:5 + ├── ordering: +5 └── sort - ├── columns: r:4 - ├── ordering: +4 + ├── columns: r:5 + ├── ordering: +5 └── project - ├── columns: r:4 + ├── columns: r:5 ├── scan xyz - │ └── columns: x:1!null y:2 z:3 + │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 └── projections - └── y:2 + x:1 [as=r:4] + └── y:2 + x:1 [as=r:5] build SELECT DISTINCT y + x AS r FROM xyz ORDER BY y + x ---- distinct-on - ├── columns: r:4 - ├── grouping columns: r:4 - ├── ordering: +4 + ├── columns: r:5 + ├── grouping columns: r:5 + ├── ordering: +5 └── sort - ├── columns: r:4 - ├── ordering: +4 + ├── columns: r:5 + ├── ordering: +5 └── project - ├── columns: r:4 + ├── columns: r:5 ├── scan xyz - │ └── columns: x:1!null y:2 z:3 + │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 └── projections - └── y:2 + x:1 [as=r:4] + └── y:2 + x:1 [as=r:5] build SELECT DISTINCT y + z FROM xyz ORDER BY y + z @@ -150,26 +150,26 @@ sort └── project ├── columns: y:2 └── scan xyz - └── columns: x:1!null y:2 z:3 + └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 build SELECT DISTINCT (y,z) AS r FROM xyz ---- distinct-on - ├── columns: r:4 - ├── grouping columns: r:4 + ├── columns: r:5 + ├── grouping columns: r:5 └── project - ├── columns: r:4 + ├── columns: r:5 ├── scan xyz - │ └── columns: x:1!null y:2 z:3 + │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 └── projections - └── (y:2, z:3) [as=r:4] + └── (y:2, z:3) [as=r:5] build SELECT count(*) FROM (SELECT DISTINCT y FROM xyz) ---- scalar-group-by - ├── columns: count:4!null + ├── columns: count:5!null ├── project │ └── distinct-on │ ├── columns: y:2 @@ -177,9 +177,9 @@ scalar-group-by │ └── project │ ├── columns: y:2 │ └── scan xyz - │ └── columns: x:1!null y:2 z:3 + │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 └── aggregations - └── count-rows [as=count_rows:4] + └── count-rows [as=count_rows:5] build SELECT DISTINCT x FROM xyz WHERE x > 0 @@ -190,9 +190,9 @@ distinct-on └── project ├── columns: x:1!null └── select - ├── columns: x:1!null y:2 z:3 + ├── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 ├── scan xyz - │ └── columns: x:1!null y:2 z:3 + │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 └── filters └── x:1 > 0 @@ -205,9 +205,9 @@ distinct-on └── project ├── columns: z:3 └── select - ├── columns: x:1!null y:2 z:3 + ├── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 ├── scan xyz - │ └── columns: x:1!null y:2 z:3 + │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 └── filters └── x:1 > 0 @@ -215,46 +215,46 @@ build SELECT DISTINCT max(x) FROM xyz GROUP BY x ---- distinct-on - ├── columns: max:4!null - ├── grouping columns: max:4!null + ├── columns: max:5!null + ├── grouping columns: max:5!null └── project - ├── columns: max:4!null + ├── columns: max:5!null └── group-by - ├── columns: x:1!null max:4!null + ├── columns: x:1!null max:5!null ├── grouping columns: x:1!null ├── project │ ├── columns: x:1!null │ └── scan xyz - │ └── columns: x:1!null y:2 z:3 + │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 └── aggregations - └── max [as=max:4] + └── max [as=max:5] └── x:1 build SELECT DISTINCT x+y AS r FROM xyz ---- distinct-on - ├── columns: r:4 - ├── grouping columns: r:4 + ├── columns: r:5 + ├── grouping columns: r:5 └── project - ├── columns: r:4 + ├── columns: r:5 ├── scan xyz - │ └── columns: x:1!null y:2 z:3 + │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 └── projections - └── x:1 + y:2 [as=r:4] + └── x:1 + y:2 [as=r:5] build SELECT DISTINCT 3 r FROM xyz ---- distinct-on - ├── columns: r:4!null - ├── grouping columns: r:4!null + ├── columns: r:5!null + ├── grouping columns: r:5!null └── project - ├── columns: r:4!null + ├── columns: r:5!null ├── scan xyz - │ └── columns: x:1!null y:2 z:3 + │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 └── projections - └── 3 [as=r:4] + └── 3 [as=r:5] build SELECT DISTINCT 3 r @@ -273,25 +273,27 @@ build SELECT DISTINCT max(z), x+y AS r, 3 AS s FROM xyz GROUP BY x, y HAVING y > 4 ---- distinct-on - ├── columns: max:4 r:5!null s:6!null - ├── grouping columns: max:4 r:5!null s:6!null + ├── columns: max:5 r:6!null s:7!null + ├── grouping columns: max:5 r:6!null s:7!null └── project - ├── columns: r:5!null s:6!null max:4 + ├── columns: r:6!null s:7!null max:5 ├── select - │ ├── columns: x:1!null y:2!null max:4 + │ ├── columns: x:1!null y:2!null max:5 │ ├── group-by - │ │ ├── columns: x:1!null y:2 max:4 + │ │ ├── columns: x:1!null y:2 max:5 │ │ ├── grouping columns: x:1!null y:2 - │ │ ├── scan xyz - │ │ │ └── columns: x:1!null y:2 z:3 + │ │ ├── project + │ │ │ ├── columns: x:1!null y:2 z:3 + │ │ │ └── scan xyz + │ │ │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 │ │ └── aggregations - │ │ └── max [as=max:4] + │ │ └── max [as=max:5] │ │ └── z:3 │ └── filters │ └── y:2 > 4 └── projections - ├── x:1 + y:2 [as=r:5] - └── 3 [as=s:6] + ├── x:1 + y:2 [as=r:6] + └── 3 [as=s:7] exec-ddl CREATE TABLE abcd ( @@ -308,15 +310,15 @@ build SELECT DISTINCT 1 AS x, d, b FROM abcd ORDER BY d, b ---- distinct-on - ├── columns: x:5!null d:4!null b:2!null - ├── grouping columns: b:2!null d:4!null x:5!null + ├── columns: x:6!null d:4!null b:2!null + ├── grouping columns: b:2!null d:4!null x:6!null ├── ordering: +4,+2 └── sort - ├── columns: b:2!null d:4!null x:5!null - ├── ordering: +4,+2 opt(5) + ├── columns: b:2!null d:4!null x:6!null + ├── ordering: +4,+2 opt(6) └── project - ├── columns: x:5!null b:2!null d:4!null + ├── columns: x:6!null b:2!null d:4!null ├── scan abcd - │ └── columns: a:1!null b:2!null c:3!null d:4!null + │ └── columns: a:1!null b:2!null c:3!null d:4!null crdb_internal_mvcc_timestamp:5 └── projections - └── 1 [as=x:5] + └── 1 [as=x:6] diff --git a/pkg/sql/opt/optbuilder/testdata/distinct_on b/pkg/sql/opt/optbuilder/testdata/distinct_on index 1772b30f5da3..89afafc095ad 100644 --- a/pkg/sql/opt/optbuilder/testdata/distinct_on +++ b/pkg/sql/opt/optbuilder/testdata/distinct_on @@ -35,7 +35,7 @@ distinct-on └── project ├── columns: x:1 y:2 z:3 └── scan xyz - └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null + └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null crdb_internal_mvcc_timestamp:6 build SELECT DISTINCT ON (z, x, y) x FROM xyz @@ -46,7 +46,7 @@ distinct-on └── project ├── columns: x:1 y:2 z:3 └── scan xyz - └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null + └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null crdb_internal_mvcc_timestamp:6 build SELECT DISTINCT ON (b, c, a) a, c, b FROM abc @@ -54,8 +54,10 @@ SELECT DISTINCT ON (b, c, a) a, c, b FROM abc distinct-on ├── columns: a:1!null c:3!null b:2!null ├── grouping columns: a:1!null b:2!null c:3!null - └── scan abc - └── columns: a:1!null b:2!null c:3!null + └── project + ├── columns: a:1!null b:2!null c:3!null + └── scan abc + └── columns: a:1!null b:2!null c:3!null crdb_internal_mvcc_timestamp:4 build SELECT DISTINCT ON (b, c, a) a FROM abc @@ -63,8 +65,10 @@ SELECT DISTINCT ON (b, c, a) a FROM abc distinct-on ├── columns: a:1!null [hidden: b:2!null c:3!null] ├── grouping columns: a:1!null b:2!null c:3!null - └── scan abc - └── columns: a:1!null b:2!null c:3!null + └── project + ├── columns: a:1!null b:2!null c:3!null + └── scan abc + └── columns: a:1!null b:2!null c:3!null crdb_internal_mvcc_timestamp:4 build SELECT DISTINCT ON (c, a, b) b FROM abc ORDER BY b @@ -76,8 +80,10 @@ distinct-on └── sort ├── columns: a:1!null b:2!null c:3!null ├── ordering: +2 - └── scan abc - └── columns: a:1!null b:2!null c:3!null + └── project + ├── columns: a:1!null b:2!null c:3!null + └── scan abc + └── columns: a:1!null b:2!null c:3!null crdb_internal_mvcc_timestamp:4 # 2/3 columns @@ -91,7 +97,7 @@ distinct-on └── project ├── columns: x:1 y:2 └── scan xyz - └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null + └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null crdb_internal_mvcc_timestamp:6 build SELECT DISTINCT ON (y, x) x FROM xyz @@ -102,7 +108,7 @@ distinct-on └── project ├── columns: x:1 y:2 └── scan xyz - └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null + └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null crdb_internal_mvcc_timestamp:6 build SELECT DISTINCT ON (y, x, x, y, x) x, y FROM xyz @@ -113,7 +119,7 @@ distinct-on └── project ├── columns: x:1 y:2 └── scan xyz - └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null + └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null crdb_internal_mvcc_timestamp:6 build SELECT DISTINCT ON(pk1, x) pk1, x FROM xyz ORDER BY pk1 @@ -126,7 +132,7 @@ distinct-on ├── columns: x:1 pk1:4!null ├── ordering: +4 └── scan xyz - ├── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null + ├── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null crdb_internal_mvcc_timestamp:6 └── ordering: +4 build @@ -135,8 +141,10 @@ SELECT DISTINCT ON (a, c) a, b FROM abc distinct-on ├── columns: a:1!null b:2!null [hidden: c:3!null] ├── grouping columns: a:1!null c:3!null - ├── scan abc - │ └── columns: a:1!null b:2!null c:3!null + ├── project + │ ├── columns: a:1!null b:2!null c:3!null + │ └── scan abc + │ └── columns: a:1!null b:2!null c:3!null crdb_internal_mvcc_timestamp:4 └── aggregations └── first-agg [as=b:2] └── b:2 @@ -147,8 +155,10 @@ SELECT DISTINCT ON (c, a) b, c, a FROM abc distinct-on ├── columns: b:2!null c:3!null a:1!null ├── grouping columns: a:1!null c:3!null - ├── scan abc - │ └── columns: a:1!null b:2!null c:3!null + ├── project + │ ├── columns: a:1!null b:2!null c:3!null + │ └── scan abc + │ └── columns: a:1!null b:2!null c:3!null crdb_internal_mvcc_timestamp:4 └── aggregations └── first-agg [as=b:2] └── b:2 @@ -169,7 +179,7 @@ sort └── project ├── columns: x:1 └── scan xyz - └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null + └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null crdb_internal_mvcc_timestamp:6 build SELECT DISTINCT ON (x, z) y, z, x FROM xyz ORDER BY z @@ -184,7 +194,7 @@ distinct-on │ └── project │ ├── columns: x:1 y:2 z:3 │ └── scan xyz - │ └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null + │ └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null crdb_internal_mvcc_timestamp:6 └── aggregations └── first-agg [as=y:2] └── y:2 @@ -203,7 +213,7 @@ distinct-on │ └── project │ ├── columns: x:1 y:2 z:3 │ └── scan xyz - │ └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null + │ └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null crdb_internal_mvcc_timestamp:6 └── aggregations ├── first-agg [as=y:2] │ └── y:2 @@ -218,45 +228,47 @@ build SELECT DISTINCT ON (max(y)) max(x) FROM xyz ---- distinct-on - ├── columns: max:6 [hidden: max:7] - ├── grouping columns: max:7 + ├── columns: max:7 [hidden: max:8] + ├── grouping columns: max:8 ├── scalar-group-by - │ ├── columns: max:6 max:7 + │ ├── columns: max:7 max:8 │ ├── project │ │ ├── columns: x:1 y:2 │ │ └── scan xyz - │ │ └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null + │ │ └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null crdb_internal_mvcc_timestamp:6 │ └── aggregations - │ ├── max [as=max:6] + │ ├── max [as=max:7] │ │ └── x:1 - │ └── max [as=max:7] + │ └── max [as=max:8] │ └── y:2 └── aggregations - └── first-agg [as=max:6] - └── max:6 + └── first-agg [as=max:7] + └── max:7 build SELECT DISTINCT ON(min(a), max(b), min(c)) max(a) FROM abc ---- distinct-on - ├── columns: max:4 [hidden: min:5 max:6 min:7] - ├── grouping columns: min:5 max:6 min:7 + ├── columns: max:5 [hidden: min:6 max:7 min:8] + ├── grouping columns: min:6 max:7 min:8 ├── scalar-group-by - │ ├── columns: max:4 min:5 max:6 min:7 - │ ├── scan abc - │ │ └── columns: a:1!null b:2!null c:3!null + │ ├── columns: max:5 min:6 max:7 min:8 + │ ├── project + │ │ ├── columns: a:1!null b:2!null c:3!null + │ │ └── scan abc + │ │ └── columns: a:1!null b:2!null c:3!null crdb_internal_mvcc_timestamp:4 │ └── aggregations - │ ├── max [as=max:4] + │ ├── max [as=max:5] │ │ └── a:1 - │ ├── min [as=min:5] + │ ├── min [as=min:6] │ │ └── a:1 - │ ├── max [as=max:6] + │ ├── max [as=max:7] │ │ └── b:2 - │ └── min [as=min:7] + │ └── min [as=min:8] │ └── c:3 └── aggregations - └── first-agg [as=max:4] - └── max:4 + └── first-agg [as=max:5] + └── max:5 ################# # With GROUP BY # @@ -266,44 +278,44 @@ build SELECT DISTINCT ON(y) min(x) FROM xyz GROUP BY y ---- distinct-on - ├── columns: min:6 [hidden: y:2] + ├── columns: min:7 [hidden: y:2] ├── grouping columns: y:2 ├── group-by - │ ├── columns: y:2 min:6 + │ ├── columns: y:2 min:7 │ ├── grouping columns: y:2 │ ├── project │ │ ├── columns: x:1 y:2 │ │ └── scan xyz - │ │ └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null + │ │ └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null crdb_internal_mvcc_timestamp:6 │ └── aggregations - │ └── min [as=min:6] + │ └── min [as=min:7] │ └── x:1 └── aggregations - └── first-agg [as=min:6] - └── min:6 + └── first-agg [as=min:7] + └── min:7 build SELECT DISTINCT ON(min(x)) min(x) FROM xyz GROUP BY y HAVING min(x) = 1 ---- distinct-on - ├── columns: min:6!null - ├── grouping columns: min:6!null + ├── columns: min:7!null + ├── grouping columns: min:7!null └── project - ├── columns: min:6!null + ├── columns: min:7!null └── select - ├── columns: y:2 min:6!null + ├── columns: y:2 min:7!null ├── group-by - │ ├── columns: y:2 min:6 + │ ├── columns: y:2 min:7 │ ├── grouping columns: y:2 │ ├── project │ │ ├── columns: x:1 y:2 │ │ └── scan xyz - │ │ └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null + │ │ └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null crdb_internal_mvcc_timestamp:6 │ └── aggregations - │ └── min [as=min:6] + │ └── min [as=min:7] │ └── x:1 └── filters - └── min:6 = 1 + └── min:7 = 1 ######################### # With window functions # @@ -313,16 +325,16 @@ build SELECT DISTINCT ON(row_number() OVER()) y FROM xyz ---- distinct-on - ├── columns: y:2 [hidden: row_number:6] - ├── grouping columns: row_number:6 + ├── columns: y:2 [hidden: row_number:7] + ├── grouping columns: row_number:7 ├── project - │ ├── columns: y:2 row_number:6 + │ ├── columns: y:2 row_number:7 │ └── window partition=() - │ ├── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null row_number:6 + │ ├── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null crdb_internal_mvcc_timestamp:6 row_number:7 │ ├── scan xyz - │ │ └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null + │ │ └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null crdb_internal_mvcc_timestamp:6 │ └── windows - │ └── row-number [as=row_number:6] + │ └── row-number [as=row_number:7] └── aggregations └── first-agg [as=y:2] └── y:2 @@ -340,7 +352,7 @@ distinct-on ├── project │ ├── columns: x:1 y:2 z:3 │ └── scan xyz - │ └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null + │ └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null crdb_internal_mvcc_timestamp:6 └── aggregations ├── first-agg [as=y:2] │ └── y:2 @@ -353,8 +365,10 @@ SELECT DISTINCT ON (1,2,3) a, b, c FROM abc distinct-on ├── columns: a:1!null b:2!null c:3!null ├── grouping columns: a:1!null b:2!null c:3!null - └── scan abc - └── columns: a:1!null b:2!null c:3!null + └── project + ├── columns: a:1!null b:2!null c:3!null + └── scan abc + └── columns: a:1!null b:2!null c:3!null crdb_internal_mvcc_timestamp:4 ######################### # With alias references # @@ -370,7 +384,7 @@ distinct-on ├── project │ ├── columns: x:1 y:2 │ └── scan xyz - │ └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null + │ └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null crdb_internal_mvcc_timestamp:6 └── aggregations └── first-agg [as=y:2] └── y:2 @@ -385,7 +399,7 @@ distinct-on └── project ├── columns: x:1 └── scan xyz - └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null + └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null crdb_internal_mvcc_timestamp:6 ################################## # With nested parentheses/tuples # @@ -400,7 +414,7 @@ distinct-on └── project ├── columns: x:1 y:2 └── scan xyz - └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null + └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null crdb_internal_mvcc_timestamp:6 ################################ # Hybrid PK and non-PK queries # @@ -416,8 +430,10 @@ distinct-on ├── sort │ ├── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null │ ├── ordering: +1,+2 - │ └── scan xyz - │ └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null + │ └── project + │ ├── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null + │ └── scan xyz + │ └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null crdb_internal_mvcc_timestamp:6 └── aggregations └── first-agg [as=z:3] └── z:3 @@ -435,7 +451,7 @@ distinct-on │ └── project │ ├── columns: x:1 y:2 z:3 pk1:4!null │ └── scan xyz - │ └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null + │ └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null crdb_internal_mvcc_timestamp:6 └── aggregations └── first-agg [as=pk1:4] └── pk1:4 @@ -454,7 +470,7 @@ distinct-on │ └── project │ ├── columns: x:1 y:2 z:3 │ └── scan xyz - │ └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null + │ └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null crdb_internal_mvcc_timestamp:6 └── aggregations └── first-agg [as=z:3] └── z:3 @@ -473,7 +489,7 @@ distinct-on │ └── project │ ├── columns: x:1 y:2 z:3 │ └── scan xyz - │ └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null + │ └── columns: x:1 y:2 z:3 pk1:4!null pk2:5!null crdb_internal_mvcc_timestamp:6 └── aggregations └── first-agg [as=z:3] └── z:3 diff --git a/pkg/sql/opt/optbuilder/testdata/explain b/pkg/sql/opt/optbuilder/testdata/explain index 35ef7c79b37e..960a27812d5f 100644 --- a/pkg/sql/opt/optbuilder/testdata/explain +++ b/pkg/sql/opt/optbuilder/testdata/explain @@ -6,126 +6,142 @@ build EXPLAIN SELECT * FROM xy ---- explain - ├── columns: tree:3 field:4 description:5 - └── scan xy - └── columns: x:1!null y:2 + ├── columns: tree:4 field:5 description:6 + └── project + ├── columns: x:1!null y:2 + └── scan xy + └── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 build EXPLAIN (TYPES) SELECT * FROM xy ---- explain - ├── columns: tree:3 field:6 description:7 columns:8 ordering:9 [hidden: level:4 node_type:5] - └── scan xy - └── columns: x:1!null y:2 + ├── columns: tree:4 field:7 description:8 columns:9 ordering:10 [hidden: level:5 node_type:6] + └── project + ├── columns: x:1!null y:2 + └── scan xy + └── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 build EXPLAIN (VERBOSE) SELECT * FROM xy ---- explain - ├── columns: tree:3 field:6 description:7 columns:8 ordering:9 [hidden: level:4 node_type:5] + ├── columns: tree:4 field:7 description:8 columns:9 ordering:10 [hidden: level:5 node_type:6] ├── mode: verbose - └── scan xy - └── columns: x:1!null y:2 + └── project + ├── columns: x:1!null y:2 + └── scan xy + └── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 # Verify we preserve the ordering requirement of the explained query. build EXPLAIN (VERBOSE) SELECT * FROM xy ORDER BY y ---- explain - ├── columns: tree:3 field:6 description:7 columns:8 ordering:9 [hidden: level:4 node_type:5] + ├── columns: tree:4 field:7 description:8 columns:9 ordering:10 [hidden: level:5 node_type:6] ├── mode: verbose └── sort ├── columns: x:1!null y:2 ├── ordering: +2 - └── scan xy - └── columns: x:1!null y:2 + └── project + ├── columns: x:1!null y:2 + └── scan xy + └── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 build EXPLAIN (VERBOSE) SELECT * FROM xy INNER JOIN (VALUES (1, 2), (3, 4)) AS t(u,v) ON x=u ---- explain - ├── columns: tree:5 field:8 description:9 columns:10 ordering:11 [hidden: level:6 node_type:7] + ├── columns: tree:6 field:9 description:10 columns:11 ordering:12 [hidden: level:7 node_type:8] ├── mode: verbose - └── inner-join (hash) - ├── columns: x:1!null y:2 u:3!null v:4!null - ├── scan xy - │ └── columns: x:1!null y:2 - ├── values - │ ├── columns: column1:3!null column2:4!null - │ ├── (1, 2) - │ └── (3, 4) - └── filters - └── x:1 = column1:3 + └── project + ├── columns: x:1!null y:2 u:4!null v:5!null + └── inner-join (hash) + ├── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 column1:4!null column2:5!null + ├── scan xy + │ └── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 + ├── values + │ ├── columns: column1:4!null column2:5!null + │ ├── (1, 2) + │ └── (3, 4) + └── filters + └── x:1 = column1:4 build SELECT tree FROM [ EXPLAIN (VERBOSE) SELECT * FROM xy ] ---- with &1 - ├── columns: tree:10 + ├── columns: tree:11 ├── explain - │ ├── columns: tree:3 level:4 node_type:5 field:6 description:7 columns:8 ordering:9 + │ ├── columns: tree:4 level:5 node_type:6 field:7 description:8 columns:9 ordering:10 │ ├── mode: verbose - │ └── scan xy - │ └── columns: x:1!null y:2 + │ └── project + │ ├── columns: x:1!null y:2 + │ └── scan xy + │ └── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 └── project - ├── columns: tree:10 + ├── columns: tree:11 └── with-scan &1 - ├── columns: tree:10 level:11 node_type:12 field:13 description:14 columns:15 ordering:16 + ├── columns: tree:11 level:12 node_type:13 field:14 description:15 columns:16 ordering:17 └── mapping: - ├── tree:3 => tree:10 - ├── level:4 => level:11 - ├── node_type:5 => node_type:12 - ├── field:6 => field:13 - ├── description:7 => description:14 - ├── columns:8 => columns:15 - └── ordering:9 => ordering:16 + ├── tree:4 => tree:11 + ├── level:5 => level:12 + ├── node_type:6 => node_type:13 + ├── field:7 => field:14 + ├── description:8 => description:15 + ├── columns:9 => columns:16 + └── ordering:10 => ordering:17 build SELECT tree FROM [ EXPLAIN (VERBOSE) SELECT x, x, y FROM xy ORDER BY y ] ---- with &1 - ├── columns: tree:10 + ├── columns: tree:11 ├── explain - │ ├── columns: tree:3 level:4 node_type:5 field:6 description:7 columns:8 ordering:9 + │ ├── columns: tree:4 level:5 node_type:6 field:7 description:8 columns:9 ordering:10 │ ├── mode: verbose │ └── sort │ ├── columns: x:1!null x:1!null y:2 │ ├── ordering: +2 - │ └── scan xy - │ └── columns: x:1!null y:2 + │ └── project + │ ├── columns: x:1!null y:2 + │ └── scan xy + │ └── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 └── project - ├── columns: tree:10 + ├── columns: tree:11 └── with-scan &1 - ├── columns: tree:10 level:11 node_type:12 field:13 description:14 columns:15 ordering:16 + ├── columns: tree:11 level:12 node_type:13 field:14 description:15 columns:16 ordering:17 └── mapping: - ├── tree:3 => tree:10 - ├── level:4 => level:11 - ├── node_type:5 => node_type:12 - ├── field:6 => field:13 - ├── description:7 => description:14 - ├── columns:8 => columns:15 - └── ordering:9 => ordering:16 + ├── tree:4 => tree:11 + ├── level:5 => level:12 + ├── node_type:6 => node_type:13 + ├── field:7 => field:14 + ├── description:8 => description:15 + ├── columns:9 => columns:16 + └── ordering:10 => ordering:17 build SELECT json FROM [EXPLAIN (DISTSQL) SELECT * FROM xy] WHERE false ---- with &1 - ├── columns: json:8 + ├── columns: json:9 ├── explain - │ ├── columns: automatic:3 url:4 json:5 + │ ├── columns: automatic:4 url:5 json:6 │ ├── mode: distsql - │ └── scan xy - │ └── columns: x:1!null y:2 + │ └── project + │ ├── columns: x:1!null y:2 + │ └── scan xy + │ └── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 └── project - ├── columns: json:8 + ├── columns: json:9 └── select - ├── columns: automatic:6 url:7 json:8 + ├── columns: automatic:7 url:8 json:9 ├── with-scan &1 - │ ├── columns: automatic:6 url:7 json:8 + │ ├── columns: automatic:7 url:8 json:9 │ └── mapping: - │ ├── automatic:3 => automatic:6 - │ ├── url:4 => url:7 - │ └── json:5 => json:8 + │ ├── automatic:4 => automatic:7 + │ ├── url:5 => url:8 + │ └── json:6 => json:9 └── filters └── false @@ -134,21 +150,23 @@ build SELECT * FROM [EXPLAIN (DISTSQL) SELECT * FROM xy] WHERE false ---- with &1 - ├── columns: automatic:6 url:7 + ├── columns: automatic:7 url:8 ├── explain - │ ├── columns: automatic:3 url:4 json:5 + │ ├── columns: automatic:4 url:5 json:6 │ ├── mode: distsql - │ └── scan xy - │ └── columns: x:1!null y:2 + │ └── project + │ ├── columns: x:1!null y:2 + │ └── scan xy + │ └── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 └── project - ├── columns: automatic:6 url:7 + ├── columns: automatic:7 url:8 └── select - ├── columns: automatic:6 url:7 json:8 + ├── columns: automatic:7 url:8 json:9 ├── with-scan &1 - │ ├── columns: automatic:6 url:7 json:8 + │ ├── columns: automatic:7 url:8 json:9 │ └── mapping: - │ ├── automatic:3 => automatic:6 - │ ├── url:4 => url:7 - │ └── json:5 => json:8 + │ ├── automatic:4 => automatic:7 + │ ├── url:5 => url:8 + │ └── json:6 => json:9 └── filters └── false diff --git a/pkg/sql/opt/optbuilder/testdata/fk-checks-delete b/pkg/sql/opt/optbuilder/testdata/fk-checks-delete index 866dd6490cd5..be5738cf2fd8 100644 --- a/pkg/sql/opt/optbuilder/testdata/fk-checks-delete +++ b/pkg/sql/opt/optbuilder/testdata/fk-checks-delete @@ -11,39 +11,39 @@ DELETE FROM child WHERE c = 4 ---- delete child ├── columns: - ├── fetch columns: c:3 p:4 + ├── fetch columns: c:4 p:5 └── select - ├── columns: c:3!null p:4!null + ├── columns: c:4!null p:5!null crdb_internal_mvcc_timestamp:6 ├── scan child - │ └── columns: c:3!null p:4!null + │ └── columns: c:4!null p:5!null crdb_internal_mvcc_timestamp:6 └── filters - └── c:3 = 4 + └── c:4 = 4 build DELETE FROM parent WHERE p = 3 ---- delete parent ├── columns: - ├── fetch columns: x:4 parent.p:5 other:6 + ├── fetch columns: x:5 parent.p:6 other:7 ├── input binding: &1 ├── select - │ ├── columns: x:4 parent.p:5!null other:6 + │ ├── columns: x:5 parent.p:6!null other:7 parent.crdb_internal_mvcc_timestamp:8 │ ├── scan parent - │ │ └── columns: x:4 parent.p:5!null other:6 + │ │ └── columns: x:5 parent.p:6!null other:7 parent.crdb_internal_mvcc_timestamp:8 │ └── filters - │ └── parent.p:5 = 3 + │ └── parent.p:6 = 3 └── f-k-checks └── f-k-checks-item: child(p) -> parent(p) └── semi-join (hash) - ├── columns: p:7!null + ├── columns: p:9!null ├── with-scan &1 - │ ├── columns: p:7!null + │ ├── columns: p:9!null │ └── mapping: - │ └── parent.p:5 => p:7 + │ └── parent.p:6 => p:9 ├── scan child - │ └── columns: child.p:9!null + │ └── columns: child.p:11!null └── filters - └── p:7 = child.p:9 + └── p:9 = child.p:11 exec-ddl CREATE TABLE child2 (c INT PRIMARY KEY, p INT NOT NULL REFERENCES parent(other)) @@ -54,37 +54,37 @@ DELETE FROM parent WHERE p = 3 ---- delete parent ├── columns: - ├── fetch columns: x:4 parent.p:5 parent.other:6 + ├── fetch columns: x:5 parent.p:6 parent.other:7 ├── input binding: &1 ├── select - │ ├── columns: x:4 parent.p:5!null parent.other:6 + │ ├── columns: x:5 parent.p:6!null parent.other:7 parent.crdb_internal_mvcc_timestamp:8 │ ├── scan parent - │ │ └── columns: x:4 parent.p:5!null parent.other:6 + │ │ └── columns: x:5 parent.p:6!null parent.other:7 parent.crdb_internal_mvcc_timestamp:8 │ └── filters - │ └── parent.p:5 = 3 + │ └── parent.p:6 = 3 └── f-k-checks ├── f-k-checks-item: child(p) -> parent(p) │ └── semi-join (hash) - │ ├── columns: p:7!null + │ ├── columns: p:9!null │ ├── with-scan &1 - │ │ ├── columns: p:7!null + │ │ ├── columns: p:9!null │ │ └── mapping: - │ │ └── parent.p:5 => p:7 + │ │ └── parent.p:6 => p:9 │ ├── scan child - │ │ └── columns: child.p:9!null + │ │ └── columns: child.p:11!null │ └── filters - │ └── p:7 = child.p:9 + │ └── p:9 = child.p:11 └── f-k-checks-item: child2(p) -> parent(other) └── semi-join (hash) - ├── columns: other:10 + ├── columns: other:13 ├── with-scan &1 - │ ├── columns: other:10 + │ ├── columns: other:13 │ └── mapping: - │ └── parent.other:6 => other:10 + │ └── parent.other:7 => other:13 ├── scan child2 - │ └── columns: child2.p:12!null + │ └── columns: child2.p:15!null └── filters - └── other:10 = child2.p:12 + └── other:13 = child2.p:15 exec-ddl CREATE TABLE doubleparent (p1 INT, p2 INT, other INT, PRIMARY KEY (p1, p2)) @@ -99,38 +99,38 @@ DELETE FROM doubleparent WHERE p1 = 10 ---- delete doubleparent ├── columns: - ├── fetch columns: doubleparent.p1:4 doubleparent.p2:5 other:6 + ├── fetch columns: doubleparent.p1:5 doubleparent.p2:6 other:7 ├── input binding: &1 ├── select - │ ├── columns: doubleparent.p1:4!null doubleparent.p2:5!null other:6 + │ ├── columns: doubleparent.p1:5!null doubleparent.p2:6!null other:7 doubleparent.crdb_internal_mvcc_timestamp:8 │ ├── scan doubleparent - │ │ └── columns: doubleparent.p1:4!null doubleparent.p2:5!null other:6 + │ │ └── columns: doubleparent.p1:5!null doubleparent.p2:6!null other:7 doubleparent.crdb_internal_mvcc_timestamp:8 │ └── filters - │ └── doubleparent.p1:4 = 10 + │ └── doubleparent.p1:5 = 10 └── f-k-checks └── f-k-checks-item: doublechild(p1,p2) -> doubleparent(p1,p2) └── semi-join (hash) - ├── columns: p1:7!null p2:8!null + ├── columns: p1:9!null p2:10!null ├── with-scan &1 - │ ├── columns: p1:7!null p2:8!null + │ ├── columns: p1:9!null p2:10!null │ └── mapping: - │ ├── doubleparent.p1:4 => p1:7 - │ └── doubleparent.p2:5 => p2:8 + │ ├── doubleparent.p1:5 => p1:9 + │ └── doubleparent.p2:6 => p2:10 ├── scan doublechild - │ └── columns: doublechild.p1:10 doublechild.p2:11 + │ └── columns: doublechild.p1:12 doublechild.p2:13 └── filters - ├── p1:7 = doublechild.p1:10 - └── p2:8 = doublechild.p2:11 + ├── p1:9 = doublechild.p1:12 + └── p2:10 = doublechild.p2:13 build DELETE FROM doublechild WHERE p1 = 10 ---- delete doublechild ├── columns: - ├── fetch columns: c:4 p1:5 p2:6 + ├── fetch columns: c:5 p1:6 p2:7 └── select - ├── columns: c:4!null p1:5!null p2:6 + ├── columns: c:5!null p1:6!null p2:7 crdb_internal_mvcc_timestamp:8 ├── scan doublechild - │ └── columns: c:4!null p1:5 p2:6 + │ └── columns: c:5!null p1:6 p2:7 crdb_internal_mvcc_timestamp:8 └── filters - └── p1:5 = 10 + └── p1:6 = 10 diff --git a/pkg/sql/opt/optbuilder/testdata/fk-checks-insert b/pkg/sql/opt/optbuilder/testdata/fk-checks-insert index 974cbd871c24..91f216bed238 100644 --- a/pkg/sql/opt/optbuilder/testdata/fk-checks-insert +++ b/pkg/sql/opt/optbuilder/testdata/fk-checks-insert @@ -12,25 +12,25 @@ INSERT INTO child VALUES (100, 1), (200, 1) insert child ├── columns: ├── insert-mapping: - │ ├── column1:3 => c:1 - │ └── column2:4 => child.p:2 + │ ├── column1:4 => c:1 + │ └── column2:5 => child.p:2 ├── input binding: &1 ├── values - │ ├── columns: column1:3!null column2:4!null + │ ├── columns: column1:4!null column2:5!null │ ├── (100, 1) │ └── (200, 1) └── f-k-checks └── f-k-checks-item: child(p) -> parent(p) └── anti-join (hash) - ├── columns: column2:5!null + ├── columns: column2:6!null ├── with-scan &1 - │ ├── columns: column2:5!null + │ ├── columns: column2:6!null │ └── mapping: - │ └── column2:4 => column2:5 + │ └── column2:5 => column2:6 ├── scan parent - │ └── columns: parent.p:6!null + │ └── columns: parent.p:7!null └── filters - └── column2:5 = parent.p:6 + └── column2:6 = parent.p:7 build INSERT INTO child VALUES (100, 1), (200, 1) ON CONFLICT DO NOTHING @@ -38,43 +38,43 @@ INSERT INTO child VALUES (100, 1), (200, 1) ON CONFLICT DO NOTHING insert child ├── columns: ├── insert-mapping: - │ ├── column1:3 => c:1 - │ └── column2:4 => child.p:2 + │ ├── column1:4 => c:1 + │ └── column2:5 => child.p:2 ├── input binding: &1 ├── upsert-distinct-on - │ ├── columns: column1:3!null column2:4!null - │ ├── grouping columns: column1:3!null + │ ├── columns: column1:4!null column2:5!null + │ ├── grouping columns: column1:4!null │ ├── project - │ │ ├── columns: column1:3!null column2:4!null + │ │ ├── columns: column1:4!null column2:5!null │ │ └── select - │ │ ├── columns: column1:3!null column2:4!null c:5 child.p:6 + │ │ ├── columns: column1:4!null column2:5!null c:6 child.p:7 child.crdb_internal_mvcc_timestamp:8 │ │ ├── left-join (hash) - │ │ │ ├── columns: column1:3!null column2:4!null c:5 child.p:6 + │ │ │ ├── columns: column1:4!null column2:5!null c:6 child.p:7 child.crdb_internal_mvcc_timestamp:8 │ │ │ ├── values - │ │ │ │ ├── columns: column1:3!null column2:4!null + │ │ │ │ ├── columns: column1:4!null column2:5!null │ │ │ │ ├── (100, 1) │ │ │ │ └── (200, 1) │ │ │ ├── scan child - │ │ │ │ └── columns: c:5!null child.p:6!null + │ │ │ │ └── columns: c:6!null child.p:7!null child.crdb_internal_mvcc_timestamp:8 │ │ │ └── filters - │ │ │ └── column1:3 = c:5 + │ │ │ └── column1:4 = c:6 │ │ └── filters - │ │ └── c:5 IS NULL + │ │ └── c:6 IS NULL │ └── aggregations - │ └── first-agg [as=column2:4] - │ └── column2:4 + │ └── first-agg [as=column2:5] + │ └── column2:5 └── f-k-checks └── f-k-checks-item: child(p) -> parent(p) └── anti-join (hash) - ├── columns: column2:7!null + ├── columns: column2:9!null ├── with-scan &1 - │ ├── columns: column2:7!null + │ ├── columns: column2:9!null │ └── mapping: - │ └── column2:4 => column2:7 + │ └── column2:5 => column2:9 ├── scan parent - │ └── columns: parent.p:8!null + │ └── columns: parent.p:10!null └── filters - └── column2:7 = parent.p:8 + └── column2:9 = parent.p:10 # Use a non-constant input. exec-ddl @@ -87,25 +87,25 @@ INSERT INTO child SELECT x, y FROM xy insert child ├── columns: ├── insert-mapping: - │ ├── x:3 => c:1 - │ └── xy.y:4 => child.p:2 + │ ├── x:4 => c:1 + │ └── xy.y:5 => child.p:2 ├── input binding: &1 ├── project - │ ├── columns: x:3 xy.y:4 + │ ├── columns: x:4 xy.y:5 │ └── scan xy - │ └── columns: x:3 xy.y:4 rowid:5!null + │ └── columns: x:4 xy.y:5 rowid:6!null xy.crdb_internal_mvcc_timestamp:7 └── f-k-checks └── f-k-checks-item: child(p) -> parent(p) └── anti-join (hash) - ├── columns: y:6 + ├── columns: y:8 ├── with-scan &1 - │ ├── columns: y:6 + │ ├── columns: y:8 │ └── mapping: - │ └── xy.y:4 => y:6 + │ └── xy.y:5 => y:8 ├── scan parent - │ └── columns: parent.p:7!null + │ └── columns: parent.p:9!null └── filters - └── y:6 = parent.p:7 + └── y:8 = parent.p:9 exec-ddl CREATE TABLE child_nullable (c INT PRIMARY KEY, p INT REFERENCES parent(p)); @@ -119,29 +119,29 @@ INSERT INTO child_nullable VALUES (100, 1), (200, NULL) insert child_nullable ├── columns: ├── insert-mapping: - │ ├── column1:3 => c:1 - │ └── column2:4 => child_nullable.p:2 + │ ├── column1:4 => c:1 + │ └── column2:5 => child_nullable.p:2 ├── input binding: &1 ├── values - │ ├── columns: column1:3!null column2:4 + │ ├── columns: column1:4!null column2:5 │ ├── (100, 1) │ └── (200, NULL::INT8) └── f-k-checks └── f-k-checks-item: child_nullable(p) -> parent(p) └── anti-join (hash) - ├── columns: column2:5!null + ├── columns: column2:6!null ├── select - │ ├── columns: column2:5!null + │ ├── columns: column2:6!null │ ├── with-scan &1 - │ │ ├── columns: column2:5 + │ │ ├── columns: column2:6 │ │ └── mapping: - │ │ └── column2:4 => column2:5 + │ │ └── column2:5 => column2:6 │ └── filters - │ └── column2:5 IS NOT NULL + │ └── column2:6 IS NOT NULL ├── scan parent - │ └── columns: parent.p:6!null + │ └── columns: parent.p:7!null └── filters - └── column2:5 = parent.p:6 + └── column2:6 = parent.p:7 # The column is nullable but we know that the input is not null, so we don't # need to plan the filter. @@ -151,25 +151,25 @@ INSERT INTO child_nullable VALUES (100, 1), (200, 1) insert child_nullable ├── columns: ├── insert-mapping: - │ ├── column1:3 => c:1 - │ └── column2:4 => child_nullable.p:2 + │ ├── column1:4 => c:1 + │ └── column2:5 => child_nullable.p:2 ├── input binding: &1 ├── values - │ ├── columns: column1:3!null column2:4!null + │ ├── columns: column1:4!null column2:5!null │ ├── (100, 1) │ └── (200, 1) └── f-k-checks └── f-k-checks-item: child_nullable(p) -> parent(p) └── anti-join (hash) - ├── columns: column2:5!null + ├── columns: column2:6!null ├── with-scan &1 - │ ├── columns: column2:5!null + │ ├── columns: column2:6!null │ └── mapping: - │ └── column2:4 => column2:5 + │ └── column2:5 => column2:6 ├── scan parent - │ └── columns: parent.p:6!null + │ └── columns: parent.p:7!null └── filters - └── column2:5 = parent.p:6 + └── column2:6 = parent.p:7 # In this case, we know that we are inserting *only* NULL values, so we don't # need to check any FKs. @@ -179,10 +179,10 @@ INSERT INTO child_nullable VALUES (100, NULL), (200, NULL) insert child_nullable ├── columns: ├── insert-mapping: - │ ├── column1:3 => c:1 - │ └── column2:4 => p:2 + │ ├── column1:4 => c:1 + │ └── column2:5 => p:2 └── values - ├── columns: column1:3!null column2:4 + ├── columns: column1:4!null column2:5 ├── (100, NULL::INT8) └── (200, NULL::INT8) @@ -193,16 +193,16 @@ INSERT INTO child_nullable (c) VALUES (100), (200) insert child_nullable ├── columns: ├── insert-mapping: - │ ├── column1:3 => c:1 - │ └── column4:4 => p:2 + │ ├── column1:4 => c:1 + │ └── column5:5 => p:2 └── project - ├── columns: column4:4 column1:3!null + ├── columns: column5:5 column1:4!null ├── values - │ ├── columns: column1:3!null + │ ├── columns: column1:4!null │ ├── (100,) │ └── (200,) └── projections - └── NULL::INT8 [as=column4:4] + └── NULL::INT8 [as=column5:5] # Check planning of filter with FULL match (which should be the same on a # single column). @@ -216,29 +216,29 @@ INSERT INTO child_nullable_full VALUES (100, 1), (200, NULL) insert child_nullable_full ├── columns: ├── insert-mapping: - │ ├── column1:3 => c:1 - │ └── column2:4 => child_nullable_full.p:2 + │ ├── column1:4 => c:1 + │ └── column2:5 => child_nullable_full.p:2 ├── input binding: &1 ├── values - │ ├── columns: column1:3!null column2:4 + │ ├── columns: column1:4!null column2:5 │ ├── (100, 1) │ └── (200, NULL::INT8) └── f-k-checks └── f-k-checks-item: child_nullable_full(p) -> parent(p) └── anti-join (hash) - ├── columns: column2:5!null + ├── columns: column2:6!null ├── select - │ ├── columns: column2:5!null + │ ├── columns: column2:6!null │ ├── with-scan &1 - │ │ ├── columns: column2:5 + │ │ ├── columns: column2:6 │ │ └── mapping: - │ │ └── column2:4 => column2:5 + │ │ └── column2:5 => column2:6 │ └── filters - │ └── column2:5 IS NOT NULL + │ └── column2:6 IS NOT NULL ├── scan parent - │ └── columns: parent.p:6!null + │ └── columns: parent.p:7!null └── filters - └── column2:5 = parent.p:6 + └── column2:6 = parent.p:7 # No FK check needed. build @@ -247,16 +247,16 @@ INSERT INTO child_nullable_full (c) VALUES (100), (200) insert child_nullable_full ├── columns: ├── insert-mapping: - │ ├── column1:3 => c:1 - │ └── column4:4 => p:2 + │ ├── column1:4 => c:1 + │ └── column5:5 => p:2 └── project - ├── columns: column4:4 column1:3!null + ├── columns: column5:5 column1:4!null ├── values - │ ├── columns: column1:3!null + │ ├── columns: column1:4!null │ ├── (100,) │ └── (200,) └── projections - └── NULL::INT8 [as=column4:4] + └── NULL::INT8 [as=column5:5] # Tests with multicolumn FKs. exec-ddl @@ -278,37 +278,37 @@ INSERT INTO multi_col_child VALUES (4, NULL, NULL, NULL), (5, 1, 2, 3) insert multi_col_child ├── columns: ├── insert-mapping: - │ ├── column1:5 => c:1 - │ ├── column2:6 => multi_col_child.p:2 - │ ├── column3:7 => multi_col_child.q:3 - │ └── column4:8 => multi_col_child.r:4 + │ ├── column1:6 => c:1 + │ ├── column2:7 => multi_col_child.p:2 + │ ├── column3:8 => multi_col_child.q:3 + │ └── column4:9 => multi_col_child.r:4 ├── input binding: &1 ├── values - │ ├── columns: column1:5!null column2:6 column3:7 column4:8 + │ ├── columns: column1:6!null column2:7 column3:8 column4:9 │ ├── (4, NULL::INT8, NULL::INT8, NULL::INT8) │ └── (5, 1, 2, 3) └── f-k-checks └── f-k-checks-item: multi_col_child(p,q,r) -> multi_col_parent(p,q,r) └── anti-join (hash) - ├── columns: column2:9!null column3:10!null column4:11!null + ├── columns: column2:10!null column3:11!null column4:12!null ├── select - │ ├── columns: column2:9!null column3:10!null column4:11!null + │ ├── columns: column2:10!null column3:11!null column4:12!null │ ├── with-scan &1 - │ │ ├── columns: column2:9 column3:10 column4:11 + │ │ ├── columns: column2:10 column3:11 column4:12 │ │ └── mapping: - │ │ ├── column2:6 => column2:9 - │ │ ├── column3:7 => column3:10 - │ │ └── column4:8 => column4:11 + │ │ ├── column2:7 => column2:10 + │ │ ├── column3:8 => column3:11 + │ │ └── column4:9 => column4:12 │ └── filters - │ ├── column2:9 IS NOT NULL - │ ├── column3:10 IS NOT NULL - │ └── column4:11 IS NOT NULL + │ ├── column2:10 IS NOT NULL + │ ├── column3:11 IS NOT NULL + │ └── column4:12 IS NOT NULL ├── scan multi_col_parent - │ └── columns: multi_col_parent.p:12!null multi_col_parent.q:13!null multi_col_parent.r:14!null + │ └── columns: multi_col_parent.p:13!null multi_col_parent.q:14!null multi_col_parent.r:15!null └── filters - ├── column2:9 = multi_col_parent.p:12 - ├── column3:10 = multi_col_parent.q:13 - └── column4:11 = multi_col_parent.r:14 + ├── column2:10 = multi_col_parent.p:13 + ├── column3:11 = multi_col_parent.q:14 + └── column4:12 = multi_col_parent.r:15 # Only p and q are nullable. build @@ -317,36 +317,36 @@ INSERT INTO multi_col_child VALUES (2, NULL, 20, 20), (3, 20, NULL, 20) insert multi_col_child ├── columns: ├── insert-mapping: - │ ├── column1:5 => c:1 - │ ├── column2:6 => multi_col_child.p:2 - │ ├── column3:7 => multi_col_child.q:3 - │ └── column4:8 => multi_col_child.r:4 + │ ├── column1:6 => c:1 + │ ├── column2:7 => multi_col_child.p:2 + │ ├── column3:8 => multi_col_child.q:3 + │ └── column4:9 => multi_col_child.r:4 ├── input binding: &1 ├── values - │ ├── columns: column1:5!null column2:6 column3:7 column4:8!null + │ ├── columns: column1:6!null column2:7 column3:8 column4:9!null │ ├── (2, NULL::INT8, 20, 20) │ └── (3, 20, NULL::INT8, 20) └── f-k-checks └── f-k-checks-item: multi_col_child(p,q,r) -> multi_col_parent(p,q,r) └── anti-join (hash) - ├── columns: column2:9!null column3:10!null column4:11!null + ├── columns: column2:10!null column3:11!null column4:12!null ├── select - │ ├── columns: column2:9!null column3:10!null column4:11!null + │ ├── columns: column2:10!null column3:11!null column4:12!null │ ├── with-scan &1 - │ │ ├── columns: column2:9 column3:10 column4:11!null + │ │ ├── columns: column2:10 column3:11 column4:12!null │ │ └── mapping: - │ │ ├── column2:6 => column2:9 - │ │ ├── column3:7 => column3:10 - │ │ └── column4:8 => column4:11 + │ │ ├── column2:7 => column2:10 + │ │ ├── column3:8 => column3:11 + │ │ └── column4:9 => column4:12 │ └── filters - │ ├── column2:9 IS NOT NULL - │ └── column3:10 IS NOT NULL + │ ├── column2:10 IS NOT NULL + │ └── column3:11 IS NOT NULL ├── scan multi_col_parent - │ └── columns: multi_col_parent.p:12!null multi_col_parent.q:13!null multi_col_parent.r:14!null + │ └── columns: multi_col_parent.p:13!null multi_col_parent.q:14!null multi_col_parent.r:15!null └── filters - ├── column2:9 = multi_col_parent.p:12 - ├── column3:10 = multi_col_parent.q:13 - └── column4:11 = multi_col_parent.r:14 + ├── column2:10 = multi_col_parent.p:13 + ├── column3:11 = multi_col_parent.q:14 + └── column4:12 = multi_col_parent.r:15 # All the FK columns are not-null; no filter necessary. build @@ -355,30 +355,30 @@ INSERT INTO multi_col_child VALUES (1, 10, 10, 10) insert multi_col_child ├── columns: ├── insert-mapping: - │ ├── column1:5 => c:1 - │ ├── column2:6 => multi_col_child.p:2 - │ ├── column3:7 => multi_col_child.q:3 - │ └── column4:8 => multi_col_child.r:4 + │ ├── column1:6 => c:1 + │ ├── column2:7 => multi_col_child.p:2 + │ ├── column3:8 => multi_col_child.q:3 + │ └── column4:9 => multi_col_child.r:4 ├── input binding: &1 ├── values - │ ├── columns: column1:5!null column2:6!null column3:7!null column4:8!null + │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null │ └── (1, 10, 10, 10) └── f-k-checks └── f-k-checks-item: multi_col_child(p,q,r) -> multi_col_parent(p,q,r) └── anti-join (hash) - ├── columns: column2:9!null column3:10!null column4:11!null + ├── columns: column2:10!null column3:11!null column4:12!null ├── with-scan &1 - │ ├── columns: column2:9!null column3:10!null column4:11!null + │ ├── columns: column2:10!null column3:11!null column4:12!null │ └── mapping: - │ ├── column2:6 => column2:9 - │ ├── column3:7 => column3:10 - │ └── column4:8 => column4:11 + │ ├── column2:7 => column2:10 + │ ├── column3:8 => column3:11 + │ └── column4:9 => column4:12 ├── scan multi_col_parent - │ └── columns: multi_col_parent.p:12!null multi_col_parent.q:13!null multi_col_parent.r:14!null + │ └── columns: multi_col_parent.p:13!null multi_col_parent.q:14!null multi_col_parent.r:15!null └── filters - ├── column2:9 = multi_col_parent.p:12 - ├── column3:10 = multi_col_parent.q:13 - └── column4:11 = multi_col_parent.r:14 + ├── column2:10 = multi_col_parent.p:13 + ├── column3:11 = multi_col_parent.q:14 + └── column4:12 = multi_col_parent.r:15 # No FK check needed - we have only NULL values for a FK column. build @@ -387,12 +387,12 @@ INSERT INTO multi_col_child VALUES (1, 10, NULL, 10) insert multi_col_child ├── columns: ├── insert-mapping: - │ ├── column1:5 => c:1 - │ ├── column2:6 => p:2 - │ ├── column3:7 => q:3 - │ └── column4:8 => r:4 + │ ├── column1:6 => c:1 + │ ├── column2:7 => p:2 + │ ├── column3:8 => q:3 + │ └── column4:9 => r:4 └── values - ├── columns: column1:5!null column2:6!null column3:7 column4:8!null + ├── columns: column1:6!null column2:7!null column3:8 column4:9!null └── (1, 10, NULL::INT8, 10) exec-ddl @@ -410,35 +410,35 @@ INSERT INTO multi_col_child_full VALUES (4, NULL, NULL, NULL), (5, 1, 2, 3) insert multi_col_child_full ├── columns: ├── insert-mapping: - │ ├── column1:5 => c:1 - │ ├── column2:6 => multi_col_child_full.p:2 - │ ├── column3:7 => multi_col_child_full.q:3 - │ └── column4:8 => multi_col_child_full.r:4 + │ ├── column1:6 => c:1 + │ ├── column2:7 => multi_col_child_full.p:2 + │ ├── column3:8 => multi_col_child_full.q:3 + │ └── column4:9 => multi_col_child_full.r:4 ├── input binding: &1 ├── values - │ ├── columns: column1:5!null column2:6 column3:7 column4:8 + │ ├── columns: column1:6!null column2:7 column3:8 column4:9 │ ├── (4, NULL::INT8, NULL::INT8, NULL::INT8) │ └── (5, 1, 2, 3) └── f-k-checks └── f-k-checks-item: multi_col_child_full(p,q,r) -> multi_col_parent(p,q,r) └── anti-join (hash) - ├── columns: column2:9 column3:10 column4:11 + ├── columns: column2:10 column3:11 column4:12 ├── select - │ ├── columns: column2:9 column3:10 column4:11 + │ ├── columns: column2:10 column3:11 column4:12 │ ├── with-scan &1 - │ │ ├── columns: column2:9 column3:10 column4:11 + │ │ ├── columns: column2:10 column3:11 column4:12 │ │ └── mapping: - │ │ ├── column2:6 => column2:9 - │ │ ├── column3:7 => column3:10 - │ │ └── column4:8 => column4:11 + │ │ ├── column2:7 => column2:10 + │ │ ├── column3:8 => column3:11 + │ │ └── column4:9 => column4:12 │ └── filters - │ └── ((column2:9 IS NOT NULL) OR (column3:10 IS NOT NULL)) OR (column4:11 IS NOT NULL) + │ └── ((column2:10 IS NOT NULL) OR (column3:11 IS NOT NULL)) OR (column4:12 IS NOT NULL) ├── scan multi_col_parent - │ └── columns: multi_col_parent.p:12!null multi_col_parent.q:13!null multi_col_parent.r:14!null + │ └── columns: multi_col_parent.p:13!null multi_col_parent.q:14!null multi_col_parent.r:15!null └── filters - ├── column2:9 = multi_col_parent.p:12 - ├── column3:10 = multi_col_parent.q:13 - └── column4:11 = multi_col_parent.r:14 + ├── column2:10 = multi_col_parent.p:13 + ├── column3:11 = multi_col_parent.q:14 + └── column4:12 = multi_col_parent.r:15 # Only p and q are nullable; no filter necessary. build @@ -447,31 +447,31 @@ INSERT INTO multi_col_child_full VALUES (2, NULL, 20, 20), (3, 20, NULL, 20) insert multi_col_child_full ├── columns: ├── insert-mapping: - │ ├── column1:5 => c:1 - │ ├── column2:6 => multi_col_child_full.p:2 - │ ├── column3:7 => multi_col_child_full.q:3 - │ └── column4:8 => multi_col_child_full.r:4 + │ ├── column1:6 => c:1 + │ ├── column2:7 => multi_col_child_full.p:2 + │ ├── column3:8 => multi_col_child_full.q:3 + │ └── column4:9 => multi_col_child_full.r:4 ├── input binding: &1 ├── values - │ ├── columns: column1:5!null column2:6 column3:7 column4:8!null + │ ├── columns: column1:6!null column2:7 column3:8 column4:9!null │ ├── (2, NULL::INT8, 20, 20) │ └── (3, 20, NULL::INT8, 20) └── f-k-checks └── f-k-checks-item: multi_col_child_full(p,q,r) -> multi_col_parent(p,q,r) └── anti-join (hash) - ├── columns: column2:9 column3:10 column4:11!null + ├── columns: column2:10 column3:11 column4:12!null ├── with-scan &1 - │ ├── columns: column2:9 column3:10 column4:11!null + │ ├── columns: column2:10 column3:11 column4:12!null │ └── mapping: - │ ├── column2:6 => column2:9 - │ ├── column3:7 => column3:10 - │ └── column4:8 => column4:11 + │ ├── column2:7 => column2:10 + │ ├── column3:8 => column3:11 + │ └── column4:9 => column4:12 ├── scan multi_col_parent - │ └── columns: multi_col_parent.p:12!null multi_col_parent.q:13!null multi_col_parent.r:14!null + │ └── columns: multi_col_parent.p:13!null multi_col_parent.q:14!null multi_col_parent.r:15!null └── filters - ├── column2:9 = multi_col_parent.p:12 - ├── column3:10 = multi_col_parent.q:13 - └── column4:11 = multi_col_parent.r:14 + ├── column2:10 = multi_col_parent.p:13 + ├── column3:11 = multi_col_parent.q:14 + └── column4:12 = multi_col_parent.r:15 # All the FK columns are not-null; no filter necessary. build @@ -480,30 +480,30 @@ INSERT INTO multi_col_child_full VALUES (1, 10, 10, 10) insert multi_col_child_full ├── columns: ├── insert-mapping: - │ ├── column1:5 => c:1 - │ ├── column2:6 => multi_col_child_full.p:2 - │ ├── column3:7 => multi_col_child_full.q:3 - │ └── column4:8 => multi_col_child_full.r:4 + │ ├── column1:6 => c:1 + │ ├── column2:7 => multi_col_child_full.p:2 + │ ├── column3:8 => multi_col_child_full.q:3 + │ └── column4:9 => multi_col_child_full.r:4 ├── input binding: &1 ├── values - │ ├── columns: column1:5!null column2:6!null column3:7!null column4:8!null + │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null │ └── (1, 10, 10, 10) └── f-k-checks └── f-k-checks-item: multi_col_child_full(p,q,r) -> multi_col_parent(p,q,r) └── anti-join (hash) - ├── columns: column2:9!null column3:10!null column4:11!null + ├── columns: column2:10!null column3:11!null column4:12!null ├── with-scan &1 - │ ├── columns: column2:9!null column3:10!null column4:11!null + │ ├── columns: column2:10!null column3:11!null column4:12!null │ └── mapping: - │ ├── column2:6 => column2:9 - │ ├── column3:7 => column3:10 - │ └── column4:8 => column4:11 + │ ├── column2:7 => column2:10 + │ ├── column3:8 => column3:11 + │ └── column4:9 => column4:12 ├── scan multi_col_parent - │ └── columns: multi_col_parent.p:12!null multi_col_parent.q:13!null multi_col_parent.r:14!null + │ └── columns: multi_col_parent.p:13!null multi_col_parent.q:14!null multi_col_parent.r:15!null └── filters - ├── column2:9 = multi_col_parent.p:12 - ├── column3:10 = multi_col_parent.q:13 - └── column4:11 = multi_col_parent.r:14 + ├── column2:10 = multi_col_parent.p:13 + ├── column3:11 = multi_col_parent.q:14 + └── column4:12 = multi_col_parent.r:15 # No FK check needed when all FK columns only have NULL values. build @@ -512,12 +512,12 @@ INSERT INTO multi_col_child_full VALUES (1, NULL, NULL, NULL) insert multi_col_child_full ├── columns: ├── insert-mapping: - │ ├── column1:5 => c:1 - │ ├── column2:6 => p:2 - │ ├── column3:7 => q:3 - │ └── column4:8 => r:4 + │ ├── column1:6 => c:1 + │ ├── column2:7 => p:2 + │ ├── column3:8 => q:3 + │ └── column4:9 => r:4 └── values - ├── columns: column1:5!null column2:6 column3:7 column4:8 + ├── columns: column1:6!null column2:7 column3:8 column4:9 └── (1, NULL::INT8, NULL::INT8, NULL::INT8) # But with MATCH FULL, the FK check is needed when only a subset of the columns @@ -528,30 +528,30 @@ INSERT INTO multi_col_child_full VALUES (1, NULL, 2, NULL) insert multi_col_child_full ├── columns: ├── insert-mapping: - │ ├── column1:5 => c:1 - │ ├── column2:6 => multi_col_child_full.p:2 - │ ├── column3:7 => multi_col_child_full.q:3 - │ └── column4:8 => multi_col_child_full.r:4 + │ ├── column1:6 => c:1 + │ ├── column2:7 => multi_col_child_full.p:2 + │ ├── column3:8 => multi_col_child_full.q:3 + │ └── column4:9 => multi_col_child_full.r:4 ├── input binding: &1 ├── values - │ ├── columns: column1:5!null column2:6 column3:7!null column4:8 + │ ├── columns: column1:6!null column2:7 column3:8!null column4:9 │ └── (1, NULL::INT8, 2, NULL::INT8) └── f-k-checks └── f-k-checks-item: multi_col_child_full(p,q,r) -> multi_col_parent(p,q,r) └── anti-join (hash) - ├── columns: column2:9 column3:10!null column4:11 + ├── columns: column2:10 column3:11!null column4:12 ├── with-scan &1 - │ ├── columns: column2:9 column3:10!null column4:11 + │ ├── columns: column2:10 column3:11!null column4:12 │ └── mapping: - │ ├── column2:6 => column2:9 - │ ├── column3:7 => column3:10 - │ └── column4:8 => column4:11 + │ ├── column2:7 => column2:10 + │ ├── column3:8 => column3:11 + │ └── column4:9 => column4:12 ├── scan multi_col_parent - │ └── columns: multi_col_parent.p:12!null multi_col_parent.q:13!null multi_col_parent.r:14!null + │ └── columns: multi_col_parent.p:13!null multi_col_parent.q:14!null multi_col_parent.r:15!null └── filters - ├── column2:9 = multi_col_parent.p:12 - ├── column3:10 = multi_col_parent.q:13 - └── column4:11 = multi_col_parent.r:14 + ├── column2:10 = multi_col_parent.p:13 + ├── column3:11 = multi_col_parent.q:14 + └── column4:12 = multi_col_parent.r:15 exec-ddl CREATE TABLE multi_ref_parent_a (a INT PRIMARY KEY, other INT) @@ -578,50 +578,50 @@ INSERT INTO multi_ref_child VALUES (1, 1, NULL, NULL), (2, NULL, 2, NULL), (3, N insert multi_ref_child ├── columns: ├── insert-mapping: - │ ├── column1:5 => k:1 - │ ├── column2:6 => multi_ref_child.a:2 - │ ├── column3:7 => multi_ref_child.b:3 - │ └── column4:8 => multi_ref_child.c:4 + │ ├── column1:6 => k:1 + │ ├── column2:7 => multi_ref_child.a:2 + │ ├── column3:8 => multi_ref_child.b:3 + │ └── column4:9 => multi_ref_child.c:4 ├── input binding: &1 ├── values - │ ├── columns: column1:5!null column2:6 column3:7 column4:8 + │ ├── columns: column1:6!null column2:7 column3:8 column4:9 │ ├── (1, 1, NULL::INT8, NULL::INT8) │ ├── (2, NULL::INT8, 2, NULL::INT8) │ └── (3, NULL::INT8, NULL::INT8, 3) └── f-k-checks ├── f-k-checks-item: multi_ref_child(a) -> multi_ref_parent_a(a) │ └── anti-join (hash) - │ ├── columns: column2:9!null + │ ├── columns: column2:10!null │ ├── select - │ │ ├── columns: column2:9!null + │ │ ├── columns: column2:10!null │ │ ├── with-scan &1 - │ │ │ ├── columns: column2:9 + │ │ │ ├── columns: column2:10 │ │ │ └── mapping: - │ │ │ └── column2:6 => column2:9 + │ │ │ └── column2:7 => column2:10 │ │ └── filters - │ │ └── column2:9 IS NOT NULL + │ │ └── column2:10 IS NOT NULL │ ├── scan multi_ref_parent_a - │ │ └── columns: multi_ref_parent_a.a:10!null + │ │ └── columns: multi_ref_parent_a.a:11!null │ └── filters - │ └── column2:9 = multi_ref_parent_a.a:10 + │ └── column2:10 = multi_ref_parent_a.a:11 └── f-k-checks-item: multi_ref_child(b,c) -> multi_ref_parent_bc(b,c) └── anti-join (hash) - ├── columns: column3:12!null column4:13!null + ├── columns: column3:14!null column4:15!null ├── select - │ ├── columns: column3:12!null column4:13!null + │ ├── columns: column3:14!null column4:15!null │ ├── with-scan &1 - │ │ ├── columns: column3:12 column4:13 + │ │ ├── columns: column3:14 column4:15 │ │ └── mapping: - │ │ ├── column3:7 => column3:12 - │ │ └── column4:8 => column4:13 + │ │ ├── column3:8 => column3:14 + │ │ └── column4:9 => column4:15 │ └── filters - │ ├── column3:12 IS NOT NULL - │ └── column4:13 IS NOT NULL + │ ├── column3:14 IS NOT NULL + │ └── column4:15 IS NOT NULL ├── scan multi_ref_parent_bc - │ └── columns: multi_ref_parent_bc.b:14!null multi_ref_parent_bc.c:15!null + │ └── columns: multi_ref_parent_bc.b:16!null multi_ref_parent_bc.c:17!null └── filters - ├── column3:12 = multi_ref_parent_bc.b:14 - └── column4:13 = multi_ref_parent_bc.c:15 + ├── column3:14 = multi_ref_parent_bc.b:16 + └── column4:15 = multi_ref_parent_bc.c:17 build INSERT INTO multi_ref_child VALUES (1, NULL, NULL, NULL) @@ -629,10 +629,10 @@ INSERT INTO multi_ref_child VALUES (1, NULL, NULL, NULL) insert multi_ref_child ├── columns: ├── insert-mapping: - │ ├── column1:5 => k:1 - │ ├── column2:6 => a:2 - │ ├── column3:7 => b:3 - │ └── column4:8 => c:4 + │ ├── column1:6 => k:1 + │ ├── column2:7 => a:2 + │ ├── column3:8 => b:3 + │ └── column4:9 => c:4 └── values - ├── columns: column1:5!null column2:6 column3:7 column4:8 + ├── columns: column1:6!null column2:7 column3:8 column4:9 └── (1, NULL::INT8, NULL::INT8, NULL::INT8) diff --git a/pkg/sql/opt/optbuilder/testdata/fk-checks-update b/pkg/sql/opt/optbuilder/testdata/fk-checks-update index ae39c1b927c8..7529f26897f6 100644 --- a/pkg/sql/opt/optbuilder/testdata/fk-checks-update +++ b/pkg/sql/opt/optbuilder/testdata/fk-checks-update @@ -11,64 +11,64 @@ UPDATE child SET p = 4 ---- update child ├── columns: - ├── fetch columns: c:3 child.p:4 + ├── fetch columns: c:4 child.p:5 ├── update-mapping: - │ └── p_new:5 => child.p:2 + │ └── p_new:7 => child.p:2 ├── input binding: &1 ├── project - │ ├── columns: p_new:5!null c:3!null child.p:4!null + │ ├── columns: p_new:7!null c:4!null child.p:5!null child.crdb_internal_mvcc_timestamp:6 │ ├── scan child - │ │ └── columns: c:3!null child.p:4!null + │ │ └── columns: c:4!null child.p:5!null child.crdb_internal_mvcc_timestamp:6 │ └── projections - │ └── 4 [as=p_new:5] + │ └── 4 [as=p_new:7] └── f-k-checks └── f-k-checks-item: child(p) -> parent(p) └── anti-join (hash) - ├── columns: p_new:6!null + ├── columns: p_new:8!null ├── with-scan &1 - │ ├── columns: p_new:6!null + │ ├── columns: p_new:8!null │ └── mapping: - │ └── p_new:5 => p_new:6 + │ └── p_new:7 => p_new:8 ├── scan parent - │ └── columns: parent.p:8!null + │ └── columns: parent.p:10!null └── filters - └── p_new:6 = parent.p:8 + └── p_new:8 = parent.p:10 build UPDATE parent SET p = p+1 ---- update parent ├── columns: - ├── fetch columns: x:4 parent.p:5 other:6 + ├── fetch columns: x:5 parent.p:6 other:7 ├── update-mapping: - │ └── p_new:7 => parent.p:2 + │ └── p_new:9 => parent.p:2 ├── input binding: &1 ├── project - │ ├── columns: p_new:7!null x:4 parent.p:5!null other:6 + │ ├── columns: p_new:9!null x:5 parent.p:6!null other:7 parent.crdb_internal_mvcc_timestamp:8 │ ├── scan parent - │ │ └── columns: x:4 parent.p:5!null other:6 + │ │ └── columns: x:5 parent.p:6!null other:7 parent.crdb_internal_mvcc_timestamp:8 │ └── projections - │ └── parent.p:5 + 1 [as=p_new:7] + │ └── parent.p:6 + 1 [as=p_new:9] └── f-k-checks └── f-k-checks-item: child(p) -> parent(p) └── semi-join (hash) - ├── columns: p:8!null + ├── columns: p:10!null ├── except - │ ├── columns: p:8!null - │ ├── left columns: p:8!null - │ ├── right columns: p_new:9 + │ ├── columns: p:10!null + │ ├── left columns: p:10!null + │ ├── right columns: p_new:11 │ ├── with-scan &1 - │ │ ├── columns: p:8!null + │ │ ├── columns: p:10!null │ │ └── mapping: - │ │ └── parent.p:5 => p:8 + │ │ └── parent.p:6 => p:10 │ └── with-scan &1 - │ ├── columns: p_new:9!null + │ ├── columns: p_new:11!null │ └── mapping: - │ └── p_new:7 => p_new:9 + │ └── p_new:9 => p_new:11 ├── scan child - │ └── columns: child.p:11!null + │ └── columns: child.p:13!null └── filters - └── p:8 = child.p:11 + └── p:10 = child.p:13 exec-ddl CREATE TABLE grandchild (g INT PRIMARY KEY, c INT NOT NULL REFERENCES child(c)) @@ -79,36 +79,36 @@ UPDATE child SET c = 4 ---- update child ├── columns: - ├── fetch columns: child.c:3 p:4 + ├── fetch columns: child.c:4 p:5 ├── update-mapping: - │ └── c_new:5 => child.c:1 + │ └── c_new:7 => child.c:1 ├── input binding: &1 ├── project - │ ├── columns: c_new:5!null child.c:3!null p:4!null + │ ├── columns: c_new:7!null child.c:4!null p:5!null child.crdb_internal_mvcc_timestamp:6 │ ├── scan child - │ │ └── columns: child.c:3!null p:4!null + │ │ └── columns: child.c:4!null p:5!null child.crdb_internal_mvcc_timestamp:6 │ └── projections - │ └── 4 [as=c_new:5] + │ └── 4 [as=c_new:7] └── f-k-checks └── f-k-checks-item: grandchild(c) -> child(c) └── semi-join (hash) - ├── columns: c:6!null + ├── columns: c:8!null ├── except - │ ├── columns: c:6!null - │ ├── left columns: c:6!null - │ ├── right columns: c_new:7 + │ ├── columns: c:8!null + │ ├── left columns: c:8!null + │ ├── right columns: c_new:9 │ ├── with-scan &1 - │ │ ├── columns: c:6!null + │ │ ├── columns: c:8!null │ │ └── mapping: - │ │ └── child.c:3 => c:6 + │ │ └── child.c:4 => c:8 │ └── with-scan &1 - │ ├── columns: c_new:7!null + │ ├── columns: c_new:9!null │ └── mapping: - │ └── c_new:5 => c_new:7 + │ └── c_new:7 => c_new:9 ├── scan grandchild - │ └── columns: grandchild.c:9!null + │ └── columns: grandchild.c:11!null └── filters - └── c:6 = grandchild.c:9 + └── c:8 = grandchild.c:11 # This update shouldn't emit checks for c, since it's unchanged. build @@ -116,101 +116,101 @@ UPDATE child SET p = 4 ---- update child ├── columns: - ├── fetch columns: c:3 child.p:4 + ├── fetch columns: c:4 child.p:5 ├── update-mapping: - │ └── p_new:5 => child.p:2 + │ └── p_new:7 => child.p:2 ├── input binding: &1 ├── project - │ ├── columns: p_new:5!null c:3!null child.p:4!null + │ ├── columns: p_new:7!null c:4!null child.p:5!null child.crdb_internal_mvcc_timestamp:6 │ ├── scan child - │ │ └── columns: c:3!null child.p:4!null + │ │ └── columns: c:4!null child.p:5!null child.crdb_internal_mvcc_timestamp:6 │ └── projections - │ └── 4 [as=p_new:5] + │ └── 4 [as=p_new:7] └── f-k-checks └── f-k-checks-item: child(p) -> parent(p) └── anti-join (hash) - ├── columns: p_new:6!null + ├── columns: p_new:8!null ├── with-scan &1 - │ ├── columns: p_new:6!null + │ ├── columns: p_new:8!null │ └── mapping: - │ └── p_new:5 => p_new:6 + │ └── p_new:7 => p_new:8 ├── scan parent - │ └── columns: parent.p:8!null + │ └── columns: parent.p:10!null └── filters - └── p_new:6 = parent.p:8 + └── p_new:8 = parent.p:10 build UPDATE child SET p = p ---- update child ├── columns: - ├── fetch columns: c:3 child.p:4 + ├── fetch columns: c:4 child.p:5 ├── update-mapping: - │ └── child.p:4 => child.p:2 + │ └── child.p:5 => child.p:2 ├── input binding: &1 ├── scan child - │ └── columns: c:3!null child.p:4!null + │ └── columns: c:4!null child.p:5!null child.crdb_internal_mvcc_timestamp:6 └── f-k-checks └── f-k-checks-item: child(p) -> parent(p) └── anti-join (hash) - ├── columns: p:5!null + ├── columns: p:7!null ├── with-scan &1 - │ ├── columns: p:5!null + │ ├── columns: p:7!null │ └── mapping: - │ └── child.p:4 => p:5 + │ └── child.p:5 => p:7 ├── scan parent - │ └── columns: parent.p:7!null + │ └── columns: parent.p:9!null └── filters - └── p:5 = parent.p:7 + └── p:7 = parent.p:9 build UPDATE child SET p = p+1, c = c+1 ---- update child ├── columns: - ├── fetch columns: child.c:3 child.p:4 + ├── fetch columns: child.c:4 child.p:5 ├── update-mapping: - │ ├── c_new:6 => child.c:1 - │ └── p_new:5 => child.p:2 + │ ├── c_new:8 => child.c:1 + │ └── p_new:7 => child.p:2 ├── input binding: &1 ├── project - │ ├── columns: p_new:5!null c_new:6!null child.c:3!null child.p:4!null + │ ├── columns: p_new:7!null c_new:8!null child.c:4!null child.p:5!null child.crdb_internal_mvcc_timestamp:6 │ ├── scan child - │ │ └── columns: child.c:3!null child.p:4!null + │ │ └── columns: child.c:4!null child.p:5!null child.crdb_internal_mvcc_timestamp:6 │ └── projections - │ ├── child.p:4 + 1 [as=p_new:5] - │ └── child.c:3 + 1 [as=c_new:6] + │ ├── child.p:5 + 1 [as=p_new:7] + │ └── child.c:4 + 1 [as=c_new:8] └── f-k-checks ├── f-k-checks-item: child(p) -> parent(p) │ └── anti-join (hash) - │ ├── columns: p_new:7!null + │ ├── columns: p_new:9!null │ ├── with-scan &1 - │ │ ├── columns: p_new:7!null + │ │ ├── columns: p_new:9!null │ │ └── mapping: - │ │ └── p_new:5 => p_new:7 + │ │ └── p_new:7 => p_new:9 │ ├── scan parent - │ │ └── columns: parent.p:9!null + │ │ └── columns: parent.p:11!null │ └── filters - │ └── p_new:7 = parent.p:9 + │ └── p_new:9 = parent.p:11 └── f-k-checks-item: grandchild(c) -> child(c) └── semi-join (hash) - ├── columns: c:11!null + ├── columns: c:14!null ├── except - │ ├── columns: c:11!null - │ ├── left columns: c:11!null - │ ├── right columns: c_new:12 + │ ├── columns: c:14!null + │ ├── left columns: c:14!null + │ ├── right columns: c_new:15 │ ├── with-scan &1 - │ │ ├── columns: c:11!null + │ │ ├── columns: c:14!null │ │ └── mapping: - │ │ └── child.c:3 => c:11 + │ │ └── child.c:4 => c:14 │ └── with-scan &1 - │ ├── columns: c_new:12!null + │ ├── columns: c_new:15!null │ └── mapping: - │ └── c_new:6 => c_new:12 + │ └── c_new:8 => c_new:15 ├── scan grandchild - │ └── columns: grandchild.c:14!null + │ └── columns: grandchild.c:17!null └── filters - └── c:11 = grandchild.c:14 + └── c:14 = grandchild.c:17 exec-ddl CREATE TABLE child_nullable (c INT PRIMARY KEY, p INT REFERENCES parent(p)) @@ -223,15 +223,15 @@ UPDATE child_nullable SET p = NULL ---- update child_nullable ├── columns: - ├── fetch columns: c:3 p:4 + ├── fetch columns: c:4 p:5 ├── update-mapping: - │ └── p_new:5 => p:2 + │ └── p_new:7 => p:2 └── project - ├── columns: p_new:5 c:3!null p:4 + ├── columns: p_new:7 c:4!null p:5 crdb_internal_mvcc_timestamp:6 ├── scan child_nullable - │ └── columns: c:3!null p:4 + │ └── columns: c:4!null p:5 crdb_internal_mvcc_timestamp:6 └── projections - └── NULL::INT8 [as=p_new:5] + └── NULL::INT8 [as=p_new:7] # Multiple grandchild tables exec-ddl @@ -243,28 +243,28 @@ UPDATE child SET p = 4 ---- update child ├── columns: - ├── fetch columns: c:3 child.p:4 + ├── fetch columns: c:4 child.p:5 ├── update-mapping: - │ └── p_new:5 => child.p:2 + │ └── p_new:7 => child.p:2 ├── input binding: &1 ├── project - │ ├── columns: p_new:5!null c:3!null child.p:4!null + │ ├── columns: p_new:7!null c:4!null child.p:5!null child.crdb_internal_mvcc_timestamp:6 │ ├── scan child - │ │ └── columns: c:3!null child.p:4!null + │ │ └── columns: c:4!null child.p:5!null child.crdb_internal_mvcc_timestamp:6 │ └── projections - │ └── 4 [as=p_new:5] + │ └── 4 [as=p_new:7] └── f-k-checks └── f-k-checks-item: child(p) -> parent(p) └── anti-join (hash) - ├── columns: p_new:6!null + ├── columns: p_new:8!null ├── with-scan &1 - │ ├── columns: p_new:6!null + │ ├── columns: p_new:8!null │ └── mapping: - │ └── p_new:5 => p_new:6 + │ └── p_new:7 => p_new:8 ├── scan parent - │ └── columns: parent.p:8!null + │ └── columns: parent.p:10!null └── filters - └── p_new:6 = parent.p:8 + └── p_new:8 = parent.p:10 exec-ddl CREATE TABLE self (x INT PRIMARY KEY, y INT NOT NULL REFERENCES self(x)) @@ -275,64 +275,64 @@ UPDATE self SET y = 3 ---- update self ├── columns: - ├── fetch columns: x:3 y:4 + ├── fetch columns: x:4 y:5 ├── update-mapping: - │ └── y_new:5 => y:2 + │ └── y_new:7 => y:2 ├── input binding: &1 ├── project - │ ├── columns: y_new:5!null x:3!null y:4!null + │ ├── columns: y_new:7!null x:4!null y:5!null crdb_internal_mvcc_timestamp:6 │ ├── scan self - │ │ └── columns: x:3!null y:4!null + │ │ └── columns: x:4!null y:5!null crdb_internal_mvcc_timestamp:6 │ └── projections - │ └── 3 [as=y_new:5] + │ └── 3 [as=y_new:7] └── f-k-checks └── f-k-checks-item: self(y) -> self(x) └── anti-join (hash) - ├── columns: y_new:6!null + ├── columns: y_new:8!null ├── with-scan &1 - │ ├── columns: y_new:6!null + │ ├── columns: y_new:8!null │ └── mapping: - │ └── y_new:5 => y_new:6 + │ └── y_new:7 => y_new:8 ├── scan self - │ └── columns: x:7!null + │ └── columns: x:9!null └── filters - └── y_new:6 = x:7 + └── y_new:8 = x:9 build UPDATE self SET x = 3 ---- update self ├── columns: - ├── fetch columns: self.x:3 y:4 + ├── fetch columns: self.x:4 y:5 ├── update-mapping: - │ └── x_new:5 => self.x:1 + │ └── x_new:7 => self.x:1 ├── input binding: &1 ├── project - │ ├── columns: x_new:5!null self.x:3!null y:4!null + │ ├── columns: x_new:7!null self.x:4!null y:5!null crdb_internal_mvcc_timestamp:6 │ ├── scan self - │ │ └── columns: self.x:3!null y:4!null + │ │ └── columns: self.x:4!null y:5!null crdb_internal_mvcc_timestamp:6 │ └── projections - │ └── 3 [as=x_new:5] + │ └── 3 [as=x_new:7] └── f-k-checks └── f-k-checks-item: self(y) -> self(x) └── semi-join (hash) - ├── columns: x:6!null + ├── columns: x:8!null ├── except - │ ├── columns: x:6!null - │ ├── left columns: x:6!null - │ ├── right columns: x_new:7 + │ ├── columns: x:8!null + │ ├── left columns: x:8!null + │ ├── right columns: x_new:9 │ ├── with-scan &1 - │ │ ├── columns: x:6!null + │ │ ├── columns: x:8!null │ │ └── mapping: - │ │ └── self.x:3 => x:6 + │ │ └── self.x:4 => x:8 │ └── with-scan &1 - │ ├── columns: x_new:7!null + │ ├── columns: x_new:9!null │ └── mapping: - │ └── x_new:5 => x_new:7 + │ └── x_new:7 => x_new:9 ├── scan self - │ └── columns: y:9!null + │ └── columns: y:11!null └── filters - └── x:6 = y:9 + └── x:8 = y:11 exec-ddl CREATE TABLE parent_multicol (a INT, b INT, c INT, PRIMARY KEY (a,b,c)) @@ -352,22 +352,22 @@ UPDATE child_multicol_simple SET a = 1, b = NULL, c = 1 WHERE k = 1 ---- update child_multicol_simple ├── columns: - ├── fetch columns: k:5 a:6 b:7 c:8 + ├── fetch columns: k:6 a:7 b:8 c:9 ├── update-mapping: - │ ├── a_new:9 => a:2 - │ ├── b_new:10 => b:3 - │ └── a_new:9 => c:4 + │ ├── a_new:11 => a:2 + │ ├── b_new:12 => b:3 + │ └── a_new:11 => c:4 └── project - ├── columns: a_new:9!null b_new:10 k:5!null a:6 b:7 c:8 + ├── columns: a_new:11!null b_new:12 k:6!null a:7 b:8 c:9 crdb_internal_mvcc_timestamp:10 ├── select - │ ├── columns: k:5!null a:6 b:7 c:8 + │ ├── columns: k:6!null a:7 b:8 c:9 crdb_internal_mvcc_timestamp:10 │ ├── scan child_multicol_simple - │ │ └── columns: k:5!null a:6 b:7 c:8 + │ │ └── columns: k:6!null a:7 b:8 c:9 crdb_internal_mvcc_timestamp:10 │ └── filters - │ └── k:5 = 1 + │ └── k:6 = 1 └── projections - ├── 1 [as=a_new:9] - └── NULL::INT8 [as=b_new:10] + ├── 1 [as=a_new:11] + └── NULL::INT8 [as=b_new:12] exec-ddl CREATE TABLE child_multicol_full ( @@ -383,60 +383,60 @@ UPDATE child_multicol_full SET a = 1, b = NULL, c = 1 WHERE k = 1 ---- update child_multicol_full ├── columns: - ├── fetch columns: k:5 child_multicol_full.a:6 child_multicol_full.b:7 child_multicol_full.c:8 + ├── fetch columns: k:6 child_multicol_full.a:7 child_multicol_full.b:8 child_multicol_full.c:9 ├── update-mapping: - │ ├── a_new:9 => child_multicol_full.a:2 - │ ├── b_new:10 => child_multicol_full.b:3 - │ └── a_new:9 => child_multicol_full.c:4 + │ ├── a_new:11 => child_multicol_full.a:2 + │ ├── b_new:12 => child_multicol_full.b:3 + │ └── a_new:11 => child_multicol_full.c:4 ├── input binding: &1 ├── project - │ ├── columns: a_new:9!null b_new:10 k:5!null child_multicol_full.a:6 child_multicol_full.b:7 child_multicol_full.c:8 + │ ├── columns: a_new:11!null b_new:12 k:6!null child_multicol_full.a:7 child_multicol_full.b:8 child_multicol_full.c:9 child_multicol_full.crdb_internal_mvcc_timestamp:10 │ ├── select - │ │ ├── columns: k:5!null child_multicol_full.a:6 child_multicol_full.b:7 child_multicol_full.c:8 + │ │ ├── columns: k:6!null child_multicol_full.a:7 child_multicol_full.b:8 child_multicol_full.c:9 child_multicol_full.crdb_internal_mvcc_timestamp:10 │ │ ├── scan child_multicol_full - │ │ │ └── columns: k:5!null child_multicol_full.a:6 child_multicol_full.b:7 child_multicol_full.c:8 + │ │ │ └── columns: k:6!null child_multicol_full.a:7 child_multicol_full.b:8 child_multicol_full.c:9 child_multicol_full.crdb_internal_mvcc_timestamp:10 │ │ └── filters - │ │ └── k:5 = 1 + │ │ └── k:6 = 1 │ └── projections - │ ├── 1 [as=a_new:9] - │ └── NULL::INT8 [as=b_new:10] + │ ├── 1 [as=a_new:11] + │ └── NULL::INT8 [as=b_new:12] └── f-k-checks └── f-k-checks-item: child_multicol_full(a,b,c) -> parent_multicol(a,b,c) └── anti-join (hash) - ├── columns: a_new:11!null b_new:12 a_new:13!null + ├── columns: a_new:13!null b_new:14 a_new:15!null ├── with-scan &1 - │ ├── columns: a_new:11!null b_new:12 a_new:13!null + │ ├── columns: a_new:13!null b_new:14 a_new:15!null │ └── mapping: - │ ├── a_new:9 => a_new:11 - │ ├── b_new:10 => b_new:12 - │ └── a_new:9 => a_new:13 + │ ├── a_new:11 => a_new:13 + │ ├── b_new:12 => b_new:14 + │ └── a_new:11 => a_new:15 ├── scan parent_multicol - │ └── columns: parent_multicol.a:14!null parent_multicol.b:15!null parent_multicol.c:16!null + │ └── columns: parent_multicol.a:16!null parent_multicol.b:17!null parent_multicol.c:18!null └── filters - ├── a_new:11 = parent_multicol.a:14 - ├── b_new:12 = parent_multicol.b:15 - └── a_new:13 = parent_multicol.c:16 + ├── a_new:13 = parent_multicol.a:16 + ├── b_new:14 = parent_multicol.b:17 + └── a_new:15 = parent_multicol.c:18 build UPDATE child_multicol_full SET a = NULL, b = NULL, c = NULL WHERE k = 1 ---- update child_multicol_full ├── columns: - ├── fetch columns: k:5 a:6 b:7 c:8 + ├── fetch columns: k:6 a:7 b:8 c:9 ├── update-mapping: - │ ├── a_new:9 => a:2 - │ ├── a_new:9 => b:3 - │ └── a_new:9 => c:4 + │ ├── a_new:11 => a:2 + │ ├── a_new:11 => b:3 + │ └── a_new:11 => c:4 └── project - ├── columns: a_new:9 k:5!null a:6 b:7 c:8 + ├── columns: a_new:11 k:6!null a:7 b:8 c:9 crdb_internal_mvcc_timestamp:10 ├── select - │ ├── columns: k:5!null a:6 b:7 c:8 + │ ├── columns: k:6!null a:7 b:8 c:9 crdb_internal_mvcc_timestamp:10 │ ├── scan child_multicol_full - │ │ └── columns: k:5!null a:6 b:7 c:8 + │ │ └── columns: k:6!null a:7 b:8 c:9 crdb_internal_mvcc_timestamp:10 │ └── filters - │ └── k:5 = 1 + │ └── k:6 = 1 └── projections - └── NULL::INT8 [as=a_new:9] + └── NULL::INT8 [as=a_new:11] exec-ddl CREATE TABLE two (a int, b int, primary key (a, b)) @@ -464,65 +464,65 @@ UPDATE fam SET c = 3 ---- update fam ├── columns: - ├── fetch columns: fam.a:7 fam.b:8 c:9 fam.d:10 rowid:12 + ├── fetch columns: fam.a:8 fam.b:9 c:10 fam.d:11 rowid:13 ├── update-mapping: - │ └── c_new:13 => c:3 + │ └── c_new:15 => c:3 ├── input binding: &1 ├── project - │ ├── columns: c_new:13!null fam.a:7 fam.b:8 c:9 fam.d:10 rowid:12!null + │ ├── columns: c_new:15!null fam.a:8 fam.b:9 c:10 fam.d:11 rowid:13!null │ ├── scan fam - │ │ └── columns: fam.a:7 fam.b:8 c:9 fam.d:10 rowid:12!null + │ │ └── columns: fam.a:8 fam.b:9 c:10 fam.d:11 rowid:13!null │ └── projections - │ └── 3 [as=c_new:13] + │ └── 3 [as=c_new:15] └── f-k-checks └── f-k-checks-item: fam(c,d) -> two(a,b) └── anti-join (hash) - ├── columns: c_new:14!null d:15!null + ├── columns: c_new:16!null d:17!null ├── select - │ ├── columns: c_new:14!null d:15!null + │ ├── columns: c_new:16!null d:17!null │ ├── with-scan &1 - │ │ ├── columns: c_new:14!null d:15 + │ │ ├── columns: c_new:16!null d:17 │ │ └── mapping: - │ │ ├── c_new:13 => c_new:14 - │ │ └── fam.d:10 => d:15 + │ │ ├── c_new:15 => c_new:16 + │ │ └── fam.d:11 => d:17 │ └── filters - │ └── d:15 IS NOT NULL + │ └── d:17 IS NOT NULL ├── scan two - │ └── columns: two.a:16!null two.b:17!null + │ └── columns: two.a:18!null two.b:19!null └── filters - ├── c_new:14 = two.a:16 - └── d:15 = two.b:17 + ├── c_new:16 = two.a:18 + └── d:17 = two.b:19 norm UPDATE fam SET d = 3 ---- update fam ├── columns: - ├── fetch columns: fam.c:9 d:10 e:11 rowid:12 + ├── fetch columns: fam.c:10 d:11 e:12 rowid:13 ├── update-mapping: - │ └── d_new:13 => d:4 + │ └── d_new:15 => d:4 ├── input binding: &1 ├── project - │ ├── columns: d_new:13!null fam.c:9 d:10 e:11 rowid:12!null + │ ├── columns: d_new:15!null fam.c:10 d:11 e:12 rowid:13!null │ ├── scan fam - │ │ └── columns: fam.c:9 d:10 e:11 rowid:12!null + │ │ └── columns: fam.c:10 d:11 e:12 rowid:13!null │ └── projections - │ └── 3 [as=d_new:13] + │ └── 3 [as=d_new:15] └── f-k-checks └── f-k-checks-item: fam(c,d) -> two(a,b) └── anti-join (hash) - ├── columns: c:14!null d_new:15!null + ├── columns: c:16!null d_new:17!null ├── select - │ ├── columns: c:14!null d_new:15!null + │ ├── columns: c:16!null d_new:17!null │ ├── with-scan &1 - │ │ ├── columns: c:14 d_new:15!null + │ │ ├── columns: c:16 d_new:17!null │ │ └── mapping: - │ │ ├── fam.c:9 => c:14 - │ │ └── d_new:13 => d_new:15 + │ │ ├── fam.c:10 => c:16 + │ │ └── d_new:15 => d_new:17 │ └── filters - │ └── c:14 IS NOT NULL + │ └── c:16 IS NOT NULL ├── scan two - │ └── columns: two.a:16!null two.b:17!null + │ └── columns: two.a:18!null two.b:19!null └── filters - ├── c:14 = two.a:16 - └── d_new:15 = two.b:17 + ├── c:16 = two.a:18 + └── d_new:17 = two.b:19 diff --git a/pkg/sql/opt/optbuilder/testdata/fk-checks-upsert b/pkg/sql/opt/optbuilder/testdata/fk-checks-upsert index 336240413fb5..1109a4a2ffe3 100644 --- a/pkg/sql/opt/optbuilder/testdata/fk-checks-upsert +++ b/pkg/sql/opt/optbuilder/testdata/fk-checks-upsert @@ -35,108 +35,108 @@ UPSERT INTO c1 VALUES (100, 1), (200, 1) ---- upsert c1 ├── columns: - ├── canary column: 7 - ├── fetch columns: c:7 c1.p:8 i:9 + ├── canary column: 8 + ├── fetch columns: c:8 c1.p:9 i:10 ├── insert-mapping: - │ ├── column1:4 => c:1 - │ ├── column2:5 => c1.p:2 - │ └── column6:6 => i:3 + │ ├── column1:5 => c:1 + │ ├── column2:6 => c1.p:2 + │ └── column7:7 => i:3 ├── update-mapping: - │ ├── column2:5 => c1.p:2 - │ └── column6:6 => i:3 + │ ├── column2:6 => c1.p:2 + │ └── column7:7 => i:3 ├── input binding: &1 ├── project - │ ├── columns: upsert_c:10 column1:4!null column2:5!null column6:6 c:7 c1.p:8 i:9 + │ ├── columns: upsert_c:12 column1:5!null column2:6!null column7:7 c:8 c1.p:9 i:10 c1.crdb_internal_mvcc_timestamp:11 │ ├── left-join (hash) - │ │ ├── columns: column1:4!null column2:5!null column6:6 c:7 c1.p:8 i:9 + │ │ ├── columns: column1:5!null column2:6!null column7:7 c:8 c1.p:9 i:10 c1.crdb_internal_mvcc_timestamp:11 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: column1:4!null column2:5!null column6:6 - │ │ │ ├── grouping columns: column1:4!null + │ │ │ ├── columns: column1:5!null column2:6!null column7:7 + │ │ │ ├── grouping columns: column1:5!null │ │ │ ├── project - │ │ │ │ ├── columns: column6:6 column1:4!null column2:5!null + │ │ │ │ ├── columns: column7:7 column1:5!null column2:6!null │ │ │ │ ├── values - │ │ │ │ │ ├── columns: column1:4!null column2:5!null + │ │ │ │ │ ├── columns: column1:5!null column2:6!null │ │ │ │ │ ├── (100, 1) │ │ │ │ │ └── (200, 1) │ │ │ │ └── projections - │ │ │ │ └── NULL::INT8 [as=column6:6] + │ │ │ │ └── NULL::INT8 [as=column7:7] │ │ │ └── aggregations - │ │ │ ├── first-agg [as=column2:5] - │ │ │ │ └── column2:5 - │ │ │ └── first-agg [as=column6:6] - │ │ │ └── column6:6 + │ │ │ ├── first-agg [as=column2:6] + │ │ │ │ └── column2:6 + │ │ │ └── first-agg [as=column7:7] + │ │ │ └── column7:7 │ │ ├── scan c1 - │ │ │ └── columns: c:7!null c1.p:8!null i:9 + │ │ │ └── columns: c:8!null c1.p:9!null i:10 c1.crdb_internal_mvcc_timestamp:11 │ │ └── filters - │ │ └── column1:4 = c:7 + │ │ └── column1:5 = c:8 │ └── projections - │ └── CASE WHEN c:7 IS NULL THEN column1:4 ELSE c:7 END [as=upsert_c:10] + │ └── CASE WHEN c:8 IS NULL THEN column1:5 ELSE c:8 END [as=upsert_c:12] └── f-k-checks └── f-k-checks-item: c1(p) -> p(p) └── anti-join (hash) - ├── columns: column2:11!null + ├── columns: column2:13!null ├── with-scan &1 - │ ├── columns: column2:11!null + │ ├── columns: column2:13!null │ └── mapping: - │ └── column2:5 => column2:11 + │ └── column2:6 => column2:13 ├── scan p - │ └── columns: p.p:12!null + │ └── columns: p.p:14!null └── filters - └── column2:11 = p.p:12 + └── column2:13 = p.p:14 build UPSERT INTO c1(c) VALUES (100), (200) ---- upsert c1 ├── columns: - ├── canary column: 7 - ├── fetch columns: c:7 c1.p:8 i:9 + ├── canary column: 8 + ├── fetch columns: c:8 c1.p:9 i:10 ├── insert-mapping: - │ ├── column1:4 => c:1 - │ ├── column5:5 => c1.p:2 - │ └── column6:6 => i:3 + │ ├── column1:5 => c:1 + │ ├── column6:6 => c1.p:2 + │ └── column7:7 => i:3 ├── input binding: &1 ├── project - │ ├── columns: upsert_c:10 upsert_p:11 upsert_i:12 column1:4!null column5:5!null column6:6 c:7 c1.p:8 i:9 + │ ├── columns: upsert_c:12 upsert_p:13 upsert_i:14 column1:5!null column6:6!null column7:7 c:8 c1.p:9 i:10 c1.crdb_internal_mvcc_timestamp:11 │ ├── left-join (hash) - │ │ ├── columns: column1:4!null column5:5!null column6:6 c:7 c1.p:8 i:9 + │ │ ├── columns: column1:5!null column6:6!null column7:7 c:8 c1.p:9 i:10 c1.crdb_internal_mvcc_timestamp:11 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: column1:4!null column5:5!null column6:6 - │ │ │ ├── grouping columns: column1:4!null + │ │ │ ├── columns: column1:5!null column6:6!null column7:7 + │ │ │ ├── grouping columns: column1:5!null │ │ │ ├── project - │ │ │ │ ├── columns: column5:5!null column6:6 column1:4!null + │ │ │ │ ├── columns: column6:6!null column7:7 column1:5!null │ │ │ │ ├── values - │ │ │ │ │ ├── columns: column1:4!null + │ │ │ │ │ ├── columns: column1:5!null │ │ │ │ │ ├── (100,) │ │ │ │ │ └── (200,) │ │ │ │ └── projections - │ │ │ │ ├── 5 [as=column5:5] - │ │ │ │ └── NULL::INT8 [as=column6:6] + │ │ │ │ ├── 5 [as=column6:6] + │ │ │ │ └── NULL::INT8 [as=column7:7] │ │ │ └── aggregations - │ │ │ ├── first-agg [as=column5:5] - │ │ │ │ └── column5:5 - │ │ │ └── first-agg [as=column6:6] - │ │ │ └── column6:6 + │ │ │ ├── first-agg [as=column6:6] + │ │ │ │ └── column6:6 + │ │ │ └── first-agg [as=column7:7] + │ │ │ └── column7:7 │ │ ├── scan c1 - │ │ │ └── columns: c:7!null c1.p:8!null i:9 + │ │ │ └── columns: c:8!null c1.p:9!null i:10 c1.crdb_internal_mvcc_timestamp:11 │ │ └── filters - │ │ └── column1:4 = c:7 + │ │ └── column1:5 = c:8 │ └── projections - │ ├── CASE WHEN c:7 IS NULL THEN column1:4 ELSE c:7 END [as=upsert_c:10] - │ ├── CASE WHEN c:7 IS NULL THEN column5:5 ELSE c1.p:8 END [as=upsert_p:11] - │ └── CASE WHEN c:7 IS NULL THEN column6:6 ELSE i:9 END [as=upsert_i:12] + │ ├── CASE WHEN c:8 IS NULL THEN column1:5 ELSE c:8 END [as=upsert_c:12] + │ ├── CASE WHEN c:8 IS NULL THEN column6:6 ELSE c1.p:9 END [as=upsert_p:13] + │ └── CASE WHEN c:8 IS NULL THEN column7:7 ELSE i:10 END [as=upsert_i:14] └── f-k-checks └── f-k-checks-item: c1(p) -> p(p) └── anti-join (hash) - ├── columns: upsert_p:13 + ├── columns: upsert_p:15 ├── with-scan &1 - │ ├── columns: upsert_p:13 + │ ├── columns: upsert_p:15 │ └── mapping: - │ └── upsert_p:11 => upsert_p:13 + │ └── upsert_p:13 => upsert_p:15 ├── scan p - │ └── columns: p.p:14!null + │ └── columns: p.p:16!null └── filters - └── upsert_p:13 = p.p:14 + └── upsert_p:15 = p.p:16 # Use a non-constant input. build @@ -144,172 +144,172 @@ UPSERT INTO c1 SELECT x, y FROM xyzw ---- upsert c1 ├── columns: - ├── canary column: 10 - ├── fetch columns: c:10 c1.p:11 i:12 + ├── canary column: 12 + ├── fetch columns: c:12 c1.p:13 i:14 ├── insert-mapping: - │ ├── x:4 => c:1 - │ ├── xyzw.y:5 => c1.p:2 - │ └── column9:9 => i:3 + │ ├── x:5 => c:1 + │ ├── xyzw.y:6 => c1.p:2 + │ └── column11:11 => i:3 ├── update-mapping: - │ ├── xyzw.y:5 => c1.p:2 - │ └── column9:9 => i:3 + │ ├── xyzw.y:6 => c1.p:2 + │ └── column11:11 => i:3 ├── input binding: &1 ├── project - │ ├── columns: upsert_c:13 x:4 xyzw.y:5 column9:9 c:10 c1.p:11 i:12 + │ ├── columns: upsert_c:16 x:5 xyzw.y:6 column11:11 c:12 c1.p:13 i:14 c1.crdb_internal_mvcc_timestamp:15 │ ├── left-join (hash) - │ │ ├── columns: x:4 xyzw.y:5 column9:9 c:10 c1.p:11 i:12 + │ │ ├── columns: x:5 xyzw.y:6 column11:11 c:12 c1.p:13 i:14 c1.crdb_internal_mvcc_timestamp:15 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: x:4 xyzw.y:5 column9:9 - │ │ │ ├── grouping columns: x:4 + │ │ │ ├── columns: x:5 xyzw.y:6 column11:11 + │ │ │ ├── grouping columns: x:5 │ │ │ ├── project - │ │ │ │ ├── columns: column9:9 x:4 xyzw.y:5 + │ │ │ │ ├── columns: column11:11 x:5 xyzw.y:6 │ │ │ │ ├── project - │ │ │ │ │ ├── columns: x:4 xyzw.y:5 + │ │ │ │ │ ├── columns: x:5 xyzw.y:6 │ │ │ │ │ └── scan xyzw - │ │ │ │ │ └── columns: x:4 xyzw.y:5 z:6 w:7 rowid:8!null + │ │ │ │ │ └── columns: x:5 xyzw.y:6 z:7 w:8 rowid:9!null xyzw.crdb_internal_mvcc_timestamp:10 │ │ │ │ └── projections - │ │ │ │ └── NULL::INT8 [as=column9:9] + │ │ │ │ └── NULL::INT8 [as=column11:11] │ │ │ └── aggregations - │ │ │ ├── first-agg [as=xyzw.y:5] - │ │ │ │ └── xyzw.y:5 - │ │ │ └── first-agg [as=column9:9] - │ │ │ └── column9:9 + │ │ │ ├── first-agg [as=xyzw.y:6] + │ │ │ │ └── xyzw.y:6 + │ │ │ └── first-agg [as=column11:11] + │ │ │ └── column11:11 │ │ ├── scan c1 - │ │ │ └── columns: c:10!null c1.p:11!null i:12 + │ │ │ └── columns: c:12!null c1.p:13!null i:14 c1.crdb_internal_mvcc_timestamp:15 │ │ └── filters - │ │ └── x:4 = c:10 + │ │ └── x:5 = c:12 │ └── projections - │ └── CASE WHEN c:10 IS NULL THEN x:4 ELSE c:10 END [as=upsert_c:13] + │ └── CASE WHEN c:12 IS NULL THEN x:5 ELSE c:12 END [as=upsert_c:16] └── f-k-checks └── f-k-checks-item: c1(p) -> p(p) └── anti-join (hash) - ├── columns: y:14 + ├── columns: y:17 ├── with-scan &1 - │ ├── columns: y:14 + │ ├── columns: y:17 │ └── mapping: - │ └── xyzw.y:5 => y:14 + │ └── xyzw.y:6 => y:17 ├── scan p - │ └── columns: p.p:15!null + │ └── columns: p.p:18!null └── filters - └── y:14 = p.p:15 + └── y:17 = p.p:18 build INSERT INTO c1 VALUES (100, 1), (200, 1) ON CONFLICT (c) DO UPDATE SET p = excluded.p + 1 ---- upsert c1 ├── columns: - ├── canary column: 7 - ├── fetch columns: c:7 c1.p:8 i:9 + ├── canary column: 8 + ├── fetch columns: c:8 c1.p:9 i:10 ├── insert-mapping: - │ ├── column1:4 => c:1 - │ ├── column2:5 => c1.p:2 - │ └── column6:6 => i:3 + │ ├── column1:5 => c:1 + │ ├── column2:6 => c1.p:2 + │ └── column7:7 => i:3 ├── update-mapping: - │ └── upsert_p:12 => c1.p:2 + │ └── upsert_p:14 => c1.p:2 ├── input binding: &1 ├── project - │ ├── columns: upsert_c:11 upsert_p:12!null upsert_i:13 column1:4!null column2:5!null column6:6 c:7 c1.p:8 i:9 p_new:10!null + │ ├── columns: upsert_c:13 upsert_p:14!null upsert_i:15 column1:5!null column2:6!null column7:7 c:8 c1.p:9 i:10 c1.crdb_internal_mvcc_timestamp:11 p_new:12!null │ ├── project - │ │ ├── columns: p_new:10!null column1:4!null column2:5!null column6:6 c:7 c1.p:8 i:9 + │ │ ├── columns: p_new:12!null column1:5!null column2:6!null column7:7 c:8 c1.p:9 i:10 c1.crdb_internal_mvcc_timestamp:11 │ │ ├── left-join (hash) - │ │ │ ├── columns: column1:4!null column2:5!null column6:6 c:7 c1.p:8 i:9 + │ │ │ ├── columns: column1:5!null column2:6!null column7:7 c:8 c1.p:9 i:10 c1.crdb_internal_mvcc_timestamp:11 │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ ├── columns: column1:4!null column2:5!null column6:6 - │ │ │ │ ├── grouping columns: column1:4!null + │ │ │ │ ├── columns: column1:5!null column2:6!null column7:7 + │ │ │ │ ├── grouping columns: column1:5!null │ │ │ │ ├── project - │ │ │ │ │ ├── columns: column6:6 column1:4!null column2:5!null + │ │ │ │ │ ├── columns: column7:7 column1:5!null column2:6!null │ │ │ │ │ ├── values - │ │ │ │ │ │ ├── columns: column1:4!null column2:5!null + │ │ │ │ │ │ ├── columns: column1:5!null column2:6!null │ │ │ │ │ │ ├── (100, 1) │ │ │ │ │ │ └── (200, 1) │ │ │ │ │ └── projections - │ │ │ │ │ └── NULL::INT8 [as=column6:6] + │ │ │ │ │ └── NULL::INT8 [as=column7:7] │ │ │ │ └── aggregations - │ │ │ │ ├── first-agg [as=column2:5] - │ │ │ │ │ └── column2:5 - │ │ │ │ └── first-agg [as=column6:6] - │ │ │ │ └── column6:6 + │ │ │ │ ├── first-agg [as=column2:6] + │ │ │ │ │ └── column2:6 + │ │ │ │ └── first-agg [as=column7:7] + │ │ │ │ └── column7:7 │ │ │ ├── scan c1 - │ │ │ │ └── columns: c:7!null c1.p:8!null i:9 + │ │ │ │ └── columns: c:8!null c1.p:9!null i:10 c1.crdb_internal_mvcc_timestamp:11 │ │ │ └── filters - │ │ │ └── column1:4 = c:7 + │ │ │ └── column1:5 = c:8 │ │ └── projections - │ │ └── column2:5 + 1 [as=p_new:10] + │ │ └── column2:6 + 1 [as=p_new:12] │ └── projections - │ ├── CASE WHEN c:7 IS NULL THEN column1:4 ELSE c:7 END [as=upsert_c:11] - │ ├── CASE WHEN c:7 IS NULL THEN column2:5 ELSE p_new:10 END [as=upsert_p:12] - │ └── CASE WHEN c:7 IS NULL THEN column6:6 ELSE i:9 END [as=upsert_i:13] + │ ├── CASE WHEN c:8 IS NULL THEN column1:5 ELSE c:8 END [as=upsert_c:13] + │ ├── CASE WHEN c:8 IS NULL THEN column2:6 ELSE p_new:12 END [as=upsert_p:14] + │ └── CASE WHEN c:8 IS NULL THEN column7:7 ELSE i:10 END [as=upsert_i:15] └── f-k-checks └── f-k-checks-item: c1(p) -> p(p) └── anti-join (hash) - ├── columns: upsert_p:14!null + ├── columns: upsert_p:16!null ├── with-scan &1 - │ ├── columns: upsert_p:14!null + │ ├── columns: upsert_p:16!null │ └── mapping: - │ └── upsert_p:12 => upsert_p:14 + │ └── upsert_p:14 => upsert_p:16 ├── scan p - │ └── columns: p.p:15!null + │ └── columns: p.p:17!null └── filters - └── upsert_p:14 = p.p:15 + └── upsert_p:16 = p.p:17 build INSERT INTO c1 SELECT u, v FROM uv ON CONFLICT (c) DO UPDATE SET i = c1.c + 1 ---- upsert c1 ├── columns: - ├── canary column: 8 - ├── fetch columns: c:8 c1.p:9 i:10 + ├── canary column: 10 + ├── fetch columns: c:10 c1.p:11 i:12 ├── insert-mapping: - │ ├── u:4 => c:1 - │ ├── v:5 => c1.p:2 - │ └── column7:7 => i:3 + │ ├── u:5 => c:1 + │ ├── v:6 => c1.p:2 + │ └── column9:9 => i:3 ├── update-mapping: - │ └── upsert_i:14 => i:3 + │ └── upsert_i:17 => i:3 ├── input binding: &1 ├── project - │ ├── columns: upsert_c:12 upsert_p:13 upsert_i:14 u:4!null v:5!null column7:7 c:8 c1.p:9 i:10 i_new:11 + │ ├── columns: upsert_c:15 upsert_p:16 upsert_i:17 u:5!null v:6!null column9:9 c:10 c1.p:11 i:12 c1.crdb_internal_mvcc_timestamp:13 i_new:14 │ ├── project - │ │ ├── columns: i_new:11 u:4!null v:5!null column7:7 c:8 c1.p:9 i:10 + │ │ ├── columns: i_new:14 u:5!null v:6!null column9:9 c:10 c1.p:11 i:12 c1.crdb_internal_mvcc_timestamp:13 │ │ ├── left-join (hash) - │ │ │ ├── columns: u:4!null v:5!null column7:7 c:8 c1.p:9 i:10 + │ │ │ ├── columns: u:5!null v:6!null column9:9 c:10 c1.p:11 i:12 c1.crdb_internal_mvcc_timestamp:13 │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ ├── columns: u:4!null v:5!null column7:7 - │ │ │ │ ├── grouping columns: u:4!null + │ │ │ │ ├── columns: u:5!null v:6!null column9:9 + │ │ │ │ ├── grouping columns: u:5!null │ │ │ │ ├── project - │ │ │ │ │ ├── columns: column7:7 u:4!null v:5!null + │ │ │ │ │ ├── columns: column9:9 u:5!null v:6!null │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: u:4!null v:5!null + │ │ │ │ │ │ ├── columns: u:5!null v:6!null │ │ │ │ │ │ └── scan uv - │ │ │ │ │ │ └── columns: u:4!null v:5!null rowid:6!null + │ │ │ │ │ │ └── columns: u:5!null v:6!null rowid:7!null uv.crdb_internal_mvcc_timestamp:8 │ │ │ │ │ └── projections - │ │ │ │ │ └── NULL::INT8 [as=column7:7] + │ │ │ │ │ └── NULL::INT8 [as=column9:9] │ │ │ │ └── aggregations - │ │ │ │ ├── first-agg [as=v:5] - │ │ │ │ │ └── v:5 - │ │ │ │ └── first-agg [as=column7:7] - │ │ │ │ └── column7:7 + │ │ │ │ ├── first-agg [as=v:6] + │ │ │ │ │ └── v:6 + │ │ │ │ └── first-agg [as=column9:9] + │ │ │ │ └── column9:9 │ │ │ ├── scan c1 - │ │ │ │ └── columns: c:8!null c1.p:9!null i:10 + │ │ │ │ └── columns: c:10!null c1.p:11!null i:12 c1.crdb_internal_mvcc_timestamp:13 │ │ │ └── filters - │ │ │ └── u:4 = c:8 + │ │ │ └── u:5 = c:10 │ │ └── projections - │ │ └── c:8 + 1 [as=i_new:11] + │ │ └── c:10 + 1 [as=i_new:14] │ └── projections - │ ├── CASE WHEN c:8 IS NULL THEN u:4 ELSE c:8 END [as=upsert_c:12] - │ ├── CASE WHEN c:8 IS NULL THEN v:5 ELSE c1.p:9 END [as=upsert_p:13] - │ └── CASE WHEN c:8 IS NULL THEN column7:7 ELSE i_new:11 END [as=upsert_i:14] + │ ├── CASE WHEN c:10 IS NULL THEN u:5 ELSE c:10 END [as=upsert_c:15] + │ ├── CASE WHEN c:10 IS NULL THEN v:6 ELSE c1.p:11 END [as=upsert_p:16] + │ └── CASE WHEN c:10 IS NULL THEN column9:9 ELSE i_new:14 END [as=upsert_i:17] └── f-k-checks └── f-k-checks-item: c1(p) -> p(p) └── anti-join (hash) - ├── columns: upsert_p:15 + ├── columns: upsert_p:18 ├── with-scan &1 - │ ├── columns: upsert_p:15 + │ ├── columns: upsert_p:18 │ └── mapping: - │ └── upsert_p:13 => upsert_p:15 + │ └── upsert_p:16 => upsert_p:18 ├── scan p - │ └── columns: p.p:16!null + │ └── columns: p.p:19!null └── filters - └── upsert_p:15 = p.p:16 + └── upsert_p:18 = p.p:19 exec-ddl CREATE TABLE c2 (c INT PRIMARY KEY, FOREIGN KEY (c) REFERENCES p(p)) @@ -320,46 +320,46 @@ INSERT INTO c2 VALUES (1), (2) ON CONFLICT (c) DO UPDATE SET c = 1 ---- upsert c2 ├── columns: - ├── canary column: 3 - ├── fetch columns: c:3 + ├── canary column: 4 + ├── fetch columns: c:4 ├── insert-mapping: - │ └── column1:2 => c:1 + │ └── column1:3 => c:1 ├── update-mapping: - │ └── upsert_c:5 => c:1 + │ └── upsert_c:7 => c:1 ├── input binding: &1 ├── project - │ ├── columns: upsert_c:5!null column1:2!null c:3 c_new:4!null + │ ├── columns: upsert_c:7!null column1:3!null c:4 c2.crdb_internal_mvcc_timestamp:5 c_new:6!null │ ├── project - │ │ ├── columns: c_new:4!null column1:2!null c:3 + │ │ ├── columns: c_new:6!null column1:3!null c:4 c2.crdb_internal_mvcc_timestamp:5 │ │ ├── left-join (hash) - │ │ │ ├── columns: column1:2!null c:3 + │ │ │ ├── columns: column1:3!null c:4 c2.crdb_internal_mvcc_timestamp:5 │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ ├── columns: column1:2!null - │ │ │ │ ├── grouping columns: column1:2!null + │ │ │ │ ├── columns: column1:3!null + │ │ │ │ ├── grouping columns: column1:3!null │ │ │ │ └── values - │ │ │ │ ├── columns: column1:2!null + │ │ │ │ ├── columns: column1:3!null │ │ │ │ ├── (1,) │ │ │ │ └── (2,) │ │ │ ├── scan c2 - │ │ │ │ └── columns: c:3!null + │ │ │ │ └── columns: c:4!null c2.crdb_internal_mvcc_timestamp:5 │ │ │ └── filters - │ │ │ └── column1:2 = c:3 + │ │ │ └── column1:3 = c:4 │ │ └── projections - │ │ └── 1 [as=c_new:4] + │ │ └── 1 [as=c_new:6] │ └── projections - │ └── CASE WHEN c:3 IS NULL THEN column1:2 ELSE c_new:4 END [as=upsert_c:5] + │ └── CASE WHEN c:4 IS NULL THEN column1:3 ELSE c_new:6 END [as=upsert_c:7] └── f-k-checks └── f-k-checks-item: c2(c) -> p(p) └── anti-join (hash) - ├── columns: upsert_c:6!null + ├── columns: upsert_c:8!null ├── with-scan &1 - │ ├── columns: upsert_c:6!null + │ ├── columns: upsert_c:8!null │ └── mapping: - │ └── upsert_c:5 => upsert_c:6 + │ └── upsert_c:7 => upsert_c:8 ├── scan p - │ └── columns: p:7!null + │ └── columns: p:9!null └── filters - └── upsert_c:6 = p:7 + └── upsert_c:8 = p:9 exec-ddl CREATE TABLE c3 (c INT PRIMARY KEY, p INT REFERENCES p(p)); @@ -372,103 +372,103 @@ UPSERT INTO c3 VALUES (100, 1), (200, NULL) ---- upsert c3 ├── columns: - ├── canary column: 5 - ├── fetch columns: c:5 c3.p:6 + ├── canary column: 6 + ├── fetch columns: c:6 c3.p:7 ├── insert-mapping: - │ ├── column1:3 => c:1 - │ └── column2:4 => c3.p:2 + │ ├── column1:4 => c:1 + │ └── column2:5 => c3.p:2 ├── update-mapping: - │ └── column2:4 => c3.p:2 + │ └── column2:5 => c3.p:2 ├── input binding: &1 ├── project - │ ├── columns: upsert_c:7 column1:3!null column2:4 c:5 c3.p:6 + │ ├── columns: upsert_c:9 column1:4!null column2:5 c:6 c3.p:7 c3.crdb_internal_mvcc_timestamp:8 │ ├── left-join (hash) - │ │ ├── columns: column1:3!null column2:4 c:5 c3.p:6 + │ │ ├── columns: column1:4!null column2:5 c:6 c3.p:7 c3.crdb_internal_mvcc_timestamp:8 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: column1:3!null column2:4 - │ │ │ ├── grouping columns: column1:3!null + │ │ │ ├── columns: column1:4!null column2:5 + │ │ │ ├── grouping columns: column1:4!null │ │ │ ├── values - │ │ │ │ ├── columns: column1:3!null column2:4 + │ │ │ │ ├── columns: column1:4!null column2:5 │ │ │ │ ├── (100, 1) │ │ │ │ └── (200, NULL::INT8) │ │ │ └── aggregations - │ │ │ └── first-agg [as=column2:4] - │ │ │ └── column2:4 + │ │ │ └── first-agg [as=column2:5] + │ │ │ └── column2:5 │ │ ├── scan c3 - │ │ │ └── columns: c:5!null c3.p:6 + │ │ │ └── columns: c:6!null c3.p:7 c3.crdb_internal_mvcc_timestamp:8 │ │ └── filters - │ │ └── column1:3 = c:5 + │ │ └── column1:4 = c:6 │ └── projections - │ └── CASE WHEN c:5 IS NULL THEN column1:3 ELSE c:5 END [as=upsert_c:7] + │ └── CASE WHEN c:6 IS NULL THEN column1:4 ELSE c:6 END [as=upsert_c:9] └── f-k-checks └── f-k-checks-item: c3(p) -> p(p) └── anti-join (hash) - ├── columns: column2:8!null + ├── columns: column2:10!null ├── select - │ ├── columns: column2:8!null + │ ├── columns: column2:10!null │ ├── with-scan &1 - │ │ ├── columns: column2:8 + │ │ ├── columns: column2:10 │ │ └── mapping: - │ │ └── column2:4 => column2:8 + │ │ └── column2:5 => column2:10 │ └── filters - │ └── column2:8 IS NOT NULL + │ └── column2:10 IS NOT NULL ├── scan p - │ └── columns: p.p:9!null + │ └── columns: p.p:11!null └── filters - └── column2:8 = p.p:9 + └── column2:10 = p.p:11 build UPSERT INTO c3(c) VALUES (100), (200) ---- upsert c3 ├── columns: - ├── canary column: 5 - ├── fetch columns: c:5 c3.p:6 + ├── canary column: 6 + ├── fetch columns: c:6 c3.p:7 ├── insert-mapping: - │ ├── column1:3 => c:1 - │ └── column4:4 => c3.p:2 + │ ├── column1:4 => c:1 + │ └── column5:5 => c3.p:2 ├── input binding: &1 ├── project - │ ├── columns: upsert_c:7 upsert_p:8 column1:3!null column4:4 c:5 c3.p:6 + │ ├── columns: upsert_c:9 upsert_p:10 column1:4!null column5:5 c:6 c3.p:7 c3.crdb_internal_mvcc_timestamp:8 │ ├── left-join (hash) - │ │ ├── columns: column1:3!null column4:4 c:5 c3.p:6 + │ │ ├── columns: column1:4!null column5:5 c:6 c3.p:7 c3.crdb_internal_mvcc_timestamp:8 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: column1:3!null column4:4 - │ │ │ ├── grouping columns: column1:3!null + │ │ │ ├── columns: column1:4!null column5:5 + │ │ │ ├── grouping columns: column1:4!null │ │ │ ├── project - │ │ │ │ ├── columns: column4:4 column1:3!null + │ │ │ │ ├── columns: column5:5 column1:4!null │ │ │ │ ├── values - │ │ │ │ │ ├── columns: column1:3!null + │ │ │ │ │ ├── columns: column1:4!null │ │ │ │ │ ├── (100,) │ │ │ │ │ └── (200,) │ │ │ │ └── projections - │ │ │ │ └── NULL::INT8 [as=column4:4] + │ │ │ │ └── NULL::INT8 [as=column5:5] │ │ │ └── aggregations - │ │ │ └── first-agg [as=column4:4] - │ │ │ └── column4:4 + │ │ │ └── first-agg [as=column5:5] + │ │ │ └── column5:5 │ │ ├── scan c3 - │ │ │ └── columns: c:5!null c3.p:6 + │ │ │ └── columns: c:6!null c3.p:7 c3.crdb_internal_mvcc_timestamp:8 │ │ └── filters - │ │ └── column1:3 = c:5 + │ │ └── column1:4 = c:6 │ └── projections - │ ├── CASE WHEN c:5 IS NULL THEN column1:3 ELSE c:5 END [as=upsert_c:7] - │ └── CASE WHEN c:5 IS NULL THEN column4:4 ELSE c3.p:6 END [as=upsert_p:8] + │ ├── CASE WHEN c:6 IS NULL THEN column1:4 ELSE c:6 END [as=upsert_c:9] + │ └── CASE WHEN c:6 IS NULL THEN column5:5 ELSE c3.p:7 END [as=upsert_p:10] └── f-k-checks └── f-k-checks-item: c3(p) -> p(p) └── anti-join (hash) - ├── columns: upsert_p:9!null + ├── columns: upsert_p:11!null ├── select - │ ├── columns: upsert_p:9!null + │ ├── columns: upsert_p:11!null │ ├── with-scan &1 - │ │ ├── columns: upsert_p:9 + │ │ ├── columns: upsert_p:11 │ │ └── mapping: - │ │ └── upsert_p:8 => upsert_p:9 + │ │ └── upsert_p:10 => upsert_p:11 │ └── filters - │ └── upsert_p:9 IS NOT NULL + │ └── upsert_p:11 IS NOT NULL ├── scan p - │ └── columns: p.p:10!null + │ └── columns: p.p:12!null └── filters - └── upsert_p:9 = p.p:10 + └── upsert_p:11 = p.p:12 exec-ddl CREATE TABLE c4 (c INT PRIMARY KEY, a INT REFERENCES p(p), other INT, UNIQUE(a)) @@ -479,118 +479,118 @@ INSERT INTO c4 SELECT x, y, z FROM xyzw ON CONFLICT (a) DO UPDATE SET other = 1 ---- upsert c4 ├── columns: - ├── canary column: 9 - ├── fetch columns: c:9 a:10 c4.other:11 + ├── canary column: 11 + ├── fetch columns: c:11 a:12 c4.other:13 ├── insert-mapping: - │ ├── x:4 => c:1 - │ ├── y:5 => a:2 - │ └── z:6 => c4.other:3 + │ ├── x:5 => c:1 + │ ├── y:6 => a:2 + │ └── z:7 => c4.other:3 ├── update-mapping: - │ └── upsert_other:15 => c4.other:3 + │ └── upsert_other:18 => c4.other:3 ├── input binding: &1 ├── project - │ ├── columns: upsert_c:13 upsert_a:14 upsert_other:15 x:4 y:5 z:6 c:9 a:10 c4.other:11 other_new:12!null + │ ├── columns: upsert_c:16 upsert_a:17 upsert_other:18 x:5 y:6 z:7 c:11 a:12 c4.other:13 c4.crdb_internal_mvcc_timestamp:14 other_new:15!null │ ├── project - │ │ ├── columns: other_new:12!null x:4 y:5 z:6 c:9 a:10 c4.other:11 + │ │ ├── columns: other_new:15!null x:5 y:6 z:7 c:11 a:12 c4.other:13 c4.crdb_internal_mvcc_timestamp:14 │ │ ├── left-join (hash) - │ │ │ ├── columns: x:4 y:5 z:6 c:9 a:10 c4.other:11 + │ │ │ ├── columns: x:5 y:6 z:7 c:11 a:12 c4.other:13 c4.crdb_internal_mvcc_timestamp:14 │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ ├── columns: x:4 y:5 z:6 - │ │ │ │ ├── grouping columns: y:5 + │ │ │ │ ├── columns: x:5 y:6 z:7 + │ │ │ │ ├── grouping columns: y:6 │ │ │ │ ├── project - │ │ │ │ │ ├── columns: x:4 y:5 z:6 + │ │ │ │ │ ├── columns: x:5 y:6 z:7 │ │ │ │ │ └── scan xyzw - │ │ │ │ │ └── columns: x:4 y:5 z:6 w:7 rowid:8!null + │ │ │ │ │ └── columns: x:5 y:6 z:7 w:8 rowid:9!null xyzw.crdb_internal_mvcc_timestamp:10 │ │ │ │ └── aggregations - │ │ │ │ ├── first-agg [as=x:4] - │ │ │ │ │ └── x:4 - │ │ │ │ └── first-agg [as=z:6] - │ │ │ │ └── z:6 + │ │ │ │ ├── first-agg [as=x:5] + │ │ │ │ │ └── x:5 + │ │ │ │ └── first-agg [as=z:7] + │ │ │ │ └── z:7 │ │ │ ├── scan c4 - │ │ │ │ └── columns: c:9!null a:10 c4.other:11 + │ │ │ │ └── columns: c:11!null a:12 c4.other:13 c4.crdb_internal_mvcc_timestamp:14 │ │ │ └── filters - │ │ │ └── y:5 = a:10 + │ │ │ └── y:6 = a:12 │ │ └── projections - │ │ └── 1 [as=other_new:12] + │ │ └── 1 [as=other_new:15] │ └── projections - │ ├── CASE WHEN c:9 IS NULL THEN x:4 ELSE c:9 END [as=upsert_c:13] - │ ├── CASE WHEN c:9 IS NULL THEN y:5 ELSE a:10 END [as=upsert_a:14] - │ └── CASE WHEN c:9 IS NULL THEN z:6 ELSE other_new:12 END [as=upsert_other:15] + │ ├── CASE WHEN c:11 IS NULL THEN x:5 ELSE c:11 END [as=upsert_c:16] + │ ├── CASE WHEN c:11 IS NULL THEN y:6 ELSE a:12 END [as=upsert_a:17] + │ └── CASE WHEN c:11 IS NULL THEN z:7 ELSE other_new:15 END [as=upsert_other:18] └── f-k-checks └── f-k-checks-item: c4(a) -> p(p) └── anti-join (hash) - ├── columns: upsert_a:16!null + ├── columns: upsert_a:19!null ├── select - │ ├── columns: upsert_a:16!null + │ ├── columns: upsert_a:19!null │ ├── with-scan &1 - │ │ ├── columns: upsert_a:16 + │ │ ├── columns: upsert_a:19 │ │ └── mapping: - │ │ └── upsert_a:14 => upsert_a:16 + │ │ └── upsert_a:17 => upsert_a:19 │ └── filters - │ └── upsert_a:16 IS NOT NULL + │ └── upsert_a:19 IS NOT NULL ├── scan p - │ └── columns: p:17!null + │ └── columns: p:20!null └── filters - └── upsert_a:16 = p:17 + └── upsert_a:19 = p:20 build INSERT INTO c4 SELECT x, y, z FROM xyzw ON CONFLICT (a) DO UPDATE SET a = 5 ---- upsert c4 ├── columns: - ├── canary column: 9 - ├── fetch columns: c:9 a:10 c4.other:11 + ├── canary column: 11 + ├── fetch columns: c:11 a:12 c4.other:13 ├── insert-mapping: - │ ├── x:4 => c:1 - │ ├── y:5 => a:2 - │ └── z:6 => c4.other:3 + │ ├── x:5 => c:1 + │ ├── y:6 => a:2 + │ └── z:7 => c4.other:3 ├── update-mapping: - │ └── upsert_a:14 => a:2 + │ └── upsert_a:17 => a:2 ├── input binding: &1 ├── project - │ ├── columns: upsert_c:13 upsert_a:14 upsert_other:15 x:4 y:5 z:6 c:9 a:10 c4.other:11 a_new:12!null + │ ├── columns: upsert_c:16 upsert_a:17 upsert_other:18 x:5 y:6 z:7 c:11 a:12 c4.other:13 c4.crdb_internal_mvcc_timestamp:14 a_new:15!null │ ├── project - │ │ ├── columns: a_new:12!null x:4 y:5 z:6 c:9 a:10 c4.other:11 + │ │ ├── columns: a_new:15!null x:5 y:6 z:7 c:11 a:12 c4.other:13 c4.crdb_internal_mvcc_timestamp:14 │ │ ├── left-join (hash) - │ │ │ ├── columns: x:4 y:5 z:6 c:9 a:10 c4.other:11 + │ │ │ ├── columns: x:5 y:6 z:7 c:11 a:12 c4.other:13 c4.crdb_internal_mvcc_timestamp:14 │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ ├── columns: x:4 y:5 z:6 - │ │ │ │ ├── grouping columns: y:5 + │ │ │ │ ├── columns: x:5 y:6 z:7 + │ │ │ │ ├── grouping columns: y:6 │ │ │ │ ├── project - │ │ │ │ │ ├── columns: x:4 y:5 z:6 + │ │ │ │ │ ├── columns: x:5 y:6 z:7 │ │ │ │ │ └── scan xyzw - │ │ │ │ │ └── columns: x:4 y:5 z:6 w:7 rowid:8!null + │ │ │ │ │ └── columns: x:5 y:6 z:7 w:8 rowid:9!null xyzw.crdb_internal_mvcc_timestamp:10 │ │ │ │ └── aggregations - │ │ │ │ ├── first-agg [as=x:4] - │ │ │ │ │ └── x:4 - │ │ │ │ └── first-agg [as=z:6] - │ │ │ │ └── z:6 + │ │ │ │ ├── first-agg [as=x:5] + │ │ │ │ │ └── x:5 + │ │ │ │ └── first-agg [as=z:7] + │ │ │ │ └── z:7 │ │ │ ├── scan c4 - │ │ │ │ └── columns: c:9!null a:10 c4.other:11 + │ │ │ │ └── columns: c:11!null a:12 c4.other:13 c4.crdb_internal_mvcc_timestamp:14 │ │ │ └── filters - │ │ │ └── y:5 = a:10 + │ │ │ └── y:6 = a:12 │ │ └── projections - │ │ └── 5 [as=a_new:12] + │ │ └── 5 [as=a_new:15] │ └── projections - │ ├── CASE WHEN c:9 IS NULL THEN x:4 ELSE c:9 END [as=upsert_c:13] - │ ├── CASE WHEN c:9 IS NULL THEN y:5 ELSE a_new:12 END [as=upsert_a:14] - │ └── CASE WHEN c:9 IS NULL THEN z:6 ELSE c4.other:11 END [as=upsert_other:15] + │ ├── CASE WHEN c:11 IS NULL THEN x:5 ELSE c:11 END [as=upsert_c:16] + │ ├── CASE WHEN c:11 IS NULL THEN y:6 ELSE a_new:15 END [as=upsert_a:17] + │ └── CASE WHEN c:11 IS NULL THEN z:7 ELSE c4.other:13 END [as=upsert_other:18] └── f-k-checks └── f-k-checks-item: c4(a) -> p(p) └── anti-join (hash) - ├── columns: upsert_a:16!null + ├── columns: upsert_a:19!null ├── select - │ ├── columns: upsert_a:16!null + │ ├── columns: upsert_a:19!null │ ├── with-scan &1 - │ │ ├── columns: upsert_a:16 + │ │ ├── columns: upsert_a:19 │ │ └── mapping: - │ │ └── upsert_a:14 => upsert_a:16 + │ │ └── upsert_a:17 => upsert_a:19 │ └── filters - │ └── upsert_a:16 IS NOT NULL + │ └── upsert_a:19 IS NOT NULL ├── scan p - │ └── columns: p:17!null + │ └── columns: p:20!null └── filters - └── upsert_a:16 = p:17 + └── upsert_a:19 = p:20 # ------------------------------------------ @@ -624,55 +624,55 @@ UPSERT INTO cpq VALUES (1, 1, 1, 1) ---- upsert cpq ├── columns: - ├── canary column: 9 - ├── fetch columns: c:9 cpq.p:10 cpq.q:11 cpq.other:12 + ├── canary column: 10 + ├── fetch columns: c:10 cpq.p:11 cpq.q:12 cpq.other:13 ├── insert-mapping: - │ ├── column1:5 => c:1 - │ ├── column2:6 => cpq.p:2 - │ ├── column3:7 => cpq.q:3 - │ └── column4:8 => cpq.other:4 + │ ├── column1:6 => c:1 + │ ├── column2:7 => cpq.p:2 + │ ├── column3:8 => cpq.q:3 + │ └── column4:9 => cpq.other:4 ├── update-mapping: - │ ├── column2:6 => cpq.p:2 - │ ├── column3:7 => cpq.q:3 - │ └── column4:8 => cpq.other:4 + │ ├── column2:7 => cpq.p:2 + │ ├── column3:8 => cpq.q:3 + │ └── column4:9 => cpq.other:4 ├── input binding: &1 ├── project - │ ├── columns: upsert_c:13 column1:5!null column2:6!null column3:7!null column4:8!null c:9 cpq.p:10 cpq.q:11 cpq.other:12 + │ ├── columns: upsert_c:15 column1:6!null column2:7!null column3:8!null column4:9!null c:10 cpq.p:11 cpq.q:12 cpq.other:13 cpq.crdb_internal_mvcc_timestamp:14 │ ├── left-join (hash) - │ │ ├── columns: column1:5!null column2:6!null column3:7!null column4:8!null c:9 cpq.p:10 cpq.q:11 cpq.other:12 + │ │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null c:10 cpq.p:11 cpq.q:12 cpq.other:13 cpq.crdb_internal_mvcc_timestamp:14 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null column4:8!null - │ │ │ ├── grouping columns: column1:5!null + │ │ │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null + │ │ │ ├── grouping columns: column1:6!null │ │ │ ├── values - │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null column4:8!null + │ │ │ │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null │ │ │ │ └── (1, 1, 1, 1) │ │ │ └── aggregations - │ │ │ ├── first-agg [as=column2:6] - │ │ │ │ └── column2:6 - │ │ │ ├── first-agg [as=column3:7] - │ │ │ │ └── column3:7 - │ │ │ └── first-agg [as=column4:8] - │ │ │ └── column4:8 + │ │ │ ├── first-agg [as=column2:7] + │ │ │ │ └── column2:7 + │ │ │ ├── first-agg [as=column3:8] + │ │ │ │ └── column3:8 + │ │ │ └── first-agg [as=column4:9] + │ │ │ └── column4:9 │ │ ├── scan cpq - │ │ │ └── columns: c:9!null cpq.p:10 cpq.q:11 cpq.other:12 + │ │ │ └── columns: c:10!null cpq.p:11 cpq.q:12 cpq.other:13 cpq.crdb_internal_mvcc_timestamp:14 │ │ └── filters - │ │ └── column1:5 = c:9 + │ │ └── column1:6 = c:10 │ └── projections - │ └── CASE WHEN c:9 IS NULL THEN column1:5 ELSE c:9 END [as=upsert_c:13] + │ └── CASE WHEN c:10 IS NULL THEN column1:6 ELSE c:10 END [as=upsert_c:15] └── f-k-checks └── f-k-checks-item: cpq(p,q) -> pq(p,q) └── anti-join (hash) - ├── columns: column2:14!null column3:15!null + ├── columns: column2:16!null column3:17!null ├── with-scan &1 - │ ├── columns: column2:14!null column3:15!null + │ ├── columns: column2:16!null column3:17!null │ └── mapping: - │ ├── column2:6 => column2:14 - │ └── column3:7 => column3:15 + │ ├── column2:7 => column2:16 + │ └── column3:8 => column3:17 ├── scan pq - │ └── columns: pq.p:17 pq.q:18 + │ └── columns: pq.p:19 pq.q:20 └── filters - ├── column2:14 = pq.p:17 - └── column3:15 = pq.q:18 + ├── column2:16 = pq.p:19 + └── column3:17 = pq.q:20 # In this case, the input columns can be null. build @@ -680,193 +680,193 @@ UPSERT INTO cpq SELECT x,y,z,w FROM xyzw ---- upsert cpq ├── columns: - ├── canary column: 10 - ├── fetch columns: c:10 cpq.p:11 cpq.q:12 cpq.other:13 + ├── canary column: 12 + ├── fetch columns: c:12 cpq.p:13 cpq.q:14 cpq.other:15 ├── insert-mapping: - │ ├── x:5 => c:1 - │ ├── xyzw.y:6 => cpq.p:2 - │ ├── xyzw.z:7 => cpq.q:3 - │ └── w:8 => cpq.other:4 + │ ├── x:6 => c:1 + │ ├── xyzw.y:7 => cpq.p:2 + │ ├── xyzw.z:8 => cpq.q:3 + │ └── w:9 => cpq.other:4 ├── update-mapping: - │ ├── xyzw.y:6 => cpq.p:2 - │ ├── xyzw.z:7 => cpq.q:3 - │ └── w:8 => cpq.other:4 + │ ├── xyzw.y:7 => cpq.p:2 + │ ├── xyzw.z:8 => cpq.q:3 + │ └── w:9 => cpq.other:4 ├── input binding: &1 ├── project - │ ├── columns: upsert_c:14 x:5 xyzw.y:6 xyzw.z:7 w:8 c:10 cpq.p:11 cpq.q:12 cpq.other:13 + │ ├── columns: upsert_c:17 x:6 xyzw.y:7 xyzw.z:8 w:9 c:12 cpq.p:13 cpq.q:14 cpq.other:15 cpq.crdb_internal_mvcc_timestamp:16 │ ├── left-join (hash) - │ │ ├── columns: x:5 xyzw.y:6 xyzw.z:7 w:8 c:10 cpq.p:11 cpq.q:12 cpq.other:13 + │ │ ├── columns: x:6 xyzw.y:7 xyzw.z:8 w:9 c:12 cpq.p:13 cpq.q:14 cpq.other:15 cpq.crdb_internal_mvcc_timestamp:16 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: x:5 xyzw.y:6 xyzw.z:7 w:8 - │ │ │ ├── grouping columns: x:5 + │ │ │ ├── columns: x:6 xyzw.y:7 xyzw.z:8 w:9 + │ │ │ ├── grouping columns: x:6 │ │ │ ├── project - │ │ │ │ ├── columns: x:5 xyzw.y:6 xyzw.z:7 w:8 + │ │ │ │ ├── columns: x:6 xyzw.y:7 xyzw.z:8 w:9 │ │ │ │ └── scan xyzw - │ │ │ │ └── columns: x:5 xyzw.y:6 xyzw.z:7 w:8 rowid:9!null + │ │ │ │ └── columns: x:6 xyzw.y:7 xyzw.z:8 w:9 rowid:10!null xyzw.crdb_internal_mvcc_timestamp:11 │ │ │ └── aggregations - │ │ │ ├── first-agg [as=xyzw.y:6] - │ │ │ │ └── xyzw.y:6 - │ │ │ ├── first-agg [as=xyzw.z:7] - │ │ │ │ └── xyzw.z:7 - │ │ │ └── first-agg [as=w:8] - │ │ │ └── w:8 + │ │ │ ├── first-agg [as=xyzw.y:7] + │ │ │ │ └── xyzw.y:7 + │ │ │ ├── first-agg [as=xyzw.z:8] + │ │ │ │ └── xyzw.z:8 + │ │ │ └── first-agg [as=w:9] + │ │ │ └── w:9 │ │ ├── scan cpq - │ │ │ └── columns: c:10!null cpq.p:11 cpq.q:12 cpq.other:13 + │ │ │ └── columns: c:12!null cpq.p:13 cpq.q:14 cpq.other:15 cpq.crdb_internal_mvcc_timestamp:16 │ │ └── filters - │ │ └── x:5 = c:10 + │ │ └── x:6 = c:12 │ └── projections - │ └── CASE WHEN c:10 IS NULL THEN x:5 ELSE c:10 END [as=upsert_c:14] + │ └── CASE WHEN c:12 IS NULL THEN x:6 ELSE c:12 END [as=upsert_c:17] └── f-k-checks └── f-k-checks-item: cpq(p,q) -> pq(p,q) └── anti-join (hash) - ├── columns: y:15!null z:16!null + ├── columns: y:18!null z:19!null ├── select - │ ├── columns: y:15!null z:16!null + │ ├── columns: y:18!null z:19!null │ ├── with-scan &1 - │ │ ├── columns: y:15 z:16 + │ │ ├── columns: y:18 z:19 │ │ └── mapping: - │ │ ├── xyzw.y:6 => y:15 - │ │ └── xyzw.z:7 => z:16 + │ │ ├── xyzw.y:7 => y:18 + │ │ └── xyzw.z:8 => z:19 │ └── filters - │ ├── y:15 IS NOT NULL - │ └── z:16 IS NOT NULL + │ ├── y:18 IS NOT NULL + │ └── z:19 IS NOT NULL ├── scan pq - │ └── columns: pq.p:18 pq.q:19 + │ └── columns: pq.p:21 pq.q:22 └── filters - ├── y:15 = pq.p:18 - └── z:16 = pq.q:19 + ├── y:18 = pq.p:21 + └── z:19 = pq.q:22 build UPSERT INTO cpq(c,p) SELECT x,y FROM xyzw ---- upsert cpq ├── columns: - ├── canary column: 12 - ├── fetch columns: c:12 cpq.p:13 cpq.q:14 cpq.other:15 + ├── canary column: 14 + ├── fetch columns: c:14 cpq.p:15 cpq.q:16 cpq.other:17 ├── insert-mapping: - │ ├── x:5 => c:1 - │ ├── xyzw.y:6 => cpq.p:2 - │ ├── column10:10 => cpq.q:3 - │ └── column11:11 => cpq.other:4 + │ ├── x:6 => c:1 + │ ├── xyzw.y:7 => cpq.p:2 + │ ├── column12:12 => cpq.q:3 + │ └── column13:13 => cpq.other:4 ├── update-mapping: - │ └── xyzw.y:6 => cpq.p:2 + │ └── xyzw.y:7 => cpq.p:2 ├── input binding: &1 ├── project - │ ├── columns: upsert_c:16 upsert_q:17 upsert_other:18 x:5 xyzw.y:6 column10:10!null column11:11 c:12 cpq.p:13 cpq.q:14 cpq.other:15 + │ ├── columns: upsert_c:19 upsert_q:20 upsert_other:21 x:6 xyzw.y:7 column12:12!null column13:13 c:14 cpq.p:15 cpq.q:16 cpq.other:17 cpq.crdb_internal_mvcc_timestamp:18 │ ├── left-join (hash) - │ │ ├── columns: x:5 xyzw.y:6 column10:10!null column11:11 c:12 cpq.p:13 cpq.q:14 cpq.other:15 + │ │ ├── columns: x:6 xyzw.y:7 column12:12!null column13:13 c:14 cpq.p:15 cpq.q:16 cpq.other:17 cpq.crdb_internal_mvcc_timestamp:18 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: x:5 xyzw.y:6 column10:10!null column11:11 - │ │ │ ├── grouping columns: x:5 + │ │ │ ├── columns: x:6 xyzw.y:7 column12:12!null column13:13 + │ │ │ ├── grouping columns: x:6 │ │ │ ├── project - │ │ │ │ ├── columns: column10:10!null column11:11 x:5 xyzw.y:6 + │ │ │ │ ├── columns: column12:12!null column13:13 x:6 xyzw.y:7 │ │ │ │ ├── project - │ │ │ │ │ ├── columns: x:5 xyzw.y:6 + │ │ │ │ │ ├── columns: x:6 xyzw.y:7 │ │ │ │ │ └── scan xyzw - │ │ │ │ │ └── columns: x:5 xyzw.y:6 z:7 w:8 rowid:9!null + │ │ │ │ │ └── columns: x:6 xyzw.y:7 z:8 w:9 rowid:10!null xyzw.crdb_internal_mvcc_timestamp:11 │ │ │ │ └── projections - │ │ │ │ ├── 8 [as=column10:10] - │ │ │ │ └── NULL::INT8 [as=column11:11] + │ │ │ │ ├── 8 [as=column12:12] + │ │ │ │ └── NULL::INT8 [as=column13:13] │ │ │ └── aggregations - │ │ │ ├── first-agg [as=xyzw.y:6] - │ │ │ │ └── xyzw.y:6 - │ │ │ ├── first-agg [as=column10:10] - │ │ │ │ └── column10:10 - │ │ │ └── first-agg [as=column11:11] - │ │ │ └── column11:11 + │ │ │ ├── first-agg [as=xyzw.y:7] + │ │ │ │ └── xyzw.y:7 + │ │ │ ├── first-agg [as=column12:12] + │ │ │ │ └── column12:12 + │ │ │ └── first-agg [as=column13:13] + │ │ │ └── column13:13 │ │ ├── scan cpq - │ │ │ └── columns: c:12!null cpq.p:13 cpq.q:14 cpq.other:15 + │ │ │ └── columns: c:14!null cpq.p:15 cpq.q:16 cpq.other:17 cpq.crdb_internal_mvcc_timestamp:18 │ │ └── filters - │ │ └── x:5 = c:12 + │ │ └── x:6 = c:14 │ └── projections - │ ├── CASE WHEN c:12 IS NULL THEN x:5 ELSE c:12 END [as=upsert_c:16] - │ ├── CASE WHEN c:12 IS NULL THEN column10:10 ELSE cpq.q:14 END [as=upsert_q:17] - │ └── CASE WHEN c:12 IS NULL THEN column11:11 ELSE cpq.other:15 END [as=upsert_other:18] + │ ├── CASE WHEN c:14 IS NULL THEN x:6 ELSE c:14 END [as=upsert_c:19] + │ ├── CASE WHEN c:14 IS NULL THEN column12:12 ELSE cpq.q:16 END [as=upsert_q:20] + │ └── CASE WHEN c:14 IS NULL THEN column13:13 ELSE cpq.other:17 END [as=upsert_other:21] └── f-k-checks └── f-k-checks-item: cpq(p,q) -> pq(p,q) └── anti-join (hash) - ├── columns: y:19!null upsert_q:20!null + ├── columns: y:22!null upsert_q:23!null ├── select - │ ├── columns: y:19!null upsert_q:20!null + │ ├── columns: y:22!null upsert_q:23!null │ ├── with-scan &1 - │ │ ├── columns: y:19 upsert_q:20 + │ │ ├── columns: y:22 upsert_q:23 │ │ └── mapping: - │ │ ├── xyzw.y:6 => y:19 - │ │ └── upsert_q:17 => upsert_q:20 + │ │ ├── xyzw.y:7 => y:22 + │ │ └── upsert_q:20 => upsert_q:23 │ └── filters - │ ├── y:19 IS NOT NULL - │ └── upsert_q:20 IS NOT NULL + │ ├── y:22 IS NOT NULL + │ └── upsert_q:23 IS NOT NULL ├── scan pq - │ └── columns: pq.p:22 pq.q:23 + │ └── columns: pq.p:25 pq.q:26 └── filters - ├── y:19 = pq.p:22 - └── upsert_q:20 = pq.q:23 + ├── y:22 = pq.p:25 + └── upsert_q:23 = pq.q:26 build UPSERT INTO cpq(c) SELECT x FROM xyzw ---- upsert cpq ├── columns: - ├── canary column: 13 - ├── fetch columns: c:13 cpq.p:14 cpq.q:15 cpq.other:16 + ├── canary column: 15 + ├── fetch columns: c:15 cpq.p:16 cpq.q:17 cpq.other:18 ├── insert-mapping: - │ ├── x:5 => c:1 - │ ├── column10:10 => cpq.p:2 - │ ├── column11:11 => cpq.q:3 - │ └── column12:12 => cpq.other:4 + │ ├── x:6 => c:1 + │ ├── column12:12 => cpq.p:2 + │ ├── column13:13 => cpq.q:3 + │ └── column14:14 => cpq.other:4 ├── input binding: &1 ├── project - │ ├── columns: upsert_c:17 upsert_p:18 upsert_q:19 upsert_other:20 x:5 column10:10!null column11:11!null column12:12 c:13 cpq.p:14 cpq.q:15 cpq.other:16 + │ ├── columns: upsert_c:20 upsert_p:21 upsert_q:22 upsert_other:23 x:6 column12:12!null column13:13!null column14:14 c:15 cpq.p:16 cpq.q:17 cpq.other:18 cpq.crdb_internal_mvcc_timestamp:19 │ ├── left-join (hash) - │ │ ├── columns: x:5 column10:10!null column11:11!null column12:12 c:13 cpq.p:14 cpq.q:15 cpq.other:16 + │ │ ├── columns: x:6 column12:12!null column13:13!null column14:14 c:15 cpq.p:16 cpq.q:17 cpq.other:18 cpq.crdb_internal_mvcc_timestamp:19 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: x:5 column10:10!null column11:11!null column12:12 - │ │ │ ├── grouping columns: x:5 + │ │ │ ├── columns: x:6 column12:12!null column13:13!null column14:14 + │ │ │ ├── grouping columns: x:6 │ │ │ ├── project - │ │ │ │ ├── columns: column10:10!null column11:11!null column12:12 x:5 + │ │ │ │ ├── columns: column12:12!null column13:13!null column14:14 x:6 │ │ │ │ ├── project - │ │ │ │ │ ├── columns: x:5 + │ │ │ │ │ ├── columns: x:6 │ │ │ │ │ └── scan xyzw - │ │ │ │ │ └── columns: x:5 y:6 z:7 w:8 rowid:9!null + │ │ │ │ │ └── columns: x:6 y:7 z:8 w:9 rowid:10!null xyzw.crdb_internal_mvcc_timestamp:11 │ │ │ │ └── projections - │ │ │ │ ├── 4 [as=column10:10] - │ │ │ │ ├── 8 [as=column11:11] - │ │ │ │ └── NULL::INT8 [as=column12:12] + │ │ │ │ ├── 4 [as=column12:12] + │ │ │ │ ├── 8 [as=column13:13] + │ │ │ │ └── NULL::INT8 [as=column14:14] │ │ │ └── aggregations - │ │ │ ├── first-agg [as=column10:10] - │ │ │ │ └── column10:10 - │ │ │ ├── first-agg [as=column11:11] - │ │ │ │ └── column11:11 - │ │ │ └── first-agg [as=column12:12] - │ │ │ └── column12:12 + │ │ │ ├── first-agg [as=column12:12] + │ │ │ │ └── column12:12 + │ │ │ ├── first-agg [as=column13:13] + │ │ │ │ └── column13:13 + │ │ │ └── first-agg [as=column14:14] + │ │ │ └── column14:14 │ │ ├── scan cpq - │ │ │ └── columns: c:13!null cpq.p:14 cpq.q:15 cpq.other:16 + │ │ │ └── columns: c:15!null cpq.p:16 cpq.q:17 cpq.other:18 cpq.crdb_internal_mvcc_timestamp:19 │ │ └── filters - │ │ └── x:5 = c:13 + │ │ └── x:6 = c:15 │ └── projections - │ ├── CASE WHEN c:13 IS NULL THEN x:5 ELSE c:13 END [as=upsert_c:17] - │ ├── CASE WHEN c:13 IS NULL THEN column10:10 ELSE cpq.p:14 END [as=upsert_p:18] - │ ├── CASE WHEN c:13 IS NULL THEN column11:11 ELSE cpq.q:15 END [as=upsert_q:19] - │ └── CASE WHEN c:13 IS NULL THEN column12:12 ELSE cpq.other:16 END [as=upsert_other:20] + │ ├── CASE WHEN c:15 IS NULL THEN x:6 ELSE c:15 END [as=upsert_c:20] + │ ├── CASE WHEN c:15 IS NULL THEN column12:12 ELSE cpq.p:16 END [as=upsert_p:21] + │ ├── CASE WHEN c:15 IS NULL THEN column13:13 ELSE cpq.q:17 END [as=upsert_q:22] + │ └── CASE WHEN c:15 IS NULL THEN column14:14 ELSE cpq.other:18 END [as=upsert_other:23] └── f-k-checks └── f-k-checks-item: cpq(p,q) -> pq(p,q) └── anti-join (hash) - ├── columns: upsert_p:21!null upsert_q:22!null + ├── columns: upsert_p:24!null upsert_q:25!null ├── select - │ ├── columns: upsert_p:21!null upsert_q:22!null + │ ├── columns: upsert_p:24!null upsert_q:25!null │ ├── with-scan &1 - │ │ ├── columns: upsert_p:21 upsert_q:22 + │ │ ├── columns: upsert_p:24 upsert_q:25 │ │ └── mapping: - │ │ ├── upsert_p:18 => upsert_p:21 - │ │ └── upsert_q:19 => upsert_q:22 + │ │ ├── upsert_p:21 => upsert_p:24 + │ │ └── upsert_q:22 => upsert_q:25 │ └── filters - │ ├── upsert_p:21 IS NOT NULL - │ └── upsert_q:22 IS NOT NULL + │ ├── upsert_p:24 IS NOT NULL + │ └── upsert_q:25 IS NOT NULL ├── scan pq - │ └── columns: pq.p:24 pq.q:25 + │ └── columns: pq.p:27 pq.q:28 └── filters - ├── upsert_p:21 = pq.p:24 - └── upsert_q:22 = pq.q:25 + ├── upsert_p:24 = pq.p:27 + └── upsert_q:25 = pq.q:28 # This has different semantics from the UPSERT INTO cpq(c) version - here we # upsert default values for all unspecified columns. @@ -875,133 +875,133 @@ UPSERT INTO cpq SELECT x FROM xyzw ---- upsert cpq ├── columns: - ├── canary column: 13 - ├── fetch columns: c:13 cpq.p:14 cpq.q:15 cpq.other:16 + ├── canary column: 15 + ├── fetch columns: c:15 cpq.p:16 cpq.q:17 cpq.other:18 ├── insert-mapping: - │ ├── x:5 => c:1 - │ ├── column10:10 => cpq.p:2 - │ ├── column11:11 => cpq.q:3 - │ └── column12:12 => cpq.other:4 + │ ├── x:6 => c:1 + │ ├── column12:12 => cpq.p:2 + │ ├── column13:13 => cpq.q:3 + │ └── column14:14 => cpq.other:4 ├── update-mapping: - │ ├── column10:10 => cpq.p:2 - │ ├── column11:11 => cpq.q:3 - │ └── column12:12 => cpq.other:4 + │ ├── column12:12 => cpq.p:2 + │ ├── column13:13 => cpq.q:3 + │ └── column14:14 => cpq.other:4 ├── input binding: &1 ├── project - │ ├── columns: upsert_c:17 x:5 column10:10!null column11:11!null column12:12 c:13 cpq.p:14 cpq.q:15 cpq.other:16 + │ ├── columns: upsert_c:20 x:6 column12:12!null column13:13!null column14:14 c:15 cpq.p:16 cpq.q:17 cpq.other:18 cpq.crdb_internal_mvcc_timestamp:19 │ ├── left-join (hash) - │ │ ├── columns: x:5 column10:10!null column11:11!null column12:12 c:13 cpq.p:14 cpq.q:15 cpq.other:16 + │ │ ├── columns: x:6 column12:12!null column13:13!null column14:14 c:15 cpq.p:16 cpq.q:17 cpq.other:18 cpq.crdb_internal_mvcc_timestamp:19 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: x:5 column10:10!null column11:11!null column12:12 - │ │ │ ├── grouping columns: x:5 + │ │ │ ├── columns: x:6 column12:12!null column13:13!null column14:14 + │ │ │ ├── grouping columns: x:6 │ │ │ ├── project - │ │ │ │ ├── columns: column10:10!null column11:11!null column12:12 x:5 + │ │ │ │ ├── columns: column12:12!null column13:13!null column14:14 x:6 │ │ │ │ ├── project - │ │ │ │ │ ├── columns: x:5 + │ │ │ │ │ ├── columns: x:6 │ │ │ │ │ └── scan xyzw - │ │ │ │ │ └── columns: x:5 y:6 z:7 w:8 rowid:9!null + │ │ │ │ │ └── columns: x:6 y:7 z:8 w:9 rowid:10!null xyzw.crdb_internal_mvcc_timestamp:11 │ │ │ │ └── projections - │ │ │ │ ├── 4 [as=column10:10] - │ │ │ │ ├── 8 [as=column11:11] - │ │ │ │ └── NULL::INT8 [as=column12:12] + │ │ │ │ ├── 4 [as=column12:12] + │ │ │ │ ├── 8 [as=column13:13] + │ │ │ │ └── NULL::INT8 [as=column14:14] │ │ │ └── aggregations - │ │ │ ├── first-agg [as=column10:10] - │ │ │ │ └── column10:10 - │ │ │ ├── first-agg [as=column11:11] - │ │ │ │ └── column11:11 - │ │ │ └── first-agg [as=column12:12] - │ │ │ └── column12:12 + │ │ │ ├── first-agg [as=column12:12] + │ │ │ │ └── column12:12 + │ │ │ ├── first-agg [as=column13:13] + │ │ │ │ └── column13:13 + │ │ │ └── first-agg [as=column14:14] + │ │ │ └── column14:14 │ │ ├── scan cpq - │ │ │ └── columns: c:13!null cpq.p:14 cpq.q:15 cpq.other:16 + │ │ │ └── columns: c:15!null cpq.p:16 cpq.q:17 cpq.other:18 cpq.crdb_internal_mvcc_timestamp:19 │ │ └── filters - │ │ └── x:5 = c:13 + │ │ └── x:6 = c:15 │ └── projections - │ └── CASE WHEN c:13 IS NULL THEN x:5 ELSE c:13 END [as=upsert_c:17] + │ └── CASE WHEN c:15 IS NULL THEN x:6 ELSE c:15 END [as=upsert_c:20] └── f-k-checks └── f-k-checks-item: cpq(p,q) -> pq(p,q) └── anti-join (hash) - ├── columns: column10:18!null column11:19!null + ├── columns: column12:21!null column13:22!null ├── with-scan &1 - │ ├── columns: column10:18!null column11:19!null + │ ├── columns: column12:21!null column13:22!null │ └── mapping: - │ ├── column10:10 => column10:18 - │ └── column11:11 => column11:19 + │ ├── column12:12 => column12:21 + │ └── column13:13 => column13:22 ├── scan pq - │ └── columns: pq.p:21 pq.q:22 + │ └── columns: pq.p:24 pq.q:25 └── filters - ├── column10:18 = pq.p:21 - └── column11:19 = pq.q:22 + ├── column12:21 = pq.p:24 + └── column13:22 = pq.q:25 build INSERT INTO cpq VALUES (1), (2) ON CONFLICT (c) DO UPDATE SET p = 10 ---- upsert cpq ├── columns: - ├── canary column: 9 - ├── fetch columns: c:9 cpq.p:10 cpq.q:11 cpq.other:12 + ├── canary column: 10 + ├── fetch columns: c:10 cpq.p:11 cpq.q:12 cpq.other:13 ├── insert-mapping: - │ ├── column1:5 => c:1 - │ ├── column6:6 => cpq.p:2 - │ ├── column7:7 => cpq.q:3 - │ └── column8:8 => cpq.other:4 + │ ├── column1:6 => c:1 + │ ├── column7:7 => cpq.p:2 + │ ├── column8:8 => cpq.q:3 + │ └── column9:9 => cpq.other:4 ├── update-mapping: - │ └── upsert_p:15 => cpq.p:2 + │ └── upsert_p:17 => cpq.p:2 ├── input binding: &1 ├── project - │ ├── columns: upsert_c:14 upsert_p:15!null upsert_q:16 upsert_other:17 column1:5!null column6:6!null column7:7!null column8:8 c:9 cpq.p:10 cpq.q:11 cpq.other:12 p_new:13!null + │ ├── columns: upsert_c:16 upsert_p:17!null upsert_q:18 upsert_other:19 column1:6!null column7:7!null column8:8!null column9:9 c:10 cpq.p:11 cpq.q:12 cpq.other:13 cpq.crdb_internal_mvcc_timestamp:14 p_new:15!null │ ├── project - │ │ ├── columns: p_new:13!null column1:5!null column6:6!null column7:7!null column8:8 c:9 cpq.p:10 cpq.q:11 cpq.other:12 + │ │ ├── columns: p_new:15!null column1:6!null column7:7!null column8:8!null column9:9 c:10 cpq.p:11 cpq.q:12 cpq.other:13 cpq.crdb_internal_mvcc_timestamp:14 │ │ ├── left-join (hash) - │ │ │ ├── columns: column1:5!null column6:6!null column7:7!null column8:8 c:9 cpq.p:10 cpq.q:11 cpq.other:12 + │ │ │ ├── columns: column1:6!null column7:7!null column8:8!null column9:9 c:10 cpq.p:11 cpq.q:12 cpq.other:13 cpq.crdb_internal_mvcc_timestamp:14 │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ ├── columns: column1:5!null column6:6!null column7:7!null column8:8 - │ │ │ │ ├── grouping columns: column1:5!null + │ │ │ │ ├── columns: column1:6!null column7:7!null column8:8!null column9:9 + │ │ │ │ ├── grouping columns: column1:6!null │ │ │ │ ├── project - │ │ │ │ │ ├── columns: column6:6!null column7:7!null column8:8 column1:5!null + │ │ │ │ │ ├── columns: column7:7!null column8:8!null column9:9 column1:6!null │ │ │ │ │ ├── values - │ │ │ │ │ │ ├── columns: column1:5!null + │ │ │ │ │ │ ├── columns: column1:6!null │ │ │ │ │ │ ├── (1,) │ │ │ │ │ │ └── (2,) │ │ │ │ │ └── projections - │ │ │ │ │ ├── 4 [as=column6:6] - │ │ │ │ │ ├── 8 [as=column7:7] - │ │ │ │ │ └── NULL::INT8 [as=column8:8] + │ │ │ │ │ ├── 4 [as=column7:7] + │ │ │ │ │ ├── 8 [as=column8:8] + │ │ │ │ │ └── NULL::INT8 [as=column9:9] │ │ │ │ └── aggregations - │ │ │ │ ├── first-agg [as=column6:6] - │ │ │ │ │ └── column6:6 │ │ │ │ ├── first-agg [as=column7:7] │ │ │ │ │ └── column7:7 - │ │ │ │ └── first-agg [as=column8:8] - │ │ │ │ └── column8:8 + │ │ │ │ ├── first-agg [as=column8:8] + │ │ │ │ │ └── column8:8 + │ │ │ │ └── first-agg [as=column9:9] + │ │ │ │ └── column9:9 │ │ │ ├── scan cpq - │ │ │ │ └── columns: c:9!null cpq.p:10 cpq.q:11 cpq.other:12 + │ │ │ │ └── columns: c:10!null cpq.p:11 cpq.q:12 cpq.other:13 cpq.crdb_internal_mvcc_timestamp:14 │ │ │ └── filters - │ │ │ └── column1:5 = c:9 + │ │ │ └── column1:6 = c:10 │ │ └── projections - │ │ └── 10 [as=p_new:13] + │ │ └── 10 [as=p_new:15] │ └── projections - │ ├── CASE WHEN c:9 IS NULL THEN column1:5 ELSE c:9 END [as=upsert_c:14] - │ ├── CASE WHEN c:9 IS NULL THEN column6:6 ELSE p_new:13 END [as=upsert_p:15] - │ ├── CASE WHEN c:9 IS NULL THEN column7:7 ELSE cpq.q:11 END [as=upsert_q:16] - │ └── CASE WHEN c:9 IS NULL THEN column8:8 ELSE cpq.other:12 END [as=upsert_other:17] + │ ├── CASE WHEN c:10 IS NULL THEN column1:6 ELSE c:10 END [as=upsert_c:16] + │ ├── CASE WHEN c:10 IS NULL THEN column7:7 ELSE p_new:15 END [as=upsert_p:17] + │ ├── CASE WHEN c:10 IS NULL THEN column8:8 ELSE cpq.q:12 END [as=upsert_q:18] + │ └── CASE WHEN c:10 IS NULL THEN column9:9 ELSE cpq.other:13 END [as=upsert_other:19] └── f-k-checks └── f-k-checks-item: cpq(p,q) -> pq(p,q) └── anti-join (hash) - ├── columns: upsert_p:18!null upsert_q:19!null + ├── columns: upsert_p:20!null upsert_q:21!null ├── select - │ ├── columns: upsert_p:18!null upsert_q:19!null + │ ├── columns: upsert_p:20!null upsert_q:21!null │ ├── with-scan &1 - │ │ ├── columns: upsert_p:18!null upsert_q:19 + │ │ ├── columns: upsert_p:20!null upsert_q:21 │ │ └── mapping: - │ │ ├── upsert_p:15 => upsert_p:18 - │ │ └── upsert_q:16 => upsert_q:19 + │ │ ├── upsert_p:17 => upsert_p:20 + │ │ └── upsert_q:18 => upsert_q:21 │ └── filters - │ └── upsert_q:19 IS NOT NULL + │ └── upsert_q:21 IS NOT NULL ├── scan pq - │ └── columns: pq.p:21 pq.q:22 + │ └── columns: pq.p:23 pq.q:24 └── filters - ├── upsert_p:18 = pq.p:21 - └── upsert_q:19 = pq.q:22 + ├── upsert_p:20 = pq.p:23 + └── upsert_q:21 = pq.q:24 # ------------------------------------------ # Multiple outbound FKs @@ -1024,136 +1024,136 @@ UPSERT INTO cmulti SELECT x,y,z,w FROM xyzw ---- upsert cmulti ├── columns: - ├── canary column: 10 - ├── fetch columns: a:10 b:11 c:12 d:13 + ├── canary column: 12 + ├── fetch columns: a:12 b:13 c:14 d:15 ├── insert-mapping: - │ ├── x:5 => a:1 - │ ├── y:6 => b:2 - │ ├── xyzw.z:7 => c:3 - │ └── w:8 => d:4 + │ ├── x:6 => a:1 + │ ├── y:7 => b:2 + │ ├── xyzw.z:8 => c:3 + │ └── w:9 => d:4 ├── update-mapping: - │ ├── xyzw.z:7 => c:3 - │ └── w:8 => d:4 + │ ├── xyzw.z:8 => c:3 + │ └── w:9 => d:4 ├── input binding: &1 ├── project - │ ├── columns: upsert_a:14 upsert_b:15 x:5 y:6 xyzw.z:7 w:8 a:10 b:11 c:12 d:13 + │ ├── columns: upsert_a:17 upsert_b:18 x:6 y:7 xyzw.z:8 w:9 a:12 b:13 c:14 d:15 cmulti.crdb_internal_mvcc_timestamp:16 │ ├── left-join (hash) - │ │ ├── columns: x:5 y:6 xyzw.z:7 w:8 a:10 b:11 c:12 d:13 + │ │ ├── columns: x:6 y:7 xyzw.z:8 w:9 a:12 b:13 c:14 d:15 cmulti.crdb_internal_mvcc_timestamp:16 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: x:5 y:6 xyzw.z:7 w:8 - │ │ │ ├── grouping columns: x:5 y:6 + │ │ │ ├── columns: x:6 y:7 xyzw.z:8 w:9 + │ │ │ ├── grouping columns: x:6 y:7 │ │ │ ├── project - │ │ │ │ ├── columns: x:5 y:6 xyzw.z:7 w:8 + │ │ │ │ ├── columns: x:6 y:7 xyzw.z:8 w:9 │ │ │ │ └── scan xyzw - │ │ │ │ └── columns: x:5 y:6 xyzw.z:7 w:8 rowid:9!null + │ │ │ │ └── columns: x:6 y:7 xyzw.z:8 w:9 rowid:10!null xyzw.crdb_internal_mvcc_timestamp:11 │ │ │ └── aggregations - │ │ │ ├── first-agg [as=xyzw.z:7] - │ │ │ │ └── xyzw.z:7 - │ │ │ └── first-agg [as=w:8] - │ │ │ └── w:8 + │ │ │ ├── first-agg [as=xyzw.z:8] + │ │ │ │ └── xyzw.z:8 + │ │ │ └── first-agg [as=w:9] + │ │ │ └── w:9 │ │ ├── scan cmulti - │ │ │ └── columns: a:10!null b:11!null c:12 d:13 + │ │ │ └── columns: a:12!null b:13!null c:14 d:15 cmulti.crdb_internal_mvcc_timestamp:16 │ │ └── filters - │ │ ├── x:5 = a:10 - │ │ └── y:6 = b:11 + │ │ ├── x:6 = a:12 + │ │ └── y:7 = b:13 │ └── projections - │ ├── CASE WHEN a:10 IS NULL THEN x:5 ELSE a:10 END [as=upsert_a:14] - │ └── CASE WHEN a:10 IS NULL THEN y:6 ELSE b:11 END [as=upsert_b:15] + │ ├── CASE WHEN a:12 IS NULL THEN x:6 ELSE a:12 END [as=upsert_a:17] + │ └── CASE WHEN a:12 IS NULL THEN y:7 ELSE b:13 END [as=upsert_b:18] └── f-k-checks ├── f-k-checks-item: cmulti(a) -> p(p) │ └── anti-join (hash) - │ ├── columns: upsert_a:16 + │ ├── columns: upsert_a:19 │ ├── with-scan &1 - │ │ ├── columns: upsert_a:16 + │ │ ├── columns: upsert_a:19 │ │ └── mapping: - │ │ └── upsert_a:14 => upsert_a:16 + │ │ └── upsert_a:17 => upsert_a:19 │ ├── scan p - │ │ └── columns: p.p:17!null + │ │ └── columns: p.p:20!null │ └── filters - │ └── upsert_a:16 = p.p:17 + │ └── upsert_a:19 = p.p:20 └── f-k-checks-item: cmulti(b,c) -> pq(p,q) └── anti-join (hash) - ├── columns: upsert_b:19 z:20 + ├── columns: upsert_b:23 z:24 ├── with-scan &1 - │ ├── columns: upsert_b:19 z:20 + │ ├── columns: upsert_b:23 z:24 │ └── mapping: - │ ├── upsert_b:15 => upsert_b:19 - │ └── xyzw.z:7 => z:20 + │ ├── upsert_b:18 => upsert_b:23 + │ └── xyzw.z:8 => z:24 ├── scan pq - │ └── columns: pq.p:22 q:23 + │ └── columns: pq.p:26 q:27 └── filters - ├── upsert_b:19 = pq.p:22 - └── z:20 = q:23 + ├── upsert_b:23 = pq.p:26 + └── z:24 = q:27 build UPSERT INTO cmulti(a,b,c) SELECT x,y,z FROM xyzw ---- upsert cmulti ├── columns: - ├── canary column: 11 - ├── fetch columns: a:11 b:12 c:13 d:14 + ├── canary column: 13 + ├── fetch columns: a:13 b:14 c:15 d:16 ├── insert-mapping: - │ ├── x:5 => a:1 - │ ├── y:6 => b:2 - │ ├── xyzw.z:7 => c:3 - │ └── column10:10 => d:4 + │ ├── x:6 => a:1 + │ ├── y:7 => b:2 + │ ├── xyzw.z:8 => c:3 + │ └── column12:12 => d:4 ├── update-mapping: - │ └── xyzw.z:7 => c:3 + │ └── xyzw.z:8 => c:3 ├── input binding: &1 ├── project - │ ├── columns: upsert_a:15 upsert_b:16 upsert_d:17 x:5 y:6 xyzw.z:7 column10:10!null a:11 b:12 c:13 d:14 + │ ├── columns: upsert_a:18 upsert_b:19 upsert_d:20 x:6 y:7 xyzw.z:8 column12:12!null a:13 b:14 c:15 d:16 cmulti.crdb_internal_mvcc_timestamp:17 │ ├── left-join (hash) - │ │ ├── columns: x:5 y:6 xyzw.z:7 column10:10!null a:11 b:12 c:13 d:14 + │ │ ├── columns: x:6 y:7 xyzw.z:8 column12:12!null a:13 b:14 c:15 d:16 cmulti.crdb_internal_mvcc_timestamp:17 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: x:5 y:6 xyzw.z:7 column10:10!null - │ │ │ ├── grouping columns: x:5 y:6 + │ │ │ ├── columns: x:6 y:7 xyzw.z:8 column12:12!null + │ │ │ ├── grouping columns: x:6 y:7 │ │ │ ├── project - │ │ │ │ ├── columns: column10:10!null x:5 y:6 xyzw.z:7 + │ │ │ │ ├── columns: column12:12!null x:6 y:7 xyzw.z:8 │ │ │ │ ├── project - │ │ │ │ │ ├── columns: x:5 y:6 xyzw.z:7 + │ │ │ │ │ ├── columns: x:6 y:7 xyzw.z:8 │ │ │ │ │ └── scan xyzw - │ │ │ │ │ └── columns: x:5 y:6 xyzw.z:7 w:8 rowid:9!null + │ │ │ │ │ └── columns: x:6 y:7 xyzw.z:8 w:9 rowid:10!null xyzw.crdb_internal_mvcc_timestamp:11 │ │ │ │ └── projections - │ │ │ │ └── 8 [as=column10:10] + │ │ │ │ └── 8 [as=column12:12] │ │ │ └── aggregations - │ │ │ ├── first-agg [as=xyzw.z:7] - │ │ │ │ └── xyzw.z:7 - │ │ │ └── first-agg [as=column10:10] - │ │ │ └── column10:10 + │ │ │ ├── first-agg [as=xyzw.z:8] + │ │ │ │ └── xyzw.z:8 + │ │ │ └── first-agg [as=column12:12] + │ │ │ └── column12:12 │ │ ├── scan cmulti - │ │ │ └── columns: a:11!null b:12!null c:13 d:14 + │ │ │ └── columns: a:13!null b:14!null c:15 d:16 cmulti.crdb_internal_mvcc_timestamp:17 │ │ └── filters - │ │ ├── x:5 = a:11 - │ │ └── y:6 = b:12 + │ │ ├── x:6 = a:13 + │ │ └── y:7 = b:14 │ └── projections - │ ├── CASE WHEN a:11 IS NULL THEN x:5 ELSE a:11 END [as=upsert_a:15] - │ ├── CASE WHEN a:11 IS NULL THEN y:6 ELSE b:12 END [as=upsert_b:16] - │ └── CASE WHEN a:11 IS NULL THEN column10:10 ELSE d:14 END [as=upsert_d:17] + │ ├── CASE WHEN a:13 IS NULL THEN x:6 ELSE a:13 END [as=upsert_a:18] + │ ├── CASE WHEN a:13 IS NULL THEN y:7 ELSE b:14 END [as=upsert_b:19] + │ └── CASE WHEN a:13 IS NULL THEN column12:12 ELSE d:16 END [as=upsert_d:20] └── f-k-checks ├── f-k-checks-item: cmulti(a) -> p(p) │ └── anti-join (hash) - │ ├── columns: upsert_a:18 + │ ├── columns: upsert_a:21 │ ├── with-scan &1 - │ │ ├── columns: upsert_a:18 + │ │ ├── columns: upsert_a:21 │ │ └── mapping: - │ │ └── upsert_a:15 => upsert_a:18 + │ │ └── upsert_a:18 => upsert_a:21 │ ├── scan p - │ │ └── columns: p.p:19!null + │ │ └── columns: p.p:22!null │ └── filters - │ └── upsert_a:18 = p.p:19 + │ └── upsert_a:21 = p.p:22 └── f-k-checks-item: cmulti(b,c) -> pq(p,q) └── anti-join (hash) - ├── columns: upsert_b:21 z:22 + ├── columns: upsert_b:25 z:26 ├── with-scan &1 - │ ├── columns: upsert_b:21 z:22 + │ ├── columns: upsert_b:25 z:26 │ └── mapping: - │ ├── upsert_b:16 => upsert_b:21 - │ └── xyzw.z:7 => z:22 + │ ├── upsert_b:19 => upsert_b:25 + │ └── xyzw.z:8 => z:26 ├── scan pq - │ └── columns: pq.p:24 q:25 + │ └── columns: pq.p:28 q:29 └── filters - ├── upsert_b:21 = pq.p:24 - └── z:22 = q:25 + ├── upsert_b:25 = pq.p:28 + └── z:26 = q:29 # --------------------------------------- # Inbound FK tests with single FK column @@ -1166,10 +1166,10 @@ UPSERT INTO p VALUES (1, 1), (2, 2) upsert p ├── columns: ├── upsert-mapping: - │ ├── column1:3 => p:1 - │ └── column2:4 => other:2 + │ ├── column1:4 => p:1 + │ └── column2:5 => other:2 └── values - ├── columns: column1:3!null column2:4!null + ├── columns: column1:4!null column2:5!null ├── (1, 1) └── (2, 2) @@ -1187,33 +1187,33 @@ UPSERT INTO p1 VALUES (1, 1), (2, 2) ---- upsert p1 ├── columns: - ├── canary column: 5 - ├── fetch columns: p:5 other:6 + ├── canary column: 6 + ├── fetch columns: p:6 other:7 ├── insert-mapping: - │ ├── column1:3 => p:1 - │ └── column2:4 => other:2 + │ ├── column1:4 => p:1 + │ └── column2:5 => other:2 ├── update-mapping: - │ └── column2:4 => other:2 + │ └── column2:5 => other:2 └── project - ├── columns: upsert_p:7 column1:3!null column2:4!null p:5 other:6 + ├── columns: upsert_p:9 column1:4!null column2:5!null p:6 other:7 crdb_internal_mvcc_timestamp:8 ├── left-join (hash) - │ ├── columns: column1:3!null column2:4!null p:5 other:6 + │ ├── columns: column1:4!null column2:5!null p:6 other:7 crdb_internal_mvcc_timestamp:8 │ ├── ensure-upsert-distinct-on - │ │ ├── columns: column1:3!null column2:4!null - │ │ ├── grouping columns: column1:3!null + │ │ ├── columns: column1:4!null column2:5!null + │ │ ├── grouping columns: column1:4!null │ │ ├── values - │ │ │ ├── columns: column1:3!null column2:4!null + │ │ │ ├── columns: column1:4!null column2:5!null │ │ │ ├── (1, 1) │ │ │ └── (2, 2) │ │ └── aggregations - │ │ └── first-agg [as=column2:4] - │ │ └── column2:4 + │ │ └── first-agg [as=column2:5] + │ │ └── column2:5 │ ├── scan p1 - │ │ └── columns: p:5!null other:6 + │ │ └── columns: p:6!null other:7 crdb_internal_mvcc_timestamp:8 │ └── filters - │ └── column1:3 = p:5 + │ └── column1:4 = p:6 └── projections - └── CASE WHEN p:5 IS NULL THEN column1:3 ELSE p:5 END [as=upsert_p:7] + └── CASE WHEN p:6 IS NULL THEN column1:4 ELSE p:6 END [as=upsert_p:9] # This statement can modify existing values of p so we need to perform the FK # check. @@ -1222,59 +1222,59 @@ INSERT INTO p1 VALUES (100, 1), (200, 1) ON CONFLICT (p) DO UPDATE SET p = exclu ---- upsert p1 ├── columns: - ├── canary column: 5 - ├── fetch columns: p1.p:5 other:6 + ├── canary column: 6 + ├── fetch columns: p1.p:6 other:7 ├── insert-mapping: - │ ├── column1:3 => p1.p:1 - │ └── column2:4 => other:2 + │ ├── column1:4 => p1.p:1 + │ └── column2:5 => other:2 ├── update-mapping: - │ └── upsert_p:8 => p1.p:1 + │ └── upsert_p:10 => p1.p:1 ├── input binding: &1 ├── project - │ ├── columns: upsert_p:8!null upsert_other:9 column1:3!null column2:4!null p1.p:5 other:6 p_new:7!null + │ ├── columns: upsert_p:10!null upsert_other:11 column1:4!null column2:5!null p1.p:6 other:7 p1.crdb_internal_mvcc_timestamp:8 p_new:9!null │ ├── project - │ │ ├── columns: p_new:7!null column1:3!null column2:4!null p1.p:5 other:6 + │ │ ├── columns: p_new:9!null column1:4!null column2:5!null p1.p:6 other:7 p1.crdb_internal_mvcc_timestamp:8 │ │ ├── left-join (hash) - │ │ │ ├── columns: column1:3!null column2:4!null p1.p:5 other:6 + │ │ │ ├── columns: column1:4!null column2:5!null p1.p:6 other:7 p1.crdb_internal_mvcc_timestamp:8 │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ ├── columns: column1:3!null column2:4!null - │ │ │ │ ├── grouping columns: column1:3!null + │ │ │ │ ├── columns: column1:4!null column2:5!null + │ │ │ │ ├── grouping columns: column1:4!null │ │ │ │ ├── values - │ │ │ │ │ ├── columns: column1:3!null column2:4!null + │ │ │ │ │ ├── columns: column1:4!null column2:5!null │ │ │ │ │ ├── (100, 1) │ │ │ │ │ └── (200, 1) │ │ │ │ └── aggregations - │ │ │ │ └── first-agg [as=column2:4] - │ │ │ │ └── column2:4 + │ │ │ │ └── first-agg [as=column2:5] + │ │ │ │ └── column2:5 │ │ │ ├── scan p1 - │ │ │ │ └── columns: p1.p:5!null other:6 + │ │ │ │ └── columns: p1.p:6!null other:7 p1.crdb_internal_mvcc_timestamp:8 │ │ │ └── filters - │ │ │ └── column1:3 = p1.p:5 + │ │ │ └── column1:4 = p1.p:6 │ │ └── projections - │ │ └── column1:3 + 1 [as=p_new:7] + │ │ └── column1:4 + 1 [as=p_new:9] │ └── projections - │ ├── CASE WHEN p1.p:5 IS NULL THEN column1:3 ELSE p_new:7 END [as=upsert_p:8] - │ └── CASE WHEN p1.p:5 IS NULL THEN column2:4 ELSE other:6 END [as=upsert_other:9] + │ ├── CASE WHEN p1.p:6 IS NULL THEN column1:4 ELSE p_new:9 END [as=upsert_p:10] + │ └── CASE WHEN p1.p:6 IS NULL THEN column2:5 ELSE other:7 END [as=upsert_other:11] └── f-k-checks └── f-k-checks-item: p1c(p) -> p1(p) └── semi-join (hash) - ├── columns: p:10 + ├── columns: p:12 ├── except - │ ├── columns: p:10 - │ ├── left columns: p:10 - │ ├── right columns: upsert_p:11 + │ ├── columns: p:12 + │ ├── left columns: p:12 + │ ├── right columns: upsert_p:13 │ ├── with-scan &1 - │ │ ├── columns: p:10 + │ │ ├── columns: p:12 │ │ └── mapping: - │ │ └── p1.p:5 => p:10 + │ │ └── p1.p:6 => p:12 │ └── with-scan &1 - │ ├── columns: upsert_p:11!null + │ ├── columns: upsert_p:13!null │ └── mapping: - │ └── upsert_p:8 => upsert_p:11 + │ └── upsert_p:10 => upsert_p:13 ├── scan p1c - │ └── columns: p1c.p:13!null + │ └── columns: p1c.p:15!null └── filters - └── p:10 = p1c.p:13 + └── p:12 = p1c.p:15 # No need to check the inbound FK: we never modify existing values of p. build @@ -1282,38 +1282,38 @@ INSERT INTO p1 VALUES (100, 1), (200, 1) ON CONFLICT (p) DO UPDATE SET other = p ---- upsert p1 ├── columns: - ├── canary column: 5 - ├── fetch columns: p:5 other:6 + ├── canary column: 6 + ├── fetch columns: p:6 other:7 ├── insert-mapping: - │ ├── column1:3 => p:1 - │ └── column2:4 => other:2 + │ ├── column1:4 => p:1 + │ └── column2:5 => other:2 ├── update-mapping: - │ └── upsert_other:9 => other:2 + │ └── upsert_other:11 => other:2 └── project - ├── columns: upsert_p:8 upsert_other:9 column1:3!null column2:4!null p:5 other:6 other_new:7 + ├── columns: upsert_p:10 upsert_other:11 column1:4!null column2:5!null p:6 other:7 crdb_internal_mvcc_timestamp:8 other_new:9 ├── project - │ ├── columns: other_new:7 column1:3!null column2:4!null p:5 other:6 + │ ├── columns: other_new:9 column1:4!null column2:5!null p:6 other:7 crdb_internal_mvcc_timestamp:8 │ ├── left-join (hash) - │ │ ├── columns: column1:3!null column2:4!null p:5 other:6 + │ │ ├── columns: column1:4!null column2:5!null p:6 other:7 crdb_internal_mvcc_timestamp:8 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: column1:3!null column2:4!null - │ │ │ ├── grouping columns: column1:3!null + │ │ │ ├── columns: column1:4!null column2:5!null + │ │ │ ├── grouping columns: column1:4!null │ │ │ ├── values - │ │ │ │ ├── columns: column1:3!null column2:4!null + │ │ │ │ ├── columns: column1:4!null column2:5!null │ │ │ │ ├── (100, 1) │ │ │ │ └── (200, 1) │ │ │ └── aggregations - │ │ │ └── first-agg [as=column2:4] - │ │ │ └── column2:4 + │ │ │ └── first-agg [as=column2:5] + │ │ │ └── column2:5 │ │ ├── scan p1 - │ │ │ └── columns: p:5!null other:6 + │ │ │ └── columns: p:6!null other:7 crdb_internal_mvcc_timestamp:8 │ │ └── filters - │ │ └── column1:3 = p:5 + │ │ └── column1:4 = p:6 │ └── projections - │ └── other:6 + 1 [as=other_new:7] + │ └── other:7 + 1 [as=other_new:9] └── projections - ├── CASE WHEN p:5 IS NULL THEN column1:3 ELSE p:5 END [as=upsert_p:8] - └── CASE WHEN p:5 IS NULL THEN column2:4 ELSE other_new:7 END [as=upsert_other:9] + ├── CASE WHEN p:6 IS NULL THEN column1:4 ELSE p:6 END [as=upsert_p:10] + └── CASE WHEN p:6 IS NULL THEN column2:5 ELSE other_new:9 END [as=upsert_other:11] # Similar tests when the FK column is not the PK. exec-ddl @@ -1329,54 +1329,54 @@ UPSERT INTO p2 VALUES (1, 1), (2, 2) ---- upsert p2 ├── columns: - ├── canary column: 5 - ├── fetch columns: p:5 p2.fk:6 + ├── canary column: 6 + ├── fetch columns: p:6 p2.fk:7 ├── insert-mapping: - │ ├── column1:3 => p:1 - │ └── column2:4 => p2.fk:2 + │ ├── column1:4 => p:1 + │ └── column2:5 => p2.fk:2 ├── update-mapping: - │ └── column2:4 => p2.fk:2 + │ └── column2:5 => p2.fk:2 ├── input binding: &1 ├── project - │ ├── columns: upsert_p:7 column1:3!null column2:4!null p:5 p2.fk:6 + │ ├── columns: upsert_p:9 column1:4!null column2:5!null p:6 p2.fk:7 p2.crdb_internal_mvcc_timestamp:8 │ ├── left-join (hash) - │ │ ├── columns: column1:3!null column2:4!null p:5 p2.fk:6 + │ │ ├── columns: column1:4!null column2:5!null p:6 p2.fk:7 p2.crdb_internal_mvcc_timestamp:8 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: column1:3!null column2:4!null - │ │ │ ├── grouping columns: column1:3!null + │ │ │ ├── columns: column1:4!null column2:5!null + │ │ │ ├── grouping columns: column1:4!null │ │ │ ├── values - │ │ │ │ ├── columns: column1:3!null column2:4!null + │ │ │ │ ├── columns: column1:4!null column2:5!null │ │ │ │ ├── (1, 1) │ │ │ │ └── (2, 2) │ │ │ └── aggregations - │ │ │ └── first-agg [as=column2:4] - │ │ │ └── column2:4 + │ │ │ └── first-agg [as=column2:5] + │ │ │ └── column2:5 │ │ ├── scan p2 - │ │ │ └── columns: p:5!null p2.fk:6 + │ │ │ └── columns: p:6!null p2.fk:7 p2.crdb_internal_mvcc_timestamp:8 │ │ └── filters - │ │ └── column1:3 = p:5 + │ │ └── column1:4 = p:6 │ └── projections - │ └── CASE WHEN p:5 IS NULL THEN column1:3 ELSE p:5 END [as=upsert_p:7] + │ └── CASE WHEN p:6 IS NULL THEN column1:4 ELSE p:6 END [as=upsert_p:9] └── f-k-checks └── f-k-checks-item: p2c(fk) -> p2(fk) └── semi-join (hash) - ├── columns: fk:8 + ├── columns: fk:10 ├── except - │ ├── columns: fk:8 - │ ├── left columns: fk:8 - │ ├── right columns: column2:9 + │ ├── columns: fk:10 + │ ├── left columns: fk:10 + │ ├── right columns: column2:11 │ ├── with-scan &1 - │ │ ├── columns: fk:8 + │ │ ├── columns: fk:10 │ │ └── mapping: - │ │ └── p2.fk:6 => fk:8 + │ │ └── p2.fk:7 => fk:10 │ └── with-scan &1 - │ ├── columns: column2:9!null + │ ├── columns: column2:11!null │ └── mapping: - │ └── column2:4 => column2:9 + │ └── column2:5 => column2:11 ├── scan p2c - │ └── columns: p2c.fk:11 + │ └── columns: p2c.fk:13 └── filters - └── fk:8 = p2c.fk:11 + └── fk:10 = p2c.fk:13 # This statement never removes existing values of the fk column; FK check is # not needed. @@ -1385,38 +1385,38 @@ INSERT INTO p2 VALUES (1, 1), (2, 2) ON CONFLICT (p) DO UPDATE SET p = excluded. ---- upsert p2 ├── columns: - ├── canary column: 5 - ├── fetch columns: p:5 fk:6 + ├── canary column: 6 + ├── fetch columns: p:6 fk:7 ├── insert-mapping: - │ ├── column1:3 => p:1 - │ └── column2:4 => fk:2 + │ ├── column1:4 => p:1 + │ └── column2:5 => fk:2 ├── update-mapping: - │ └── upsert_p:8 => p:1 + │ └── upsert_p:10 => p:1 └── project - ├── columns: upsert_p:8!null upsert_fk:9 column1:3!null column2:4!null p:5 fk:6 p_new:7!null + ├── columns: upsert_p:10!null upsert_fk:11 column1:4!null column2:5!null p:6 fk:7 crdb_internal_mvcc_timestamp:8 p_new:9!null ├── project - │ ├── columns: p_new:7!null column1:3!null column2:4!null p:5 fk:6 + │ ├── columns: p_new:9!null column1:4!null column2:5!null p:6 fk:7 crdb_internal_mvcc_timestamp:8 │ ├── left-join (hash) - │ │ ├── columns: column1:3!null column2:4!null p:5 fk:6 + │ │ ├── columns: column1:4!null column2:5!null p:6 fk:7 crdb_internal_mvcc_timestamp:8 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: column1:3!null column2:4!null - │ │ │ ├── grouping columns: column1:3!null + │ │ │ ├── columns: column1:4!null column2:5!null + │ │ │ ├── grouping columns: column1:4!null │ │ │ ├── values - │ │ │ │ ├── columns: column1:3!null column2:4!null + │ │ │ │ ├── columns: column1:4!null column2:5!null │ │ │ │ ├── (1, 1) │ │ │ │ └── (2, 2) │ │ │ └── aggregations - │ │ │ └── first-agg [as=column2:4] - │ │ │ └── column2:4 + │ │ │ └── first-agg [as=column2:5] + │ │ │ └── column2:5 │ │ ├── scan p2 - │ │ │ └── columns: p:5!null fk:6 + │ │ │ └── columns: p:6!null fk:7 crdb_internal_mvcc_timestamp:8 │ │ └── filters - │ │ └── column1:3 = p:5 + │ │ └── column1:4 = p:6 │ └── projections - │ └── column1:3 + 1 [as=p_new:7] + │ └── column1:4 + 1 [as=p_new:9] └── projections - ├── CASE WHEN p:5 IS NULL THEN column1:3 ELSE p_new:7 END [as=upsert_p:8] - └── CASE WHEN p:5 IS NULL THEN column2:4 ELSE fk:6 END [as=upsert_fk:9] + ├── CASE WHEN p:6 IS NULL THEN column1:4 ELSE p_new:9 END [as=upsert_p:10] + └── CASE WHEN p:6 IS NULL THEN column2:5 ELSE fk:7 END [as=upsert_fk:11] # This statement can change existing values of the fk column, so the FK check # is needed. @@ -1425,59 +1425,59 @@ INSERT INTO p2 VALUES (1, 1), (2, 2) ON CONFLICT (p) DO UPDATE SET fk = excluded ---- upsert p2 ├── columns: - ├── canary column: 5 - ├── fetch columns: p:5 p2.fk:6 + ├── canary column: 6 + ├── fetch columns: p:6 p2.fk:7 ├── insert-mapping: - │ ├── column1:3 => p:1 - │ └── column2:4 => p2.fk:2 + │ ├── column1:4 => p:1 + │ └── column2:5 => p2.fk:2 ├── update-mapping: - │ └── upsert_fk:9 => p2.fk:2 + │ └── upsert_fk:11 => p2.fk:2 ├── input binding: &1 ├── project - │ ├── columns: upsert_p:8 upsert_fk:9!null column1:3!null column2:4!null p:5 p2.fk:6 fk_new:7!null + │ ├── columns: upsert_p:10 upsert_fk:11!null column1:4!null column2:5!null p:6 p2.fk:7 p2.crdb_internal_mvcc_timestamp:8 fk_new:9!null │ ├── project - │ │ ├── columns: fk_new:7!null column1:3!null column2:4!null p:5 p2.fk:6 + │ │ ├── columns: fk_new:9!null column1:4!null column2:5!null p:6 p2.fk:7 p2.crdb_internal_mvcc_timestamp:8 │ │ ├── left-join (hash) - │ │ │ ├── columns: column1:3!null column2:4!null p:5 p2.fk:6 + │ │ │ ├── columns: column1:4!null column2:5!null p:6 p2.fk:7 p2.crdb_internal_mvcc_timestamp:8 │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ ├── columns: column1:3!null column2:4!null - │ │ │ │ ├── grouping columns: column1:3!null + │ │ │ │ ├── columns: column1:4!null column2:5!null + │ │ │ │ ├── grouping columns: column1:4!null │ │ │ │ ├── values - │ │ │ │ │ ├── columns: column1:3!null column2:4!null + │ │ │ │ │ ├── columns: column1:4!null column2:5!null │ │ │ │ │ ├── (1, 1) │ │ │ │ │ └── (2, 2) │ │ │ │ └── aggregations - │ │ │ │ └── first-agg [as=column2:4] - │ │ │ │ └── column2:4 + │ │ │ │ └── first-agg [as=column2:5] + │ │ │ │ └── column2:5 │ │ │ ├── scan p2 - │ │ │ │ └── columns: p:5!null p2.fk:6 + │ │ │ │ └── columns: p:6!null p2.fk:7 p2.crdb_internal_mvcc_timestamp:8 │ │ │ └── filters - │ │ │ └── column1:3 = p:5 + │ │ │ └── column1:4 = p:6 │ │ └── projections - │ │ └── column2:4 + 1 [as=fk_new:7] + │ │ └── column2:5 + 1 [as=fk_new:9] │ └── projections - │ ├── CASE WHEN p:5 IS NULL THEN column1:3 ELSE p:5 END [as=upsert_p:8] - │ └── CASE WHEN p:5 IS NULL THEN column2:4 ELSE fk_new:7 END [as=upsert_fk:9] + │ ├── CASE WHEN p:6 IS NULL THEN column1:4 ELSE p:6 END [as=upsert_p:10] + │ └── CASE WHEN p:6 IS NULL THEN column2:5 ELSE fk_new:9 END [as=upsert_fk:11] └── f-k-checks └── f-k-checks-item: p2c(fk) -> p2(fk) └── semi-join (hash) - ├── columns: fk:10 + ├── columns: fk:12 ├── except - │ ├── columns: fk:10 - │ ├── left columns: fk:10 - │ ├── right columns: upsert_fk:11 + │ ├── columns: fk:12 + │ ├── left columns: fk:12 + │ ├── right columns: upsert_fk:13 │ ├── with-scan &1 - │ │ ├── columns: fk:10 + │ │ ├── columns: fk:12 │ │ └── mapping: - │ │ └── p2.fk:6 => fk:10 + │ │ └── p2.fk:7 => fk:12 │ └── with-scan &1 - │ ├── columns: upsert_fk:11!null + │ ├── columns: upsert_fk:13!null │ └── mapping: - │ └── upsert_fk:9 => upsert_fk:11 + │ └── upsert_fk:11 => upsert_fk:13 ├── scan p2c - │ └── columns: p2c.fk:13 + │ └── columns: p2c.fk:15 └── filters - └── fk:10 = p2c.fk:13 + └── fk:12 = p2c.fk:15 # This statement never removes existing values of the fk column; the FK check is # not needed. @@ -1486,97 +1486,97 @@ INSERT INTO p2 VALUES (1, 1), (2, 2) ON CONFLICT (fk) DO UPDATE SET p = excluded ---- upsert p2 ├── columns: - ├── canary column: 5 - ├── fetch columns: p:5 fk:6 + ├── canary column: 6 + ├── fetch columns: p:6 fk:7 ├── insert-mapping: - │ ├── column1:3 => p:1 - │ └── column2:4 => fk:2 + │ ├── column1:4 => p:1 + │ └── column2:5 => fk:2 ├── update-mapping: - │ └── upsert_p:8 => p:1 + │ └── upsert_p:10 => p:1 └── project - ├── columns: upsert_p:8!null upsert_fk:9 column1:3!null column2:4!null p:5 fk:6 p_new:7!null + ├── columns: upsert_p:10!null upsert_fk:11 column1:4!null column2:5!null p:6 fk:7 crdb_internal_mvcc_timestamp:8 p_new:9!null ├── project - │ ├── columns: p_new:7!null column1:3!null column2:4!null p:5 fk:6 + │ ├── columns: p_new:9!null column1:4!null column2:5!null p:6 fk:7 crdb_internal_mvcc_timestamp:8 │ ├── left-join (hash) - │ │ ├── columns: column1:3!null column2:4!null p:5 fk:6 + │ │ ├── columns: column1:4!null column2:5!null p:6 fk:7 crdb_internal_mvcc_timestamp:8 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: column1:3!null column2:4!null - │ │ │ ├── grouping columns: column2:4!null + │ │ │ ├── columns: column1:4!null column2:5!null + │ │ │ ├── grouping columns: column2:5!null │ │ │ ├── values - │ │ │ │ ├── columns: column1:3!null column2:4!null + │ │ │ │ ├── columns: column1:4!null column2:5!null │ │ │ │ ├── (1, 1) │ │ │ │ └── (2, 2) │ │ │ └── aggregations - │ │ │ └── first-agg [as=column1:3] - │ │ │ └── column1:3 + │ │ │ └── first-agg [as=column1:4] + │ │ │ └── column1:4 │ │ ├── scan p2 - │ │ │ └── columns: p:5!null fk:6 + │ │ │ └── columns: p:6!null fk:7 crdb_internal_mvcc_timestamp:8 │ │ └── filters - │ │ └── column2:4 = fk:6 + │ │ └── column2:5 = fk:7 │ └── projections - │ └── column1:3 + 1 [as=p_new:7] + │ └── column1:4 + 1 [as=p_new:9] └── projections - ├── CASE WHEN p:5 IS NULL THEN column1:3 ELSE p_new:7 END [as=upsert_p:8] - └── CASE WHEN p:5 IS NULL THEN column2:4 ELSE fk:6 END [as=upsert_fk:9] + ├── CASE WHEN p:6 IS NULL THEN column1:4 ELSE p_new:9 END [as=upsert_p:10] + └── CASE WHEN p:6 IS NULL THEN column2:5 ELSE fk:7 END [as=upsert_fk:11] build INSERT INTO p2 VALUES (1, 1), (2, 2) ON CONFLICT (fk) DO UPDATE SET fk = excluded.fk + 1 ---- upsert p2 ├── columns: - ├── canary column: 5 - ├── fetch columns: p:5 p2.fk:6 + ├── canary column: 6 + ├── fetch columns: p:6 p2.fk:7 ├── insert-mapping: - │ ├── column1:3 => p:1 - │ └── column2:4 => p2.fk:2 + │ ├── column1:4 => p:1 + │ └── column2:5 => p2.fk:2 ├── update-mapping: - │ └── upsert_fk:9 => p2.fk:2 + │ └── upsert_fk:11 => p2.fk:2 ├── input binding: &1 ├── project - │ ├── columns: upsert_p:8 upsert_fk:9!null column1:3!null column2:4!null p:5 p2.fk:6 fk_new:7!null + │ ├── columns: upsert_p:10 upsert_fk:11!null column1:4!null column2:5!null p:6 p2.fk:7 p2.crdb_internal_mvcc_timestamp:8 fk_new:9!null │ ├── project - │ │ ├── columns: fk_new:7!null column1:3!null column2:4!null p:5 p2.fk:6 + │ │ ├── columns: fk_new:9!null column1:4!null column2:5!null p:6 p2.fk:7 p2.crdb_internal_mvcc_timestamp:8 │ │ ├── left-join (hash) - │ │ │ ├── columns: column1:3!null column2:4!null p:5 p2.fk:6 + │ │ │ ├── columns: column1:4!null column2:5!null p:6 p2.fk:7 p2.crdb_internal_mvcc_timestamp:8 │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ ├── columns: column1:3!null column2:4!null - │ │ │ │ ├── grouping columns: column2:4!null + │ │ │ │ ├── columns: column1:4!null column2:5!null + │ │ │ │ ├── grouping columns: column2:5!null │ │ │ │ ├── values - │ │ │ │ │ ├── columns: column1:3!null column2:4!null + │ │ │ │ │ ├── columns: column1:4!null column2:5!null │ │ │ │ │ ├── (1, 1) │ │ │ │ │ └── (2, 2) │ │ │ │ └── aggregations - │ │ │ │ └── first-agg [as=column1:3] - │ │ │ │ └── column1:3 + │ │ │ │ └── first-agg [as=column1:4] + │ │ │ │ └── column1:4 │ │ │ ├── scan p2 - │ │ │ │ └── columns: p:5!null p2.fk:6 + │ │ │ │ └── columns: p:6!null p2.fk:7 p2.crdb_internal_mvcc_timestamp:8 │ │ │ └── filters - │ │ │ └── column2:4 = p2.fk:6 + │ │ │ └── column2:5 = p2.fk:7 │ │ └── projections - │ │ └── column2:4 + 1 [as=fk_new:7] + │ │ └── column2:5 + 1 [as=fk_new:9] │ └── projections - │ ├── CASE WHEN p:5 IS NULL THEN column1:3 ELSE p:5 END [as=upsert_p:8] - │ └── CASE WHEN p:5 IS NULL THEN column2:4 ELSE fk_new:7 END [as=upsert_fk:9] + │ ├── CASE WHEN p:6 IS NULL THEN column1:4 ELSE p:6 END [as=upsert_p:10] + │ └── CASE WHEN p:6 IS NULL THEN column2:5 ELSE fk_new:9 END [as=upsert_fk:11] └── f-k-checks └── f-k-checks-item: p2c(fk) -> p2(fk) └── semi-join (hash) - ├── columns: fk:10 + ├── columns: fk:12 ├── except - │ ├── columns: fk:10 - │ ├── left columns: fk:10 - │ ├── right columns: upsert_fk:11 + │ ├── columns: fk:12 + │ ├── left columns: fk:12 + │ ├── right columns: upsert_fk:13 │ ├── with-scan &1 - │ │ ├── columns: fk:10 + │ │ ├── columns: fk:12 │ │ └── mapping: - │ │ └── p2.fk:6 => fk:10 + │ │ └── p2.fk:7 => fk:12 │ └── with-scan &1 - │ ├── columns: upsert_fk:11!null + │ ├── columns: upsert_fk:13!null │ └── mapping: - │ └── upsert_fk:9 => upsert_fk:11 + │ └── upsert_fk:11 => upsert_fk:13 ├── scan p2c - │ └── columns: p2c.fk:13 + │ └── columns: p2c.fk:15 └── filters - └── fk:10 = p2c.fk:13 + └── fk:12 = p2c.fk:15 # This partial upsert never removes existing values of the fk column; the FK # check is not needed. @@ -1585,36 +1585,36 @@ UPSERT INTO p2(p) VALUES (1), (2) ---- upsert p2 ├── columns: - ├── canary column: 5 - ├── fetch columns: p:5 fk:6 + ├── canary column: 6 + ├── fetch columns: p:6 fk:7 ├── insert-mapping: - │ ├── column1:3 => p:1 - │ └── column4:4 => fk:2 + │ ├── column1:4 => p:1 + │ └── column5:5 => fk:2 └── project - ├── columns: upsert_p:7 upsert_fk:8 column1:3!null column4:4 p:5 fk:6 + ├── columns: upsert_p:9 upsert_fk:10 column1:4!null column5:5 p:6 fk:7 crdb_internal_mvcc_timestamp:8 ├── left-join (hash) - │ ├── columns: column1:3!null column4:4 p:5 fk:6 + │ ├── columns: column1:4!null column5:5 p:6 fk:7 crdb_internal_mvcc_timestamp:8 │ ├── ensure-upsert-distinct-on - │ │ ├── columns: column1:3!null column4:4 - │ │ ├── grouping columns: column1:3!null + │ │ ├── columns: column1:4!null column5:5 + │ │ ├── grouping columns: column1:4!null │ │ ├── project - │ │ │ ├── columns: column4:4 column1:3!null + │ │ │ ├── columns: column5:5 column1:4!null │ │ │ ├── values - │ │ │ │ ├── columns: column1:3!null + │ │ │ │ ├── columns: column1:4!null │ │ │ │ ├── (1,) │ │ │ │ └── (2,) │ │ │ └── projections - │ │ │ └── NULL::INT8 [as=column4:4] + │ │ │ └── NULL::INT8 [as=column5:5] │ │ └── aggregations - │ │ └── first-agg [as=column4:4] - │ │ └── column4:4 + │ │ └── first-agg [as=column5:5] + │ │ └── column5:5 │ ├── scan p2 - │ │ └── columns: p:5!null fk:6 + │ │ └── columns: p:6!null fk:7 crdb_internal_mvcc_timestamp:8 │ └── filters - │ └── column1:3 = p:5 + │ └── column1:4 = p:6 └── projections - ├── CASE WHEN p:5 IS NULL THEN column1:3 ELSE p:5 END [as=upsert_p:7] - └── CASE WHEN p:5 IS NULL THEN column4:4 ELSE fk:6 END [as=upsert_fk:8] + ├── CASE WHEN p:6 IS NULL THEN column1:4 ELSE p:6 END [as=upsert_p:9] + └── CASE WHEN p:6 IS NULL THEN column5:5 ELSE fk:7 END [as=upsert_fk:10] # ------------------------------------------ # Inbound FK tests with multiple FK columns @@ -1625,87 +1625,87 @@ UPSERT INTO pq VALUES (1, 1, 1, 1), (2, 2, 2, 2) ---- upsert pq ├── columns: - ├── canary column: 9 - ├── fetch columns: k:9 pq.p:10 pq.q:11 pq.other:12 + ├── canary column: 10 + ├── fetch columns: k:10 pq.p:11 pq.q:12 pq.other:13 ├── insert-mapping: - │ ├── column1:5 => k:1 - │ ├── column2:6 => pq.p:2 - │ ├── column3:7 => pq.q:3 - │ └── column4:8 => pq.other:4 + │ ├── column1:6 => k:1 + │ ├── column2:7 => pq.p:2 + │ ├── column3:8 => pq.q:3 + │ └── column4:9 => pq.other:4 ├── update-mapping: - │ ├── column2:6 => pq.p:2 - │ ├── column3:7 => pq.q:3 - │ └── column4:8 => pq.other:4 + │ ├── column2:7 => pq.p:2 + │ ├── column3:8 => pq.q:3 + │ └── column4:9 => pq.other:4 ├── input binding: &1 ├── project - │ ├── columns: upsert_k:13 column1:5!null column2:6!null column3:7!null column4:8!null k:9 pq.p:10 pq.q:11 pq.other:12 + │ ├── columns: upsert_k:15 column1:6!null column2:7!null column3:8!null column4:9!null k:10 pq.p:11 pq.q:12 pq.other:13 pq.crdb_internal_mvcc_timestamp:14 │ ├── left-join (hash) - │ │ ├── columns: column1:5!null column2:6!null column3:7!null column4:8!null k:9 pq.p:10 pq.q:11 pq.other:12 + │ │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null k:10 pq.p:11 pq.q:12 pq.other:13 pq.crdb_internal_mvcc_timestamp:14 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null column4:8!null - │ │ │ ├── grouping columns: column1:5!null + │ │ │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null + │ │ │ ├── grouping columns: column1:6!null │ │ │ ├── values - │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null column4:8!null + │ │ │ │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null │ │ │ │ ├── (1, 1, 1, 1) │ │ │ │ └── (2, 2, 2, 2) │ │ │ └── aggregations - │ │ │ ├── first-agg [as=column2:6] - │ │ │ │ └── column2:6 - │ │ │ ├── first-agg [as=column3:7] - │ │ │ │ └── column3:7 - │ │ │ └── first-agg [as=column4:8] - │ │ │ └── column4:8 + │ │ │ ├── first-agg [as=column2:7] + │ │ │ │ └── column2:7 + │ │ │ ├── first-agg [as=column3:8] + │ │ │ │ └── column3:8 + │ │ │ └── first-agg [as=column4:9] + │ │ │ └── column4:9 │ │ ├── scan pq - │ │ │ └── columns: k:9!null pq.p:10 pq.q:11 pq.other:12 + │ │ │ └── columns: k:10!null pq.p:11 pq.q:12 pq.other:13 pq.crdb_internal_mvcc_timestamp:14 │ │ └── filters - │ │ └── column1:5 = k:9 + │ │ └── column1:6 = k:10 │ └── projections - │ └── CASE WHEN k:9 IS NULL THEN column1:5 ELSE k:9 END [as=upsert_k:13] + │ └── CASE WHEN k:10 IS NULL THEN column1:6 ELSE k:10 END [as=upsert_k:15] └── f-k-checks ├── f-k-checks-item: cpq(p,q) -> pq(p,q) │ └── semi-join (hash) - │ ├── columns: p:14 q:15 + │ ├── columns: p:16 q:17 │ ├── except - │ │ ├── columns: p:14 q:15 - │ │ ├── left columns: p:14 q:15 - │ │ ├── right columns: column2:16 column3:17 + │ │ ├── columns: p:16 q:17 + │ │ ├── left columns: p:16 q:17 + │ │ ├── right columns: column2:18 column3:19 │ │ ├── with-scan &1 - │ │ │ ├── columns: p:14 q:15 + │ │ │ ├── columns: p:16 q:17 │ │ │ └── mapping: - │ │ │ ├── pq.p:10 => p:14 - │ │ │ └── pq.q:11 => q:15 + │ │ │ ├── pq.p:11 => p:16 + │ │ │ └── pq.q:12 => q:17 │ │ └── with-scan &1 - │ │ ├── columns: column2:16!null column3:17!null + │ │ ├── columns: column2:18!null column3:19!null │ │ └── mapping: - │ │ ├── column2:6 => column2:16 - │ │ └── column3:7 => column3:17 + │ │ ├── column2:7 => column2:18 + │ │ └── column3:8 => column3:19 │ ├── scan cpq - │ │ └── columns: cpq.p:19 cpq.q:20 + │ │ └── columns: cpq.p:21 cpq.q:22 │ └── filters - │ ├── p:14 = cpq.p:19 - │ └── q:15 = cpq.q:20 + │ ├── p:16 = cpq.p:21 + │ └── q:17 = cpq.q:22 └── f-k-checks-item: cmulti(b,c) -> pq(p,q) └── semi-join (hash) - ├── columns: p:22 q:23 + ├── columns: p:25 q:26 ├── except - │ ├── columns: p:22 q:23 - │ ├── left columns: p:22 q:23 - │ ├── right columns: column2:24 column3:25 + │ ├── columns: p:25 q:26 + │ ├── left columns: p:25 q:26 + │ ├── right columns: column2:27 column3:28 │ ├── with-scan &1 - │ │ ├── columns: p:22 q:23 + │ │ ├── columns: p:25 q:26 │ │ └── mapping: - │ │ ├── pq.p:10 => p:22 - │ │ └── pq.q:11 => q:23 + │ │ ├── pq.p:11 => p:25 + │ │ └── pq.q:12 => q:26 │ └── with-scan &1 - │ ├── columns: column2:24!null column3:25!null + │ ├── columns: column2:27!null column3:28!null │ └── mapping: - │ ├── column2:6 => column2:24 - │ └── column3:7 => column3:25 + │ ├── column2:7 => column2:27 + │ └── column3:8 => column3:28 ├── scan cmulti - │ └── columns: b:27!null cmulti.c:28 + │ └── columns: b:30!null cmulti.c:31 └── filters - ├── p:22 = b:27 - └── q:23 = cmulti.c:28 + ├── p:25 = b:30 + └── q:26 = cmulti.c:31 # Partial UPSERT doesn't remove (p,q) values; FK check not needed. build @@ -1713,129 +1713,129 @@ UPSERT INTO pq (k) VALUES (1), (2) ---- upsert pq ├── columns: - ├── canary column: 7 - ├── fetch columns: k:7 p:8 q:9 other:10 + ├── canary column: 8 + ├── fetch columns: k:8 p:9 q:10 other:11 ├── insert-mapping: - │ ├── column1:5 => k:1 - │ ├── column6:6 => p:2 - │ ├── column6:6 => q:3 - │ └── column6:6 => other:4 + │ ├── column1:6 => k:1 + │ ├── column7:7 => p:2 + │ ├── column7:7 => q:3 + │ └── column7:7 => other:4 └── project - ├── columns: upsert_k:11 upsert_p:12 upsert_q:13 upsert_other:14 column1:5!null column6:6 k:7 p:8 q:9 other:10 + ├── columns: upsert_k:13 upsert_p:14 upsert_q:15 upsert_other:16 column1:6!null column7:7 k:8 p:9 q:10 other:11 crdb_internal_mvcc_timestamp:12 ├── left-join (hash) - │ ├── columns: column1:5!null column6:6 k:7 p:8 q:9 other:10 + │ ├── columns: column1:6!null column7:7 k:8 p:9 q:10 other:11 crdb_internal_mvcc_timestamp:12 │ ├── ensure-upsert-distinct-on - │ │ ├── columns: column1:5!null column6:6 - │ │ ├── grouping columns: column1:5!null + │ │ ├── columns: column1:6!null column7:7 + │ │ ├── grouping columns: column1:6!null │ │ ├── project - │ │ │ ├── columns: column6:6 column1:5!null + │ │ │ ├── columns: column7:7 column1:6!null │ │ │ ├── values - │ │ │ │ ├── columns: column1:5!null + │ │ │ │ ├── columns: column1:6!null │ │ │ │ ├── (1,) │ │ │ │ └── (2,) │ │ │ └── projections - │ │ │ └── NULL::INT8 [as=column6:6] + │ │ │ └── NULL::INT8 [as=column7:7] │ │ └── aggregations - │ │ └── first-agg [as=column6:6] - │ │ └── column6:6 + │ │ └── first-agg [as=column7:7] + │ │ └── column7:7 │ ├── scan pq - │ │ └── columns: k:7!null p:8 q:9 other:10 + │ │ └── columns: k:8!null p:9 q:10 other:11 crdb_internal_mvcc_timestamp:12 │ └── filters - │ └── column1:5 = k:7 + │ └── column1:6 = k:8 └── projections - ├── CASE WHEN k:7 IS NULL THEN column1:5 ELSE k:7 END [as=upsert_k:11] - ├── CASE WHEN k:7 IS NULL THEN column6:6 ELSE p:8 END [as=upsert_p:12] - ├── CASE WHEN k:7 IS NULL THEN column6:6 ELSE q:9 END [as=upsert_q:13] - └── CASE WHEN k:7 IS NULL THEN column6:6 ELSE other:10 END [as=upsert_other:14] + ├── CASE WHEN k:8 IS NULL THEN column1:6 ELSE k:8 END [as=upsert_k:13] + ├── CASE WHEN k:8 IS NULL THEN column7:7 ELSE p:9 END [as=upsert_p:14] + ├── CASE WHEN k:8 IS NULL THEN column7:7 ELSE q:10 END [as=upsert_q:15] + └── CASE WHEN k:8 IS NULL THEN column7:7 ELSE other:11 END [as=upsert_other:16] build UPSERT INTO pq (k,q) VALUES (1, 1), (2, 2) ---- upsert pq ├── columns: - ├── canary column: 8 - ├── fetch columns: k:8 pq.p:9 pq.q:10 pq.other:11 + ├── canary column: 9 + ├── fetch columns: k:9 pq.p:10 pq.q:11 pq.other:12 ├── insert-mapping: - │ ├── column1:5 => k:1 - │ ├── column7:7 => pq.p:2 - │ ├── column2:6 => pq.q:3 - │ └── column7:7 => pq.other:4 + │ ├── column1:6 => k:1 + │ ├── column8:8 => pq.p:2 + │ ├── column2:7 => pq.q:3 + │ └── column8:8 => pq.other:4 ├── update-mapping: - │ └── column2:6 => pq.q:3 + │ └── column2:7 => pq.q:3 ├── input binding: &1 ├── project - │ ├── columns: upsert_k:12 upsert_p:13 upsert_other:14 column1:5!null column2:6!null column7:7 k:8 pq.p:9 pq.q:10 pq.other:11 + │ ├── columns: upsert_k:14 upsert_p:15 upsert_other:16 column1:6!null column2:7!null column8:8 k:9 pq.p:10 pq.q:11 pq.other:12 pq.crdb_internal_mvcc_timestamp:13 │ ├── left-join (hash) - │ │ ├── columns: column1:5!null column2:6!null column7:7 k:8 pq.p:9 pq.q:10 pq.other:11 + │ │ ├── columns: column1:6!null column2:7!null column8:8 k:9 pq.p:10 pq.q:11 pq.other:12 pq.crdb_internal_mvcc_timestamp:13 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: column1:5!null column2:6!null column7:7 - │ │ │ ├── grouping columns: column1:5!null + │ │ │ ├── columns: column1:6!null column2:7!null column8:8 + │ │ │ ├── grouping columns: column1:6!null │ │ │ ├── project - │ │ │ │ ├── columns: column7:7 column1:5!null column2:6!null + │ │ │ │ ├── columns: column8:8 column1:6!null column2:7!null │ │ │ │ ├── values - │ │ │ │ │ ├── columns: column1:5!null column2:6!null + │ │ │ │ │ ├── columns: column1:6!null column2:7!null │ │ │ │ │ ├── (1, 1) │ │ │ │ │ └── (2, 2) │ │ │ │ └── projections - │ │ │ │ └── NULL::INT8 [as=column7:7] + │ │ │ │ └── NULL::INT8 [as=column8:8] │ │ │ └── aggregations - │ │ │ ├── first-agg [as=column2:6] - │ │ │ │ └── column2:6 - │ │ │ └── first-agg [as=column7:7] - │ │ │ └── column7:7 + │ │ │ ├── first-agg [as=column2:7] + │ │ │ │ └── column2:7 + │ │ │ └── first-agg [as=column8:8] + │ │ │ └── column8:8 │ │ ├── scan pq - │ │ │ └── columns: k:8!null pq.p:9 pq.q:10 pq.other:11 + │ │ │ └── columns: k:9!null pq.p:10 pq.q:11 pq.other:12 pq.crdb_internal_mvcc_timestamp:13 │ │ └── filters - │ │ └── column1:5 = k:8 + │ │ └── column1:6 = k:9 │ └── projections - │ ├── CASE WHEN k:8 IS NULL THEN column1:5 ELSE k:8 END [as=upsert_k:12] - │ ├── CASE WHEN k:8 IS NULL THEN column7:7 ELSE pq.p:9 END [as=upsert_p:13] - │ └── CASE WHEN k:8 IS NULL THEN column7:7 ELSE pq.other:11 END [as=upsert_other:14] + │ ├── CASE WHEN k:9 IS NULL THEN column1:6 ELSE k:9 END [as=upsert_k:14] + │ ├── CASE WHEN k:9 IS NULL THEN column8:8 ELSE pq.p:10 END [as=upsert_p:15] + │ └── CASE WHEN k:9 IS NULL THEN column8:8 ELSE pq.other:12 END [as=upsert_other:16] └── f-k-checks ├── f-k-checks-item: cpq(p,q) -> pq(p,q) │ └── semi-join (hash) - │ ├── columns: p:15 q:16 + │ ├── columns: p:17 q:18 │ ├── except - │ │ ├── columns: p:15 q:16 - │ │ ├── left columns: p:15 q:16 - │ │ ├── right columns: upsert_p:17 column2:18 + │ │ ├── columns: p:17 q:18 + │ │ ├── left columns: p:17 q:18 + │ │ ├── right columns: upsert_p:19 column2:20 │ │ ├── with-scan &1 - │ │ │ ├── columns: p:15 q:16 + │ │ │ ├── columns: p:17 q:18 │ │ │ └── mapping: - │ │ │ ├── pq.p:9 => p:15 - │ │ │ └── pq.q:10 => q:16 + │ │ │ ├── pq.p:10 => p:17 + │ │ │ └── pq.q:11 => q:18 │ │ └── with-scan &1 - │ │ ├── columns: upsert_p:17 column2:18!null + │ │ ├── columns: upsert_p:19 column2:20!null │ │ └── mapping: - │ │ ├── upsert_p:13 => upsert_p:17 - │ │ └── column2:6 => column2:18 + │ │ ├── upsert_p:15 => upsert_p:19 + │ │ └── column2:7 => column2:20 │ ├── scan cpq - │ │ └── columns: cpq.p:20 cpq.q:21 + │ │ └── columns: cpq.p:22 cpq.q:23 │ └── filters - │ ├── p:15 = cpq.p:20 - │ └── q:16 = cpq.q:21 + │ ├── p:17 = cpq.p:22 + │ └── q:18 = cpq.q:23 └── f-k-checks-item: cmulti(b,c) -> pq(p,q) └── semi-join (hash) - ├── columns: p:23 q:24 + ├── columns: p:26 q:27 ├── except - │ ├── columns: p:23 q:24 - │ ├── left columns: p:23 q:24 - │ ├── right columns: upsert_p:25 column2:26 + │ ├── columns: p:26 q:27 + │ ├── left columns: p:26 q:27 + │ ├── right columns: upsert_p:28 column2:29 │ ├── with-scan &1 - │ │ ├── columns: p:23 q:24 + │ │ ├── columns: p:26 q:27 │ │ └── mapping: - │ │ ├── pq.p:9 => p:23 - │ │ └── pq.q:10 => q:24 + │ │ ├── pq.p:10 => p:26 + │ │ └── pq.q:11 => q:27 │ └── with-scan &1 - │ ├── columns: upsert_p:25 column2:26!null + │ ├── columns: upsert_p:28 column2:29!null │ └── mapping: - │ ├── upsert_p:13 => upsert_p:25 - │ └── column2:6 => column2:26 + │ ├── upsert_p:15 => upsert_p:28 + │ └── column2:7 => column2:29 ├── scan cmulti - │ └── columns: b:28!null cmulti.c:29 + │ └── columns: b:31!null cmulti.c:32 └── filters - ├── p:23 = b:28 - └── q:24 = cmulti.c:29 + ├── p:26 = b:31 + └── q:27 = cmulti.c:32 # Statement doesn't remove (p,q) values; FK check not needed. build @@ -1843,136 +1843,136 @@ INSERT INTO pq VALUES (1, 1, 1, 1), (2, 2, 2, 2) ON CONFLICT (p,q) DO UPDATE SET ---- upsert pq ├── columns: - ├── canary column: 9 - ├── fetch columns: k:9 p:10 q:11 other:12 + ├── canary column: 10 + ├── fetch columns: k:10 p:11 q:12 other:13 ├── insert-mapping: - │ ├── column1:5 => k:1 - │ ├── column2:6 => p:2 - │ ├── column3:7 => q:3 - │ └── column4:8 => other:4 + │ ├── column1:6 => k:1 + │ ├── column2:7 => p:2 + │ ├── column3:8 => q:3 + │ └── column4:9 => other:4 ├── update-mapping: - │ └── upsert_k:14 => k:1 + │ └── upsert_k:16 => k:1 └── project - ├── columns: upsert_k:14 upsert_p:15 upsert_q:16 upsert_other:17 column1:5!null column2:6!null column3:7!null column4:8!null k:9 p:10 q:11 other:12 k_new:13 + ├── columns: upsert_k:16 upsert_p:17 upsert_q:18 upsert_other:19 column1:6!null column2:7!null column3:8!null column4:9!null k:10 p:11 q:12 other:13 crdb_internal_mvcc_timestamp:14 k_new:15 ├── project - │ ├── columns: k_new:13 column1:5!null column2:6!null column3:7!null column4:8!null k:9 p:10 q:11 other:12 + │ ├── columns: k_new:15 column1:6!null column2:7!null column3:8!null column4:9!null k:10 p:11 q:12 other:13 crdb_internal_mvcc_timestamp:14 │ ├── left-join (hash) - │ │ ├── columns: column1:5!null column2:6!null column3:7!null column4:8!null k:9 p:10 q:11 other:12 + │ │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null k:10 p:11 q:12 other:13 crdb_internal_mvcc_timestamp:14 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null column4:8!null - │ │ │ ├── grouping columns: column2:6!null column3:7!null + │ │ │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null + │ │ │ ├── grouping columns: column2:7!null column3:8!null │ │ │ ├── values - │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null column4:8!null + │ │ │ │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null │ │ │ │ ├── (1, 1, 1, 1) │ │ │ │ └── (2, 2, 2, 2) │ │ │ └── aggregations - │ │ │ ├── first-agg [as=column1:5] - │ │ │ │ └── column1:5 - │ │ │ └── first-agg [as=column4:8] - │ │ │ └── column4:8 + │ │ │ ├── first-agg [as=column1:6] + │ │ │ │ └── column1:6 + │ │ │ └── first-agg [as=column4:9] + │ │ │ └── column4:9 │ │ ├── scan pq - │ │ │ └── columns: k:9!null p:10 q:11 other:12 + │ │ │ └── columns: k:10!null p:11 q:12 other:13 crdb_internal_mvcc_timestamp:14 │ │ └── filters - │ │ ├── column2:6 = p:10 - │ │ └── column3:7 = q:11 + │ │ ├── column2:7 = p:11 + │ │ └── column3:8 = q:12 │ └── projections - │ └── k:9 + 1 [as=k_new:13] + │ └── k:10 + 1 [as=k_new:15] └── projections - ├── CASE WHEN k:9 IS NULL THEN column1:5 ELSE k_new:13 END [as=upsert_k:14] - ├── CASE WHEN k:9 IS NULL THEN column2:6 ELSE p:10 END [as=upsert_p:15] - ├── CASE WHEN k:9 IS NULL THEN column3:7 ELSE q:11 END [as=upsert_q:16] - └── CASE WHEN k:9 IS NULL THEN column4:8 ELSE other:12 END [as=upsert_other:17] + ├── CASE WHEN k:10 IS NULL THEN column1:6 ELSE k_new:15 END [as=upsert_k:16] + ├── CASE WHEN k:10 IS NULL THEN column2:7 ELSE p:11 END [as=upsert_p:17] + ├── CASE WHEN k:10 IS NULL THEN column3:8 ELSE q:12 END [as=upsert_q:18] + └── CASE WHEN k:10 IS NULL THEN column4:9 ELSE other:13 END [as=upsert_other:19] build INSERT INTO pq VALUES (1, 1, 1, 1), (2, 2, 2, 2) ON CONFLICT (p,q) DO UPDATE SET p = pq.p + 1 ---- upsert pq ├── columns: - ├── canary column: 9 - ├── fetch columns: k:9 pq.p:10 pq.q:11 pq.other:12 + ├── canary column: 10 + ├── fetch columns: k:10 pq.p:11 pq.q:12 pq.other:13 ├── insert-mapping: - │ ├── column1:5 => k:1 - │ ├── column2:6 => pq.p:2 - │ ├── column3:7 => pq.q:3 - │ └── column4:8 => pq.other:4 + │ ├── column1:6 => k:1 + │ ├── column2:7 => pq.p:2 + │ ├── column3:8 => pq.q:3 + │ └── column4:9 => pq.other:4 ├── update-mapping: - │ └── upsert_p:15 => pq.p:2 + │ └── upsert_p:17 => pq.p:2 ├── input binding: &1 ├── project - │ ├── columns: upsert_k:14 upsert_p:15 upsert_q:16 upsert_other:17 column1:5!null column2:6!null column3:7!null column4:8!null k:9 pq.p:10 pq.q:11 pq.other:12 p_new:13 + │ ├── columns: upsert_k:16 upsert_p:17 upsert_q:18 upsert_other:19 column1:6!null column2:7!null column3:8!null column4:9!null k:10 pq.p:11 pq.q:12 pq.other:13 pq.crdb_internal_mvcc_timestamp:14 p_new:15 │ ├── project - │ │ ├── columns: p_new:13 column1:5!null column2:6!null column3:7!null column4:8!null k:9 pq.p:10 pq.q:11 pq.other:12 + │ │ ├── columns: p_new:15 column1:6!null column2:7!null column3:8!null column4:9!null k:10 pq.p:11 pq.q:12 pq.other:13 pq.crdb_internal_mvcc_timestamp:14 │ │ ├── left-join (hash) - │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null column4:8!null k:9 pq.p:10 pq.q:11 pq.other:12 + │ │ │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null k:10 pq.p:11 pq.q:12 pq.other:13 pq.crdb_internal_mvcc_timestamp:14 │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null column4:8!null - │ │ │ │ ├── grouping columns: column2:6!null column3:7!null + │ │ │ │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null + │ │ │ │ ├── grouping columns: column2:7!null column3:8!null │ │ │ │ ├── values - │ │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null column4:8!null + │ │ │ │ │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null │ │ │ │ │ ├── (1, 1, 1, 1) │ │ │ │ │ └── (2, 2, 2, 2) │ │ │ │ └── aggregations - │ │ │ │ ├── first-agg [as=column1:5] - │ │ │ │ │ └── column1:5 - │ │ │ │ └── first-agg [as=column4:8] - │ │ │ │ └── column4:8 + │ │ │ │ ├── first-agg [as=column1:6] + │ │ │ │ │ └── column1:6 + │ │ │ │ └── first-agg [as=column4:9] + │ │ │ │ └── column4:9 │ │ │ ├── scan pq - │ │ │ │ └── columns: k:9!null pq.p:10 pq.q:11 pq.other:12 + │ │ │ │ └── columns: k:10!null pq.p:11 pq.q:12 pq.other:13 pq.crdb_internal_mvcc_timestamp:14 │ │ │ └── filters - │ │ │ ├── column2:6 = pq.p:10 - │ │ │ └── column3:7 = pq.q:11 + │ │ │ ├── column2:7 = pq.p:11 + │ │ │ └── column3:8 = pq.q:12 │ │ └── projections - │ │ └── pq.p:10 + 1 [as=p_new:13] + │ │ └── pq.p:11 + 1 [as=p_new:15] │ └── projections - │ ├── CASE WHEN k:9 IS NULL THEN column1:5 ELSE k:9 END [as=upsert_k:14] - │ ├── CASE WHEN k:9 IS NULL THEN column2:6 ELSE p_new:13 END [as=upsert_p:15] - │ ├── CASE WHEN k:9 IS NULL THEN column3:7 ELSE pq.q:11 END [as=upsert_q:16] - │ └── CASE WHEN k:9 IS NULL THEN column4:8 ELSE pq.other:12 END [as=upsert_other:17] + │ ├── CASE WHEN k:10 IS NULL THEN column1:6 ELSE k:10 END [as=upsert_k:16] + │ ├── CASE WHEN k:10 IS NULL THEN column2:7 ELSE p_new:15 END [as=upsert_p:17] + │ ├── CASE WHEN k:10 IS NULL THEN column3:8 ELSE pq.q:12 END [as=upsert_q:18] + │ └── CASE WHEN k:10 IS NULL THEN column4:9 ELSE pq.other:13 END [as=upsert_other:19] └── f-k-checks ├── f-k-checks-item: cpq(p,q) -> pq(p,q) │ └── semi-join (hash) - │ ├── columns: p:18 q:19 + │ ├── columns: p:20 q:21 │ ├── except - │ │ ├── columns: p:18 q:19 - │ │ ├── left columns: p:18 q:19 - │ │ ├── right columns: upsert_p:20 upsert_q:21 + │ │ ├── columns: p:20 q:21 + │ │ ├── left columns: p:20 q:21 + │ │ ├── right columns: upsert_p:22 upsert_q:23 │ │ ├── with-scan &1 - │ │ │ ├── columns: p:18 q:19 + │ │ │ ├── columns: p:20 q:21 │ │ │ └── mapping: - │ │ │ ├── pq.p:10 => p:18 - │ │ │ └── pq.q:11 => q:19 + │ │ │ ├── pq.p:11 => p:20 + │ │ │ └── pq.q:12 => q:21 │ │ └── with-scan &1 - │ │ ├── columns: upsert_p:20 upsert_q:21 + │ │ ├── columns: upsert_p:22 upsert_q:23 │ │ └── mapping: - │ │ ├── upsert_p:15 => upsert_p:20 - │ │ └── upsert_q:16 => upsert_q:21 + │ │ ├── upsert_p:17 => upsert_p:22 + │ │ └── upsert_q:18 => upsert_q:23 │ ├── scan cpq - │ │ └── columns: cpq.p:23 cpq.q:24 + │ │ └── columns: cpq.p:25 cpq.q:26 │ └── filters - │ ├── p:18 = cpq.p:23 - │ └── q:19 = cpq.q:24 + │ ├── p:20 = cpq.p:25 + │ └── q:21 = cpq.q:26 └── f-k-checks-item: cmulti(b,c) -> pq(p,q) └── semi-join (hash) - ├── columns: p:26 q:27 + ├── columns: p:29 q:30 ├── except - │ ├── columns: p:26 q:27 - │ ├── left columns: p:26 q:27 - │ ├── right columns: upsert_p:28 upsert_q:29 + │ ├── columns: p:29 q:30 + │ ├── left columns: p:29 q:30 + │ ├── right columns: upsert_p:31 upsert_q:32 │ ├── with-scan &1 - │ │ ├── columns: p:26 q:27 + │ │ ├── columns: p:29 q:30 │ │ └── mapping: - │ │ ├── pq.p:10 => p:26 - │ │ └── pq.q:11 => q:27 + │ │ ├── pq.p:11 => p:29 + │ │ └── pq.q:12 => q:30 │ └── with-scan &1 - │ ├── columns: upsert_p:28 upsert_q:29 + │ ├── columns: upsert_p:31 upsert_q:32 │ └── mapping: - │ ├── upsert_p:15 => upsert_p:28 - │ └── upsert_q:16 => upsert_q:29 + │ ├── upsert_p:17 => upsert_p:31 + │ └── upsert_q:18 => upsert_q:32 ├── scan cmulti - │ └── columns: b:31!null cmulti.c:32 + │ └── columns: b:34!null cmulti.c:35 └── filters - ├── p:26 = b:31 - └── q:27 = cmulti.c:32 + ├── p:29 = b:34 + └── q:30 = cmulti.c:35 # Statement never removes (p,q) values; FK check not needed. build @@ -1980,138 +1980,138 @@ INSERT INTO pq VALUES (1, 1, 1, 1), (2, 2, 2, 2) ON CONFLICT (k) DO UPDATE SET o ---- upsert pq ├── columns: - ├── canary column: 9 - ├── fetch columns: k:9 p:10 q:11 other:12 + ├── canary column: 10 + ├── fetch columns: k:10 p:11 q:12 other:13 ├── insert-mapping: - │ ├── column1:5 => k:1 - │ ├── column2:6 => p:2 - │ ├── column3:7 => q:3 - │ └── column4:8 => other:4 + │ ├── column1:6 => k:1 + │ ├── column2:7 => p:2 + │ ├── column3:8 => q:3 + │ └── column4:9 => other:4 ├── update-mapping: - │ └── upsert_other:17 => other:4 + │ └── upsert_other:19 => other:4 └── project - ├── columns: upsert_k:14 upsert_p:15 upsert_q:16 upsert_other:17!null column1:5!null column2:6!null column3:7!null column4:8!null k:9 p:10 q:11 other:12 other_new:13!null + ├── columns: upsert_k:16 upsert_p:17 upsert_q:18 upsert_other:19!null column1:6!null column2:7!null column3:8!null column4:9!null k:10 p:11 q:12 other:13 crdb_internal_mvcc_timestamp:14 other_new:15!null ├── project - │ ├── columns: other_new:13!null column1:5!null column2:6!null column3:7!null column4:8!null k:9 p:10 q:11 other:12 + │ ├── columns: other_new:15!null column1:6!null column2:7!null column3:8!null column4:9!null k:10 p:11 q:12 other:13 crdb_internal_mvcc_timestamp:14 │ ├── left-join (hash) - │ │ ├── columns: column1:5!null column2:6!null column3:7!null column4:8!null k:9 p:10 q:11 other:12 + │ │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null k:10 p:11 q:12 other:13 crdb_internal_mvcc_timestamp:14 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null column4:8!null - │ │ │ ├── grouping columns: column1:5!null + │ │ │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null + │ │ │ ├── grouping columns: column1:6!null │ │ │ ├── values - │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null column4:8!null + │ │ │ │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null │ │ │ │ ├── (1, 1, 1, 1) │ │ │ │ └── (2, 2, 2, 2) │ │ │ └── aggregations - │ │ │ ├── first-agg [as=column2:6] - │ │ │ │ └── column2:6 - │ │ │ ├── first-agg [as=column3:7] - │ │ │ │ └── column3:7 - │ │ │ └── first-agg [as=column4:8] - │ │ │ └── column4:8 + │ │ │ ├── first-agg [as=column2:7] + │ │ │ │ └── column2:7 + │ │ │ ├── first-agg [as=column3:8] + │ │ │ │ └── column3:8 + │ │ │ └── first-agg [as=column4:9] + │ │ │ └── column4:9 │ │ ├── scan pq - │ │ │ └── columns: k:9!null p:10 q:11 other:12 + │ │ │ └── columns: k:10!null p:11 q:12 other:13 crdb_internal_mvcc_timestamp:14 │ │ └── filters - │ │ └── column1:5 = k:9 + │ │ └── column1:6 = k:10 │ └── projections - │ └── 5 [as=other_new:13] + │ └── 5 [as=other_new:15] └── projections - ├── CASE WHEN k:9 IS NULL THEN column1:5 ELSE k:9 END [as=upsert_k:14] - ├── CASE WHEN k:9 IS NULL THEN column2:6 ELSE p:10 END [as=upsert_p:15] - ├── CASE WHEN k:9 IS NULL THEN column3:7 ELSE q:11 END [as=upsert_q:16] - └── CASE WHEN k:9 IS NULL THEN column4:8 ELSE other_new:13 END [as=upsert_other:17] + ├── CASE WHEN k:10 IS NULL THEN column1:6 ELSE k:10 END [as=upsert_k:16] + ├── CASE WHEN k:10 IS NULL THEN column2:7 ELSE p:11 END [as=upsert_p:17] + ├── CASE WHEN k:10 IS NULL THEN column3:8 ELSE q:12 END [as=upsert_q:18] + └── CASE WHEN k:10 IS NULL THEN column4:9 ELSE other_new:15 END [as=upsert_other:19] build INSERT INTO pq VALUES (1, 1, 1, 1), (2, 2, 2, 2) ON CONFLICT (k) DO UPDATE SET q = 5 ---- upsert pq ├── columns: - ├── canary column: 9 - ├── fetch columns: k:9 pq.p:10 pq.q:11 pq.other:12 + ├── canary column: 10 + ├── fetch columns: k:10 pq.p:11 pq.q:12 pq.other:13 ├── insert-mapping: - │ ├── column1:5 => k:1 - │ ├── column2:6 => pq.p:2 - │ ├── column3:7 => pq.q:3 - │ └── column4:8 => pq.other:4 + │ ├── column1:6 => k:1 + │ ├── column2:7 => pq.p:2 + │ ├── column3:8 => pq.q:3 + │ └── column4:9 => pq.other:4 ├── update-mapping: - │ └── upsert_q:16 => pq.q:3 + │ └── upsert_q:18 => pq.q:3 ├── input binding: &1 ├── project - │ ├── columns: upsert_k:14 upsert_p:15 upsert_q:16!null upsert_other:17 column1:5!null column2:6!null column3:7!null column4:8!null k:9 pq.p:10 pq.q:11 pq.other:12 q_new:13!null + │ ├── columns: upsert_k:16 upsert_p:17 upsert_q:18!null upsert_other:19 column1:6!null column2:7!null column3:8!null column4:9!null k:10 pq.p:11 pq.q:12 pq.other:13 pq.crdb_internal_mvcc_timestamp:14 q_new:15!null │ ├── project - │ │ ├── columns: q_new:13!null column1:5!null column2:6!null column3:7!null column4:8!null k:9 pq.p:10 pq.q:11 pq.other:12 + │ │ ├── columns: q_new:15!null column1:6!null column2:7!null column3:8!null column4:9!null k:10 pq.p:11 pq.q:12 pq.other:13 pq.crdb_internal_mvcc_timestamp:14 │ │ ├── left-join (hash) - │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null column4:8!null k:9 pq.p:10 pq.q:11 pq.other:12 + │ │ │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null k:10 pq.p:11 pq.q:12 pq.other:13 pq.crdb_internal_mvcc_timestamp:14 │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null column4:8!null - │ │ │ │ ├── grouping columns: column1:5!null + │ │ │ │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null + │ │ │ │ ├── grouping columns: column1:6!null │ │ │ │ ├── values - │ │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null column4:8!null + │ │ │ │ │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null │ │ │ │ │ ├── (1, 1, 1, 1) │ │ │ │ │ └── (2, 2, 2, 2) │ │ │ │ └── aggregations - │ │ │ │ ├── first-agg [as=column2:6] - │ │ │ │ │ └── column2:6 - │ │ │ │ ├── first-agg [as=column3:7] - │ │ │ │ │ └── column3:7 - │ │ │ │ └── first-agg [as=column4:8] - │ │ │ │ └── column4:8 + │ │ │ │ ├── first-agg [as=column2:7] + │ │ │ │ │ └── column2:7 + │ │ │ │ ├── first-agg [as=column3:8] + │ │ │ │ │ └── column3:8 + │ │ │ │ └── first-agg [as=column4:9] + │ │ │ │ └── column4:9 │ │ │ ├── scan pq - │ │ │ │ └── columns: k:9!null pq.p:10 pq.q:11 pq.other:12 + │ │ │ │ └── columns: k:10!null pq.p:11 pq.q:12 pq.other:13 pq.crdb_internal_mvcc_timestamp:14 │ │ │ └── filters - │ │ │ └── column1:5 = k:9 + │ │ │ └── column1:6 = k:10 │ │ └── projections - │ │ └── 5 [as=q_new:13] + │ │ └── 5 [as=q_new:15] │ └── projections - │ ├── CASE WHEN k:9 IS NULL THEN column1:5 ELSE k:9 END [as=upsert_k:14] - │ ├── CASE WHEN k:9 IS NULL THEN column2:6 ELSE pq.p:10 END [as=upsert_p:15] - │ ├── CASE WHEN k:9 IS NULL THEN column3:7 ELSE q_new:13 END [as=upsert_q:16] - │ └── CASE WHEN k:9 IS NULL THEN column4:8 ELSE pq.other:12 END [as=upsert_other:17] + │ ├── CASE WHEN k:10 IS NULL THEN column1:6 ELSE k:10 END [as=upsert_k:16] + │ ├── CASE WHEN k:10 IS NULL THEN column2:7 ELSE pq.p:11 END [as=upsert_p:17] + │ ├── CASE WHEN k:10 IS NULL THEN column3:8 ELSE q_new:15 END [as=upsert_q:18] + │ └── CASE WHEN k:10 IS NULL THEN column4:9 ELSE pq.other:13 END [as=upsert_other:19] └── f-k-checks ├── f-k-checks-item: cpq(p,q) -> pq(p,q) │ └── semi-join (hash) - │ ├── columns: p:18 q:19 + │ ├── columns: p:20 q:21 │ ├── except - │ │ ├── columns: p:18 q:19 - │ │ ├── left columns: p:18 q:19 - │ │ ├── right columns: upsert_p:20 upsert_q:21 + │ │ ├── columns: p:20 q:21 + │ │ ├── left columns: p:20 q:21 + │ │ ├── right columns: upsert_p:22 upsert_q:23 │ │ ├── with-scan &1 - │ │ │ ├── columns: p:18 q:19 + │ │ │ ├── columns: p:20 q:21 │ │ │ └── mapping: - │ │ │ ├── pq.p:10 => p:18 - │ │ │ └── pq.q:11 => q:19 + │ │ │ ├── pq.p:11 => p:20 + │ │ │ └── pq.q:12 => q:21 │ │ └── with-scan &1 - │ │ ├── columns: upsert_p:20 upsert_q:21!null + │ │ ├── columns: upsert_p:22 upsert_q:23!null │ │ └── mapping: - │ │ ├── upsert_p:15 => upsert_p:20 - │ │ └── upsert_q:16 => upsert_q:21 + │ │ ├── upsert_p:17 => upsert_p:22 + │ │ └── upsert_q:18 => upsert_q:23 │ ├── scan cpq - │ │ └── columns: cpq.p:23 cpq.q:24 + │ │ └── columns: cpq.p:25 cpq.q:26 │ └── filters - │ ├── p:18 = cpq.p:23 - │ └── q:19 = cpq.q:24 + │ ├── p:20 = cpq.p:25 + │ └── q:21 = cpq.q:26 └── f-k-checks-item: cmulti(b,c) -> pq(p,q) └── semi-join (hash) - ├── columns: p:26 q:27 + ├── columns: p:29 q:30 ├── except - │ ├── columns: p:26 q:27 - │ ├── left columns: p:26 q:27 - │ ├── right columns: upsert_p:28 upsert_q:29 + │ ├── columns: p:29 q:30 + │ ├── left columns: p:29 q:30 + │ ├── right columns: upsert_p:31 upsert_q:32 │ ├── with-scan &1 - │ │ ├── columns: p:26 q:27 + │ │ ├── columns: p:29 q:30 │ │ └── mapping: - │ │ ├── pq.p:10 => p:26 - │ │ └── pq.q:11 => q:27 + │ │ ├── pq.p:11 => p:29 + │ │ └── pq.q:12 => q:30 │ └── with-scan &1 - │ ├── columns: upsert_p:28 upsert_q:29!null + │ ├── columns: upsert_p:31 upsert_q:32!null │ └── mapping: - │ ├── upsert_p:15 => upsert_p:28 - │ └── upsert_q:16 => upsert_q:29 + │ ├── upsert_p:17 => upsert_p:31 + │ └── upsert_q:18 => upsert_q:32 ├── scan cmulti - │ └── columns: b:31!null cmulti.c:32 + │ └── columns: b:34!null cmulti.c:35 └── filters - ├── p:26 = b:31 - └── q:27 = cmulti.c:32 + ├── p:29 = b:34 + └── q:30 = cmulti.c:35 # ------------------------------------- # Inbound + outbound combination tests @@ -2144,150 +2144,150 @@ UPSERT INTO tab2 VALUES (1,NULL,NULL), (2,2,2) ---- upsert tab2 ├── columns: - ├── canary column: 7 - ├── fetch columns: c:7 d:8 tab2.e:9 + ├── canary column: 8 + ├── fetch columns: c:8 d:9 tab2.e:10 ├── insert-mapping: - │ ├── column1:4 => c:1 - │ ├── column2:5 => d:2 - │ └── column3:6 => tab2.e:3 + │ ├── column1:5 => c:1 + │ ├── column2:6 => d:2 + │ └── column3:7 => tab2.e:3 ├── update-mapping: - │ ├── column2:5 => d:2 - │ └── column3:6 => tab2.e:3 + │ ├── column2:6 => d:2 + │ └── column3:7 => tab2.e:3 ├── input binding: &1 ├── project - │ ├── columns: upsert_c:10 column1:4!null column2:5 column3:6 c:7 d:8 tab2.e:9 + │ ├── columns: upsert_c:12 column1:5!null column2:6 column3:7 c:8 d:9 tab2.e:10 tab2.crdb_internal_mvcc_timestamp:11 │ ├── left-join (hash) - │ │ ├── columns: column1:4!null column2:5 column3:6 c:7 d:8 tab2.e:9 + │ │ ├── columns: column1:5!null column2:6 column3:7 c:8 d:9 tab2.e:10 tab2.crdb_internal_mvcc_timestamp:11 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: column1:4!null column2:5 column3:6 - │ │ │ ├── grouping columns: column1:4!null + │ │ │ ├── columns: column1:5!null column2:6 column3:7 + │ │ │ ├── grouping columns: column1:5!null │ │ │ ├── values - │ │ │ │ ├── columns: column1:4!null column2:5 column3:6 + │ │ │ │ ├── columns: column1:5!null column2:6 column3:7 │ │ │ │ ├── (1, NULL::INT8, NULL::INT8) │ │ │ │ └── (2, 2, 2) │ │ │ └── aggregations - │ │ │ ├── first-agg [as=column2:5] - │ │ │ │ └── column2:5 - │ │ │ └── first-agg [as=column3:6] - │ │ │ └── column3:6 + │ │ │ ├── first-agg [as=column2:6] + │ │ │ │ └── column2:6 + │ │ │ └── first-agg [as=column3:7] + │ │ │ └── column3:7 │ │ ├── scan tab2 - │ │ │ └── columns: c:7!null d:8 tab2.e:9 + │ │ │ └── columns: c:8!null d:9 tab2.e:10 tab2.crdb_internal_mvcc_timestamp:11 │ │ └── filters - │ │ └── column1:4 = c:7 + │ │ └── column1:5 = c:8 │ └── projections - │ └── CASE WHEN c:7 IS NULL THEN column1:4 ELSE c:7 END [as=upsert_c:10] + │ └── CASE WHEN c:8 IS NULL THEN column1:5 ELSE c:8 END [as=upsert_c:12] └── f-k-checks ├── f-k-checks-item: tab2(d) -> tab1(b) │ └── anti-join (hash) - │ ├── columns: column2:11!null + │ ├── columns: column2:13!null │ ├── select - │ │ ├── columns: column2:11!null + │ │ ├── columns: column2:13!null │ │ ├── with-scan &1 - │ │ │ ├── columns: column2:11 + │ │ │ ├── columns: column2:13 │ │ │ └── mapping: - │ │ │ └── column2:5 => column2:11 + │ │ │ └── column2:6 => column2:13 │ │ └── filters - │ │ └── column2:11 IS NOT NULL + │ │ └── column2:13 IS NOT NULL │ ├── scan tab1 - │ │ └── columns: b:13 + │ │ └── columns: b:15 │ └── filters - │ └── column2:11 = b:13 + │ └── column2:13 = b:15 └── f-k-checks-item: tab3(g) -> tab2(e) └── semi-join (hash) - ├── columns: e:14 + ├── columns: e:17 ├── except - │ ├── columns: e:14 - │ ├── left columns: e:14 - │ ├── right columns: column3:15 + │ ├── columns: e:17 + │ ├── left columns: e:17 + │ ├── right columns: column3:18 │ ├── with-scan &1 - │ │ ├── columns: e:14 + │ │ ├── columns: e:17 │ │ └── mapping: - │ │ └── tab2.e:9 => e:14 + │ │ └── tab2.e:10 => e:17 │ └── with-scan &1 - │ ├── columns: column3:15 + │ ├── columns: column3:18 │ └── mapping: - │ └── column3:6 => column3:15 + │ └── column3:7 => column3:18 ├── scan tab3 - │ └── columns: g:17 + │ └── columns: g:20 └── filters - └── e:14 = g:17 + └── e:17 = g:20 build INSERT INTO tab2 VALUES (1,1,1) ON CONFLICT (c) DO UPDATE SET e = tab2.e + 1 ---- upsert tab2 ├── columns: - ├── canary column: 7 - ├── fetch columns: c:7 d:8 tab2.e:9 + ├── canary column: 8 + ├── fetch columns: c:8 d:9 tab2.e:10 ├── insert-mapping: - │ ├── column1:4 => c:1 - │ ├── column2:5 => d:2 - │ └── column3:6 => tab2.e:3 + │ ├── column1:5 => c:1 + │ ├── column2:6 => d:2 + │ └── column3:7 => tab2.e:3 ├── update-mapping: - │ └── upsert_e:13 => tab2.e:3 + │ └── upsert_e:15 => tab2.e:3 ├── input binding: &1 ├── project - │ ├── columns: upsert_c:11 upsert_d:12 upsert_e:13 column1:4!null column2:5!null column3:6!null c:7 d:8 tab2.e:9 e_new:10 + │ ├── columns: upsert_c:13 upsert_d:14 upsert_e:15 column1:5!null column2:6!null column3:7!null c:8 d:9 tab2.e:10 tab2.crdb_internal_mvcc_timestamp:11 e_new:12 │ ├── project - │ │ ├── columns: e_new:10 column1:4!null column2:5!null column3:6!null c:7 d:8 tab2.e:9 + │ │ ├── columns: e_new:12 column1:5!null column2:6!null column3:7!null c:8 d:9 tab2.e:10 tab2.crdb_internal_mvcc_timestamp:11 │ │ ├── left-join (hash) - │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null c:7 d:8 tab2.e:9 + │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null c:8 d:9 tab2.e:10 tab2.crdb_internal_mvcc_timestamp:11 │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null - │ │ │ │ ├── grouping columns: column1:4!null + │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null + │ │ │ │ ├── grouping columns: column1:5!null │ │ │ │ ├── values - │ │ │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null + │ │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null │ │ │ │ │ └── (1, 1, 1) │ │ │ │ └── aggregations - │ │ │ │ ├── first-agg [as=column2:5] - │ │ │ │ │ └── column2:5 - │ │ │ │ └── first-agg [as=column3:6] - │ │ │ │ └── column3:6 + │ │ │ │ ├── first-agg [as=column2:6] + │ │ │ │ │ └── column2:6 + │ │ │ │ └── first-agg [as=column3:7] + │ │ │ │ └── column3:7 │ │ │ ├── scan tab2 - │ │ │ │ └── columns: c:7!null d:8 tab2.e:9 + │ │ │ │ └── columns: c:8!null d:9 tab2.e:10 tab2.crdb_internal_mvcc_timestamp:11 │ │ │ └── filters - │ │ │ └── column1:4 = c:7 + │ │ │ └── column1:5 = c:8 │ │ └── projections - │ │ └── tab2.e:9 + 1 [as=e_new:10] + │ │ └── tab2.e:10 + 1 [as=e_new:12] │ └── projections - │ ├── CASE WHEN c:7 IS NULL THEN column1:4 ELSE c:7 END [as=upsert_c:11] - │ ├── CASE WHEN c:7 IS NULL THEN column2:5 ELSE d:8 END [as=upsert_d:12] - │ └── CASE WHEN c:7 IS NULL THEN column3:6 ELSE e_new:10 END [as=upsert_e:13] + │ ├── CASE WHEN c:8 IS NULL THEN column1:5 ELSE c:8 END [as=upsert_c:13] + │ ├── CASE WHEN c:8 IS NULL THEN column2:6 ELSE d:9 END [as=upsert_d:14] + │ └── CASE WHEN c:8 IS NULL THEN column3:7 ELSE e_new:12 END [as=upsert_e:15] └── f-k-checks ├── f-k-checks-item: tab2(d) -> tab1(b) │ └── anti-join (hash) - │ ├── columns: upsert_d:14!null + │ ├── columns: upsert_d:16!null │ ├── select - │ │ ├── columns: upsert_d:14!null + │ │ ├── columns: upsert_d:16!null │ │ ├── with-scan &1 - │ │ │ ├── columns: upsert_d:14 + │ │ │ ├── columns: upsert_d:16 │ │ │ └── mapping: - │ │ │ └── upsert_d:12 => upsert_d:14 + │ │ │ └── upsert_d:14 => upsert_d:16 │ │ └── filters - │ │ └── upsert_d:14 IS NOT NULL + │ │ └── upsert_d:16 IS NOT NULL │ ├── scan tab1 - │ │ └── columns: b:16 + │ │ └── columns: b:18 │ └── filters - │ └── upsert_d:14 = b:16 + │ └── upsert_d:16 = b:18 └── f-k-checks-item: tab3(g) -> tab2(e) └── semi-join (hash) - ├── columns: e:17 + ├── columns: e:20 ├── except - │ ├── columns: e:17 - │ ├── left columns: e:17 - │ ├── right columns: upsert_e:18 + │ ├── columns: e:20 + │ ├── left columns: e:20 + │ ├── right columns: upsert_e:21 │ ├── with-scan &1 - │ │ ├── columns: e:17 + │ │ ├── columns: e:20 │ │ └── mapping: - │ │ └── tab2.e:9 => e:17 + │ │ └── tab2.e:10 => e:20 │ └── with-scan &1 - │ ├── columns: upsert_e:18 + │ ├── columns: upsert_e:21 │ └── mapping: - │ └── upsert_e:13 => upsert_e:18 + │ └── upsert_e:15 => upsert_e:21 ├── scan tab3 - │ └── columns: g:20 + │ └── columns: g:23 └── filters - └── e:17 = g:20 + └── e:20 = g:23 # Statement never removes values from e column; the inbound check is not necessary. build @@ -2295,58 +2295,58 @@ INSERT INTO tab2 VALUES (1,1,1) ON CONFLICT (e) DO UPDATE SET d = tab2.d + 1 ---- upsert tab2 ├── columns: - ├── canary column: 7 - ├── fetch columns: c:7 d:8 e:9 + ├── canary column: 8 + ├── fetch columns: c:8 d:9 e:10 ├── insert-mapping: - │ ├── column1:4 => c:1 - │ ├── column2:5 => d:2 - │ └── column3:6 => e:3 + │ ├── column1:5 => c:1 + │ ├── column2:6 => d:2 + │ └── column3:7 => e:3 ├── update-mapping: - │ └── upsert_d:12 => d:2 + │ └── upsert_d:14 => d:2 ├── input binding: &1 ├── project - │ ├── columns: upsert_c:11 upsert_d:12 upsert_e:13 column1:4!null column2:5!null column3:6!null c:7 d:8 e:9 d_new:10 + │ ├── columns: upsert_c:13 upsert_d:14 upsert_e:15 column1:5!null column2:6!null column3:7!null c:8 d:9 e:10 tab2.crdb_internal_mvcc_timestamp:11 d_new:12 │ ├── project - │ │ ├── columns: d_new:10 column1:4!null column2:5!null column3:6!null c:7 d:8 e:9 + │ │ ├── columns: d_new:12 column1:5!null column2:6!null column3:7!null c:8 d:9 e:10 tab2.crdb_internal_mvcc_timestamp:11 │ │ ├── left-join (hash) - │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null c:7 d:8 e:9 + │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null c:8 d:9 e:10 tab2.crdb_internal_mvcc_timestamp:11 │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null - │ │ │ │ ├── grouping columns: column3:6!null + │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null + │ │ │ │ ├── grouping columns: column3:7!null │ │ │ │ ├── values - │ │ │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null + │ │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null │ │ │ │ │ └── (1, 1, 1) │ │ │ │ └── aggregations - │ │ │ │ ├── first-agg [as=column1:4] - │ │ │ │ │ └── column1:4 - │ │ │ │ └── first-agg [as=column2:5] - │ │ │ │ └── column2:5 + │ │ │ │ ├── first-agg [as=column1:5] + │ │ │ │ │ └── column1:5 + │ │ │ │ └── first-agg [as=column2:6] + │ │ │ │ └── column2:6 │ │ │ ├── scan tab2 - │ │ │ │ └── columns: c:7!null d:8 e:9 + │ │ │ │ └── columns: c:8!null d:9 e:10 tab2.crdb_internal_mvcc_timestamp:11 │ │ │ └── filters - │ │ │ └── column3:6 = e:9 + │ │ │ └── column3:7 = e:10 │ │ └── projections - │ │ └── d:8 + 1 [as=d_new:10] + │ │ └── d:9 + 1 [as=d_new:12] │ └── projections - │ ├── CASE WHEN c:7 IS NULL THEN column1:4 ELSE c:7 END [as=upsert_c:11] - │ ├── CASE WHEN c:7 IS NULL THEN column2:5 ELSE d_new:10 END [as=upsert_d:12] - │ └── CASE WHEN c:7 IS NULL THEN column3:6 ELSE e:9 END [as=upsert_e:13] + │ ├── CASE WHEN c:8 IS NULL THEN column1:5 ELSE c:8 END [as=upsert_c:13] + │ ├── CASE WHEN c:8 IS NULL THEN column2:6 ELSE d_new:12 END [as=upsert_d:14] + │ └── CASE WHEN c:8 IS NULL THEN column3:7 ELSE e:10 END [as=upsert_e:15] └── f-k-checks └── f-k-checks-item: tab2(d) -> tab1(b) └── anti-join (hash) - ├── columns: upsert_d:14!null + ├── columns: upsert_d:16!null ├── select - │ ├── columns: upsert_d:14!null + │ ├── columns: upsert_d:16!null │ ├── with-scan &1 - │ │ ├── columns: upsert_d:14 + │ │ ├── columns: upsert_d:16 │ │ └── mapping: - │ │ └── upsert_d:12 => upsert_d:14 + │ │ └── upsert_d:14 => upsert_d:16 │ └── filters - │ └── upsert_d:14 IS NOT NULL + │ └── upsert_d:16 IS NOT NULL ├── scan tab1 - │ └── columns: b:16 + │ └── columns: b:18 └── filters - └── upsert_d:14 = b:16 + └── upsert_d:16 = b:18 exec-ddl CREATE TABLE self ( @@ -2367,108 +2367,108 @@ UPSERT INTO self SELECT x, y, z, w FROM xyzw ---- upsert self ├── columns: - ├── canary column: 10 - ├── fetch columns: a:10 self.b:11 self.c:12 self.d:13 + ├── canary column: 12 + ├── fetch columns: a:12 self.b:13 self.c:14 self.d:15 ├── insert-mapping: - │ ├── x:5 => a:1 - │ ├── y:6 => self.b:2 - │ ├── xyzw.z:7 => self.c:3 - │ └── xyzw.w:8 => self.d:4 + │ ├── x:6 => a:1 + │ ├── y:7 => self.b:2 + │ ├── xyzw.z:8 => self.c:3 + │ └── xyzw.w:9 => self.d:4 ├── update-mapping: - │ ├── xyzw.z:7 => self.c:3 - │ └── xyzw.w:8 => self.d:4 + │ ├── xyzw.z:8 => self.c:3 + │ └── xyzw.w:9 => self.d:4 ├── input binding: &1 ├── project - │ ├── columns: upsert_a:14 upsert_b:15 x:5 y:6 xyzw.z:7 xyzw.w:8 a:10 self.b:11 self.c:12 self.d:13 + │ ├── columns: upsert_a:17 upsert_b:18 x:6 y:7 xyzw.z:8 xyzw.w:9 a:12 self.b:13 self.c:14 self.d:15 self.crdb_internal_mvcc_timestamp:16 │ ├── left-join (hash) - │ │ ├── columns: x:5 y:6 xyzw.z:7 xyzw.w:8 a:10 self.b:11 self.c:12 self.d:13 + │ │ ├── columns: x:6 y:7 xyzw.z:8 xyzw.w:9 a:12 self.b:13 self.c:14 self.d:15 self.crdb_internal_mvcc_timestamp:16 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: x:5 y:6 xyzw.z:7 xyzw.w:8 - │ │ │ ├── grouping columns: x:5 y:6 + │ │ │ ├── columns: x:6 y:7 xyzw.z:8 xyzw.w:9 + │ │ │ ├── grouping columns: x:6 y:7 │ │ │ ├── project - │ │ │ │ ├── columns: x:5 y:6 xyzw.z:7 xyzw.w:8 + │ │ │ │ ├── columns: x:6 y:7 xyzw.z:8 xyzw.w:9 │ │ │ │ └── scan xyzw - │ │ │ │ └── columns: x:5 y:6 xyzw.z:7 xyzw.w:8 rowid:9!null + │ │ │ │ └── columns: x:6 y:7 xyzw.z:8 xyzw.w:9 rowid:10!null xyzw.crdb_internal_mvcc_timestamp:11 │ │ │ └── aggregations - │ │ │ ├── first-agg [as=xyzw.z:7] - │ │ │ │ └── xyzw.z:7 - │ │ │ └── first-agg [as=xyzw.w:8] - │ │ │ └── xyzw.w:8 + │ │ │ ├── first-agg [as=xyzw.z:8] + │ │ │ │ └── xyzw.z:8 + │ │ │ └── first-agg [as=xyzw.w:9] + │ │ │ └── xyzw.w:9 │ │ ├── scan self - │ │ │ └── columns: a:10!null self.b:11!null self.c:12 self.d:13 + │ │ │ └── columns: a:12!null self.b:13!null self.c:14 self.d:15 self.crdb_internal_mvcc_timestamp:16 │ │ └── filters - │ │ ├── x:5 = a:10 - │ │ └── y:6 = self.b:11 + │ │ ├── x:6 = a:12 + │ │ └── y:7 = self.b:13 │ └── projections - │ ├── CASE WHEN a:10 IS NULL THEN x:5 ELSE a:10 END [as=upsert_a:14] - │ └── CASE WHEN a:10 IS NULL THEN y:6 ELSE self.b:11 END [as=upsert_b:15] + │ ├── CASE WHEN a:12 IS NULL THEN x:6 ELSE a:12 END [as=upsert_a:17] + │ └── CASE WHEN a:12 IS NULL THEN y:7 ELSE self.b:13 END [as=upsert_b:18] └── f-k-checks ├── f-k-checks-item: self(a,b) -> self(b,d) │ └── anti-join (hash) - │ ├── columns: upsert_a:16 upsert_b:17 + │ ├── columns: upsert_a:19 upsert_b:20 │ ├── with-scan &1 - │ │ ├── columns: upsert_a:16 upsert_b:17 + │ │ ├── columns: upsert_a:19 upsert_b:20 │ │ └── mapping: - │ │ ├── upsert_a:14 => upsert_a:16 - │ │ └── upsert_b:15 => upsert_b:17 + │ │ ├── upsert_a:17 => upsert_a:19 + │ │ └── upsert_b:18 => upsert_b:20 │ ├── scan self - │ │ └── columns: self.b:19!null self.d:21 + │ │ └── columns: self.b:22!null self.d:24 │ └── filters - │ ├── upsert_a:16 = self.b:19 - │ └── upsert_b:17 = self.d:21 + │ ├── upsert_a:19 = self.b:22 + │ └── upsert_b:20 = self.d:24 ├── f-k-checks-item: self(d) -> self(c) │ └── anti-join (hash) - │ ├── columns: w:22!null + │ ├── columns: w:26!null │ ├── select - │ │ ├── columns: w:22!null + │ │ ├── columns: w:26!null │ │ ├── with-scan &1 - │ │ │ ├── columns: w:22 + │ │ │ ├── columns: w:26 │ │ │ └── mapping: - │ │ │ └── xyzw.w:8 => w:22 + │ │ │ └── xyzw.w:9 => w:26 │ │ └── filters - │ │ └── w:22 IS NOT NULL + │ │ └── w:26 IS NOT NULL │ ├── scan self - │ │ └── columns: self.c:25 + │ │ └── columns: self.c:29 │ └── filters - │ └── w:22 = self.c:25 + │ └── w:26 = self.c:29 ├── f-k-checks-item: self(a,b) -> self(b,d) │ └── semi-join (hash) - │ ├── columns: b:27 d:28 + │ ├── columns: b:32 d:33 │ ├── except - │ │ ├── columns: b:27 d:28 - │ │ ├── left columns: b:27 d:28 - │ │ ├── right columns: upsert_b:29 w:30 + │ │ ├── columns: b:32 d:33 + │ │ ├── left columns: b:32 d:33 + │ │ ├── right columns: upsert_b:34 w:35 │ │ ├── with-scan &1 - │ │ │ ├── columns: b:27 d:28 + │ │ │ ├── columns: b:32 d:33 │ │ │ └── mapping: - │ │ │ ├── self.b:11 => b:27 - │ │ │ └── self.d:13 => d:28 + │ │ │ ├── self.b:13 => b:32 + │ │ │ └── self.d:15 => d:33 │ │ └── with-scan &1 - │ │ ├── columns: upsert_b:29 w:30 + │ │ ├── columns: upsert_b:34 w:35 │ │ └── mapping: - │ │ ├── upsert_b:15 => upsert_b:29 - │ │ └── xyzw.w:8 => w:30 + │ │ ├── upsert_b:18 => upsert_b:34 + │ │ └── xyzw.w:9 => w:35 │ ├── scan self - │ │ └── columns: a:31!null self.b:32!null + │ │ └── columns: a:36!null self.b:37!null │ └── filters - │ ├── b:27 = a:31 - │ └── d:28 = self.b:32 + │ ├── b:32 = a:36 + │ └── d:33 = self.b:37 └── f-k-checks-item: self(d) -> self(c) └── semi-join (hash) - ├── columns: c:35 + ├── columns: c:41 ├── except - │ ├── columns: c:35 - │ ├── left columns: c:35 - │ ├── right columns: z:36 + │ ├── columns: c:41 + │ ├── left columns: c:41 + │ ├── right columns: z:42 │ ├── with-scan &1 - │ │ ├── columns: c:35 + │ │ ├── columns: c:41 │ │ └── mapping: - │ │ └── self.c:12 => c:35 + │ │ └── self.c:14 => c:41 │ └── with-scan &1 - │ ├── columns: z:36 + │ ├── columns: z:42 │ └── mapping: - │ └── xyzw.z:7 => z:36 + │ └── xyzw.z:8 => z:42 ├── scan self - │ └── columns: self.d:40 + │ └── columns: self.d:46 └── filters - └── c:35 = self.d:40 + └── c:41 = self.d:46 diff --git a/pkg/sql/opt/optbuilder/testdata/fk-on-delete-cascade b/pkg/sql/opt/optbuilder/testdata/fk-on-delete-cascade index a3fb83c5e1f7..1466f69e5fa2 100644 --- a/pkg/sql/opt/optbuilder/testdata/fk-on-delete-cascade +++ b/pkg/sql/opt/optbuilder/testdata/fk-on-delete-cascade @@ -13,30 +13,30 @@ DELETE FROM parent WHERE p > 1 root ├── delete parent │ ├── columns: - │ ├── fetch columns: p:2 + │ ├── fetch columns: p:3 │ ├── input binding: &1 │ ├── cascades │ │ └── fk_p_ref_parent │ └── select - │ ├── columns: p:2!null + │ ├── columns: p:3!null crdb_internal_mvcc_timestamp:4 │ ├── scan parent - │ │ └── columns: p:2!null + │ │ └── columns: p:3!null crdb_internal_mvcc_timestamp:4 │ └── filters - │ └── p:2 > 1 + │ └── p:3 > 1 └── cascade └── delete child ├── columns: - ├── fetch columns: c:5 child.p:6 + ├── fetch columns: c:8 child.p:9 └── semi-join (hash) - ├── columns: c:5!null child.p:6!null + ├── columns: c:8!null child.p:9!null child.crdb_internal_mvcc_timestamp:10 ├── scan child - │ └── columns: c:5!null child.p:6!null + │ └── columns: c:8!null child.p:9!null child.crdb_internal_mvcc_timestamp:10 ├── with-scan &1 - │ ├── columns: p:7!null + │ ├── columns: p:11!null │ └── mapping: - │ └── parent.p:2 => p:7 + │ └── parent.p:3 => p:11 └── filters - └── child.p:6 = p:7 + └── child.p:9 = p:11 exec-ddl CREATE TABLE grandchild (g INT PRIMARY KEY, c INT REFERENCES child(c) ON DELETE CASCADE) @@ -49,47 +49,47 @@ DELETE FROM parent WHERE p > 1 root ├── delete parent │ ├── columns: - │ ├── fetch columns: p:2 + │ ├── fetch columns: p:3 │ ├── input binding: &1 │ ├── cascades │ │ └── fk_p_ref_parent │ └── select - │ ├── columns: p:2!null + │ ├── columns: p:3!null crdb_internal_mvcc_timestamp:4 │ ├── scan parent - │ │ └── columns: p:2!null + │ │ └── columns: p:3!null crdb_internal_mvcc_timestamp:4 │ └── filters - │ └── p:2 > 1 + │ └── p:3 > 1 └── cascade ├── delete child │ ├── columns: - │ ├── fetch columns: c:5 child.p:6 + │ ├── fetch columns: c:8 child.p:9 │ ├── input binding: &2 │ ├── cascades │ │ └── fk_c_ref_child │ └── semi-join (hash) - │ ├── columns: c:5!null child.p:6!null + │ ├── columns: c:8!null child.p:9!null child.crdb_internal_mvcc_timestamp:10 │ ├── scan child - │ │ └── columns: c:5!null child.p:6!null + │ │ └── columns: c:8!null child.p:9!null child.crdb_internal_mvcc_timestamp:10 │ ├── with-scan &1 - │ │ ├── columns: p:7!null + │ │ ├── columns: p:11!null │ │ └── mapping: - │ │ └── parent.p:2 => p:7 + │ │ └── parent.p:3 => p:11 │ └── filters - │ └── child.p:6 = p:7 + │ └── child.p:9 = p:11 └── cascade └── delete grandchild ├── columns: - ├── fetch columns: g:10 grandchild.c:11 + ├── fetch columns: g:15 grandchild.c:16 └── semi-join (hash) - ├── columns: g:10!null grandchild.c:11 + ├── columns: g:15!null grandchild.c:16 grandchild.crdb_internal_mvcc_timestamp:17 ├── scan grandchild - │ └── columns: g:10!null grandchild.c:11 + │ └── columns: g:15!null grandchild.c:16 grandchild.crdb_internal_mvcc_timestamp:17 ├── with-scan &2 - │ ├── columns: c:12!null + │ ├── columns: c:18!null │ └── mapping: - │ └── child.c:5 => c:12 + │ └── child.c:8 => c:18 └── filters - └── grandchild.c:11 = c:12 + └── grandchild.c:16 = c:18 # Cascade with check query. exec-ddl @@ -106,43 +106,43 @@ DELETE FROM parent WHERE p > 1 root ├── delete parent │ ├── columns: - │ ├── fetch columns: p:2 + │ ├── fetch columns: p:3 │ ├── input binding: &1 │ ├── cascades │ │ └── fk_p_ref_parent │ └── select - │ ├── columns: p:2!null + │ ├── columns: p:3!null crdb_internal_mvcc_timestamp:4 │ ├── scan parent - │ │ └── columns: p:2!null + │ │ └── columns: p:3!null crdb_internal_mvcc_timestamp:4 │ └── filters - │ └── p:2 > 1 + │ └── p:3 > 1 └── cascade └── delete child ├── columns: - ├── fetch columns: child.c:5 child.p:6 + ├── fetch columns: child.c:8 child.p:9 ├── input binding: &2 ├── semi-join (hash) - │ ├── columns: child.c:5!null child.p:6!null + │ ├── columns: child.c:8!null child.p:9!null child.crdb_internal_mvcc_timestamp:10 │ ├── scan child - │ │ └── columns: child.c:5!null child.p:6!null + │ │ └── columns: child.c:8!null child.p:9!null child.crdb_internal_mvcc_timestamp:10 │ ├── with-scan &1 - │ │ ├── columns: p:7!null + │ │ ├── columns: p:11!null │ │ └── mapping: - │ │ └── parent.p:2 => p:7 + │ │ └── parent.p:3 => p:11 │ └── filters - │ └── child.p:6 = p:7 + │ └── child.p:9 = p:11 └── f-k-checks └── f-k-checks-item: grandchild(c) -> child(c) └── semi-join (hash) - ├── columns: c:8!null + ├── columns: c:12!null ├── with-scan &2 - │ ├── columns: c:8!null + │ ├── columns: c:12!null │ └── mapping: - │ └── child.c:5 => c:8 + │ └── child.c:8 => c:12 ├── scan grandchild - │ └── columns: grandchild.c:10 + │ └── columns: grandchild.c:14 └── filters - └── c:8 = grandchild.c:10 + └── c:12 = grandchild.c:14 # Self-reference with cascade. exec-ddl @@ -155,67 +155,67 @@ DELETE FROM self WHERE a=1 root ├── delete self │ ├── columns: - │ ├── fetch columns: a:3 b:4 + │ ├── fetch columns: a:4 b:5 │ ├── input binding: &1 │ ├── cascades │ │ └── fk_b_ref_self │ └── select - │ ├── columns: a:3!null b:4 + │ ├── columns: a:4!null b:5 crdb_internal_mvcc_timestamp:6 │ ├── scan self - │ │ └── columns: a:3!null b:4 + │ │ └── columns: a:4!null b:5 crdb_internal_mvcc_timestamp:6 │ └── filters - │ └── a:3 = 1 + │ └── a:4 = 1 └── cascade ├── delete self │ ├── columns: - │ ├── fetch columns: self.a:7 b:8 + │ ├── fetch columns: self.a:10 b:11 │ ├── input binding: &2 │ ├── cascades │ │ └── fk_b_ref_self │ └── semi-join (hash) - │ ├── columns: self.a:7!null b:8 + │ ├── columns: self.a:10!null b:11 crdb_internal_mvcc_timestamp:12 │ ├── scan self - │ │ └── columns: self.a:7!null b:8 + │ │ └── columns: self.a:10!null b:11 crdb_internal_mvcc_timestamp:12 │ ├── with-scan &1 - │ │ ├── columns: a:9!null + │ │ ├── columns: a:13!null │ │ └── mapping: - │ │ └── self.a:3 => a:9 + │ │ └── self.a:4 => a:13 │ └── filters - │ └── b:8 = a:9 + │ └── b:11 = a:13 └── cascade ├── delete self │ ├── columns: - │ ├── fetch columns: self.a:12 b:13 + │ ├── fetch columns: self.a:17 b:18 │ ├── input binding: &3 │ ├── cascades │ │ └── fk_b_ref_self │ └── semi-join (hash) - │ ├── columns: self.a:12!null b:13 + │ ├── columns: self.a:17!null b:18 crdb_internal_mvcc_timestamp:19 │ ├── scan self - │ │ └── columns: self.a:12!null b:13 + │ │ └── columns: self.a:17!null b:18 crdb_internal_mvcc_timestamp:19 │ ├── with-scan &2 - │ │ ├── columns: a:14!null + │ │ ├── columns: a:20!null │ │ └── mapping: - │ │ └── self.a:7 => a:14 + │ │ └── self.a:10 => a:20 │ └── filters - │ └── b:13 = a:14 + │ └── b:18 = a:20 └── cascade └── delete self ├── columns: - ├── fetch columns: self.a:17 b:18 + ├── fetch columns: self.a:24 b:25 ├── input binding: &4 ├── cascades │ └── fk_b_ref_self └── semi-join (hash) - ├── columns: self.a:17!null b:18 + ├── columns: self.a:24!null b:25 crdb_internal_mvcc_timestamp:26 ├── scan self - │ └── columns: self.a:17!null b:18 + │ └── columns: self.a:24!null b:25 crdb_internal_mvcc_timestamp:26 ├── with-scan &3 - │ ├── columns: a:19!null + │ ├── columns: a:27!null │ └── mapping: - │ └── self.a:12 => a:19 + │ └── self.a:17 => a:27 └── filters - └── b:18 = a:19 + └── b:25 = a:27 # Cascade cycle. exec-ddl @@ -248,64 +248,64 @@ DELETE FROM ab WHERE a = 1 root ├── delete ab │ ├── columns: - │ ├── fetch columns: a:3 b:4 + │ ├── fetch columns: a:4 b:5 │ ├── input binding: &1 │ ├── cascades │ │ └── ef_ab │ └── select - │ ├── columns: a:3!null b:4 + │ ├── columns: a:4!null b:5 crdb_internal_mvcc_timestamp:6 │ ├── scan ab - │ │ └── columns: a:3!null b:4 + │ │ └── columns: a:4!null b:5 crdb_internal_mvcc_timestamp:6 │ └── filters - │ └── a:3 = 1 + │ └── a:4 = 1 └── cascade ├── delete ef │ ├── columns: - │ ├── fetch columns: e:7 f:8 + │ ├── fetch columns: e:10 f:11 │ ├── input binding: &2 │ ├── cascades │ │ └── cd_ef │ └── semi-join (hash) - │ ├── columns: e:7!null f:8 + │ ├── columns: e:10!null f:11 ef.crdb_internal_mvcc_timestamp:12 │ ├── scan ef - │ │ └── columns: e:7!null f:8 + │ │ └── columns: e:10!null f:11 ef.crdb_internal_mvcc_timestamp:12 │ ├── with-scan &1 - │ │ ├── columns: a:9!null + │ │ ├── columns: a:13!null │ │ └── mapping: - │ │ └── ab.a:3 => a:9 + │ │ └── ab.a:4 => a:13 │ └── filters - │ └── f:8 = a:9 + │ └── f:11 = a:13 └── cascade ├── delete cd │ ├── columns: - │ ├── fetch columns: c:12 d:13 + │ ├── fetch columns: c:17 d:18 │ ├── input binding: &3 │ ├── cascades │ │ └── ab_cd │ └── semi-join (hash) - │ ├── columns: c:12!null d:13 + │ ├── columns: c:17!null d:18 cd.crdb_internal_mvcc_timestamp:19 │ ├── scan cd - │ │ └── columns: c:12!null d:13 + │ │ └── columns: c:17!null d:18 cd.crdb_internal_mvcc_timestamp:19 │ ├── with-scan &2 - │ │ ├── columns: e:14!null + │ │ ├── columns: e:20!null │ │ └── mapping: - │ │ └── ef.e:7 => e:14 + │ │ └── ef.e:10 => e:20 │ └── filters - │ └── d:13 = e:14 + │ └── d:18 = e:20 └── cascade └── delete ab ├── columns: - ├── fetch columns: ab.a:17 b:18 + ├── fetch columns: ab.a:24 b:25 ├── input binding: &4 ├── cascades │ └── ef_ab └── semi-join (hash) - ├── columns: ab.a:17!null b:18 + ├── columns: ab.a:24!null b:25 ab.crdb_internal_mvcc_timestamp:26 ├── scan ab - │ └── columns: ab.a:17!null b:18 + │ └── columns: ab.a:24!null b:25 ab.crdb_internal_mvcc_timestamp:26 ├── with-scan &3 - │ ├── columns: c:19!null + │ ├── columns: c:27!null │ └── mapping: - │ └── cd.c:12 => c:19 + │ └── cd.c:17 => c:27 └── filters - └── b:18 = c:19 + └── b:25 = c:27 diff --git a/pkg/sql/opt/optbuilder/testdata/fk-on-delete-set-default b/pkg/sql/opt/optbuilder/testdata/fk-on-delete-set-default index ff8e1503d5fe..00fd1e587ef2 100644 --- a/pkg/sql/opt/optbuilder/testdata/fk-on-delete-set-default +++ b/pkg/sql/opt/optbuilder/testdata/fk-on-delete-set-default @@ -12,49 +12,49 @@ DELETE FROM parent WHERE p > 1 root ├── delete parent │ ├── columns: - │ ├── fetch columns: p:2 + │ ├── fetch columns: p:3 │ ├── input binding: &1 │ ├── cascades │ │ └── fk_p_ref_parent │ └── select - │ ├── columns: p:2!null + │ ├── columns: p:3!null crdb_internal_mvcc_timestamp:4 │ ├── scan parent - │ │ └── columns: p:2!null + │ │ └── columns: p:3!null crdb_internal_mvcc_timestamp:4 │ └── filters - │ └── p:2 > 1 + │ └── p:3 > 1 └── cascade └── update child ├── columns: - ├── fetch columns: c:5 child.p:6 + ├── fetch columns: c:8 child.p:9 ├── update-mapping: - │ └── p_new:8 => child.p:4 + │ └── p_new:12 => child.p:6 ├── input binding: &2 ├── project - │ ├── columns: p_new:8!null c:5!null child.p:6 + │ ├── columns: p_new:12!null c:8!null child.p:9 child.crdb_internal_mvcc_timestamp:10 │ ├── semi-join (hash) - │ │ ├── columns: c:5!null child.p:6 + │ │ ├── columns: c:8!null child.p:9 child.crdb_internal_mvcc_timestamp:10 │ │ ├── scan child - │ │ │ └── columns: c:5!null child.p:6 + │ │ │ └── columns: c:8!null child.p:9 child.crdb_internal_mvcc_timestamp:10 │ │ ├── with-scan &1 - │ │ │ ├── columns: p:7!null + │ │ │ ├── columns: p:11!null │ │ │ └── mapping: - │ │ │ └── parent.p:2 => p:7 + │ │ │ └── parent.p:3 => p:11 │ │ └── filters - │ │ └── child.p:6 = p:7 + │ │ └── child.p:9 = p:11 │ └── projections - │ └── 0 [as=p_new:8] + │ └── 0 [as=p_new:12] └── f-k-checks └── f-k-checks-item: child(p) -> parent(p) └── anti-join (hash) - ├── columns: p_new:9!null + ├── columns: p_new:13!null ├── with-scan &2 - │ ├── columns: p_new:9!null + │ ├── columns: p_new:13!null │ └── mapping: - │ └── p_new:8 => p_new:9 + │ └── p_new:12 => p_new:13 ├── scan parent - │ └── columns: parent.p:10!null + │ └── columns: parent.p:14!null └── filters - └── p_new:9 = parent.p:10 + └── p_new:13 = parent.p:14 exec-ddl DROP TABLE child @@ -72,36 +72,36 @@ DELETE FROM parent WHERE p > 1 root ├── delete parent │ ├── columns: - │ ├── fetch columns: p:2 + │ ├── fetch columns: p:3 │ ├── input binding: &1 │ ├── cascades │ │ └── fk_p_ref_parent │ └── select - │ ├── columns: p:2!null + │ ├── columns: p:3!null crdb_internal_mvcc_timestamp:4 │ ├── scan parent - │ │ └── columns: p:2!null + │ │ └── columns: p:3!null crdb_internal_mvcc_timestamp:4 │ └── filters - │ └── p:2 > 1 + │ └── p:3 > 1 └── cascade └── update child_null ├── columns: - ├── fetch columns: c:5 child_null.p:6 + ├── fetch columns: c:8 child_null.p:9 ├── update-mapping: - │ └── p_new:8 => child_null.p:4 + │ └── p_new:12 => child_null.p:6 └── project - ├── columns: p_new:8 c:5!null child_null.p:6 + ├── columns: p_new:12 c:8!null child_null.p:9 child_null.crdb_internal_mvcc_timestamp:10 ├── semi-join (hash) - │ ├── columns: c:5!null child_null.p:6 + │ ├── columns: c:8!null child_null.p:9 child_null.crdb_internal_mvcc_timestamp:10 │ ├── scan child_null - │ │ └── columns: c:5!null child_null.p:6 + │ │ └── columns: c:8!null child_null.p:9 child_null.crdb_internal_mvcc_timestamp:10 │ ├── with-scan &1 - │ │ ├── columns: p:7!null + │ │ ├── columns: p:11!null │ │ └── mapping: - │ │ └── parent.p:2 => p:7 + │ │ └── parent.p:3 => p:11 │ └── filters - │ └── child_null.p:6 = p:7 + │ └── child_null.p:9 = p:11 └── projections - └── NULL::INT8 [as=p_new:8] + └── NULL::INT8 [as=p_new:12] exec-ddl CREATE TABLE parent_multicol (p INT, q INT, r INT, PRIMARY KEY (p, q, r)) @@ -128,75 +128,75 @@ DELETE FROM parent_multicol WHERE p > 1 root ├── delete parent_multicol │ ├── columns: - │ ├── fetch columns: p:4 q:5 r:6 + │ ├── fetch columns: p:5 q:6 r:7 │ ├── input binding: &1 │ ├── cascades │ │ └── fk │ └── select - │ ├── columns: p:4!null q:5!null r:6!null + │ ├── columns: p:5!null q:6!null r:7!null crdb_internal_mvcc_timestamp:8 │ ├── scan parent_multicol - │ │ └── columns: p:4!null q:5!null r:6!null + │ │ └── columns: p:5!null q:6!null r:7!null crdb_internal_mvcc_timestamp:8 │ └── filters - │ └── p:4 > 1 + │ └── p:5 > 1 └── cascade └── update child_multicol ├── columns: - ├── fetch columns: child_multicol.c:12 child_multicol.p:13 child_multicol.q:14 child_multicol.r:15 x:16 + ├── fetch columns: child_multicol.c:15 child_multicol.p:16 child_multicol.q:17 child_multicol.r:18 x:19 ├── update-mapping: - │ ├── child_multicol.c:12 => child_multicol.p:8 - │ ├── q_new:20 => child_multicol.q:9 - │ ├── r_new:21 => child_multicol.r:10 - │ └── column22:22 => x:11 - ├── check columns: check1:23 + │ ├── child_multicol.c:15 => child_multicol.p:10 + │ ├── q_new:24 => child_multicol.q:11 + │ ├── r_new:25 => child_multicol.r:12 + │ └── column26:26 => x:13 + ├── check columns: check1:27 ├── input binding: &2 ├── project - │ ├── columns: check1:23!null child_multicol.c:12!null child_multicol.p:13 child_multicol.q:14 child_multicol.r:15 x:16 q_new:20 r_new:21 column22:22 + │ ├── columns: check1:27!null child_multicol.c:15!null child_multicol.p:16 child_multicol.q:17 child_multicol.r:18 x:19 child_multicol.crdb_internal_mvcc_timestamp:20 q_new:24 r_new:25 column26:26 │ ├── project - │ │ ├── columns: column22:22 child_multicol.c:12!null child_multicol.p:13 child_multicol.q:14 child_multicol.r:15 x:16 q_new:20 r_new:21 + │ │ ├── columns: column26:26 child_multicol.c:15!null child_multicol.p:16 child_multicol.q:17 child_multicol.r:18 x:19 child_multicol.crdb_internal_mvcc_timestamp:20 q_new:24 r_new:25 │ │ ├── project - │ │ │ ├── columns: q_new:20 r_new:21 child_multicol.c:12!null child_multicol.p:13 child_multicol.q:14 child_multicol.r:15 x:16 + │ │ │ ├── columns: q_new:24 r_new:25 child_multicol.c:15!null child_multicol.p:16 child_multicol.q:17 child_multicol.r:18 x:19 child_multicol.crdb_internal_mvcc_timestamp:20 │ │ │ ├── semi-join (hash) - │ │ │ │ ├── columns: child_multicol.c:12!null child_multicol.p:13 child_multicol.q:14 child_multicol.r:15 x:16 + │ │ │ │ ├── columns: child_multicol.c:15!null child_multicol.p:16 child_multicol.q:17 child_multicol.r:18 x:19 child_multicol.crdb_internal_mvcc_timestamp:20 │ │ │ │ ├── scan child_multicol - │ │ │ │ │ ├── columns: child_multicol.c:12!null child_multicol.p:13 child_multicol.q:14 child_multicol.r:15 x:16 + │ │ │ │ │ ├── columns: child_multicol.c:15!null child_multicol.p:16 child_multicol.q:17 child_multicol.r:18 x:19 child_multicol.crdb_internal_mvcc_timestamp:20 │ │ │ │ │ └── computed column expressions - │ │ │ │ │ └── x:16 - │ │ │ │ │ └── (child_multicol.p:13 + child_multicol.q:14) + child_multicol.r:15 + │ │ │ │ │ └── x:19 + │ │ │ │ │ └── (child_multicol.p:16 + child_multicol.q:17) + child_multicol.r:18 │ │ │ │ ├── with-scan &1 - │ │ │ │ │ ├── columns: p:17!null q:18!null r:19!null + │ │ │ │ │ ├── columns: p:21!null q:22!null r:23!null │ │ │ │ │ └── mapping: - │ │ │ │ │ ├── parent_multicol.p:4 => p:17 - │ │ │ │ │ ├── parent_multicol.q:5 => q:18 - │ │ │ │ │ └── parent_multicol.r:6 => r:19 + │ │ │ │ │ ├── parent_multicol.p:5 => p:21 + │ │ │ │ │ ├── parent_multicol.q:6 => q:22 + │ │ │ │ │ └── parent_multicol.r:7 => r:23 │ │ │ │ └── filters - │ │ │ │ ├── child_multicol.p:13 = p:17 - │ │ │ │ ├── child_multicol.q:14 = q:18 - │ │ │ │ └── child_multicol.r:15 = r:19 + │ │ │ │ ├── child_multicol.p:16 = p:21 + │ │ │ │ ├── child_multicol.q:17 = q:22 + │ │ │ │ └── child_multicol.r:18 = r:23 │ │ │ └── projections - │ │ │ ├── child_multicol.p:13 + 1 [as=q_new:20] - │ │ │ └── child_multicol.p:13 + child_multicol.q:14 [as=r_new:21] + │ │ │ ├── child_multicol.p:16 + 1 [as=q_new:24] + │ │ │ └── child_multicol.p:16 + child_multicol.q:17 [as=r_new:25] │ │ └── projections - │ │ └── (child_multicol.c:12 + q_new:20) + r_new:21 [as=column22:22] + │ │ └── (child_multicol.c:15 + q_new:24) + r_new:25 [as=column26:26] │ └── projections - │ └── (child_multicol.c:12 > 100) OR (child_multicol.c:12 > child_multicol.c:12) [as=check1:23] + │ └── (child_multicol.c:15 > 100) OR (child_multicol.c:15 > child_multicol.c:15) [as=check1:27] └── f-k-checks └── f-k-checks-item: child_multicol(p,q,r) -> parent_multicol(p,q,r) └── anti-join (hash) - ├── columns: c:24!null q_new:25!null r_new:26!null + ├── columns: c:28!null q_new:29!null r_new:30!null ├── select - │ ├── columns: c:24!null q_new:25!null r_new:26!null + │ ├── columns: c:28!null q_new:29!null r_new:30!null │ ├── with-scan &2 - │ │ ├── columns: c:24!null q_new:25 r_new:26 + │ │ ├── columns: c:28!null q_new:29 r_new:30 │ │ └── mapping: - │ │ ├── child_multicol.c:12 => c:24 - │ │ ├── q_new:20 => q_new:25 - │ │ └── r_new:21 => r_new:26 + │ │ ├── child_multicol.c:15 => c:28 + │ │ ├── q_new:24 => q_new:29 + │ │ └── r_new:25 => r_new:30 │ └── filters - │ ├── q_new:25 IS NOT NULL - │ └── r_new:26 IS NOT NULL + │ ├── q_new:29 IS NOT NULL + │ └── r_new:30 IS NOT NULL ├── scan parent_multicol - │ └── columns: parent_multicol.p:27!null parent_multicol.q:28!null parent_multicol.r:29!null + │ └── columns: parent_multicol.p:31!null parent_multicol.q:32!null parent_multicol.r:33!null └── filters - ├── c:24 = parent_multicol.p:27 - ├── q_new:25 = parent_multicol.q:28 - └── r_new:26 = parent_multicol.r:29 + ├── c:28 = parent_multicol.p:31 + ├── q_new:29 = parent_multicol.q:32 + └── r_new:30 = parent_multicol.r:33 diff --git a/pkg/sql/opt/optbuilder/testdata/fk-on-delete-set-null b/pkg/sql/opt/optbuilder/testdata/fk-on-delete-set-null index 741233cc09dc..b9fcd5e870e5 100644 --- a/pkg/sql/opt/optbuilder/testdata/fk-on-delete-set-null +++ b/pkg/sql/opt/optbuilder/testdata/fk-on-delete-set-null @@ -12,36 +12,36 @@ DELETE FROM parent WHERE p > 1 root ├── delete parent │ ├── columns: - │ ├── fetch columns: p:2 + │ ├── fetch columns: p:3 │ ├── input binding: &1 │ ├── cascades │ │ └── fk_p_ref_parent │ └── select - │ ├── columns: p:2!null + │ ├── columns: p:3!null crdb_internal_mvcc_timestamp:4 │ ├── scan parent - │ │ └── columns: p:2!null + │ │ └── columns: p:3!null crdb_internal_mvcc_timestamp:4 │ └── filters - │ └── p:2 > 1 + │ └── p:3 > 1 └── cascade └── update child ├── columns: - ├── fetch columns: c:5 child.p:6 + ├── fetch columns: c:8 child.p:9 ├── update-mapping: - │ └── p_new:8 => child.p:4 + │ └── p_new:12 => child.p:6 └── project - ├── columns: p_new:8 c:5!null child.p:6 + ├── columns: p_new:12 c:8!null child.p:9 child.crdb_internal_mvcc_timestamp:10 ├── semi-join (hash) - │ ├── columns: c:5!null child.p:6 + │ ├── columns: c:8!null child.p:9 child.crdb_internal_mvcc_timestamp:10 │ ├── scan child - │ │ └── columns: c:5!null child.p:6 + │ │ └── columns: c:8!null child.p:9 child.crdb_internal_mvcc_timestamp:10 │ ├── with-scan &1 - │ │ ├── columns: p:7!null + │ │ ├── columns: p:11!null │ │ └── mapping: - │ │ └── parent.p:2 => p:7 + │ │ └── parent.p:3 => p:11 │ └── filters - │ └── child.p:6 = p:7 + │ └── child.p:9 = p:11 └── projections - └── NULL::INT8 [as=p_new:8] + └── NULL::INT8 [as=p_new:12] exec-ddl CREATE TABLE parent_multicol (p INT, q INT, r INT, PRIMARY KEY (p, q, r)) @@ -67,54 +67,54 @@ DELETE FROM parent_multicol WHERE p > 1 root ├── delete parent_multicol │ ├── columns: - │ ├── fetch columns: p:4 q:5 r:6 + │ ├── fetch columns: p:5 q:6 r:7 │ ├── input binding: &1 │ ├── cascades │ │ └── fk │ └── select - │ ├── columns: p:4!null q:5!null r:6!null + │ ├── columns: p:5!null q:6!null r:7!null crdb_internal_mvcc_timestamp:8 │ ├── scan parent_multicol - │ │ └── columns: p:4!null q:5!null r:6!null + │ │ └── columns: p:5!null q:6!null r:7!null crdb_internal_mvcc_timestamp:8 │ └── filters - │ └── p:4 > 1 + │ └── p:5 > 1 └── cascade └── update child_multicol ├── columns: - ├── fetch columns: c:12 child_multicol.p:13 child_multicol.q:14 child_multicol.r:15 x:16 + ├── fetch columns: c:15 child_multicol.p:16 child_multicol.q:17 child_multicol.r:18 x:19 ├── update-mapping: - │ ├── p_new:20 => child_multicol.p:8 - │ ├── p_new:20 => child_multicol.q:9 - │ ├── p_new:20 => child_multicol.r:10 - │ └── column21:21 => x:11 - ├── check columns: check1:22 + │ ├── p_new:24 => child_multicol.p:10 + │ ├── p_new:24 => child_multicol.q:11 + │ ├── p_new:24 => child_multicol.r:12 + │ └── column25:25 => x:13 + ├── check columns: check1:26 └── project - ├── columns: check1:22!null c:12!null child_multicol.p:13 child_multicol.q:14 child_multicol.r:15 x:16 p_new:20 column21:21 + ├── columns: check1:26!null c:15!null child_multicol.p:16 child_multicol.q:17 child_multicol.r:18 x:19 child_multicol.crdb_internal_mvcc_timestamp:20 p_new:24 column25:25 ├── project - │ ├── columns: column21:21 c:12!null child_multicol.p:13 child_multicol.q:14 child_multicol.r:15 x:16 p_new:20 + │ ├── columns: column25:25 c:15!null child_multicol.p:16 child_multicol.q:17 child_multicol.r:18 x:19 child_multicol.crdb_internal_mvcc_timestamp:20 p_new:24 │ ├── project - │ │ ├── columns: p_new:20 c:12!null child_multicol.p:13 child_multicol.q:14 child_multicol.r:15 x:16 + │ │ ├── columns: p_new:24 c:15!null child_multicol.p:16 child_multicol.q:17 child_multicol.r:18 x:19 child_multicol.crdb_internal_mvcc_timestamp:20 │ │ ├── semi-join (hash) - │ │ │ ├── columns: c:12!null child_multicol.p:13 child_multicol.q:14 child_multicol.r:15 x:16 + │ │ │ ├── columns: c:15!null child_multicol.p:16 child_multicol.q:17 child_multicol.r:18 x:19 child_multicol.crdb_internal_mvcc_timestamp:20 │ │ │ ├── scan child_multicol - │ │ │ │ ├── columns: c:12!null child_multicol.p:13 child_multicol.q:14 child_multicol.r:15 x:16 + │ │ │ │ ├── columns: c:15!null child_multicol.p:16 child_multicol.q:17 child_multicol.r:18 x:19 child_multicol.crdb_internal_mvcc_timestamp:20 │ │ │ │ ├── check constraint expressions - │ │ │ │ │ └── (c:12 > 100) OR (child_multicol.p:13 IS NOT NULL) + │ │ │ │ │ └── (c:15 > 100) OR (child_multicol.p:16 IS NOT NULL) │ │ │ │ └── computed column expressions - │ │ │ │ └── x:16 - │ │ │ │ └── (child_multicol.p:13 + child_multicol.q:14) + child_multicol.r:15 + │ │ │ │ └── x:19 + │ │ │ │ └── (child_multicol.p:16 + child_multicol.q:17) + child_multicol.r:18 │ │ │ ├── with-scan &1 - │ │ │ │ ├── columns: p:17!null q:18!null r:19!null + │ │ │ │ ├── columns: p:21!null q:22!null r:23!null │ │ │ │ └── mapping: - │ │ │ │ ├── parent_multicol.p:4 => p:17 - │ │ │ │ ├── parent_multicol.q:5 => q:18 - │ │ │ │ └── parent_multicol.r:6 => r:19 + │ │ │ │ ├── parent_multicol.p:5 => p:21 + │ │ │ │ ├── parent_multicol.q:6 => q:22 + │ │ │ │ └── parent_multicol.r:7 => r:23 │ │ │ └── filters - │ │ │ ├── child_multicol.p:13 = p:17 - │ │ │ ├── child_multicol.q:14 = q:18 - │ │ │ └── child_multicol.r:15 = r:19 + │ │ │ ├── child_multicol.p:16 = p:21 + │ │ │ ├── child_multicol.q:17 = q:22 + │ │ │ └── child_multicol.r:18 = r:23 │ │ └── projections - │ │ └── NULL::INT8 [as=p_new:20] + │ │ └── NULL::INT8 [as=p_new:24] │ └── projections - │ └── (p_new:20 + p_new:20) + p_new:20 [as=column21:21] + │ └── (p_new:24 + p_new:24) + p_new:24 [as=column25:25] └── projections - └── (c:12 > 100) OR (p_new:20 IS NOT NULL) [as=check1:22] + └── (c:15 > 100) OR (p_new:24 IS NOT NULL) [as=check1:26] diff --git a/pkg/sql/opt/optbuilder/testdata/fk-on-update-cascade b/pkg/sql/opt/optbuilder/testdata/fk-on-update-cascade index b8feb03d5433..caebddab3521 100644 --- a/pkg/sql/opt/optbuilder/testdata/fk-on-update-cascade +++ b/pkg/sql/opt/optbuilder/testdata/fk-on-update-cascade @@ -12,56 +12,56 @@ UPDATE parent SET p = p * 10 WHERE p > 1 root ├── update parent │ ├── columns: - │ ├── fetch columns: p:2 + │ ├── fetch columns: p:3 │ ├── update-mapping: - │ │ └── p_new:3 => p:1 + │ │ └── p_new:5 => p:1 │ ├── input binding: &1 │ ├── cascades │ │ └── fk_p_ref_parent │ └── project - │ ├── columns: p_new:3!null p:2!null + │ ├── columns: p_new:5!null p:3!null crdb_internal_mvcc_timestamp:4 │ ├── select - │ │ ├── columns: p:2!null + │ │ ├── columns: p:3!null crdb_internal_mvcc_timestamp:4 │ │ ├── scan parent - │ │ │ └── columns: p:2!null + │ │ │ └── columns: p:3!null crdb_internal_mvcc_timestamp:4 │ │ └── filters - │ │ └── p:2 > 1 + │ │ └── p:3 > 1 │ └── projections - │ └── p:2 * 10 [as=p_new:3] + │ └── p:3 * 10 [as=p_new:5] └── cascade └── update child ├── columns: - ├── fetch columns: c:6 child.p:7 + ├── fetch columns: c:9 child.p:10 ├── update-mapping: - │ └── p_new:9 => child.p:5 + │ └── p_new:13 => child.p:7 ├── input binding: &2 ├── inner-join (hash) - │ ├── columns: c:6!null child.p:7!null p:8!null p_new:9!null + │ ├── columns: c:9!null child.p:10!null child.crdb_internal_mvcc_timestamp:11 p:12!null p_new:13!null │ ├── scan child - │ │ └── columns: c:6!null child.p:7!null + │ │ └── columns: c:9!null child.p:10!null child.crdb_internal_mvcc_timestamp:11 │ ├── select - │ │ ├── columns: p:8!null p_new:9!null + │ │ ├── columns: p:12!null p_new:13!null │ │ ├── with-scan &1 - │ │ │ ├── columns: p:8!null p_new:9!null + │ │ │ ├── columns: p:12!null p_new:13!null │ │ │ └── mapping: - │ │ │ ├── parent.p:2 => p:8 - │ │ │ └── p_new:3 => p_new:9 + │ │ │ ├── parent.p:3 => p:12 + │ │ │ └── p_new:5 => p_new:13 │ │ └── filters - │ │ └── p:8 IS DISTINCT FROM p_new:9 + │ │ └── p:12 IS DISTINCT FROM p_new:13 │ └── filters - │ └── child.p:7 = p:8 + │ └── child.p:10 = p:12 └── f-k-checks └── f-k-checks-item: child(p) -> parent(p) └── anti-join (hash) - ├── columns: p_new:10!null + ├── columns: p_new:14!null ├── with-scan &2 - │ ├── columns: p_new:10!null + │ ├── columns: p_new:14!null │ └── mapping: - │ └── p_new:9 => p_new:10 + │ └── p_new:13 => p_new:14 ├── scan parent - │ └── columns: parent.p:11!null + │ └── columns: parent.p:15!null └── filters - └── p_new:10 = parent.p:11 + └── p_new:14 = parent.p:15 exec-ddl CREATE TABLE parent_multi ( @@ -89,69 +89,69 @@ UPDATE parent_multi SET p = p * 10, q = q + 1 WHERE pk > 1 root ├── update parent_multi │ ├── columns: - │ ├── fetch columns: pk:4 p:5 q:6 + │ ├── fetch columns: pk:5 p:6 q:7 │ ├── update-mapping: - │ │ ├── p_new:7 => p:2 - │ │ └── q_new:8 => q:3 + │ │ ├── p_new:9 => p:2 + │ │ └── q_new:10 => q:3 │ ├── input binding: &1 │ ├── cascades │ │ └── fk │ └── project - │ ├── columns: p_new:7 q_new:8 pk:4!null p:5 q:6 + │ ├── columns: p_new:9 q_new:10 pk:5!null p:6 q:7 crdb_internal_mvcc_timestamp:8 │ ├── select - │ │ ├── columns: pk:4!null p:5 q:6 + │ │ ├── columns: pk:5!null p:6 q:7 crdb_internal_mvcc_timestamp:8 │ │ ├── scan parent_multi - │ │ │ └── columns: pk:4!null p:5 q:6 + │ │ │ └── columns: pk:5!null p:6 q:7 crdb_internal_mvcc_timestamp:8 │ │ └── filters - │ │ └── pk:4 > 1 + │ │ └── pk:5 > 1 │ └── projections - │ ├── p:5 * 10 [as=p_new:7] - │ └── q:6 + 1 [as=q_new:8] + │ ├── p:6 * 10 [as=p_new:9] + │ └── q:7 + 1 [as=q_new:10] └── cascade └── update child_multi ├── columns: - ├── fetch columns: c:12 child_multi.p:13 child_multi.q:14 + ├── fetch columns: c:15 child_multi.p:16 child_multi.q:17 ├── update-mapping: - │ ├── p_new:17 => child_multi.p:10 - │ └── q_new:18 => child_multi.q:11 + │ ├── p_new:21 => child_multi.p:12 + │ └── q_new:22 => child_multi.q:13 ├── input binding: &2 ├── inner-join (hash) - │ ├── columns: c:12!null child_multi.p:13!null child_multi.q:14!null p:15!null q:16!null p_new:17 q_new:18 + │ ├── columns: c:15!null child_multi.p:16!null child_multi.q:17!null child_multi.crdb_internal_mvcc_timestamp:18 p:19!null q:20!null p_new:21 q_new:22 │ ├── scan child_multi - │ │ └── columns: c:12!null child_multi.p:13 child_multi.q:14 + │ │ └── columns: c:15!null child_multi.p:16 child_multi.q:17 child_multi.crdb_internal_mvcc_timestamp:18 │ ├── select - │ │ ├── columns: p:15 q:16 p_new:17 q_new:18 + │ │ ├── columns: p:19 q:20 p_new:21 q_new:22 │ │ ├── with-scan &1 - │ │ │ ├── columns: p:15 q:16 p_new:17 q_new:18 + │ │ │ ├── columns: p:19 q:20 p_new:21 q_new:22 │ │ │ └── mapping: - │ │ │ ├── parent_multi.p:5 => p:15 - │ │ │ ├── parent_multi.q:6 => q:16 - │ │ │ ├── p_new:7 => p_new:17 - │ │ │ └── q_new:8 => q_new:18 + │ │ │ ├── parent_multi.p:6 => p:19 + │ │ │ ├── parent_multi.q:7 => q:20 + │ │ │ ├── p_new:9 => p_new:21 + │ │ │ └── q_new:10 => q_new:22 │ │ └── filters - │ │ └── (p:15 IS DISTINCT FROM p_new:17) OR (q:16 IS DISTINCT FROM q_new:18) + │ │ └── (p:19 IS DISTINCT FROM p_new:21) OR (q:20 IS DISTINCT FROM q_new:22) │ └── filters - │ ├── child_multi.p:13 = p:15 - │ └── child_multi.q:14 = q:16 + │ ├── child_multi.p:16 = p:19 + │ └── child_multi.q:17 = q:20 └── f-k-checks └── f-k-checks-item: child_multi(p,q) -> parent_multi(p,q) └── anti-join (hash) - ├── columns: p_new:19!null q_new:20!null + ├── columns: p_new:23!null q_new:24!null ├── select - │ ├── columns: p_new:19!null q_new:20!null + │ ├── columns: p_new:23!null q_new:24!null │ ├── with-scan &2 - │ │ ├── columns: p_new:19 q_new:20 + │ │ ├── columns: p_new:23 q_new:24 │ │ └── mapping: - │ │ ├── p_new:17 => p_new:19 - │ │ └── q_new:18 => q_new:20 + │ │ ├── p_new:21 => p_new:23 + │ │ └── q_new:22 => q_new:24 │ └── filters - │ ├── p_new:19 IS NOT NULL - │ └── q_new:20 IS NOT NULL + │ ├── p_new:23 IS NOT NULL + │ └── q_new:24 IS NOT NULL ├── scan parent_multi - │ └── columns: parent_multi.p:22 parent_multi.q:23 + │ └── columns: parent_multi.p:26 parent_multi.q:27 └── filters - ├── p_new:19 = parent_multi.p:22 - └── q_new:20 = parent_multi.q:23 + ├── p_new:23 = parent_multi.p:26 + └── q_new:24 = parent_multi.q:27 # Update only one of the two FK columns. The "before" and "after" values of q # come from the same column in the mutation input. @@ -161,66 +161,66 @@ UPDATE parent_multi SET p = p * 10 WHERE p > 1 root ├── update parent_multi │ ├── columns: - │ ├── fetch columns: pk:4 p:5 q:6 + │ ├── fetch columns: pk:5 p:6 q:7 │ ├── update-mapping: - │ │ └── p_new:7 => p:2 + │ │ └── p_new:9 => p:2 │ ├── input binding: &1 │ ├── cascades │ │ └── fk │ └── project - │ ├── columns: p_new:7!null pk:4!null p:5!null q:6 + │ ├── columns: p_new:9!null pk:5!null p:6!null q:7 crdb_internal_mvcc_timestamp:8 │ ├── select - │ │ ├── columns: pk:4!null p:5!null q:6 + │ │ ├── columns: pk:5!null p:6!null q:7 crdb_internal_mvcc_timestamp:8 │ │ ├── scan parent_multi - │ │ │ └── columns: pk:4!null p:5 q:6 + │ │ │ └── columns: pk:5!null p:6 q:7 crdb_internal_mvcc_timestamp:8 │ │ └── filters - │ │ └── p:5 > 1 + │ │ └── p:6 > 1 │ └── projections - │ └── p:5 * 10 [as=p_new:7] + │ └── p:6 * 10 [as=p_new:9] └── cascade └── update child_multi ├── columns: - ├── fetch columns: c:11 child_multi.p:12 child_multi.q:13 + ├── fetch columns: c:14 child_multi.p:15 child_multi.q:16 ├── update-mapping: - │ ├── p_new:16 => child_multi.p:9 - │ └── q:17 => child_multi.q:10 + │ ├── p_new:20 => child_multi.p:11 + │ └── q:21 => child_multi.q:12 ├── input binding: &2 ├── inner-join (hash) - │ ├── columns: c:11!null child_multi.p:12!null child_multi.q:13!null p:14!null q:15!null p_new:16!null q:17 + │ ├── columns: c:14!null child_multi.p:15!null child_multi.q:16!null child_multi.crdb_internal_mvcc_timestamp:17 p:18!null q:19!null p_new:20!null q:21 │ ├── scan child_multi - │ │ └── columns: c:11!null child_multi.p:12 child_multi.q:13 + │ │ └── columns: c:14!null child_multi.p:15 child_multi.q:16 child_multi.crdb_internal_mvcc_timestamp:17 │ ├── select - │ │ ├── columns: p:14!null q:15 p_new:16!null q:17 + │ │ ├── columns: p:18!null q:19 p_new:20!null q:21 │ │ ├── with-scan &1 - │ │ │ ├── columns: p:14!null q:15 p_new:16!null q:17 + │ │ │ ├── columns: p:18!null q:19 p_new:20!null q:21 │ │ │ └── mapping: - │ │ │ ├── parent_multi.p:5 => p:14 - │ │ │ ├── parent_multi.q:6 => q:15 - │ │ │ ├── p_new:7 => p_new:16 - │ │ │ └── parent_multi.q:6 => q:17 + │ │ │ ├── parent_multi.p:6 => p:18 + │ │ │ ├── parent_multi.q:7 => q:19 + │ │ │ ├── p_new:9 => p_new:20 + │ │ │ └── parent_multi.q:7 => q:21 │ │ └── filters - │ │ └── (p:14 IS DISTINCT FROM p_new:16) OR (q:15 IS DISTINCT FROM q:17) + │ │ └── (p:18 IS DISTINCT FROM p_new:20) OR (q:19 IS DISTINCT FROM q:21) │ └── filters - │ ├── child_multi.p:12 = p:14 - │ └── child_multi.q:13 = q:15 + │ ├── child_multi.p:15 = p:18 + │ └── child_multi.q:16 = q:19 └── f-k-checks └── f-k-checks-item: child_multi(p,q) -> parent_multi(p,q) └── anti-join (hash) - ├── columns: p_new:18!null q:19!null + ├── columns: p_new:22!null q:23!null ├── select - │ ├── columns: p_new:18!null q:19!null + │ ├── columns: p_new:22!null q:23!null │ ├── with-scan &2 - │ │ ├── columns: p_new:18!null q:19 + │ │ ├── columns: p_new:22!null q:23 │ │ └── mapping: - │ │ ├── p_new:16 => p_new:18 - │ │ └── q:17 => q:19 + │ │ ├── p_new:20 => p_new:22 + │ │ └── q:21 => q:23 │ └── filters - │ └── q:19 IS NOT NULL + │ └── q:23 IS NOT NULL ├── scan parent_multi - │ └── columns: parent_multi.p:21 parent_multi.q:22 + │ └── columns: parent_multi.p:25 parent_multi.q:26 └── filters - ├── p_new:18 = parent_multi.p:21 - └── q:19 = parent_multi.q:22 + ├── p_new:22 = parent_multi.p:25 + └── q:23 = parent_multi.q:26 build-cascades UPSERT INTO parent_multi VALUES (1, 10, 10), (2, 20, 20) @@ -228,80 +228,80 @@ UPSERT INTO parent_multi VALUES (1, 10, 10), (2, 20, 20) root ├── upsert parent_multi │ ├── columns: - │ ├── canary column: 7 - │ ├── fetch columns: pk:7 p:8 q:9 + │ ├── canary column: 8 + │ ├── fetch columns: pk:8 p:9 q:10 │ ├── insert-mapping: - │ │ ├── column1:4 => pk:1 - │ │ ├── column2:5 => p:2 - │ │ └── column3:6 => q:3 + │ │ ├── column1:5 => pk:1 + │ │ ├── column2:6 => p:2 + │ │ └── column3:7 => q:3 │ ├── update-mapping: - │ │ ├── column2:5 => p:2 - │ │ └── column3:6 => q:3 + │ │ ├── column2:6 => p:2 + │ │ └── column3:7 => q:3 │ ├── input binding: &1 │ ├── cascades │ │ └── fk │ └── project - │ ├── columns: upsert_pk:10 column1:4!null column2:5!null column3:6!null pk:7 p:8 q:9 + │ ├── columns: upsert_pk:12 column1:5!null column2:6!null column3:7!null pk:8 p:9 q:10 crdb_internal_mvcc_timestamp:11 │ ├── left-join (hash) - │ │ ├── columns: column1:4!null column2:5!null column3:6!null pk:7 p:8 q:9 + │ │ ├── columns: column1:5!null column2:6!null column3:7!null pk:8 p:9 q:10 crdb_internal_mvcc_timestamp:11 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null - │ │ │ ├── grouping columns: column1:4!null + │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null + │ │ │ ├── grouping columns: column1:5!null │ │ │ ├── values - │ │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null + │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null │ │ │ │ ├── (1, 10, 10) │ │ │ │ └── (2, 20, 20) │ │ │ └── aggregations - │ │ │ ├── first-agg [as=column2:5] - │ │ │ │ └── column2:5 - │ │ │ └── first-agg [as=column3:6] - │ │ │ └── column3:6 + │ │ │ ├── first-agg [as=column2:6] + │ │ │ │ └── column2:6 + │ │ │ └── first-agg [as=column3:7] + │ │ │ └── column3:7 │ │ ├── scan parent_multi - │ │ │ └── columns: pk:7!null p:8 q:9 + │ │ │ └── columns: pk:8!null p:9 q:10 crdb_internal_mvcc_timestamp:11 │ │ └── filters - │ │ └── column1:4 = pk:7 + │ │ └── column1:5 = pk:8 │ └── projections - │ └── CASE WHEN pk:7 IS NULL THEN column1:4 ELSE pk:7 END [as=upsert_pk:10] + │ └── CASE WHEN pk:8 IS NULL THEN column1:5 ELSE pk:8 END [as=upsert_pk:12] └── cascade └── update child_multi ├── columns: - ├── fetch columns: c:14 child_multi.p:15 child_multi.q:16 + ├── fetch columns: c:17 child_multi.p:18 child_multi.q:19 ├── update-mapping: - │ ├── column2:19 => child_multi.p:12 - │ └── column3:20 => child_multi.q:13 + │ ├── column2:23 => child_multi.p:14 + │ └── column3:24 => child_multi.q:15 ├── input binding: &2 ├── inner-join (hash) - │ ├── columns: c:14!null child_multi.p:15!null child_multi.q:16!null p:17!null q:18!null column2:19!null column3:20!null + │ ├── columns: c:17!null child_multi.p:18!null child_multi.q:19!null child_multi.crdb_internal_mvcc_timestamp:20 p:21!null q:22!null column2:23!null column3:24!null │ ├── scan child_multi - │ │ └── columns: c:14!null child_multi.p:15 child_multi.q:16 + │ │ └── columns: c:17!null child_multi.p:18 child_multi.q:19 child_multi.crdb_internal_mvcc_timestamp:20 │ ├── select - │ │ ├── columns: p:17 q:18 column2:19!null column3:20!null + │ │ ├── columns: p:21 q:22 column2:23!null column3:24!null │ │ ├── with-scan &1 - │ │ │ ├── columns: p:17 q:18 column2:19!null column3:20!null + │ │ │ ├── columns: p:21 q:22 column2:23!null column3:24!null │ │ │ └── mapping: - │ │ │ ├── parent_multi.p:8 => p:17 - │ │ │ ├── parent_multi.q:9 => q:18 - │ │ │ ├── column2:5 => column2:19 - │ │ │ └── column3:6 => column3:20 + │ │ │ ├── parent_multi.p:9 => p:21 + │ │ │ ├── parent_multi.q:10 => q:22 + │ │ │ ├── column2:6 => column2:23 + │ │ │ └── column3:7 => column3:24 │ │ └── filters - │ │ └── (p:17 IS DISTINCT FROM column2:19) OR (q:18 IS DISTINCT FROM column3:20) + │ │ └── (p:21 IS DISTINCT FROM column2:23) OR (q:22 IS DISTINCT FROM column3:24) │ └── filters - │ ├── child_multi.p:15 = p:17 - │ └── child_multi.q:16 = q:18 + │ ├── child_multi.p:18 = p:21 + │ └── child_multi.q:19 = q:22 └── f-k-checks └── f-k-checks-item: child_multi(p,q) -> parent_multi(p,q) └── anti-join (hash) - ├── columns: column2:21!null column3:22!null + ├── columns: column2:25!null column3:26!null ├── with-scan &2 - │ ├── columns: column2:21!null column3:22!null + │ ├── columns: column2:25!null column3:26!null │ └── mapping: - │ ├── column2:19 => column2:21 - │ └── column3:20 => column3:22 + │ ├── column2:23 => column2:25 + │ └── column3:24 => column3:26 ├── scan parent_multi - │ └── columns: parent_multi.p:24 parent_multi.q:25 + │ └── columns: parent_multi.p:28 parent_multi.q:29 └── filters - ├── column2:21 = parent_multi.p:24 - └── column3:22 = parent_multi.q:25 + ├── column2:25 = parent_multi.p:28 + └── column3:26 = parent_multi.q:29 # Upsert that only touches one of the FK columns. build-cascades @@ -310,88 +310,88 @@ UPSERT INTO parent_multi(pk, p) VALUES (1, 10), (2, 20) root ├── upsert parent_multi │ ├── columns: - │ ├── canary column: 7 - │ ├── fetch columns: pk:7 p:8 q:9 + │ ├── canary column: 8 + │ ├── fetch columns: pk:8 p:9 q:10 │ ├── insert-mapping: - │ │ ├── column1:4 => pk:1 - │ │ ├── column2:5 => p:2 - │ │ └── column6:6 => q:3 + │ │ ├── column1:5 => pk:1 + │ │ ├── column2:6 => p:2 + │ │ └── column7:7 => q:3 │ ├── update-mapping: - │ │ └── column2:5 => p:2 + │ │ └── column2:6 => p:2 │ ├── input binding: &1 │ ├── cascades │ │ └── fk │ └── project - │ ├── columns: upsert_pk:10 upsert_q:11 column1:4!null column2:5!null column6:6 pk:7 p:8 q:9 + │ ├── columns: upsert_pk:12 upsert_q:13 column1:5!null column2:6!null column7:7 pk:8 p:9 q:10 crdb_internal_mvcc_timestamp:11 │ ├── left-join (hash) - │ │ ├── columns: column1:4!null column2:5!null column6:6 pk:7 p:8 q:9 + │ │ ├── columns: column1:5!null column2:6!null column7:7 pk:8 p:9 q:10 crdb_internal_mvcc_timestamp:11 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: column1:4!null column2:5!null column6:6 - │ │ │ ├── grouping columns: column1:4!null + │ │ │ ├── columns: column1:5!null column2:6!null column7:7 + │ │ │ ├── grouping columns: column1:5!null │ │ │ ├── project - │ │ │ │ ├── columns: column6:6 column1:4!null column2:5!null + │ │ │ │ ├── columns: column7:7 column1:5!null column2:6!null │ │ │ │ ├── values - │ │ │ │ │ ├── columns: column1:4!null column2:5!null + │ │ │ │ │ ├── columns: column1:5!null column2:6!null │ │ │ │ │ ├── (1, 10) │ │ │ │ │ └── (2, 20) │ │ │ │ └── projections - │ │ │ │ └── NULL::INT8 [as=column6:6] + │ │ │ │ └── NULL::INT8 [as=column7:7] │ │ │ └── aggregations - │ │ │ ├── first-agg [as=column2:5] - │ │ │ │ └── column2:5 - │ │ │ └── first-agg [as=column6:6] - │ │ │ └── column6:6 + │ │ │ ├── first-agg [as=column2:6] + │ │ │ │ └── column2:6 + │ │ │ └── first-agg [as=column7:7] + │ │ │ └── column7:7 │ │ ├── scan parent_multi - │ │ │ └── columns: pk:7!null p:8 q:9 + │ │ │ └── columns: pk:8!null p:9 q:10 crdb_internal_mvcc_timestamp:11 │ │ └── filters - │ │ └── column1:4 = pk:7 + │ │ └── column1:5 = pk:8 │ └── projections - │ ├── CASE WHEN pk:7 IS NULL THEN column1:4 ELSE pk:7 END [as=upsert_pk:10] - │ └── CASE WHEN pk:7 IS NULL THEN column6:6 ELSE q:9 END [as=upsert_q:11] + │ ├── CASE WHEN pk:8 IS NULL THEN column1:5 ELSE pk:8 END [as=upsert_pk:12] + │ └── CASE WHEN pk:8 IS NULL THEN column7:7 ELSE q:10 END [as=upsert_q:13] └── cascade └── update child_multi ├── columns: - ├── fetch columns: c:15 child_multi.p:16 child_multi.q:17 + ├── fetch columns: c:18 child_multi.p:19 child_multi.q:20 ├── update-mapping: - │ ├── column2:20 => child_multi.p:13 - │ └── q:21 => child_multi.q:14 + │ ├── column2:24 => child_multi.p:15 + │ └── q:25 => child_multi.q:16 ├── input binding: &2 ├── inner-join (hash) - │ ├── columns: c:15!null child_multi.p:16!null child_multi.q:17!null p:18!null q:19!null column2:20!null q:21 + │ ├── columns: c:18!null child_multi.p:19!null child_multi.q:20!null child_multi.crdb_internal_mvcc_timestamp:21 p:22!null q:23!null column2:24!null q:25 │ ├── scan child_multi - │ │ └── columns: c:15!null child_multi.p:16 child_multi.q:17 + │ │ └── columns: c:18!null child_multi.p:19 child_multi.q:20 child_multi.crdb_internal_mvcc_timestamp:21 │ ├── select - │ │ ├── columns: p:18 q:19 column2:20!null q:21 + │ │ ├── columns: p:22 q:23 column2:24!null q:25 │ │ ├── with-scan &1 - │ │ │ ├── columns: p:18 q:19 column2:20!null q:21 + │ │ │ ├── columns: p:22 q:23 column2:24!null q:25 │ │ │ └── mapping: - │ │ │ ├── parent_multi.p:8 => p:18 - │ │ │ ├── parent_multi.q:9 => q:19 - │ │ │ ├── column2:5 => column2:20 - │ │ │ └── parent_multi.q:9 => q:21 + │ │ │ ├── parent_multi.p:9 => p:22 + │ │ │ ├── parent_multi.q:10 => q:23 + │ │ │ ├── column2:6 => column2:24 + │ │ │ └── parent_multi.q:10 => q:25 │ │ └── filters - │ │ └── (p:18 IS DISTINCT FROM column2:20) OR (q:19 IS DISTINCT FROM q:21) + │ │ └── (p:22 IS DISTINCT FROM column2:24) OR (q:23 IS DISTINCT FROM q:25) │ └── filters - │ ├── child_multi.p:16 = p:18 - │ └── child_multi.q:17 = q:19 + │ ├── child_multi.p:19 = p:22 + │ └── child_multi.q:20 = q:23 └── f-k-checks └── f-k-checks-item: child_multi(p,q) -> parent_multi(p,q) └── anti-join (hash) - ├── columns: column2:22!null q:23!null + ├── columns: column2:26!null q:27!null ├── select - │ ├── columns: column2:22!null q:23!null + │ ├── columns: column2:26!null q:27!null │ ├── with-scan &2 - │ │ ├── columns: column2:22!null q:23 + │ │ ├── columns: column2:26!null q:27 │ │ └── mapping: - │ │ ├── column2:20 => column2:22 - │ │ └── q:21 => q:23 + │ │ ├── column2:24 => column2:26 + │ │ └── q:25 => q:27 │ └── filters - │ └── q:23 IS NOT NULL + │ └── q:27 IS NOT NULL ├── scan parent_multi - │ └── columns: parent_multi.p:25 parent_multi.q:26 + │ └── columns: parent_multi.p:29 parent_multi.q:30 └── filters - ├── column2:22 = parent_multi.p:25 - └── q:23 = parent_multi.q:26 + ├── column2:26 = parent_multi.p:29 + └── q:27 = parent_multi.q:30 build-cascades INSERT INTO parent_multi VALUES (1, 10, 10), (2, 20, 20) ON CONFLICT (p,q) DO UPDATE SET p = 100 @@ -399,88 +399,88 @@ INSERT INTO parent_multi VALUES (1, 10, 10), (2, 20, 20) ON CONFLICT (p,q) DO UP root ├── upsert parent_multi │ ├── columns: - │ ├── canary column: 7 - │ ├── fetch columns: pk:7 p:8 q:9 + │ ├── canary column: 8 + │ ├── fetch columns: pk:8 p:9 q:10 │ ├── insert-mapping: - │ │ ├── column1:4 => pk:1 - │ │ ├── column2:5 => p:2 - │ │ └── column3:6 => q:3 + │ │ ├── column1:5 => pk:1 + │ │ ├── column2:6 => p:2 + │ │ └── column3:7 => q:3 │ ├── update-mapping: - │ │ └── upsert_p:12 => p:2 + │ │ └── upsert_p:14 => p:2 │ ├── input binding: &1 │ ├── cascades │ │ └── fk │ └── project - │ ├── columns: upsert_pk:11 upsert_p:12!null upsert_q:13 column1:4!null column2:5!null column3:6!null pk:7 p:8 q:9 p_new:10!null + │ ├── columns: upsert_pk:13 upsert_p:14!null upsert_q:15 column1:5!null column2:6!null column3:7!null pk:8 p:9 q:10 crdb_internal_mvcc_timestamp:11 p_new:12!null │ ├── project - │ │ ├── columns: p_new:10!null column1:4!null column2:5!null column3:6!null pk:7 p:8 q:9 + │ │ ├── columns: p_new:12!null column1:5!null column2:6!null column3:7!null pk:8 p:9 q:10 crdb_internal_mvcc_timestamp:11 │ │ ├── left-join (hash) - │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null pk:7 p:8 q:9 + │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null pk:8 p:9 q:10 crdb_internal_mvcc_timestamp:11 │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null - │ │ │ │ ├── grouping columns: column2:5!null column3:6!null + │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null + │ │ │ │ ├── grouping columns: column2:6!null column3:7!null │ │ │ │ ├── values - │ │ │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null + │ │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null │ │ │ │ │ ├── (1, 10, 10) │ │ │ │ │ └── (2, 20, 20) │ │ │ │ └── aggregations - │ │ │ │ └── first-agg [as=column1:4] - │ │ │ │ └── column1:4 + │ │ │ │ └── first-agg [as=column1:5] + │ │ │ │ └── column1:5 │ │ │ ├── scan parent_multi - │ │ │ │ └── columns: pk:7!null p:8 q:9 + │ │ │ │ └── columns: pk:8!null p:9 q:10 crdb_internal_mvcc_timestamp:11 │ │ │ └── filters - │ │ │ ├── column2:5 = p:8 - │ │ │ └── column3:6 = q:9 + │ │ │ ├── column2:6 = p:9 + │ │ │ └── column3:7 = q:10 │ │ └── projections - │ │ └── 100 [as=p_new:10] + │ │ └── 100 [as=p_new:12] │ └── projections - │ ├── CASE WHEN pk:7 IS NULL THEN column1:4 ELSE pk:7 END [as=upsert_pk:11] - │ ├── CASE WHEN pk:7 IS NULL THEN column2:5 ELSE p_new:10 END [as=upsert_p:12] - │ └── CASE WHEN pk:7 IS NULL THEN column3:6 ELSE q:9 END [as=upsert_q:13] + │ ├── CASE WHEN pk:8 IS NULL THEN column1:5 ELSE pk:8 END [as=upsert_pk:13] + │ ├── CASE WHEN pk:8 IS NULL THEN column2:6 ELSE p_new:12 END [as=upsert_p:14] + │ └── CASE WHEN pk:8 IS NULL THEN column3:7 ELSE q:10 END [as=upsert_q:15] └── cascade └── update child_multi ├── columns: - ├── fetch columns: c:17 child_multi.p:18 child_multi.q:19 + ├── fetch columns: c:20 child_multi.p:21 child_multi.q:22 ├── update-mapping: - │ ├── upsert_p:22 => child_multi.p:15 - │ └── q:23 => child_multi.q:16 + │ ├── upsert_p:26 => child_multi.p:17 + │ └── q:27 => child_multi.q:18 ├── input binding: &2 ├── inner-join (hash) - │ ├── columns: c:17!null child_multi.p:18!null child_multi.q:19!null p:20!null q:21!null upsert_p:22!null q:23 + │ ├── columns: c:20!null child_multi.p:21!null child_multi.q:22!null child_multi.crdb_internal_mvcc_timestamp:23 p:24!null q:25!null upsert_p:26!null q:27 │ ├── scan child_multi - │ │ └── columns: c:17!null child_multi.p:18 child_multi.q:19 + │ │ └── columns: c:20!null child_multi.p:21 child_multi.q:22 child_multi.crdb_internal_mvcc_timestamp:23 │ ├── select - │ │ ├── columns: p:20 q:21 upsert_p:22!null q:23 + │ │ ├── columns: p:24 q:25 upsert_p:26!null q:27 │ │ ├── with-scan &1 - │ │ │ ├── columns: p:20 q:21 upsert_p:22!null q:23 + │ │ │ ├── columns: p:24 q:25 upsert_p:26!null q:27 │ │ │ └── mapping: - │ │ │ ├── parent_multi.p:8 => p:20 - │ │ │ ├── parent_multi.q:9 => q:21 - │ │ │ ├── upsert_p:12 => upsert_p:22 - │ │ │ └── parent_multi.q:9 => q:23 + │ │ │ ├── parent_multi.p:9 => p:24 + │ │ │ ├── parent_multi.q:10 => q:25 + │ │ │ ├── upsert_p:14 => upsert_p:26 + │ │ │ └── parent_multi.q:10 => q:27 │ │ └── filters - │ │ └── (p:20 IS DISTINCT FROM upsert_p:22) OR (q:21 IS DISTINCT FROM q:23) + │ │ └── (p:24 IS DISTINCT FROM upsert_p:26) OR (q:25 IS DISTINCT FROM q:27) │ └── filters - │ ├── child_multi.p:18 = p:20 - │ └── child_multi.q:19 = q:21 + │ ├── child_multi.p:21 = p:24 + │ └── child_multi.q:22 = q:25 └── f-k-checks └── f-k-checks-item: child_multi(p,q) -> parent_multi(p,q) └── anti-join (hash) - ├── columns: upsert_p:24!null q:25!null + ├── columns: upsert_p:28!null q:29!null ├── select - │ ├── columns: upsert_p:24!null q:25!null + │ ├── columns: upsert_p:28!null q:29!null │ ├── with-scan &2 - │ │ ├── columns: upsert_p:24!null q:25 + │ │ ├── columns: upsert_p:28!null q:29 │ │ └── mapping: - │ │ ├── upsert_p:22 => upsert_p:24 - │ │ └── q:23 => q:25 + │ │ ├── upsert_p:26 => upsert_p:28 + │ │ └── q:27 => q:29 │ └── filters - │ └── q:25 IS NOT NULL + │ └── q:29 IS NOT NULL ├── scan parent_multi - │ └── columns: parent_multi.p:27 parent_multi.q:28 + │ └── columns: parent_multi.p:31 parent_multi.q:32 └── filters - ├── upsert_p:24 = parent_multi.p:27 - └── q:25 = parent_multi.q:28 + ├── upsert_p:28 = parent_multi.p:31 + └── q:29 = parent_multi.q:32 # Test a two-level cascade. exec-ddl @@ -497,112 +497,112 @@ UPDATE parent_multi SET q = q * 10 WHERE p > 1 root ├── update parent_multi │ ├── columns: - │ ├── fetch columns: pk:4 p:5 q:6 + │ ├── fetch columns: pk:5 p:6 q:7 │ ├── update-mapping: - │ │ └── q_new:7 => q:3 + │ │ └── q_new:9 => q:3 │ ├── input binding: &1 │ ├── cascades │ │ └── fk │ └── project - │ ├── columns: q_new:7 pk:4!null p:5!null q:6 + │ ├── columns: q_new:9 pk:5!null p:6!null q:7 crdb_internal_mvcc_timestamp:8 │ ├── select - │ │ ├── columns: pk:4!null p:5!null q:6 + │ │ ├── columns: pk:5!null p:6!null q:7 crdb_internal_mvcc_timestamp:8 │ │ ├── scan parent_multi - │ │ │ └── columns: pk:4!null p:5 q:6 + │ │ │ └── columns: pk:5!null p:6 q:7 crdb_internal_mvcc_timestamp:8 │ │ └── filters - │ │ └── p:5 > 1 + │ │ └── p:6 > 1 │ └── projections - │ └── q:6 * 10 [as=q_new:7] + │ └── q:7 * 10 [as=q_new:9] └── cascade ├── update child_multi │ ├── columns: - │ ├── fetch columns: c:11 child_multi.p:12 child_multi.q:13 + │ ├── fetch columns: c:14 child_multi.p:15 child_multi.q:16 │ ├── update-mapping: - │ │ ├── p:16 => child_multi.p:9 - │ │ └── q_new:17 => child_multi.q:10 + │ │ ├── p:20 => child_multi.p:11 + │ │ └── q_new:21 => child_multi.q:12 │ ├── input binding: &2 │ ├── cascades │ │ └── fk2 │ ├── inner-join (hash) - │ │ ├── columns: c:11!null child_multi.p:12!null child_multi.q:13!null p:14!null q:15!null p:16!null q_new:17 + │ │ ├── columns: c:14!null child_multi.p:15!null child_multi.q:16!null child_multi.crdb_internal_mvcc_timestamp:17 p:18!null q:19!null p:20!null q_new:21 │ │ ├── scan child_multi - │ │ │ └── columns: c:11!null child_multi.p:12 child_multi.q:13 + │ │ │ └── columns: c:14!null child_multi.p:15 child_multi.q:16 child_multi.crdb_internal_mvcc_timestamp:17 │ │ ├── select - │ │ │ ├── columns: p:14!null q:15 p:16!null q_new:17 + │ │ │ ├── columns: p:18!null q:19 p:20!null q_new:21 │ │ │ ├── with-scan &1 - │ │ │ │ ├── columns: p:14!null q:15 p:16!null q_new:17 + │ │ │ │ ├── columns: p:18!null q:19 p:20!null q_new:21 │ │ │ │ └── mapping: - │ │ │ │ ├── parent_multi.p:5 => p:14 - │ │ │ │ ├── parent_multi.q:6 => q:15 - │ │ │ │ ├── parent_multi.p:5 => p:16 - │ │ │ │ └── q_new:7 => q_new:17 + │ │ │ │ ├── parent_multi.p:6 => p:18 + │ │ │ │ ├── parent_multi.q:7 => q:19 + │ │ │ │ ├── parent_multi.p:6 => p:20 + │ │ │ │ └── q_new:9 => q_new:21 │ │ │ └── filters - │ │ │ └── (p:14 IS DISTINCT FROM p:16) OR (q:15 IS DISTINCT FROM q_new:17) + │ │ │ └── (p:18 IS DISTINCT FROM p:20) OR (q:19 IS DISTINCT FROM q_new:21) │ │ └── filters - │ │ ├── child_multi.p:12 = p:14 - │ │ └── child_multi.q:13 = q:15 + │ │ ├── child_multi.p:15 = p:18 + │ │ └── child_multi.q:16 = q:19 │ └── f-k-checks │ └── f-k-checks-item: child_multi(p,q) -> parent_multi(p,q) │ └── anti-join (hash) - │ ├── columns: p:18!null q_new:19!null + │ ├── columns: p:22!null q_new:23!null │ ├── select - │ │ ├── columns: p:18!null q_new:19!null + │ │ ├── columns: p:22!null q_new:23!null │ │ ├── with-scan &2 - │ │ │ ├── columns: p:18!null q_new:19 + │ │ │ ├── columns: p:22!null q_new:23 │ │ │ └── mapping: - │ │ │ ├── p:16 => p:18 - │ │ │ └── q_new:17 => q_new:19 + │ │ │ ├── p:20 => p:22 + │ │ │ └── q_new:21 => q_new:23 │ │ └── filters - │ │ └── q_new:19 IS NOT NULL + │ │ └── q_new:23 IS NOT NULL │ ├── scan parent_multi - │ │ └── columns: parent_multi.p:21 parent_multi.q:22 + │ │ └── columns: parent_multi.p:25 parent_multi.q:26 │ └── filters - │ ├── p:18 = parent_multi.p:21 - │ └── q_new:19 = parent_multi.q:22 + │ ├── p:22 = parent_multi.p:25 + │ └── q_new:23 = parent_multi.q:26 └── cascade └── update grandchild ├── columns: - ├── fetch columns: g:26 grandchild.c:27 grandchild.q:28 + ├── fetch columns: g:32 grandchild.c:33 grandchild.q:34 ├── update-mapping: - │ ├── c:31 => grandchild.c:24 - │ └── q_new:32 => grandchild.q:25 + │ ├── c:38 => grandchild.c:29 + │ └── q_new:39 => grandchild.q:30 ├── input binding: &3 ├── inner-join (hash) - │ ├── columns: g:26!null grandchild.c:27!null grandchild.q:28!null c:29!null q:30!null c:31!null q_new:32 + │ ├── columns: g:32!null grandchild.c:33!null grandchild.q:34!null grandchild.crdb_internal_mvcc_timestamp:35 c:36!null q:37!null c:38!null q_new:39 │ ├── scan grandchild - │ │ └── columns: g:26!null grandchild.c:27 grandchild.q:28 + │ │ └── columns: g:32!null grandchild.c:33 grandchild.q:34 grandchild.crdb_internal_mvcc_timestamp:35 │ ├── select - │ │ ├── columns: c:29!null q:30!null c:31!null q_new:32 + │ │ ├── columns: c:36!null q:37!null c:38!null q_new:39 │ │ ├── with-scan &2 - │ │ │ ├── columns: c:29!null q:30!null c:31!null q_new:32 + │ │ │ ├── columns: c:36!null q:37!null c:38!null q_new:39 │ │ │ └── mapping: - │ │ │ ├── child_multi.c:11 => c:29 - │ │ │ ├── child_multi.q:13 => q:30 - │ │ │ ├── child_multi.c:11 => c:31 - │ │ │ └── q_new:17 => q_new:32 + │ │ │ ├── child_multi.c:14 => c:36 + │ │ │ ├── child_multi.q:16 => q:37 + │ │ │ ├── child_multi.c:14 => c:38 + │ │ │ └── q_new:21 => q_new:39 │ │ └── filters - │ │ └── (c:29 IS DISTINCT FROM c:31) OR (q:30 IS DISTINCT FROM q_new:32) + │ │ └── (c:36 IS DISTINCT FROM c:38) OR (q:37 IS DISTINCT FROM q_new:39) │ └── filters - │ ├── grandchild.c:27 = c:29 - │ └── grandchild.q:28 = q:30 + │ ├── grandchild.c:33 = c:36 + │ └── grandchild.q:34 = q:37 └── f-k-checks └── f-k-checks-item: grandchild(c,q) -> child_multi(c,q) └── anti-join (hash) - ├── columns: c:33!null q_new:34!null + ├── columns: c:40!null q_new:41!null ├── select - │ ├── columns: c:33!null q_new:34!null + │ ├── columns: c:40!null q_new:41!null │ ├── with-scan &3 - │ │ ├── columns: c:33!null q_new:34 + │ │ ├── columns: c:40!null q_new:41 │ │ └── mapping: - │ │ ├── c:31 => c:33 - │ │ └── q_new:32 => q_new:34 + │ │ ├── c:38 => c:40 + │ │ └── q_new:39 => q_new:41 │ └── filters - │ └── q_new:34 IS NOT NULL + │ └── q_new:41 IS NOT NULL ├── scan child_multi - │ └── columns: child_multi.c:35!null child_multi.q:37 + │ └── columns: child_multi.c:42!null child_multi.q:44 └── filters - ├── c:33 = child_multi.c:35 - └── q_new:34 = child_multi.q:37 + ├── c:40 = child_multi.c:42 + └── q_new:41 = child_multi.q:44 build-cascades UPSERT INTO parent_multi VALUES (1, 10, 10), (2, 20, 20) @@ -610,119 +610,119 @@ UPSERT INTO parent_multi VALUES (1, 10, 10), (2, 20, 20) root ├── upsert parent_multi │ ├── columns: - │ ├── canary column: 7 - │ ├── fetch columns: pk:7 p:8 q:9 + │ ├── canary column: 8 + │ ├── fetch columns: pk:8 p:9 q:10 │ ├── insert-mapping: - │ │ ├── column1:4 => pk:1 - │ │ ├── column2:5 => p:2 - │ │ └── column3:6 => q:3 + │ │ ├── column1:5 => pk:1 + │ │ ├── column2:6 => p:2 + │ │ └── column3:7 => q:3 │ ├── update-mapping: - │ │ ├── column2:5 => p:2 - │ │ └── column3:6 => q:3 + │ │ ├── column2:6 => p:2 + │ │ └── column3:7 => q:3 │ ├── input binding: &1 │ ├── cascades │ │ └── fk │ └── project - │ ├── columns: upsert_pk:10 column1:4!null column2:5!null column3:6!null pk:7 p:8 q:9 + │ ├── columns: upsert_pk:12 column1:5!null column2:6!null column3:7!null pk:8 p:9 q:10 crdb_internal_mvcc_timestamp:11 │ ├── left-join (hash) - │ │ ├── columns: column1:4!null column2:5!null column3:6!null pk:7 p:8 q:9 + │ │ ├── columns: column1:5!null column2:6!null column3:7!null pk:8 p:9 q:10 crdb_internal_mvcc_timestamp:11 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null - │ │ │ ├── grouping columns: column1:4!null + │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null + │ │ │ ├── grouping columns: column1:5!null │ │ │ ├── values - │ │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null + │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null │ │ │ │ ├── (1, 10, 10) │ │ │ │ └── (2, 20, 20) │ │ │ └── aggregations - │ │ │ ├── first-agg [as=column2:5] - │ │ │ │ └── column2:5 - │ │ │ └── first-agg [as=column3:6] - │ │ │ └── column3:6 + │ │ │ ├── first-agg [as=column2:6] + │ │ │ │ └── column2:6 + │ │ │ └── first-agg [as=column3:7] + │ │ │ └── column3:7 │ │ ├── scan parent_multi - │ │ │ └── columns: pk:7!null p:8 q:9 + │ │ │ └── columns: pk:8!null p:9 q:10 crdb_internal_mvcc_timestamp:11 │ │ └── filters - │ │ └── column1:4 = pk:7 + │ │ └── column1:5 = pk:8 │ └── projections - │ └── CASE WHEN pk:7 IS NULL THEN column1:4 ELSE pk:7 END [as=upsert_pk:10] + │ └── CASE WHEN pk:8 IS NULL THEN column1:5 ELSE pk:8 END [as=upsert_pk:12] └── cascade ├── update child_multi │ ├── columns: - │ ├── fetch columns: c:14 child_multi.p:15 child_multi.q:16 + │ ├── fetch columns: c:17 child_multi.p:18 child_multi.q:19 │ ├── update-mapping: - │ │ ├── column2:19 => child_multi.p:12 - │ │ └── column3:20 => child_multi.q:13 + │ │ ├── column2:23 => child_multi.p:14 + │ │ └── column3:24 => child_multi.q:15 │ ├── input binding: &2 │ ├── cascades │ │ └── fk2 │ ├── inner-join (hash) - │ │ ├── columns: c:14!null child_multi.p:15!null child_multi.q:16!null p:17!null q:18!null column2:19!null column3:20!null + │ │ ├── columns: c:17!null child_multi.p:18!null child_multi.q:19!null child_multi.crdb_internal_mvcc_timestamp:20 p:21!null q:22!null column2:23!null column3:24!null │ │ ├── scan child_multi - │ │ │ └── columns: c:14!null child_multi.p:15 child_multi.q:16 + │ │ │ └── columns: c:17!null child_multi.p:18 child_multi.q:19 child_multi.crdb_internal_mvcc_timestamp:20 │ │ ├── select - │ │ │ ├── columns: p:17 q:18 column2:19!null column3:20!null + │ │ │ ├── columns: p:21 q:22 column2:23!null column3:24!null │ │ │ ├── with-scan &1 - │ │ │ │ ├── columns: p:17 q:18 column2:19!null column3:20!null + │ │ │ │ ├── columns: p:21 q:22 column2:23!null column3:24!null │ │ │ │ └── mapping: - │ │ │ │ ├── parent_multi.p:8 => p:17 - │ │ │ │ ├── parent_multi.q:9 => q:18 - │ │ │ │ ├── column2:5 => column2:19 - │ │ │ │ └── column3:6 => column3:20 + │ │ │ │ ├── parent_multi.p:9 => p:21 + │ │ │ │ ├── parent_multi.q:10 => q:22 + │ │ │ │ ├── column2:6 => column2:23 + │ │ │ │ └── column3:7 => column3:24 │ │ │ └── filters - │ │ │ └── (p:17 IS DISTINCT FROM column2:19) OR (q:18 IS DISTINCT FROM column3:20) + │ │ │ └── (p:21 IS DISTINCT FROM column2:23) OR (q:22 IS DISTINCT FROM column3:24) │ │ └── filters - │ │ ├── child_multi.p:15 = p:17 - │ │ └── child_multi.q:16 = q:18 + │ │ ├── child_multi.p:18 = p:21 + │ │ └── child_multi.q:19 = q:22 │ └── f-k-checks │ └── f-k-checks-item: child_multi(p,q) -> parent_multi(p,q) │ └── anti-join (hash) - │ ├── columns: column2:21!null column3:22!null + │ ├── columns: column2:25!null column3:26!null │ ├── with-scan &2 - │ │ ├── columns: column2:21!null column3:22!null + │ │ ├── columns: column2:25!null column3:26!null │ │ └── mapping: - │ │ ├── column2:19 => column2:21 - │ │ └── column3:20 => column3:22 + │ │ ├── column2:23 => column2:25 + │ │ └── column3:24 => column3:26 │ ├── scan parent_multi - │ │ └── columns: parent_multi.p:24 parent_multi.q:25 + │ │ └── columns: parent_multi.p:28 parent_multi.q:29 │ └── filters - │ ├── column2:21 = parent_multi.p:24 - │ └── column3:22 = parent_multi.q:25 + │ ├── column2:25 = parent_multi.p:28 + │ └── column3:26 = parent_multi.q:29 └── cascade └── update grandchild ├── columns: - ├── fetch columns: g:29 grandchild.c:30 grandchild.q:31 + ├── fetch columns: g:35 grandchild.c:36 grandchild.q:37 ├── update-mapping: - │ ├── c:34 => grandchild.c:27 - │ └── column3:35 => grandchild.q:28 + │ ├── c:41 => grandchild.c:32 + │ └── column3:42 => grandchild.q:33 ├── input binding: &3 ├── inner-join (hash) - │ ├── columns: g:29!null grandchild.c:30!null grandchild.q:31!null c:32!null q:33!null c:34!null column3:35!null + │ ├── columns: g:35!null grandchild.c:36!null grandchild.q:37!null grandchild.crdb_internal_mvcc_timestamp:38 c:39!null q:40!null c:41!null column3:42!null │ ├── scan grandchild - │ │ └── columns: g:29!null grandchild.c:30 grandchild.q:31 + │ │ └── columns: g:35!null grandchild.c:36 grandchild.q:37 grandchild.crdb_internal_mvcc_timestamp:38 │ ├── select - │ │ ├── columns: c:32!null q:33!null c:34!null column3:35!null + │ │ ├── columns: c:39!null q:40!null c:41!null column3:42!null │ │ ├── with-scan &2 - │ │ │ ├── columns: c:32!null q:33!null c:34!null column3:35!null + │ │ │ ├── columns: c:39!null q:40!null c:41!null column3:42!null │ │ │ └── mapping: - │ │ │ ├── child_multi.c:14 => c:32 - │ │ │ ├── child_multi.q:16 => q:33 - │ │ │ ├── child_multi.c:14 => c:34 - │ │ │ └── column3:20 => column3:35 + │ │ │ ├── child_multi.c:17 => c:39 + │ │ │ ├── child_multi.q:19 => q:40 + │ │ │ ├── child_multi.c:17 => c:41 + │ │ │ └── column3:24 => column3:42 │ │ └── filters - │ │ └── (c:32 IS DISTINCT FROM c:34) OR (q:33 IS DISTINCT FROM column3:35) + │ │ └── (c:39 IS DISTINCT FROM c:41) OR (q:40 IS DISTINCT FROM column3:42) │ └── filters - │ ├── grandchild.c:30 = c:32 - │ └── grandchild.q:31 = q:33 + │ ├── grandchild.c:36 = c:39 + │ └── grandchild.q:37 = q:40 └── f-k-checks └── f-k-checks-item: grandchild(c,q) -> child_multi(c,q) └── anti-join (hash) - ├── columns: c:36!null column3:37!null + ├── columns: c:43!null column3:44!null ├── with-scan &3 - │ ├── columns: c:36!null column3:37!null + │ ├── columns: c:43!null column3:44!null │ └── mapping: - │ ├── c:34 => c:36 - │ └── column3:35 => column3:37 + │ ├── c:41 => c:43 + │ └── column3:42 => column3:44 ├── scan child_multi - │ └── columns: child_multi.c:38!null child_multi.q:40 + │ └── columns: child_multi.c:45!null child_multi.q:47 └── filters - ├── c:36 = child_multi.c:38 - └── column3:37 = child_multi.q:40 + ├── c:43 = child_multi.c:45 + └── column3:44 = child_multi.q:47 diff --git a/pkg/sql/opt/optbuilder/testdata/fk-on-update-set-default b/pkg/sql/opt/optbuilder/testdata/fk-on-update-set-default index e91fcafc8e6d..6e6b5908d854 100644 --- a/pkg/sql/opt/optbuilder/testdata/fk-on-update-set-default +++ b/pkg/sql/opt/optbuilder/testdata/fk-on-update-set-default @@ -12,60 +12,60 @@ UPDATE parent SET p = p * 10 WHERE p > 1 root ├── update parent │ ├── columns: - │ ├── fetch columns: p:2 + │ ├── fetch columns: p:3 │ ├── update-mapping: - │ │ └── p_new:3 => p:1 + │ │ └── p_new:5 => p:1 │ ├── input binding: &1 │ ├── cascades │ │ └── fk_p_ref_parent │ └── project - │ ├── columns: p_new:3!null p:2!null + │ ├── columns: p_new:5!null p:3!null crdb_internal_mvcc_timestamp:4 │ ├── select - │ │ ├── columns: p:2!null + │ │ ├── columns: p:3!null crdb_internal_mvcc_timestamp:4 │ │ ├── scan parent - │ │ │ └── columns: p:2!null + │ │ │ └── columns: p:3!null crdb_internal_mvcc_timestamp:4 │ │ └── filters - │ │ └── p:2 > 1 + │ │ └── p:3 > 1 │ └── projections - │ └── p:2 * 10 [as=p_new:3] + │ └── p:3 * 10 [as=p_new:5] └── cascade └── update child ├── columns: - ├── fetch columns: c:6 child.p:7 + ├── fetch columns: c:9 child.p:10 ├── update-mapping: - │ └── p_new:10 => child.p:5 + │ └── p_new:14 => child.p:7 ├── input binding: &2 ├── project - │ ├── columns: p_new:10!null c:6!null child.p:7!null p:8!null p_new:9!null + │ ├── columns: p_new:14!null c:9!null child.p:10!null child.crdb_internal_mvcc_timestamp:11 p:12!null p_new:13!null │ ├── inner-join (hash) - │ │ ├── columns: c:6!null child.p:7!null p:8!null p_new:9!null + │ │ ├── columns: c:9!null child.p:10!null child.crdb_internal_mvcc_timestamp:11 p:12!null p_new:13!null │ │ ├── scan child - │ │ │ └── columns: c:6!null child.p:7!null + │ │ │ └── columns: c:9!null child.p:10!null child.crdb_internal_mvcc_timestamp:11 │ │ ├── select - │ │ │ ├── columns: p:8!null p_new:9!null + │ │ │ ├── columns: p:12!null p_new:13!null │ │ │ ├── with-scan &1 - │ │ │ │ ├── columns: p:8!null p_new:9!null + │ │ │ │ ├── columns: p:12!null p_new:13!null │ │ │ │ └── mapping: - │ │ │ │ ├── parent.p:2 => p:8 - │ │ │ │ └── p_new:3 => p_new:9 + │ │ │ │ ├── parent.p:3 => p:12 + │ │ │ │ └── p_new:5 => p_new:13 │ │ │ └── filters - │ │ │ └── p:8 IS DISTINCT FROM p_new:9 + │ │ │ └── p:12 IS DISTINCT FROM p_new:13 │ │ └── filters - │ │ └── child.p:7 = p:8 + │ │ └── child.p:10 = p:12 │ └── projections - │ └── 0 [as=p_new:10] + │ └── 0 [as=p_new:14] └── f-k-checks └── f-k-checks-item: child(p) -> parent(p) └── anti-join (hash) - ├── columns: p_new:11!null + ├── columns: p_new:15!null ├── with-scan &2 - │ ├── columns: p_new:11!null + │ ├── columns: p_new:15!null │ └── mapping: - │ └── p_new:10 => p_new:11 + │ └── p_new:14 => p_new:15 ├── scan parent - │ └── columns: parent.p:12!null + │ └── columns: parent.p:16!null └── filters - └── p_new:11 = parent.p:12 + └── p_new:15 = parent.p:16 exec-ddl CREATE TABLE parent_multi ( @@ -94,69 +94,69 @@ UPDATE parent_multi SET p = p * 10, q = q + 1 WHERE pk > 1 root ├── update parent_multi │ ├── columns: - │ ├── fetch columns: pk:4 p:5 q:6 + │ ├── fetch columns: pk:5 p:6 q:7 │ ├── update-mapping: - │ │ ├── p_new:7 => p:2 - │ │ └── q_new:8 => q:3 + │ │ ├── p_new:9 => p:2 + │ │ └── q_new:10 => q:3 │ ├── input binding: &1 │ ├── cascades │ │ └── fk │ └── project - │ ├── columns: p_new:7 q_new:8 pk:4!null p:5 q:6 + │ ├── columns: p_new:9 q_new:10 pk:5!null p:6 q:7 crdb_internal_mvcc_timestamp:8 │ ├── select - │ │ ├── columns: pk:4!null p:5 q:6 + │ │ ├── columns: pk:5!null p:6 q:7 crdb_internal_mvcc_timestamp:8 │ │ ├── scan parent_multi - │ │ │ └── columns: pk:4!null p:5 q:6 + │ │ │ └── columns: pk:5!null p:6 q:7 crdb_internal_mvcc_timestamp:8 │ │ └── filters - │ │ └── pk:4 > 1 + │ │ └── pk:5 > 1 │ └── projections - │ ├── p:5 * 10 [as=p_new:7] - │ └── q:6 + 1 [as=q_new:8] + │ ├── p:6 * 10 [as=p_new:9] + │ └── q:7 + 1 [as=q_new:10] └── cascade └── update child_multi ├── columns: - ├── fetch columns: c:12 child_multi.p:13 child_multi.q:14 + ├── fetch columns: c:15 child_multi.p:16 child_multi.q:17 ├── update-mapping: - │ ├── p_new:19 => child_multi.p:10 - │ └── q_new:20 => child_multi.q:11 + │ ├── p_new:23 => child_multi.p:12 + │ └── q_new:24 => child_multi.q:13 ├── input binding: &2 ├── project - │ ├── columns: p_new:19!null q_new:20!null c:12!null child_multi.p:13!null child_multi.q:14!null p:15!null q:16!null p_new:17 q_new:18 + │ ├── columns: p_new:23!null q_new:24!null c:15!null child_multi.p:16!null child_multi.q:17!null child_multi.crdb_internal_mvcc_timestamp:18 p:19!null q:20!null p_new:21 q_new:22 │ ├── inner-join (hash) - │ │ ├── columns: c:12!null child_multi.p:13!null child_multi.q:14!null p:15!null q:16!null p_new:17 q_new:18 + │ │ ├── columns: c:15!null child_multi.p:16!null child_multi.q:17!null child_multi.crdb_internal_mvcc_timestamp:18 p:19!null q:20!null p_new:21 q_new:22 │ │ ├── scan child_multi - │ │ │ └── columns: c:12!null child_multi.p:13 child_multi.q:14 + │ │ │ └── columns: c:15!null child_multi.p:16 child_multi.q:17 child_multi.crdb_internal_mvcc_timestamp:18 │ │ ├── select - │ │ │ ├── columns: p:15 q:16 p_new:17 q_new:18 + │ │ │ ├── columns: p:19 q:20 p_new:21 q_new:22 │ │ │ ├── with-scan &1 - │ │ │ │ ├── columns: p:15 q:16 p_new:17 q_new:18 + │ │ │ │ ├── columns: p:19 q:20 p_new:21 q_new:22 │ │ │ │ └── mapping: - │ │ │ │ ├── parent_multi.p:5 => p:15 - │ │ │ │ ├── parent_multi.q:6 => q:16 - │ │ │ │ ├── p_new:7 => p_new:17 - │ │ │ │ └── q_new:8 => q_new:18 + │ │ │ │ ├── parent_multi.p:6 => p:19 + │ │ │ │ ├── parent_multi.q:7 => q:20 + │ │ │ │ ├── p_new:9 => p_new:21 + │ │ │ │ └── q_new:10 => q_new:22 │ │ │ └── filters - │ │ │ └── (p:15 IS DISTINCT FROM p_new:17) OR (q:16 IS DISTINCT FROM q_new:18) + │ │ │ └── (p:19 IS DISTINCT FROM p_new:21) OR (q:20 IS DISTINCT FROM q_new:22) │ │ └── filters - │ │ ├── child_multi.p:13 = p:15 - │ │ └── child_multi.q:14 = q:16 + │ │ ├── child_multi.p:16 = p:19 + │ │ └── child_multi.q:17 = q:20 │ └── projections - │ ├── 0 [as=p_new:19] - │ └── 1 [as=q_new:20] + │ ├── 0 [as=p_new:23] + │ └── 1 [as=q_new:24] └── f-k-checks └── f-k-checks-item: child_multi(p,q) -> parent_multi(p,q) └── anti-join (hash) - ├── columns: p_new:21!null q_new:22!null + ├── columns: p_new:25!null q_new:26!null ├── with-scan &2 - │ ├── columns: p_new:21!null q_new:22!null + │ ├── columns: p_new:25!null q_new:26!null │ └── mapping: - │ ├── p_new:19 => p_new:21 - │ └── q_new:20 => q_new:22 + │ ├── p_new:23 => p_new:25 + │ └── q_new:24 => q_new:26 ├── scan parent_multi - │ └── columns: parent_multi.p:24 parent_multi.q:25 + │ └── columns: parent_multi.p:28 parent_multi.q:29 └── filters - ├── p_new:21 = parent_multi.p:24 - └── q_new:22 = parent_multi.q:25 + ├── p_new:25 = parent_multi.p:28 + └── q_new:26 = parent_multi.q:29 # Update only one of the two FK columns. The "before" and "after" values of q # come from the same column in the mutation input. @@ -166,67 +166,67 @@ UPDATE parent_multi SET p = p * 10 WHERE p > 1 root ├── update parent_multi │ ├── columns: - │ ├── fetch columns: pk:4 p:5 q:6 + │ ├── fetch columns: pk:5 p:6 q:7 │ ├── update-mapping: - │ │ └── p_new:7 => p:2 + │ │ └── p_new:9 => p:2 │ ├── input binding: &1 │ ├── cascades │ │ └── fk │ └── project - │ ├── columns: p_new:7!null pk:4!null p:5!null q:6 + │ ├── columns: p_new:9!null pk:5!null p:6!null q:7 crdb_internal_mvcc_timestamp:8 │ ├── select - │ │ ├── columns: pk:4!null p:5!null q:6 + │ │ ├── columns: pk:5!null p:6!null q:7 crdb_internal_mvcc_timestamp:8 │ │ ├── scan parent_multi - │ │ │ └── columns: pk:4!null p:5 q:6 + │ │ │ └── columns: pk:5!null p:6 q:7 crdb_internal_mvcc_timestamp:8 │ │ └── filters - │ │ └── p:5 > 1 + │ │ └── p:6 > 1 │ └── projections - │ └── p:5 * 10 [as=p_new:7] + │ └── p:6 * 10 [as=p_new:9] └── cascade └── update child_multi ├── columns: - ├── fetch columns: c:11 child_multi.p:12 child_multi.q:13 + ├── fetch columns: c:14 child_multi.p:15 child_multi.q:16 ├── update-mapping: - │ ├── p_new:18 => child_multi.p:9 - │ └── q_new:19 => child_multi.q:10 + │ ├── p_new:22 => child_multi.p:11 + │ └── q_new:23 => child_multi.q:12 ├── input binding: &2 ├── project - │ ├── columns: p_new:18!null q_new:19!null c:11!null child_multi.p:12!null child_multi.q:13!null p:14!null q:15!null p_new:16!null q:17 + │ ├── columns: p_new:22!null q_new:23!null c:14!null child_multi.p:15!null child_multi.q:16!null child_multi.crdb_internal_mvcc_timestamp:17 p:18!null q:19!null p_new:20!null q:21 │ ├── inner-join (hash) - │ │ ├── columns: c:11!null child_multi.p:12!null child_multi.q:13!null p:14!null q:15!null p_new:16!null q:17 + │ │ ├── columns: c:14!null child_multi.p:15!null child_multi.q:16!null child_multi.crdb_internal_mvcc_timestamp:17 p:18!null q:19!null p_new:20!null q:21 │ │ ├── scan child_multi - │ │ │ └── columns: c:11!null child_multi.p:12 child_multi.q:13 + │ │ │ └── columns: c:14!null child_multi.p:15 child_multi.q:16 child_multi.crdb_internal_mvcc_timestamp:17 │ │ ├── select - │ │ │ ├── columns: p:14!null q:15 p_new:16!null q:17 + │ │ │ ├── columns: p:18!null q:19 p_new:20!null q:21 │ │ │ ├── with-scan &1 - │ │ │ │ ├── columns: p:14!null q:15 p_new:16!null q:17 + │ │ │ │ ├── columns: p:18!null q:19 p_new:20!null q:21 │ │ │ │ └── mapping: - │ │ │ │ ├── parent_multi.p:5 => p:14 - │ │ │ │ ├── parent_multi.q:6 => q:15 - │ │ │ │ ├── p_new:7 => p_new:16 - │ │ │ │ └── parent_multi.q:6 => q:17 + │ │ │ │ ├── parent_multi.p:6 => p:18 + │ │ │ │ ├── parent_multi.q:7 => q:19 + │ │ │ │ ├── p_new:9 => p_new:20 + │ │ │ │ └── parent_multi.q:7 => q:21 │ │ │ └── filters - │ │ │ └── (p:14 IS DISTINCT FROM p_new:16) OR (q:15 IS DISTINCT FROM q:17) + │ │ │ └── (p:18 IS DISTINCT FROM p_new:20) OR (q:19 IS DISTINCT FROM q:21) │ │ └── filters - │ │ ├── child_multi.p:12 = p:14 - │ │ └── child_multi.q:13 = q:15 + │ │ ├── child_multi.p:15 = p:18 + │ │ └── child_multi.q:16 = q:19 │ └── projections - │ ├── 0 [as=p_new:18] - │ └── 1 [as=q_new:19] + │ ├── 0 [as=p_new:22] + │ └── 1 [as=q_new:23] └── f-k-checks └── f-k-checks-item: child_multi(p,q) -> parent_multi(p,q) └── anti-join (hash) - ├── columns: p_new:20!null q_new:21!null + ├── columns: p_new:24!null q_new:25!null ├── with-scan &2 - │ ├── columns: p_new:20!null q_new:21!null + │ ├── columns: p_new:24!null q_new:25!null │ └── mapping: - │ ├── p_new:18 => p_new:20 - │ └── q_new:19 => q_new:21 + │ ├── p_new:22 => p_new:24 + │ └── q_new:23 => q_new:25 ├── scan parent_multi - │ └── columns: parent_multi.p:23 parent_multi.q:24 + │ └── columns: parent_multi.p:27 parent_multi.q:28 └── filters - ├── p_new:20 = parent_multi.p:23 - └── q_new:21 = parent_multi.q:24 + ├── p_new:24 = parent_multi.p:27 + └── q_new:25 = parent_multi.q:28 # Test a two-level cascade. exec-ddl @@ -243,114 +243,114 @@ UPDATE parent_multi SET q = q * 10 WHERE p > 1 root ├── update parent_multi │ ├── columns: - │ ├── fetch columns: pk:4 p:5 q:6 + │ ├── fetch columns: pk:5 p:6 q:7 │ ├── update-mapping: - │ │ └── q_new:7 => q:3 + │ │ └── q_new:9 => q:3 │ ├── input binding: &1 │ ├── cascades │ │ └── fk │ └── project - │ ├── columns: q_new:7 pk:4!null p:5!null q:6 + │ ├── columns: q_new:9 pk:5!null p:6!null q:7 crdb_internal_mvcc_timestamp:8 │ ├── select - │ │ ├── columns: pk:4!null p:5!null q:6 + │ │ ├── columns: pk:5!null p:6!null q:7 crdb_internal_mvcc_timestamp:8 │ │ ├── scan parent_multi - │ │ │ └── columns: pk:4!null p:5 q:6 + │ │ │ └── columns: pk:5!null p:6 q:7 crdb_internal_mvcc_timestamp:8 │ │ └── filters - │ │ └── p:5 > 1 + │ │ └── p:6 > 1 │ └── projections - │ └── q:6 * 10 [as=q_new:7] + │ └── q:7 * 10 [as=q_new:9] └── cascade ├── update child_multi │ ├── columns: - │ ├── fetch columns: c:11 child_multi.p:12 child_multi.q:13 + │ ├── fetch columns: c:14 child_multi.p:15 child_multi.q:16 │ ├── update-mapping: - │ │ ├── p_new:18 => child_multi.p:9 - │ │ └── q_new:19 => child_multi.q:10 + │ │ ├── p_new:22 => child_multi.p:11 + │ │ └── q_new:23 => child_multi.q:12 │ ├── input binding: &2 │ ├── cascades │ │ └── fk2 │ ├── project - │ │ ├── columns: p_new:18!null q_new:19!null c:11!null child_multi.p:12!null child_multi.q:13!null p:14!null q:15!null p:16!null q_new:17 + │ │ ├── columns: p_new:22!null q_new:23!null c:14!null child_multi.p:15!null child_multi.q:16!null child_multi.crdb_internal_mvcc_timestamp:17 p:18!null q:19!null p:20!null q_new:21 │ │ ├── inner-join (hash) - │ │ │ ├── columns: c:11!null child_multi.p:12!null child_multi.q:13!null p:14!null q:15!null p:16!null q_new:17 + │ │ │ ├── columns: c:14!null child_multi.p:15!null child_multi.q:16!null child_multi.crdb_internal_mvcc_timestamp:17 p:18!null q:19!null p:20!null q_new:21 │ │ │ ├── scan child_multi - │ │ │ │ └── columns: c:11!null child_multi.p:12 child_multi.q:13 + │ │ │ │ └── columns: c:14!null child_multi.p:15 child_multi.q:16 child_multi.crdb_internal_mvcc_timestamp:17 │ │ │ ├── select - │ │ │ │ ├── columns: p:14!null q:15 p:16!null q_new:17 + │ │ │ │ ├── columns: p:18!null q:19 p:20!null q_new:21 │ │ │ │ ├── with-scan &1 - │ │ │ │ │ ├── columns: p:14!null q:15 p:16!null q_new:17 + │ │ │ │ │ ├── columns: p:18!null q:19 p:20!null q_new:21 │ │ │ │ │ └── mapping: - │ │ │ │ │ ├── parent_multi.p:5 => p:14 - │ │ │ │ │ ├── parent_multi.q:6 => q:15 - │ │ │ │ │ ├── parent_multi.p:5 => p:16 - │ │ │ │ │ └── q_new:7 => q_new:17 + │ │ │ │ │ ├── parent_multi.p:6 => p:18 + │ │ │ │ │ ├── parent_multi.q:7 => q:19 + │ │ │ │ │ ├── parent_multi.p:6 => p:20 + │ │ │ │ │ └── q_new:9 => q_new:21 │ │ │ │ └── filters - │ │ │ │ └── (p:14 IS DISTINCT FROM p:16) OR (q:15 IS DISTINCT FROM q_new:17) + │ │ │ │ └── (p:18 IS DISTINCT FROM p:20) OR (q:19 IS DISTINCT FROM q_new:21) │ │ │ └── filters - │ │ │ ├── child_multi.p:12 = p:14 - │ │ │ └── child_multi.q:13 = q:15 + │ │ │ ├── child_multi.p:15 = p:18 + │ │ │ └── child_multi.q:16 = q:19 │ │ └── projections - │ │ ├── 0 [as=p_new:18] - │ │ └── 1 [as=q_new:19] + │ │ ├── 0 [as=p_new:22] + │ │ └── 1 [as=q_new:23] │ └── f-k-checks │ └── f-k-checks-item: child_multi(p,q) -> parent_multi(p,q) │ └── anti-join (hash) - │ ├── columns: p_new:20!null q_new:21!null + │ ├── columns: p_new:24!null q_new:25!null │ ├── with-scan &2 - │ │ ├── columns: p_new:20!null q_new:21!null + │ │ ├── columns: p_new:24!null q_new:25!null │ │ └── mapping: - │ │ ├── p_new:18 => p_new:20 - │ │ └── q_new:19 => q_new:21 + │ │ ├── p_new:22 => p_new:24 + │ │ └── q_new:23 => q_new:25 │ ├── scan parent_multi - │ │ └── columns: parent_multi.p:23 parent_multi.q:24 + │ │ └── columns: parent_multi.p:27 parent_multi.q:28 │ └── filters - │ ├── p_new:20 = parent_multi.p:23 - │ └── q_new:21 = parent_multi.q:24 + │ ├── p_new:24 = parent_multi.p:27 + │ └── q_new:25 = parent_multi.q:28 └── cascade └── update grandchild ├── columns: - ├── fetch columns: g:28 grandchild.c:29 grandchild.q:30 + ├── fetch columns: g:34 grandchild.c:35 grandchild.q:36 ├── update-mapping: - │ ├── c_new:35 => grandchild.c:26 - │ └── q_new:36 => grandchild.q:27 + │ ├── c_new:42 => grandchild.c:31 + │ └── q_new:43 => grandchild.q:32 ├── input binding: &3 ├── project - │ ├── columns: c_new:35!null q_new:36!null g:28!null grandchild.c:29!null grandchild.q:30!null c:31!null q:32!null c:33!null q_new:34!null + │ ├── columns: c_new:42!null q_new:43!null g:34!null grandchild.c:35!null grandchild.q:36!null grandchild.crdb_internal_mvcc_timestamp:37 c:38!null q:39!null c:40!null q_new:41!null │ ├── inner-join (hash) - │ │ ├── columns: g:28!null grandchild.c:29!null grandchild.q:30!null c:31!null q:32!null c:33!null q_new:34!null + │ │ ├── columns: g:34!null grandchild.c:35!null grandchild.q:36!null grandchild.crdb_internal_mvcc_timestamp:37 c:38!null q:39!null c:40!null q_new:41!null │ │ ├── scan grandchild - │ │ │ └── columns: g:28!null grandchild.c:29 grandchild.q:30 + │ │ │ └── columns: g:34!null grandchild.c:35 grandchild.q:36 grandchild.crdb_internal_mvcc_timestamp:37 │ │ ├── select - │ │ │ ├── columns: c:31!null q:32!null c:33!null q_new:34!null + │ │ │ ├── columns: c:38!null q:39!null c:40!null q_new:41!null │ │ │ ├── with-scan &2 - │ │ │ │ ├── columns: c:31!null q:32!null c:33!null q_new:34!null + │ │ │ │ ├── columns: c:38!null q:39!null c:40!null q_new:41!null │ │ │ │ └── mapping: - │ │ │ │ ├── child_multi.c:11 => c:31 - │ │ │ │ ├── child_multi.q:13 => q:32 - │ │ │ │ ├── child_multi.c:11 => c:33 - │ │ │ │ └── q_new:19 => q_new:34 + │ │ │ │ ├── child_multi.c:14 => c:38 + │ │ │ │ ├── child_multi.q:16 => q:39 + │ │ │ │ ├── child_multi.c:14 => c:40 + │ │ │ │ └── q_new:23 => q_new:41 │ │ │ └── filters - │ │ │ └── (c:31 IS DISTINCT FROM c:33) OR (q:32 IS DISTINCT FROM q_new:34) + │ │ │ └── (c:38 IS DISTINCT FROM c:40) OR (q:39 IS DISTINCT FROM q_new:41) │ │ └── filters - │ │ ├── grandchild.c:29 = c:31 - │ │ └── grandchild.q:30 = q:32 + │ │ ├── grandchild.c:35 = c:38 + │ │ └── grandchild.q:36 = q:39 │ └── projections - │ ├── 10 [as=c_new:35] - │ └── 11 [as=q_new:36] + │ ├── 10 [as=c_new:42] + │ └── 11 [as=q_new:43] └── f-k-checks └── f-k-checks-item: grandchild(c,q) -> child_multi(c,q) └── anti-join (hash) - ├── columns: c_new:37!null q_new:38!null + ├── columns: c_new:44!null q_new:45!null ├── with-scan &3 - │ ├── columns: c_new:37!null q_new:38!null + │ ├── columns: c_new:44!null q_new:45!null │ └── mapping: - │ ├── c_new:35 => c_new:37 - │ └── q_new:36 => q_new:38 + │ ├── c_new:42 => c_new:44 + │ └── q_new:43 => q_new:45 ├── scan child_multi - │ └── columns: child_multi.c:39!null child_multi.q:41 + │ └── columns: child_multi.c:46!null child_multi.q:48 └── filters - ├── c_new:37 = child_multi.c:39 - └── q_new:38 = child_multi.q:41 + ├── c_new:44 = child_multi.c:46 + └── q_new:45 = child_multi.q:48 build-cascades UPSERT INTO parent_multi VALUES (1, 10, 10), (2, 20, 20) @@ -358,132 +358,132 @@ UPSERT INTO parent_multi VALUES (1, 10, 10), (2, 20, 20) root ├── upsert parent_multi │ ├── columns: - │ ├── canary column: 7 - │ ├── fetch columns: pk:7 p:8 q:9 + │ ├── canary column: 8 + │ ├── fetch columns: pk:8 p:9 q:10 │ ├── insert-mapping: - │ │ ├── column1:4 => pk:1 - │ │ ├── column2:5 => p:2 - │ │ └── column3:6 => q:3 + │ │ ├── column1:5 => pk:1 + │ │ ├── column2:6 => p:2 + │ │ └── column3:7 => q:3 │ ├── update-mapping: - │ │ ├── column2:5 => p:2 - │ │ └── column3:6 => q:3 + │ │ ├── column2:6 => p:2 + │ │ └── column3:7 => q:3 │ ├── input binding: &1 │ ├── cascades │ │ └── fk │ └── project - │ ├── columns: upsert_pk:10 column1:4!null column2:5!null column3:6!null pk:7 p:8 q:9 + │ ├── columns: upsert_pk:12 column1:5!null column2:6!null column3:7!null pk:8 p:9 q:10 crdb_internal_mvcc_timestamp:11 │ ├── left-join (hash) - │ │ ├── columns: column1:4!null column2:5!null column3:6!null pk:7 p:8 q:9 + │ │ ├── columns: column1:5!null column2:6!null column3:7!null pk:8 p:9 q:10 crdb_internal_mvcc_timestamp:11 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null - │ │ │ ├── grouping columns: column1:4!null + │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null + │ │ │ ├── grouping columns: column1:5!null │ │ │ ├── values - │ │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null + │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null │ │ │ │ ├── (1, 10, 10) │ │ │ │ └── (2, 20, 20) │ │ │ └── aggregations - │ │ │ ├── first-agg [as=column2:5] - │ │ │ │ └── column2:5 - │ │ │ └── first-agg [as=column3:6] - │ │ │ └── column3:6 + │ │ │ ├── first-agg [as=column2:6] + │ │ │ │ └── column2:6 + │ │ │ └── first-agg [as=column3:7] + │ │ │ └── column3:7 │ │ ├── scan parent_multi - │ │ │ └── columns: pk:7!null p:8 q:9 + │ │ │ └── columns: pk:8!null p:9 q:10 crdb_internal_mvcc_timestamp:11 │ │ └── filters - │ │ └── column1:4 = pk:7 + │ │ └── column1:5 = pk:8 │ └── projections - │ └── CASE WHEN pk:7 IS NULL THEN column1:4 ELSE pk:7 END [as=upsert_pk:10] + │ └── CASE WHEN pk:8 IS NULL THEN column1:5 ELSE pk:8 END [as=upsert_pk:12] └── cascade ├── update child_multi │ ├── columns: - │ ├── fetch columns: c:14 child_multi.p:15 child_multi.q:16 + │ ├── fetch columns: c:17 child_multi.p:18 child_multi.q:19 │ ├── update-mapping: - │ │ ├── p_new:21 => child_multi.p:12 - │ │ └── q_new:22 => child_multi.q:13 + │ │ ├── p_new:25 => child_multi.p:14 + │ │ └── q_new:26 => child_multi.q:15 │ ├── input binding: &2 │ ├── cascades │ │ └── fk2 │ ├── project - │ │ ├── columns: p_new:21!null q_new:22!null c:14!null child_multi.p:15!null child_multi.q:16!null p:17!null q:18!null column2:19!null column3:20!null + │ │ ├── columns: p_new:25!null q_new:26!null c:17!null child_multi.p:18!null child_multi.q:19!null child_multi.crdb_internal_mvcc_timestamp:20 p:21!null q:22!null column2:23!null column3:24!null │ │ ├── inner-join (hash) - │ │ │ ├── columns: c:14!null child_multi.p:15!null child_multi.q:16!null p:17!null q:18!null column2:19!null column3:20!null + │ │ │ ├── columns: c:17!null child_multi.p:18!null child_multi.q:19!null child_multi.crdb_internal_mvcc_timestamp:20 p:21!null q:22!null column2:23!null column3:24!null │ │ │ ├── scan child_multi - │ │ │ │ └── columns: c:14!null child_multi.p:15 child_multi.q:16 + │ │ │ │ └── columns: c:17!null child_multi.p:18 child_multi.q:19 child_multi.crdb_internal_mvcc_timestamp:20 │ │ │ ├── select - │ │ │ │ ├── columns: p:17 q:18 column2:19!null column3:20!null + │ │ │ │ ├── columns: p:21 q:22 column2:23!null column3:24!null │ │ │ │ ├── with-scan &1 - │ │ │ │ │ ├── columns: p:17 q:18 column2:19!null column3:20!null + │ │ │ │ │ ├── columns: p:21 q:22 column2:23!null column3:24!null │ │ │ │ │ └── mapping: - │ │ │ │ │ ├── parent_multi.p:8 => p:17 - │ │ │ │ │ ├── parent_multi.q:9 => q:18 - │ │ │ │ │ ├── column2:5 => column2:19 - │ │ │ │ │ └── column3:6 => column3:20 + │ │ │ │ │ ├── parent_multi.p:9 => p:21 + │ │ │ │ │ ├── parent_multi.q:10 => q:22 + │ │ │ │ │ ├── column2:6 => column2:23 + │ │ │ │ │ └── column3:7 => column3:24 │ │ │ │ └── filters - │ │ │ │ └── (p:17 IS DISTINCT FROM column2:19) OR (q:18 IS DISTINCT FROM column3:20) + │ │ │ │ └── (p:21 IS DISTINCT FROM column2:23) OR (q:22 IS DISTINCT FROM column3:24) │ │ │ └── filters - │ │ │ ├── child_multi.p:15 = p:17 - │ │ │ └── child_multi.q:16 = q:18 + │ │ │ ├── child_multi.p:18 = p:21 + │ │ │ └── child_multi.q:19 = q:22 │ │ └── projections - │ │ ├── 0 [as=p_new:21] - │ │ └── 1 [as=q_new:22] + │ │ ├── 0 [as=p_new:25] + │ │ └── 1 [as=q_new:26] │ └── f-k-checks │ └── f-k-checks-item: child_multi(p,q) -> parent_multi(p,q) │ └── anti-join (hash) - │ ├── columns: p_new:23!null q_new:24!null + │ ├── columns: p_new:27!null q_new:28!null │ ├── with-scan &2 - │ │ ├── columns: p_new:23!null q_new:24!null + │ │ ├── columns: p_new:27!null q_new:28!null │ │ └── mapping: - │ │ ├── p_new:21 => p_new:23 - │ │ └── q_new:22 => q_new:24 + │ │ ├── p_new:25 => p_new:27 + │ │ └── q_new:26 => q_new:28 │ ├── scan parent_multi - │ │ └── columns: parent_multi.p:26 parent_multi.q:27 + │ │ └── columns: parent_multi.p:30 parent_multi.q:31 │ └── filters - │ ├── p_new:23 = parent_multi.p:26 - │ └── q_new:24 = parent_multi.q:27 + │ ├── p_new:27 = parent_multi.p:30 + │ └── q_new:28 = parent_multi.q:31 └── cascade └── update grandchild ├── columns: - ├── fetch columns: g:31 grandchild.c:32 grandchild.q:33 + ├── fetch columns: g:37 grandchild.c:38 grandchild.q:39 ├── update-mapping: - │ ├── c_new:38 => grandchild.c:29 - │ └── q_new:39 => grandchild.q:30 + │ ├── c_new:45 => grandchild.c:34 + │ └── q_new:46 => grandchild.q:35 ├── input binding: &3 ├── project - │ ├── columns: c_new:38!null q_new:39!null g:31!null grandchild.c:32!null grandchild.q:33!null c:34!null q:35!null c:36!null q_new:37!null + │ ├── columns: c_new:45!null q_new:46!null g:37!null grandchild.c:38!null grandchild.q:39!null grandchild.crdb_internal_mvcc_timestamp:40 c:41!null q:42!null c:43!null q_new:44!null │ ├── inner-join (hash) - │ │ ├── columns: g:31!null grandchild.c:32!null grandchild.q:33!null c:34!null q:35!null c:36!null q_new:37!null + │ │ ├── columns: g:37!null grandchild.c:38!null grandchild.q:39!null grandchild.crdb_internal_mvcc_timestamp:40 c:41!null q:42!null c:43!null q_new:44!null │ │ ├── scan grandchild - │ │ │ └── columns: g:31!null grandchild.c:32 grandchild.q:33 + │ │ │ └── columns: g:37!null grandchild.c:38 grandchild.q:39 grandchild.crdb_internal_mvcc_timestamp:40 │ │ ├── select - │ │ │ ├── columns: c:34!null q:35!null c:36!null q_new:37!null + │ │ │ ├── columns: c:41!null q:42!null c:43!null q_new:44!null │ │ │ ├── with-scan &2 - │ │ │ │ ├── columns: c:34!null q:35!null c:36!null q_new:37!null + │ │ │ │ ├── columns: c:41!null q:42!null c:43!null q_new:44!null │ │ │ │ └── mapping: - │ │ │ │ ├── child_multi.c:14 => c:34 - │ │ │ │ ├── child_multi.q:16 => q:35 - │ │ │ │ ├── child_multi.c:14 => c:36 - │ │ │ │ └── q_new:22 => q_new:37 + │ │ │ │ ├── child_multi.c:17 => c:41 + │ │ │ │ ├── child_multi.q:19 => q:42 + │ │ │ │ ├── child_multi.c:17 => c:43 + │ │ │ │ └── q_new:26 => q_new:44 │ │ │ └── filters - │ │ │ └── (c:34 IS DISTINCT FROM c:36) OR (q:35 IS DISTINCT FROM q_new:37) + │ │ │ └── (c:41 IS DISTINCT FROM c:43) OR (q:42 IS DISTINCT FROM q_new:44) │ │ └── filters - │ │ ├── grandchild.c:32 = c:34 - │ │ └── grandchild.q:33 = q:35 + │ │ ├── grandchild.c:38 = c:41 + │ │ └── grandchild.q:39 = q:42 │ └── projections - │ ├── 10 [as=c_new:38] - │ └── 11 [as=q_new:39] + │ ├── 10 [as=c_new:45] + │ └── 11 [as=q_new:46] └── f-k-checks └── f-k-checks-item: grandchild(c,q) -> child_multi(c,q) └── anti-join (hash) - ├── columns: c_new:40!null q_new:41!null + ├── columns: c_new:47!null q_new:48!null ├── with-scan &3 - │ ├── columns: c_new:40!null q_new:41!null + │ ├── columns: c_new:47!null q_new:48!null │ └── mapping: - │ ├── c_new:38 => c_new:40 - │ └── q_new:39 => q_new:41 + │ ├── c_new:45 => c_new:47 + │ └── q_new:46 => q_new:48 ├── scan child_multi - │ └── columns: child_multi.c:42!null child_multi.q:44 + │ └── columns: child_multi.c:49!null child_multi.q:51 └── filters - ├── c_new:40 = child_multi.c:42 - └── q_new:41 = child_multi.q:44 + ├── c_new:47 = child_multi.c:49 + └── q_new:48 = child_multi.q:51 # Upsert that only touches one of the FK columns. build-cascades @@ -492,136 +492,136 @@ UPSERT INTO parent_multi(pk, p) VALUES (1, 10), (2, 20) root ├── upsert parent_multi │ ├── columns: - │ ├── canary column: 7 - │ ├── fetch columns: pk:7 p:8 q:9 + │ ├── canary column: 8 + │ ├── fetch columns: pk:8 p:9 q:10 │ ├── insert-mapping: - │ │ ├── column1:4 => pk:1 - │ │ ├── column2:5 => p:2 - │ │ └── column6:6 => q:3 + │ │ ├── column1:5 => pk:1 + │ │ ├── column2:6 => p:2 + │ │ └── column7:7 => q:3 │ ├── update-mapping: - │ │ └── column2:5 => p:2 + │ │ └── column2:6 => p:2 │ ├── input binding: &1 │ ├── cascades │ │ └── fk │ └── project - │ ├── columns: upsert_pk:10 upsert_q:11 column1:4!null column2:5!null column6:6 pk:7 p:8 q:9 + │ ├── columns: upsert_pk:12 upsert_q:13 column1:5!null column2:6!null column7:7 pk:8 p:9 q:10 crdb_internal_mvcc_timestamp:11 │ ├── left-join (hash) - │ │ ├── columns: column1:4!null column2:5!null column6:6 pk:7 p:8 q:9 + │ │ ├── columns: column1:5!null column2:6!null column7:7 pk:8 p:9 q:10 crdb_internal_mvcc_timestamp:11 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: column1:4!null column2:5!null column6:6 - │ │ │ ├── grouping columns: column1:4!null + │ │ │ ├── columns: column1:5!null column2:6!null column7:7 + │ │ │ ├── grouping columns: column1:5!null │ │ │ ├── project - │ │ │ │ ├── columns: column6:6 column1:4!null column2:5!null + │ │ │ │ ├── columns: column7:7 column1:5!null column2:6!null │ │ │ │ ├── values - │ │ │ │ │ ├── columns: column1:4!null column2:5!null + │ │ │ │ │ ├── columns: column1:5!null column2:6!null │ │ │ │ │ ├── (1, 10) │ │ │ │ │ └── (2, 20) │ │ │ │ └── projections - │ │ │ │ └── NULL::INT8 [as=column6:6] + │ │ │ │ └── NULL::INT8 [as=column7:7] │ │ │ └── aggregations - │ │ │ ├── first-agg [as=column2:5] - │ │ │ │ └── column2:5 - │ │ │ └── first-agg [as=column6:6] - │ │ │ └── column6:6 + │ │ │ ├── first-agg [as=column2:6] + │ │ │ │ └── column2:6 + │ │ │ └── first-agg [as=column7:7] + │ │ │ └── column7:7 │ │ ├── scan parent_multi - │ │ │ └── columns: pk:7!null p:8 q:9 + │ │ │ └── columns: pk:8!null p:9 q:10 crdb_internal_mvcc_timestamp:11 │ │ └── filters - │ │ └── column1:4 = pk:7 + │ │ └── column1:5 = pk:8 │ └── projections - │ ├── CASE WHEN pk:7 IS NULL THEN column1:4 ELSE pk:7 END [as=upsert_pk:10] - │ └── CASE WHEN pk:7 IS NULL THEN column6:6 ELSE q:9 END [as=upsert_q:11] + │ ├── CASE WHEN pk:8 IS NULL THEN column1:5 ELSE pk:8 END [as=upsert_pk:12] + │ └── CASE WHEN pk:8 IS NULL THEN column7:7 ELSE q:10 END [as=upsert_q:13] └── cascade ├── update child_multi │ ├── columns: - │ ├── fetch columns: c:15 child_multi.p:16 child_multi.q:17 + │ ├── fetch columns: c:18 child_multi.p:19 child_multi.q:20 │ ├── update-mapping: - │ │ ├── p_new:22 => child_multi.p:13 - │ │ └── q_new:23 => child_multi.q:14 + │ │ ├── p_new:26 => child_multi.p:15 + │ │ └── q_new:27 => child_multi.q:16 │ ├── input binding: &2 │ ├── cascades │ │ └── fk2 │ ├── project - │ │ ├── columns: p_new:22!null q_new:23!null c:15!null child_multi.p:16!null child_multi.q:17!null p:18!null q:19!null column2:20!null q:21 + │ │ ├── columns: p_new:26!null q_new:27!null c:18!null child_multi.p:19!null child_multi.q:20!null child_multi.crdb_internal_mvcc_timestamp:21 p:22!null q:23!null column2:24!null q:25 │ │ ├── inner-join (hash) - │ │ │ ├── columns: c:15!null child_multi.p:16!null child_multi.q:17!null p:18!null q:19!null column2:20!null q:21 + │ │ │ ├── columns: c:18!null child_multi.p:19!null child_multi.q:20!null child_multi.crdb_internal_mvcc_timestamp:21 p:22!null q:23!null column2:24!null q:25 │ │ │ ├── scan child_multi - │ │ │ │ └── columns: c:15!null child_multi.p:16 child_multi.q:17 + │ │ │ │ └── columns: c:18!null child_multi.p:19 child_multi.q:20 child_multi.crdb_internal_mvcc_timestamp:21 │ │ │ ├── select - │ │ │ │ ├── columns: p:18 q:19 column2:20!null q:21 + │ │ │ │ ├── columns: p:22 q:23 column2:24!null q:25 │ │ │ │ ├── with-scan &1 - │ │ │ │ │ ├── columns: p:18 q:19 column2:20!null q:21 + │ │ │ │ │ ├── columns: p:22 q:23 column2:24!null q:25 │ │ │ │ │ └── mapping: - │ │ │ │ │ ├── parent_multi.p:8 => p:18 - │ │ │ │ │ ├── parent_multi.q:9 => q:19 - │ │ │ │ │ ├── column2:5 => column2:20 - │ │ │ │ │ └── parent_multi.q:9 => q:21 + │ │ │ │ │ ├── parent_multi.p:9 => p:22 + │ │ │ │ │ ├── parent_multi.q:10 => q:23 + │ │ │ │ │ ├── column2:6 => column2:24 + │ │ │ │ │ └── parent_multi.q:10 => q:25 │ │ │ │ └── filters - │ │ │ │ └── (p:18 IS DISTINCT FROM column2:20) OR (q:19 IS DISTINCT FROM q:21) + │ │ │ │ └── (p:22 IS DISTINCT FROM column2:24) OR (q:23 IS DISTINCT FROM q:25) │ │ │ └── filters - │ │ │ ├── child_multi.p:16 = p:18 - │ │ │ └── child_multi.q:17 = q:19 + │ │ │ ├── child_multi.p:19 = p:22 + │ │ │ └── child_multi.q:20 = q:23 │ │ └── projections - │ │ ├── 0 [as=p_new:22] - │ │ └── 1 [as=q_new:23] + │ │ ├── 0 [as=p_new:26] + │ │ └── 1 [as=q_new:27] │ └── f-k-checks │ └── f-k-checks-item: child_multi(p,q) -> parent_multi(p,q) │ └── anti-join (hash) - │ ├── columns: p_new:24!null q_new:25!null + │ ├── columns: p_new:28!null q_new:29!null │ ├── with-scan &2 - │ │ ├── columns: p_new:24!null q_new:25!null + │ │ ├── columns: p_new:28!null q_new:29!null │ │ └── mapping: - │ │ ├── p_new:22 => p_new:24 - │ │ └── q_new:23 => q_new:25 + │ │ ├── p_new:26 => p_new:28 + │ │ └── q_new:27 => q_new:29 │ ├── scan parent_multi - │ │ └── columns: parent_multi.p:27 parent_multi.q:28 + │ │ └── columns: parent_multi.p:31 parent_multi.q:32 │ └── filters - │ ├── p_new:24 = parent_multi.p:27 - │ └── q_new:25 = parent_multi.q:28 + │ ├── p_new:28 = parent_multi.p:31 + │ └── q_new:29 = parent_multi.q:32 └── cascade └── update grandchild ├── columns: - ├── fetch columns: g:32 grandchild.c:33 grandchild.q:34 + ├── fetch columns: g:38 grandchild.c:39 grandchild.q:40 ├── update-mapping: - │ ├── c_new:39 => grandchild.c:30 - │ └── q_new:40 => grandchild.q:31 + │ ├── c_new:46 => grandchild.c:35 + │ └── q_new:47 => grandchild.q:36 ├── input binding: &3 ├── project - │ ├── columns: c_new:39!null q_new:40!null g:32!null grandchild.c:33!null grandchild.q:34!null c:35!null q:36!null c:37!null q_new:38!null + │ ├── columns: c_new:46!null q_new:47!null g:38!null grandchild.c:39!null grandchild.q:40!null grandchild.crdb_internal_mvcc_timestamp:41 c:42!null q:43!null c:44!null q_new:45!null │ ├── inner-join (hash) - │ │ ├── columns: g:32!null grandchild.c:33!null grandchild.q:34!null c:35!null q:36!null c:37!null q_new:38!null + │ │ ├── columns: g:38!null grandchild.c:39!null grandchild.q:40!null grandchild.crdb_internal_mvcc_timestamp:41 c:42!null q:43!null c:44!null q_new:45!null │ │ ├── scan grandchild - │ │ │ └── columns: g:32!null grandchild.c:33 grandchild.q:34 + │ │ │ └── columns: g:38!null grandchild.c:39 grandchild.q:40 grandchild.crdb_internal_mvcc_timestamp:41 │ │ ├── select - │ │ │ ├── columns: c:35!null q:36!null c:37!null q_new:38!null + │ │ │ ├── columns: c:42!null q:43!null c:44!null q_new:45!null │ │ │ ├── with-scan &2 - │ │ │ │ ├── columns: c:35!null q:36!null c:37!null q_new:38!null + │ │ │ │ ├── columns: c:42!null q:43!null c:44!null q_new:45!null │ │ │ │ └── mapping: - │ │ │ │ ├── child_multi.c:15 => c:35 - │ │ │ │ ├── child_multi.q:17 => q:36 - │ │ │ │ ├── child_multi.c:15 => c:37 - │ │ │ │ └── q_new:23 => q_new:38 + │ │ │ │ ├── child_multi.c:18 => c:42 + │ │ │ │ ├── child_multi.q:20 => q:43 + │ │ │ │ ├── child_multi.c:18 => c:44 + │ │ │ │ └── q_new:27 => q_new:45 │ │ │ └── filters - │ │ │ └── (c:35 IS DISTINCT FROM c:37) OR (q:36 IS DISTINCT FROM q_new:38) + │ │ │ └── (c:42 IS DISTINCT FROM c:44) OR (q:43 IS DISTINCT FROM q_new:45) │ │ └── filters - │ │ ├── grandchild.c:33 = c:35 - │ │ └── grandchild.q:34 = q:36 + │ │ ├── grandchild.c:39 = c:42 + │ │ └── grandchild.q:40 = q:43 │ └── projections - │ ├── 10 [as=c_new:39] - │ └── 11 [as=q_new:40] + │ ├── 10 [as=c_new:46] + │ └── 11 [as=q_new:47] └── f-k-checks └── f-k-checks-item: grandchild(c,q) -> child_multi(c,q) └── anti-join (hash) - ├── columns: c_new:41!null q_new:42!null + ├── columns: c_new:48!null q_new:49!null ├── with-scan &3 - │ ├── columns: c_new:41!null q_new:42!null + │ ├── columns: c_new:48!null q_new:49!null │ └── mapping: - │ ├── c_new:39 => c_new:41 - │ └── q_new:40 => q_new:42 + │ ├── c_new:46 => c_new:48 + │ └── q_new:47 => q_new:49 ├── scan child_multi - │ └── columns: child_multi.c:43!null child_multi.q:45 + │ └── columns: child_multi.c:50!null child_multi.q:52 └── filters - ├── c_new:41 = child_multi.c:43 - └── q_new:42 = child_multi.q:45 + ├── c_new:48 = child_multi.c:50 + └── q_new:49 = child_multi.q:52 build-cascades INSERT INTO parent_multi VALUES (1, 10, 10), (2, 20, 20) ON CONFLICT (p,q) DO UPDATE SET p = 100 @@ -629,133 +629,133 @@ INSERT INTO parent_multi VALUES (1, 10, 10), (2, 20, 20) ON CONFLICT (p,q) DO UP root ├── upsert parent_multi │ ├── columns: - │ ├── canary column: 7 - │ ├── fetch columns: pk:7 p:8 q:9 + │ ├── canary column: 8 + │ ├── fetch columns: pk:8 p:9 q:10 │ ├── insert-mapping: - │ │ ├── column1:4 => pk:1 - │ │ ├── column2:5 => p:2 - │ │ └── column3:6 => q:3 + │ │ ├── column1:5 => pk:1 + │ │ ├── column2:6 => p:2 + │ │ └── column3:7 => q:3 │ ├── update-mapping: - │ │ └── upsert_p:12 => p:2 + │ │ └── upsert_p:14 => p:2 │ ├── input binding: &1 │ ├── cascades │ │ └── fk │ └── project - │ ├── columns: upsert_pk:11 upsert_p:12!null upsert_q:13 column1:4!null column2:5!null column3:6!null pk:7 p:8 q:9 p_new:10!null + │ ├── columns: upsert_pk:13 upsert_p:14!null upsert_q:15 column1:5!null column2:6!null column3:7!null pk:8 p:9 q:10 crdb_internal_mvcc_timestamp:11 p_new:12!null │ ├── project - │ │ ├── columns: p_new:10!null column1:4!null column2:5!null column3:6!null pk:7 p:8 q:9 + │ │ ├── columns: p_new:12!null column1:5!null column2:6!null column3:7!null pk:8 p:9 q:10 crdb_internal_mvcc_timestamp:11 │ │ ├── left-join (hash) - │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null pk:7 p:8 q:9 + │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null pk:8 p:9 q:10 crdb_internal_mvcc_timestamp:11 │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null - │ │ │ │ ├── grouping columns: column2:5!null column3:6!null + │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null + │ │ │ │ ├── grouping columns: column2:6!null column3:7!null │ │ │ │ ├── values - │ │ │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null + │ │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null │ │ │ │ │ ├── (1, 10, 10) │ │ │ │ │ └── (2, 20, 20) │ │ │ │ └── aggregations - │ │ │ │ └── first-agg [as=column1:4] - │ │ │ │ └── column1:4 + │ │ │ │ └── first-agg [as=column1:5] + │ │ │ │ └── column1:5 │ │ │ ├── scan parent_multi - │ │ │ │ └── columns: pk:7!null p:8 q:9 + │ │ │ │ └── columns: pk:8!null p:9 q:10 crdb_internal_mvcc_timestamp:11 │ │ │ └── filters - │ │ │ ├── column2:5 = p:8 - │ │ │ └── column3:6 = q:9 + │ │ │ ├── column2:6 = p:9 + │ │ │ └── column3:7 = q:10 │ │ └── projections - │ │ └── 100 [as=p_new:10] + │ │ └── 100 [as=p_new:12] │ └── projections - │ ├── CASE WHEN pk:7 IS NULL THEN column1:4 ELSE pk:7 END [as=upsert_pk:11] - │ ├── CASE WHEN pk:7 IS NULL THEN column2:5 ELSE p_new:10 END [as=upsert_p:12] - │ └── CASE WHEN pk:7 IS NULL THEN column3:6 ELSE q:9 END [as=upsert_q:13] + │ ├── CASE WHEN pk:8 IS NULL THEN column1:5 ELSE pk:8 END [as=upsert_pk:13] + │ ├── CASE WHEN pk:8 IS NULL THEN column2:6 ELSE p_new:12 END [as=upsert_p:14] + │ └── CASE WHEN pk:8 IS NULL THEN column3:7 ELSE q:10 END [as=upsert_q:15] └── cascade ├── update child_multi │ ├── columns: - │ ├── fetch columns: c:17 child_multi.p:18 child_multi.q:19 + │ ├── fetch columns: c:20 child_multi.p:21 child_multi.q:22 │ ├── update-mapping: - │ │ ├── p_new:24 => child_multi.p:15 - │ │ └── q_new:25 => child_multi.q:16 + │ │ ├── p_new:28 => child_multi.p:17 + │ │ └── q_new:29 => child_multi.q:18 │ ├── input binding: &2 │ ├── cascades │ │ └── fk2 │ ├── project - │ │ ├── columns: p_new:24!null q_new:25!null c:17!null child_multi.p:18!null child_multi.q:19!null p:20!null q:21!null upsert_p:22!null q:23 + │ │ ├── columns: p_new:28!null q_new:29!null c:20!null child_multi.p:21!null child_multi.q:22!null child_multi.crdb_internal_mvcc_timestamp:23 p:24!null q:25!null upsert_p:26!null q:27 │ │ ├── inner-join (hash) - │ │ │ ├── columns: c:17!null child_multi.p:18!null child_multi.q:19!null p:20!null q:21!null upsert_p:22!null q:23 + │ │ │ ├── columns: c:20!null child_multi.p:21!null child_multi.q:22!null child_multi.crdb_internal_mvcc_timestamp:23 p:24!null q:25!null upsert_p:26!null q:27 │ │ │ ├── scan child_multi - │ │ │ │ └── columns: c:17!null child_multi.p:18 child_multi.q:19 + │ │ │ │ └── columns: c:20!null child_multi.p:21 child_multi.q:22 child_multi.crdb_internal_mvcc_timestamp:23 │ │ │ ├── select - │ │ │ │ ├── columns: p:20 q:21 upsert_p:22!null q:23 + │ │ │ │ ├── columns: p:24 q:25 upsert_p:26!null q:27 │ │ │ │ ├── with-scan &1 - │ │ │ │ │ ├── columns: p:20 q:21 upsert_p:22!null q:23 + │ │ │ │ │ ├── columns: p:24 q:25 upsert_p:26!null q:27 │ │ │ │ │ └── mapping: - │ │ │ │ │ ├── parent_multi.p:8 => p:20 - │ │ │ │ │ ├── parent_multi.q:9 => q:21 - │ │ │ │ │ ├── upsert_p:12 => upsert_p:22 - │ │ │ │ │ └── parent_multi.q:9 => q:23 + │ │ │ │ │ ├── parent_multi.p:9 => p:24 + │ │ │ │ │ ├── parent_multi.q:10 => q:25 + │ │ │ │ │ ├── upsert_p:14 => upsert_p:26 + │ │ │ │ │ └── parent_multi.q:10 => q:27 │ │ │ │ └── filters - │ │ │ │ └── (p:20 IS DISTINCT FROM upsert_p:22) OR (q:21 IS DISTINCT FROM q:23) + │ │ │ │ └── (p:24 IS DISTINCT FROM upsert_p:26) OR (q:25 IS DISTINCT FROM q:27) │ │ │ └── filters - │ │ │ ├── child_multi.p:18 = p:20 - │ │ │ └── child_multi.q:19 = q:21 + │ │ │ ├── child_multi.p:21 = p:24 + │ │ │ └── child_multi.q:22 = q:25 │ │ └── projections - │ │ ├── 0 [as=p_new:24] - │ │ └── 1 [as=q_new:25] + │ │ ├── 0 [as=p_new:28] + │ │ └── 1 [as=q_new:29] │ └── f-k-checks │ └── f-k-checks-item: child_multi(p,q) -> parent_multi(p,q) │ └── anti-join (hash) - │ ├── columns: p_new:26!null q_new:27!null + │ ├── columns: p_new:30!null q_new:31!null │ ├── with-scan &2 - │ │ ├── columns: p_new:26!null q_new:27!null + │ │ ├── columns: p_new:30!null q_new:31!null │ │ └── mapping: - │ │ ├── p_new:24 => p_new:26 - │ │ └── q_new:25 => q_new:27 + │ │ ├── p_new:28 => p_new:30 + │ │ └── q_new:29 => q_new:31 │ ├── scan parent_multi - │ │ └── columns: parent_multi.p:29 parent_multi.q:30 + │ │ └── columns: parent_multi.p:33 parent_multi.q:34 │ └── filters - │ ├── p_new:26 = parent_multi.p:29 - │ └── q_new:27 = parent_multi.q:30 + │ ├── p_new:30 = parent_multi.p:33 + │ └── q_new:31 = parent_multi.q:34 └── cascade └── update grandchild ├── columns: - ├── fetch columns: g:34 grandchild.c:35 grandchild.q:36 + ├── fetch columns: g:40 grandchild.c:41 grandchild.q:42 ├── update-mapping: - │ ├── c_new:41 => grandchild.c:32 - │ └── q_new:42 => grandchild.q:33 + │ ├── c_new:48 => grandchild.c:37 + │ └── q_new:49 => grandchild.q:38 ├── input binding: &3 ├── project - │ ├── columns: c_new:41!null q_new:42!null g:34!null grandchild.c:35!null grandchild.q:36!null c:37!null q:38!null c:39!null q_new:40!null + │ ├── columns: c_new:48!null q_new:49!null g:40!null grandchild.c:41!null grandchild.q:42!null grandchild.crdb_internal_mvcc_timestamp:43 c:44!null q:45!null c:46!null q_new:47!null │ ├── inner-join (hash) - │ │ ├── columns: g:34!null grandchild.c:35!null grandchild.q:36!null c:37!null q:38!null c:39!null q_new:40!null + │ │ ├── columns: g:40!null grandchild.c:41!null grandchild.q:42!null grandchild.crdb_internal_mvcc_timestamp:43 c:44!null q:45!null c:46!null q_new:47!null │ │ ├── scan grandchild - │ │ │ └── columns: g:34!null grandchild.c:35 grandchild.q:36 + │ │ │ └── columns: g:40!null grandchild.c:41 grandchild.q:42 grandchild.crdb_internal_mvcc_timestamp:43 │ │ ├── select - │ │ │ ├── columns: c:37!null q:38!null c:39!null q_new:40!null + │ │ │ ├── columns: c:44!null q:45!null c:46!null q_new:47!null │ │ │ ├── with-scan &2 - │ │ │ │ ├── columns: c:37!null q:38!null c:39!null q_new:40!null + │ │ │ │ ├── columns: c:44!null q:45!null c:46!null q_new:47!null │ │ │ │ └── mapping: - │ │ │ │ ├── child_multi.c:17 => c:37 - │ │ │ │ ├── child_multi.q:19 => q:38 - │ │ │ │ ├── child_multi.c:17 => c:39 - │ │ │ │ └── q_new:25 => q_new:40 + │ │ │ │ ├── child_multi.c:20 => c:44 + │ │ │ │ ├── child_multi.q:22 => q:45 + │ │ │ │ ├── child_multi.c:20 => c:46 + │ │ │ │ └── q_new:29 => q_new:47 │ │ │ └── filters - │ │ │ └── (c:37 IS DISTINCT FROM c:39) OR (q:38 IS DISTINCT FROM q_new:40) + │ │ │ └── (c:44 IS DISTINCT FROM c:46) OR (q:45 IS DISTINCT FROM q_new:47) │ │ └── filters - │ │ ├── grandchild.c:35 = c:37 - │ │ └── grandchild.q:36 = q:38 + │ │ ├── grandchild.c:41 = c:44 + │ │ └── grandchild.q:42 = q:45 │ └── projections - │ ├── 10 [as=c_new:41] - │ └── 11 [as=q_new:42] + │ ├── 10 [as=c_new:48] + │ └── 11 [as=q_new:49] └── f-k-checks └── f-k-checks-item: grandchild(c,q) -> child_multi(c,q) └── anti-join (hash) - ├── columns: c_new:43!null q_new:44!null + ├── columns: c_new:50!null q_new:51!null ├── with-scan &3 - │ ├── columns: c_new:43!null q_new:44!null + │ ├── columns: c_new:50!null q_new:51!null │ └── mapping: - │ ├── c_new:41 => c_new:43 - │ └── q_new:42 => q_new:44 + │ ├── c_new:48 => c_new:50 + │ └── q_new:49 => q_new:51 ├── scan child_multi - │ └── columns: child_multi.c:45!null child_multi.q:47 + │ └── columns: child_multi.c:52!null child_multi.q:54 └── filters - ├── c_new:43 = child_multi.c:45 - └── q_new:44 = child_multi.q:47 + ├── c_new:50 = child_multi.c:52 + └── q_new:51 = child_multi.q:54 diff --git a/pkg/sql/opt/optbuilder/testdata/fk-on-update-set-null b/pkg/sql/opt/optbuilder/testdata/fk-on-update-set-null index 46bdaf40a06c..64997edf5e49 100644 --- a/pkg/sql/opt/optbuilder/testdata/fk-on-update-set-null +++ b/pkg/sql/opt/optbuilder/testdata/fk-on-update-set-null @@ -12,47 +12,47 @@ UPDATE parent SET p = p * 10 WHERE p > 1 root ├── update parent │ ├── columns: - │ ├── fetch columns: p:2 + │ ├── fetch columns: p:3 │ ├── update-mapping: - │ │ └── p_new:3 => p:1 + │ │ └── p_new:5 => p:1 │ ├── input binding: &1 │ ├── cascades │ │ └── fk_p_ref_parent │ └── project - │ ├── columns: p_new:3!null p:2!null + │ ├── columns: p_new:5!null p:3!null crdb_internal_mvcc_timestamp:4 │ ├── select - │ │ ├── columns: p:2!null + │ │ ├── columns: p:3!null crdb_internal_mvcc_timestamp:4 │ │ ├── scan parent - │ │ │ └── columns: p:2!null + │ │ │ └── columns: p:3!null crdb_internal_mvcc_timestamp:4 │ │ └── filters - │ │ └── p:2 > 1 + │ │ └── p:3 > 1 │ └── projections - │ └── p:2 * 10 [as=p_new:3] + │ └── p:3 * 10 [as=p_new:5] └── cascade └── update child ├── columns: - ├── fetch columns: c:6 child.p:7 + ├── fetch columns: c:9 child.p:10 ├── update-mapping: - │ └── p_new:10 => child.p:5 + │ └── p_new:14 => child.p:7 └── project - ├── columns: p_new:10 c:6!null child.p:7!null p:8!null p_new:9!null + ├── columns: p_new:14 c:9!null child.p:10!null child.crdb_internal_mvcc_timestamp:11 p:12!null p_new:13!null ├── inner-join (hash) - │ ├── columns: c:6!null child.p:7!null p:8!null p_new:9!null + │ ├── columns: c:9!null child.p:10!null child.crdb_internal_mvcc_timestamp:11 p:12!null p_new:13!null │ ├── scan child - │ │ └── columns: c:6!null child.p:7!null + │ │ └── columns: c:9!null child.p:10!null child.crdb_internal_mvcc_timestamp:11 │ ├── select - │ │ ├── columns: p:8!null p_new:9!null + │ │ ├── columns: p:12!null p_new:13!null │ │ ├── with-scan &1 - │ │ │ ├── columns: p:8!null p_new:9!null + │ │ │ ├── columns: p:12!null p_new:13!null │ │ │ └── mapping: - │ │ │ ├── parent.p:2 => p:8 - │ │ │ └── p_new:3 => p_new:9 + │ │ │ ├── parent.p:3 => p:12 + │ │ │ └── p_new:5 => p_new:13 │ │ └── filters - │ │ └── p:8 IS DISTINCT FROM p_new:9 + │ │ └── p:12 IS DISTINCT FROM p_new:13 │ └── filters - │ └── child.p:7 = p:8 + │ └── child.p:10 = p:12 └── projections - └── NULL::INT8 [as=p_new:10] + └── NULL::INT8 [as=p_new:14] exec-ddl CREATE TABLE parent_multi ( @@ -80,53 +80,53 @@ UPDATE parent_multi SET p = p * 10, q = q + 1 WHERE pk > 1 root ├── update parent_multi │ ├── columns: - │ ├── fetch columns: pk:4 p:5 q:6 + │ ├── fetch columns: pk:5 p:6 q:7 │ ├── update-mapping: - │ │ ├── p_new:7 => p:2 - │ │ └── q_new:8 => q:3 + │ │ ├── p_new:9 => p:2 + │ │ └── q_new:10 => q:3 │ ├── input binding: &1 │ ├── cascades │ │ └── fk │ └── project - │ ├── columns: p_new:7 q_new:8 pk:4!null p:5 q:6 + │ ├── columns: p_new:9 q_new:10 pk:5!null p:6 q:7 crdb_internal_mvcc_timestamp:8 │ ├── select - │ │ ├── columns: pk:4!null p:5 q:6 + │ │ ├── columns: pk:5!null p:6 q:7 crdb_internal_mvcc_timestamp:8 │ │ ├── scan parent_multi - │ │ │ └── columns: pk:4!null p:5 q:6 + │ │ │ └── columns: pk:5!null p:6 q:7 crdb_internal_mvcc_timestamp:8 │ │ └── filters - │ │ └── pk:4 > 1 + │ │ └── pk:5 > 1 │ └── projections - │ ├── p:5 * 10 [as=p_new:7] - │ └── q:6 + 1 [as=q_new:8] + │ ├── p:6 * 10 [as=p_new:9] + │ └── q:7 + 1 [as=q_new:10] └── cascade └── update child_multi ├── columns: - ├── fetch columns: c:12 child_multi.p:13 child_multi.q:14 + ├── fetch columns: c:15 child_multi.p:16 child_multi.q:17 ├── update-mapping: - │ ├── p_new:19 => child_multi.p:10 - │ └── p_new:19 => child_multi.q:11 + │ ├── p_new:23 => child_multi.p:12 + │ └── p_new:23 => child_multi.q:13 └── project - ├── columns: p_new:19 c:12!null child_multi.p:13!null child_multi.q:14!null p:15!null q:16!null p_new:17 q_new:18 + ├── columns: p_new:23 c:15!null child_multi.p:16!null child_multi.q:17!null child_multi.crdb_internal_mvcc_timestamp:18 p:19!null q:20!null p_new:21 q_new:22 ├── inner-join (hash) - │ ├── columns: c:12!null child_multi.p:13!null child_multi.q:14!null p:15!null q:16!null p_new:17 q_new:18 + │ ├── columns: c:15!null child_multi.p:16!null child_multi.q:17!null child_multi.crdb_internal_mvcc_timestamp:18 p:19!null q:20!null p_new:21 q_new:22 │ ├── scan child_multi - │ │ └── columns: c:12!null child_multi.p:13 child_multi.q:14 + │ │ └── columns: c:15!null child_multi.p:16 child_multi.q:17 child_multi.crdb_internal_mvcc_timestamp:18 │ ├── select - │ │ ├── columns: p:15 q:16 p_new:17 q_new:18 + │ │ ├── columns: p:19 q:20 p_new:21 q_new:22 │ │ ├── with-scan &1 - │ │ │ ├── columns: p:15 q:16 p_new:17 q_new:18 + │ │ │ ├── columns: p:19 q:20 p_new:21 q_new:22 │ │ │ └── mapping: - │ │ │ ├── parent_multi.p:5 => p:15 - │ │ │ ├── parent_multi.q:6 => q:16 - │ │ │ ├── p_new:7 => p_new:17 - │ │ │ └── q_new:8 => q_new:18 + │ │ │ ├── parent_multi.p:6 => p:19 + │ │ │ ├── parent_multi.q:7 => q:20 + │ │ │ ├── p_new:9 => p_new:21 + │ │ │ └── q_new:10 => q_new:22 │ │ └── filters - │ │ └── (p:15 IS DISTINCT FROM p_new:17) OR (q:16 IS DISTINCT FROM q_new:18) + │ │ └── (p:19 IS DISTINCT FROM p_new:21) OR (q:20 IS DISTINCT FROM q_new:22) │ └── filters - │ ├── child_multi.p:13 = p:15 - │ └── child_multi.q:14 = q:16 + │ ├── child_multi.p:16 = p:19 + │ └── child_multi.q:17 = q:20 └── projections - └── NULL::INT8 [as=p_new:19] + └── NULL::INT8 [as=p_new:23] # Update only one of the two FK columns. build-cascades @@ -135,51 +135,51 @@ UPDATE parent_multi SET p = p * 10 WHERE p > 1 root ├── update parent_multi │ ├── columns: - │ ├── fetch columns: pk:4 p:5 q:6 + │ ├── fetch columns: pk:5 p:6 q:7 │ ├── update-mapping: - │ │ └── p_new:7 => p:2 + │ │ └── p_new:9 => p:2 │ ├── input binding: &1 │ ├── cascades │ │ └── fk │ └── project - │ ├── columns: p_new:7!null pk:4!null p:5!null q:6 + │ ├── columns: p_new:9!null pk:5!null p:6!null q:7 crdb_internal_mvcc_timestamp:8 │ ├── select - │ │ ├── columns: pk:4!null p:5!null q:6 + │ │ ├── columns: pk:5!null p:6!null q:7 crdb_internal_mvcc_timestamp:8 │ │ ├── scan parent_multi - │ │ │ └── columns: pk:4!null p:5 q:6 + │ │ │ └── columns: pk:5!null p:6 q:7 crdb_internal_mvcc_timestamp:8 │ │ └── filters - │ │ └── p:5 > 1 + │ │ └── p:6 > 1 │ └── projections - │ └── p:5 * 10 [as=p_new:7] + │ └── p:6 * 10 [as=p_new:9] └── cascade └── update child_multi ├── columns: - ├── fetch columns: c:11 child_multi.p:12 child_multi.q:13 + ├── fetch columns: c:14 child_multi.p:15 child_multi.q:16 ├── update-mapping: - │ ├── p_new:18 => child_multi.p:9 - │ └── p_new:18 => child_multi.q:10 + │ ├── p_new:22 => child_multi.p:11 + │ └── p_new:22 => child_multi.q:12 └── project - ├── columns: p_new:18 c:11!null child_multi.p:12!null child_multi.q:13!null p:14!null q:15!null p_new:16!null q:17 + ├── columns: p_new:22 c:14!null child_multi.p:15!null child_multi.q:16!null child_multi.crdb_internal_mvcc_timestamp:17 p:18!null q:19!null p_new:20!null q:21 ├── inner-join (hash) - │ ├── columns: c:11!null child_multi.p:12!null child_multi.q:13!null p:14!null q:15!null p_new:16!null q:17 + │ ├── columns: c:14!null child_multi.p:15!null child_multi.q:16!null child_multi.crdb_internal_mvcc_timestamp:17 p:18!null q:19!null p_new:20!null q:21 │ ├── scan child_multi - │ │ └── columns: c:11!null child_multi.p:12 child_multi.q:13 + │ │ └── columns: c:14!null child_multi.p:15 child_multi.q:16 child_multi.crdb_internal_mvcc_timestamp:17 │ ├── select - │ │ ├── columns: p:14!null q:15 p_new:16!null q:17 + │ │ ├── columns: p:18!null q:19 p_new:20!null q:21 │ │ ├── with-scan &1 - │ │ │ ├── columns: p:14!null q:15 p_new:16!null q:17 + │ │ │ ├── columns: p:18!null q:19 p_new:20!null q:21 │ │ │ └── mapping: - │ │ │ ├── parent_multi.p:5 => p:14 - │ │ │ ├── parent_multi.q:6 => q:15 - │ │ │ ├── p_new:7 => p_new:16 - │ │ │ └── parent_multi.q:6 => q:17 + │ │ │ ├── parent_multi.p:6 => p:18 + │ │ │ ├── parent_multi.q:7 => q:19 + │ │ │ ├── p_new:9 => p_new:20 + │ │ │ └── parent_multi.q:7 => q:21 │ │ └── filters - │ │ └── (p:14 IS DISTINCT FROM p_new:16) OR (q:15 IS DISTINCT FROM q:17) + │ │ └── (p:18 IS DISTINCT FROM p_new:20) OR (q:19 IS DISTINCT FROM q:21) │ └── filters - │ ├── child_multi.p:12 = p:14 - │ └── child_multi.q:13 = q:15 + │ ├── child_multi.p:15 = p:18 + │ └── child_multi.q:16 = q:19 └── projections - └── NULL::INT8 [as=p_new:18] + └── NULL::INT8 [as=p_new:22] # Test a two-level cascade. exec-ddl @@ -196,83 +196,83 @@ UPDATE parent_multi SET q = q * 10 WHERE p > 1 root ├── update parent_multi │ ├── columns: - │ ├── fetch columns: pk:4 p:5 q:6 + │ ├── fetch columns: pk:5 p:6 q:7 │ ├── update-mapping: - │ │ └── q_new:7 => q:3 + │ │ └── q_new:9 => q:3 │ ├── input binding: &1 │ ├── cascades │ │ └── fk │ └── project - │ ├── columns: q_new:7 pk:4!null p:5!null q:6 + │ ├── columns: q_new:9 pk:5!null p:6!null q:7 crdb_internal_mvcc_timestamp:8 │ ├── select - │ │ ├── columns: pk:4!null p:5!null q:6 + │ │ ├── columns: pk:5!null p:6!null q:7 crdb_internal_mvcc_timestamp:8 │ │ ├── scan parent_multi - │ │ │ └── columns: pk:4!null p:5 q:6 + │ │ │ └── columns: pk:5!null p:6 q:7 crdb_internal_mvcc_timestamp:8 │ │ └── filters - │ │ └── p:5 > 1 + │ │ └── p:6 > 1 │ └── projections - │ └── q:6 * 10 [as=q_new:7] + │ └── q:7 * 10 [as=q_new:9] └── cascade ├── update child_multi │ ├── columns: - │ ├── fetch columns: c:11 child_multi.p:12 child_multi.q:13 + │ ├── fetch columns: c:14 child_multi.p:15 child_multi.q:16 │ ├── update-mapping: - │ │ ├── p_new:18 => child_multi.p:9 - │ │ └── p_new:18 => child_multi.q:10 + │ │ ├── p_new:22 => child_multi.p:11 + │ │ └── p_new:22 => child_multi.q:12 │ ├── input binding: &2 │ ├── cascades │ │ └── fk2 │ └── project - │ ├── columns: p_new:18 c:11!null child_multi.p:12!null child_multi.q:13!null p:14!null q:15!null p:16!null q_new:17 + │ ├── columns: p_new:22 c:14!null child_multi.p:15!null child_multi.q:16!null child_multi.crdb_internal_mvcc_timestamp:17 p:18!null q:19!null p:20!null q_new:21 │ ├── inner-join (hash) - │ │ ├── columns: c:11!null child_multi.p:12!null child_multi.q:13!null p:14!null q:15!null p:16!null q_new:17 + │ │ ├── columns: c:14!null child_multi.p:15!null child_multi.q:16!null child_multi.crdb_internal_mvcc_timestamp:17 p:18!null q:19!null p:20!null q_new:21 │ │ ├── scan child_multi - │ │ │ └── columns: c:11!null child_multi.p:12 child_multi.q:13 + │ │ │ └── columns: c:14!null child_multi.p:15 child_multi.q:16 child_multi.crdb_internal_mvcc_timestamp:17 │ │ ├── select - │ │ │ ├── columns: p:14!null q:15 p:16!null q_new:17 + │ │ │ ├── columns: p:18!null q:19 p:20!null q_new:21 │ │ │ ├── with-scan &1 - │ │ │ │ ├── columns: p:14!null q:15 p:16!null q_new:17 + │ │ │ │ ├── columns: p:18!null q:19 p:20!null q_new:21 │ │ │ │ └── mapping: - │ │ │ │ ├── parent_multi.p:5 => p:14 - │ │ │ │ ├── parent_multi.q:6 => q:15 - │ │ │ │ ├── parent_multi.p:5 => p:16 - │ │ │ │ └── q_new:7 => q_new:17 + │ │ │ │ ├── parent_multi.p:6 => p:18 + │ │ │ │ ├── parent_multi.q:7 => q:19 + │ │ │ │ ├── parent_multi.p:6 => p:20 + │ │ │ │ └── q_new:9 => q_new:21 │ │ │ └── filters - │ │ │ └── (p:14 IS DISTINCT FROM p:16) OR (q:15 IS DISTINCT FROM q_new:17) + │ │ │ └── (p:18 IS DISTINCT FROM p:20) OR (q:19 IS DISTINCT FROM q_new:21) │ │ └── filters - │ │ ├── child_multi.p:12 = p:14 - │ │ └── child_multi.q:13 = q:15 + │ │ ├── child_multi.p:15 = p:18 + │ │ └── child_multi.q:16 = q:19 │ └── projections - │ └── NULL::INT8 [as=p_new:18] + │ └── NULL::INT8 [as=p_new:22] └── cascade └── update grandchild ├── columns: - ├── fetch columns: g:22 grandchild.c:23 grandchild.q:24 + ├── fetch columns: g:27 grandchild.c:28 grandchild.q:29 ├── update-mapping: - │ ├── c_new:29 => grandchild.c:20 - │ └── c_new:29 => grandchild.q:21 + │ ├── c_new:35 => grandchild.c:24 + │ └── c_new:35 => grandchild.q:25 └── project - ├── columns: c_new:29 g:22!null grandchild.c:23!null grandchild.q:24!null c:25!null q:26!null c:27!null p_new:28 + ├── columns: c_new:35 g:27!null grandchild.c:28!null grandchild.q:29!null grandchild.crdb_internal_mvcc_timestamp:30 c:31!null q:32!null c:33!null p_new:34 ├── inner-join (hash) - │ ├── columns: g:22!null grandchild.c:23!null grandchild.q:24!null c:25!null q:26!null c:27!null p_new:28 + │ ├── columns: g:27!null grandchild.c:28!null grandchild.q:29!null grandchild.crdb_internal_mvcc_timestamp:30 c:31!null q:32!null c:33!null p_new:34 │ ├── scan grandchild - │ │ └── columns: g:22!null grandchild.c:23 grandchild.q:24 + │ │ └── columns: g:27!null grandchild.c:28 grandchild.q:29 grandchild.crdb_internal_mvcc_timestamp:30 │ ├── select - │ │ ├── columns: c:25!null q:26!null c:27!null p_new:28 + │ │ ├── columns: c:31!null q:32!null c:33!null p_new:34 │ │ ├── with-scan &2 - │ │ │ ├── columns: c:25!null q:26!null c:27!null p_new:28 + │ │ │ ├── columns: c:31!null q:32!null c:33!null p_new:34 │ │ │ └── mapping: - │ │ │ ├── child_multi.c:11 => c:25 - │ │ │ ├── child_multi.q:13 => q:26 - │ │ │ ├── child_multi.c:11 => c:27 - │ │ │ └── p_new:18 => p_new:28 + │ │ │ ├── child_multi.c:14 => c:31 + │ │ │ ├── child_multi.q:16 => q:32 + │ │ │ ├── child_multi.c:14 => c:33 + │ │ │ └── p_new:22 => p_new:34 │ │ └── filters - │ │ └── (c:25 IS DISTINCT FROM c:27) OR (q:26 IS DISTINCT FROM p_new:28) + │ │ └── (c:31 IS DISTINCT FROM c:33) OR (q:32 IS DISTINCT FROM p_new:34) │ └── filters - │ ├── grandchild.c:23 = c:25 - │ └── grandchild.q:24 = q:26 + │ ├── grandchild.c:28 = c:31 + │ └── grandchild.q:29 = q:32 └── projections - └── NULL::INT8 [as=c_new:29] + └── NULL::INT8 [as=c_new:35] build-cascades UPSERT INTO parent_multi VALUES (1, 10, 10), (2, 20, 20) @@ -280,101 +280,101 @@ UPSERT INTO parent_multi VALUES (1, 10, 10), (2, 20, 20) root ├── upsert parent_multi │ ├── columns: - │ ├── canary column: 7 - │ ├── fetch columns: pk:7 p:8 q:9 + │ ├── canary column: 8 + │ ├── fetch columns: pk:8 p:9 q:10 │ ├── insert-mapping: - │ │ ├── column1:4 => pk:1 - │ │ ├── column2:5 => p:2 - │ │ └── column3:6 => q:3 + │ │ ├── column1:5 => pk:1 + │ │ ├── column2:6 => p:2 + │ │ └── column3:7 => q:3 │ ├── update-mapping: - │ │ ├── column2:5 => p:2 - │ │ └── column3:6 => q:3 + │ │ ├── column2:6 => p:2 + │ │ └── column3:7 => q:3 │ ├── input binding: &1 │ ├── cascades │ │ └── fk │ └── project - │ ├── columns: upsert_pk:10 column1:4!null column2:5!null column3:6!null pk:7 p:8 q:9 + │ ├── columns: upsert_pk:12 column1:5!null column2:6!null column3:7!null pk:8 p:9 q:10 crdb_internal_mvcc_timestamp:11 │ ├── left-join (hash) - │ │ ├── columns: column1:4!null column2:5!null column3:6!null pk:7 p:8 q:9 + │ │ ├── columns: column1:5!null column2:6!null column3:7!null pk:8 p:9 q:10 crdb_internal_mvcc_timestamp:11 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null - │ │ │ ├── grouping columns: column1:4!null + │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null + │ │ │ ├── grouping columns: column1:5!null │ │ │ ├── values - │ │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null + │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null │ │ │ │ ├── (1, 10, 10) │ │ │ │ └── (2, 20, 20) │ │ │ └── aggregations - │ │ │ ├── first-agg [as=column2:5] - │ │ │ │ └── column2:5 - │ │ │ └── first-agg [as=column3:6] - │ │ │ └── column3:6 + │ │ │ ├── first-agg [as=column2:6] + │ │ │ │ └── column2:6 + │ │ │ └── first-agg [as=column3:7] + │ │ │ └── column3:7 │ │ ├── scan parent_multi - │ │ │ └── columns: pk:7!null p:8 q:9 + │ │ │ └── columns: pk:8!null p:9 q:10 crdb_internal_mvcc_timestamp:11 │ │ └── filters - │ │ └── column1:4 = pk:7 + │ │ └── column1:5 = pk:8 │ └── projections - │ └── CASE WHEN pk:7 IS NULL THEN column1:4 ELSE pk:7 END [as=upsert_pk:10] + │ └── CASE WHEN pk:8 IS NULL THEN column1:5 ELSE pk:8 END [as=upsert_pk:12] └── cascade ├── update child_multi │ ├── columns: - │ ├── fetch columns: c:14 child_multi.p:15 child_multi.q:16 + │ ├── fetch columns: c:17 child_multi.p:18 child_multi.q:19 │ ├── update-mapping: - │ │ ├── p_new:21 => child_multi.p:12 - │ │ └── p_new:21 => child_multi.q:13 + │ │ ├── p_new:25 => child_multi.p:14 + │ │ └── p_new:25 => child_multi.q:15 │ ├── input binding: &2 │ ├── cascades │ │ └── fk2 │ └── project - │ ├── columns: p_new:21 c:14!null child_multi.p:15!null child_multi.q:16!null p:17!null q:18!null column2:19!null column3:20!null + │ ├── columns: p_new:25 c:17!null child_multi.p:18!null child_multi.q:19!null child_multi.crdb_internal_mvcc_timestamp:20 p:21!null q:22!null column2:23!null column3:24!null │ ├── inner-join (hash) - │ │ ├── columns: c:14!null child_multi.p:15!null child_multi.q:16!null p:17!null q:18!null column2:19!null column3:20!null + │ │ ├── columns: c:17!null child_multi.p:18!null child_multi.q:19!null child_multi.crdb_internal_mvcc_timestamp:20 p:21!null q:22!null column2:23!null column3:24!null │ │ ├── scan child_multi - │ │ │ └── columns: c:14!null child_multi.p:15 child_multi.q:16 + │ │ │ └── columns: c:17!null child_multi.p:18 child_multi.q:19 child_multi.crdb_internal_mvcc_timestamp:20 │ │ ├── select - │ │ │ ├── columns: p:17 q:18 column2:19!null column3:20!null + │ │ │ ├── columns: p:21 q:22 column2:23!null column3:24!null │ │ │ ├── with-scan &1 - │ │ │ │ ├── columns: p:17 q:18 column2:19!null column3:20!null + │ │ │ │ ├── columns: p:21 q:22 column2:23!null column3:24!null │ │ │ │ └── mapping: - │ │ │ │ ├── parent_multi.p:8 => p:17 - │ │ │ │ ├── parent_multi.q:9 => q:18 - │ │ │ │ ├── column2:5 => column2:19 - │ │ │ │ └── column3:6 => column3:20 + │ │ │ │ ├── parent_multi.p:9 => p:21 + │ │ │ │ ├── parent_multi.q:10 => q:22 + │ │ │ │ ├── column2:6 => column2:23 + │ │ │ │ └── column3:7 => column3:24 │ │ │ └── filters - │ │ │ └── (p:17 IS DISTINCT FROM column2:19) OR (q:18 IS DISTINCT FROM column3:20) + │ │ │ └── (p:21 IS DISTINCT FROM column2:23) OR (q:22 IS DISTINCT FROM column3:24) │ │ └── filters - │ │ ├── child_multi.p:15 = p:17 - │ │ └── child_multi.q:16 = q:18 + │ │ ├── child_multi.p:18 = p:21 + │ │ └── child_multi.q:19 = q:22 │ └── projections - │ └── NULL::INT8 [as=p_new:21] + │ └── NULL::INT8 [as=p_new:25] └── cascade └── update grandchild ├── columns: - ├── fetch columns: g:25 grandchild.c:26 grandchild.q:27 + ├── fetch columns: g:30 grandchild.c:31 grandchild.q:32 ├── update-mapping: - │ ├── c_new:32 => grandchild.c:23 - │ └── c_new:32 => grandchild.q:24 + │ ├── c_new:38 => grandchild.c:27 + │ └── c_new:38 => grandchild.q:28 └── project - ├── columns: c_new:32 g:25!null grandchild.c:26!null grandchild.q:27!null c:28!null q:29!null c:30!null p_new:31 + ├── columns: c_new:38 g:30!null grandchild.c:31!null grandchild.q:32!null grandchild.crdb_internal_mvcc_timestamp:33 c:34!null q:35!null c:36!null p_new:37 ├── inner-join (hash) - │ ├── columns: g:25!null grandchild.c:26!null grandchild.q:27!null c:28!null q:29!null c:30!null p_new:31 + │ ├── columns: g:30!null grandchild.c:31!null grandchild.q:32!null grandchild.crdb_internal_mvcc_timestamp:33 c:34!null q:35!null c:36!null p_new:37 │ ├── scan grandchild - │ │ └── columns: g:25!null grandchild.c:26 grandchild.q:27 + │ │ └── columns: g:30!null grandchild.c:31 grandchild.q:32 grandchild.crdb_internal_mvcc_timestamp:33 │ ├── select - │ │ ├── columns: c:28!null q:29!null c:30!null p_new:31 + │ │ ├── columns: c:34!null q:35!null c:36!null p_new:37 │ │ ├── with-scan &2 - │ │ │ ├── columns: c:28!null q:29!null c:30!null p_new:31 + │ │ │ ├── columns: c:34!null q:35!null c:36!null p_new:37 │ │ │ └── mapping: - │ │ │ ├── child_multi.c:14 => c:28 - │ │ │ ├── child_multi.q:16 => q:29 - │ │ │ ├── child_multi.c:14 => c:30 - │ │ │ └── p_new:21 => p_new:31 + │ │ │ ├── child_multi.c:17 => c:34 + │ │ │ ├── child_multi.q:19 => q:35 + │ │ │ ├── child_multi.c:17 => c:36 + │ │ │ └── p_new:25 => p_new:37 │ │ └── filters - │ │ └── (c:28 IS DISTINCT FROM c:30) OR (q:29 IS DISTINCT FROM p_new:31) + │ │ └── (c:34 IS DISTINCT FROM c:36) OR (q:35 IS DISTINCT FROM p_new:37) │ └── filters - │ ├── grandchild.c:26 = c:28 - │ └── grandchild.q:27 = q:29 + │ ├── grandchild.c:31 = c:34 + │ └── grandchild.q:32 = q:35 └── projections - └── NULL::INT8 [as=c_new:32] + └── NULL::INT8 [as=c_new:38] # Upsert that only touches one of the FK columns. build-cascades @@ -383,105 +383,105 @@ UPSERT INTO parent_multi(pk, p) VALUES (1, 10), (2, 20) root ├── upsert parent_multi │ ├── columns: - │ ├── canary column: 7 - │ ├── fetch columns: pk:7 p:8 q:9 + │ ├── canary column: 8 + │ ├── fetch columns: pk:8 p:9 q:10 │ ├── insert-mapping: - │ │ ├── column1:4 => pk:1 - │ │ ├── column2:5 => p:2 - │ │ └── column6:6 => q:3 + │ │ ├── column1:5 => pk:1 + │ │ ├── column2:6 => p:2 + │ │ └── column7:7 => q:3 │ ├── update-mapping: - │ │ └── column2:5 => p:2 + │ │ └── column2:6 => p:2 │ ├── input binding: &1 │ ├── cascades │ │ └── fk │ └── project - │ ├── columns: upsert_pk:10 upsert_q:11 column1:4!null column2:5!null column6:6 pk:7 p:8 q:9 + │ ├── columns: upsert_pk:12 upsert_q:13 column1:5!null column2:6!null column7:7 pk:8 p:9 q:10 crdb_internal_mvcc_timestamp:11 │ ├── left-join (hash) - │ │ ├── columns: column1:4!null column2:5!null column6:6 pk:7 p:8 q:9 + │ │ ├── columns: column1:5!null column2:6!null column7:7 pk:8 p:9 q:10 crdb_internal_mvcc_timestamp:11 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: column1:4!null column2:5!null column6:6 - │ │ │ ├── grouping columns: column1:4!null + │ │ │ ├── columns: column1:5!null column2:6!null column7:7 + │ │ │ ├── grouping columns: column1:5!null │ │ │ ├── project - │ │ │ │ ├── columns: column6:6 column1:4!null column2:5!null + │ │ │ │ ├── columns: column7:7 column1:5!null column2:6!null │ │ │ │ ├── values - │ │ │ │ │ ├── columns: column1:4!null column2:5!null + │ │ │ │ │ ├── columns: column1:5!null column2:6!null │ │ │ │ │ ├── (1, 10) │ │ │ │ │ └── (2, 20) │ │ │ │ └── projections - │ │ │ │ └── NULL::INT8 [as=column6:6] + │ │ │ │ └── NULL::INT8 [as=column7:7] │ │ │ └── aggregations - │ │ │ ├── first-agg [as=column2:5] - │ │ │ │ └── column2:5 - │ │ │ └── first-agg [as=column6:6] - │ │ │ └── column6:6 + │ │ │ ├── first-agg [as=column2:6] + │ │ │ │ └── column2:6 + │ │ │ └── first-agg [as=column7:7] + │ │ │ └── column7:7 │ │ ├── scan parent_multi - │ │ │ └── columns: pk:7!null p:8 q:9 + │ │ │ └── columns: pk:8!null p:9 q:10 crdb_internal_mvcc_timestamp:11 │ │ └── filters - │ │ └── column1:4 = pk:7 + │ │ └── column1:5 = pk:8 │ └── projections - │ ├── CASE WHEN pk:7 IS NULL THEN column1:4 ELSE pk:7 END [as=upsert_pk:10] - │ └── CASE WHEN pk:7 IS NULL THEN column6:6 ELSE q:9 END [as=upsert_q:11] + │ ├── CASE WHEN pk:8 IS NULL THEN column1:5 ELSE pk:8 END [as=upsert_pk:12] + │ └── CASE WHEN pk:8 IS NULL THEN column7:7 ELSE q:10 END [as=upsert_q:13] └── cascade ├── update child_multi │ ├── columns: - │ ├── fetch columns: c:15 child_multi.p:16 child_multi.q:17 + │ ├── fetch columns: c:18 child_multi.p:19 child_multi.q:20 │ ├── update-mapping: - │ │ ├── p_new:22 => child_multi.p:13 - │ │ └── p_new:22 => child_multi.q:14 + │ │ ├── p_new:26 => child_multi.p:15 + │ │ └── p_new:26 => child_multi.q:16 │ ├── input binding: &2 │ ├── cascades │ │ └── fk2 │ └── project - │ ├── columns: p_new:22 c:15!null child_multi.p:16!null child_multi.q:17!null p:18!null q:19!null column2:20!null q:21 + │ ├── columns: p_new:26 c:18!null child_multi.p:19!null child_multi.q:20!null child_multi.crdb_internal_mvcc_timestamp:21 p:22!null q:23!null column2:24!null q:25 │ ├── inner-join (hash) - │ │ ├── columns: c:15!null child_multi.p:16!null child_multi.q:17!null p:18!null q:19!null column2:20!null q:21 + │ │ ├── columns: c:18!null child_multi.p:19!null child_multi.q:20!null child_multi.crdb_internal_mvcc_timestamp:21 p:22!null q:23!null column2:24!null q:25 │ │ ├── scan child_multi - │ │ │ └── columns: c:15!null child_multi.p:16 child_multi.q:17 + │ │ │ └── columns: c:18!null child_multi.p:19 child_multi.q:20 child_multi.crdb_internal_mvcc_timestamp:21 │ │ ├── select - │ │ │ ├── columns: p:18 q:19 column2:20!null q:21 + │ │ │ ├── columns: p:22 q:23 column2:24!null q:25 │ │ │ ├── with-scan &1 - │ │ │ │ ├── columns: p:18 q:19 column2:20!null q:21 + │ │ │ │ ├── columns: p:22 q:23 column2:24!null q:25 │ │ │ │ └── mapping: - │ │ │ │ ├── parent_multi.p:8 => p:18 - │ │ │ │ ├── parent_multi.q:9 => q:19 - │ │ │ │ ├── column2:5 => column2:20 - │ │ │ │ └── parent_multi.q:9 => q:21 + │ │ │ │ ├── parent_multi.p:9 => p:22 + │ │ │ │ ├── parent_multi.q:10 => q:23 + │ │ │ │ ├── column2:6 => column2:24 + │ │ │ │ └── parent_multi.q:10 => q:25 │ │ │ └── filters - │ │ │ └── (p:18 IS DISTINCT FROM column2:20) OR (q:19 IS DISTINCT FROM q:21) + │ │ │ └── (p:22 IS DISTINCT FROM column2:24) OR (q:23 IS DISTINCT FROM q:25) │ │ └── filters - │ │ ├── child_multi.p:16 = p:18 - │ │ └── child_multi.q:17 = q:19 + │ │ ├── child_multi.p:19 = p:22 + │ │ └── child_multi.q:20 = q:23 │ └── projections - │ └── NULL::INT8 [as=p_new:22] + │ └── NULL::INT8 [as=p_new:26] └── cascade └── update grandchild ├── columns: - ├── fetch columns: g:26 grandchild.c:27 grandchild.q:28 + ├── fetch columns: g:31 grandchild.c:32 grandchild.q:33 ├── update-mapping: - │ ├── c_new:33 => grandchild.c:24 - │ └── c_new:33 => grandchild.q:25 + │ ├── c_new:39 => grandchild.c:28 + │ └── c_new:39 => grandchild.q:29 └── project - ├── columns: c_new:33 g:26!null grandchild.c:27!null grandchild.q:28!null c:29!null q:30!null c:31!null p_new:32 + ├── columns: c_new:39 g:31!null grandchild.c:32!null grandchild.q:33!null grandchild.crdb_internal_mvcc_timestamp:34 c:35!null q:36!null c:37!null p_new:38 ├── inner-join (hash) - │ ├── columns: g:26!null grandchild.c:27!null grandchild.q:28!null c:29!null q:30!null c:31!null p_new:32 + │ ├── columns: g:31!null grandchild.c:32!null grandchild.q:33!null grandchild.crdb_internal_mvcc_timestamp:34 c:35!null q:36!null c:37!null p_new:38 │ ├── scan grandchild - │ │ └── columns: g:26!null grandchild.c:27 grandchild.q:28 + │ │ └── columns: g:31!null grandchild.c:32 grandchild.q:33 grandchild.crdb_internal_mvcc_timestamp:34 │ ├── select - │ │ ├── columns: c:29!null q:30!null c:31!null p_new:32 + │ │ ├── columns: c:35!null q:36!null c:37!null p_new:38 │ │ ├── with-scan &2 - │ │ │ ├── columns: c:29!null q:30!null c:31!null p_new:32 + │ │ │ ├── columns: c:35!null q:36!null c:37!null p_new:38 │ │ │ └── mapping: - │ │ │ ├── child_multi.c:15 => c:29 - │ │ │ ├── child_multi.q:17 => q:30 - │ │ │ ├── child_multi.c:15 => c:31 - │ │ │ └── p_new:22 => p_new:32 + │ │ │ ├── child_multi.c:18 => c:35 + │ │ │ ├── child_multi.q:20 => q:36 + │ │ │ ├── child_multi.c:18 => c:37 + │ │ │ └── p_new:26 => p_new:38 │ │ └── filters - │ │ └── (c:29 IS DISTINCT FROM c:31) OR (q:30 IS DISTINCT FROM p_new:32) + │ │ └── (c:35 IS DISTINCT FROM c:37) OR (q:36 IS DISTINCT FROM p_new:38) │ └── filters - │ ├── grandchild.c:27 = c:29 - │ └── grandchild.q:28 = q:30 + │ ├── grandchild.c:32 = c:35 + │ └── grandchild.q:33 = q:36 └── projections - └── NULL::INT8 [as=c_new:33] + └── NULL::INT8 [as=c_new:39] build-cascades INSERT INTO parent_multi VALUES (1, 10, 10), (2, 20, 20) ON CONFLICT (p,q) DO UPDATE SET p = 100 @@ -489,102 +489,102 @@ INSERT INTO parent_multi VALUES (1, 10, 10), (2, 20, 20) ON CONFLICT (p,q) DO UP root ├── upsert parent_multi │ ├── columns: - │ ├── canary column: 7 - │ ├── fetch columns: pk:7 p:8 q:9 + │ ├── canary column: 8 + │ ├── fetch columns: pk:8 p:9 q:10 │ ├── insert-mapping: - │ │ ├── column1:4 => pk:1 - │ │ ├── column2:5 => p:2 - │ │ └── column3:6 => q:3 + │ │ ├── column1:5 => pk:1 + │ │ ├── column2:6 => p:2 + │ │ └── column3:7 => q:3 │ ├── update-mapping: - │ │ └── upsert_p:12 => p:2 + │ │ └── upsert_p:14 => p:2 │ ├── input binding: &1 │ ├── cascades │ │ └── fk │ └── project - │ ├── columns: upsert_pk:11 upsert_p:12!null upsert_q:13 column1:4!null column2:5!null column3:6!null pk:7 p:8 q:9 p_new:10!null + │ ├── columns: upsert_pk:13 upsert_p:14!null upsert_q:15 column1:5!null column2:6!null column3:7!null pk:8 p:9 q:10 crdb_internal_mvcc_timestamp:11 p_new:12!null │ ├── project - │ │ ├── columns: p_new:10!null column1:4!null column2:5!null column3:6!null pk:7 p:8 q:9 + │ │ ├── columns: p_new:12!null column1:5!null column2:6!null column3:7!null pk:8 p:9 q:10 crdb_internal_mvcc_timestamp:11 │ │ ├── left-join (hash) - │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null pk:7 p:8 q:9 + │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null pk:8 p:9 q:10 crdb_internal_mvcc_timestamp:11 │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null - │ │ │ │ ├── grouping columns: column2:5!null column3:6!null + │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null + │ │ │ │ ├── grouping columns: column2:6!null column3:7!null │ │ │ │ ├── values - │ │ │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null + │ │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null │ │ │ │ │ ├── (1, 10, 10) │ │ │ │ │ └── (2, 20, 20) │ │ │ │ └── aggregations - │ │ │ │ └── first-agg [as=column1:4] - │ │ │ │ └── column1:4 + │ │ │ │ └── first-agg [as=column1:5] + │ │ │ │ └── column1:5 │ │ │ ├── scan parent_multi - │ │ │ │ └── columns: pk:7!null p:8 q:9 + │ │ │ │ └── columns: pk:8!null p:9 q:10 crdb_internal_mvcc_timestamp:11 │ │ │ └── filters - │ │ │ ├── column2:5 = p:8 - │ │ │ └── column3:6 = q:9 + │ │ │ ├── column2:6 = p:9 + │ │ │ └── column3:7 = q:10 │ │ └── projections - │ │ └── 100 [as=p_new:10] + │ │ └── 100 [as=p_new:12] │ └── projections - │ ├── CASE WHEN pk:7 IS NULL THEN column1:4 ELSE pk:7 END [as=upsert_pk:11] - │ ├── CASE WHEN pk:7 IS NULL THEN column2:5 ELSE p_new:10 END [as=upsert_p:12] - │ └── CASE WHEN pk:7 IS NULL THEN column3:6 ELSE q:9 END [as=upsert_q:13] + │ ├── CASE WHEN pk:8 IS NULL THEN column1:5 ELSE pk:8 END [as=upsert_pk:13] + │ ├── CASE WHEN pk:8 IS NULL THEN column2:6 ELSE p_new:12 END [as=upsert_p:14] + │ └── CASE WHEN pk:8 IS NULL THEN column3:7 ELSE q:10 END [as=upsert_q:15] └── cascade ├── update child_multi │ ├── columns: - │ ├── fetch columns: c:17 child_multi.p:18 child_multi.q:19 + │ ├── fetch columns: c:20 child_multi.p:21 child_multi.q:22 │ ├── update-mapping: - │ │ ├── p_new:24 => child_multi.p:15 - │ │ └── p_new:24 => child_multi.q:16 + │ │ ├── p_new:28 => child_multi.p:17 + │ │ └── p_new:28 => child_multi.q:18 │ ├── input binding: &2 │ ├── cascades │ │ └── fk2 │ └── project - │ ├── columns: p_new:24 c:17!null child_multi.p:18!null child_multi.q:19!null p:20!null q:21!null upsert_p:22!null q:23 + │ ├── columns: p_new:28 c:20!null child_multi.p:21!null child_multi.q:22!null child_multi.crdb_internal_mvcc_timestamp:23 p:24!null q:25!null upsert_p:26!null q:27 │ ├── inner-join (hash) - │ │ ├── columns: c:17!null child_multi.p:18!null child_multi.q:19!null p:20!null q:21!null upsert_p:22!null q:23 + │ │ ├── columns: c:20!null child_multi.p:21!null child_multi.q:22!null child_multi.crdb_internal_mvcc_timestamp:23 p:24!null q:25!null upsert_p:26!null q:27 │ │ ├── scan child_multi - │ │ │ └── columns: c:17!null child_multi.p:18 child_multi.q:19 + │ │ │ └── columns: c:20!null child_multi.p:21 child_multi.q:22 child_multi.crdb_internal_mvcc_timestamp:23 │ │ ├── select - │ │ │ ├── columns: p:20 q:21 upsert_p:22!null q:23 + │ │ │ ├── columns: p:24 q:25 upsert_p:26!null q:27 │ │ │ ├── with-scan &1 - │ │ │ │ ├── columns: p:20 q:21 upsert_p:22!null q:23 + │ │ │ │ ├── columns: p:24 q:25 upsert_p:26!null q:27 │ │ │ │ └── mapping: - │ │ │ │ ├── parent_multi.p:8 => p:20 - │ │ │ │ ├── parent_multi.q:9 => q:21 - │ │ │ │ ├── upsert_p:12 => upsert_p:22 - │ │ │ │ └── parent_multi.q:9 => q:23 + │ │ │ │ ├── parent_multi.p:9 => p:24 + │ │ │ │ ├── parent_multi.q:10 => q:25 + │ │ │ │ ├── upsert_p:14 => upsert_p:26 + │ │ │ │ └── parent_multi.q:10 => q:27 │ │ │ └── filters - │ │ │ └── (p:20 IS DISTINCT FROM upsert_p:22) OR (q:21 IS DISTINCT FROM q:23) + │ │ │ └── (p:24 IS DISTINCT FROM upsert_p:26) OR (q:25 IS DISTINCT FROM q:27) │ │ └── filters - │ │ ├── child_multi.p:18 = p:20 - │ │ └── child_multi.q:19 = q:21 + │ │ ├── child_multi.p:21 = p:24 + │ │ └── child_multi.q:22 = q:25 │ └── projections - │ └── NULL::INT8 [as=p_new:24] + │ └── NULL::INT8 [as=p_new:28] └── cascade └── update grandchild ├── columns: - ├── fetch columns: g:28 grandchild.c:29 grandchild.q:30 + ├── fetch columns: g:33 grandchild.c:34 grandchild.q:35 ├── update-mapping: - │ ├── c_new:35 => grandchild.c:26 - │ └── c_new:35 => grandchild.q:27 + │ ├── c_new:41 => grandchild.c:30 + │ └── c_new:41 => grandchild.q:31 └── project - ├── columns: c_new:35 g:28!null grandchild.c:29!null grandchild.q:30!null c:31!null q:32!null c:33!null p_new:34 + ├── columns: c_new:41 g:33!null grandchild.c:34!null grandchild.q:35!null grandchild.crdb_internal_mvcc_timestamp:36 c:37!null q:38!null c:39!null p_new:40 ├── inner-join (hash) - │ ├── columns: g:28!null grandchild.c:29!null grandchild.q:30!null c:31!null q:32!null c:33!null p_new:34 + │ ├── columns: g:33!null grandchild.c:34!null grandchild.q:35!null grandchild.crdb_internal_mvcc_timestamp:36 c:37!null q:38!null c:39!null p_new:40 │ ├── scan grandchild - │ │ └── columns: g:28!null grandchild.c:29 grandchild.q:30 + │ │ └── columns: g:33!null grandchild.c:34 grandchild.q:35 grandchild.crdb_internal_mvcc_timestamp:36 │ ├── select - │ │ ├── columns: c:31!null q:32!null c:33!null p_new:34 + │ │ ├── columns: c:37!null q:38!null c:39!null p_new:40 │ │ ├── with-scan &2 - │ │ │ ├── columns: c:31!null q:32!null c:33!null p_new:34 + │ │ │ ├── columns: c:37!null q:38!null c:39!null p_new:40 │ │ │ └── mapping: - │ │ │ ├── child_multi.c:17 => c:31 - │ │ │ ├── child_multi.q:19 => q:32 - │ │ │ ├── child_multi.c:17 => c:33 - │ │ │ └── p_new:24 => p_new:34 + │ │ │ ├── child_multi.c:20 => c:37 + │ │ │ ├── child_multi.q:22 => q:38 + │ │ │ ├── child_multi.c:20 => c:39 + │ │ │ └── p_new:28 => p_new:40 │ │ └── filters - │ │ └── (c:31 IS DISTINCT FROM c:33) OR (q:32 IS DISTINCT FROM p_new:34) + │ │ └── (c:37 IS DISTINCT FROM c:39) OR (q:38 IS DISTINCT FROM p_new:40) │ └── filters - │ ├── grandchild.c:29 = c:31 - │ └── grandchild.q:30 = q:32 + │ ├── grandchild.c:34 = c:37 + │ └── grandchild.q:35 = q:38 └── projections - └── NULL::INT8 [as=c_new:35] + └── NULL::INT8 [as=c_new:41] diff --git a/pkg/sql/opt/optbuilder/testdata/having b/pkg/sql/opt/optbuilder/testdata/having index f90bb15a7ebf..f7ccb526296f 100644 --- a/pkg/sql/opt/optbuilder/testdata/having +++ b/pkg/sql/opt/optbuilder/testdata/having @@ -14,75 +14,75 @@ build SELECT 3 r FROM kv HAVING TRUE ---- project - ├── columns: r:5!null + ├── columns: r:6!null ├── select │ ├── scalar-group-by │ │ └── project │ │ └── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── filters │ └── true └── projections - └── 3 [as=r:5] + └── 3 [as=r:6] build SELECT s, count(*) FROM kv GROUP BY s HAVING count(*) > 1 ---- select - ├── columns: s:4 count:5!null + ├── columns: s:4 count:6!null ├── group-by - │ ├── columns: s:4 count_rows:5!null + │ ├── columns: s:4 count_rows:6!null │ ├── grouping columns: s:4 │ ├── project │ │ ├── columns: s:4 │ │ └── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── aggregations - │ └── count-rows [as=count_rows:5] + │ └── count-rows [as=count_rows:6] └── filters - └── count_rows:5 > 1 + └── count_rows:6 > 1 build SELECT max(k), min(v) FROM kv HAVING min(v) > 2 ---- select - ├── columns: max:5 min:6!null + ├── columns: max:6 min:7!null ├── scalar-group-by - │ ├── columns: max:5 min:6 + │ ├── columns: max:6 min:7 │ ├── project │ │ ├── columns: k:1!null v:2 │ │ └── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── aggregations - │ ├── max [as=max:5] + │ ├── max [as=max:6] │ │ └── k:1 - │ └── min [as=min:6] + │ └── min [as=min:7] │ └── v:2 └── filters - └── min:6 > 2 + └── min:7 > 2 build SELECT max(k), min(v) FROM kv HAVING max(v) > 2 ---- project - ├── columns: max:5 min:6 + ├── columns: max:6 min:7 └── select - ├── columns: max:5 min:6 max:7!null + ├── columns: max:6 min:7 max:8!null ├── scalar-group-by - │ ├── columns: max:5 min:6 max:7 + │ ├── columns: max:6 min:7 max:8 │ ├── project │ │ ├── columns: k:1!null v:2 │ │ └── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── aggregations - │ ├── max [as=max:5] + │ ├── max [as=max:6] │ │ └── k:1 - │ ├── min [as=min:6] + │ ├── min [as=min:7] │ │ └── v:2 - │ └── max [as=max:7] + │ └── max [as=max:8] │ └── v:2 └── filters - └── max:7 > 2 + └── max:8 > 2 build SELECT max(k), min(v) FROM kv HAVING max(min(v)) > 2 @@ -105,7 +105,7 @@ build SELECT 3 FROM kv GROUP BY k HAVING v > 2 ---- project - ├── columns: "?column?":5!null + ├── columns: "?column?":6!null ├── select │ ├── columns: k:1!null v:2!null │ ├── group-by @@ -114,11 +114,11 @@ project │ │ └── project │ │ ├── columns: k:1!null v:2 │ │ └── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── filters │ └── v:2 > 2 └── projections - └── 3 [as="?column?":5] + └── 3 [as="?column?":6] build SELECT k FROM kv HAVING k > 7 @@ -129,20 +129,20 @@ build SELECT count(*), k+w AS r FROM kv GROUP BY k+w HAVING (k+w) > 5 ---- select - ├── columns: count:5!null r:6!null + ├── columns: count:6!null r:7!null ├── group-by - │ ├── columns: count_rows:5!null column6:6 - │ ├── grouping columns: column6:6 + │ ├── columns: count_rows:6!null column7:7 + │ ├── grouping columns: column7:7 │ ├── project - │ │ ├── columns: column6:6 + │ │ ├── columns: column7:7 │ │ ├── scan kv - │ │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ │ └── projections - │ │ └── k:1 + w:3 [as=column6:6] + │ │ └── k:1 + w:3 [as=column7:7] │ └── aggregations - │ └── count-rows [as=count_rows:5] + │ └── count-rows [as=count_rows:6] └── filters - └── column6:6 > 5 + └── column7:7 > 5 build SELECT count(*), k+w FROM kv GROUP BY k+w HAVING (k+v) > 5 @@ -154,18 +154,18 @@ build SELECT max(kv.v) FROM kv GROUP BY v HAVING kv.v > 5 ---- project - ├── columns: max:5 + ├── columns: max:6 └── select - ├── columns: v:2!null max:5 + ├── columns: v:2!null max:6 ├── group-by - │ ├── columns: v:2 max:5 + │ ├── columns: v:2 max:6 │ ├── grouping columns: v:2 │ ├── project │ │ ├── columns: v:2 │ │ └── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── aggregations - │ └── max [as=max:5] + │ └── max [as=max:6] │ └── v:2 └── filters └── v:2 > 5 @@ -174,45 +174,45 @@ build SELECT sum(kv.w) FROM kv GROUP BY lower(s) HAVING lower(kv.s) LIKE 'test%' ---- project - ├── columns: sum:5 + ├── columns: sum:6 └── select - ├── columns: sum:5 column6:6!null + ├── columns: sum:6 column7:7!null ├── group-by - │ ├── columns: sum:5 column6:6 - │ ├── grouping columns: column6:6 + │ ├── columns: sum:6 column7:7 + │ ├── grouping columns: column7:7 │ ├── project - │ │ ├── columns: column6:6 w:3 + │ │ ├── columns: column7:7 w:3 │ │ ├── scan kv - │ │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ │ └── projections - │ │ └── lower(s:4) [as=column6:6] + │ │ └── lower(s:4) [as=column7:7] │ └── aggregations - │ └── sum [as=sum:5] + │ └── sum [as=sum:6] │ └── w:3 └── filters - └── column6:6 LIKE 'test%' + └── column7:7 LIKE 'test%' build SELECT sum(kv.w) FROM kv GROUP BY lower(s) HAVING sum(w) IN (4, 5, 6) ---- project - ├── columns: sum:5!null + ├── columns: sum:6!null └── select - ├── columns: sum:5!null column6:6 + ├── columns: sum:6!null column7:7 ├── group-by - │ ├── columns: sum:5 column6:6 - │ ├── grouping columns: column6:6 + │ ├── columns: sum:6 column7:7 + │ ├── grouping columns: column7:7 │ ├── project - │ │ ├── columns: column6:6 w:3 + │ │ ├── columns: column7:7 w:3 │ │ ├── scan kv - │ │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ │ └── projections - │ │ └── lower(s:4) [as=column6:6] + │ │ └── lower(s:4) [as=column7:7] │ └── aggregations - │ └── sum [as=sum:5] + │ └── sum [as=sum:6] │ └── w:3 └── filters - └── sum:5 IN (4, 5, 6) + └── sum:6 IN (4, 5, 6) build fully-qualify-names SELECT t.kv.v FROM t.kv GROUP BY v, kv.k * w HAVING k * kv.w > 5 @@ -220,18 +220,18 @@ SELECT t.kv.v FROM t.kv GROUP BY v, kv.k * w HAVING k * kv.w > 5 project ├── columns: v:2 └── select - ├── columns: t.public.kv.v:2 column5:5!null + ├── columns: t.public.kv.v:2 column6:6!null ├── group-by - │ ├── columns: t.public.kv.v:2 column5:5 - │ ├── grouping columns: t.public.kv.v:2 column5:5 + │ ├── columns: t.public.kv.v:2 column6:6 + │ ├── grouping columns: t.public.kv.v:2 column6:6 │ └── project - │ ├── columns: column5:5 t.public.kv.v:2 + │ ├── columns: column6:6 t.public.kv.v:2 │ ├── scan t.public.kv - │ │ └── columns: t.public.kv.k:1!null t.public.kv.v:2 t.public.kv.w:3 t.public.kv.s:4 + │ │ └── columns: t.public.kv.k:1!null t.public.kv.v:2 t.public.kv.w:3 t.public.kv.s:4 t.public.kv.crdb_internal_mvcc_timestamp:5 │ └── projections - │ └── t.public.kv.k:1 * t.public.kv.w:3 [as=column5:5] + │ └── t.public.kv.k:1 * t.public.kv.w:3 [as=column6:6] └── filters - └── column5:5 > 5 + └── column6:6 > 5 build fully-qualify-names SELECT t.kv.v FROM t.kv GROUP BY v, kv.k * w HAVING w > 5 @@ -242,90 +242,90 @@ build fully-qualify-names SELECT upper(s), count(s), count(upper(s)) FROM t.kv GROUP BY upper(s) HAVING count(s) > 1 ---- select - ├── columns: upper:6 count:5!null count:7!null + ├── columns: upper:7 count:6!null count:8!null ├── group-by - │ ├── columns: count:5!null column6:6 count:7!null - │ ├── grouping columns: column6:6 + │ ├── columns: count:6!null column7:7 count:8!null + │ ├── grouping columns: column7:7 │ ├── project - │ │ ├── columns: column6:6 t.public.kv.s:4 + │ │ ├── columns: column7:7 t.public.kv.s:4 │ │ ├── scan t.public.kv - │ │ │ └── columns: t.public.kv.k:1!null t.public.kv.v:2 t.public.kv.w:3 t.public.kv.s:4 + │ │ │ └── columns: t.public.kv.k:1!null t.public.kv.v:2 t.public.kv.w:3 t.public.kv.s:4 t.public.kv.crdb_internal_mvcc_timestamp:5 │ │ └── projections - │ │ └── upper(t.public.kv.s:4) [as=column6:6] + │ │ └── upper(t.public.kv.s:4) [as=column7:7] │ └── aggregations - │ ├── count [as=count:5] + │ ├── count [as=count:6] │ │ └── t.public.kv.s:4 - │ └── count [as=count:7] - │ └── column6:6 + │ └── count [as=count:8] + │ └── column7:7 └── filters - └── count:5 > 1 + └── count:6 > 1 # Check that ordering by an alias of an aggregate works when HAVING is present. build SELECT sum(k) AS mk FROM kv GROUP BY v HAVING sum(k)=10 ORDER BY mk ---- project - ├── columns: mk:5!null - ├── ordering: +5 + ├── columns: mk:6!null + ├── ordering: +6 └── select - ├── columns: v:2 sum:5!null + ├── columns: v:2 sum:6!null ├── group-by - │ ├── columns: v:2 sum:5!null + │ ├── columns: v:2 sum:6!null │ ├── grouping columns: v:2 │ ├── project │ │ ├── columns: k:1!null v:2 │ │ └── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── aggregations - │ └── sum [as=sum:5] + │ └── sum [as=sum:6] │ └── k:1 └── filters - └── sum:5 = 10 + └── sum:6 = 10 build SELECT sum(k) AS mk FROM kv GROUP BY v HAVING max(k) > 10 ORDER BY mk ---- sort - ├── columns: mk:5!null - ├── ordering: +5 + ├── columns: mk:6!null + ├── ordering: +6 └── project - ├── columns: sum:5!null + ├── columns: sum:6!null └── select - ├── columns: v:2 sum:5!null max:6!null + ├── columns: v:2 sum:6!null max:7!null ├── group-by - │ ├── columns: v:2 sum:5!null max:6!null + │ ├── columns: v:2 sum:6!null max:7!null │ ├── grouping columns: v:2 │ ├── project │ │ ├── columns: k:1!null v:2 │ │ └── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── aggregations - │ ├── sum [as=sum:5] + │ ├── sum [as=sum:6] │ │ └── k:1 - │ └── max [as=max:6] + │ └── max [as=max:7] │ └── k:1 └── filters - └── max:6 > 10 + └── max:7 > 10 build SELECT sum(k) AS mk FROM kv GROUP BY v HAVING v > 10 ORDER BY mk ---- sort - ├── columns: mk:5!null - ├── ordering: +5 + ├── columns: mk:6!null + ├── ordering: +6 └── project - ├── columns: sum:5!null + ├── columns: sum:6!null └── select - ├── columns: v:2!null sum:5!null + ├── columns: v:2!null sum:6!null ├── group-by - │ ├── columns: v:2 sum:5!null + │ ├── columns: v:2 sum:6!null │ ├── grouping columns: v:2 │ ├── project │ │ ├── columns: k:1!null v:2 │ │ └── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── aggregations - │ └── sum [as=sum:5] + │ └── sum [as=sum:6] │ └── k:1 └── filters └── v:2 > 10 @@ -334,44 +334,44 @@ build SELECT max(k) AS mk1, max(k) AS mk2 FROM kv GROUP BY v HAVING max(k) > 10 ORDER BY mk1 ---- sort - ├── columns: mk1:5!null mk2:5!null - ├── ordering: +5 + ├── columns: mk1:6!null mk2:6!null + ├── ordering: +6 └── project - ├── columns: max:5!null + ├── columns: max:6!null └── select - ├── columns: v:2 max:5!null + ├── columns: v:2 max:6!null ├── group-by - │ ├── columns: v:2 max:5!null + │ ├── columns: v:2 max:6!null │ ├── grouping columns: v:2 │ ├── project │ │ ├── columns: k:1!null v:2 │ │ └── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── aggregations - │ └── max [as=max:5] + │ └── max [as=max:6] │ └── k:1 └── filters - └── max:5 > 10 + └── max:6 > 10 build SELECT max(k) AS mk1, max(k) AS mk2 FROM kv GROUP BY v HAVING max(k) > 10 ORDER BY mk2 ---- sort - ├── columns: mk1:5!null mk2:5!null - ├── ordering: +5 + ├── columns: mk1:6!null mk2:6!null + ├── ordering: +6 └── project - ├── columns: max:5!null + ├── columns: max:6!null └── select - ├── columns: v:2 max:5!null + ├── columns: v:2 max:6!null ├── group-by - │ ├── columns: v:2 max:5!null + │ ├── columns: v:2 max:6!null │ ├── grouping columns: v:2 │ ├── project │ │ ├── columns: k:1!null v:2 │ │ └── scan kv - │ │ └── columns: k:1!null v:2 w:3 s:4 + │ │ └── columns: k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 │ └── aggregations - │ └── max [as=max:5] + │ └── max [as=max:6] │ └── k:1 └── filters - └── max:5 > 10 + └── max:6 > 10 diff --git a/pkg/sql/opt/optbuilder/testdata/inner-join b/pkg/sql/opt/optbuilder/testdata/inner-join index de4cab7caa9e..7e7d2d758da1 100644 --- a/pkg/sql/opt/optbuilder/testdata/inner-join +++ b/pkg/sql/opt/optbuilder/testdata/inner-join @@ -14,53 +14,53 @@ build SELECT * FROM a, b ---- project - ├── columns: x:1!null y:2 x:3 y:4 + ├── columns: x:1!null y:2 x:4 y:5 └── inner-join (cross) - ├── columns: a.x:1!null a.y:2 b.x:3 b.y:4 rowid:5!null + ├── columns: a.x:1!null a.y:2 a.crdb_internal_mvcc_timestamp:3 b.x:4 b.y:5 rowid:6!null b.crdb_internal_mvcc_timestamp:7 ├── scan a - │ └── columns: a.x:1!null a.y:2 + │ └── columns: a.x:1!null a.y:2 a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: b.x:3 b.y:4 rowid:5!null + │ └── columns: b.x:4 b.y:5 rowid:6!null b.crdb_internal_mvcc_timestamp:7 └── filters (true) build SELECT a.x, b.y FROM a, b WHERE a.x = b.x ---- project - ├── columns: x:1!null y:4 + ├── columns: x:1!null y:5 └── select - ├── columns: a.x:1!null a.y:2 b.x:3!null b.y:4 rowid:5!null + ├── columns: a.x:1!null a.y:2 a.crdb_internal_mvcc_timestamp:3 b.x:4!null b.y:5 rowid:6!null b.crdb_internal_mvcc_timestamp:7 ├── inner-join (cross) - │ ├── columns: a.x:1!null a.y:2 b.x:3 b.y:4 rowid:5!null + │ ├── columns: a.x:1!null a.y:2 a.crdb_internal_mvcc_timestamp:3 b.x:4 b.y:5 rowid:6!null b.crdb_internal_mvcc_timestamp:7 │ ├── scan a - │ │ └── columns: a.x:1!null a.y:2 + │ │ └── columns: a.x:1!null a.y:2 a.crdb_internal_mvcc_timestamp:3 │ ├── scan b - │ │ └── columns: b.x:3 b.y:4 rowid:5!null + │ │ └── columns: b.x:4 b.y:5 rowid:6!null b.crdb_internal_mvcc_timestamp:7 │ └── filters (true) └── filters - └── a.x:1 = b.x:3 + └── a.x:1 = b.x:4 build SELECT * FROM c, b, a WHERE c.x = a.x AND b.x = a.x ---- project - ├── columns: x:1!null y:2 z:3 x:5!null y:6 x:8!null y:9 + ├── columns: x:1!null y:2 z:3 x:6!null y:7 x:10!null y:11 └── select - ├── columns: c.x:1!null c.y:2 z:3 c.rowid:4!null b.x:5!null b.y:6 b.rowid:7!null a.x:8!null a.y:9 + ├── columns: c.x:1!null c.y:2 z:3 c.rowid:4!null c.crdb_internal_mvcc_timestamp:5 b.x:6!null b.y:7 b.rowid:8!null b.crdb_internal_mvcc_timestamp:9 a.x:10!null a.y:11 a.crdb_internal_mvcc_timestamp:12 ├── inner-join (cross) - │ ├── columns: c.x:1 c.y:2 z:3 c.rowid:4!null b.x:5 b.y:6 b.rowid:7!null a.x:8!null a.y:9 + │ ├── columns: c.x:1 c.y:2 z:3 c.rowid:4!null c.crdb_internal_mvcc_timestamp:5 b.x:6 b.y:7 b.rowid:8!null b.crdb_internal_mvcc_timestamp:9 a.x:10!null a.y:11 a.crdb_internal_mvcc_timestamp:12 │ ├── scan c - │ │ └── columns: c.x:1 c.y:2 z:3 c.rowid:4!null + │ │ └── columns: c.x:1 c.y:2 z:3 c.rowid:4!null c.crdb_internal_mvcc_timestamp:5 │ ├── inner-join (cross) - │ │ ├── columns: b.x:5 b.y:6 b.rowid:7!null a.x:8!null a.y:9 + │ │ ├── columns: b.x:6 b.y:7 b.rowid:8!null b.crdb_internal_mvcc_timestamp:9 a.x:10!null a.y:11 a.crdb_internal_mvcc_timestamp:12 │ │ ├── scan b - │ │ │ └── columns: b.x:5 b.y:6 b.rowid:7!null + │ │ │ └── columns: b.x:6 b.y:7 b.rowid:8!null b.crdb_internal_mvcc_timestamp:9 │ │ ├── scan a - │ │ │ └── columns: a.x:8!null a.y:9 + │ │ │ └── columns: a.x:10!null a.y:11 a.crdb_internal_mvcc_timestamp:12 │ │ └── filters (true) │ └── filters (true) └── filters - └── (c.x:1 = a.x:8) AND (b.x:5 = a.x:8) + └── (c.x:1 = a.x:10) AND (b.x:6 = a.x:10) exec-ddl CREATE TABLE db1.a (x INT PRIMARY KEY, y FLOAT, z STRING) @@ -83,13 +83,15 @@ error (42702): column reference "x" is ambiguous (candidates: a.x, b.x) build fully-qualify-names SELECT * FROM db1.a, db2.a ---- -inner-join (cross) - ├── columns: x:1!null y:2 z:3 x:4!null y:5 - ├── scan db1.public.a - │ └── columns: db1.public.a.x:1!null db1.public.a.y:2 db1.public.a.z:3 - ├── scan db2.public.a - │ └── columns: db2.public.a.x:4!null db2.public.a.y:5 - └── filters (true) +project + ├── columns: x:1!null y:2 z:3 x:5!null y:6 + └── inner-join (cross) + ├── columns: db1.public.a.x:1!null db1.public.a.y:2 db1.public.a.z:3 db1.public.a.crdb_internal_mvcc_timestamp:4 db2.public.a.x:5!null db2.public.a.y:6 db2.public.a.crdb_internal_mvcc_timestamp:7 + ├── scan db1.public.a + │ └── columns: db1.public.a.x:1!null db1.public.a.y:2 db1.public.a.z:3 db1.public.a.crdb_internal_mvcc_timestamp:4 + ├── scan db2.public.a + │ └── columns: db2.public.a.x:5!null db2.public.a.y:6 db2.public.a.crdb_internal_mvcc_timestamp:7 + └── filters (true) build fully-qualify-names SELECT * FROM a, a @@ -100,13 +102,17 @@ error (42712): source name "a" specified more than once (missing AS clause) build fully-qualify-names SELECT * FROM a, (SELECT * FROM a) AS a ---- -inner-join (cross) - ├── columns: x:1!null y:2 x:3!null y:4 - ├── scan t.public.a - │ └── columns: t.public.a.x:1!null t.public.a.y:2 - ├── scan t.public.a - │ └── columns: t.public.a.x:3!null t.public.a.y:4 - └── filters (true) +project + ├── columns: x:1!null y:2 x:4!null y:5 + └── inner-join (cross) + ├── columns: t.public.a.x:1!null t.public.a.y:2 t.public.a.crdb_internal_mvcc_timestamp:3 t.public.a.x:4!null t.public.a.y:5 + ├── scan t.public.a + │ └── columns: t.public.a.x:1!null t.public.a.y:2 t.public.a.crdb_internal_mvcc_timestamp:3 + ├── project + │ ├── columns: t.public.a.x:4!null t.public.a.y:5 + │ └── scan t.public.a + │ └── columns: t.public.a.x:4!null t.public.a.y:5 t.public.a.crdb_internal_mvcc_timestamp:6 + └── filters (true) build fully-qualify-names SELECT * FROM t.a, a @@ -116,13 +122,15 @@ error (42712): source name "a" specified more than once (missing AS clause) build fully-qualify-names SELECT * FROM t.a, a AS a ---- -inner-join (cross) - ├── columns: x:1!null y:2 x:3!null y:4 - ├── scan t.public.a - │ └── columns: t.public.a.x:1!null t.public.a.y:2 - ├── scan t.public.a - │ └── columns: t.public.a.x:3!null t.public.a.y:4 - └── filters (true) +project + ├── columns: x:1!null y:2 x:4!null y:5 + └── inner-join (cross) + ├── columns: t.public.a.x:1!null t.public.a.y:2 t.public.a.crdb_internal_mvcc_timestamp:3 t.public.a.x:4!null t.public.a.y:5 t.public.a.crdb_internal_mvcc_timestamp:6 + ├── scan t.public.a + │ └── columns: t.public.a.x:1!null t.public.a.y:2 t.public.a.crdb_internal_mvcc_timestamp:3 + ├── scan t.public.a + │ └── columns: t.public.a.x:4!null t.public.a.y:5 t.public.a.crdb_internal_mvcc_timestamp:6 + └── filters (true) build fully-qualify-names SELECT a.* FROM t.a, a AS a diff --git a/pkg/sql/opt/optbuilder/testdata/insert b/pkg/sql/opt/optbuilder/testdata/insert index 3c516ce1f071..2757ec26754d 100644 --- a/pkg/sql/opt/optbuilder/testdata/insert +++ b/pkg/sql/opt/optbuilder/testdata/insert @@ -70,23 +70,23 @@ INSERT INTO abcde VALUES (1, 2, 3) insert abcde ├── columns: ├── insert-mapping: - │ ├── column1:7 => a:1 - │ ├── column2:8 => b:2 - │ ├── column3:9 => c:3 - │ ├── column11:11 => d:4 - │ ├── column1:7 => e:5 - │ └── column10:10 => rowid:6 + │ ├── column1:8 => a:1 + │ ├── column2:9 => b:2 + │ ├── column3:10 => c:3 + │ ├── column12:12 => d:4 + │ ├── column1:8 => e:5 + │ └── column11:11 => rowid:6 └── project - ├── columns: column11:11!null column1:7!null column2:8!null column3:9!null column10:10 + ├── columns: column12:12!null column1:8!null column2:9!null column3:10!null column11:11 ├── project - │ ├── columns: column10:10 column1:7!null column2:8!null column3:9!null + │ ├── columns: column11:11 column1:8!null column2:9!null column3:10!null │ ├── values - │ │ ├── columns: column1:7!null column2:8!null column3:9!null + │ │ ├── columns: column1:8!null column2:9!null column3:10!null │ │ └── (1, 2, 3) │ └── projections - │ └── unique_rowid() [as=column10:10] + │ └── unique_rowid() [as=column11:11] └── projections - └── (column2:8 + column3:9) + 1 [as=column11:11] + └── (column2:9 + column3:10) + 1 [as=column12:12] # Don't specify values for null or default columns. build @@ -95,25 +95,25 @@ INSERT INTO abcde VALUES (1) insert abcde ├── columns: ├── insert-mapping: - │ ├── column1:7 => a:1 - │ ├── column8:8 => b:2 - │ ├── column9:9 => c:3 - │ ├── column11:11 => d:4 - │ ├── column1:7 => e:5 - │ └── column10:10 => rowid:6 + │ ├── column1:8 => a:1 + │ ├── column9:9 => b:2 + │ ├── column10:10 => c:3 + │ ├── column12:12 => d:4 + │ ├── column1:8 => e:5 + │ └── column11:11 => rowid:6 └── project - ├── columns: column11:11 column1:7!null column8:8 column9:9!null column10:10 + ├── columns: column12:12 column1:8!null column9:9 column10:10!null column11:11 ├── project - │ ├── columns: column8:8 column9:9!null column10:10 column1:7!null + │ ├── columns: column9:9 column10:10!null column11:11 column1:8!null │ ├── values - │ │ ├── columns: column1:7!null + │ │ ├── columns: column1:8!null │ │ └── (1,) │ └── projections - │ ├── NULL::INT8 [as=column8:8] - │ ├── 10 [as=column9:9] - │ └── unique_rowid() [as=column10:10] + │ ├── NULL::INT8 [as=column9:9] + │ ├── 10 [as=column10:10] + │ └── unique_rowid() [as=column11:11] └── projections - └── (column8:8 + column9:9) + 1 [as=column11:11] + └── (column9:9 + column10:10) + 1 [as=column12:12] # Ordered input. build @@ -122,34 +122,34 @@ INSERT INTO abcde SELECT y FROM xyz ORDER BY y, z LIMIT 10 insert abcde ├── columns: ├── insert-mapping: - │ ├── y:8 => a:1 - │ ├── column10:10 => b:2 - │ ├── column11:11 => c:3 - │ ├── column13:13 => d:4 - │ ├── y:8 => e:5 - │ └── column12:12 => rowid:6 + │ ├── y:9 => a:1 + │ ├── column12:12 => b:2 + │ ├── column13:13 => c:3 + │ ├── column15:15 => d:4 + │ ├── y:9 => e:5 + │ └── column14:14 => rowid:6 └── project - ├── columns: column13:13 y:8 column10:10 column11:11!null column12:12 + ├── columns: column15:15 y:9 column12:12 column13:13!null column14:14 ├── project - │ ├── columns: column10:10 column11:11!null column12:12 y:8 + │ ├── columns: column12:12 column13:13!null column14:14 y:9 │ ├── limit - │ │ ├── columns: y:8 z:9 - │ │ ├── internal-ordering: +8,+9 + │ │ ├── columns: y:9 z:10 + │ │ ├── internal-ordering: +9,+10 │ │ ├── sort - │ │ │ ├── columns: y:8 z:9 - │ │ │ ├── ordering: +8,+9 + │ │ │ ├── columns: y:9 z:10 + │ │ │ ├── ordering: +9,+10 │ │ │ ├── limit hint: 10.00 │ │ │ └── project - │ │ │ ├── columns: y:8 z:9 + │ │ │ ├── columns: y:9 z:10 │ │ │ └── scan xyz - │ │ │ └── columns: x:7!null y:8 z:9 + │ │ │ └── columns: x:8!null y:9 z:10 xyz.crdb_internal_mvcc_timestamp:11 │ │ └── 10 │ └── projections - │ ├── NULL::INT8 [as=column10:10] - │ ├── 10 [as=column11:11] - │ └── unique_rowid() [as=column12:12] + │ ├── NULL::INT8 [as=column12:12] + │ ├── 10 [as=column13:13] + │ └── unique_rowid() [as=column14:14] └── projections - └── (column10:10 + column11:11) + 1 [as=column13:13] + └── (column12:12 + column13:13) + 1 [as=column15:15] # Ignore ORDER BY without LIMIT. build @@ -158,26 +158,26 @@ INSERT INTO abcde SELECT y FROM xyz ORDER BY y, z insert abcde ├── columns: ├── insert-mapping: - │ ├── y:8 => a:1 - │ ├── column10:10 => b:2 - │ ├── column11:11 => c:3 - │ ├── column13:13 => d:4 - │ ├── y:8 => e:5 - │ └── column12:12 => rowid:6 + │ ├── y:9 => a:1 + │ ├── column12:12 => b:2 + │ ├── column13:13 => c:3 + │ ├── column15:15 => d:4 + │ ├── y:9 => e:5 + │ └── column14:14 => rowid:6 └── project - ├── columns: column13:13 y:8 column10:10 column11:11!null column12:12 + ├── columns: column15:15 y:9 column12:12 column13:13!null column14:14 ├── project - │ ├── columns: column10:10 column11:11!null column12:12 y:8 + │ ├── columns: column12:12 column13:13!null column14:14 y:9 │ ├── project - │ │ ├── columns: y:8 z:9 + │ │ ├── columns: y:9 z:10 │ │ └── scan xyz - │ │ └── columns: x:7!null y:8 z:9 + │ │ └── columns: x:8!null y:9 z:10 xyz.crdb_internal_mvcc_timestamp:11 │ └── projections - │ ├── NULL::INT8 [as=column10:10] - │ ├── 10 [as=column11:11] - │ └── unique_rowid() [as=column12:12] + │ ├── NULL::INT8 [as=column12:12] + │ ├── 10 [as=column13:13] + │ └── unique_rowid() [as=column14:14] └── projections - └── (column10:10 + column11:11) + 1 [as=column13:13] + └── (column12:12 + column13:13) + 1 [as=column15:15] # Use placeholders. build @@ -186,11 +186,11 @@ INSERT INTO xyz VALUES ($1, $2, $3) insert xyz ├── columns: ├── insert-mapping: - │ ├── column1:4 => x:1 - │ ├── column2:5 => y:2 - │ └── column3:6 => z:3 + │ ├── column1:5 => x:1 + │ ├── column2:6 => y:2 + │ └── column3:7 => z:3 └── values - ├── columns: column1:4 column2:5 column3:6 + ├── columns: column1:5 column2:6 column3:7 └── ($1, $2, $3) # Null expressions. @@ -200,23 +200,23 @@ INSERT INTO abcde VALUES (2, null, null) insert abcde ├── columns: ├── insert-mapping: - │ ├── column1:7 => a:1 - │ ├── column2:8 => b:2 - │ ├── column3:9 => c:3 - │ ├── column11:11 => d:4 - │ ├── column1:7 => e:5 - │ └── column10:10 => rowid:6 + │ ├── column1:8 => a:1 + │ ├── column2:9 => b:2 + │ ├── column3:10 => c:3 + │ ├── column12:12 => d:4 + │ ├── column1:8 => e:5 + │ └── column11:11 => rowid:6 └── project - ├── columns: column11:11 column1:7!null column2:8 column3:9 column10:10 + ├── columns: column12:12 column1:8!null column2:9 column3:10 column11:11 ├── project - │ ├── columns: column10:10 column1:7!null column2:8 column3:9 + │ ├── columns: column11:11 column1:8!null column2:9 column3:10 │ ├── values - │ │ ├── columns: column1:7!null column2:8 column3:9 + │ │ ├── columns: column1:8!null column2:9 column3:10 │ │ └── (2, NULL::INT8, NULL::INT8) │ └── projections - │ └── unique_rowid() [as=column10:10] + │ └── unique_rowid() [as=column11:11] └── projections - └── (column2:8 + column3:9) + 1 [as=column11:11] + └── (column2:9 + column3:10) + 1 [as=column12:12] # Duplicate expressions. build @@ -225,27 +225,27 @@ INSERT INTO abcde SELECT 2, $1 + 1, $1 + 1 insert abcde ├── columns: ├── insert-mapping: - │ ├── "?column?":7 => a:1 - │ ├── "?column?":8 => b:2 - │ ├── "?column?":8 => c:3 - │ ├── column10:10 => d:4 - │ ├── "?column?":7 => e:5 - │ └── column9:9 => rowid:6 + │ ├── "?column?":8 => a:1 + │ ├── "?column?":9 => b:2 + │ ├── "?column?":9 => c:3 + │ ├── column11:11 => d:4 + │ ├── "?column?":8 => e:5 + │ └── column10:10 => rowid:6 └── project - ├── columns: column10:10 "?column?":7!null "?column?":8 column9:9 + ├── columns: column11:11 "?column?":8!null "?column?":9 column10:10 ├── project - │ ├── columns: column9:9 "?column?":7!null "?column?":8 + │ ├── columns: column10:10 "?column?":8!null "?column?":9 │ ├── project - │ │ ├── columns: "?column?":7!null "?column?":8 + │ │ ├── columns: "?column?":8!null "?column?":9 │ │ ├── values │ │ │ └── () │ │ └── projections - │ │ ├── 2 [as="?column?":7] - │ │ └── $1 + 1 [as="?column?":8] + │ │ ├── 2 [as="?column?":8] + │ │ └── $1 + 1 [as="?column?":9] │ └── projections - │ └── unique_rowid() [as=column9:9] + │ └── unique_rowid() [as=column10:10] └── projections - └── ("?column?":8 + "?column?":8) + 1 [as=column10:10] + └── ("?column?":9 + "?column?":9) + 1 [as=column11:11] # Use DEFAULT VALUES. build @@ -254,17 +254,17 @@ INSERT INTO uv DEFAULT VALUES insert uv ├── columns: ├── insert-mapping: - │ ├── column4:4 => u:1 - │ ├── column5:5 => v:2 - │ └── column6:6 => rowid:3 + │ ├── column5:5 => u:1 + │ ├── column6:6 => v:2 + │ └── column7:7 => rowid:3 └── project - ├── columns: column4:4 column5:5 column6:6 + ├── columns: column5:5 column6:6 column7:7 ├── values │ └── () └── projections - ├── NULL::DECIMAL [as=column4:4] - ├── NULL::BYTES [as=column5:5] - └── unique_rowid() [as=column6:6] + ├── NULL::DECIMAL [as=column5:5] + ├── NULL::BYTES [as=column6:6] + └── unique_rowid() [as=column7:7] # Use DEFAULT expressions in VALUES expression. build @@ -273,26 +273,26 @@ INSERT INTO abcde ((VALUES (1, DEFAULT, 2), (2, 3, 4), (3, 2, DEFAULT), (4, DEFA insert abcde ├── columns: ├── insert-mapping: - │ ├── column1:7 => a:1 - │ ├── column2:8 => b:2 - │ ├── column3:9 => c:3 - │ ├── column11:11 => d:4 - │ ├── column1:7 => e:5 - │ └── column10:10 => rowid:6 + │ ├── column1:8 => a:1 + │ ├── column2:9 => b:2 + │ ├── column3:10 => c:3 + │ ├── column12:12 => d:4 + │ ├── column1:8 => e:5 + │ └── column11:11 => rowid:6 └── project - ├── columns: column11:11 column1:7!null column2:8 column3:9!null column10:10 + ├── columns: column12:12 column1:8!null column2:9 column3:10!null column11:11 ├── project - │ ├── columns: column10:10 column1:7!null column2:8 column3:9!null + │ ├── columns: column11:11 column1:8!null column2:9 column3:10!null │ ├── values - │ │ ├── columns: column1:7!null column2:8 column3:9!null + │ │ ├── columns: column1:8!null column2:9 column3:10!null │ │ ├── (1, NULL::INT8, 2) │ │ ├── (2, 3, 4) │ │ ├── (3, 2, 10) │ │ └── (4, NULL::INT8, 10) │ └── projections - │ └── unique_rowid() [as=column10:10] + │ └── unique_rowid() [as=column11:11] └── projections - └── (column2:8 + column3:9) + 1 [as=column11:11] + └── (column2:9 + column3:10) + 1 [as=column12:12] # Use DEFAULT expressions in VALUES expression wrapped by WITH clause (error). build @@ -315,102 +315,102 @@ project └── insert abcde ├── columns: a:1!null b:2 c:3!null d:4 e:5!null rowid:6!null ├── insert-mapping: - │ ├── "?column?":7 => a:1 - │ ├── column8:8 => b:2 - │ ├── column9:9 => c:3 - │ ├── column11:11 => d:4 - │ ├── "?column?":7 => e:5 - │ └── column10:10 => rowid:6 + │ ├── "?column?":8 => a:1 + │ ├── column9:9 => b:2 + │ ├── column10:10 => c:3 + │ ├── column12:12 => d:4 + │ ├── "?column?":8 => e:5 + │ └── column11:11 => rowid:6 └── project - ├── columns: column11:11 "?column?":7!null column8:8 column9:9!null column10:10 + ├── columns: column12:12 "?column?":8!null column9:9 column10:10!null column11:11 ├── project - │ ├── columns: column8:8 column9:9!null column10:10 "?column?":7!null + │ ├── columns: column9:9 column10:10!null column11:11 "?column?":8!null │ ├── project - │ │ ├── columns: "?column?":7!null + │ │ ├── columns: "?column?":8!null │ │ ├── values │ │ │ └── () │ │ └── projections - │ │ └── 1 [as="?column?":7] + │ │ └── 1 [as="?column?":8] │ └── projections - │ ├── NULL::INT8 [as=column8:8] - │ ├── 10 [as=column9:9] - │ └── unique_rowid() [as=column10:10] + │ ├── NULL::INT8 [as=column9:9] + │ ├── 10 [as=column10:10] + │ └── unique_rowid() [as=column11:11] └── projections - └── (column8:8 + column9:9) + 1 [as=column11:11] + └── (column9:9 + column10:10) + 1 [as=column12:12] # Return values from aliased table. build INSERT INTO abcde AS foo SELECT 1 RETURNING foo.a + 1, foo.b * foo.c ---- project - ├── columns: "?column?":12!null "?column?":13 + ├── columns: "?column?":13!null "?column?":14 ├── insert foo │ ├── columns: a:1!null b:2 c:3!null d:4 e:5!null rowid:6!null │ ├── insert-mapping: - │ │ ├── "?column?":7 => a:1 - │ │ ├── column8:8 => b:2 - │ │ ├── column9:9 => c:3 - │ │ ├── column11:11 => d:4 - │ │ ├── "?column?":7 => e:5 - │ │ └── column10:10 => rowid:6 + │ │ ├── "?column?":8 => a:1 + │ │ ├── column9:9 => b:2 + │ │ ├── column10:10 => c:3 + │ │ ├── column12:12 => d:4 + │ │ ├── "?column?":8 => e:5 + │ │ └── column11:11 => rowid:6 │ └── project - │ ├── columns: column11:11 "?column?":7!null column8:8 column9:9!null column10:10 + │ ├── columns: column12:12 "?column?":8!null column9:9 column10:10!null column11:11 │ ├── project - │ │ ├── columns: column8:8 column9:9!null column10:10 "?column?":7!null + │ │ ├── columns: column9:9 column10:10!null column11:11 "?column?":8!null │ │ ├── project - │ │ │ ├── columns: "?column?":7!null + │ │ │ ├── columns: "?column?":8!null │ │ │ ├── values │ │ │ │ └── () │ │ │ └── projections - │ │ │ └── 1 [as="?column?":7] + │ │ │ └── 1 [as="?column?":8] │ │ └── projections - │ │ ├── NULL::INT8 [as=column8:8] - │ │ ├── 10 [as=column9:9] - │ │ └── unique_rowid() [as=column10:10] + │ │ ├── NULL::INT8 [as=column9:9] + │ │ ├── 10 [as=column10:10] + │ │ └── unique_rowid() [as=column11:11] │ └── projections - │ └── (column8:8 + column9:9) + 1 [as=column11:11] + │ └── (column9:9 + column10:10) + 1 [as=column12:12] └── projections - ├── a:1 + 1 [as="?column?":12] - └── b:2 * c:3 [as="?column?":13] + ├── a:1 + 1 [as="?column?":13] + └── b:2 * c:3 [as="?column?":14] # Use returning INSERT as a FROM expression. build SELECT * FROM [INSERT INTO abcde VALUES (1) RETURNING *] ---- with &1 - ├── columns: a:12!null b:13 c:14!null d:15 e:16!null + ├── columns: a:13!null b:14 c:15!null d:16 e:17!null ├── project │ ├── columns: abcde.a:1!null abcde.b:2 abcde.c:3!null abcde.d:4 abcde.e:5!null │ └── insert abcde │ ├── columns: abcde.a:1!null abcde.b:2 abcde.c:3!null abcde.d:4 abcde.e:5!null rowid:6!null │ ├── insert-mapping: - │ │ ├── column1:7 => abcde.a:1 - │ │ ├── column8:8 => abcde.b:2 - │ │ ├── column9:9 => abcde.c:3 - │ │ ├── column11:11 => abcde.d:4 - │ │ ├── column1:7 => abcde.e:5 - │ │ └── column10:10 => rowid:6 + │ │ ├── column1:8 => abcde.a:1 + │ │ ├── column9:9 => abcde.b:2 + │ │ ├── column10:10 => abcde.c:3 + │ │ ├── column12:12 => abcde.d:4 + │ │ ├── column1:8 => abcde.e:5 + │ │ └── column11:11 => rowid:6 │ └── project - │ ├── columns: column11:11 column1:7!null column8:8 column9:9!null column10:10 + │ ├── columns: column12:12 column1:8!null column9:9 column10:10!null column11:11 │ ├── project - │ │ ├── columns: column8:8 column9:9!null column10:10 column1:7!null + │ │ ├── columns: column9:9 column10:10!null column11:11 column1:8!null │ │ ├── values - │ │ │ ├── columns: column1:7!null + │ │ │ ├── columns: column1:8!null │ │ │ └── (1,) │ │ └── projections - │ │ ├── NULL::INT8 [as=column8:8] - │ │ ├── 10 [as=column9:9] - │ │ └── unique_rowid() [as=column10:10] + │ │ ├── NULL::INT8 [as=column9:9] + │ │ ├── 10 [as=column10:10] + │ │ └── unique_rowid() [as=column11:11] │ └── projections - │ └── (column8:8 + column9:9) + 1 [as=column11:11] + │ └── (column9:9 + column10:10) + 1 [as=column12:12] └── with-scan &1 - ├── columns: a:12!null b:13 c:14!null d:15 e:16!null + ├── columns: a:13!null b:14 c:15!null d:16 e:17!null └── mapping: - ├── abcde.a:1 => a:12 - ├── abcde.b:2 => b:13 - ├── abcde.c:3 => c:14 - ├── abcde.d:4 => d:15 - └── abcde.e:5 => e:16 + ├── abcde.a:1 => a:13 + ├── abcde.b:2 => b:14 + ├── abcde.c:3 => c:15 + ├── abcde.d:4 => d:16 + └── abcde.e:5 => e:17 # Try to use aggregate function in RETURNING clause. build @@ -436,34 +436,34 @@ WITH a AS (SELECT y, y+1 FROM xyz) INSERT INTO abcde SELECT * FROM a ---- with &1 (a) ├── project - │ ├── columns: "?column?":4 xyz.y:2 + │ ├── columns: "?column?":5 xyz.y:2 │ ├── scan xyz - │ │ └── columns: x:1!null xyz.y:2 z:3 + │ │ └── columns: x:1!null xyz.y:2 z:3 xyz.crdb_internal_mvcc_timestamp:4 │ └── projections - │ └── xyz.y:2 + 1 [as="?column?":4] + │ └── xyz.y:2 + 1 [as="?column?":5] └── insert abcde ├── columns: ├── insert-mapping: - │ ├── y:11 => a:5 - │ ├── "?column?":12 => b:6 - │ ├── column13:13 => c:7 - │ ├── column15:15 => d:8 - │ ├── y:11 => e:9 - │ └── column14:14 => rowid:10 + │ ├── y:13 => a:6 + │ ├── "?column?":14 => b:7 + │ ├── column15:15 => c:8 + │ ├── column17:17 => d:9 + │ ├── y:13 => e:10 + │ └── column16:16 => rowid:11 └── project - ├── columns: column15:15 y:11 "?column?":12 column13:13!null column14:14 + ├── columns: column17:17 y:13 "?column?":14 column15:15!null column16:16 ├── project - │ ├── columns: column13:13!null column14:14 y:11 "?column?":12 + │ ├── columns: column15:15!null column16:16 y:13 "?column?":14 │ ├── with-scan &1 (a) - │ │ ├── columns: y:11 "?column?":12 + │ │ ├── columns: y:13 "?column?":14 │ │ └── mapping: - │ │ ├── xyz.y:2 => y:11 - │ │ └── "?column?":4 => "?column?":12 + │ │ ├── xyz.y:2 => y:13 + │ │ └── "?column?":5 => "?column?":14 │ └── projections - │ ├── 10 [as=column13:13] - │ └── unique_rowid() [as=column14:14] + │ ├── 10 [as=column15:15] + │ └── unique_rowid() [as=column16:16] └── projections - └── ("?column?":12 + column13:13) + 1 [as=column15:15] + └── ("?column?":14 + column15:15) + 1 [as=column17:17] # Use CTE with multiple variables. build @@ -472,50 +472,50 @@ INSERT INTO abcde TABLE a UNION TABLE b ---- with &1 (a) ├── project - │ ├── columns: "?column?":4 xyz.y:2 + │ ├── columns: "?column?":5 xyz.y:2 │ ├── scan xyz - │ │ └── columns: x:1!null xyz.y:2 z:3 + │ │ └── columns: x:1!null xyz.y:2 z:3 xyz.crdb_internal_mvcc_timestamp:4 │ └── projections - │ └── xyz.y:2 + 1 [as="?column?":4] + │ └── xyz.y:2 + 1 [as="?column?":5] └── with &2 (b) ├── project - │ ├── columns: "?column?":8 xyz.y:6 + │ ├── columns: "?column?":10 xyz.y:7 │ ├── scan xyz - │ │ └── columns: x:5!null xyz.y:6 z:7 + │ │ └── columns: x:6!null xyz.y:7 z:8 xyz.crdb_internal_mvcc_timestamp:9 │ └── projections - │ └── xyz.y:6 + 1 [as="?column?":8] + │ └── xyz.y:7 + 1 [as="?column?":10] └── insert abcde ├── columns: ├── insert-mapping: - │ ├── y:19 => a:9 - │ ├── "?column?":20 => b:10 - │ ├── column21:21 => c:11 - │ ├── column23:23 => d:12 - │ ├── y:19 => e:13 - │ └── column22:22 => rowid:14 + │ ├── y:22 => a:11 + │ ├── "?column?":23 => b:12 + │ ├── column24:24 => c:13 + │ ├── column26:26 => d:14 + │ ├── y:22 => e:15 + │ └── column25:25 => rowid:16 └── project - ├── columns: column23:23 y:19 "?column?":20 column21:21!null column22:22 + ├── columns: column26:26 y:22 "?column?":23 column24:24!null column25:25 ├── project - │ ├── columns: column21:21!null column22:22 y:19 "?column?":20 + │ ├── columns: column24:24!null column25:25 y:22 "?column?":23 │ ├── union - │ │ ├── columns: y:19 "?column?":20 - │ │ ├── left columns: y:15 "?column?":16 - │ │ ├── right columns: "?column?":17 y:18 + │ │ ├── columns: y:22 "?column?":23 + │ │ ├── left columns: y:18 "?column?":19 + │ │ ├── right columns: "?column?":20 y:21 │ │ ├── with-scan &1 (a) - │ │ │ ├── columns: y:15 "?column?":16 + │ │ │ ├── columns: y:18 "?column?":19 │ │ │ └── mapping: - │ │ │ ├── xyz.y:2 => y:15 - │ │ │ └── "?column?":4 => "?column?":16 + │ │ │ ├── xyz.y:2 => y:18 + │ │ │ └── "?column?":5 => "?column?":19 │ │ └── with-scan &2 (b) - │ │ ├── columns: "?column?":17 y:18 + │ │ ├── columns: "?column?":20 y:21 │ │ └── mapping: - │ │ ├── "?column?":8 => "?column?":17 - │ │ └── xyz.y:6 => y:18 + │ │ ├── "?column?":10 => "?column?":20 + │ │ └── xyz.y:7 => y:21 │ └── projections - │ ├── 10 [as=column21:21] - │ └── unique_rowid() [as=column22:22] + │ ├── 10 [as=column24:24] + │ └── unique_rowid() [as=column25:25] └── projections - └── ("?column?":20 + column21:21) + 1 [as=column23:23] + └── ("?column?":23 + column24:24) + 1 [as=column26:26] # Non-referenced CTE with mutation. build @@ -527,58 +527,58 @@ with &1 │ └── insert abcde │ ├── columns: abcde.a:1!null abcde.b:2 abcde.c:3!null abcde.d:4 abcde.e:5!null rowid:6!null │ ├── insert-mapping: - │ │ ├── column1:7 => abcde.a:1 - │ │ ├── column8:8 => abcde.b:2 - │ │ ├── column9:9 => abcde.c:3 - │ │ ├── column11:11 => abcde.d:4 - │ │ ├── column1:7 => abcde.e:5 - │ │ └── column10:10 => rowid:6 + │ │ ├── column1:8 => abcde.a:1 + │ │ ├── column9:9 => abcde.b:2 + │ │ ├── column10:10 => abcde.c:3 + │ │ ├── column12:12 => abcde.d:4 + │ │ ├── column1:8 => abcde.e:5 + │ │ └── column11:11 => rowid:6 │ └── project - │ ├── columns: column11:11 column1:7!null column8:8 column9:9!null column10:10 + │ ├── columns: column12:12 column1:8!null column9:9 column10:10!null column11:11 │ ├── project - │ │ ├── columns: column8:8 column9:9!null column10:10 column1:7!null + │ │ ├── columns: column9:9 column10:10!null column11:11 column1:8!null │ │ ├── values - │ │ │ ├── columns: column1:7!null + │ │ │ ├── columns: column1:8!null │ │ │ └── (1,) │ │ └── projections - │ │ ├── NULL::INT8 [as=column8:8] - │ │ ├── 10 [as=column9:9] - │ │ └── unique_rowid() [as=column10:10] + │ │ ├── NULL::INT8 [as=column9:9] + │ │ ├── 10 [as=column10:10] + │ │ └── unique_rowid() [as=column11:11] │ └── projections - │ └── (column8:8 + column9:9) + 1 [as=column11:11] + │ └── (column9:9 + column10:10) + 1 [as=column12:12] └── with &2 (cte) ├── project - │ ├── columns: b:13 + │ ├── columns: b:14 │ └── with-scan &1 - │ ├── columns: a:12!null b:13 c:14!null d:15 e:16!null + │ ├── columns: a:13!null b:14 c:15!null d:16 e:17!null │ └── mapping: - │ ├── abcde.a:1 => a:12 - │ ├── abcde.b:2 => b:13 - │ ├── abcde.c:3 => c:14 - │ ├── abcde.d:4 => d:15 - │ └── abcde.e:5 => e:16 + │ ├── abcde.a:1 => a:13 + │ ├── abcde.b:2 => b:14 + │ ├── abcde.c:3 => c:15 + │ ├── abcde.d:4 => d:16 + │ └── abcde.e:5 => e:17 └── insert abcde ├── columns: ├── insert-mapping: - │ ├── column1:23 => abcde.a:17 - │ ├── column24:24 => abcde.b:18 - │ ├── column25:25 => abcde.c:19 - │ ├── column27:27 => abcde.d:20 - │ ├── column1:23 => abcde.e:21 - │ └── column26:26 => rowid:22 + │ ├── column1:25 => abcde.a:18 + │ ├── column26:26 => abcde.b:19 + │ ├── column27:27 => abcde.c:20 + │ ├── column29:29 => abcde.d:21 + │ ├── column1:25 => abcde.e:22 + │ └── column28:28 => rowid:23 └── project - ├── columns: column27:27 column1:23!null column24:24 column25:25!null column26:26 + ├── columns: column29:29 column1:25!null column26:26 column27:27!null column28:28 ├── project - │ ├── columns: column24:24 column25:25!null column26:26 column1:23!null + │ ├── columns: column26:26 column27:27!null column28:28 column1:25!null │ ├── values - │ │ ├── columns: column1:23!null + │ │ ├── columns: column1:25!null │ │ └── (1,) │ └── projections - │ ├── NULL::INT8 [as=column24:24] - │ ├── 10 [as=column25:25] - │ └── unique_rowid() [as=column26:26] + │ ├── NULL::INT8 [as=column26:26] + │ ├── 10 [as=column27:27] + │ └── unique_rowid() [as=column28:28] └── projections - └── (column24:24 + column25:25) + 1 [as=column27:27] + └── (column26:26 + column27:27) + 1 [as=column29:29] # Insert CTE that returns no columns. build @@ -603,23 +603,23 @@ INSERT INTO abcde (c, b, a) VALUES (1, 2, 3) insert abcde ├── columns: ├── insert-mapping: - │ ├── column3:9 => a:1 - │ ├── column2:8 => b:2 - │ ├── column1:7 => c:3 - │ ├── column11:11 => d:4 - │ ├── column3:9 => e:5 - │ └── column10:10 => rowid:6 + │ ├── column3:10 => a:1 + │ ├── column2:9 => b:2 + │ ├── column1:8 => c:3 + │ ├── column12:12 => d:4 + │ ├── column3:10 => e:5 + │ └── column11:11 => rowid:6 └── project - ├── columns: column11:11!null column1:7!null column2:8!null column3:9!null column10:10 + ├── columns: column12:12!null column1:8!null column2:9!null column3:10!null column11:11 ├── project - │ ├── columns: column10:10 column1:7!null column2:8!null column3:9!null + │ ├── columns: column11:11 column1:8!null column2:9!null column3:10!null │ ├── values - │ │ ├── columns: column1:7!null column2:8!null column3:9!null + │ │ ├── columns: column1:8!null column2:9!null column3:10!null │ │ └── (1, 2, 3) │ └── projections - │ └── unique_rowid() [as=column10:10] + │ └── unique_rowid() [as=column11:11] └── projections - └── (column2:8 + column1:7) + 1 [as=column11:11] + └── (column2:9 + column1:8) + 1 [as=column12:12] # Don't specify values for null or default columns. build @@ -628,25 +628,25 @@ INSERT INTO abcde (a) VALUES (1) insert abcde ├── columns: ├── insert-mapping: - │ ├── column1:7 => a:1 - │ ├── column8:8 => b:2 - │ ├── column9:9 => c:3 - │ ├── column11:11 => d:4 - │ ├── column1:7 => e:5 - │ └── column10:10 => rowid:6 + │ ├── column1:8 => a:1 + │ ├── column9:9 => b:2 + │ ├── column10:10 => c:3 + │ ├── column12:12 => d:4 + │ ├── column1:8 => e:5 + │ └── column11:11 => rowid:6 └── project - ├── columns: column11:11 column1:7!null column8:8 column9:9!null column10:10 + ├── columns: column12:12 column1:8!null column9:9 column10:10!null column11:11 ├── project - │ ├── columns: column8:8 column9:9!null column10:10 column1:7!null + │ ├── columns: column9:9 column10:10!null column11:11 column1:8!null │ ├── values - │ │ ├── columns: column1:7!null + │ │ ├── columns: column1:8!null │ │ └── (1,) │ └── projections - │ ├── NULL::INT8 [as=column8:8] - │ ├── 10 [as=column9:9] - │ └── unique_rowid() [as=column10:10] + │ ├── NULL::INT8 [as=column9:9] + │ ├── 10 [as=column10:10] + │ └── unique_rowid() [as=column11:11] └── projections - └── (column8:8 + column9:9) + 1 [as=column11:11] + └── (column9:9 + column10:10) + 1 [as=column12:12] # Insert value into hidden rowid column. build @@ -657,24 +657,24 @@ project └── insert abcde ├── columns: a:1!null b:2 c:3!null d:4 e:5!null rowid:6!null ├── insert-mapping: - │ ├── column1:7 => a:1 - │ ├── column9:9 => b:2 - │ ├── column10:10 => c:3 - │ ├── column11:11 => d:4 - │ ├── column1:7 => e:5 - │ └── column2:8 => rowid:6 + │ ├── column1:8 => a:1 + │ ├── column10:10 => b:2 + │ ├── column11:11 => c:3 + │ ├── column12:12 => d:4 + │ ├── column1:8 => e:5 + │ └── column2:9 => rowid:6 └── project - ├── columns: column11:11 column1:7!null column2:8!null column9:9 column10:10!null + ├── columns: column12:12 column1:8!null column2:9!null column10:10 column11:11!null ├── project - │ ├── columns: column9:9 column10:10!null column1:7!null column2:8!null + │ ├── columns: column10:10 column11:11!null column1:8!null column2:9!null │ ├── values - │ │ ├── columns: column1:7!null column2:8!null + │ │ ├── columns: column1:8!null column2:9!null │ │ └── (1, 2) │ └── projections - │ ├── NULL::INT8 [as=column9:9] - │ └── 10 [as=column10:10] + │ ├── NULL::INT8 [as=column10:10] + │ └── 10 [as=column11:11] └── projections - └── (column9:9 + column10:10) + 1 [as=column11:11] + └── (column10:10 + column11:11) + 1 [as=column12:12] # Use DEFAULT expressions in VALUES expression. build @@ -685,21 +685,21 @@ RETURNING *, rowid insert abcde ├── columns: a:1!null b:2 c:3!null d:4 e:5!null rowid:6!null ├── insert-mapping: - │ ├── column3:9 => a:1 - │ ├── column2:8 => b:2 - │ ├── column1:7 => c:3 - │ ├── column11:11 => d:4 - │ ├── column3:9 => e:5 - │ └── column4:10 => rowid:6 + │ ├── column3:10 => a:1 + │ ├── column2:9 => b:2 + │ ├── column1:8 => c:3 + │ ├── column12:12 => d:4 + │ ├── column3:10 => e:5 + │ └── column4:11 => rowid:6 └── project - ├── columns: column11:11 column1:7!null column2:8 column3:9!null column4:10 + ├── columns: column12:12 column1:8!null column2:9 column3:10!null column4:11 ├── values - │ ├── columns: column1:7!null column2:8 column3:9!null column4:10 + │ ├── columns: column1:8!null column2:9 column3:10!null column4:11 │ ├── (10, NULL::INT8, 1, unique_rowid()) │ ├── (3, 2, 1, unique_rowid()) │ └── (10, NULL::INT8, 2, 100) └── projections - └── (column2:8 + column1:7) + 1 [as=column11:11] + └── (column2:9 + column1:8) + 1 [as=column12:12] # Verify that there is no compile-time error when trying to insert a NULL # DEFAULT value into a not-null column (it will fail at runtime). @@ -709,25 +709,25 @@ INSERT INTO abcde (a) VALUES (DEFAULT) insert abcde ├── columns: ├── insert-mapping: - │ ├── column1:7 => a:1 - │ ├── column8:8 => b:2 - │ ├── column9:9 => c:3 - │ ├── column11:11 => d:4 - │ ├── column1:7 => e:5 - │ └── column10:10 => rowid:6 + │ ├── column1:8 => a:1 + │ ├── column9:9 => b:2 + │ ├── column10:10 => c:3 + │ ├── column12:12 => d:4 + │ ├── column1:8 => e:5 + │ └── column11:11 => rowid:6 └── project - ├── columns: column11:11 column1:7 column8:8 column9:9!null column10:10 + ├── columns: column12:12 column1:8 column9:9 column10:10!null column11:11 ├── project - │ ├── columns: column8:8 column9:9!null column10:10 column1:7 + │ ├── columns: column9:9 column10:10!null column11:11 column1:8 │ ├── values - │ │ ├── columns: column1:7 + │ │ ├── columns: column1:8 │ │ └── (NULL::INT8,) │ └── projections - │ ├── NULL::INT8 [as=column8:8] - │ ├── 10 [as=column9:9] - │ └── unique_rowid() [as=column10:10] + │ ├── NULL::INT8 [as=column9:9] + │ ├── 10 [as=column10:10] + │ └── unique_rowid() [as=column11:11] └── projections - └── (column8:8 + column9:9) + 1 [as=column11:11] + └── (column9:9 + column10:10) + 1 [as=column12:12] # Mismatched type. build @@ -780,27 +780,27 @@ project └── insert abcde ├── columns: a:1!null b:2!null c:3!null d:4!null e:5 rowid:6!null ├── insert-mapping: - │ ├── y:8 => a:1 - │ ├── x:10 => b:2 - │ ├── column11:11 => c:3 - │ ├── column13:13 => d:4 - │ ├── y:8 => e:5 - │ └── column12:12 => rowid:6 + │ ├── y:9 => a:1 + │ ├── x:12 => b:2 + │ ├── column13:13 => c:3 + │ ├── column15:15 => d:4 + │ ├── y:9 => e:5 + │ └── column14:14 => rowid:6 └── project - ├── columns: column13:13!null y:8 x:10!null column11:11!null column12:12 + ├── columns: column15:15!null y:9 x:12!null column13:13!null column14:14 ├── project - │ ├── columns: column11:11!null column12:12 y:8 x:10!null + │ ├── columns: column13:13!null column14:14 y:9 x:12!null │ ├── project - │ │ ├── columns: x:10!null y:8 + │ │ ├── columns: x:12!null y:9 │ │ ├── scan xyz - │ │ │ └── columns: xyz.x:7!null y:8 z:9 + │ │ │ └── columns: xyz.x:8!null y:9 z:10 xyz.crdb_internal_mvcc_timestamp:11 │ │ └── projections - │ │ └── xyz.x:7::INT8 [as=x:10] + │ │ └── xyz.x:8::INT8 [as=x:12] │ └── projections - │ ├── 10 [as=column11:11] - │ └── unique_rowid() [as=column12:12] + │ ├── 10 [as=column13:13] + │ └── unique_rowid() [as=column14:14] └── projections - └── (x:10 + column11:11) + 1 [as=column13:13] + └── (x:12 + column13:13) + 1 [as=column15:15] # Return hidden column. build @@ -809,65 +809,65 @@ INSERT INTO abcde (rowid, a) VALUES (1, 2) RETURNING *, rowid insert abcde ├── columns: a:1!null b:2 c:3!null d:4 e:5!null rowid:6!null ├── insert-mapping: - │ ├── column2:8 => a:1 - │ ├── column9:9 => b:2 - │ ├── column10:10 => c:3 - │ ├── column11:11 => d:4 - │ ├── column2:8 => e:5 - │ └── column1:7 => rowid:6 + │ ├── column2:9 => a:1 + │ ├── column10:10 => b:2 + │ ├── column11:11 => c:3 + │ ├── column12:12 => d:4 + │ ├── column2:9 => e:5 + │ └── column1:8 => rowid:6 └── project - ├── columns: column11:11 column1:7!null column2:8!null column9:9 column10:10!null + ├── columns: column12:12 column1:8!null column2:9!null column10:10 column11:11!null ├── project - │ ├── columns: column9:9 column10:10!null column1:7!null column2:8!null + │ ├── columns: column10:10 column11:11!null column1:8!null column2:9!null │ ├── values - │ │ ├── columns: column1:7!null column2:8!null + │ │ ├── columns: column1:8!null column2:9!null │ │ └── (1, 2) │ └── projections - │ ├── NULL::INT8 [as=column9:9] - │ └── 10 [as=column10:10] + │ ├── NULL::INT8 [as=column10:10] + │ └── 10 [as=column11:11] └── projections - └── (column9:9 + column10:10) + 1 [as=column11:11] + └── (column10:10 + column11:11) + 1 [as=column12:12] # Use returning INSERT as a FROM expression. build SELECT * FROM [INSERT INTO abcde (a, b) SELECT y+1, y FROM xyz RETURNING *] ---- with &1 - ├── columns: a:14!null b:15 c:16!null d:17 e:18 + ├── columns: a:16!null b:17 c:18!null d:19 e:20 ├── project │ ├── columns: abcde.a:1!null abcde.b:2 abcde.c:3!null abcde.d:4 abcde.e:5 │ └── insert abcde │ ├── columns: abcde.a:1!null abcde.b:2 abcde.c:3!null abcde.d:4 abcde.e:5 rowid:6!null │ ├── insert-mapping: - │ │ ├── "?column?":10 => abcde.a:1 - │ │ ├── y:8 => abcde.b:2 - │ │ ├── column11:11 => abcde.c:3 - │ │ ├── column13:13 => abcde.d:4 - │ │ ├── "?column?":10 => abcde.e:5 - │ │ └── column12:12 => rowid:6 + │ │ ├── "?column?":12 => abcde.a:1 + │ │ ├── y:9 => abcde.b:2 + │ │ ├── column13:13 => abcde.c:3 + │ │ ├── column15:15 => abcde.d:4 + │ │ ├── "?column?":12 => abcde.e:5 + │ │ └── column14:14 => rowid:6 │ └── project - │ ├── columns: column13:13 y:8 "?column?":10 column11:11!null column12:12 + │ ├── columns: column15:15 y:9 "?column?":12 column13:13!null column14:14 │ ├── project - │ │ ├── columns: column11:11!null column12:12 y:8 "?column?":10 + │ │ ├── columns: column13:13!null column14:14 y:9 "?column?":12 │ │ ├── project - │ │ │ ├── columns: "?column?":10 y:8 + │ │ │ ├── columns: "?column?":12 y:9 │ │ │ ├── scan xyz - │ │ │ │ └── columns: x:7!null y:8 z:9 + │ │ │ │ └── columns: x:8!null y:9 z:10 xyz.crdb_internal_mvcc_timestamp:11 │ │ │ └── projections - │ │ │ └── y:8 + 1 [as="?column?":10] + │ │ │ └── y:9 + 1 [as="?column?":12] │ │ └── projections - │ │ ├── 10 [as=column11:11] - │ │ └── unique_rowid() [as=column12:12] + │ │ ├── 10 [as=column13:13] + │ │ └── unique_rowid() [as=column14:14] │ └── projections - │ └── (y:8 + column11:11) + 1 [as=column13:13] + │ └── (y:9 + column13:13) + 1 [as=column15:15] └── with-scan &1 - ├── columns: a:14!null b:15 c:16!null d:17 e:18 + ├── columns: a:16!null b:17 c:18!null d:19 e:20 └── mapping: - ├── abcde.a:1 => a:14 - ├── abcde.b:2 => b:15 - ├── abcde.c:3 => c:16 - ├── abcde.d:4 => d:17 - └── abcde.e:5 => e:18 + ├── abcde.a:1 => a:16 + ├── abcde.b:2 => b:17 + ├── abcde.c:3 => c:18 + ├── abcde.d:4 => d:19 + └── abcde.e:5 => e:20 # ------------------------------------------------------------------------------ # Propagate desired INSERT types. @@ -880,11 +880,11 @@ INSERT INTO xyz VALUES ($1, $2 + 1, $3 + 1) insert xyz ├── columns: ├── insert-mapping: - │ ├── column1:4 => x:1 - │ ├── column2:5 => y:2 - │ └── column3:6 => z:3 + │ ├── column1:5 => x:1 + │ ├── column2:6 => y:2 + │ └── column3:7 => z:3 └── values - ├── columns: column1:4 column2:5 column3:6 + ├── columns: column1:5 column2:6 column3:7 └── ($1, $2 + 1, $3 + 1.0) # Propagate types to VALUES (named columns). @@ -894,11 +894,11 @@ INSERT INTO xyz (z, y, x) VALUES ($1 + 1, $2 + 1, $3) insert xyz ├── columns: ├── insert-mapping: - │ ├── column3:6 => x:1 - │ ├── column2:5 => y:2 - │ └── column1:4 => z:3 + │ ├── column3:7 => x:1 + │ ├── column2:6 => y:2 + │ └── column1:5 => z:3 └── values - ├── columns: column1:4 column2:5 column3:6 + ├── columns: column1:5 column2:6 column3:7 └── ($1 + 1.0, $2 + 1, $3) # Propagate types to projection list. @@ -908,17 +908,17 @@ INSERT INTO xyz ((SELECT $1, $2 + 1, $3 + 1)) insert xyz ├── columns: ├── insert-mapping: - │ ├── "?column?":4 => x:1 - │ ├── "?column?":5 => y:2 - │ └── "?column?":6 => z:3 + │ ├── "?column?":5 => x:1 + │ ├── "?column?":6 => y:2 + │ └── "?column?":7 => z:3 └── project - ├── columns: "?column?":4 "?column?":5 "?column?":6 + ├── columns: "?column?":5 "?column?":6 "?column?":7 ├── values │ └── () └── projections - ├── $1 [as="?column?":4] - ├── $2 + 1 [as="?column?":5] - └── $3 + 1.0 [as="?column?":6] + ├── $1 [as="?column?":5] + ├── $2 + 1 [as="?column?":6] + └── $3 + 1.0 [as="?column?":7] # Propagate types to projection list (named columns). build @@ -927,17 +927,17 @@ INSERT INTO xyz (x, y, z) SELECT $1, $2 + 1, $3 + 1 insert xyz ├── columns: ├── insert-mapping: - │ ├── "?column?":4 => x:1 - │ ├── "?column?":5 => y:2 - │ └── "?column?":6 => z:3 + │ ├── "?column?":5 => x:1 + │ ├── "?column?":6 => y:2 + │ └── "?column?":7 => z:3 └── project - ├── columns: "?column?":4 "?column?":5 "?column?":6 + ├── columns: "?column?":5 "?column?":6 "?column?":7 ├── values │ └── () └── projections - ├── $1 [as="?column?":4] - ├── $2 + 1 [as="?column?":5] - └── $3 + 1.0 [as="?column?":6] + ├── $1 [as="?column?":5] + ├── $2 + 1 [as="?column?":6] + └── $3 + 1.0 [as="?column?":7] # Propagate types to UNION. build @@ -946,29 +946,29 @@ INSERT INTO xyz (SELECT $1, $2 + 1, $3 + 1) UNION ALL (SELECT $1, $2 + 1, $3 + 1 insert xyz ├── columns: ├── insert-mapping: - │ ├── "?column?":10 => x:1 - │ ├── "?column?":11 => y:2 - │ └── "?column?":12 => z:3 + │ ├── "?column?":11 => x:1 + │ ├── "?column?":12 => y:2 + │ └── "?column?":13 => z:3 └── union-all - ├── columns: "?column?":10 "?column?":11 "?column?":12 - ├── left columns: "?column?":4 "?column?":5 "?column?":6 - ├── right columns: "?column?":7 "?column?":8 "?column?":9 + ├── columns: "?column?":11 "?column?":12 "?column?":13 + ├── left columns: "?column?":5 "?column?":6 "?column?":7 + ├── right columns: "?column?":8 "?column?":9 "?column?":10 ├── project - │ ├── columns: "?column?":4 "?column?":5 "?column?":6 + │ ├── columns: "?column?":5 "?column?":6 "?column?":7 │ ├── values │ │ └── () │ └── projections - │ ├── $1 [as="?column?":4] - │ ├── $2 + 1 [as="?column?":5] - │ └── $3 + 1.0 [as="?column?":6] + │ ├── $1 [as="?column?":5] + │ ├── $2 + 1 [as="?column?":6] + │ └── $3 + 1.0 [as="?column?":7] └── project - ├── columns: "?column?":7 "?column?":8 "?column?":9 + ├── columns: "?column?":8 "?column?":9 "?column?":10 ├── values │ └── () └── projections - ├── $1 [as="?column?":7] - ├── $2 + 1 [as="?column?":8] - └── $3 + 1.0 [as="?column?":9] + ├── $1 [as="?column?":8] + ├── $2 + 1 [as="?column?":9] + └── $3 + 1.0 [as="?column?":10] # Propagate types to UNION (named columns). build @@ -977,29 +977,29 @@ INSERT INTO xyz (x, z, y) SELECT $1, $2 + 1, $3 + 1 UNION ALL SELECT $1, $2 + 1, insert xyz ├── columns: ├── insert-mapping: - │ ├── "?column?":10 => x:1 - │ ├── "?column?":12 => y:2 - │ └── "?column?":11 => z:3 + │ ├── "?column?":11 => x:1 + │ ├── "?column?":13 => y:2 + │ └── "?column?":12 => z:3 └── union-all - ├── columns: "?column?":10 "?column?":11 "?column?":12 - ├── left columns: "?column?":4 "?column?":5 "?column?":6 - ├── right columns: "?column?":7 "?column?":8 "?column?":9 + ├── columns: "?column?":11 "?column?":12 "?column?":13 + ├── left columns: "?column?":5 "?column?":6 "?column?":7 + ├── right columns: "?column?":8 "?column?":9 "?column?":10 ├── project - │ ├── columns: "?column?":4 "?column?":5 "?column?":6 + │ ├── columns: "?column?":5 "?column?":6 "?column?":7 │ ├── values │ │ └── () │ └── projections - │ ├── $1 [as="?column?":4] - │ ├── $2 + 1.0 [as="?column?":5] - │ └── $3 + 1 [as="?column?":6] + │ ├── $1 [as="?column?":5] + │ ├── $2 + 1.0 [as="?column?":6] + │ └── $3 + 1 [as="?column?":7] └── project - ├── columns: "?column?":7 "?column?":8 "?column?":9 + ├── columns: "?column?":8 "?column?":9 "?column?":10 ├── values │ └── () └── projections - ├── $1 [as="?column?":7] - ├── $2 + 1.0 [as="?column?":8] - └── $3 + 1 [as="?column?":9] + ├── $1 [as="?column?":8] + ├── $2 + 1.0 [as="?column?":9] + └── $3 + 1 [as="?column?":10] # ------------------------------------------------------------------------------ # Tests with mutation columns. @@ -1012,26 +1012,26 @@ INSERT INTO mutation (m, n) VALUES (1, 2) insert mutation ├── columns: ├── insert-mapping: - │ ├── column1:6 => m:1 - │ ├── column2:7 => n:2 - │ ├── column8:8 => o:3 - │ └── column9:9 => p:4 - ├── check columns: check1:10 + │ ├── column1:7 => m:1 + │ ├── column2:8 => n:2 + │ ├── column9:9 => o:3 + │ └── column10:10 => p:4 + ├── check columns: check1:11 └── project - ├── columns: check1:10!null column1:6!null column2:7!null column8:8!null column9:9!null + ├── columns: check1:11!null column1:7!null column2:8!null column9:9!null column10:10!null ├── project - │ ├── columns: column9:9!null column1:6!null column2:7!null column8:8!null + │ ├── columns: column10:10!null column1:7!null column2:8!null column9:9!null │ ├── project - │ │ ├── columns: column8:8!null column1:6!null column2:7!null + │ │ ├── columns: column9:9!null column1:7!null column2:8!null │ │ ├── values - │ │ │ ├── columns: column1:6!null column2:7!null + │ │ │ ├── columns: column1:7!null column2:8!null │ │ │ └── (1, 2) │ │ └── projections - │ │ └── 10 [as=column8:8] + │ │ └── 10 [as=column9:9] │ └── projections - │ └── column8:8 + column2:7 [as=column9:9] + │ └── column9:9 + column2:8 [as=column10:10] └── projections - └── column1:6 > 0 [as=check1:10] + └── column1:7 > 0 [as=check1:11] # Use RETURNING clause and ensure that mutation columns aren't projected. build @@ -1040,26 +1040,26 @@ INSERT INTO mutation (m, n) VALUES (1, 2) RETURNING * insert mutation ├── columns: m:1!null n:2!null ├── insert-mapping: - │ ├── column1:6 => m:1 - │ ├── column2:7 => n:2 - │ ├── column8:8 => o:3 - │ └── column9:9 => p:4 - ├── check columns: check1:10 + │ ├── column1:7 => m:1 + │ ├── column2:8 => n:2 + │ ├── column9:9 => o:3 + │ └── column10:10 => p:4 + ├── check columns: check1:11 └── project - ├── columns: check1:10!null column1:6!null column2:7!null column8:8!null column9:9!null + ├── columns: check1:11!null column1:7!null column2:8!null column9:9!null column10:10!null ├── project - │ ├── columns: column9:9!null column1:6!null column2:7!null column8:8!null + │ ├── columns: column10:10!null column1:7!null column2:8!null column9:9!null │ ├── project - │ │ ├── columns: column8:8!null column1:6!null column2:7!null + │ │ ├── columns: column9:9!null column1:7!null column2:8!null │ │ ├── values - │ │ │ ├── columns: column1:6!null column2:7!null + │ │ │ ├── columns: column1:7!null column2:8!null │ │ │ └── (1, 2) │ │ └── projections - │ │ └── 10 [as=column8:8] + │ │ └── 10 [as=column9:9] │ └── projections - │ └── column8:8 + column2:7 [as=column9:9] + │ └── column9:9 + column2:8 [as=column10:10] └── projections - └── column1:6 > 0 [as=check1:10] + └── column1:7 > 0 [as=check1:11] # Try to reference write-only mutation column in RETURNING clause. build @@ -1090,23 +1090,23 @@ INSERT INTO checks (a, b, c) VALUES (1, 2, 3) insert checks ├── columns: ├── insert-mapping: - │ ├── column1:5 => a:1 - │ ├── column2:6 => b:2 - │ ├── column3:7 => c:3 - │ └── column8:8 => d:4 - ├── check columns: check1:9 check2:10 + │ ├── column1:6 => a:1 + │ ├── column2:7 => b:2 + │ ├── column3:8 => c:3 + │ └── column9:9 => d:4 + ├── check columns: check1:10 check2:11 └── project - ├── columns: check1:9!null check2:10!null column1:5!null column2:6!null column3:7!null column8:8!null + ├── columns: check1:10!null check2:11!null column1:6!null column2:7!null column3:8!null column9:9!null ├── project - │ ├── columns: column8:8!null column1:5!null column2:6!null column3:7!null + │ ├── columns: column9:9!null column1:6!null column2:7!null column3:8!null │ ├── values - │ │ ├── columns: column1:5!null column2:6!null column3:7!null + │ │ ├── columns: column1:6!null column2:7!null column3:8!null │ │ └── (1, 2, 3) │ └── projections - │ └── column3:7 + 1 [as=column8:8] + │ └── column3:8 + 1 [as=column9:9] └── projections - ├── column2:6 < column8:8 [as=check1:9] - └── column1:5 > 0 [as=check2:10] + ├── column2:7 < column9:9 [as=check1:10] + └── column1:6 > 0 [as=check2:11] # Insert results of SELECT. build @@ -1115,29 +1115,29 @@ INSERT INTO checks SELECT a, b, c FROM abcde insert checks ├── columns: ├── insert-mapping: - │ ├── abcde.a:5 => checks.a:1 - │ ├── abcde.b:6 => checks.b:2 - │ ├── abcde.c:7 => checks.c:3 - │ └── column11:11 => checks.d:4 - ├── check columns: check1:12 check2:13 + │ ├── abcde.a:6 => checks.a:1 + │ ├── abcde.b:7 => checks.b:2 + │ ├── abcde.c:8 => checks.c:3 + │ └── column13:13 => checks.d:4 + ├── check columns: check1:14 check2:15 └── project - ├── columns: check1:12 check2:13!null abcde.a:5!null abcde.b:6 abcde.c:7 column11:11 + ├── columns: check1:14 check2:15!null abcde.a:6!null abcde.b:7 abcde.c:8 column13:13 ├── project - │ ├── columns: column11:11 abcde.a:5!null abcde.b:6 abcde.c:7 + │ ├── columns: column13:13 abcde.a:6!null abcde.b:7 abcde.c:8 │ ├── project - │ │ ├── columns: abcde.a:5!null abcde.b:6 abcde.c:7 + │ │ ├── columns: abcde.a:6!null abcde.b:7 abcde.c:8 │ │ └── scan abcde - │ │ ├── columns: abcde.a:5!null abcde.b:6 abcde.c:7 abcde.d:8 e:9 rowid:10!null + │ │ ├── columns: abcde.a:6!null abcde.b:7 abcde.c:8 abcde.d:9 e:10 rowid:11!null abcde.crdb_internal_mvcc_timestamp:12 │ │ └── computed column expressions - │ │ ├── abcde.d:8 - │ │ │ └── (abcde.b:6 + abcde.c:7) + 1 - │ │ └── e:9 - │ │ └── abcde.a:5 + │ │ ├── abcde.d:9 + │ │ │ └── (abcde.b:7 + abcde.c:8) + 1 + │ │ └── e:10 + │ │ └── abcde.a:6 │ └── projections - │ └── abcde.c:7 + 1 [as=column11:11] + │ └── abcde.c:8 + 1 [as=column13:13] └── projections - ├── abcde.b:6 < column11:11 [as=check1:12] - └── abcde.a:5 > 0 [as=check2:13] + ├── abcde.b:7 < column13:13 [as=check1:14] + └── abcde.a:6 > 0 [as=check2:15] # ------------------------------------------------------------------------------ # Test decimal column rounding. @@ -1149,37 +1149,37 @@ INSERT INTO decimals (a, b) VALUES (1.1, ARRAY[0.95, NULL, 15]) insert decimals ├── columns: ├── insert-mapping: - │ ├── a:8 => decimals.a:1 - │ ├── b:9 => decimals.b:2 - │ ├── c:10 => decimals.c:3 - │ └── d:12 => decimals.d:4 - ├── check columns: check1:13 check2:14 + │ ├── a:9 => decimals.a:1 + │ ├── b:10 => decimals.b:2 + │ ├── c:11 => decimals.c:3 + │ └── d:13 => decimals.d:4 + ├── check columns: check1:14 check2:15 └── project - ├── columns: check1:13 check2:14 a:8 b:9 c:10 d:12 + ├── columns: check1:14 check2:15 a:9 b:10 c:11 d:13 ├── project - │ ├── columns: d:12 a:8 b:9 c:10 + │ ├── columns: d:13 a:9 b:10 c:11 │ ├── project - │ │ ├── columns: column11:11 a:8 b:9 c:10 + │ │ ├── columns: column12:12 a:9 b:10 c:11 │ │ ├── project - │ │ │ ├── columns: a:8 b:9 c:10 + │ │ │ ├── columns: a:9 b:10 c:11 │ │ │ ├── project - │ │ │ │ ├── columns: column7:7!null column1:5!null column2:6 + │ │ │ │ ├── columns: column8:8!null column1:6!null column2:7 │ │ │ │ ├── values - │ │ │ │ │ ├── columns: column1:5!null column2:6 + │ │ │ │ │ ├── columns: column1:6!null column2:7 │ │ │ │ │ └── (1.1, ARRAY[0.95,NULL,15]) │ │ │ │ └── projections - │ │ │ │ └── 1.23 [as=column7:7] + │ │ │ │ └── 1.23 [as=column8:8] │ │ │ └── projections - │ │ │ ├── crdb_internal.round_decimal_values(column1:5, 0) [as=a:8] - │ │ │ ├── crdb_internal.round_decimal_values(column2:6, 1) [as=b:9] - │ │ │ └── crdb_internal.round_decimal_values(column7:7, 1) [as=c:10] + │ │ │ ├── crdb_internal.round_decimal_values(column1:6, 0) [as=a:9] + │ │ │ ├── crdb_internal.round_decimal_values(column2:7, 1) [as=b:10] + │ │ │ └── crdb_internal.round_decimal_values(column8:8, 1) [as=c:11] │ │ └── projections - │ │ └── a:8 + c:10 [as=column11:11] + │ │ └── a:9 + c:11 [as=column12:12] │ └── projections - │ └── crdb_internal.round_decimal_values(column11:11, 1) [as=d:12] + │ └── crdb_internal.round_decimal_values(column12:12, 1) [as=d:13] └── projections - ├── round(a:8) = a:8 [as=check1:13] - └── b:9[0] > 1 [as=check2:14] + ├── round(a:9) = a:9 [as=check1:14] + └── b:10[0] > 1 [as=check2:15] # Regression test for #38293; the default values should be separate projections. exec-ddl @@ -1196,17 +1196,17 @@ INSERT INTO defvals(id) VALUES (1) insert defvals ├── columns: ├── insert-mapping: - │ ├── column1:4 => id:1 - │ ├── column5:5 => arr1:2 - │ └── column6:6 => arr2:3 + │ ├── column1:5 => id:1 + │ ├── column6:6 => arr1:2 + │ └── column7:7 => arr2:3 └── project - ├── columns: column5:5!null column6:6!null column1:4!null + ├── columns: column6:6!null column7:7!null column1:5!null ├── values - │ ├── columns: column1:4!null + │ ├── columns: column1:5!null │ └── (1,) └── projections - ├── ARRAY[] [as=column5:5] - └── ARRAY[] [as=column6:6] + ├── ARRAY[] [as=column6:6] + └── ARRAY[] [as=column7:7] exec-ddl CREATE TABLE defvals2 ( @@ -1222,17 +1222,17 @@ INSERT INTO defvals2(id) VALUES (1) insert defvals2 ├── columns: ├── insert-mapping: - │ ├── column1:4 => id:1 - │ ├── column5:5 => arr1:2 - │ └── column6:6 => arr2:3 + │ ├── column1:5 => id:1 + │ ├── column6:6 => arr1:2 + │ └── column7:7 => arr2:3 └── project - ├── columns: column5:5 column6:6 column1:4!null + ├── columns: column6:6 column7:7 column1:5!null ├── values - │ ├── columns: column1:4!null + │ ├── columns: column1:5!null │ └── (1,) └── projections - ├── ARRAY[NULL] [as=column5:5] - └── ARRAY[NULL] [as=column6:6] + ├── ARRAY[NULL] [as=column6:6] + └── ARRAY[NULL] [as=column7:7] # ------------------------------------------------------------------------------ # Test partial index column values. @@ -1257,17 +1257,17 @@ INSERT INTO partial_indexes VALUES (2, 1, 'bar') insert partial_indexes ├── columns: ├── insert-mapping: - │ ├── column1:4 => a:1 - │ ├── column2:5 => b:2 - │ └── column3:6 => c:3 - ├── partial index put columns: partial_index_put1:7 partial_index_put2:8 partial_index_put3:9 partial_index_put4:10 + │ ├── column1:5 => a:1 + │ ├── column2:6 => b:2 + │ └── column3:7 => c:3 + ├── partial index put columns: partial_index_put1:8 partial_index_put2:9 partial_index_put3:10 partial_index_put4:11 └── project - ├── columns: partial_index_put1:7!null partial_index_put2:8!null partial_index_put3:9!null partial_index_put4:10!null column1:4!null column2:5!null column3:6!null + ├── columns: partial_index_put1:8!null partial_index_put2:9!null partial_index_put3:10!null partial_index_put4:11!null column1:5!null column2:6!null column3:7!null ├── values - │ ├── columns: column1:4!null column2:5!null column3:6!null + │ ├── columns: column1:5!null column2:6!null column3:7!null │ └── (2, 1, 'bar') └── projections - ├── column3:6 = 'foo' [as=partial_index_put1:7] - ├── (column1:4 > column2:5) AND (column3:6 = 'bar') [as=partial_index_put2:8] - ├── column3:6 = 'delete-only' [as=partial_index_put3:9] - └── column3:6 = 'write-only' [as=partial_index_put4:10] + ├── column3:7 = 'foo' [as=partial_index_put1:8] + ├── (column1:5 > column2:6) AND (column3:7 = 'bar') [as=partial_index_put2:9] + ├── column3:7 = 'delete-only' [as=partial_index_put3:10] + └── column3:7 = 'write-only' [as=partial_index_put4:11] diff --git a/pkg/sql/opt/optbuilder/testdata/join b/pkg/sql/opt/optbuilder/testdata/join index bca0d691f82b..5fb16b49584b 100644 --- a/pkg/sql/opt/optbuilder/testdata/join +++ b/pkg/sql/opt/optbuilder/testdata/join @@ -8,13 +8,13 @@ build SELECT * FROM onecolumn AS a(x) CROSS JOIN onecolumn AS b(y) ---- project - ├── columns: x:1 y:3 + ├── columns: x:1 y:4 └── inner-join (cross) - ├── columns: x:1 a.rowid:2!null y:3 b.rowid:4!null + ├── columns: x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 y:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 ├── scan a - │ └── columns: x:1 a.rowid:2!null + │ └── columns: x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: y:3 b.rowid:4!null + │ └── columns: y:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 └── filters (true) # Check that name resolution chokes on ambiguity when it needs to. @@ -33,7 +33,7 @@ SELECT x FROM (SELECT 1 AS x), onecolumn AS a, onecolumn AS b project ├── columns: x:1!null └── inner-join (cross) - ├── columns: x:1!null a.x:2 a.rowid:3!null b.x:4 b.rowid:5!null + ├── columns: x:1!null a.x:2 a.rowid:3!null a.crdb_internal_mvcc_timestamp:4 b.x:5 b.rowid:6!null b.crdb_internal_mvcc_timestamp:7 ├── project │ ├── columns: x:1!null │ ├── values @@ -41,11 +41,11 @@ project │ └── projections │ └── 1 [as=x:1] ├── inner-join (cross) - │ ├── columns: a.x:2 a.rowid:3!null b.x:4 b.rowid:5!null + │ ├── columns: a.x:2 a.rowid:3!null a.crdb_internal_mvcc_timestamp:4 b.x:5 b.rowid:6!null b.crdb_internal_mvcc_timestamp:7 │ ├── scan a - │ │ └── columns: a.x:2 a.rowid:3!null + │ │ └── columns: a.x:2 a.rowid:3!null a.crdb_internal_mvcc_timestamp:4 │ ├── scan b - │ │ └── columns: b.x:4 b.rowid:5!null + │ │ └── columns: b.x:5 b.rowid:6!null b.crdb_internal_mvcc_timestamp:7 │ └── filters (true) └── filters (true) @@ -53,15 +53,15 @@ build SELECT * FROM onecolumn AS a(x) JOIN onecolumn AS b(y) ON a.x = b.y ---- project - ├── columns: x:1!null y:3!null + ├── columns: x:1!null y:4!null └── inner-join (hash) - ├── columns: x:1!null a.rowid:2!null y:3!null b.rowid:4!null + ├── columns: x:1!null a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 y:4!null b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 ├── scan a - │ └── columns: x:1 a.rowid:2!null + │ └── columns: x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: y:3 b.rowid:4!null + │ └── columns: y:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 └── filters - └── x:1 = y:3 + └── x:1 = y:4 build SELECT * FROM onecolumn AS a JOIN onecolumn as b USING(x) ORDER BY x @@ -72,13 +72,13 @@ sort └── project ├── columns: a.x:1!null └── inner-join (hash) - ├── columns: a.x:1!null a.rowid:2!null b.x:3!null b.rowid:4!null + ├── columns: a.x:1!null a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 b.x:4!null b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 ├── scan a - │ └── columns: a.x:1 a.rowid:2!null + │ └── columns: a.x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: b.x:3 b.rowid:4!null + │ └── columns: b.x:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 └── filters - └── a.x:1 = b.x:3 + └── a.x:1 = b.x:4 build SELECT * FROM onecolumn AS a NATURAL JOIN onecolumn as b @@ -86,27 +86,27 @@ SELECT * FROM onecolumn AS a NATURAL JOIN onecolumn as b project ├── columns: x:1!null └── inner-join (hash) - ├── columns: a.x:1!null a.rowid:2!null b.x:3!null b.rowid:4!null + ├── columns: a.x:1!null a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 b.x:4!null b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 ├── scan a - │ └── columns: a.x:1 a.rowid:2!null + │ └── columns: a.x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: b.x:3 b.rowid:4!null + │ └── columns: b.x:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 └── filters - └── a.x:1 = b.x:3 + └── a.x:1 = b.x:4 build SELECT * FROM onecolumn AS a(x) LEFT OUTER JOIN onecolumn AS b(y) ON a.x = b.y ---- project - ├── columns: x:1 y:3 + ├── columns: x:1 y:4 └── left-join (hash) - ├── columns: x:1 a.rowid:2!null y:3 b.rowid:4 + ├── columns: x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 y:4 b.rowid:5 b.crdb_internal_mvcc_timestamp:6 ├── scan a - │ └── columns: x:1 a.rowid:2!null + │ └── columns: x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: y:3 b.rowid:4!null + │ └── columns: y:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 └── filters - └── x:1 = y:3 + └── x:1 = y:4 build SELECT * FROM onecolumn AS a LEFT OUTER JOIN onecolumn AS b USING(x) ORDER BY x @@ -117,13 +117,13 @@ sort └── project ├── columns: a.x:1 └── left-join (hash) - ├── columns: a.x:1 a.rowid:2!null b.x:3 b.rowid:4 + ├── columns: a.x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 b.x:4 b.rowid:5 b.crdb_internal_mvcc_timestamp:6 ├── scan a - │ └── columns: a.x:1 a.rowid:2!null + │ └── columns: a.x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: b.x:3 b.rowid:4!null + │ └── columns: b.x:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 └── filters - └── a.x:1 = b.x:3 + └── a.x:1 = b.x:4 # Check that ORDER BY chokes on ambiguity if no table less columns # were introduced by USING. (#12239) @@ -158,58 +158,58 @@ SELECT * FROM onecolumn AS a NATURAL LEFT OUTER JOIN onecolumn AS b project ├── columns: x:1 └── left-join (hash) - ├── columns: a.x:1 a.rowid:2!null b.x:3 b.rowid:4 + ├── columns: a.x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 b.x:4 b.rowid:5 b.crdb_internal_mvcc_timestamp:6 ├── scan a - │ └── columns: a.x:1 a.rowid:2!null + │ └── columns: a.x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: b.x:3 b.rowid:4!null + │ └── columns: b.x:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 └── filters - └── a.x:1 = b.x:3 + └── a.x:1 = b.x:4 build SELECT * FROM onecolumn AS a(x) RIGHT OUTER JOIN onecolumn AS b(y) ON a.x = b.y ---- project - ├── columns: x:1 y:3 + ├── columns: x:1 y:4 └── right-join (hash) - ├── columns: x:1 a.rowid:2 y:3 b.rowid:4!null + ├── columns: x:1 a.rowid:2 a.crdb_internal_mvcc_timestamp:3 y:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 ├── scan a - │ └── columns: x:1 a.rowid:2!null + │ └── columns: x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: y:3 b.rowid:4!null + │ └── columns: y:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 └── filters - └── x:1 = y:3 + └── x:1 = y:4 build SELECT * FROM onecolumn AS a RIGHT OUTER JOIN onecolumn AS b USING(x) ORDER BY x ---- sort - ├── columns: x:3 - ├── ordering: +3 + ├── columns: x:4 + ├── ordering: +4 └── project - ├── columns: b.x:3 + ├── columns: b.x:4 └── right-join (hash) - ├── columns: a.x:1 a.rowid:2 b.x:3 b.rowid:4!null + ├── columns: a.x:1 a.rowid:2 a.crdb_internal_mvcc_timestamp:3 b.x:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 ├── scan a - │ └── columns: a.x:1 a.rowid:2!null + │ └── columns: a.x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: b.x:3 b.rowid:4!null + │ └── columns: b.x:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 └── filters - └── a.x:1 = b.x:3 + └── a.x:1 = b.x:4 build SELECT * FROM onecolumn AS a NATURAL RIGHT OUTER JOIN onecolumn AS b ---- project - ├── columns: x:3 + ├── columns: x:4 └── right-join (hash) - ├── columns: a.x:1 a.rowid:2 b.x:3 b.rowid:4!null + ├── columns: a.x:1 a.rowid:2 a.crdb_internal_mvcc_timestamp:3 b.x:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 ├── scan a - │ └── columns: a.x:1 a.rowid:2!null + │ └── columns: a.x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: b.x:3 b.rowid:4!null + │ └── columns: b.x:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 └── filters - └── a.x:1 = b.x:3 + └── a.x:1 = b.x:4 exec-ddl CREATE TABLE onecolumn_w(w INT) @@ -219,13 +219,13 @@ build SELECT * FROM onecolumn AS a NATURAL JOIN onecolumn_w as b ---- project - ├── columns: x:1 w:3 + ├── columns: x:1 w:4 └── inner-join (cross) - ├── columns: x:1 a.rowid:2!null w:3 b.rowid:4!null + ├── columns: x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 w:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 ├── scan a - │ └── columns: x:1 a.rowid:2!null + │ └── columns: x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: w:3 b.rowid:4!null + │ └── columns: w:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 └── filters (true) exec-ddl @@ -236,39 +236,39 @@ build SELECT * FROM onecolumn AS a FULL OUTER JOIN othercolumn AS b ON a.x = b.x ORDER BY a.x,b.x ---- sort - ├── columns: x:1 x:3 - ├── ordering: +1,+3 + ├── columns: x:1 x:4 + ├── ordering: +1,+4 └── project - ├── columns: a.x:1 b.x:3 + ├── columns: a.x:1 b.x:4 └── full-join (hash) - ├── columns: a.x:1 a.rowid:2 b.x:3 b.rowid:4 + ├── columns: a.x:1 a.rowid:2 a.crdb_internal_mvcc_timestamp:3 b.x:4 b.rowid:5 b.crdb_internal_mvcc_timestamp:6 ├── scan a - │ └── columns: a.x:1 a.rowid:2!null + │ └── columns: a.x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: b.x:3 b.rowid:4!null + │ └── columns: b.x:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 └── filters - └── a.x:1 = b.x:3 + └── a.x:1 = b.x:4 build SELECT * FROM onecolumn AS a FULL OUTER JOIN othercolumn AS b USING(x) ORDER BY x ---- sort - ├── columns: x:5 - ├── ordering: +5 + ├── columns: x:7 + ├── ordering: +7 └── project - ├── columns: x:5 + ├── columns: x:7 └── project - ├── columns: x:5 a.x:1 a.rowid:2 b.x:3 b.rowid:4 + ├── columns: x:7 a.x:1 a.rowid:2 a.crdb_internal_mvcc_timestamp:3 b.x:4 b.rowid:5 b.crdb_internal_mvcc_timestamp:6 ├── full-join (hash) - │ ├── columns: a.x:1 a.rowid:2 b.x:3 b.rowid:4 + │ ├── columns: a.x:1 a.rowid:2 a.crdb_internal_mvcc_timestamp:3 b.x:4 b.rowid:5 b.crdb_internal_mvcc_timestamp:6 │ ├── scan a - │ │ └── columns: a.x:1 a.rowid:2!null + │ │ └── columns: a.x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 │ ├── scan b - │ │ └── columns: b.x:3 b.rowid:4!null + │ │ └── columns: b.x:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 │ └── filters - │ └── a.x:1 = b.x:3 + │ └── a.x:1 = b.x:4 └── projections - └── COALESCE(a.x:1, b.x:3) [as=x:5] + └── COALESCE(a.x:1, b.x:4) [as=x:7] # Check that the source columns can be selected separately from the # USING column (#12033). @@ -276,43 +276,43 @@ build SELECT x AS s, a.x, b.x FROM onecolumn AS a FULL OUTER JOIN othercolumn AS b USING(x) ORDER BY s ---- sort - ├── columns: s:5 x:1 x:3 - ├── ordering: +5 + ├── columns: s:7 x:1 x:4 + ├── ordering: +7 └── project - ├── columns: a.x:1 b.x:3 x:5 + ├── columns: a.x:1 b.x:4 x:7 └── project - ├── columns: x:5 a.x:1 a.rowid:2 b.x:3 b.rowid:4 + ├── columns: x:7 a.x:1 a.rowid:2 a.crdb_internal_mvcc_timestamp:3 b.x:4 b.rowid:5 b.crdb_internal_mvcc_timestamp:6 ├── full-join (hash) - │ ├── columns: a.x:1 a.rowid:2 b.x:3 b.rowid:4 + │ ├── columns: a.x:1 a.rowid:2 a.crdb_internal_mvcc_timestamp:3 b.x:4 b.rowid:5 b.crdb_internal_mvcc_timestamp:6 │ ├── scan a - │ │ └── columns: a.x:1 a.rowid:2!null + │ │ └── columns: a.x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 │ ├── scan b - │ │ └── columns: b.x:3 b.rowid:4!null + │ │ └── columns: b.x:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 │ └── filters - │ └── a.x:1 = b.x:3 + │ └── a.x:1 = b.x:4 └── projections - └── COALESCE(a.x:1, b.x:3) [as=x:5] + └── COALESCE(a.x:1, b.x:4) [as=x:7] build SELECT * FROM onecolumn AS a NATURAL FULL OUTER JOIN othercolumn AS b ORDER BY x ---- sort - ├── columns: x:5 - ├── ordering: +5 + ├── columns: x:7 + ├── ordering: +7 └── project - ├── columns: x:5 + ├── columns: x:7 └── project - ├── columns: x:5 a.x:1 a.rowid:2 b.x:3 b.rowid:4 + ├── columns: x:7 a.x:1 a.rowid:2 a.crdb_internal_mvcc_timestamp:3 b.x:4 b.rowid:5 b.crdb_internal_mvcc_timestamp:6 ├── full-join (hash) - │ ├── columns: a.x:1 a.rowid:2 b.x:3 b.rowid:4 + │ ├── columns: a.x:1 a.rowid:2 a.crdb_internal_mvcc_timestamp:3 b.x:4 b.rowid:5 b.crdb_internal_mvcc_timestamp:6 │ ├── scan a - │ │ └── columns: a.x:1 a.rowid:2!null + │ │ └── columns: a.x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 │ ├── scan b - │ │ └── columns: b.x:3 b.rowid:4!null + │ │ └── columns: b.x:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 │ └── filters - │ └── a.x:1 = b.x:3 + │ └── a.x:1 = b.x:4 └── projections - └── COALESCE(a.x:1, b.x:3) [as=x:5] + └── COALESCE(a.x:1, b.x:4) [as=x:7] # Check that a limit on the JOIN's result do not cause rows from the # JOIN operands to become invisible to the JOIN. @@ -325,17 +325,17 @@ limit │ ├── columns: x:1!null │ ├── limit hint: 1.00 │ └── inner-join (hash) - │ ├── columns: x:1!null column1:3!null + │ ├── columns: x:1!null column1:4!null │ ├── limit hint: 1.00 │ ├── project │ │ ├── columns: x:1 │ │ └── scan onecolumn - │ │ └── columns: x:1 rowid:2!null + │ │ └── columns: x:1 rowid:2!null crdb_internal_mvcc_timestamp:3 │ ├── values - │ │ ├── columns: column1:3!null + │ │ ├── columns: column1:4!null │ │ └── (42,) │ └── filters - │ └── x:1 = column1:3 + │ └── x:1 = column1:4 └── 1 exec-ddl @@ -346,41 +346,41 @@ build SELECT * FROM onecolumn AS a(x) CROSS JOIN empty AS b(y) ---- project - ├── columns: x:1 y:3 + ├── columns: x:1 y:4 └── inner-join (cross) - ├── columns: x:1 a.rowid:2!null y:3 b.rowid:4!null + ├── columns: x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 y:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 ├── scan a - │ └── columns: x:1 a.rowid:2!null + │ └── columns: x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: y:3 b.rowid:4!null + │ └── columns: y:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 └── filters (true) build SELECT * FROM empty AS a CROSS JOIN onecolumn AS b ---- project - ├── columns: x:1 x:3 + ├── columns: x:1 x:4 └── inner-join (cross) - ├── columns: a.x:1 a.rowid:2!null b.x:3 b.rowid:4!null + ├── columns: a.x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 b.x:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 ├── scan a - │ └── columns: a.x:1 a.rowid:2!null + │ └── columns: a.x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: b.x:3 b.rowid:4!null + │ └── columns: b.x:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 └── filters (true) build SELECT * FROM onecolumn AS a(x) JOIN empty AS b(y) ON a.x = b.y ---- project - ├── columns: x:1!null y:3!null + ├── columns: x:1!null y:4!null └── inner-join (hash) - ├── columns: x:1!null a.rowid:2!null y:3!null b.rowid:4!null + ├── columns: x:1!null a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 y:4!null b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 ├── scan a - │ └── columns: x:1 a.rowid:2!null + │ └── columns: x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: y:3 b.rowid:4!null + │ └── columns: y:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 └── filters - └── x:1 = y:3 + └── x:1 = y:4 build SELECT * FROM onecolumn AS a JOIN empty AS b USING(x) @@ -388,27 +388,27 @@ SELECT * FROM onecolumn AS a JOIN empty AS b USING(x) project ├── columns: x:1!null └── inner-join (hash) - ├── columns: a.x:1!null a.rowid:2!null b.x:3!null b.rowid:4!null + ├── columns: a.x:1!null a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 b.x:4!null b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 ├── scan a - │ └── columns: a.x:1 a.rowid:2!null + │ └── columns: a.x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: b.x:3 b.rowid:4!null + │ └── columns: b.x:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 └── filters - └── a.x:1 = b.x:3 + └── a.x:1 = b.x:4 build SELECT * FROM empty AS a(x) JOIN onecolumn AS b(y) ON a.x = b.y ---- project - ├── columns: x:1!null y:3!null + ├── columns: x:1!null y:4!null └── inner-join (hash) - ├── columns: x:1!null a.rowid:2!null y:3!null b.rowid:4!null + ├── columns: x:1!null a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 y:4!null b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 ├── scan a - │ └── columns: x:1 a.rowid:2!null + │ └── columns: x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: y:3 b.rowid:4!null + │ └── columns: y:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 └── filters - └── x:1 = y:3 + └── x:1 = y:4 build SELECT * FROM empty AS a JOIN onecolumn AS b USING(x) @@ -416,30 +416,30 @@ SELECT * FROM empty AS a JOIN onecolumn AS b USING(x) project ├── columns: x:1!null └── inner-join (hash) - ├── columns: a.x:1!null a.rowid:2!null b.x:3!null b.rowid:4!null + ├── columns: a.x:1!null a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 b.x:4!null b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 ├── scan a - │ └── columns: a.x:1 a.rowid:2!null + │ └── columns: a.x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: b.x:3 b.rowid:4!null + │ └── columns: b.x:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 └── filters - └── a.x:1 = b.x:3 + └── a.x:1 = b.x:4 build SELECT * FROM onecolumn AS a(x) LEFT OUTER JOIN empty AS b(y) ON a.x = b.y ORDER BY a.x ---- sort - ├── columns: x:1 y:3 + ├── columns: x:1 y:4 ├── ordering: +1 └── project - ├── columns: x:1 y:3 + ├── columns: x:1 y:4 └── left-join (hash) - ├── columns: x:1 a.rowid:2!null y:3 b.rowid:4 + ├── columns: x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 y:4 b.rowid:5 b.crdb_internal_mvcc_timestamp:6 ├── scan a - │ └── columns: x:1 a.rowid:2!null + │ └── columns: x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: y:3 b.rowid:4!null + │ └── columns: y:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 └── filters - └── x:1 = y:3 + └── x:1 = y:4 build SELECT * FROM onecolumn AS a LEFT OUTER JOIN empty AS b USING(x) ORDER BY x @@ -450,27 +450,27 @@ sort └── project ├── columns: a.x:1 └── left-join (hash) - ├── columns: a.x:1 a.rowid:2!null b.x:3 b.rowid:4 + ├── columns: a.x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 b.x:4 b.rowid:5 b.crdb_internal_mvcc_timestamp:6 ├── scan a - │ └── columns: a.x:1 a.rowid:2!null + │ └── columns: a.x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: b.x:3 b.rowid:4!null + │ └── columns: b.x:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 └── filters - └── a.x:1 = b.x:3 + └── a.x:1 = b.x:4 build SELECT * FROM empty AS a(x) LEFT OUTER JOIN onecolumn AS b(y) ON a.x = b.y ---- project - ├── columns: x:1 y:3 + ├── columns: x:1 y:4 └── left-join (hash) - ├── columns: x:1 a.rowid:2!null y:3 b.rowid:4 + ├── columns: x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 y:4 b.rowid:5 b.crdb_internal_mvcc_timestamp:6 ├── scan a - │ └── columns: x:1 a.rowid:2!null + │ └── columns: x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: y:3 b.rowid:4!null + │ └── columns: y:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 └── filters - └── x:1 = y:3 + └── x:1 = y:4 build SELECT * FROM empty AS a LEFT OUTER JOIN onecolumn AS b USING(x) @@ -478,117 +478,117 @@ SELECT * FROM empty AS a LEFT OUTER JOIN onecolumn AS b USING(x) project ├── columns: x:1 └── left-join (hash) - ├── columns: a.x:1 a.rowid:2!null b.x:3 b.rowid:4 + ├── columns: a.x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 b.x:4 b.rowid:5 b.crdb_internal_mvcc_timestamp:6 ├── scan a - │ └── columns: a.x:1 a.rowid:2!null + │ └── columns: a.x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: b.x:3 b.rowid:4!null + │ └── columns: b.x:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 └── filters - └── a.x:1 = b.x:3 + └── a.x:1 = b.x:4 build SELECT * FROM onecolumn AS a(x) RIGHT OUTER JOIN empty AS b(y) ON a.x = b.y ---- project - ├── columns: x:1 y:3 + ├── columns: x:1 y:4 └── right-join (hash) - ├── columns: x:1 a.rowid:2 y:3 b.rowid:4!null + ├── columns: x:1 a.rowid:2 a.crdb_internal_mvcc_timestamp:3 y:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 ├── scan a - │ └── columns: x:1 a.rowid:2!null + │ └── columns: x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: y:3 b.rowid:4!null + │ └── columns: y:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 └── filters - └── x:1 = y:3 + └── x:1 = y:4 build SELECT * FROM onecolumn AS a RIGHT OUTER JOIN empty AS b USING(x) ---- project - ├── columns: x:3 + ├── columns: x:4 └── right-join (hash) - ├── columns: a.x:1 a.rowid:2 b.x:3 b.rowid:4!null + ├── columns: a.x:1 a.rowid:2 a.crdb_internal_mvcc_timestamp:3 b.x:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 ├── scan a - │ └── columns: a.x:1 a.rowid:2!null + │ └── columns: a.x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: b.x:3 b.rowid:4!null + │ └── columns: b.x:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 └── filters - └── a.x:1 = b.x:3 + └── a.x:1 = b.x:4 build SELECT * FROM empty AS a(x) FULL OUTER JOIN onecolumn AS b(y) ON a.x = b.y ORDER BY b.y ---- sort - ├── columns: x:1 y:3 - ├── ordering: +3 + ├── columns: x:1 y:4 + ├── ordering: +4 └── project - ├── columns: x:1 y:3 + ├── columns: x:1 y:4 └── full-join (hash) - ├── columns: x:1 a.rowid:2 y:3 b.rowid:4 + ├── columns: x:1 a.rowid:2 a.crdb_internal_mvcc_timestamp:3 y:4 b.rowid:5 b.crdb_internal_mvcc_timestamp:6 ├── scan a - │ └── columns: x:1 a.rowid:2!null + │ └── columns: x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: y:3 b.rowid:4!null + │ └── columns: y:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 └── filters - └── x:1 = y:3 + └── x:1 = y:4 build SELECT * FROM empty AS a FULL OUTER JOIN onecolumn AS b USING(x) ORDER BY x ---- sort - ├── columns: x:5 - ├── ordering: +5 + ├── columns: x:7 + ├── ordering: +7 └── project - ├── columns: x:5 + ├── columns: x:7 └── project - ├── columns: x:5 a.x:1 a.rowid:2 b.x:3 b.rowid:4 + ├── columns: x:7 a.x:1 a.rowid:2 a.crdb_internal_mvcc_timestamp:3 b.x:4 b.rowid:5 b.crdb_internal_mvcc_timestamp:6 ├── full-join (hash) - │ ├── columns: a.x:1 a.rowid:2 b.x:3 b.rowid:4 + │ ├── columns: a.x:1 a.rowid:2 a.crdb_internal_mvcc_timestamp:3 b.x:4 b.rowid:5 b.crdb_internal_mvcc_timestamp:6 │ ├── scan a - │ │ └── columns: a.x:1 a.rowid:2!null + │ │ └── columns: a.x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 │ ├── scan b - │ │ └── columns: b.x:3 b.rowid:4!null + │ │ └── columns: b.x:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 │ └── filters - │ └── a.x:1 = b.x:3 + │ └── a.x:1 = b.x:4 └── projections - └── COALESCE(a.x:1, b.x:3) [as=x:5] + └── COALESCE(a.x:1, b.x:4) [as=x:7] build SELECT * FROM onecolumn AS a(x) FULL OUTER JOIN empty AS b(y) ON a.x = b.y ORDER BY a.x ---- sort - ├── columns: x:1 y:3 + ├── columns: x:1 y:4 ├── ordering: +1 └── project - ├── columns: x:1 y:3 + ├── columns: x:1 y:4 └── full-join (hash) - ├── columns: x:1 a.rowid:2 y:3 b.rowid:4 + ├── columns: x:1 a.rowid:2 a.crdb_internal_mvcc_timestamp:3 y:4 b.rowid:5 b.crdb_internal_mvcc_timestamp:6 ├── scan a - │ └── columns: x:1 a.rowid:2!null + │ └── columns: x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: y:3 b.rowid:4!null + │ └── columns: y:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 └── filters - └── x:1 = y:3 + └── x:1 = y:4 build SELECT * FROM onecolumn AS a FULL OUTER JOIN empty AS b USING(x) ORDER BY x ---- sort - ├── columns: x:5 - ├── ordering: +5 + ├── columns: x:7 + ├── ordering: +7 └── project - ├── columns: x:5 + ├── columns: x:7 └── project - ├── columns: x:5 a.x:1 a.rowid:2 b.x:3 b.rowid:4 + ├── columns: x:7 a.x:1 a.rowid:2 a.crdb_internal_mvcc_timestamp:3 b.x:4 b.rowid:5 b.crdb_internal_mvcc_timestamp:6 ├── full-join (hash) - │ ├── columns: a.x:1 a.rowid:2 b.x:3 b.rowid:4 + │ ├── columns: a.x:1 a.rowid:2 a.crdb_internal_mvcc_timestamp:3 b.x:4 b.rowid:5 b.crdb_internal_mvcc_timestamp:6 │ ├── scan a - │ │ └── columns: a.x:1 a.rowid:2!null + │ │ └── columns: a.x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 │ ├── scan b - │ │ └── columns: b.x:3 b.rowid:4!null + │ │ └── columns: b.x:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 │ └── filters - │ └── a.x:1 = b.x:3 + │ └── a.x:1 = b.x:4 └── projections - └── COALESCE(a.x:1, b.x:3) [as=x:5] + └── COALESCE(a.x:1, b.x:4) [as=x:7] exec-ddl CREATE TABLE twocolumn (x INT, y INT) @@ -599,55 +599,55 @@ build SELECT * FROM onecolumn NATURAL JOIN twocolumn ---- project - ├── columns: x:1!null y:4 + ├── columns: x:1!null y:5 └── inner-join (hash) - ├── columns: onecolumn.x:1!null onecolumn.rowid:2!null twocolumn.x:3!null y:4 twocolumn.rowid:5!null + ├── columns: onecolumn.x:1!null onecolumn.rowid:2!null onecolumn.crdb_internal_mvcc_timestamp:3 twocolumn.x:4!null y:5 twocolumn.rowid:6!null twocolumn.crdb_internal_mvcc_timestamp:7 ├── scan onecolumn - │ └── columns: onecolumn.x:1 onecolumn.rowid:2!null + │ └── columns: onecolumn.x:1 onecolumn.rowid:2!null onecolumn.crdb_internal_mvcc_timestamp:3 ├── scan twocolumn - │ └── columns: twocolumn.x:3 y:4 twocolumn.rowid:5!null + │ └── columns: twocolumn.x:4 y:5 twocolumn.rowid:6!null twocolumn.crdb_internal_mvcc_timestamp:7 └── filters - └── onecolumn.x:1 = twocolumn.x:3 + └── onecolumn.x:1 = twocolumn.x:4 build SELECT * FROM onecolumn JOIN twocolumn USING(x) ---- project - ├── columns: x:1!null y:4 + ├── columns: x:1!null y:5 └── inner-join (hash) - ├── columns: onecolumn.x:1!null onecolumn.rowid:2!null twocolumn.x:3!null y:4 twocolumn.rowid:5!null + ├── columns: onecolumn.x:1!null onecolumn.rowid:2!null onecolumn.crdb_internal_mvcc_timestamp:3 twocolumn.x:4!null y:5 twocolumn.rowid:6!null twocolumn.crdb_internal_mvcc_timestamp:7 ├── scan onecolumn - │ └── columns: onecolumn.x:1 onecolumn.rowid:2!null + │ └── columns: onecolumn.x:1 onecolumn.rowid:2!null onecolumn.crdb_internal_mvcc_timestamp:3 ├── scan twocolumn - │ └── columns: twocolumn.x:3 y:4 twocolumn.rowid:5!null + │ └── columns: twocolumn.x:4 y:5 twocolumn.rowid:6!null twocolumn.crdb_internal_mvcc_timestamp:7 └── filters - └── onecolumn.x:1 = twocolumn.x:3 + └── onecolumn.x:1 = twocolumn.x:4 build SELECT * FROM twocolumn AS a JOIN twocolumn AS b ON a.x = b.y ---- project - ├── columns: x:1!null y:2 x:4 y:5!null + ├── columns: x:1!null y:2 x:5 y:6!null └── inner-join (hash) - ├── columns: a.x:1!null a.y:2 a.rowid:3!null b.x:4 b.y:5!null b.rowid:6!null + ├── columns: a.x:1!null a.y:2 a.rowid:3!null a.crdb_internal_mvcc_timestamp:4 b.x:5 b.y:6!null b.rowid:7!null b.crdb_internal_mvcc_timestamp:8 ├── scan a - │ └── columns: a.x:1 a.y:2 a.rowid:3!null + │ └── columns: a.x:1 a.y:2 a.rowid:3!null a.crdb_internal_mvcc_timestamp:4 ├── scan b - │ └── columns: b.x:4 b.y:5 b.rowid:6!null + │ └── columns: b.x:5 b.y:6 b.rowid:7!null b.crdb_internal_mvcc_timestamp:8 └── filters - └── a.x:1 = b.y:5 + └── a.x:1 = b.y:6 build SELECT * FROM twocolumn AS a JOIN twocolumn AS b ON a.x = a.y ---- project - ├── columns: x:1!null y:2!null x:4 y:5 + ├── columns: x:1!null y:2!null x:5 y:6 └── inner-join (cross) - ├── columns: a.x:1!null a.y:2!null a.rowid:3!null b.x:4 b.y:5 b.rowid:6!null + ├── columns: a.x:1!null a.y:2!null a.rowid:3!null a.crdb_internal_mvcc_timestamp:4 b.x:5 b.y:6 b.rowid:7!null b.crdb_internal_mvcc_timestamp:8 ├── scan a - │ └── columns: a.x:1 a.y:2 a.rowid:3!null + │ └── columns: a.x:1 a.y:2 a.rowid:3!null a.crdb_internal_mvcc_timestamp:4 ├── scan b - │ └── columns: b.x:4 b.y:5 b.rowid:6!null + │ └── columns: b.x:5 b.y:6 b.rowid:7!null b.crdb_internal_mvcc_timestamp:8 └── filters └── a.x:1 = a.y:2 @@ -655,42 +655,42 @@ build SELECT * FROM onecolumn AS a JOIN twocolumn AS b ON ((a.x)) = ((b.y)) ---- project - ├── columns: x:1!null x:3 y:4!null + ├── columns: x:1!null x:4 y:5!null └── inner-join (hash) - ├── columns: a.x:1!null a.rowid:2!null b.x:3 y:4!null b.rowid:5!null + ├── columns: a.x:1!null a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 b.x:4 y:5!null b.rowid:6!null b.crdb_internal_mvcc_timestamp:7 ├── scan a - │ └── columns: a.x:1 a.rowid:2!null + │ └── columns: a.x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: b.x:3 y:4 b.rowid:5!null + │ └── columns: b.x:4 y:5 b.rowid:6!null b.crdb_internal_mvcc_timestamp:7 └── filters - └── a.x:1 = y:4 + └── a.x:1 = y:5 build SELECT * FROM onecolumn JOIN twocolumn ON onecolumn.x = twocolumn.y ---- project - ├── columns: x:1!null x:3 y:4!null + ├── columns: x:1!null x:4 y:5!null └── inner-join (hash) - ├── columns: onecolumn.x:1!null onecolumn.rowid:2!null twocolumn.x:3 y:4!null twocolumn.rowid:5!null + ├── columns: onecolumn.x:1!null onecolumn.rowid:2!null onecolumn.crdb_internal_mvcc_timestamp:3 twocolumn.x:4 y:5!null twocolumn.rowid:6!null twocolumn.crdb_internal_mvcc_timestamp:7 ├── scan onecolumn - │ └── columns: onecolumn.x:1 onecolumn.rowid:2!null + │ └── columns: onecolumn.x:1 onecolumn.rowid:2!null onecolumn.crdb_internal_mvcc_timestamp:3 ├── scan twocolumn - │ └── columns: twocolumn.x:3 y:4 twocolumn.rowid:5!null + │ └── columns: twocolumn.x:4 y:5 twocolumn.rowid:6!null twocolumn.crdb_internal_mvcc_timestamp:7 └── filters - └── onecolumn.x:1 = y:4 + └── onecolumn.x:1 = y:5 # Inner join with filter predicate build SELECT * FROM twocolumn AS a JOIN twocolumn AS b ON a.x = 44 ---- project - ├── columns: x:1!null y:2 x:4 y:5 + ├── columns: x:1!null y:2 x:5 y:6 └── inner-join (cross) - ├── columns: a.x:1!null a.y:2 a.rowid:3!null b.x:4 b.y:5 b.rowid:6!null + ├── columns: a.x:1!null a.y:2 a.rowid:3!null a.crdb_internal_mvcc_timestamp:4 b.x:5 b.y:6 b.rowid:7!null b.crdb_internal_mvcc_timestamp:8 ├── scan a - │ └── columns: a.x:1 a.y:2 a.rowid:3!null + │ └── columns: a.x:1 a.y:2 a.rowid:3!null a.crdb_internal_mvcc_timestamp:4 ├── scan b - │ └── columns: b.x:4 b.y:5 b.rowid:6!null + │ └── columns: b.x:5 b.y:6 b.rowid:7!null b.crdb_internal_mvcc_timestamp:8 └── filters └── a.x:1 = 44 @@ -698,58 +698,58 @@ build SELECT o.x, t.y FROM onecolumn o INNER JOIN twocolumn t ON (o.x=t.x AND t.y=53) ---- project - ├── columns: x:1!null y:4!null + ├── columns: x:1!null y:5!null └── inner-join (cross) - ├── columns: o.x:1!null o.rowid:2!null t.x:3!null y:4!null t.rowid:5!null + ├── columns: o.x:1!null o.rowid:2!null o.crdb_internal_mvcc_timestamp:3 t.x:4!null y:5!null t.rowid:6!null t.crdb_internal_mvcc_timestamp:7 ├── scan o - │ └── columns: o.x:1 o.rowid:2!null + │ └── columns: o.x:1 o.rowid:2!null o.crdb_internal_mvcc_timestamp:3 ├── scan t - │ └── columns: t.x:3 y:4 t.rowid:5!null + │ └── columns: t.x:4 y:5 t.rowid:6!null t.crdb_internal_mvcc_timestamp:7 └── filters - └── (o.x:1 = t.x:3) AND (y:4 = 53) + └── (o.x:1 = t.x:4) AND (y:5 = 53) # Outer joins with filter predicate build SELECT o.x, t.y FROM onecolumn o LEFT OUTER JOIN twocolumn t ON (o.x=t.x AND t.y=53) ---- project - ├── columns: x:1 y:4 + ├── columns: x:1 y:5 └── left-join (cross) - ├── columns: o.x:1 o.rowid:2!null t.x:3 y:4 t.rowid:5 + ├── columns: o.x:1 o.rowid:2!null o.crdb_internal_mvcc_timestamp:3 t.x:4 y:5 t.rowid:6 t.crdb_internal_mvcc_timestamp:7 ├── scan o - │ └── columns: o.x:1 o.rowid:2!null + │ └── columns: o.x:1 o.rowid:2!null o.crdb_internal_mvcc_timestamp:3 ├── scan t - │ └── columns: t.x:3 y:4 t.rowid:5!null + │ └── columns: t.x:4 y:5 t.rowid:6!null t.crdb_internal_mvcc_timestamp:7 └── filters - └── (o.x:1 = t.x:3) AND (y:4 = 53) + └── (o.x:1 = t.x:4) AND (y:5 = 53) build SELECT o.x, t.y FROM onecolumn o LEFT OUTER JOIN twocolumn t ON (o.x=t.x AND o.x=44) ---- project - ├── columns: x:1 y:4 + ├── columns: x:1 y:5 └── left-join (cross) - ├── columns: o.x:1 o.rowid:2!null t.x:3 y:4 t.rowid:5 + ├── columns: o.x:1 o.rowid:2!null o.crdb_internal_mvcc_timestamp:3 t.x:4 y:5 t.rowid:6 t.crdb_internal_mvcc_timestamp:7 ├── scan o - │ └── columns: o.x:1 o.rowid:2!null + │ └── columns: o.x:1 o.rowid:2!null o.crdb_internal_mvcc_timestamp:3 ├── scan t - │ └── columns: t.x:3 y:4 t.rowid:5!null + │ └── columns: t.x:4 y:5 t.rowid:6!null t.crdb_internal_mvcc_timestamp:7 └── filters - └── (o.x:1 = t.x:3) AND (o.x:1 = 44) + └── (o.x:1 = t.x:4) AND (o.x:1 = 44) build SELECT o.x, t.y FROM onecolumn o LEFT OUTER JOIN twocolumn t ON (o.x=t.x AND t.x=44) ---- project - ├── columns: x:1 y:4 + ├── columns: x:1 y:5 └── left-join (cross) - ├── columns: o.x:1 o.rowid:2!null t.x:3 y:4 t.rowid:5 + ├── columns: o.x:1 o.rowid:2!null o.crdb_internal_mvcc_timestamp:3 t.x:4 y:5 t.rowid:6 t.crdb_internal_mvcc_timestamp:7 ├── scan o - │ └── columns: o.x:1 o.rowid:2!null + │ └── columns: o.x:1 o.rowid:2!null o.crdb_internal_mvcc_timestamp:3 ├── scan t - │ └── columns: t.x:3 y:4 t.rowid:5!null + │ └── columns: t.x:4 y:5 t.rowid:6!null t.crdb_internal_mvcc_timestamp:7 └── filters - └── (o.x:1 = t.x:3) AND (t.x:3 = 44) + └── (o.x:1 = t.x:4) AND (t.x:4 = 44) build SELECT x, a.x, b.y FROM (SELECT * FROM onecolumn AS a NATURAL JOIN twocolumn AS b) AS q @@ -776,115 +776,115 @@ build SELECT * FROM a INNER JOIN b ON a.i = b.i ---- project - ├── columns: i:1!null i:3!null b:4 + ├── columns: i:1!null i:4!null b:5 └── inner-join (hash) - ├── columns: a.i:1!null a.rowid:2!null b.i:3!null b:4 b.rowid:5!null + ├── columns: a.i:1!null a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 b.i:4!null b:5 b.rowid:6!null b.crdb_internal_mvcc_timestamp:7 ├── scan a - │ └── columns: a.i:1 a.rowid:2!null + │ └── columns: a.i:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: b.i:3 b:4 b.rowid:5!null + │ └── columns: b.i:4 b:5 b.rowid:6!null b.crdb_internal_mvcc_timestamp:7 └── filters - └── a.i:1 = b.i:3 + └── a.i:1 = b.i:4 build SELECT * FROM a LEFT OUTER JOIN b ON a.i = b.i ---- project - ├── columns: i:1 i:3 b:4 + ├── columns: i:1 i:4 b:5 └── left-join (hash) - ├── columns: a.i:1 a.rowid:2!null b.i:3 b:4 b.rowid:5 + ├── columns: a.i:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 b.i:4 b:5 b.rowid:6 b.crdb_internal_mvcc_timestamp:7 ├── scan a - │ └── columns: a.i:1 a.rowid:2!null + │ └── columns: a.i:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: b.i:3 b:4 b.rowid:5!null + │ └── columns: b.i:4 b:5 b.rowid:6!null b.crdb_internal_mvcc_timestamp:7 └── filters - └── a.i:1 = b.i:3 + └── a.i:1 = b.i:4 build SELECT * FROM a RIGHT OUTER JOIN b ON a.i = b.i ORDER BY b.i, b.b ---- sort - ├── columns: i:1 i:3 b:4 - ├── ordering: +3,+4 + ├── columns: i:1 i:4 b:5 + ├── ordering: +4,+5 └── project - ├── columns: a.i:1 b.i:3 b:4 + ├── columns: a.i:1 b.i:4 b:5 └── right-join (hash) - ├── columns: a.i:1 a.rowid:2 b.i:3 b:4 b.rowid:5!null + ├── columns: a.i:1 a.rowid:2 a.crdb_internal_mvcc_timestamp:3 b.i:4 b:5 b.rowid:6!null b.crdb_internal_mvcc_timestamp:7 ├── scan a - │ └── columns: a.i:1 a.rowid:2!null + │ └── columns: a.i:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: b.i:3 b:4 b.rowid:5!null + │ └── columns: b.i:4 b:5 b.rowid:6!null b.crdb_internal_mvcc_timestamp:7 └── filters - └── a.i:1 = b.i:3 + └── a.i:1 = b.i:4 build SELECT * FROM a FULL OUTER JOIN b ON a.i = b.i ORDER BY b.i, b.b ---- sort - ├── columns: i:1 i:3 b:4 - ├── ordering: +3,+4 + ├── columns: i:1 i:4 b:5 + ├── ordering: +4,+5 └── project - ├── columns: a.i:1 b.i:3 b:4 + ├── columns: a.i:1 b.i:4 b:5 └── full-join (hash) - ├── columns: a.i:1 a.rowid:2 b.i:3 b:4 b.rowid:5 + ├── columns: a.i:1 a.rowid:2 a.crdb_internal_mvcc_timestamp:3 b.i:4 b:5 b.rowid:6 b.crdb_internal_mvcc_timestamp:7 ├── scan a - │ └── columns: a.i:1 a.rowid:2!null + │ └── columns: a.i:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: b.i:3 b:4 b.rowid:5!null + │ └── columns: b.i:4 b:5 b.rowid:6!null b.crdb_internal_mvcc_timestamp:7 └── filters - └── a.i:1 = b.i:3 + └── a.i:1 = b.i:4 # Full outer join with filter predicate build SELECT * FROM a FULL OUTER JOIN b ON (a.i = b.i and a.i>2) ORDER BY a.i, b.i ---- sort - ├── columns: i:1 i:3 b:4 - ├── ordering: +1,+3 + ├── columns: i:1 i:4 b:5 + ├── ordering: +1,+4 └── project - ├── columns: a.i:1 b.i:3 b:4 + ├── columns: a.i:1 b.i:4 b:5 └── full-join (cross) - ├── columns: a.i:1 a.rowid:2 b.i:3 b:4 b.rowid:5 + ├── columns: a.i:1 a.rowid:2 a.crdb_internal_mvcc_timestamp:3 b.i:4 b:5 b.rowid:6 b.crdb_internal_mvcc_timestamp:7 ├── scan a - │ └── columns: a.i:1 a.rowid:2!null + │ └── columns: a.i:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: b.i:3 b:4 b.rowid:5!null + │ └── columns: b.i:4 b:5 b.rowid:6!null b.crdb_internal_mvcc_timestamp:7 └── filters - └── (a.i:1 = b.i:3) AND (a.i:1 > 2) + └── (a.i:1 = b.i:4) AND (a.i:1 > 2) # Check column orders and names. build SELECT * FROM (onecolumn CROSS JOIN twocolumn JOIN onecolumn AS a(b) ON a.b=twocolumn.x JOIN twocolumn AS c(d,e) ON a.b=c.d AND c.d=onecolumn.x) ORDER BY 1 LIMIT 1 ---- limit - ├── columns: x:1!null x:3!null y:4 b:6!null d:8!null e:9 + ├── columns: x:1!null x:4!null y:5 b:8!null d:11!null e:12 ├── internal-ordering: +1 ├── ordering: +1 ├── sort - │ ├── columns: onecolumn.x:1!null twocolumn.x:3!null y:4 b:6!null d:8!null e:9 + │ ├── columns: onecolumn.x:1!null twocolumn.x:4!null y:5 b:8!null d:11!null e:12 │ ├── ordering: +1 │ ├── limit hint: 1.00 │ └── project - │ ├── columns: onecolumn.x:1!null twocolumn.x:3!null y:4 b:6!null d:8!null e:9 + │ ├── columns: onecolumn.x:1!null twocolumn.x:4!null y:5 b:8!null d:11!null e:12 │ └── inner-join (cross) - │ ├── columns: onecolumn.x:1!null onecolumn.rowid:2!null twocolumn.x:3!null y:4 twocolumn.rowid:5!null b:6!null a.rowid:7!null d:8!null e:9 c.rowid:10!null + │ ├── columns: onecolumn.x:1!null onecolumn.rowid:2!null onecolumn.crdb_internal_mvcc_timestamp:3 twocolumn.x:4!null y:5 twocolumn.rowid:6!null twocolumn.crdb_internal_mvcc_timestamp:7 b:8!null a.rowid:9!null a.crdb_internal_mvcc_timestamp:10 d:11!null e:12 c.rowid:13!null c.crdb_internal_mvcc_timestamp:14 │ ├── inner-join (hash) - │ │ ├── columns: onecolumn.x:1 onecolumn.rowid:2!null twocolumn.x:3!null y:4 twocolumn.rowid:5!null b:6!null a.rowid:7!null + │ │ ├── columns: onecolumn.x:1 onecolumn.rowid:2!null onecolumn.crdb_internal_mvcc_timestamp:3 twocolumn.x:4!null y:5 twocolumn.rowid:6!null twocolumn.crdb_internal_mvcc_timestamp:7 b:8!null a.rowid:9!null a.crdb_internal_mvcc_timestamp:10 │ │ ├── inner-join (cross) - │ │ │ ├── columns: onecolumn.x:1 onecolumn.rowid:2!null twocolumn.x:3 y:4 twocolumn.rowid:5!null + │ │ │ ├── columns: onecolumn.x:1 onecolumn.rowid:2!null onecolumn.crdb_internal_mvcc_timestamp:3 twocolumn.x:4 y:5 twocolumn.rowid:6!null twocolumn.crdb_internal_mvcc_timestamp:7 │ │ │ ├── scan onecolumn - │ │ │ │ └── columns: onecolumn.x:1 onecolumn.rowid:2!null + │ │ │ │ └── columns: onecolumn.x:1 onecolumn.rowid:2!null onecolumn.crdb_internal_mvcc_timestamp:3 │ │ │ ├── scan twocolumn - │ │ │ │ └── columns: twocolumn.x:3 y:4 twocolumn.rowid:5!null + │ │ │ │ └── columns: twocolumn.x:4 y:5 twocolumn.rowid:6!null twocolumn.crdb_internal_mvcc_timestamp:7 │ │ │ └── filters (true) │ │ ├── scan a - │ │ │ └── columns: b:6 a.rowid:7!null + │ │ │ └── columns: b:8 a.rowid:9!null a.crdb_internal_mvcc_timestamp:10 │ │ └── filters - │ │ └── b:6 = twocolumn.x:3 + │ │ └── b:8 = twocolumn.x:4 │ ├── scan c - │ │ └── columns: d:8 e:9 c.rowid:10!null + │ │ └── columns: d:11 e:12 c.rowid:13!null c.crdb_internal_mvcc_timestamp:14 │ └── filters - │ └── (b:6 = d:8) AND (d:8 = onecolumn.x:1) + │ └── (b:8 = d:11) AND (d:11 = onecolumn.x:1) └── 1 # Check sub-queries in ON conditions. @@ -892,25 +892,25 @@ build SELECT * FROM onecolumn JOIN twocolumn ON twocolumn.x = onecolumn.x AND onecolumn.x IN (SELECT x FROM twocolumn WHERE y >= 52) ---- project - ├── columns: x:1!null x:3!null y:4 + ├── columns: x:1!null x:4!null y:5 └── inner-join (cross) - ├── columns: onecolumn.x:1!null onecolumn.rowid:2!null twocolumn.x:3!null y:4 twocolumn.rowid:5!null + ├── columns: onecolumn.x:1!null onecolumn.rowid:2!null onecolumn.crdb_internal_mvcc_timestamp:3 twocolumn.x:4!null y:5 twocolumn.rowid:6!null twocolumn.crdb_internal_mvcc_timestamp:7 ├── scan onecolumn - │ └── columns: onecolumn.x:1 onecolumn.rowid:2!null + │ └── columns: onecolumn.x:1 onecolumn.rowid:2!null onecolumn.crdb_internal_mvcc_timestamp:3 ├── scan twocolumn - │ └── columns: twocolumn.x:3 y:4 twocolumn.rowid:5!null + │ └── columns: twocolumn.x:4 y:5 twocolumn.rowid:6!null twocolumn.crdb_internal_mvcc_timestamp:7 └── filters └── and - ├── twocolumn.x:3 = onecolumn.x:1 + ├── twocolumn.x:4 = onecolumn.x:1 └── any: eq ├── project - │ ├── columns: twocolumn.x:6 + │ ├── columns: twocolumn.x:8 │ └── select - │ ├── columns: twocolumn.x:6 y:7!null twocolumn.rowid:8!null + │ ├── columns: twocolumn.x:8 y:9!null twocolumn.rowid:10!null twocolumn.crdb_internal_mvcc_timestamp:11 │ ├── scan twocolumn - │ │ └── columns: twocolumn.x:6 y:7 twocolumn.rowid:8!null + │ │ └── columns: twocolumn.x:8 y:9 twocolumn.rowid:10!null twocolumn.crdb_internal_mvcc_timestamp:11 │ └── filters - │ └── y:7 >= 52 + │ └── y:9 >= 52 └── onecolumn.x:1 # Check sub-queries as data sources. @@ -920,16 +920,16 @@ SELECT * FROM onecolumn JOIN (VALUES (41),(42),(43)) AS a(x) USING(x) project ├── columns: x:1!null └── inner-join (hash) - ├── columns: x:1!null rowid:2!null column1:3!null + ├── columns: x:1!null rowid:2!null crdb_internal_mvcc_timestamp:3 column1:4!null ├── scan onecolumn - │ └── columns: x:1 rowid:2!null + │ └── columns: x:1 rowid:2!null crdb_internal_mvcc_timestamp:3 ├── values - │ ├── columns: column1:3!null + │ ├── columns: column1:4!null │ ├── (41,) │ ├── (42,) │ └── (43,) └── filters - └── x:1 = column1:3 + └── x:1 = column1:4 build SELECT * FROM onecolumn JOIN (SELECT x + 2 AS x FROM onecolumn) USING(x) @@ -937,70 +937,70 @@ SELECT * FROM onecolumn JOIN (SELECT x + 2 AS x FROM onecolumn) USING(x) project ├── columns: x:1!null └── inner-join (hash) - ├── columns: onecolumn.x:1!null rowid:2!null x:5!null + ├── columns: onecolumn.x:1!null rowid:2!null crdb_internal_mvcc_timestamp:3 x:7!null ├── scan onecolumn - │ └── columns: onecolumn.x:1 rowid:2!null + │ └── columns: onecolumn.x:1 rowid:2!null crdb_internal_mvcc_timestamp:3 ├── project - │ ├── columns: x:5 + │ ├── columns: x:7 │ ├── scan onecolumn - │ │ └── columns: onecolumn.x:3 rowid:4!null + │ │ └── columns: onecolumn.x:4 rowid:5!null crdb_internal_mvcc_timestamp:6 │ └── projections - │ └── onecolumn.x:3 + 2 [as=x:5] + │ └── onecolumn.x:4 + 2 [as=x:7] └── filters - └── onecolumn.x:1 = x:5 + └── onecolumn.x:1 = x:7 # Check that a single column can have multiple table aliases. build SELECT * FROM (twocolumn AS a JOIN twocolumn AS b USING(x) JOIN twocolumn AS c USING(x)) ORDER BY x LIMIT 1 ---- limit - ├── columns: x:1!null y:2 y:5 y:8 + ├── columns: x:1!null y:2 y:6 y:10 ├── internal-ordering: +1 ├── ordering: +1 ├── sort - │ ├── columns: a.x:1!null a.y:2 b.y:5 c.y:8 + │ ├── columns: a.x:1!null a.y:2 b.y:6 c.y:10 │ ├── ordering: +1 │ ├── limit hint: 1.00 │ └── project - │ ├── columns: a.x:1!null a.y:2 b.y:5 c.y:8 + │ ├── columns: a.x:1!null a.y:2 b.y:6 c.y:10 │ └── inner-join (hash) - │ ├── columns: a.x:1!null a.y:2 a.rowid:3!null b.x:4!null b.y:5 b.rowid:6!null c.x:7!null c.y:8 c.rowid:9!null + │ ├── columns: a.x:1!null a.y:2 a.rowid:3!null a.crdb_internal_mvcc_timestamp:4 b.x:5!null b.y:6 b.rowid:7!null b.crdb_internal_mvcc_timestamp:8 c.x:9!null c.y:10 c.rowid:11!null c.crdb_internal_mvcc_timestamp:12 │ ├── inner-join (hash) - │ │ ├── columns: a.x:1!null a.y:2 a.rowid:3!null b.x:4!null b.y:5 b.rowid:6!null + │ │ ├── columns: a.x:1!null a.y:2 a.rowid:3!null a.crdb_internal_mvcc_timestamp:4 b.x:5!null b.y:6 b.rowid:7!null b.crdb_internal_mvcc_timestamp:8 │ │ ├── scan a - │ │ │ └── columns: a.x:1 a.y:2 a.rowid:3!null + │ │ │ └── columns: a.x:1 a.y:2 a.rowid:3!null a.crdb_internal_mvcc_timestamp:4 │ │ ├── scan b - │ │ │ └── columns: b.x:4 b.y:5 b.rowid:6!null + │ │ │ └── columns: b.x:5 b.y:6 b.rowid:7!null b.crdb_internal_mvcc_timestamp:8 │ │ └── filters - │ │ └── a.x:1 = b.x:4 + │ │ └── a.x:1 = b.x:5 │ ├── scan c - │ │ └── columns: c.x:7 c.y:8 c.rowid:9!null + │ │ └── columns: c.x:9 c.y:10 c.rowid:11!null c.crdb_internal_mvcc_timestamp:12 │ └── filters - │ └── a.x:1 = c.x:7 + │ └── a.x:1 = c.x:9 └── 1 build SELECT a.x AS s, b.x, c.x, a.y, b.y, c.y FROM (twocolumn AS a JOIN twocolumn AS b USING(x) JOIN twocolumn AS c USING(x)) ORDER BY s ---- sort - ├── columns: s:1!null x:4!null x:7!null y:2 y:5 y:8 + ├── columns: s:1!null x:5!null x:9!null y:2 y:6 y:10 ├── ordering: +1 └── project - ├── columns: a.x:1!null a.y:2 b.x:4!null b.y:5 c.x:7!null c.y:8 + ├── columns: a.x:1!null a.y:2 b.x:5!null b.y:6 c.x:9!null c.y:10 └── inner-join (hash) - ├── columns: a.x:1!null a.y:2 a.rowid:3!null b.x:4!null b.y:5 b.rowid:6!null c.x:7!null c.y:8 c.rowid:9!null + ├── columns: a.x:1!null a.y:2 a.rowid:3!null a.crdb_internal_mvcc_timestamp:4 b.x:5!null b.y:6 b.rowid:7!null b.crdb_internal_mvcc_timestamp:8 c.x:9!null c.y:10 c.rowid:11!null c.crdb_internal_mvcc_timestamp:12 ├── inner-join (hash) - │ ├── columns: a.x:1!null a.y:2 a.rowid:3!null b.x:4!null b.y:5 b.rowid:6!null + │ ├── columns: a.x:1!null a.y:2 a.rowid:3!null a.crdb_internal_mvcc_timestamp:4 b.x:5!null b.y:6 b.rowid:7!null b.crdb_internal_mvcc_timestamp:8 │ ├── scan a - │ │ └── columns: a.x:1 a.y:2 a.rowid:3!null + │ │ └── columns: a.x:1 a.y:2 a.rowid:3!null a.crdb_internal_mvcc_timestamp:4 │ ├── scan b - │ │ └── columns: b.x:4 b.y:5 b.rowid:6!null + │ │ └── columns: b.x:5 b.y:6 b.rowid:7!null b.crdb_internal_mvcc_timestamp:8 │ └── filters - │ └── a.x:1 = b.x:4 + │ └── a.x:1 = b.x:5 ├── scan c - │ └── columns: c.x:7 c.y:8 c.rowid:9!null + │ └── columns: c.x:9 c.y:10 c.rowid:11!null c.crdb_internal_mvcc_timestamp:12 └── filters - └── a.x:1 = c.x:7 + └── a.x:1 = c.x:9 build SELECT * FROM (onecolumn AS a JOIN onecolumn AS b USING(y)) @@ -1036,15 +1036,15 @@ build SELECT * FROM (SELECT * FROM onecolumn), (SELECT * FROM onecolumn) ---- inner-join (cross) - ├── columns: x:1 x:3 + ├── columns: x:1 x:4 ├── project │ ├── columns: x:1 │ └── scan onecolumn - │ └── columns: x:1 rowid:2!null + │ └── columns: x:1 rowid:2!null crdb_internal_mvcc_timestamp:3 ├── project - │ ├── columns: x:3 + │ ├── columns: x:4 │ └── scan onecolumn - │ └── columns: x:3 rowid:4!null + │ └── columns: x:4 rowid:5!null crdb_internal_mvcc_timestamp:6 └── filters (true) # Check that anonymous sources are properly looked up without ambiguity. @@ -1054,25 +1054,25 @@ SELECT x FROM (onecolumn JOIN othercolumn USING (x)) JOIN (onecolumn AS a JOIN o project ├── columns: x:1!null └── inner-join (hash) - ├── columns: onecolumn.x:1!null onecolumn.rowid:2!null othercolumn.x:3!null othercolumn.rowid:4!null a.x:5!null a.rowid:6!null b.x:7!null b.rowid:8!null + ├── columns: onecolumn.x:1!null onecolumn.rowid:2!null onecolumn.crdb_internal_mvcc_timestamp:3 othercolumn.x:4!null othercolumn.rowid:5!null othercolumn.crdb_internal_mvcc_timestamp:6 a.x:7!null a.rowid:8!null a.crdb_internal_mvcc_timestamp:9 b.x:10!null b.rowid:11!null b.crdb_internal_mvcc_timestamp:12 ├── inner-join (hash) - │ ├── columns: onecolumn.x:1!null onecolumn.rowid:2!null othercolumn.x:3!null othercolumn.rowid:4!null + │ ├── columns: onecolumn.x:1!null onecolumn.rowid:2!null onecolumn.crdb_internal_mvcc_timestamp:3 othercolumn.x:4!null othercolumn.rowid:5!null othercolumn.crdb_internal_mvcc_timestamp:6 │ ├── scan onecolumn - │ │ └── columns: onecolumn.x:1 onecolumn.rowid:2!null + │ │ └── columns: onecolumn.x:1 onecolumn.rowid:2!null onecolumn.crdb_internal_mvcc_timestamp:3 │ ├── scan othercolumn - │ │ └── columns: othercolumn.x:3 othercolumn.rowid:4!null + │ │ └── columns: othercolumn.x:4 othercolumn.rowid:5!null othercolumn.crdb_internal_mvcc_timestamp:6 │ └── filters - │ └── onecolumn.x:1 = othercolumn.x:3 + │ └── onecolumn.x:1 = othercolumn.x:4 ├── inner-join (hash) - │ ├── columns: a.x:5!null a.rowid:6!null b.x:7!null b.rowid:8!null + │ ├── columns: a.x:7!null a.rowid:8!null a.crdb_internal_mvcc_timestamp:9 b.x:10!null b.rowid:11!null b.crdb_internal_mvcc_timestamp:12 │ ├── scan a - │ │ └── columns: a.x:5 a.rowid:6!null + │ │ └── columns: a.x:7 a.rowid:8!null a.crdb_internal_mvcc_timestamp:9 │ ├── scan b - │ │ └── columns: b.x:7 b.rowid:8!null + │ │ └── columns: b.x:10 b.rowid:11!null b.crdb_internal_mvcc_timestamp:12 │ └── filters - │ └── a.x:5 = b.x:7 + │ └── a.x:7 = b.x:10 └── filters - └── onecolumn.x:1 = a.x:5 + └── onecolumn.x:1 = a.x:7 # Check that multiple anonymous sources cause proper ambiguity errors. build @@ -1095,42 +1095,42 @@ build SELECT a.x, b.y FROM twocolumn AS a, twocolumn AS b ---- project - ├── columns: x:1 y:5 + ├── columns: x:1 y:6 └── inner-join (cross) - ├── columns: a.x:1 a.y:2 a.rowid:3!null b.x:4 b.y:5 b.rowid:6!null + ├── columns: a.x:1 a.y:2 a.rowid:3!null a.crdb_internal_mvcc_timestamp:4 b.x:5 b.y:6 b.rowid:7!null b.crdb_internal_mvcc_timestamp:8 ├── scan a - │ └── columns: a.x:1 a.y:2 a.rowid:3!null + │ └── columns: a.x:1 a.y:2 a.rowid:3!null a.crdb_internal_mvcc_timestamp:4 ├── scan b - │ └── columns: b.x:4 b.y:5 b.rowid:6!null + │ └── columns: b.x:5 b.y:6 b.rowid:7!null b.crdb_internal_mvcc_timestamp:8 └── filters (true) build SELECT b.y FROM (twocolumn AS a JOIN twocolumn AS b USING(x)) ---- project - ├── columns: y:5 + ├── columns: y:6 └── inner-join (hash) - ├── columns: a.x:1!null a.y:2 a.rowid:3!null b.x:4!null b.y:5 b.rowid:6!null + ├── columns: a.x:1!null a.y:2 a.rowid:3!null a.crdb_internal_mvcc_timestamp:4 b.x:5!null b.y:6 b.rowid:7!null b.crdb_internal_mvcc_timestamp:8 ├── scan a - │ └── columns: a.x:1 a.y:2 a.rowid:3!null + │ └── columns: a.x:1 a.y:2 a.rowid:3!null a.crdb_internal_mvcc_timestamp:4 ├── scan b - │ └── columns: b.x:4 b.y:5 b.rowid:6!null + │ └── columns: b.x:5 b.y:6 b.rowid:7!null b.crdb_internal_mvcc_timestamp:8 └── filters - └── a.x:1 = b.x:4 + └── a.x:1 = b.x:5 build SELECT b.y FROM (twocolumn AS a JOIN twocolumn AS b ON a.x = b.x) ---- project - ├── columns: y:5 + ├── columns: y:6 └── inner-join (hash) - ├── columns: a.x:1!null a.y:2 a.rowid:3!null b.x:4!null b.y:5 b.rowid:6!null + ├── columns: a.x:1!null a.y:2 a.rowid:3!null a.crdb_internal_mvcc_timestamp:4 b.x:5!null b.y:6 b.rowid:7!null b.crdb_internal_mvcc_timestamp:8 ├── scan a - │ └── columns: a.x:1 a.y:2 a.rowid:3!null + │ └── columns: a.x:1 a.y:2 a.rowid:3!null a.crdb_internal_mvcc_timestamp:4 ├── scan b - │ └── columns: b.x:4 b.y:5 b.rowid:6!null + │ └── columns: b.x:5 b.y:6 b.rowid:7!null b.crdb_internal_mvcc_timestamp:8 └── filters - └── a.x:1 = b.x:4 + └── a.x:1 = b.x:5 build SELECT a.x FROM (twocolumn AS a JOIN twocolumn AS b ON a.x < b.y) @@ -1138,13 +1138,13 @@ SELECT a.x FROM (twocolumn AS a JOIN twocolumn AS b ON a.x < b.y) project ├── columns: x:1!null └── inner-join (cross) - ├── columns: a.x:1!null a.y:2 a.rowid:3!null b.x:4 b.y:5!null b.rowid:6!null + ├── columns: a.x:1!null a.y:2 a.rowid:3!null a.crdb_internal_mvcc_timestamp:4 b.x:5 b.y:6!null b.rowid:7!null b.crdb_internal_mvcc_timestamp:8 ├── scan a - │ └── columns: a.x:1 a.y:2 a.rowid:3!null + │ └── columns: a.x:1 a.y:2 a.rowid:3!null a.crdb_internal_mvcc_timestamp:4 ├── scan b - │ └── columns: b.x:4 b.y:5 b.rowid:6!null + │ └── columns: b.x:5 b.y:6 b.rowid:7!null b.crdb_internal_mvcc_timestamp:8 └── filters - └── a.x:1 < b.y:5 + └── a.x:1 < b.y:6 build SELECT * FROM (SELECT * FROM (VALUES (9, 1), (8, 2)) AS a (u, k) ORDER BY k) @@ -1183,36 +1183,36 @@ build SELECT * FROM pairs, square WHERE pairs.b = square.n ---- project - ├── columns: a:1 b:2!null n:4!null sq:5 + ├── columns: a:1 b:2!null n:5!null sq:6 └── select - ├── columns: a:1 b:2!null rowid:3!null n:4!null sq:5 + ├── columns: a:1 b:2!null rowid:3!null pairs.crdb_internal_mvcc_timestamp:4 n:5!null sq:6 square.crdb_internal_mvcc_timestamp:7 ├── inner-join (cross) - │ ├── columns: a:1 b:2 rowid:3!null n:4!null sq:5 + │ ├── columns: a:1 b:2 rowid:3!null pairs.crdb_internal_mvcc_timestamp:4 n:5!null sq:6 square.crdb_internal_mvcc_timestamp:7 │ ├── scan pairs - │ │ └── columns: a:1 b:2 rowid:3!null + │ │ └── columns: a:1 b:2 rowid:3!null pairs.crdb_internal_mvcc_timestamp:4 │ ├── scan square - │ │ └── columns: n:4!null sq:5 + │ │ └── columns: n:5!null sq:6 square.crdb_internal_mvcc_timestamp:7 │ └── filters (true) └── filters - └── b:2 = n:4 + └── b:2 = n:5 # The filter expression becomes an ON predicate. build SELECT * FROM pairs, square WHERE pairs.a + pairs.b = square.sq ---- project - ├── columns: a:1 b:2 n:4!null sq:5 + ├── columns: a:1 b:2 n:5!null sq:6 └── select - ├── columns: a:1 b:2 rowid:3!null n:4!null sq:5 + ├── columns: a:1 b:2 rowid:3!null pairs.crdb_internal_mvcc_timestamp:4 n:5!null sq:6 square.crdb_internal_mvcc_timestamp:7 ├── inner-join (cross) - │ ├── columns: a:1 b:2 rowid:3!null n:4!null sq:5 + │ ├── columns: a:1 b:2 rowid:3!null pairs.crdb_internal_mvcc_timestamp:4 n:5!null sq:6 square.crdb_internal_mvcc_timestamp:7 │ ├── scan pairs - │ │ └── columns: a:1 b:2 rowid:3!null + │ │ └── columns: a:1 b:2 rowid:3!null pairs.crdb_internal_mvcc_timestamp:4 │ ├── scan square - │ │ └── columns: n:4!null sq:5 + │ │ └── columns: n:5!null sq:6 square.crdb_internal_mvcc_timestamp:7 │ └── filters (true) └── filters - └── (a:1 + b:2) = sq:5 + └── (a:1 + b:2) = sq:6 # Query similar to the one above, but the filter refers to a rendered # expression and can't "break through". See the comment for propagateFilters @@ -1221,55 +1221,55 @@ build SELECT a, b, n, sq FROM (SELECT a, b, a + b AS sum, n, sq FROM pairs, square) WHERE sum = sq ---- project - ├── columns: a:1 b:2 n:4!null sq:5!null + ├── columns: a:1 b:2 n:5!null sq:6!null └── select - ├── columns: a:1 b:2 n:4!null sq:5!null sum:6!null + ├── columns: a:1 b:2 n:5!null sq:6!null sum:8!null ├── project - │ ├── columns: sum:6 a:1 b:2 n:4!null sq:5 + │ ├── columns: sum:8 a:1 b:2 n:5!null sq:6 │ ├── inner-join (cross) - │ │ ├── columns: a:1 b:2 rowid:3!null n:4!null sq:5 + │ │ ├── columns: a:1 b:2 rowid:3!null pairs.crdb_internal_mvcc_timestamp:4 n:5!null sq:6 square.crdb_internal_mvcc_timestamp:7 │ │ ├── scan pairs - │ │ │ └── columns: a:1 b:2 rowid:3!null + │ │ │ └── columns: a:1 b:2 rowid:3!null pairs.crdb_internal_mvcc_timestamp:4 │ │ ├── scan square - │ │ │ └── columns: n:4!null sq:5 + │ │ │ └── columns: n:5!null sq:6 square.crdb_internal_mvcc_timestamp:7 │ │ └── filters (true) │ └── projections - │ └── a:1 + b:2 [as=sum:6] + │ └── a:1 + b:2 [as=sum:8] └── filters - └── sum:6 = sq:5 + └── sum:8 = sq:6 # The filter expression must stay on top of the outer join. build SELECT * FROM pairs FULL OUTER JOIN square ON pairs.a + pairs.b = square.sq ---- project - ├── columns: a:1 b:2 n:4 sq:5 + ├── columns: a:1 b:2 n:5 sq:6 └── full-join (cross) - ├── columns: a:1 b:2 rowid:3 n:4 sq:5 + ├── columns: a:1 b:2 rowid:3 pairs.crdb_internal_mvcc_timestamp:4 n:5 sq:6 square.crdb_internal_mvcc_timestamp:7 ├── scan pairs - │ └── columns: a:1 b:2 rowid:3!null + │ └── columns: a:1 b:2 rowid:3!null pairs.crdb_internal_mvcc_timestamp:4 ├── scan square - │ └── columns: n:4!null sq:5 + │ └── columns: n:5!null sq:6 square.crdb_internal_mvcc_timestamp:7 └── filters - └── (a:1 + b:2) = sq:5 + └── (a:1 + b:2) = sq:6 build SELECT * FROM pairs FULL OUTER JOIN square ON pairs.a + pairs.b = square.sq WHERE pairs.b%2 <> square.sq%2 ---- project - ├── columns: a:1 b:2 n:4 sq:5 + ├── columns: a:1 b:2 n:5 sq:6 └── select - ├── columns: a:1 b:2 rowid:3 n:4 sq:5 + ├── columns: a:1 b:2 rowid:3 pairs.crdb_internal_mvcc_timestamp:4 n:5 sq:6 square.crdb_internal_mvcc_timestamp:7 ├── full-join (cross) - │ ├── columns: a:1 b:2 rowid:3 n:4 sq:5 + │ ├── columns: a:1 b:2 rowid:3 pairs.crdb_internal_mvcc_timestamp:4 n:5 sq:6 square.crdb_internal_mvcc_timestamp:7 │ ├── scan pairs - │ │ └── columns: a:1 b:2 rowid:3!null + │ │ └── columns: a:1 b:2 rowid:3!null pairs.crdb_internal_mvcc_timestamp:4 │ ├── scan square - │ │ └── columns: n:4!null sq:5 + │ │ └── columns: n:5!null sq:6 square.crdb_internal_mvcc_timestamp:7 │ └── filters - │ └── (a:1 + b:2) = sq:5 + │ └── (a:1 + b:2) = sq:6 └── filters - └── (b:2 % 2) != (sq:5 % 2) + └── (b:2 % 2) != (sq:6 % 2) # Filter propagation through outer joins. @@ -1279,19 +1279,19 @@ SELECT * WHERE b > 1 AND (n IS NULL OR n > 1) AND (n IS NULL OR a < sq) ---- select - ├── columns: a:1 b:2!null n:4 sq:5 + ├── columns: a:1 b:2!null n:5 sq:6 ├── project - │ ├── columns: a:1 b:2 n:4 sq:5 + │ ├── columns: a:1 b:2 n:5 sq:6 │ └── left-join (cross) - │ ├── columns: a:1 b:2 rowid:3!null n:4 sq:5 + │ ├── columns: a:1 b:2 rowid:3!null pairs.crdb_internal_mvcc_timestamp:4 n:5 sq:6 square.crdb_internal_mvcc_timestamp:7 │ ├── scan pairs - │ │ └── columns: a:1 b:2 rowid:3!null + │ │ └── columns: a:1 b:2 rowid:3!null pairs.crdb_internal_mvcc_timestamp:4 │ ├── scan square - │ │ └── columns: n:4!null sq:5 + │ │ └── columns: n:5!null sq:6 square.crdb_internal_mvcc_timestamp:7 │ └── filters - │ └── ((b:2 = sq:5) AND (a:1 > 1)) AND (n:4 < 6) + │ └── ((b:2 = sq:6) AND (a:1 > 1)) AND (n:5 < 6) └── filters - └── ((b:2 > 1) AND ((n:4 IS NULL) OR (n:4 > 1))) AND ((n:4 IS NULL) OR (a:1 < sq:5)) + └── ((b:2 > 1) AND ((n:5 IS NULL) OR (n:5 > 1))) AND ((n:5 IS NULL) OR (a:1 < sq:6)) build SELECT * @@ -1299,19 +1299,19 @@ SELECT * WHERE (a IS NULL OR a > 2) AND n > 1 AND (a IS NULL OR a < sq) ---- select - ├── columns: a:1 b:2 n:4!null sq:5 + ├── columns: a:1 b:2 n:5!null sq:6 ├── project - │ ├── columns: a:1 b:2 n:4!null sq:5 + │ ├── columns: a:1 b:2 n:5!null sq:6 │ └── right-join (cross) - │ ├── columns: a:1 b:2 rowid:3 n:4!null sq:5 + │ ├── columns: a:1 b:2 rowid:3 pairs.crdb_internal_mvcc_timestamp:4 n:5!null sq:6 square.crdb_internal_mvcc_timestamp:7 │ ├── scan pairs - │ │ └── columns: a:1 b:2 rowid:3!null + │ │ └── columns: a:1 b:2 rowid:3!null pairs.crdb_internal_mvcc_timestamp:4 │ ├── scan square - │ │ └── columns: n:4!null sq:5 + │ │ └── columns: n:5!null sq:6 square.crdb_internal_mvcc_timestamp:7 │ └── filters - │ └── ((b:2 = sq:5) AND (a:1 > 1)) AND (n:4 < 6) + │ └── ((b:2 = sq:6) AND (a:1 > 1)) AND (n:5 < 6) └── filters - └── (((a:1 IS NULL) OR (a:1 > 2)) AND (n:4 > 1)) AND ((a:1 IS NULL) OR (a:1 < sq:5)) + └── (((a:1 IS NULL) OR (a:1 > 2)) AND (n:5 > 1)) AND ((a:1 IS NULL) OR (a:1 < sq:6)) # The simpler plan for an inner join, to compare. build @@ -1320,19 +1320,19 @@ SELECT * WHERE (a IS NULL OR a > 2) AND n > 1 AND (a IS NULL OR a < sq) ---- select - ├── columns: a:1!null b:2!null n:4!null sq:5!null + ├── columns: a:1!null b:2!null n:5!null sq:6!null ├── project - │ ├── columns: a:1!null b:2!null n:4!null sq:5!null + │ ├── columns: a:1!null b:2!null n:5!null sq:6!null │ └── inner-join (cross) - │ ├── columns: a:1!null b:2!null rowid:3!null n:4!null sq:5!null + │ ├── columns: a:1!null b:2!null rowid:3!null pairs.crdb_internal_mvcc_timestamp:4 n:5!null sq:6!null square.crdb_internal_mvcc_timestamp:7 │ ├── scan pairs - │ │ └── columns: a:1 b:2 rowid:3!null + │ │ └── columns: a:1 b:2 rowid:3!null pairs.crdb_internal_mvcc_timestamp:4 │ ├── scan square - │ │ └── columns: n:4!null sq:5 + │ │ └── columns: n:5!null sq:6 square.crdb_internal_mvcc_timestamp:7 │ └── filters - │ └── ((b:2 = sq:5) AND (a:1 > 1)) AND (n:4 < 6) + │ └── ((b:2 = sq:6) AND (a:1 > 1)) AND (n:5 < 6) └── filters - └── (((a:1 IS NULL) OR (a:1 > 2)) AND (n:4 > 1)) AND ((a:1 IS NULL) OR (a:1 < sq:5)) + └── (((a:1 IS NULL) OR (a:1 > 2)) AND (n:5 > 1)) AND ((a:1 IS NULL) OR (a:1 < sq:6)) exec-ddl @@ -1347,185 +1347,185 @@ build SELECT * FROM t1 JOIN t2 USING(x) ---- project - ├── columns: x:2!null col1:1 col2:3 y:4 col3:6 y:7 col4:9 + ├── columns: x:2!null col1:1 col2:3 y:4 col3:7 y:8 col4:10 └── inner-join (hash) - ├── columns: col1:1 t1.x:2!null col2:3 t1.y:4 t1.rowid:5!null col3:6 t2.y:7 t2.x:8!null col4:9 t2.rowid:10!null + ├── columns: col1:1 t1.x:2!null col2:3 t1.y:4 t1.rowid:5!null t1.crdb_internal_mvcc_timestamp:6 col3:7 t2.y:8 t2.x:9!null col4:10 t2.rowid:11!null t2.crdb_internal_mvcc_timestamp:12 ├── scan t1 - │ └── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5!null + │ └── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5!null t1.crdb_internal_mvcc_timestamp:6 ├── scan t2 - │ └── columns: col3:6 t2.y:7 t2.x:8 col4:9 t2.rowid:10!null + │ └── columns: col3:7 t2.y:8 t2.x:9 col4:10 t2.rowid:11!null t2.crdb_internal_mvcc_timestamp:12 └── filters - └── t1.x:2 = t2.x:8 + └── t1.x:2 = t2.x:9 build SELECT * FROM t1 NATURAL JOIN t2 ---- project - ├── columns: x:2!null y:4!null col1:1 col2:3 col3:6 col4:9 + ├── columns: x:2!null y:4!null col1:1 col2:3 col3:7 col4:10 └── inner-join (hash) - ├── columns: col1:1 t1.x:2!null col2:3 t1.y:4!null t1.rowid:5!null col3:6 t2.y:7!null t2.x:8!null col4:9 t2.rowid:10!null + ├── columns: col1:1 t1.x:2!null col2:3 t1.y:4!null t1.rowid:5!null t1.crdb_internal_mvcc_timestamp:6 col3:7 t2.y:8!null t2.x:9!null col4:10 t2.rowid:11!null t2.crdb_internal_mvcc_timestamp:12 ├── scan t1 - │ └── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5!null + │ └── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5!null t1.crdb_internal_mvcc_timestamp:6 ├── scan t2 - │ └── columns: col3:6 t2.y:7 t2.x:8 col4:9 t2.rowid:10!null + │ └── columns: col3:7 t2.y:8 t2.x:9 col4:10 t2.rowid:11!null t2.crdb_internal_mvcc_timestamp:12 └── filters - ├── t1.x:2 = t2.x:8 - └── t1.y:4 = t2.y:7 + ├── t1.x:2 = t2.x:9 + └── t1.y:4 = t2.y:8 build SELECT x, t1.x, t2.x FROM t1 NATURAL JOIN t2 ---- project - ├── columns: x:2!null x:2!null x:8!null + ├── columns: x:2!null x:2!null x:9!null └── inner-join (hash) - ├── columns: col1:1 t1.x:2!null col2:3 t1.y:4!null t1.rowid:5!null col3:6 t2.y:7!null t2.x:8!null col4:9 t2.rowid:10!null + ├── columns: col1:1 t1.x:2!null col2:3 t1.y:4!null t1.rowid:5!null t1.crdb_internal_mvcc_timestamp:6 col3:7 t2.y:8!null t2.x:9!null col4:10 t2.rowid:11!null t2.crdb_internal_mvcc_timestamp:12 ├── scan t1 - │ └── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5!null + │ └── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5!null t1.crdb_internal_mvcc_timestamp:6 ├── scan t2 - │ └── columns: col3:6 t2.y:7 t2.x:8 col4:9 t2.rowid:10!null + │ └── columns: col3:7 t2.y:8 t2.x:9 col4:10 t2.rowid:11!null t2.crdb_internal_mvcc_timestamp:12 └── filters - ├── t1.x:2 = t2.x:8 - └── t1.y:4 = t2.y:7 + ├── t1.x:2 = t2.x:9 + └── t1.y:4 = t2.y:8 build SELECT t1.*, t2.* FROM t1 NATURAL JOIN t2 ---- project - ├── columns: x:2!null y:4!null col1:1 col2:3 col3:6 col4:9 + ├── columns: x:2!null y:4!null col1:1 col2:3 col3:7 col4:10 └── inner-join (hash) - ├── columns: col1:1 t1.x:2!null col2:3 t1.y:4!null t1.rowid:5!null col3:6 t2.y:7!null t2.x:8!null col4:9 t2.rowid:10!null + ├── columns: col1:1 t1.x:2!null col2:3 t1.y:4!null t1.rowid:5!null t1.crdb_internal_mvcc_timestamp:6 col3:7 t2.y:8!null t2.x:9!null col4:10 t2.rowid:11!null t2.crdb_internal_mvcc_timestamp:12 ├── scan t1 - │ └── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5!null + │ └── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5!null t1.crdb_internal_mvcc_timestamp:6 ├── scan t2 - │ └── columns: col3:6 t2.y:7 t2.x:8 col4:9 t2.rowid:10!null + │ └── columns: col3:7 t2.y:8 t2.x:9 col4:10 t2.rowid:11!null t2.crdb_internal_mvcc_timestamp:12 └── filters - ├── t1.x:2 = t2.x:8 - └── t1.y:4 = t2.y:7 + ├── t1.x:2 = t2.x:9 + └── t1.y:4 = t2.y:8 build SELECT * FROM t1 JOIN t2 ON t2.x=t1.x ---- project - ├── columns: col1:1 x:2!null col2:3 y:4 col3:6 y:7 x:8!null col4:9 + ├── columns: col1:1 x:2!null col2:3 y:4 col3:7 y:8 x:9!null col4:10 └── inner-join (hash) - ├── columns: col1:1 t1.x:2!null col2:3 t1.y:4 t1.rowid:5!null col3:6 t2.y:7 t2.x:8!null col4:9 t2.rowid:10!null + ├── columns: col1:1 t1.x:2!null col2:3 t1.y:4 t1.rowid:5!null t1.crdb_internal_mvcc_timestamp:6 col3:7 t2.y:8 t2.x:9!null col4:10 t2.rowid:11!null t2.crdb_internal_mvcc_timestamp:12 ├── scan t1 - │ └── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5!null + │ └── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5!null t1.crdb_internal_mvcc_timestamp:6 ├── scan t2 - │ └── columns: col3:6 t2.y:7 t2.x:8 col4:9 t2.rowid:10!null + │ └── columns: col3:7 t2.y:8 t2.x:9 col4:10 t2.rowid:11!null t2.crdb_internal_mvcc_timestamp:12 └── filters - └── t2.x:8 = t1.x:2 + └── t2.x:9 = t1.x:2 build SELECT * FROM t1 FULL OUTER JOIN t2 USING(x) ---- project - ├── columns: x:11 col1:1 col2:3 y:4 col3:6 y:7 col4:9 + ├── columns: x:13 col1:1 col2:3 y:4 col3:7 y:8 col4:10 └── project - ├── columns: x:11 col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5 col3:6 t2.y:7 t2.x:8 col4:9 t2.rowid:10 + ├── columns: x:13 col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5 t1.crdb_internal_mvcc_timestamp:6 col3:7 t2.y:8 t2.x:9 col4:10 t2.rowid:11 t2.crdb_internal_mvcc_timestamp:12 ├── full-join (hash) - │ ├── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5 col3:6 t2.y:7 t2.x:8 col4:9 t2.rowid:10 + │ ├── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5 t1.crdb_internal_mvcc_timestamp:6 col3:7 t2.y:8 t2.x:9 col4:10 t2.rowid:11 t2.crdb_internal_mvcc_timestamp:12 │ ├── scan t1 - │ │ └── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5!null + │ │ └── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5!null t1.crdb_internal_mvcc_timestamp:6 │ ├── scan t2 - │ │ └── columns: col3:6 t2.y:7 t2.x:8 col4:9 t2.rowid:10!null + │ │ └── columns: col3:7 t2.y:8 t2.x:9 col4:10 t2.rowid:11!null t2.crdb_internal_mvcc_timestamp:12 │ └── filters - │ └── t1.x:2 = t2.x:8 + │ └── t1.x:2 = t2.x:9 └── projections - └── COALESCE(t1.x:2, t2.x:8) [as=x:11] + └── COALESCE(t1.x:2, t2.x:9) [as=x:13] build SELECT * FROM t1 NATURAL FULL OUTER JOIN t2 ---- project - ├── columns: x:11 y:12 col1:1 col2:3 col3:6 col4:9 + ├── columns: x:13 y:14 col1:1 col2:3 col3:7 col4:10 └── project - ├── columns: x:11 y:12 col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5 col3:6 t2.y:7 t2.x:8 col4:9 t2.rowid:10 + ├── columns: x:13 y:14 col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5 t1.crdb_internal_mvcc_timestamp:6 col3:7 t2.y:8 t2.x:9 col4:10 t2.rowid:11 t2.crdb_internal_mvcc_timestamp:12 ├── full-join (hash) - │ ├── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5 col3:6 t2.y:7 t2.x:8 col4:9 t2.rowid:10 + │ ├── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5 t1.crdb_internal_mvcc_timestamp:6 col3:7 t2.y:8 t2.x:9 col4:10 t2.rowid:11 t2.crdb_internal_mvcc_timestamp:12 │ ├── scan t1 - │ │ └── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5!null + │ │ └── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5!null t1.crdb_internal_mvcc_timestamp:6 │ ├── scan t2 - │ │ └── columns: col3:6 t2.y:7 t2.x:8 col4:9 t2.rowid:10!null + │ │ └── columns: col3:7 t2.y:8 t2.x:9 col4:10 t2.rowid:11!null t2.crdb_internal_mvcc_timestamp:12 │ └── filters - │ ├── t1.x:2 = t2.x:8 - │ └── t1.y:4 = t2.y:7 + │ ├── t1.x:2 = t2.x:9 + │ └── t1.y:4 = t2.y:8 └── projections - ├── COALESCE(t1.x:2, t2.x:8) [as=x:11] - └── COALESCE(t1.y:4, t2.y:7) [as=y:12] + ├── COALESCE(t1.x:2, t2.x:9) [as=x:13] + └── COALESCE(t1.y:4, t2.y:8) [as=y:14] # Regression: computed columns are not wrapped with Variable outside join. build SELECT * FROM (SELECT x, x+1 AS plus1 FROM t1) NATURAL FULL OUTER JOIN (SELECT x, 2 AS two FROM t2) ---- project - ├── columns: x:13 plus1:6 two:12 + ├── columns: x:15 plus1:7 two:14 └── project - ├── columns: x:13 t1.x:2 plus1:6 t2.x:9 two:12 + ├── columns: x:15 t1.x:2 plus1:7 t2.x:10 two:14 ├── full-join (hash) - │ ├── columns: t1.x:2 plus1:6 t2.x:9 two:12 + │ ├── columns: t1.x:2 plus1:7 t2.x:10 two:14 │ ├── project - │ │ ├── columns: plus1:6 t1.x:2 + │ │ ├── columns: plus1:7 t1.x:2 │ │ ├── scan t1 - │ │ │ └── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5!null + │ │ │ └── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5!null t1.crdb_internal_mvcc_timestamp:6 │ │ └── projections - │ │ └── t1.x:2 + 1 [as=plus1:6] + │ │ └── t1.x:2 + 1 [as=plus1:7] │ ├── project - │ │ ├── columns: two:12!null t2.x:9 + │ │ ├── columns: two:14!null t2.x:10 │ │ ├── scan t2 - │ │ │ └── columns: col3:7 t2.y:8 t2.x:9 col4:10 t2.rowid:11!null + │ │ │ └── columns: col3:8 t2.y:9 t2.x:10 col4:11 t2.rowid:12!null t2.crdb_internal_mvcc_timestamp:13 │ │ └── projections - │ │ └── 2 [as=two:12] + │ │ └── 2 [as=two:14] │ └── filters - │ └── t1.x:2 = t2.x:9 + │ └── t1.x:2 = t2.x:10 └── projections - └── COALESCE(t1.x:2, t2.x:9) [as=x:13] + └── COALESCE(t1.x:2, t2.x:10) [as=x:15] build SELECT * FROM t1 FULL OUTER JOIN t2 ON t1.x=t2.x ---- project - ├── columns: col1:1 x:2 col2:3 y:4 col3:6 y:7 x:8 col4:9 + ├── columns: col1:1 x:2 col2:3 y:4 col3:7 y:8 x:9 col4:10 └── full-join (hash) - ├── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5 col3:6 t2.y:7 t2.x:8 col4:9 t2.rowid:10 + ├── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5 t1.crdb_internal_mvcc_timestamp:6 col3:7 t2.y:8 t2.x:9 col4:10 t2.rowid:11 t2.crdb_internal_mvcc_timestamp:12 ├── scan t1 - │ └── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5!null + │ └── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5!null t1.crdb_internal_mvcc_timestamp:6 ├── scan t2 - │ └── columns: col3:6 t2.y:7 t2.x:8 col4:9 t2.rowid:10!null + │ └── columns: col3:7 t2.y:8 t2.x:9 col4:10 t2.rowid:11!null t2.crdb_internal_mvcc_timestamp:12 └── filters - └── t1.x:2 = t2.x:8 + └── t1.x:2 = t2.x:9 build SELECT t2.x, t1.x, x FROM t1 JOIN t2 USING(x) ---- project - ├── columns: x:8!null x:2!null x:2!null + ├── columns: x:9!null x:2!null x:2!null └── inner-join (hash) - ├── columns: col1:1 t1.x:2!null col2:3 t1.y:4 t1.rowid:5!null col3:6 t2.y:7 t2.x:8!null col4:9 t2.rowid:10!null + ├── columns: col1:1 t1.x:2!null col2:3 t1.y:4 t1.rowid:5!null t1.crdb_internal_mvcc_timestamp:6 col3:7 t2.y:8 t2.x:9!null col4:10 t2.rowid:11!null t2.crdb_internal_mvcc_timestamp:12 ├── scan t1 - │ └── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5!null + │ └── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5!null t1.crdb_internal_mvcc_timestamp:6 ├── scan t2 - │ └── columns: col3:6 t2.y:7 t2.x:8 col4:9 t2.rowid:10!null + │ └── columns: col3:7 t2.y:8 t2.x:9 col4:10 t2.rowid:11!null t2.crdb_internal_mvcc_timestamp:12 └── filters - └── t1.x:2 = t2.x:8 + └── t1.x:2 = t2.x:9 build SELECT t2.x, t1.x, x FROM t1 FULL OUTER JOIN t2 USING(x) ---- project - ├── columns: x:8 x:2 x:11 + ├── columns: x:9 x:2 x:13 └── project - ├── columns: x:11 col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5 col3:6 t2.y:7 t2.x:8 col4:9 t2.rowid:10 + ├── columns: x:13 col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5 t1.crdb_internal_mvcc_timestamp:6 col3:7 t2.y:8 t2.x:9 col4:10 t2.rowid:11 t2.crdb_internal_mvcc_timestamp:12 ├── full-join (hash) - │ ├── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5 col3:6 t2.y:7 t2.x:8 col4:9 t2.rowid:10 + │ ├── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5 t1.crdb_internal_mvcc_timestamp:6 col3:7 t2.y:8 t2.x:9 col4:10 t2.rowid:11 t2.crdb_internal_mvcc_timestamp:12 │ ├── scan t1 - │ │ └── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5!null + │ │ └── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5!null t1.crdb_internal_mvcc_timestamp:6 │ ├── scan t2 - │ │ └── columns: col3:6 t2.y:7 t2.x:8 col4:9 t2.rowid:10!null + │ │ └── columns: col3:7 t2.y:8 t2.x:9 col4:10 t2.rowid:11!null t2.crdb_internal_mvcc_timestamp:12 │ └── filters - │ └── t1.x:2 = t2.x:8 + │ └── t1.x:2 = t2.x:9 └── projections - └── COALESCE(t1.x:2, t2.x:8) [as=x:11] + └── COALESCE(t1.x:2, t2.x:9) [as=x:13] # Test for #19536. build @@ -1534,16 +1534,16 @@ SELECT x FROM t1 NATURAL JOIN (SELECT * FROM t2) project ├── columns: x:2!null └── inner-join (hash) - ├── columns: col1:1 t1.x:2!null col2:3 t1.y:4!null t1.rowid:5!null col3:6 t2.y:7!null t2.x:8!null col4:9 + ├── columns: col1:1 t1.x:2!null col2:3 t1.y:4!null t1.rowid:5!null t1.crdb_internal_mvcc_timestamp:6 col3:7 t2.y:8!null t2.x:9!null col4:10 ├── scan t1 - │ └── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5!null + │ └── columns: col1:1 t1.x:2 col2:3 t1.y:4 t1.rowid:5!null t1.crdb_internal_mvcc_timestamp:6 ├── project - │ ├── columns: col3:6 t2.y:7 t2.x:8 col4:9 + │ ├── columns: col3:7 t2.y:8 t2.x:9 col4:10 │ └── scan t2 - │ └── columns: col3:6 t2.y:7 t2.x:8 col4:9 t2.rowid:10!null + │ └── columns: col3:7 t2.y:8 t2.x:9 col4:10 t2.rowid:11!null t2.crdb_internal_mvcc_timestamp:12 └── filters - ├── t1.x:2 = t2.x:8 - └── t1.y:4 = t2.y:7 + ├── t1.x:2 = t2.x:9 + └── t1.y:4 = t2.y:8 # Tests for merge join ordering information. exec-ddl @@ -1565,14 +1565,16 @@ CREATE TABLE pkBAD (a INT, b INT, c INT, d INT, PRIMARY KEY(b,a,d)) build SELECT * FROM pkBA AS l JOIN pkBC AS r ON l.a = r.a AND l.b = r.b AND l.c = r.c ---- -inner-join (cross) - ├── columns: a:1!null b:2!null c:3!null d:4 a:5!null b:6!null c:7!null d:8 - ├── scan l - │ └── columns: l.a:1!null l.b:2!null l.c:3 l.d:4 - ├── scan r - │ └── columns: r.a:5 r.b:6!null r.c:7!null r.d:8 - └── filters - └── ((l.a:1 = r.a:5) AND (l.b:2 = r.b:6)) AND (l.c:3 = r.c:7) +project + ├── columns: a:1!null b:2!null c:3!null d:4 a:6!null b:7!null c:8!null d:9 + └── inner-join (cross) + ├── columns: l.a:1!null l.b:2!null l.c:3!null l.d:4 l.crdb_internal_mvcc_timestamp:5 r.a:6!null r.b:7!null r.c:8!null r.d:9 r.crdb_internal_mvcc_timestamp:10 + ├── scan l + │ └── columns: l.a:1!null l.b:2!null l.c:3 l.d:4 l.crdb_internal_mvcc_timestamp:5 + ├── scan r + │ └── columns: r.a:6 r.b:7!null r.c:8!null r.d:9 r.crdb_internal_mvcc_timestamp:10 + └── filters + └── ((l.a:1 = r.a:6) AND (l.b:2 = r.b:7)) AND (l.c:3 = r.c:8) build SELECT * FROM pkBA NATURAL JOIN pkBAD @@ -1580,44 +1582,46 @@ SELECT * FROM pkBA NATURAL JOIN pkBAD project ├── columns: a:1!null b:2!null c:3!null d:4!null └── inner-join (hash) - ├── columns: pkba.a:1!null pkba.b:2!null pkba.c:3!null pkba.d:4!null pkbad.a:5!null pkbad.b:6!null pkbad.c:7!null pkbad.d:8!null + ├── columns: pkba.a:1!null pkba.b:2!null pkba.c:3!null pkba.d:4!null pkba.crdb_internal_mvcc_timestamp:5 pkbad.a:6!null pkbad.b:7!null pkbad.c:8!null pkbad.d:9!null pkbad.crdb_internal_mvcc_timestamp:10 ├── scan pkba - │ └── columns: pkba.a:1!null pkba.b:2!null pkba.c:3 pkba.d:4 + │ └── columns: pkba.a:1!null pkba.b:2!null pkba.c:3 pkba.d:4 pkba.crdb_internal_mvcc_timestamp:5 ├── scan pkbad - │ └── columns: pkbad.a:5!null pkbad.b:6!null pkbad.c:7 pkbad.d:8!null + │ └── columns: pkbad.a:6!null pkbad.b:7!null pkbad.c:8 pkbad.d:9!null pkbad.crdb_internal_mvcc_timestamp:10 └── filters - ├── pkba.a:1 = pkbad.a:5 - ├── pkba.b:2 = pkbad.b:6 - ├── pkba.c:3 = pkbad.c:7 - └── pkba.d:4 = pkbad.d:8 + ├── pkba.a:1 = pkbad.a:6 + ├── pkba.b:2 = pkbad.b:7 + ├── pkba.c:3 = pkbad.c:8 + └── pkba.d:4 = pkbad.d:9 build SELECT * FROM pkBAC AS l JOIN pkBAC AS r USING(a, b, c) ---- project - ├── columns: a:1!null b:2!null c:3!null d:4 d:8 + ├── columns: a:1!null b:2!null c:3!null d:4 d:9 └── inner-join (hash) - ├── columns: l.a:1!null l.b:2!null l.c:3!null l.d:4 r.a:5!null r.b:6!null r.c:7!null r.d:8 + ├── columns: l.a:1!null l.b:2!null l.c:3!null l.d:4 l.crdb_internal_mvcc_timestamp:5 r.a:6!null r.b:7!null r.c:8!null r.d:9 r.crdb_internal_mvcc_timestamp:10 ├── scan l - │ └── columns: l.a:1!null l.b:2!null l.c:3!null l.d:4 + │ └── columns: l.a:1!null l.b:2!null l.c:3!null l.d:4 l.crdb_internal_mvcc_timestamp:5 ├── scan r - │ └── columns: r.a:5!null r.b:6!null r.c:7!null r.d:8 + │ └── columns: r.a:6!null r.b:7!null r.c:8!null r.d:9 r.crdb_internal_mvcc_timestamp:10 └── filters - ├── l.a:1 = r.a:5 - ├── l.b:2 = r.b:6 - └── l.c:3 = r.c:7 + ├── l.a:1 = r.a:6 + ├── l.b:2 = r.b:7 + └── l.c:3 = r.c:8 build SELECT * FROM pkBAC AS l JOIN pkBAD AS r ON l.c = r.d AND l.a = r.a AND l.b = r.b ---- -inner-join (cross) - ├── columns: a:1!null b:2!null c:3!null d:4 a:5!null b:6!null c:7 d:8!null - ├── scan l - │ └── columns: l.a:1!null l.b:2!null l.c:3!null l.d:4 - ├── scan r - │ └── columns: r.a:5!null r.b:6!null r.c:7 r.d:8!null - └── filters - └── ((l.c:3 = r.d:8) AND (l.a:1 = r.a:5)) AND (l.b:2 = r.b:6) +project + ├── columns: a:1!null b:2!null c:3!null d:4 a:6!null b:7!null c:8 d:9!null + └── inner-join (cross) + ├── columns: l.a:1!null l.b:2!null l.c:3!null l.d:4 l.crdb_internal_mvcc_timestamp:5 r.a:6!null r.b:7!null r.c:8 r.d:9!null r.crdb_internal_mvcc_timestamp:10 + ├── scan l + │ └── columns: l.a:1!null l.b:2!null l.c:3!null l.d:4 l.crdb_internal_mvcc_timestamp:5 + ├── scan r + │ └── columns: r.a:6!null r.b:7!null r.c:8 r.d:9!null r.crdb_internal_mvcc_timestamp:10 + └── filters + └── ((l.c:3 = r.d:9) AND (l.a:1 = r.a:6)) AND (l.b:2 = r.b:7) # Tests with joins with merged columns of collated string type. exec-ddl @@ -1632,65 +1636,65 @@ build SELECT s, str1.s, str2.s FROM str1 INNER JOIN str2 USING(s) ---- project - ├── columns: s:2!null s:2!null s:4!null + ├── columns: s:2!null s:2!null s:5!null └── inner-join (hash) - ├── columns: str1.a:1!null str1.s:2!null str2.a:3!null str2.s:4!null + ├── columns: str1.a:1!null str1.s:2!null str1.crdb_internal_mvcc_timestamp:3 str2.a:4!null str2.s:5!null str2.crdb_internal_mvcc_timestamp:6 ├── scan str1 - │ └── columns: str1.a:1!null str1.s:2 + │ └── columns: str1.a:1!null str1.s:2 str1.crdb_internal_mvcc_timestamp:3 ├── scan str2 - │ └── columns: str2.a:3!null str2.s:4 + │ └── columns: str2.a:4!null str2.s:5 str2.crdb_internal_mvcc_timestamp:6 └── filters - └── str1.s:2 = str2.s:4 + └── str1.s:2 = str2.s:5 build SELECT s, str1.s, str2.s FROM str1 LEFT OUTER JOIN str2 USING(s) ---- project - ├── columns: s:2 s:2 s:4 + ├── columns: s:2 s:2 s:5 └── left-join (hash) - ├── columns: str1.a:1!null str1.s:2 str2.a:3 str2.s:4 + ├── columns: str1.a:1!null str1.s:2 str1.crdb_internal_mvcc_timestamp:3 str2.a:4 str2.s:5 str2.crdb_internal_mvcc_timestamp:6 ├── scan str1 - │ └── columns: str1.a:1!null str1.s:2 + │ └── columns: str1.a:1!null str1.s:2 str1.crdb_internal_mvcc_timestamp:3 ├── scan str2 - │ └── columns: str2.a:3!null str2.s:4 + │ └── columns: str2.a:4!null str2.s:5 str2.crdb_internal_mvcc_timestamp:6 └── filters - └── str1.s:2 = str2.s:4 + └── str1.s:2 = str2.s:5 build SELECT s, str1.s, str2.s FROM str1 RIGHT OUTER JOIN str2 USING(s) ---- project - ├── columns: s:5 s:2 s:4 + ├── columns: s:7 s:2 s:5 └── project - ├── columns: s:5 str1.a:1 str1.s:2 str2.a:3!null str2.s:4 + ├── columns: s:7 str1.a:1 str1.s:2 str1.crdb_internal_mvcc_timestamp:3 str2.a:4!null str2.s:5 str2.crdb_internal_mvcc_timestamp:6 ├── right-join (hash) - │ ├── columns: str1.a:1 str1.s:2 str2.a:3!null str2.s:4 + │ ├── columns: str1.a:1 str1.s:2 str1.crdb_internal_mvcc_timestamp:3 str2.a:4!null str2.s:5 str2.crdb_internal_mvcc_timestamp:6 │ ├── scan str1 - │ │ └── columns: str1.a:1!null str1.s:2 + │ │ └── columns: str1.a:1!null str1.s:2 str1.crdb_internal_mvcc_timestamp:3 │ ├── scan str2 - │ │ └── columns: str2.a:3!null str2.s:4 + │ │ └── columns: str2.a:4!null str2.s:5 str2.crdb_internal_mvcc_timestamp:6 │ └── filters - │ └── str1.s:2 = str2.s:4 + │ └── str1.s:2 = str2.s:5 └── projections - └── COALESCE(str1.s:2, str2.s:4) [as=s:5] + └── COALESCE(str1.s:2, str2.s:5) [as=s:7] build SELECT s, str1.s, str2.s FROM str1 FULL OUTER JOIN str2 USING(s) ---- project - ├── columns: s:5 s:2 s:4 + ├── columns: s:7 s:2 s:5 └── project - ├── columns: s:5 str1.a:1 str1.s:2 str2.a:3 str2.s:4 + ├── columns: s:7 str1.a:1 str1.s:2 str1.crdb_internal_mvcc_timestamp:3 str2.a:4 str2.s:5 str2.crdb_internal_mvcc_timestamp:6 ├── full-join (hash) - │ ├── columns: str1.a:1 str1.s:2 str2.a:3 str2.s:4 + │ ├── columns: str1.a:1 str1.s:2 str1.crdb_internal_mvcc_timestamp:3 str2.a:4 str2.s:5 str2.crdb_internal_mvcc_timestamp:6 │ ├── scan str1 - │ │ └── columns: str1.a:1!null str1.s:2 + │ │ └── columns: str1.a:1!null str1.s:2 str1.crdb_internal_mvcc_timestamp:3 │ ├── scan str2 - │ │ └── columns: str2.a:3!null str2.s:4 + │ │ └── columns: str2.a:4!null str2.s:5 str2.crdb_internal_mvcc_timestamp:6 │ └── filters - │ └── str1.s:2 = str2.s:4 + │ └── str1.s:2 = str2.s:5 └── projections - └── COALESCE(str1.s:2, str2.s:4) [as=s:5] + └── COALESCE(str1.s:2, str2.s:5) [as=s:7] # Verify that we resolve the merged column a to str2.a but use IFNULL for # column s which is a collated string. @@ -1698,20 +1702,20 @@ build SELECT * FROM str1 RIGHT OUTER JOIN str2 USING(a, s) ---- project - ├── columns: a:3!null s:5 + ├── columns: a:4!null s:7 └── project - ├── columns: s:5 str1.a:1 str1.s:2 str2.a:3!null str2.s:4 + ├── columns: s:7 str1.a:1 str1.s:2 str1.crdb_internal_mvcc_timestamp:3 str2.a:4!null str2.s:5 str2.crdb_internal_mvcc_timestamp:6 ├── right-join (hash) - │ ├── columns: str1.a:1 str1.s:2 str2.a:3!null str2.s:4 + │ ├── columns: str1.a:1 str1.s:2 str1.crdb_internal_mvcc_timestamp:3 str2.a:4!null str2.s:5 str2.crdb_internal_mvcc_timestamp:6 │ ├── scan str1 - │ │ └── columns: str1.a:1!null str1.s:2 + │ │ └── columns: str1.a:1!null str1.s:2 str1.crdb_internal_mvcc_timestamp:3 │ ├── scan str2 - │ │ └── columns: str2.a:3!null str2.s:4 + │ │ └── columns: str2.a:4!null str2.s:5 str2.crdb_internal_mvcc_timestamp:6 │ └── filters - │ ├── str1.a:1 = str2.a:3 - │ └── str1.s:2 = str2.s:4 + │ ├── str1.a:1 = str2.a:4 + │ └── str1.s:2 = str2.s:5 └── projections - └── COALESCE(str1.s:2, str2.s:4) [as=s:5] + └── COALESCE(str1.s:2, str2.s:5) [as=s:7] exec-ddl @@ -1726,18 +1730,18 @@ build SELECT * FROM xyu INNER JOIN xyv USING(x, y) WHERE x > 2 ---- project - ├── columns: x:1!null y:2!null u:3!null v:6!null + ├── columns: x:1!null y:2!null u:3!null v:7!null └── select - ├── columns: xyu.x:1!null xyu.y:2!null u:3!null xyv.x:4!null xyv.y:5!null v:6!null + ├── columns: xyu.x:1!null xyu.y:2!null u:3!null xyu.crdb_internal_mvcc_timestamp:4 xyv.x:5!null xyv.y:6!null v:7!null xyv.crdb_internal_mvcc_timestamp:8 ├── inner-join (hash) - │ ├── columns: xyu.x:1!null xyu.y:2!null u:3!null xyv.x:4!null xyv.y:5!null v:6!null + │ ├── columns: xyu.x:1!null xyu.y:2!null u:3!null xyu.crdb_internal_mvcc_timestamp:4 xyv.x:5!null xyv.y:6!null v:7!null xyv.crdb_internal_mvcc_timestamp:8 │ ├── scan xyu - │ │ └── columns: xyu.x:1!null xyu.y:2!null u:3!null + │ │ └── columns: xyu.x:1!null xyu.y:2!null u:3!null xyu.crdb_internal_mvcc_timestamp:4 │ ├── scan xyv - │ │ └── columns: xyv.x:4!null xyv.y:5!null v:6!null + │ │ └── columns: xyv.x:5!null xyv.y:6!null v:7!null xyv.crdb_internal_mvcc_timestamp:8 │ └── filters - │ ├── xyu.x:1 = xyv.x:4 - │ └── xyu.y:2 = xyv.y:5 + │ ├── xyu.x:1 = xyv.x:5 + │ └── xyu.y:2 = xyv.y:6 └── filters └── xyu.x:1 > 2 @@ -1745,18 +1749,18 @@ build SELECT * FROM xyu LEFT OUTER JOIN xyv USING(x, y) WHERE x > 2 ---- project - ├── columns: x:1!null y:2!null u:3!null v:6 + ├── columns: x:1!null y:2!null u:3!null v:7 └── select - ├── columns: xyu.x:1!null xyu.y:2!null u:3!null xyv.x:4 xyv.y:5 v:6 + ├── columns: xyu.x:1!null xyu.y:2!null u:3!null xyu.crdb_internal_mvcc_timestamp:4 xyv.x:5 xyv.y:6 v:7 xyv.crdb_internal_mvcc_timestamp:8 ├── left-join (hash) - │ ├── columns: xyu.x:1!null xyu.y:2!null u:3!null xyv.x:4 xyv.y:5 v:6 + │ ├── columns: xyu.x:1!null xyu.y:2!null u:3!null xyu.crdb_internal_mvcc_timestamp:4 xyv.x:5 xyv.y:6 v:7 xyv.crdb_internal_mvcc_timestamp:8 │ ├── scan xyu - │ │ └── columns: xyu.x:1!null xyu.y:2!null u:3!null + │ │ └── columns: xyu.x:1!null xyu.y:2!null u:3!null xyu.crdb_internal_mvcc_timestamp:4 │ ├── scan xyv - │ │ └── columns: xyv.x:4!null xyv.y:5!null v:6!null + │ │ └── columns: xyv.x:5!null xyv.y:6!null v:7!null xyv.crdb_internal_mvcc_timestamp:8 │ └── filters - │ ├── xyu.x:1 = xyv.x:4 - │ └── xyu.y:2 = xyv.y:5 + │ ├── xyu.x:1 = xyv.x:5 + │ └── xyu.y:2 = xyv.y:6 └── filters └── xyu.x:1 > 2 @@ -1764,97 +1768,105 @@ build SELECT * FROM xyu RIGHT OUTER JOIN xyv USING(x, y) WHERE x > 2 ---- project - ├── columns: x:4!null y:5!null u:3 v:6!null + ├── columns: x:5!null y:6!null u:3 v:7!null └── select - ├── columns: xyu.x:1 xyu.y:2 u:3 xyv.x:4!null xyv.y:5!null v:6!null + ├── columns: xyu.x:1 xyu.y:2 u:3 xyu.crdb_internal_mvcc_timestamp:4 xyv.x:5!null xyv.y:6!null v:7!null xyv.crdb_internal_mvcc_timestamp:8 ├── right-join (hash) - │ ├── columns: xyu.x:1 xyu.y:2 u:3 xyv.x:4!null xyv.y:5!null v:6!null + │ ├── columns: xyu.x:1 xyu.y:2 u:3 xyu.crdb_internal_mvcc_timestamp:4 xyv.x:5!null xyv.y:6!null v:7!null xyv.crdb_internal_mvcc_timestamp:8 │ ├── scan xyu - │ │ └── columns: xyu.x:1!null xyu.y:2!null u:3!null + │ │ └── columns: xyu.x:1!null xyu.y:2!null u:3!null xyu.crdb_internal_mvcc_timestamp:4 │ ├── scan xyv - │ │ └── columns: xyv.x:4!null xyv.y:5!null v:6!null + │ │ └── columns: xyv.x:5!null xyv.y:6!null v:7!null xyv.crdb_internal_mvcc_timestamp:8 │ └── filters - │ ├── xyu.x:1 = xyv.x:4 - │ └── xyu.y:2 = xyv.y:5 + │ ├── xyu.x:1 = xyv.x:5 + │ └── xyu.y:2 = xyv.y:6 └── filters - └── xyv.x:4 > 2 + └── xyv.x:5 > 2 build SELECT * FROM xyu FULL OUTER JOIN xyv USING(x, y) WHERE x > 2 ---- project - ├── columns: x:7!null y:8 u:3 v:6 + ├── columns: x:9!null y:10 u:3 v:7 └── select - ├── columns: xyu.x:1 xyu.y:2 u:3 xyv.x:4 xyv.y:5 v:6 x:7!null y:8 + ├── columns: xyu.x:1 xyu.y:2 u:3 xyu.crdb_internal_mvcc_timestamp:4 xyv.x:5 xyv.y:6 v:7 xyv.crdb_internal_mvcc_timestamp:8 x:9!null y:10 ├── project - │ ├── columns: x:7 y:8 xyu.x:1 xyu.y:2 u:3 xyv.x:4 xyv.y:5 v:6 + │ ├── columns: x:9 y:10 xyu.x:1 xyu.y:2 u:3 xyu.crdb_internal_mvcc_timestamp:4 xyv.x:5 xyv.y:6 v:7 xyv.crdb_internal_mvcc_timestamp:8 │ ├── full-join (hash) - │ │ ├── columns: xyu.x:1 xyu.y:2 u:3 xyv.x:4 xyv.y:5 v:6 + │ │ ├── columns: xyu.x:1 xyu.y:2 u:3 xyu.crdb_internal_mvcc_timestamp:4 xyv.x:5 xyv.y:6 v:7 xyv.crdb_internal_mvcc_timestamp:8 │ │ ├── scan xyu - │ │ │ └── columns: xyu.x:1!null xyu.y:2!null u:3!null + │ │ │ └── columns: xyu.x:1!null xyu.y:2!null u:3!null xyu.crdb_internal_mvcc_timestamp:4 │ │ ├── scan xyv - │ │ │ └── columns: xyv.x:4!null xyv.y:5!null v:6!null + │ │ │ └── columns: xyv.x:5!null xyv.y:6!null v:7!null xyv.crdb_internal_mvcc_timestamp:8 │ │ └── filters - │ │ ├── xyu.x:1 = xyv.x:4 - │ │ └── xyu.y:2 = xyv.y:5 + │ │ ├── xyu.x:1 = xyv.x:5 + │ │ └── xyu.y:2 = xyv.y:6 │ └── projections - │ ├── COALESCE(xyu.x:1, xyv.x:4) [as=x:7] - │ └── COALESCE(xyu.y:2, xyv.y:5) [as=y:8] + │ ├── COALESCE(xyu.x:1, xyv.x:5) [as=x:9] + │ └── COALESCE(xyu.y:2, xyv.y:6) [as=y:10] └── filters - └── x:7 > 2 + └── x:9 > 2 # Verify that we transfer constraints between the two sides. build SELECT * FROM xyu INNER JOIN xyv ON xyu.x = xyv.x AND xyu.y = xyv.y WHERE xyu.x = 1 AND xyu.y < 10 ---- -select - ├── columns: x:1!null y:2!null u:3!null x:4!null y:5!null v:6!null - ├── inner-join (cross) - │ ├── columns: xyu.x:1!null xyu.y:2!null u:3!null xyv.x:4!null xyv.y:5!null v:6!null - │ ├── scan xyu - │ │ └── columns: xyu.x:1!null xyu.y:2!null u:3!null - │ ├── scan xyv - │ │ └── columns: xyv.x:4!null xyv.y:5!null v:6!null - │ └── filters - │ └── (xyu.x:1 = xyv.x:4) AND (xyu.y:2 = xyv.y:5) - └── filters - └── (xyu.x:1 = 1) AND (xyu.y:2 < 10) +project + ├── columns: x:1!null y:2!null u:3!null x:5!null y:6!null v:7!null + └── select + ├── columns: xyu.x:1!null xyu.y:2!null u:3!null xyu.crdb_internal_mvcc_timestamp:4 xyv.x:5!null xyv.y:6!null v:7!null xyv.crdb_internal_mvcc_timestamp:8 + ├── inner-join (cross) + │ ├── columns: xyu.x:1!null xyu.y:2!null u:3!null xyu.crdb_internal_mvcc_timestamp:4 xyv.x:5!null xyv.y:6!null v:7!null xyv.crdb_internal_mvcc_timestamp:8 + │ ├── scan xyu + │ │ └── columns: xyu.x:1!null xyu.y:2!null u:3!null xyu.crdb_internal_mvcc_timestamp:4 + │ ├── scan xyv + │ │ └── columns: xyv.x:5!null xyv.y:6!null v:7!null xyv.crdb_internal_mvcc_timestamp:8 + │ └── filters + │ └── (xyu.x:1 = xyv.x:5) AND (xyu.y:2 = xyv.y:6) + └── filters + └── (xyu.x:1 = 1) AND (xyu.y:2 < 10) build SELECT * FROM xyu INNER JOIN xyv ON xyu.x = xyv.x AND xyu.y = xyv.y AND xyu.x = 1 AND xyu.y < 10 ---- -inner-join (cross) - ├── columns: x:1!null y:2!null u:3!null x:4!null y:5!null v:6!null - ├── scan xyu - │ └── columns: xyu.x:1!null xyu.y:2!null u:3!null - ├── scan xyv - │ └── columns: xyv.x:4!null xyv.y:5!null v:6!null - └── filters - └── (((xyu.x:1 = xyv.x:4) AND (xyu.y:2 = xyv.y:5)) AND (xyu.x:1 = 1)) AND (xyu.y:2 < 10) +project + ├── columns: x:1!null y:2!null u:3!null x:5!null y:6!null v:7!null + └── inner-join (cross) + ├── columns: xyu.x:1!null xyu.y:2!null u:3!null xyu.crdb_internal_mvcc_timestamp:4 xyv.x:5!null xyv.y:6!null v:7!null xyv.crdb_internal_mvcc_timestamp:8 + ├── scan xyu + │ └── columns: xyu.x:1!null xyu.y:2!null u:3!null xyu.crdb_internal_mvcc_timestamp:4 + ├── scan xyv + │ └── columns: xyv.x:5!null xyv.y:6!null v:7!null xyv.crdb_internal_mvcc_timestamp:8 + └── filters + └── (((xyu.x:1 = xyv.x:5) AND (xyu.y:2 = xyv.y:6)) AND (xyu.x:1 = 1)) AND (xyu.y:2 < 10) build SELECT * FROM xyu LEFT OUTER JOIN xyv ON xyu.x = xyv.x AND xyu.y = xyv.y AND xyu.x = 1 AND xyu.y < 10 ---- -left-join (cross) - ├── columns: x:1!null y:2!null u:3!null x:4 y:5 v:6 - ├── scan xyu - │ └── columns: xyu.x:1!null xyu.y:2!null u:3!null - ├── scan xyv - │ └── columns: xyv.x:4!null xyv.y:5!null v:6!null - └── filters - └── (((xyu.x:1 = xyv.x:4) AND (xyu.y:2 = xyv.y:5)) AND (xyu.x:1 = 1)) AND (xyu.y:2 < 10) +project + ├── columns: x:1!null y:2!null u:3!null x:5 y:6 v:7 + └── left-join (cross) + ├── columns: xyu.x:1!null xyu.y:2!null u:3!null xyu.crdb_internal_mvcc_timestamp:4 xyv.x:5 xyv.y:6 v:7 xyv.crdb_internal_mvcc_timestamp:8 + ├── scan xyu + │ └── columns: xyu.x:1!null xyu.y:2!null u:3!null xyu.crdb_internal_mvcc_timestamp:4 + ├── scan xyv + │ └── columns: xyv.x:5!null xyv.y:6!null v:7!null xyv.crdb_internal_mvcc_timestamp:8 + └── filters + └── (((xyu.x:1 = xyv.x:5) AND (xyu.y:2 = xyv.y:6)) AND (xyu.x:1 = 1)) AND (xyu.y:2 < 10) build SELECT * FROM xyu RIGHT OUTER JOIN xyv ON xyu.x = xyv.x AND xyu.y = xyv.y AND xyu.x = 1 AND xyu.y < 10 ---- -right-join (cross) - ├── columns: x:1 y:2 u:3 x:4!null y:5!null v:6!null - ├── scan xyu - │ └── columns: xyu.x:1!null xyu.y:2!null u:3!null - ├── scan xyv - │ └── columns: xyv.x:4!null xyv.y:5!null v:6!null - └── filters - └── (((xyu.x:1 = xyv.x:4) AND (xyu.y:2 = xyv.y:5)) AND (xyu.x:1 = 1)) AND (xyu.y:2 < 10) +project + ├── columns: x:1 y:2 u:3 x:5!null y:6!null v:7!null + └── right-join (cross) + ├── columns: xyu.x:1 xyu.y:2 u:3 xyu.crdb_internal_mvcc_timestamp:4 xyv.x:5!null xyv.y:6!null v:7!null xyv.crdb_internal_mvcc_timestamp:8 + ├── scan xyu + │ └── columns: xyu.x:1!null xyu.y:2!null u:3!null xyu.crdb_internal_mvcc_timestamp:4 + ├── scan xyv + │ └── columns: xyv.x:5!null xyv.y:6!null v:7!null xyv.crdb_internal_mvcc_timestamp:8 + └── filters + └── (((xyu.x:1 = xyv.x:5) AND (xyu.y:2 = xyv.y:6)) AND (xyu.x:1 = 1)) AND (xyu.y:2 < 10) # Test OUTER joins that are run in the distSQL merge joiner @@ -1863,18 +1875,22 @@ build SELECT * FROM (SELECT * FROM xyu ORDER BY x, y) AS xyu LEFT OUTER JOIN (SELECT * FROM xyv ORDER BY x, y) AS xyv USING(x, y) WHERE x > 2 ---- project - ├── columns: x:1!null y:2!null u:3!null v:6 + ├── columns: x:1!null y:2!null u:3!null v:7 └── select - ├── columns: xyu.x:1!null xyu.y:2!null u:3!null xyv.x:4 xyv.y:5 v:6 + ├── columns: xyu.x:1!null xyu.y:2!null u:3!null xyv.x:5 xyv.y:6 v:7 ├── left-join (hash) - │ ├── columns: xyu.x:1!null xyu.y:2!null u:3!null xyv.x:4 xyv.y:5 v:6 - │ ├── scan xyu - │ │ └── columns: xyu.x:1!null xyu.y:2!null u:3!null - │ ├── scan xyv - │ │ └── columns: xyv.x:4!null xyv.y:5!null v:6!null + │ ├── columns: xyu.x:1!null xyu.y:2!null u:3!null xyv.x:5 xyv.y:6 v:7 + │ ├── project + │ │ ├── columns: xyu.x:1!null xyu.y:2!null u:3!null + │ │ └── scan xyu + │ │ └── columns: xyu.x:1!null xyu.y:2!null u:3!null xyu.crdb_internal_mvcc_timestamp:4 + │ ├── project + │ │ ├── columns: xyv.x:5!null xyv.y:6!null v:7!null + │ │ └── scan xyv + │ │ └── columns: xyv.x:5!null xyv.y:6!null v:7!null xyv.crdb_internal_mvcc_timestamp:8 │ └── filters - │ ├── xyu.x:1 = xyv.x:4 - │ └── xyu.y:2 = xyv.y:5 + │ ├── xyu.x:1 = xyv.x:5 + │ └── xyu.y:2 = xyv.y:6 └── filters └── xyu.x:1 > 2 @@ -1882,86 +1898,102 @@ build SELECT * FROM (SELECT * FROM xyu ORDER BY x, y) AS xyu RIGHT OUTER JOIN (SELECT * FROM xyv ORDER BY x, y) AS xyv USING(x, y) WHERE x > 2 ---- project - ├── columns: x:4!null y:5!null u:3 v:6!null + ├── columns: x:5!null y:6!null u:3 v:7!null └── select - ├── columns: xyu.x:1 xyu.y:2 u:3 xyv.x:4!null xyv.y:5!null v:6!null + ├── columns: xyu.x:1 xyu.y:2 u:3 xyv.x:5!null xyv.y:6!null v:7!null ├── right-join (hash) - │ ├── columns: xyu.x:1 xyu.y:2 u:3 xyv.x:4!null xyv.y:5!null v:6!null - │ ├── scan xyu - │ │ └── columns: xyu.x:1!null xyu.y:2!null u:3!null - │ ├── scan xyv - │ │ └── columns: xyv.x:4!null xyv.y:5!null v:6!null + │ ├── columns: xyu.x:1 xyu.y:2 u:3 xyv.x:5!null xyv.y:6!null v:7!null + │ ├── project + │ │ ├── columns: xyu.x:1!null xyu.y:2!null u:3!null + │ │ └── scan xyu + │ │ └── columns: xyu.x:1!null xyu.y:2!null u:3!null xyu.crdb_internal_mvcc_timestamp:4 + │ ├── project + │ │ ├── columns: xyv.x:5!null xyv.y:6!null v:7!null + │ │ └── scan xyv + │ │ └── columns: xyv.x:5!null xyv.y:6!null v:7!null xyv.crdb_internal_mvcc_timestamp:8 │ └── filters - │ ├── xyu.x:1 = xyv.x:4 - │ └── xyu.y:2 = xyv.y:5 + │ ├── xyu.x:1 = xyv.x:5 + │ └── xyu.y:2 = xyv.y:6 └── filters - └── xyv.x:4 > 2 + └── xyv.x:5 > 2 build SELECT * FROM (SELECT * FROM xyu ORDER BY x, y) AS xyu FULL OUTER JOIN (SELECT * FROM xyv ORDER BY x, y) AS xyv USING(x, y) WHERE x > 2 ---- project - ├── columns: x:7!null y:8 u:3 v:6 + ├── columns: x:9!null y:10 u:3 v:7 └── select - ├── columns: xyu.x:1 xyu.y:2 u:3 xyv.x:4 xyv.y:5 v:6 x:7!null y:8 + ├── columns: xyu.x:1 xyu.y:2 u:3 xyv.x:5 xyv.y:6 v:7 x:9!null y:10 ├── project - │ ├── columns: x:7 y:8 xyu.x:1 xyu.y:2 u:3 xyv.x:4 xyv.y:5 v:6 + │ ├── columns: x:9 y:10 xyu.x:1 xyu.y:2 u:3 xyv.x:5 xyv.y:6 v:7 │ ├── full-join (hash) - │ │ ├── columns: xyu.x:1 xyu.y:2 u:3 xyv.x:4 xyv.y:5 v:6 - │ │ ├── scan xyu - │ │ │ └── columns: xyu.x:1!null xyu.y:2!null u:3!null - │ │ ├── scan xyv - │ │ │ └── columns: xyv.x:4!null xyv.y:5!null v:6!null + │ │ ├── columns: xyu.x:1 xyu.y:2 u:3 xyv.x:5 xyv.y:6 v:7 + │ │ ├── project + │ │ │ ├── columns: xyu.x:1!null xyu.y:2!null u:3!null + │ │ │ └── scan xyu + │ │ │ └── columns: xyu.x:1!null xyu.y:2!null u:3!null xyu.crdb_internal_mvcc_timestamp:4 + │ │ ├── project + │ │ │ ├── columns: xyv.x:5!null xyv.y:6!null v:7!null + │ │ │ └── scan xyv + │ │ │ └── columns: xyv.x:5!null xyv.y:6!null v:7!null xyv.crdb_internal_mvcc_timestamp:8 │ │ └── filters - │ │ ├── xyu.x:1 = xyv.x:4 - │ │ └── xyu.y:2 = xyv.y:5 + │ │ ├── xyu.x:1 = xyv.x:5 + │ │ └── xyu.y:2 = xyv.y:6 │ └── projections - │ ├── COALESCE(xyu.x:1, xyv.x:4) [as=x:7] - │ └── COALESCE(xyu.y:2, xyv.y:5) [as=y:8] + │ ├── COALESCE(xyu.x:1, xyv.x:5) [as=x:9] + │ └── COALESCE(xyu.y:2, xyv.y:6) [as=y:10] └── filters - └── x:7 > 2 + └── x:9 > 2 build SELECT * FROM (SELECT * FROM xyu ORDER BY x, y) AS xyu LEFT OUTER JOIN (SELECT * FROM xyv ORDER BY x, y) AS xyv ON xyu.x = xyv.x AND xyu.y = xyv.y AND xyu.x = 1 AND xyu.y < 10 ---- left-join (cross) - ├── columns: x:1!null y:2!null u:3!null x:4 y:5 v:6 - ├── scan xyu - │ └── columns: xyu.x:1!null xyu.y:2!null u:3!null - ├── scan xyv - │ └── columns: xyv.x:4!null xyv.y:5!null v:6!null + ├── columns: x:1!null y:2!null u:3!null x:5 y:6 v:7 + ├── project + │ ├── columns: xyu.x:1!null xyu.y:2!null u:3!null + │ └── scan xyu + │ └── columns: xyu.x:1!null xyu.y:2!null u:3!null xyu.crdb_internal_mvcc_timestamp:4 + ├── project + │ ├── columns: xyv.x:5!null xyv.y:6!null v:7!null + │ └── scan xyv + │ └── columns: xyv.x:5!null xyv.y:6!null v:7!null xyv.crdb_internal_mvcc_timestamp:8 └── filters - └── (((xyu.x:1 = xyv.x:4) AND (xyu.y:2 = xyv.y:5)) AND (xyu.x:1 = 1)) AND (xyu.y:2 < 10) + └── (((xyu.x:1 = xyv.x:5) AND (xyu.y:2 = xyv.y:6)) AND (xyu.x:1 = 1)) AND (xyu.y:2 < 10) build SELECT * FROM xyu RIGHT OUTER JOIN (SELECT * FROM xyv ORDER BY x, y) AS xyv ON xyu.x = xyv.x AND xyu.y = xyv.y AND xyu.x = 1 AND xyu.y < 10 ---- -right-join (cross) - ├── columns: x:1 y:2 u:3 x:4!null y:5!null v:6!null - ├── scan xyu - │ └── columns: xyu.x:1!null xyu.y:2!null u:3!null - ├── scan xyv - │ └── columns: xyv.x:4!null xyv.y:5!null v:6!null - └── filters - └── (((xyu.x:1 = xyv.x:4) AND (xyu.y:2 = xyv.y:5)) AND (xyu.x:1 = 1)) AND (xyu.y:2 < 10) +project + ├── columns: x:1 y:2 u:3 x:5!null y:6!null v:7!null + └── right-join (cross) + ├── columns: xyu.x:1 xyu.y:2 u:3 xyu.crdb_internal_mvcc_timestamp:4 xyv.x:5!null xyv.y:6!null v:7!null + ├── scan xyu + │ └── columns: xyu.x:1!null xyu.y:2!null u:3!null xyu.crdb_internal_mvcc_timestamp:4 + ├── project + │ ├── columns: xyv.x:5!null xyv.y:6!null v:7!null + │ └── scan xyv + │ └── columns: xyv.x:5!null xyv.y:6!null v:7!null xyv.crdb_internal_mvcc_timestamp:8 + └── filters + └── (((xyu.x:1 = xyv.x:5) AND (xyu.y:2 = xyv.y:6)) AND (xyu.x:1 = 1)) AND (xyu.y:2 < 10) # Regression test for #20472: break up tuple inequalities. build SELECT * FROM xyu JOIN xyv USING(x, y) WHERE (x, y, u) > (1, 2, 3) ---- project - ├── columns: x:1!null y:2!null u:3!null v:6!null + ├── columns: x:1!null y:2!null u:3!null v:7!null └── select - ├── columns: xyu.x:1!null xyu.y:2!null u:3!null xyv.x:4!null xyv.y:5!null v:6!null + ├── columns: xyu.x:1!null xyu.y:2!null u:3!null xyu.crdb_internal_mvcc_timestamp:4 xyv.x:5!null xyv.y:6!null v:7!null xyv.crdb_internal_mvcc_timestamp:8 ├── inner-join (hash) - │ ├── columns: xyu.x:1!null xyu.y:2!null u:3!null xyv.x:4!null xyv.y:5!null v:6!null + │ ├── columns: xyu.x:1!null xyu.y:2!null u:3!null xyu.crdb_internal_mvcc_timestamp:4 xyv.x:5!null xyv.y:6!null v:7!null xyv.crdb_internal_mvcc_timestamp:8 │ ├── scan xyu - │ │ └── columns: xyu.x:1!null xyu.y:2!null u:3!null + │ │ └── columns: xyu.x:1!null xyu.y:2!null u:3!null xyu.crdb_internal_mvcc_timestamp:4 │ ├── scan xyv - │ │ └── columns: xyv.x:4!null xyv.y:5!null v:6!null + │ │ └── columns: xyv.x:5!null xyv.y:6!null v:7!null xyv.crdb_internal_mvcc_timestamp:8 │ └── filters - │ ├── xyu.x:1 = xyv.x:4 - │ └── xyu.y:2 = xyv.y:5 + │ ├── xyu.x:1 = xyv.x:5 + │ └── xyu.y:2 = xyv.y:6 └── filters └── (xyu.x:1, xyu.y:2, u:3) > (1, 2, 3) @@ -1979,34 +2011,38 @@ CREATE TABLE r (a INT PRIMARY KEY) build SELECT * FROM l LEFT OUTER JOIN r ON l.a = r.a WHERE l.a = 3; ---- -select - ├── columns: a:1!null a:2 - ├── left-join (hash) - │ ├── columns: l.a:1!null r.a:2 - │ ├── scan l - │ │ └── columns: l.a:1!null - │ ├── scan r - │ │ └── columns: r.a:2!null - │ └── filters - │ └── l.a:1 = r.a:2 - └── filters - └── l.a:1 = 3 +project + ├── columns: a:1!null a:3 + └── select + ├── columns: l.a:1!null l.crdb_internal_mvcc_timestamp:2 r.a:3 r.crdb_internal_mvcc_timestamp:4 + ├── left-join (hash) + │ ├── columns: l.a:1!null l.crdb_internal_mvcc_timestamp:2 r.a:3 r.crdb_internal_mvcc_timestamp:4 + │ ├── scan l + │ │ └── columns: l.a:1!null l.crdb_internal_mvcc_timestamp:2 + │ ├── scan r + │ │ └── columns: r.a:3!null r.crdb_internal_mvcc_timestamp:4 + │ └── filters + │ └── l.a:1 = r.a:3 + └── filters + └── l.a:1 = 3 build SELECT * FROM l RIGHT OUTER JOIN r ON l.a = r.a WHERE r.a = 3; ---- -select - ├── columns: a:1 a:2!null - ├── right-join (hash) - │ ├── columns: l.a:1 r.a:2!null - │ ├── scan l - │ │ └── columns: l.a:1!null - │ ├── scan r - │ │ └── columns: r.a:2!null - │ └── filters - │ └── l.a:1 = r.a:2 - └── filters - └── r.a:2 = 3 +project + ├── columns: a:1 a:3!null + └── select + ├── columns: l.a:1 l.crdb_internal_mvcc_timestamp:2 r.a:3!null r.crdb_internal_mvcc_timestamp:4 + ├── right-join (hash) + │ ├── columns: l.a:1 l.crdb_internal_mvcc_timestamp:2 r.a:3!null r.crdb_internal_mvcc_timestamp:4 + │ ├── scan l + │ │ └── columns: l.a:1!null l.crdb_internal_mvcc_timestamp:2 + │ ├── scan r + │ │ └── columns: r.a:3!null r.crdb_internal_mvcc_timestamp:4 + │ └── filters + │ └── l.a:1 = r.a:3 + └── filters + └── r.a:3 = 3 build SELECT * FROM l LEFT OUTER JOIN r USING(a) WHERE a = 1 @@ -2014,15 +2050,15 @@ SELECT * FROM l LEFT OUTER JOIN r USING(a) WHERE a = 1 project ├── columns: a:1!null └── select - ├── columns: l.a:1!null r.a:2 + ├── columns: l.a:1!null l.crdb_internal_mvcc_timestamp:2 r.a:3 r.crdb_internal_mvcc_timestamp:4 ├── left-join (hash) - │ ├── columns: l.a:1!null r.a:2 + │ ├── columns: l.a:1!null l.crdb_internal_mvcc_timestamp:2 r.a:3 r.crdb_internal_mvcc_timestamp:4 │ ├── scan l - │ │ └── columns: l.a:1!null + │ │ └── columns: l.a:1!null l.crdb_internal_mvcc_timestamp:2 │ ├── scan r - │ │ └── columns: r.a:2!null + │ │ └── columns: r.a:3!null r.crdb_internal_mvcc_timestamp:4 │ └── filters - │ └── l.a:1 = r.a:2 + │ └── l.a:1 = r.a:3 └── filters └── l.a:1 = 1 @@ -2030,19 +2066,19 @@ build SELECT * FROM l RIGHT OUTER JOIN r USING(a) WHERE a = 3 ---- project - ├── columns: a:2!null + ├── columns: a:3!null └── select - ├── columns: l.a:1 r.a:2!null + ├── columns: l.a:1 l.crdb_internal_mvcc_timestamp:2 r.a:3!null r.crdb_internal_mvcc_timestamp:4 ├── right-join (hash) - │ ├── columns: l.a:1 r.a:2!null + │ ├── columns: l.a:1 l.crdb_internal_mvcc_timestamp:2 r.a:3!null r.crdb_internal_mvcc_timestamp:4 │ ├── scan l - │ │ └── columns: l.a:1!null + │ │ └── columns: l.a:1!null l.crdb_internal_mvcc_timestamp:2 │ ├── scan r - │ │ └── columns: r.a:2!null + │ │ └── columns: r.a:3!null r.crdb_internal_mvcc_timestamp:4 │ └── filters - │ └── l.a:1 = r.a:2 + │ └── l.a:1 = r.a:3 └── filters - └── r.a:2 = 3 + └── r.a:3 = 3 # Regression tests for #21243 exec-ddl @@ -2070,18 +2106,20 @@ build SELECT * FROM abcdef join (select * from abg) USING (a,b) WHERE ((a,b)>(1,2) OR ((a,b)=(1,2) AND c < 6) OR ((a,b,c)=(1,2,6) AND d > 8)) ---- project - ├── columns: a:1!null b:2!null c:3!null d:4!null e:5 f:6 g:9 + ├── columns: a:1!null b:2!null c:3!null d:4!null e:5 f:6 g:10 └── select - ├── columns: abcdef.a:1!null abcdef.b:2!null c:3!null d:4!null e:5 f:6 abg.a:7!null abg.b:8!null g:9 + ├── columns: abcdef.a:1!null abcdef.b:2!null c:3!null d:4!null e:5 f:6 abcdef.crdb_internal_mvcc_timestamp:7 abg.a:8!null abg.b:9!null g:10 ├── inner-join (hash) - │ ├── columns: abcdef.a:1!null abcdef.b:2!null c:3!null d:4!null e:5 f:6 abg.a:7!null abg.b:8!null g:9 + │ ├── columns: abcdef.a:1!null abcdef.b:2!null c:3!null d:4!null e:5 f:6 abcdef.crdb_internal_mvcc_timestamp:7 abg.a:8!null abg.b:9!null g:10 │ ├── scan abcdef - │ │ └── columns: abcdef.a:1!null abcdef.b:2!null c:3!null d:4!null e:5 f:6 - │ ├── scan abg - │ │ └── columns: abg.a:7!null abg.b:8!null g:9 + │ │ └── columns: abcdef.a:1!null abcdef.b:2!null c:3!null d:4!null e:5 f:6 abcdef.crdb_internal_mvcc_timestamp:7 + │ ├── project + │ │ ├── columns: abg.a:8!null abg.b:9!null g:10 + │ │ └── scan abg + │ │ └── columns: abg.a:8!null abg.b:9!null g:10 abg.crdb_internal_mvcc_timestamp:11 │ └── filters - │ ├── abcdef.a:1 = abg.a:7 - │ └── abcdef.b:2 = abg.b:8 + │ ├── abcdef.a:1 = abg.a:8 + │ └── abcdef.b:2 = abg.b:9 └── filters └── (((abcdef.a:1, abcdef.b:2) > (1, 2)) OR (((abcdef.a:1, abcdef.b:2) = (1, 2)) AND (c:3 < 6))) OR (((abcdef.a:1, abcdef.b:2, c:3) = (1, 2, 6)) AND (d:4 > 8)) @@ -2111,149 +2149,149 @@ SELECT * FROM foo NATURAL JOIN bar project ├── columns: a:1!null b:2!null c:3!null d:4!null └── inner-join (hash) - ├── columns: foo.a:1!null foo.b:2!null foo.c:3!null foo.d:4!null foo.rowid:5!null bar.a:6!null bar.b:7!null bar.c:8!null bar.d:9!null bar.rowid:10!null + ├── columns: foo.a:1!null foo.b:2!null foo.c:3!null foo.d:4!null foo.rowid:5!null foo.crdb_internal_mvcc_timestamp:6 bar.a:7!null bar.b:8!null bar.c:9!null bar.d:10!null bar.rowid:11!null bar.crdb_internal_mvcc_timestamp:12 ├── scan foo - │ └── columns: foo.a:1 foo.b:2 foo.c:3 foo.d:4 foo.rowid:5!null + │ └── columns: foo.a:1 foo.b:2 foo.c:3 foo.d:4 foo.rowid:5!null foo.crdb_internal_mvcc_timestamp:6 ├── scan bar - │ └── columns: bar.a:6 bar.b:7 bar.c:8 bar.d:9 bar.rowid:10!null + │ └── columns: bar.a:7 bar.b:8 bar.c:9 bar.d:10 bar.rowid:11!null bar.crdb_internal_mvcc_timestamp:12 └── filters - ├── foo.a:1 = bar.a:6 - ├── foo.b:2 = bar.b:7 - ├── foo.c:3 = bar.c:8 - └── foo.d:4 = bar.d:9 + ├── foo.a:1 = bar.a:7 + ├── foo.b:2 = bar.b:8 + ├── foo.c:3 = bar.c:9 + └── foo.d:4 = bar.d:10 # b can't be an equality column. build SELECT * FROM foo JOIN bar USING (b) ---- project - ├── columns: b:2!null a:1 c:3 d:4 a:6 c:8 d:9 + ├── columns: b:2!null a:1 c:3 d:4 a:7 c:9 d:10 └── inner-join (cross) - ├── columns: foo.a:1 foo.b:2!null foo.c:3 foo.d:4 foo.rowid:5!null bar.a:6 bar.b:7!null bar.c:8 bar.d:9 bar.rowid:10!null + ├── columns: foo.a:1 foo.b:2!null foo.c:3 foo.d:4 foo.rowid:5!null foo.crdb_internal_mvcc_timestamp:6 bar.a:7 bar.b:8!null bar.c:9 bar.d:10 bar.rowid:11!null bar.crdb_internal_mvcc_timestamp:12 ├── scan foo - │ └── columns: foo.a:1 foo.b:2 foo.c:3 foo.d:4 foo.rowid:5!null + │ └── columns: foo.a:1 foo.b:2 foo.c:3 foo.d:4 foo.rowid:5!null foo.crdb_internal_mvcc_timestamp:6 ├── scan bar - │ └── columns: bar.a:6 bar.b:7 bar.c:8 bar.d:9 bar.rowid:10!null + │ └── columns: bar.a:7 bar.b:8 bar.c:9 bar.d:10 bar.rowid:11!null bar.crdb_internal_mvcc_timestamp:12 └── filters - └── foo.b:2 = bar.b:7 + └── foo.b:2 = bar.b:8 # Only a can be an equality column. build SELECT * FROM foo JOIN bar USING (a, b) ---- project - ├── columns: a:1!null b:2!null c:3 d:4 c:8 d:9 + ├── columns: a:1!null b:2!null c:3 d:4 c:9 d:10 └── inner-join (hash) - ├── columns: foo.a:1!null foo.b:2!null foo.c:3 foo.d:4 foo.rowid:5!null bar.a:6!null bar.b:7!null bar.c:8 bar.d:9 bar.rowid:10!null + ├── columns: foo.a:1!null foo.b:2!null foo.c:3 foo.d:4 foo.rowid:5!null foo.crdb_internal_mvcc_timestamp:6 bar.a:7!null bar.b:8!null bar.c:9 bar.d:10 bar.rowid:11!null bar.crdb_internal_mvcc_timestamp:12 ├── scan foo - │ └── columns: foo.a:1 foo.b:2 foo.c:3 foo.d:4 foo.rowid:5!null + │ └── columns: foo.a:1 foo.b:2 foo.c:3 foo.d:4 foo.rowid:5!null foo.crdb_internal_mvcc_timestamp:6 ├── scan bar - │ └── columns: bar.a:6 bar.b:7 bar.c:8 bar.d:9 bar.rowid:10!null + │ └── columns: bar.a:7 bar.b:8 bar.c:9 bar.d:10 bar.rowid:11!null bar.crdb_internal_mvcc_timestamp:12 └── filters - ├── foo.a:1 = bar.a:6 - └── foo.b:2 = bar.b:7 + ├── foo.a:1 = bar.a:7 + └── foo.b:2 = bar.b:8 # Only a and c can be equality columns. build SELECT * FROM foo JOIN bar USING (a, b, c) ---- project - ├── columns: a:1!null b:2!null c:3!null d:4 d:9 + ├── columns: a:1!null b:2!null c:3!null d:4 d:10 └── inner-join (hash) - ├── columns: foo.a:1!null foo.b:2!null foo.c:3!null foo.d:4 foo.rowid:5!null bar.a:6!null bar.b:7!null bar.c:8!null bar.d:9 bar.rowid:10!null + ├── columns: foo.a:1!null foo.b:2!null foo.c:3!null foo.d:4 foo.rowid:5!null foo.crdb_internal_mvcc_timestamp:6 bar.a:7!null bar.b:8!null bar.c:9!null bar.d:10 bar.rowid:11!null bar.crdb_internal_mvcc_timestamp:12 ├── scan foo - │ └── columns: foo.a:1 foo.b:2 foo.c:3 foo.d:4 foo.rowid:5!null + │ └── columns: foo.a:1 foo.b:2 foo.c:3 foo.d:4 foo.rowid:5!null foo.crdb_internal_mvcc_timestamp:6 ├── scan bar - │ └── columns: bar.a:6 bar.b:7 bar.c:8 bar.d:9 bar.rowid:10!null + │ └── columns: bar.a:7 bar.b:8 bar.c:9 bar.d:10 bar.rowid:11!null bar.crdb_internal_mvcc_timestamp:12 └── filters - ├── foo.a:1 = bar.a:6 - ├── foo.b:2 = bar.b:7 - └── foo.c:3 = bar.c:8 + ├── foo.a:1 = bar.a:7 + ├── foo.b:2 = bar.b:8 + └── foo.c:3 = bar.c:9 # b can't be an equality column. build SELECT * FROM foo JOIN bar ON foo.b = bar.b ---- project - ├── columns: a:1 b:2!null c:3 d:4 a:6 b:7!null c:8 d:9 + ├── columns: a:1 b:2!null c:3 d:4 a:7 b:8!null c:9 d:10 └── inner-join (cross) - ├── columns: foo.a:1 foo.b:2!null foo.c:3 foo.d:4 foo.rowid:5!null bar.a:6 bar.b:7!null bar.c:8 bar.d:9 bar.rowid:10!null + ├── columns: foo.a:1 foo.b:2!null foo.c:3 foo.d:4 foo.rowid:5!null foo.crdb_internal_mvcc_timestamp:6 bar.a:7 bar.b:8!null bar.c:9 bar.d:10 bar.rowid:11!null bar.crdb_internal_mvcc_timestamp:12 ├── scan foo - │ └── columns: foo.a:1 foo.b:2 foo.c:3 foo.d:4 foo.rowid:5!null + │ └── columns: foo.a:1 foo.b:2 foo.c:3 foo.d:4 foo.rowid:5!null foo.crdb_internal_mvcc_timestamp:6 ├── scan bar - │ └── columns: bar.a:6 bar.b:7 bar.c:8 bar.d:9 bar.rowid:10!null + │ └── columns: bar.a:7 bar.b:8 bar.c:9 bar.d:10 bar.rowid:11!null bar.crdb_internal_mvcc_timestamp:12 └── filters - └── foo.b:2 = bar.b:7 + └── foo.b:2 = bar.b:8 # Only a can be an equality column. build SELECT * FROM foo JOIN bar ON foo.a = bar.a AND foo.b = bar.b ---- project - ├── columns: a:1!null b:2!null c:3 d:4 a:6!null b:7!null c:8 d:9 + ├── columns: a:1!null b:2!null c:3 d:4 a:7!null b:8!null c:9 d:10 └── inner-join (cross) - ├── columns: foo.a:1!null foo.b:2!null foo.c:3 foo.d:4 foo.rowid:5!null bar.a:6!null bar.b:7!null bar.c:8 bar.d:9 bar.rowid:10!null + ├── columns: foo.a:1!null foo.b:2!null foo.c:3 foo.d:4 foo.rowid:5!null foo.crdb_internal_mvcc_timestamp:6 bar.a:7!null bar.b:8!null bar.c:9 bar.d:10 bar.rowid:11!null bar.crdb_internal_mvcc_timestamp:12 ├── scan foo - │ └── columns: foo.a:1 foo.b:2 foo.c:3 foo.d:4 foo.rowid:5!null + │ └── columns: foo.a:1 foo.b:2 foo.c:3 foo.d:4 foo.rowid:5!null foo.crdb_internal_mvcc_timestamp:6 ├── scan bar - │ └── columns: bar.a:6 bar.b:7 bar.c:8 bar.d:9 bar.rowid:10!null + │ └── columns: bar.a:7 bar.b:8 bar.c:9 bar.d:10 bar.rowid:11!null bar.crdb_internal_mvcc_timestamp:12 └── filters - └── (foo.a:1 = bar.a:6) AND (foo.b:2 = bar.b:7) + └── (foo.a:1 = bar.a:7) AND (foo.b:2 = bar.b:8) build SELECT * FROM foo, bar WHERE foo.b = bar.b ---- project - ├── columns: a:1 b:2!null c:3 d:4 a:6 b:7!null c:8 d:9 + ├── columns: a:1 b:2!null c:3 d:4 a:7 b:8!null c:9 d:10 └── select - ├── columns: foo.a:1 foo.b:2!null foo.c:3 foo.d:4 foo.rowid:5!null bar.a:6 bar.b:7!null bar.c:8 bar.d:9 bar.rowid:10!null + ├── columns: foo.a:1 foo.b:2!null foo.c:3 foo.d:4 foo.rowid:5!null foo.crdb_internal_mvcc_timestamp:6 bar.a:7 bar.b:8!null bar.c:9 bar.d:10 bar.rowid:11!null bar.crdb_internal_mvcc_timestamp:12 ├── inner-join (cross) - │ ├── columns: foo.a:1 foo.b:2 foo.c:3 foo.d:4 foo.rowid:5!null bar.a:6 bar.b:7 bar.c:8 bar.d:9 bar.rowid:10!null + │ ├── columns: foo.a:1 foo.b:2 foo.c:3 foo.d:4 foo.rowid:5!null foo.crdb_internal_mvcc_timestamp:6 bar.a:7 bar.b:8 bar.c:9 bar.d:10 bar.rowid:11!null bar.crdb_internal_mvcc_timestamp:12 │ ├── scan foo - │ │ └── columns: foo.a:1 foo.b:2 foo.c:3 foo.d:4 foo.rowid:5!null + │ │ └── columns: foo.a:1 foo.b:2 foo.c:3 foo.d:4 foo.rowid:5!null foo.crdb_internal_mvcc_timestamp:6 │ ├── scan bar - │ │ └── columns: bar.a:6 bar.b:7 bar.c:8 bar.d:9 bar.rowid:10!null + │ │ └── columns: bar.a:7 bar.b:8 bar.c:9 bar.d:10 bar.rowid:11!null bar.crdb_internal_mvcc_timestamp:12 │ └── filters (true) └── filters - └── foo.b:2 = bar.b:7 + └── foo.b:2 = bar.b:8 # Only a can be an equality column. build SELECT * FROM foo, bar WHERE foo.a = bar.a AND foo.b = bar.b ---- project - ├── columns: a:1!null b:2!null c:3 d:4 a:6!null b:7!null c:8 d:9 + ├── columns: a:1!null b:2!null c:3 d:4 a:7!null b:8!null c:9 d:10 └── select - ├── columns: foo.a:1!null foo.b:2!null foo.c:3 foo.d:4 foo.rowid:5!null bar.a:6!null bar.b:7!null bar.c:8 bar.d:9 bar.rowid:10!null + ├── columns: foo.a:1!null foo.b:2!null foo.c:3 foo.d:4 foo.rowid:5!null foo.crdb_internal_mvcc_timestamp:6 bar.a:7!null bar.b:8!null bar.c:9 bar.d:10 bar.rowid:11!null bar.crdb_internal_mvcc_timestamp:12 ├── inner-join (cross) - │ ├── columns: foo.a:1 foo.b:2 foo.c:3 foo.d:4 foo.rowid:5!null bar.a:6 bar.b:7 bar.c:8 bar.d:9 bar.rowid:10!null + │ ├── columns: foo.a:1 foo.b:2 foo.c:3 foo.d:4 foo.rowid:5!null foo.crdb_internal_mvcc_timestamp:6 bar.a:7 bar.b:8 bar.c:9 bar.d:10 bar.rowid:11!null bar.crdb_internal_mvcc_timestamp:12 │ ├── scan foo - │ │ └── columns: foo.a:1 foo.b:2 foo.c:3 foo.d:4 foo.rowid:5!null + │ │ └── columns: foo.a:1 foo.b:2 foo.c:3 foo.d:4 foo.rowid:5!null foo.crdb_internal_mvcc_timestamp:6 │ ├── scan bar - │ │ └── columns: bar.a:6 bar.b:7 bar.c:8 bar.d:9 bar.rowid:10!null + │ │ └── columns: bar.a:7 bar.b:8 bar.c:9 bar.d:10 bar.rowid:11!null bar.crdb_internal_mvcc_timestamp:12 │ └── filters (true) └── filters - └── (foo.a:1 = bar.a:6) AND (foo.b:2 = bar.b:7) + └── (foo.a:1 = bar.a:7) AND (foo.b:2 = bar.b:8) # Only a and c can be equality columns. build SELECT * FROM foo JOIN bar USING (a, b) WHERE foo.c = bar.c AND foo.d = bar.d ---- project - ├── columns: a:1!null b:2!null c:3!null d:4!null c:8!null d:9!null + ├── columns: a:1!null b:2!null c:3!null d:4!null c:9!null d:10!null └── select - ├── columns: foo.a:1!null foo.b:2!null foo.c:3!null foo.d:4!null foo.rowid:5!null bar.a:6!null bar.b:7!null bar.c:8!null bar.d:9!null bar.rowid:10!null + ├── columns: foo.a:1!null foo.b:2!null foo.c:3!null foo.d:4!null foo.rowid:5!null foo.crdb_internal_mvcc_timestamp:6 bar.a:7!null bar.b:8!null bar.c:9!null bar.d:10!null bar.rowid:11!null bar.crdb_internal_mvcc_timestamp:12 ├── inner-join (hash) - │ ├── columns: foo.a:1!null foo.b:2!null foo.c:3 foo.d:4 foo.rowid:5!null bar.a:6!null bar.b:7!null bar.c:8 bar.d:9 bar.rowid:10!null + │ ├── columns: foo.a:1!null foo.b:2!null foo.c:3 foo.d:4 foo.rowid:5!null foo.crdb_internal_mvcc_timestamp:6 bar.a:7!null bar.b:8!null bar.c:9 bar.d:10 bar.rowid:11!null bar.crdb_internal_mvcc_timestamp:12 │ ├── scan foo - │ │ └── columns: foo.a:1 foo.b:2 foo.c:3 foo.d:4 foo.rowid:5!null + │ │ └── columns: foo.a:1 foo.b:2 foo.c:3 foo.d:4 foo.rowid:5!null foo.crdb_internal_mvcc_timestamp:6 │ ├── scan bar - │ │ └── columns: bar.a:6 bar.b:7 bar.c:8 bar.d:9 bar.rowid:10!null + │ │ └── columns: bar.a:7 bar.b:8 bar.c:9 bar.d:10 bar.rowid:11!null bar.crdb_internal_mvcc_timestamp:12 │ └── filters - │ ├── foo.a:1 = bar.a:6 - │ └── foo.b:2 = bar.b:7 + │ ├── foo.a:1 = bar.a:7 + │ └── foo.b:2 = bar.b:8 └── filters - └── (foo.c:3 = bar.c:8) AND (foo.d:4 = bar.d:9) + └── (foo.c:3 = bar.c:9) AND (foo.d:4 = bar.d:10) exec-ddl CREATE TABLE t.kv ( @@ -2268,17 +2306,17 @@ build SELECT k FROM kv, (SELECT 1 AS k) ---- project - ├── columns: k:5!null + ├── columns: k:6!null └── inner-join (cross) - ├── columns: kv.k:1!null v:2 w:3 s:4 k:5!null + ├── columns: kv.k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 k:6!null ├── scan kv - │ └── columns: kv.k:1!null v:2 w:3 s:4 + │ └── columns: kv.k:1!null v:2 w:3 s:4 crdb_internal_mvcc_timestamp:5 ├── project - │ ├── columns: k:5!null + │ ├── columns: k:6!null │ ├── values │ │ └── () │ └── projections - │ └── 1 [as=k:5] + │ └── 1 [as=k:6] └── filters (true) build @@ -2306,7 +2344,7 @@ select ├── project │ ├── columns: kv.k:3!null │ └── scan kv - │ └── columns: kv.k:3!null v:4 w:5 s:6 + │ └── columns: kv.k:3!null v:4 w:5 s:6 crdb_internal_mvcc_timestamp:7 └── 1 # Test natural outer join when the left side has unknown type @@ -2387,16 +2425,16 @@ build SELECT * FROM onecolumn AS a(x) INNER MERGE JOIN onecolumn AS b(y) ON a.x = b.y ---- project - ├── columns: x:1!null y:3!null + ├── columns: x:1!null y:4!null └── inner-join (hash) - ├── columns: x:1!null a.rowid:2!null y:3!null b.rowid:4!null + ├── columns: x:1!null a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 y:4!null b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 ├── flags: force merge join ├── scan a - │ └── columns: x:1 a.rowid:2!null + │ └── columns: x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: y:3 b.rowid:4!null + │ └── columns: y:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 └── filters - └── x:1 = y:3 + └── x:1 = y:4 build SELECT * FROM onecolumn AS a NATURAL LEFT LOOKUP JOIN onecolumn as b USING(x) @@ -2407,16 +2445,16 @@ build SELECT * FROM onecolumn AS a(x) FULL OUTER HASH JOIN onecolumn AS b(y) ON a.x = b.y ---- project - ├── columns: x:1 y:3 + ├── columns: x:1 y:4 └── full-join (hash) - ├── columns: x:1 a.rowid:2 y:3 b.rowid:4 + ├── columns: x:1 a.rowid:2 a.crdb_internal_mvcc_timestamp:3 y:4 b.rowid:5 b.crdb_internal_mvcc_timestamp:6 ├── flags: force hash join (store right side) ├── scan a - │ └── columns: x:1 a.rowid:2!null + │ └── columns: x:1 a.rowid:2!null a.crdb_internal_mvcc_timestamp:3 ├── scan b - │ └── columns: y:3 b.rowid:4!null + │ └── columns: y:4 b.rowid:5!null b.crdb_internal_mvcc_timestamp:6 └── filters - └── x:1 = y:3 + └── x:1 = y:4 # Regression test for #46403. exec-ddl @@ -2433,37 +2471,37 @@ SELECT * FROM v0 NATURAL JOIN t0 project ├── columns: c0:1!null c1:1!null └── inner-join (hash) - ├── columns: c0:1!null c0:3!null rowid:4!null + ├── columns: c0:1!null c0:4!null rowid:5!null crdb_internal_mvcc_timestamp:6 ├── distinct-on │ ├── columns: c0:1 │ ├── grouping columns: c0:1 │ └── project │ ├── columns: c0:1 │ └── scan t0 - │ └── columns: c0:1 rowid:2!null + │ └── columns: c0:1 rowid:2!null crdb_internal_mvcc_timestamp:3 ├── scan t0 - │ └── columns: c0:3 rowid:4!null + │ └── columns: c0:4 rowid:5!null crdb_internal_mvcc_timestamp:6 └── filters - └── c0:1 = c0:3 + └── c0:1 = c0:4 build SELECT * FROM t0 NATURAL JOIN v0 ---- project - ├── columns: c0:1!null c1:3!null + ├── columns: c0:1!null c1:4!null └── inner-join (hash) - ├── columns: c0:1!null rowid:2!null c0:3!null + ├── columns: c0:1!null rowid:2!null crdb_internal_mvcc_timestamp:3 c0:4!null ├── scan t0 - │ └── columns: c0:1 rowid:2!null + │ └── columns: c0:1 rowid:2!null crdb_internal_mvcc_timestamp:3 ├── distinct-on - │ ├── columns: c0:3 - │ ├── grouping columns: c0:3 + │ ├── columns: c0:4 + │ ├── grouping columns: c0:4 │ └── project - │ ├── columns: c0:3 + │ ├── columns: c0:4 │ └── scan t0 - │ └── columns: c0:3 rowid:4!null + │ └── columns: c0:4 rowid:5!null crdb_internal_mvcc_timestamp:6 └── filters - └── c0:1 = c0:3 + └── c0:1 = c0:4 build SELECT * FROM v0 NATURAL JOIN v0 AS v1 @@ -2471,24 +2509,24 @@ SELECT * FROM v0 NATURAL JOIN v0 AS v1 project ├── columns: c0:1!null c1:1!null └── inner-join (hash) - ├── columns: c0:1!null c0:3!null + ├── columns: c0:1!null c0:4!null ├── distinct-on │ ├── columns: c0:1 │ ├── grouping columns: c0:1 │ └── project │ ├── columns: c0:1 │ └── scan t0 - │ └── columns: c0:1 rowid:2!null + │ └── columns: c0:1 rowid:2!null crdb_internal_mvcc_timestamp:3 ├── distinct-on - │ ├── columns: c0:3 - │ ├── grouping columns: c0:3 + │ ├── columns: c0:4 + │ ├── grouping columns: c0:4 │ └── project - │ ├── columns: c0:3 + │ ├── columns: c0:4 │ └── scan t0 - │ └── columns: c0:3 rowid:4!null + │ └── columns: c0:4 rowid:5!null crdb_internal_mvcc_timestamp:6 └── filters - ├── c0:1 = c0:3 - └── c0:1 = c0:3 + ├── c0:1 = c0:4 + └── c0:1 = c0:4 build SELECT * FROM v0 NATURAL LEFT JOIN v0 AS v1 @@ -2496,79 +2534,79 @@ SELECT * FROM v0 NATURAL LEFT JOIN v0 AS v1 project ├── columns: c0:1 c1:1 └── left-join (hash) - ├── columns: c0:1 c0:3 + ├── columns: c0:1 c0:4 ├── distinct-on │ ├── columns: c0:1 │ ├── grouping columns: c0:1 │ └── project │ ├── columns: c0:1 │ └── scan t0 - │ └── columns: c0:1 rowid:2!null + │ └── columns: c0:1 rowid:2!null crdb_internal_mvcc_timestamp:3 ├── distinct-on - │ ├── columns: c0:3 - │ ├── grouping columns: c0:3 + │ ├── columns: c0:4 + │ ├── grouping columns: c0:4 │ └── project - │ ├── columns: c0:3 + │ ├── columns: c0:4 │ └── scan t0 - │ └── columns: c0:3 rowid:4!null + │ └── columns: c0:4 rowid:5!null crdb_internal_mvcc_timestamp:6 └── filters - ├── c0:1 = c0:3 - └── c0:1 = c0:3 + ├── c0:1 = c0:4 + └── c0:1 = c0:4 build SELECT * FROM v0 NATURAL RIGHT JOIN v0 AS v1 ---- project - ├── columns: c0:3 c1:3 + ├── columns: c0:4 c1:4 └── right-join (hash) - ├── columns: c0:1 c0:3 + ├── columns: c0:1 c0:4 ├── distinct-on │ ├── columns: c0:1 │ ├── grouping columns: c0:1 │ └── project │ ├── columns: c0:1 │ └── scan t0 - │ └── columns: c0:1 rowid:2!null + │ └── columns: c0:1 rowid:2!null crdb_internal_mvcc_timestamp:3 ├── distinct-on - │ ├── columns: c0:3 - │ ├── grouping columns: c0:3 + │ ├── columns: c0:4 + │ ├── grouping columns: c0:4 │ └── project - │ ├── columns: c0:3 + │ ├── columns: c0:4 │ └── scan t0 - │ └── columns: c0:3 rowid:4!null + │ └── columns: c0:4 rowid:5!null crdb_internal_mvcc_timestamp:6 └── filters - ├── c0:1 = c0:3 - └── c0:1 = c0:3 + ├── c0:1 = c0:4 + └── c0:1 = c0:4 build SELECT * FROM v0 NATURAL FULL OUTER JOIN v0 AS v1 ---- project - ├── columns: c0:5 c1:6 + ├── columns: c0:7 c1:8 └── project - ├── columns: c0:5 c1:6 t0.c0:1 t0.c0:3 + ├── columns: c0:7 c1:8 t0.c0:1 t0.c0:4 ├── full-join (hash) - │ ├── columns: t0.c0:1 t0.c0:3 + │ ├── columns: t0.c0:1 t0.c0:4 │ ├── distinct-on │ │ ├── columns: t0.c0:1 │ │ ├── grouping columns: t0.c0:1 │ │ └── project │ │ ├── columns: t0.c0:1 │ │ └── scan t0 - │ │ └── columns: t0.c0:1 rowid:2!null + │ │ └── columns: t0.c0:1 rowid:2!null crdb_internal_mvcc_timestamp:3 │ ├── distinct-on - │ │ ├── columns: t0.c0:3 - │ │ ├── grouping columns: t0.c0:3 + │ │ ├── columns: t0.c0:4 + │ │ ├── grouping columns: t0.c0:4 │ │ └── project - │ │ ├── columns: t0.c0:3 + │ │ ├── columns: t0.c0:4 │ │ └── scan t0 - │ │ └── columns: t0.c0:3 rowid:4!null + │ │ └── columns: t0.c0:4 rowid:5!null crdb_internal_mvcc_timestamp:6 │ └── filters - │ ├── t0.c0:1 = t0.c0:3 - │ └── t0.c0:1 = t0.c0:3 + │ ├── t0.c0:1 = t0.c0:4 + │ └── t0.c0:1 = t0.c0:4 └── projections - ├── COALESCE(t0.c0:1, t0.c0:3) [as=c0:5] - └── COALESCE(t0.c0:1, t0.c0:3) [as=c1:6] + ├── COALESCE(t0.c0:1, t0.c0:4) [as=c0:7] + └── COALESCE(t0.c0:1, t0.c0:4) [as=c1:8] build SELECT * FROM (SELECT DISTINCT c0, c0 FROM t0) AS v1(c0, c1) NATURAL JOIN t0 @@ -2576,59 +2614,59 @@ SELECT * FROM (SELECT DISTINCT c0, c0 FROM t0) AS v1(c0, c1) NATURAL JOIN t0 project ├── columns: c0:1!null c1:1!null └── inner-join (hash) - ├── columns: c0:1!null c0:3!null rowid:4!null + ├── columns: c0:1!null c0:4!null rowid:5!null crdb_internal_mvcc_timestamp:6 ├── distinct-on │ ├── columns: c0:1 │ ├── grouping columns: c0:1 │ └── project │ ├── columns: c0:1 │ └── scan t0 - │ └── columns: c0:1 rowid:2!null + │ └── columns: c0:1 rowid:2!null crdb_internal_mvcc_timestamp:3 ├── scan t0 - │ └── columns: c0:3 rowid:4!null + │ └── columns: c0:4 rowid:5!null crdb_internal_mvcc_timestamp:6 └── filters - └── c0:1 = c0:3 + └── c0:1 = c0:4 build SELECT * FROM v0 JOIN v0 AS v1 USING (c0) ---- inner-join (hash) - ├── columns: c0:1!null c1:1!null c1:3!null + ├── columns: c0:1!null c1:1!null c1:4!null ├── distinct-on │ ├── columns: c0:1 │ ├── grouping columns: c0:1 │ └── project │ ├── columns: c0:1 │ └── scan t0 - │ └── columns: c0:1 rowid:2!null + │ └── columns: c0:1 rowid:2!null crdb_internal_mvcc_timestamp:3 ├── distinct-on - │ ├── columns: c0:3 - │ ├── grouping columns: c0:3 + │ ├── columns: c0:4 + │ ├── grouping columns: c0:4 │ └── project - │ ├── columns: c0:3 + │ ├── columns: c0:4 │ └── scan t0 - │ └── columns: c0:3 rowid:4!null + │ └── columns: c0:4 rowid:5!null crdb_internal_mvcc_timestamp:6 └── filters - └── c0:1 = c0:3 + └── c0:1 = c0:4 build SELECT * FROM v0 JOIN v0 AS v1 USING (c1) ---- inner-join (hash) - ├── columns: c1:1!null c0:1!null c0:3!null + ├── columns: c1:1!null c0:1!null c0:4!null ├── distinct-on │ ├── columns: c0:1 │ ├── grouping columns: c0:1 │ └── project │ ├── columns: c0:1 │ └── scan t0 - │ └── columns: c0:1 rowid:2!null + │ └── columns: c0:1 rowid:2!null crdb_internal_mvcc_timestamp:3 ├── distinct-on - │ ├── columns: c0:3 - │ ├── grouping columns: c0:3 + │ ├── columns: c0:4 + │ ├── grouping columns: c0:4 │ └── project - │ ├── columns: c0:3 + │ ├── columns: c0:4 │ └── scan t0 - │ └── columns: c0:3 rowid:4!null + │ └── columns: c0:4 rowid:5!null crdb_internal_mvcc_timestamp:6 └── filters - └── c0:1 = c0:3 + └── c0:1 = c0:4 diff --git a/pkg/sql/opt/optbuilder/testdata/lateral b/pkg/sql/opt/optbuilder/testdata/lateral index 14d25b7d55fc..16bfe70547a8 100644 --- a/pkg/sql/opt/optbuilder/testdata/lateral +++ b/pkg/sql/opt/optbuilder/testdata/lateral @@ -13,33 +13,39 @@ CREATE TABLE z (c INT PRIMARY KEY) build SELECT * FROM x, y, z ---- -inner-join (cross) - ├── columns: a:1!null b:2!null c:3!null - ├── scan x - │ └── columns: a:1!null - ├── inner-join (cross) - │ ├── columns: b:2!null c:3!null - │ ├── scan y - │ │ └── columns: b:2!null - │ ├── scan z - │ │ └── columns: c:3!null - │ └── filters (true) - └── filters (true) +project + ├── columns: a:1!null b:3!null c:5!null + └── inner-join (cross) + ├── columns: a:1!null x.crdb_internal_mvcc_timestamp:2 b:3!null y.crdb_internal_mvcc_timestamp:4 c:5!null z.crdb_internal_mvcc_timestamp:6 + ├── scan x + │ └── columns: a:1!null x.crdb_internal_mvcc_timestamp:2 + ├── inner-join (cross) + │ ├── columns: b:3!null y.crdb_internal_mvcc_timestamp:4 c:5!null z.crdb_internal_mvcc_timestamp:6 + │ ├── scan y + │ │ └── columns: b:3!null y.crdb_internal_mvcc_timestamp:4 + │ ├── scan z + │ │ └── columns: c:5!null z.crdb_internal_mvcc_timestamp:6 + │ └── filters (true) + └── filters (true) build SELECT * FROM x, LATERAL (SELECT * FROM y WHERE b = a) ---- -inner-join-apply - ├── columns: a:1!null b:2!null - ├── scan x - │ └── columns: a:1!null - ├── select - │ ├── columns: b:2!null - │ ├── scan y - │ │ └── columns: b:2!null - │ └── filters - │ └── b:2 = a:1 - └── filters (true) +project + ├── columns: a:1!null b:3!null + └── inner-join-apply + ├── columns: a:1!null x.crdb_internal_mvcc_timestamp:2 b:3!null + ├── scan x + │ └── columns: a:1!null x.crdb_internal_mvcc_timestamp:2 + ├── project + │ ├── columns: b:3!null + │ └── select + │ ├── columns: b:3!null y.crdb_internal_mvcc_timestamp:4 + │ ├── scan y + │ │ └── columns: b:3!null y.crdb_internal_mvcc_timestamp:4 + │ └── filters + │ └── b:3 = a:1 + └── filters (true) build SELECT * FROM x, (SELECT * FROM y WHERE b = a) @@ -54,25 +60,29 @@ SELECT FROM x ---- project - ├── columns: b:4 + ├── columns: b:7 ├── scan x - │ └── columns: a:1!null + │ └── columns: a:1!null x.crdb_internal_mvcc_timestamp:2 └── projections - └── subquery [as=b:4] + └── subquery [as=b:7] └── max1-row - ├── columns: y.b:3!null + ├── columns: y.b:5!null └── project - ├── columns: y.b:3!null + ├── columns: y.b:5!null └── inner-join-apply - ├── columns: c:2!null y.b:3!null - ├── scan z - │ └── columns: c:2!null - ├── select - │ ├── columns: y.b:3!null - │ ├── scan y - │ │ └── columns: y.b:3!null - │ └── filters - │ └── y.b:3 = c:2 + ├── columns: c:3!null y.b:5!null + ├── project + │ ├── columns: c:3!null + │ └── scan z + │ └── columns: c:3!null z.crdb_internal_mvcc_timestamp:4 + ├── project + │ ├── columns: y.b:5!null + │ └── select + │ ├── columns: y.b:5!null y.crdb_internal_mvcc_timestamp:6 + │ ├── scan y + │ │ └── columns: y.b:5!null y.crdb_internal_mvcc_timestamp:6 + │ └── filters + │ └── y.b:5 = c:3 └── filters (true) build @@ -81,67 +91,81 @@ SELECT FROM x ---- project - ├── columns: b:4 + ├── columns: b:7 ├── scan x - │ └── columns: a:1!null + │ └── columns: a:1!null x.crdb_internal_mvcc_timestamp:2 └── projections - └── subquery [as=b:4] + └── subquery [as=b:7] └── max1-row - ├── columns: y.b:3!null + ├── columns: y.b:5!null └── project - ├── columns: y.b:3!null + ├── columns: y.b:5!null └── inner-join (cross) - ├── columns: c:2!null y.b:3!null - ├── scan z - │ └── columns: c:2!null - ├── select - │ ├── columns: y.b:3!null - │ ├── scan y - │ │ └── columns: y.b:3!null - │ └── filters - │ └── y.b:3 = a:1 + ├── columns: c:3!null y.b:5!null + ├── project + │ ├── columns: c:3!null + │ └── scan z + │ └── columns: c:3!null z.crdb_internal_mvcc_timestamp:4 + ├── project + │ ├── columns: y.b:5!null + │ └── select + │ ├── columns: y.b:5!null y.crdb_internal_mvcc_timestamp:6 + │ ├── scan y + │ │ └── columns: y.b:5!null y.crdb_internal_mvcc_timestamp:6 + │ └── filters + │ └── y.b:5 = a:1 └── filters (true) build SELECT * FROM x AS o WHERE EXISTS(SELECT * FROM x, LATERAL (SELECT * FROM y WHERE b = x.a AND o.a = x.a)) ---- -select +project ├── columns: a:1!null - ├── scan o - │ └── columns: o.a:1!null - └── filters - └── exists - └── inner-join-apply - ├── columns: x.a:2!null b:3!null - ├── scan x - │ └── columns: x.a:2!null - ├── select - │ ├── columns: b:3!null - │ ├── scan y - │ │ └── columns: b:3!null - │ └── filters - │ └── (b:3 = x.a:2) AND (o.a:1 = x.a:2) - └── filters (true) + └── select + ├── columns: o.a:1!null o.crdb_internal_mvcc_timestamp:2 + ├── scan o + │ └── columns: o.a:1!null o.crdb_internal_mvcc_timestamp:2 + └── filters + └── exists + └── project + ├── columns: x.a:3!null b:5!null + └── inner-join-apply + ├── columns: x.a:3!null x.crdb_internal_mvcc_timestamp:4 b:5!null + ├── scan x + │ └── columns: x.a:3!null x.crdb_internal_mvcc_timestamp:4 + ├── project + │ ├── columns: b:5!null + │ └── select + │ ├── columns: b:5!null y.crdb_internal_mvcc_timestamp:6 + │ ├── scan y + │ │ └── columns: b:5!null y.crdb_internal_mvcc_timestamp:6 + │ └── filters + │ └── (b:5 = x.a:3) AND (o.a:1 = x.a:3) + └── filters (true) build SELECT * FROM x, LATERAL (SELECT * FROM y WHERE b = a), z ---- -inner-join-apply - ├── columns: a:1!null b:2!null c:3!null - ├── inner-join-apply - │ ├── columns: a:1!null b:2!null - │ ├── scan x - │ │ └── columns: a:1!null - │ ├── select - │ │ ├── columns: b:2!null - │ │ ├── scan y - │ │ │ └── columns: b:2!null - │ │ └── filters - │ │ └── b:2 = a:1 - │ └── filters (true) - ├── scan z - │ └── columns: c:3!null - └── filters (true) +project + ├── columns: a:1!null b:3!null c:5!null + └── inner-join-apply + ├── columns: a:1!null x.crdb_internal_mvcc_timestamp:2 b:3!null c:5!null z.crdb_internal_mvcc_timestamp:6 + ├── inner-join-apply + │ ├── columns: a:1!null x.crdb_internal_mvcc_timestamp:2 b:3!null + │ ├── scan x + │ │ └── columns: a:1!null x.crdb_internal_mvcc_timestamp:2 + │ ├── project + │ │ ├── columns: b:3!null + │ │ └── select + │ │ ├── columns: b:3!null y.crdb_internal_mvcc_timestamp:4 + │ │ ├── scan y + │ │ │ └── columns: b:3!null y.crdb_internal_mvcc_timestamp:4 + │ │ └── filters + │ │ └── b:3 = a:1 + │ └── filters (true) + ├── scan z + │ └── columns: c:5!null z.crdb_internal_mvcc_timestamp:6 + └── filters (true) build SELECT * FROM x, LATERAL (SELECT * FROM y WHERE b = a), x @@ -158,60 +182,66 @@ CREATE TABLE j ( build SELECT * FROM j, jsonb_array_elements(j.j->'foo') ---- -inner-join-apply - ├── columns: id:1!null j:2 value:3 - ├── scan j - │ └── columns: id:1!null j:2 - ├── project-set - │ ├── columns: value:3 - │ ├── values - │ │ └── () - │ └── zip - │ └── jsonb_array_elements(j:2->'foo') - └── filters (true) +project + ├── columns: id:1!null j:2 value:4 + └── inner-join-apply + ├── columns: id:1!null j:2 crdb_internal_mvcc_timestamp:3 value:4 + ├── scan j + │ └── columns: id:1!null j:2 crdb_internal_mvcc_timestamp:3 + ├── project-set + │ ├── columns: value:4 + │ ├── values + │ │ └── () + │ └── zip + │ └── jsonb_array_elements(j:2->'foo') + └── filters (true) # Explicit LATERAL makes no difference for SRFs. build SELECT * FROM j, LATERAL jsonb_array_elements(j.j->'foo') ---- -inner-join-apply - ├── columns: id:1!null j:2 value:3 - ├── scan j - │ └── columns: id:1!null j:2 - ├── project-set - │ ├── columns: value:3 - │ ├── values - │ │ └── () - │ └── zip - │ └── jsonb_array_elements(j:2->'foo') - └── filters (true) +project + ├── columns: id:1!null j:2 value:4 + └── inner-join-apply + ├── columns: id:1!null j:2 crdb_internal_mvcc_timestamp:3 value:4 + ├── scan j + │ └── columns: id:1!null j:2 crdb_internal_mvcc_timestamp:3 + ├── project-set + │ ├── columns: value:4 + │ ├── values + │ │ └── () + │ └── zip + │ └── jsonb_array_elements(j:2->'foo') + └── filters (true) # Referencing a lateral SRF from a lateral SRF. build SELECT * FROM j, jsonb_array_elements(j.j->'foo') AS e, jsonb_array_elements(e.e->'bar') ---- -inner-join-apply - ├── columns: id:1!null j:2 e:3 value:4 - ├── inner-join-apply - │ ├── columns: id:1!null j:2 value:3 - │ ├── scan j - │ │ └── columns: id:1!null j:2 - │ ├── project-set - │ │ ├── columns: value:3 - │ │ ├── values - │ │ │ └── () - │ │ └── zip - │ │ └── jsonb_array_elements(j:2->'foo') - │ └── filters (true) - ├── project-set - │ ├── columns: value:4 - │ ├── values - │ │ └── () - │ └── zip - │ └── jsonb_array_elements(value:3->'bar') - └── filters (true) +project + ├── columns: id:1!null j:2 e:4 value:5 + └── inner-join-apply + ├── columns: id:1!null j:2 crdb_internal_mvcc_timestamp:3 value:4 value:5 + ├── inner-join-apply + │ ├── columns: id:1!null j:2 crdb_internal_mvcc_timestamp:3 value:4 + │ ├── scan j + │ │ └── columns: id:1!null j:2 crdb_internal_mvcc_timestamp:3 + │ ├── project-set + │ │ ├── columns: value:4 + │ │ ├── values + │ │ │ └── () + │ │ └── zip + │ │ └── jsonb_array_elements(j:2->'foo') + │ └── filters (true) + ├── project-set + │ ├── columns: value:5 + │ ├── values + │ │ └── () + │ └── zip + │ └── jsonb_array_elements(value:4->'bar') + └── filters (true) build SELECT @@ -229,62 +259,72 @@ FROM ) ) ---- -inner-join-apply - ├── columns: id:1!null j:2 value:6 - ├── scan j - │ └── columns: j.id:1!null j.j:2 - ├── project-set - │ ├── columns: value:6 - │ ├── values - │ │ └── () - │ └── zip - │ └── function: jsonb_array_elements - │ └── subquery - │ └── max1-row - │ ├── columns: members:5 - │ └── project - │ ├── columns: members:5 - │ ├── select - │ │ ├── columns: j2.id:3!null j2.j:4 - │ │ ├── scan j2 - │ │ │ └── columns: j2.id:3!null j2.j:4 - │ │ └── filters - │ │ └── (j2.j:4->>'name') = (j.j:2->>'name') - │ └── projections - │ └── j2.j:4->'members' [as=members:5] - └── filters (true) +project + ├── columns: id:1!null j:2 value:8 + └── inner-join-apply + ├── columns: j.id:1!null j.j:2 j.crdb_internal_mvcc_timestamp:3 value:8 + ├── scan j + │ └── columns: j.id:1!null j.j:2 j.crdb_internal_mvcc_timestamp:3 + ├── project-set + │ ├── columns: value:8 + │ ├── values + │ │ └── () + │ └── zip + │ └── function: jsonb_array_elements + │ └── subquery + │ └── max1-row + │ ├── columns: members:7 + │ └── project + │ ├── columns: members:7 + │ ├── select + │ │ ├── columns: j2.id:4!null j2.j:5 j2.crdb_internal_mvcc_timestamp:6 + │ │ ├── scan j2 + │ │ │ └── columns: j2.id:4!null j2.j:5 j2.crdb_internal_mvcc_timestamp:6 + │ │ └── filters + │ │ └── (j2.j:5->>'name') = (j.j:2->>'name') + │ └── projections + │ └── j2.j:5->'members' [as=members:7] + └── filters (true) build SELECT * FROM x JOIN LATERAL (SELECT * FROM y WHERE b = x.a) ON true ---- -inner-join-apply - ├── columns: a:1!null b:2!null - ├── scan x - │ └── columns: a:1!null - ├── select - │ ├── columns: b:2!null - │ ├── scan y - │ │ └── columns: b:2!null - │ └── filters - │ └── b:2 = a:1 - └── filters - └── true +project + ├── columns: a:1!null b:3!null + └── inner-join-apply + ├── columns: a:1!null x.crdb_internal_mvcc_timestamp:2 b:3!null + ├── scan x + │ └── columns: a:1!null x.crdb_internal_mvcc_timestamp:2 + ├── project + │ ├── columns: b:3!null + │ └── select + │ ├── columns: b:3!null y.crdb_internal_mvcc_timestamp:4 + │ ├── scan y + │ │ └── columns: b:3!null y.crdb_internal_mvcc_timestamp:4 + │ └── filters + │ └── b:3 = a:1 + └── filters + └── true build SELECT * FROM x LEFT JOIN LATERAL (SELECT * FROM y WHERE b = x.a) ON true ---- -left-join-apply - ├── columns: a:1!null b:2 - ├── scan x - │ └── columns: a:1!null - ├── select - │ ├── columns: b:2!null - │ ├── scan y - │ │ └── columns: b:2!null - │ └── filters - │ └── b:2 = a:1 - └── filters - └── true +project + ├── columns: a:1!null b:3 + └── left-join-apply + ├── columns: a:1!null x.crdb_internal_mvcc_timestamp:2 b:3 + ├── scan x + │ └── columns: a:1!null x.crdb_internal_mvcc_timestamp:2 + ├── project + │ ├── columns: b:3!null + │ └── select + │ ├── columns: b:3!null y.crdb_internal_mvcc_timestamp:4 + │ ├── scan y + │ │ └── columns: b:3!null y.crdb_internal_mvcc_timestamp:4 + │ └── filters + │ └── b:3 = a:1 + └── filters + └── true build SELECT * FROM x RIGHT JOIN LATERAL (SELECT * FROM y WHERE b = x.a) ON true diff --git a/pkg/sql/opt/optbuilder/testdata/limit b/pkg/sql/opt/optbuilder/testdata/limit index 4366b7ceb4ed..28536177bfcb 100644 --- a/pkg/sql/opt/optbuilder/testdata/limit +++ b/pkg/sql/opt/optbuilder/testdata/limit @@ -16,7 +16,7 @@ limit │ ├── ordering: +1 │ ├── limit hint: 5.00 │ └── scan t - │ ├── columns: k:1!null v:2 w:3 + │ ├── columns: k:1!null v:2 w:3 crdb_internal_mvcc_timestamp:4 │ ├── ordering: +1 │ └── limit hint: 5.00 └── 5 @@ -35,7 +35,7 @@ limit │ └── project │ ├── columns: k:1!null v:2 │ └── scan t - │ └── columns: k:1!null v:2 w:3 + │ └── columns: k:1!null v:2 w:3 crdb_internal_mvcc_timestamp:4 └── 5 build @@ -46,7 +46,7 @@ limit ├── project │ ├── columns: k:1!null v:2 │ └── scan t - │ └── columns: k:1!null v:2 w:3 + │ └── columns: k:1!null v:2 w:3 crdb_internal_mvcc_timestamp:4 └── 1 + 2 build @@ -61,7 +61,7 @@ limit │ ├── ordering: +1 │ ├── limit hint: 1.00 │ └── scan t - │ ├── columns: k:1!null v:2 w:3 + │ ├── columns: k:1!null v:2 w:3 crdb_internal_mvcc_timestamp:4 │ ├── ordering: +1 │ └── limit hint: 1.00 └── 1 @@ -83,7 +83,7 @@ limit │ │ ├── ordering: +1 │ │ ├── limit hint: 4.00 │ │ └── scan t - │ │ ├── columns: k:1!null v:2 w:3 + │ │ ├── columns: k:1!null v:2 w:3 crdb_internal_mvcc_timestamp:4 │ │ ├── ordering: +1 │ │ └── limit hint: 4.00 │ └── 3 @@ -100,7 +100,7 @@ offset │ ├── columns: k:1!null v:2 │ ├── ordering: +1 │ └── scan t - │ ├── columns: k:1!null v:2 w:3 + │ ├── columns: k:1!null v:2 w:3 crdb_internal_mvcc_timestamp:4 │ └── ordering: +1 └── 5 @@ -115,7 +115,7 @@ limit │ ├── columns: k:1!null │ ├── ordering: +1 │ └── scan t - │ ├── columns: k:1!null v:2 w:3 + │ ├── columns: k:1!null v:2 w:3 crdb_internal_mvcc_timestamp:4 │ └── ordering: +1 └── 1 + 1 @@ -133,25 +133,27 @@ build SELECT sum(w) FROM t GROUP BY k, v ORDER BY v DESC LIMIT 10 ---- limit - ├── columns: sum:4 [hidden: v:2] + ├── columns: sum:5 [hidden: v:2] ├── internal-ordering: -2 ├── ordering: -2 ├── project - │ ├── columns: v:2 sum:4 + │ ├── columns: v:2 sum:5 │ ├── ordering: -2 │ ├── limit hint: 10.00 │ └── group-by - │ ├── columns: k:1!null v:2 sum:4 + │ ├── columns: k:1!null v:2 sum:5 │ ├── grouping columns: k:1!null v:2 │ ├── ordering: -2 │ ├── limit hint: 10.00 │ ├── sort │ │ ├── columns: k:1!null v:2 w:3 │ │ ├── ordering: -2 - │ │ └── scan t - │ │ └── columns: k:1!null v:2 w:3 + │ │ └── project + │ │ ├── columns: k:1!null v:2 w:3 + │ │ └── scan t + │ │ └── columns: k:1!null v:2 w:3 crdb_internal_mvcc_timestamp:4 │ └── aggregations - │ └── sum [as=sum:4] + │ └── sum [as=sum:5] │ └── w:3 └── 10 @@ -172,7 +174,7 @@ limit │ └── project │ ├── columns: v:2 │ └── scan t - │ └── columns: k:1!null v:2 w:3 + │ └── columns: k:1!null v:2 w:3 crdb_internal_mvcc_timestamp:4 └── 10 build @@ -266,7 +268,7 @@ project │ └── project │ ├── columns: k:1!null v:2 │ └── scan t - │ └── columns: k:1!null v:2 w:3 + │ └── columns: k:1!null v:2 w:3 crdb_internal_mvcc_timestamp:4 └── 10 # This kind of query can be used to work around memory usage limits. We need to @@ -296,7 +298,7 @@ limit │ │ └── project │ │ ├── columns: w:3 │ │ └── scan t - │ │ └── columns: k:1!null v:2 w:3 + │ │ └── columns: k:1!null v:2 w:3 crdb_internal_mvcc_timestamp:4 │ └── 100 └── 25 diff --git a/pkg/sql/opt/optbuilder/testdata/misc_statements b/pkg/sql/opt/optbuilder/testdata/misc_statements index 23cd9964a503..9020858270a4 100644 --- a/pkg/sql/opt/optbuilder/testdata/misc_statements +++ b/pkg/sql/opt/optbuilder/testdata/misc_statements @@ -33,7 +33,7 @@ control-jobs (PAUSE) └── project ├── columns: a:1 b:2 └── scan ab - └── columns: a:1 b:2 rowid:3!null + └── columns: a:1 b:2 rowid:3!null crdb_internal_mvcc_timestamp:4 build PAUSE JOB 1 @@ -80,7 +80,7 @@ cancel-sessions └── project ├── columns: a:1 b:2 └── scan ab - └── columns: a:1 b:2 rowid:3!null + └── columns: a:1 b:2 rowid:3!null crdb_internal_mvcc_timestamp:4 build CANCEL SESSION 1 @@ -119,7 +119,7 @@ cancel-queries └── project ├── columns: a:1 b:2 └── scan ab - └── columns: a:1 b:2 rowid:3!null + └── columns: a:1 b:2 rowid:3!null crdb_internal_mvcc_timestamp:4 build CANCEL QUERY 1 @@ -135,24 +135,24 @@ build EXPORT INTO CSV 'nodelocal://0/foo' FROM SELECT * FROM ab ---- export - ├── columns: filename:4 rows:5 bytes:6 + ├── columns: filename:5 rows:6 bytes:7 ├── format: CSV ├── project │ ├── columns: a:1 b:2 │ └── scan ab - │ └── columns: a:1 b:2 rowid:3!null + │ └── columns: a:1 b:2 rowid:3!null crdb_internal_mvcc_timestamp:4 └── 'nodelocal://0/foo' build EXPORT INTO CSV 'nodelocal://0/foo' WITH 'foo', 'bar'='baz' FROM SELECT * FROM ab ---- export - ├── columns: filename:4 rows:5 bytes:6 + ├── columns: filename:5 rows:6 bytes:7 ├── format: CSV ├── project │ ├── columns: a:1 b:2 │ └── scan ab - │ └── columns: a:1 b:2 rowid:3!null + │ └── columns: a:1 b:2 rowid:3!null crdb_internal_mvcc_timestamp:4 ├── 'nodelocal://0/foo' └── k-v-options ├── k-v-options-item foo @@ -164,12 +164,12 @@ build EXPORT INTO CSV 'nodelocal://0/foo' WITH 'foo' = $1 FROM SELECT * FROM ab ---- export - ├── columns: filename:4 rows:5 bytes:6 + ├── columns: filename:5 rows:6 bytes:7 ├── format: CSV ├── project │ ├── columns: a:1 b:2 │ └── scan ab - │ └── columns: a:1 b:2 rowid:3!null + │ └── columns: a:1 b:2 rowid:3!null crdb_internal_mvcc_timestamp:4 ├── 'nodelocal://0/foo' └── k-v-options └── k-v-options-item foo diff --git a/pkg/sql/opt/optbuilder/testdata/orderby b/pkg/sql/opt/optbuilder/testdata/orderby index e543fe8dad0b..7940874a0a19 100644 --- a/pkg/sql/opt/optbuilder/testdata/orderby +++ b/pkg/sql/opt/optbuilder/testdata/orderby @@ -17,7 +17,7 @@ sort └── project ├── columns: c:3 └── scan t - └── columns: a:1!null b:2 c:3 + └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 build SELECT c FROM t ORDER BY c DESC @@ -28,7 +28,7 @@ sort └── project ├── columns: c:3 └── scan t - └── columns: a:1!null b:2 c:3 + └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 build SELECT a, b FROM t ORDER BY b @@ -39,7 +39,7 @@ sort └── project ├── columns: a:1!null b:2 └── scan t - └── columns: a:1!null b:2 c:3 + └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 build SELECT a, b FROM t ORDER BY b DESC @@ -50,7 +50,7 @@ sort └── project ├── columns: a:1!null b:2 └── scan t - └── columns: a:1!null b:2 c:3 + └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 build SELECT a, b FROM t ORDER BY b DESC LIMIT 2 @@ -66,7 +66,7 @@ limit │ └── project │ ├── columns: a:1!null b:2 │ └── scan t - │ └── columns: a:1!null b:2 c:3 + │ └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 └── 2 build @@ -76,7 +76,7 @@ project ├── columns: a:1!null ├── ordering: -1 └── scan t,rev - ├── columns: a:1!null b:2 c:3 + ├── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 └── ordering: -1 # This query causes an error in Postgres, and the optimizer has followed @@ -95,7 +95,7 @@ project ├── columns: foo:1!null b:2 ├── ordering: -1 └── scan t,rev - ├── columns: a:1!null b:2 c:3 + ├── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 └── ordering: -1 # Check that ambiguous references to renders are properly reported. @@ -118,7 +118,7 @@ limit │ ├── ordering: +1 │ ├── limit hint: 1.00 │ └── scan t - │ ├── columns: a:1!null b:2 c:3 + │ ├── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 │ ├── ordering: +1 │ └── limit hint: 1.00 └── 1 @@ -132,7 +132,7 @@ project ├── columns: b:1!null c:2 ├── ordering: +1 └── scan t - ├── columns: a:1!null b:2 c:3 + ├── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 └── ordering: +1 build @@ -142,7 +142,7 @@ project ├── columns: foo.bar:1!null b:2 ├── ordering: -1 └── scan t,rev - ├── columns: a:1!null b:2 c:3 + ├── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 └── ordering: -1 build @@ -152,7 +152,7 @@ project ├── columns: foo:1!null b:2 ├── ordering: -1 └── scan t,rev - ├── columns: a:1!null b:2 c:3 + ├── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 └── ordering: -1 build @@ -162,7 +162,7 @@ project ├── columns: b:2 [hidden: a:1!null] ├── ordering: -1 └── scan t,rev - ├── columns: a:1!null b:2 c:3 + ├── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 └── ordering: -1 build @@ -177,7 +177,7 @@ limit │ ├── ordering: +1 │ ├── limit hint: 1.00 │ └── scan t - │ ├── columns: a:1!null b:2 c:3 + │ ├── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 │ ├── ordering: +1 │ └── limit hint: 1.00 └── 1 @@ -189,7 +189,7 @@ project ├── columns: b:2 [hidden: a:1!null] ├── ordering: -1,+2 └── scan t,rev - ├── columns: a:1!null b:2 c:3 + ├── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 └── ordering: -1 build @@ -199,7 +199,7 @@ project ├── columns: b:2 [hidden: a:1!null] ├── ordering: -1,-2 └── scan t,rev - ├── columns: a:1!null b:2 c:3 + ├── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 └── ordering: -1 # both presentation and ordering @@ -209,44 +209,58 @@ SELECT a, b, b FROM t ORDER BY c sort ├── columns: a:1!null b:2 b:2 [hidden: c:3] ├── ordering: +3 - └── scan t - └── columns: a:1!null b:2 c:3 + └── project + ├── columns: a:1!null b:2 c:3 + └── scan t + └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 build SELECT * FROM t ORDER BY (b, t.*) ---- -sort +project ├── columns: a:1!null b:2 c:3 ├── ordering: +2,+1,+2,+3 - └── scan t - └── columns: a:1!null b:2 c:3 + └── sort + ├── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 + ├── ordering: +2,+1 + └── scan t + └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 build SELECT * FROM t ORDER BY (b, a), c ---- -sort +project ├── columns: a:1!null b:2 c:3 ├── ordering: +2,+1,+3 - └── scan t - └── columns: a:1!null b:2 c:3 + └── sort + ├── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 + ├── ordering: +2,+1 + └── scan t + └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 build SELECT * FROM t ORDER BY b, (a, c) ---- -sort +project ├── columns: a:1!null b:2 c:3 ├── ordering: +2,+1,+3 - └── scan t - └── columns: a:1!null b:2 c:3 + └── sort + ├── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 + ├── ordering: +2,+1 + └── scan t + └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 build SELECT * FROM t ORDER BY (b, (a, c)) ---- -sort +project ├── columns: a:1!null b:2 c:3 ├── ordering: +2,+1,+3 - └── scan t - └── columns: a:1!null b:2 c:3 + └── sort + ├── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 + ├── ordering: +2,+1 + └── scan t + └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 build SELECT a, b FROM t WHERE b = 7 ORDER BY b, a @@ -255,10 +269,10 @@ project ├── columns: a:1!null b:2!null ├── ordering: +2,+1 └── select - ├── columns: a:1!null b:2!null c:3 + ├── columns: a:1!null b:2!null c:3 crdb_internal_mvcc_timestamp:4 ├── ordering: +1 opt(2) ├── scan t - │ ├── columns: a:1!null b:2 c:3 + │ ├── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 │ └── ordering: +1 opt(2) └── filters └── b:2 = 7 @@ -272,37 +286,37 @@ sort └── project ├── columns: a:1!null b:2 └── scan t - └── columns: a:1!null b:2 c:3 + └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 build SELECT a, b, a+b AS ab FROM t WHERE b = 7 ORDER BY ab DESC, a ---- sort - ├── columns: a:1!null b:2!null ab:4!null - ├── ordering: -4,+1 + ├── columns: a:1!null b:2!null ab:5!null + ├── ordering: -5,+1 └── project - ├── columns: ab:4!null a:1!null b:2!null + ├── columns: ab:5!null a:1!null b:2!null ├── select - │ ├── columns: a:1!null b:2!null c:3 + │ ├── columns: a:1!null b:2!null c:3 crdb_internal_mvcc_timestamp:4 │ ├── scan t - │ │ └── columns: a:1!null b:2 c:3 + │ │ └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 │ └── filters │ └── b:2 = 7 └── projections - └── a:1 + b:2 [as=ab:4] + └── a:1 + b:2 [as=ab:5] build SELECT a FROM t ORDER BY a+b DESC, a ---- sort - ├── columns: a:1!null [hidden: column4:4] - ├── ordering: -4,+1 + ├── columns: a:1!null [hidden: column5:5] + ├── ordering: -5,+1 └── project - ├── columns: column4:4 a:1!null + ├── columns: column5:5 a:1!null ├── scan t - │ └── columns: a:1!null b:2 c:3 + │ └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 └── projections - └── a:1 + b:2 [as=column4:4] + └── a:1 + b:2 [as=column5:5] build SELECT a FROM t ORDER BY (((a))) @@ -311,7 +325,7 @@ project ├── columns: a:1!null ├── ordering: +1 └── scan t - ├── columns: a:1!null b:2 c:3 + ├── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 └── ordering: +1 build @@ -326,7 +340,7 @@ limit │ ├── ordering: -1 │ ├── limit hint: 4.00 │ └── scan t,rev - │ ├── columns: a:1!null b:2 c:3 + │ ├── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 │ ├── ordering: -1 │ └── limit hint: 4.00 └── 4 @@ -343,7 +357,7 @@ limit │ ├── ordering: -1 │ ├── limit hint: 4.00 │ └── scan t,rev - │ ├── columns: a:1!null b:2 c:3 + │ ├── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 │ ├── ordering: -1 │ └── limit hint: 4.00 └── 4 @@ -514,92 +528,100 @@ build SELECT * FROM t ORDER BY 1+2 ---- project - ├── columns: a:1!null b:2 c:3 [hidden: column4:4!null] - ├── ordering: +4 + ├── columns: a:1!null b:2 c:3 [hidden: column5:5!null] + ├── ordering: +5 ├── scan t - │ └── columns: a:1!null b:2 c:3 + │ └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 └── projections - └── 1 + 2 [as=column4:4] + └── 1 + 2 [as=column5:5] build SELECT 1 AS r, * FROM t ORDER BY 1 ---- project - ├── columns: r:4!null a:1!null b:2 c:3 - ├── ordering: +4 + ├── columns: r:5!null a:1!null b:2 c:3 + ├── ordering: +5 ├── scan t - │ └── columns: a:1!null b:2 c:3 + │ └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 └── projections - └── 1 [as=r:4] + └── 1 [as=r:5] build SELECT * FROM t ORDER BY length('abc') ---- project - ├── columns: a:1!null b:2 c:3 [hidden: column4:4] - ├── ordering: +4 + ├── columns: a:1!null b:2 c:3 [hidden: column5:5] + ├── ordering: +5 ├── scan t - │ └── columns: a:1!null b:2 c:3 + │ └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 └── projections - └── length('abc') [as=column4:4] + └── length('abc') [as=column5:5] build SELECT b+2 AS r FROM t ORDER BY b+2 ---- sort - ├── columns: r:4 - ├── ordering: +4 + ├── columns: r:5 + ├── ordering: +5 └── project - ├── columns: r:4 + ├── columns: r:5 ├── scan t - │ └── columns: a:1!null b:2 c:3 + │ └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 └── projections - └── b:2 + 2 [as=r:4] + └── b:2 + 2 [as=r:5] # Check that the sort picks up a renamed render properly. build SELECT b+2 AS y FROM t ORDER BY y ---- sort - ├── columns: y:4 - ├── ordering: +4 + ├── columns: y:5 + ├── ordering: +5 └── project - ├── columns: y:4 + ├── columns: y:5 ├── scan t - │ └── columns: a:1!null b:2 c:3 + │ └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 └── projections - └── b:2 + 2 [as=y:4] + └── b:2 + 2 [as=y:5] build SELECT b+2 AS y FROM t ORDER BY b+2 ---- sort - ├── columns: y:4 - ├── ordering: +4 + ├── columns: y:5 + ├── ordering: +5 └── project - ├── columns: y:4 + ├── columns: y:5 ├── scan t - │ └── columns: a:1!null b:2 c:3 + │ └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 └── projections - └── b:2 + 2 [as=y:4] + └── b:2 + 2 [as=y:5] build SELECT b, c FROM t ORDER BY @2 ---- sort - ├── columns: b:2 c:3 [hidden: column4:4] - ├── ordering: +4 + ├── columns: b:2 c:3 [hidden: column5:5] + ├── ordering: +5 └── project - ├── columns: column4:4 b:2 c:3 + ├── columns: column5:5 b:2 c:3 ├── scan t - │ └── columns: a:1!null b:2 c:3 + │ └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 └── projections - └── b:2 [as=column4:4] + └── b:2 [as=column5:5] build SELECT b, c FROM t ORDER BY @4 ---- -error (42703): invalid column ordinal: @4 +sort + ├── columns: b:2 c:3 [hidden: column5:5] + ├── ordering: +5 + └── project + ├── columns: column5:5 b:2 c:3 + ├── scan t + │ └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 + └── projections + └── crdb_internal_mvcc_timestamp:4 [as=column5:5] exec-ddl CREATE TABLE abc ( @@ -623,21 +645,24 @@ build SELECT d FROM abc ORDER BY lower(d) ---- sort - ├── columns: d:4 [hidden: column5:5] - ├── ordering: +5 + ├── columns: d:4 [hidden: column6:6] + ├── ordering: +6 └── project - ├── columns: column5:5 d:4 + ├── columns: column6:6 d:4 ├── scan abc - │ └── columns: a:1!null b:2!null c:3!null d:4 + │ └── columns: a:1!null b:2!null c:3!null d:4 crdb_internal_mvcc_timestamp:5 └── projections - └── lower(d:4) [as=column5:5] + └── lower(d:4) [as=column6:6] build SELECT * FROM abc ORDER BY a ---- -scan abc +project ├── columns: a:1!null b:2!null c:3!null d:4 - └── ordering: +1 + ├── ordering: +1 + └── scan abc + ├── columns: a:1!null b:2!null c:3!null d:4 crdb_internal_mvcc_timestamp:5 + └── ordering: +1 build SELECT a, b FROM abc ORDER BY b, a @@ -648,7 +673,7 @@ sort └── project ├── columns: a:1!null b:2!null └── scan abc - └── columns: a:1!null b:2!null c:3!null d:4 + └── columns: a:1!null b:2!null c:3!null d:4 crdb_internal_mvcc_timestamp:5 build SELECT a, b FROM abc ORDER BY b, c @@ -659,7 +684,7 @@ sort └── project ├── columns: a:1!null b:2!null c:3!null └── scan abc - └── columns: a:1!null b:2!null c:3!null d:4 + └── columns: a:1!null b:2!null c:3!null d:4 crdb_internal_mvcc_timestamp:5 build SELECT a, b FROM abc ORDER BY b, c, a DESC @@ -668,10 +693,10 @@ project ├── columns: a:1!null b:2!null [hidden: c:3!null] ├── ordering: +2,+3,-1 └── sort - ├── columns: a:1!null b:2!null c:3!null d:4 + ├── columns: a:1!null b:2!null c:3!null d:4 crdb_internal_mvcc_timestamp:5 ├── ordering: +2,+3 └── scan abc - └── columns: a:1!null b:2!null c:3!null d:4 + └── columns: a:1!null b:2!null c:3!null d:4 crdb_internal_mvcc_timestamp:5 build SELECT a FROM abc ORDER BY a DESC @@ -680,7 +705,7 @@ project ├── columns: a:1!null ├── ordering: -1 └── scan abc,rev - ├── columns: a:1!null b:2!null c:3!null d:4 + ├── columns: a:1!null b:2!null c:3!null d:4 crdb_internal_mvcc_timestamp:5 └── ordering: -1 build @@ -695,7 +720,7 @@ limit │ ├── ordering: -1 │ ├── limit hint: 1.00 │ └── scan abc,rev - │ ├── columns: a:1!null b:2!null c:3!null d:4 + │ ├── columns: a:1!null b:2!null c:3!null d:4 crdb_internal_mvcc_timestamp:5 │ ├── ordering: -1 │ └── limit hint: 1.00 └── 1 @@ -711,7 +736,7 @@ offset │ ├── columns: a:1!null │ ├── ordering: -1 │ └── scan abc,rev - │ ├── columns: a:1!null b:2!null c:3!null d:4 + │ ├── columns: a:1!null b:2!null c:3!null d:4 crdb_internal_mvcc_timestamp:5 │ └── ordering: -1 └── 1 @@ -724,9 +749,9 @@ sort └── project ├── columns: c:3!null └── select - ├── columns: a:1!null b:2!null c:3!null d:4 + ├── columns: a:1!null b:2!null c:3!null d:4 crdb_internal_mvcc_timestamp:5 ├── scan abc - │ └── columns: a:1!null b:2!null c:3!null d:4 + │ └── columns: a:1!null b:2!null c:3!null d:4 crdb_internal_mvcc_timestamp:5 └── filters └── b:2 = 2 @@ -739,9 +764,9 @@ sort └── project ├── columns: c:3!null └── select - ├── columns: a:1!null b:2!null c:3!null d:4 + ├── columns: a:1!null b:2!null c:3!null d:4 crdb_internal_mvcc_timestamp:5 ├── scan abc - │ └── columns: a:1!null b:2!null c:3!null d:4 + │ └── columns: a:1!null b:2!null c:3!null d:4 crdb_internal_mvcc_timestamp:5 └── filters └── b:2 = 2 @@ -755,10 +780,10 @@ project ├── columns: a:1!null b:2!null c:3!null ├── ordering: +2,+3 opt(1) └── select - ├── columns: a:1!null b:2!null c:3!null d:4 + ├── columns: a:1!null b:2!null c:3!null d:4 crdb_internal_mvcc_timestamp:5 ├── ordering: +2,+3 opt(1) ├── scan abc - │ ├── columns: a:1!null b:2!null c:3!null d:4 + │ ├── columns: a:1!null b:2!null c:3!null d:4 crdb_internal_mvcc_timestamp:5 │ └── ordering: +2,+3 opt(1) └── filters └── a:1 = 1 @@ -772,7 +797,7 @@ sort └── project ├── columns: a:1!null b:2!null c:3!null └── scan abc - └── columns: a:1!null b:2!null c:3!null d:4 + └── columns: a:1!null b:2!null c:3!null d:4 crdb_internal_mvcc_timestamp:5 build SELECT a FROM abc ORDER BY PRIMARY KEY a @@ -794,8 +819,10 @@ SELECT * FROM bar ORDER BY baz, id sort ├── columns: id:1!null baz:2 ├── ordering: +2,+1 - └── scan bar - └── columns: id:1!null baz:2 + └── project + ├── columns: id:1!null baz:2 + └── scan bar + └── columns: id:1!null baz:2 crdb_internal_mvcc_timestamp:3 exec-ddl CREATE TABLE abcd ( @@ -815,21 +842,25 @@ build SELECT a+b AS r FROM (SELECT * FROM abcd ORDER BY d) ---- project - ├── columns: r:5 - ├── scan abcd - │ └── columns: a:1!null b:2 c:3 d:4 + ├── columns: r:6 + ├── project + │ ├── columns: a:1!null b:2 c:3 d:4 + │ └── scan abcd + │ └── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 └── projections - └── a:1 + b:2 [as=r:5] + └── a:1 + b:2 [as=r:6] build SELECT b+d AS r FROM (SELECT * FROM abcd ORDER BY a,d) ---- project - ├── columns: r:5 - ├── scan abcd - │ └── columns: a:1!null b:2 c:3 d:4 + ├── columns: r:6 + ├── project + │ ├── columns: a:1!null b:2 c:3 d:4 + │ └── scan abcd + │ └── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 └── projections - └── b:2 + d:4 [as=r:5] + └── b:2 + d:4 [as=r:6] build SELECT * FROM (VALUES ('a'), ('b'), ('c')) AS c(x) ORDER BY x @@ -875,7 +906,7 @@ limit │ ├── ordering: +1,+2,+3 │ ├── limit hint: 1.00 │ └── scan blocks - │ ├── columns: block_id:1!null writer_id:2!null block_num:3!null raw_bytes:4 + │ ├── columns: block_id:1!null writer_id:2!null block_num:3!null raw_bytes:4 crdb_internal_mvcc_timestamp:5 │ ├── ordering: +1,+2,+3 │ └── limit hint: 1.00 └── 1 @@ -887,7 +918,7 @@ project ├── columns: a:1!null ├── ordering: +1 └── scan abcd - ├── columns: a:1!null b:2 c:3 d:4 + ├── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 └── ordering: +1 build @@ -899,7 +930,7 @@ sort └── project ├── columns: a:1!null b:2 └── scan abcd - └── columns: a:1!null b:2 c:3 d:4 + └── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 build SELECT a FROM abcd ORDER BY INDEX abcd@abc @@ -908,7 +939,7 @@ project ├── columns: a:1!null [hidden: b:2 c:3] ├── ordering: +1,+2,+3 └── scan abcd - ├── columns: a:1!null b:2 c:3 d:4 + ├── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 └── ordering: +1 build @@ -918,7 +949,7 @@ project ├── columns: a:1!null [hidden: b:2 c:3] ├── ordering: -1,-2,-3 └── scan abcd,rev - ├── columns: a:1!null b:2 c:3 d:4 + ├── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 └── ordering: -1 build @@ -937,8 +968,10 @@ SELECT a FROM abcd ORDER BY INDEX abcd@bcd sort ├── columns: a:1!null [hidden: b:2 c:3 d:4] ├── ordering: +2,-3,+4,+1 - └── scan abcd - └── columns: a:1!null b:2 c:3 d:4 + └── project + ├── columns: a:1!null b:2 c:3 d:4 + └── scan abcd + └── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 build SELECT a FROM abcd ORDER BY INDEX abcd@bcd DESC @@ -946,8 +979,10 @@ SELECT a FROM abcd ORDER BY INDEX abcd@bcd DESC sort ├── columns: a:1!null [hidden: b:2 c:3 d:4] ├── ordering: -2,+3,-4,-1 - └── scan abcd - └── columns: a:1!null b:2 c:3 d:4 + └── project + ├── columns: a:1!null b:2 c:3 d:4 + └── scan abcd + └── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 build @@ -961,8 +996,10 @@ SELECT a FROM t.public.abcd ORDER BY INDEX t.public.abcd@bcd sort ├── columns: a:1!null [hidden: b:2 c:3 d:4] ├── ordering: +2,-3,+4,+1 - └── scan t.public.abcd - └── columns: a:1!null b:2 c:3 d:4 + └── project + ├── columns: a:1!null b:2 c:3 d:4 + └── scan t.public.abcd + └── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 build SELECT a FROM t.abcd ORDER BY INDEX t.abcd@bcd @@ -970,8 +1007,10 @@ SELECT a FROM t.abcd ORDER BY INDEX t.abcd@bcd sort ├── columns: a:1!null [hidden: b:2 c:3 d:4] ├── ordering: +2,-3,+4,+1 - └── scan t.public.abcd - └── columns: a:1!null b:2 c:3 d:4 + └── project + ├── columns: a:1!null b:2 c:3 d:4 + └── scan t.public.abcd + └── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 build SELECT a FROM public.abcd ORDER BY INDEX public.abcd@bcd @@ -979,8 +1018,10 @@ SELECT a FROM public.abcd ORDER BY INDEX public.abcd@bcd sort ├── columns: a:1!null [hidden: b:2 c:3 d:4] ├── ordering: +2,-3,+4,+1 - └── scan public.abcd - └── columns: a:1!null b:2 c:3 d:4 + └── project + ├── columns: a:1!null b:2 c:3 d:4 + └── scan public.abcd + └── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 build SELECT a FROM (SELECT a FROM abcd) ORDER BY INDEX abcd@bcd @@ -1010,7 +1051,7 @@ sort └── project ├── columns: a:1!null b:2 c:3 └── scan abcd - └── columns: a:1!null b:2 c:3 d:4 + └── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 build SELECT a FROM abcd ORDER BY b, c @@ -1021,7 +1062,7 @@ sort └── project ├── columns: a:1!null b:2 c:3 └── scan abcd - └── columns: a:1!null b:2 c:3 d:4 + └── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 build SELECT a FROM abcd ORDER BY a, b, c @@ -1030,18 +1071,18 @@ project ├── columns: a:1!null [hidden: b:2 c:3] ├── ordering: +1,+2,+3 └── scan abcd - ├── columns: a:1!null b:2 c:3 d:4 + ├── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 └── ordering: +1 build SELECT ARRAY[a] FROM abcd ORDER BY 1 ---- sort - ├── columns: array:5!null - ├── ordering: +5 + ├── columns: array:6!null + ├── ordering: +6 └── project - ├── columns: array:5!null + ├── columns: array:6!null ├── scan abcd - │ └── columns: a:1!null b:2 c:3 d:4 + │ └── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 └── projections - └── ARRAY[a:1] [as=array:5] + └── ARRAY[a:1] [as=array:6] diff --git a/pkg/sql/opt/optbuilder/testdata/ordinality b/pkg/sql/opt/optbuilder/testdata/ordinality index f0678087be27..7a082e0dc5fb 100644 --- a/pkg/sql/opt/optbuilder/testdata/ordinality +++ b/pkg/sql/opt/optbuilder/testdata/ordinality @@ -22,57 +22,63 @@ build SELECT a, ordinality FROM abcd WITH ORDINALITY ---- project - ├── columns: a:1!null ordinality:5!null + ├── columns: a:1!null ordinality:6!null └── ordinality - ├── columns: a:1!null b:2 c:3 d:4 ordinality:5!null + ├── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 ordinality:6!null └── scan abcd - └── columns: a:1!null b:2 c:3 d:4 + └── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 build SELECT a, ordinality FROM (SELECT * FROM abcd ORDER BY a) WITH ORDINALITY ---- project - ├── columns: a:1!null ordinality:5!null + ├── columns: a:1!null ordinality:6!null └── ordinality - ├── columns: a:1!null b:2 c:3 d:4 ordinality:5!null - └── scan abcd + ├── columns: a:1!null b:2 c:3 d:4 ordinality:6!null + └── project ├── columns: a:1!null b:2 c:3 d:4 - └── ordering: +1 + ├── ordering: +1 + └── scan abcd + ├── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 + └── ordering: +1 build SELECT a, ordinality FROM (SELECT * FROM abcd ORDER BY a) WITH ORDINALITY ORDER BY ordinality ---- project - ├── columns: a:1!null ordinality:5!null - ├── ordering: +5 + ├── columns: a:1!null ordinality:6!null + ├── ordering: +6 └── ordinality - ├── columns: a:1!null b:2 c:3 d:4 ordinality:5!null - ├── ordering: +5 - └── scan abcd + ├── columns: a:1!null b:2 c:3 d:4 ordinality:6!null + ├── ordering: +6 + └── project ├── columns: a:1!null b:2 c:3 d:4 - └── ordering: +1 + ├── ordering: +1 + └── scan abcd + ├── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 + └── ordering: +1 build SELECT a FROM abcd WITH ORDINALITY ORDER BY ordinality ---- project - ├── columns: a:1!null [hidden: ordinality:5!null] - ├── ordering: +5 + ├── columns: a:1!null [hidden: ordinality:6!null] + ├── ordering: +6 └── ordinality - ├── columns: a:1!null b:2 c:3 d:4 ordinality:5!null - ├── ordering: +5 + ├── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 ordinality:6!null + ├── ordering: +6 └── scan abcd - └── columns: a:1!null b:2 c:3 d:4 + └── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 build SELECT ordinality FROM abcd WITH ORDINALITY ORDER BY a ---- sort - ├── columns: ordinality:5!null [hidden: a:1!null] + ├── columns: ordinality:6!null [hidden: a:1!null] ├── ordering: +1 └── project - ├── columns: a:1!null ordinality:5!null + ├── columns: a:1!null ordinality:6!null └── ordinality - ├── columns: a:1!null b:2 c:3 d:4 ordinality:5!null + ├── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 ordinality:6!null └── scan abcd - └── columns: a:1!null b:2 c:3 d:4 + └── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 diff --git a/pkg/sql/opt/optbuilder/testdata/project b/pkg/sql/opt/optbuilder/testdata/project index c3551d35cabf..94558bd2dffa 100644 --- a/pkg/sql/opt/optbuilder/testdata/project +++ b/pkg/sql/opt/optbuilder/testdata/project @@ -22,25 +22,31 @@ SELECT a.x FROM a project ├── columns: x:1!null └── scan a - └── columns: x:1!null y:2 + └── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 build SELECT a.x, a.y FROM a ---- -scan a - └── columns: x:1!null y:2 +project + ├── columns: x:1!null y:2 + └── scan a + └── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 build SELECT a.y, a.x FROM a ---- -scan a - └── columns: y:2 x:1!null +project + ├── columns: y:2 x:1!null + └── scan a + └── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 build SELECT * FROM a ---- -scan a - └── columns: x:1!null y:2 +project + ├── columns: x:1!null y:2 + └── scan a + └── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 # Note that an explicit projection operator is added for table b (unlike for # table a) to avoid projecting the hidden rowid column. @@ -50,54 +56,54 @@ SELECT * FROM b project ├── columns: x:1 y:2 └── scan b - └── columns: x:1 y:2 rowid:3!null + └── columns: x:1 y:2 rowid:3!null crdb_internal_mvcc_timestamp:4 build SELECT (a.x + 3) AS "X", false AS "Y" FROM a ---- project - ├── columns: X:3!null Y:4!null + ├── columns: X:4!null Y:5!null ├── scan a - │ └── columns: x:1!null y:2 + │ └── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 └── projections - ├── x:1 + 3 [as=X:3] - └── false [as=Y:4] + ├── x:1 + 3 [as=X:4] + └── false [as=Y:5] build SELECT *, ((x < y) OR x > 1000) AS r FROM a ---- project - ├── columns: x:1!null y:2 r:3 + ├── columns: x:1!null y:2 r:4 ├── scan a - │ └── columns: x:1!null y:2 + │ └── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 └── projections - └── (x:1 < y:2) OR (x:1 > 1000) [as=r:3] + └── (x:1 < y:2) OR (x:1 > 1000) [as=r:4] build SELECT a.*, true FROM a ---- project - ├── columns: x:1!null y:2 bool:3!null + ├── columns: x:1!null y:2 bool:4!null ├── scan a - │ └── columns: x:1!null y:2 + │ └── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 └── projections - └── true [as=bool:3] + └── true [as=bool:4] build SELECT u + 1 AS r, v + 1 AS s FROM (SELECT a.x + 3 AS t, a.y + 1.0 AS u FROM a) AS foo(u, v) ---- project - ├── columns: r:5!null s:6 + ├── columns: r:6!null s:7 ├── project - │ ├── columns: t:3!null u:4 + │ ├── columns: t:4!null u:5 │ ├── scan a - │ │ └── columns: x:1!null y:2 + │ │ └── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 │ └── projections - │ ├── x:1 + 3 [as=t:3] - │ └── y:2 + 1.0 [as=u:4] + │ ├── x:1 + 3 [as=t:4] + │ └── y:2 + 1.0 [as=u:5] └── projections - ├── t:3 + 1 [as=r:5] - └── u:4 + 1.0 [as=s:6] + ├── t:4 + 1 [as=r:6] + └── u:5 + 1.0 [as=s:7] build SELECT rowid FROM b; @@ -105,7 +111,7 @@ SELECT rowid FROM b; project ├── columns: rowid:3!null └── scan b - └── columns: x:1 y:2 rowid:3!null + └── columns: x:1 y:2 rowid:3!null crdb_internal_mvcc_timestamp:4 build SELECT rowid FROM (SELECT * FROM b) @@ -118,7 +124,7 @@ SELECT rowid FROM (SELECT rowid FROM b) project ├── columns: rowid:3!null └── scan b - └── columns: x:1 y:2 rowid:3!null + └── columns: x:1 y:2 rowid:3!null crdb_internal_mvcc_timestamp:4 build SELECT q.r FROM (SELECT rowid FROM b) AS q(r) @@ -126,7 +132,7 @@ SELECT q.r FROM (SELECT rowid FROM b) AS q(r) project ├── columns: r:3!null └── scan b - └── columns: x:1 y:2 rowid:3!null + └── columns: x:1 y:2 rowid:3!null crdb_internal_mvcc_timestamp:4 build SELECT r FROM (SELECT rowid FROM b) AS q(r) @@ -134,7 +140,7 @@ SELECT r FROM (SELECT rowid FROM b) AS q(r) project ├── columns: r:3!null └── scan b - └── columns: x:1 y:2 rowid:3!null + └── columns: x:1 y:2 rowid:3!null crdb_internal_mvcc_timestamp:4 exec-ddl CREATE TABLE c (x INT, y FLOAT) @@ -148,30 +154,34 @@ error (42702): column reference "rowid" is ambiguous (candidates: b.rowid, c.row build SELECT x, y, rowid FROM c WHERE rowid > 0 ---- -select +project ├── columns: x:1 y:2 rowid:3!null - ├── scan c - │ └── columns: x:1 y:2 rowid:3!null - └── filters - └── rowid:3 > 0 + └── select + ├── columns: x:1 y:2 rowid:3!null crdb_internal_mvcc_timestamp:4 + ├── scan c + │ └── columns: x:1 y:2 rowid:3!null crdb_internal_mvcc_timestamp:4 + └── filters + └── rowid:3 > 0 build SELECT r FROM (SELECT x, y, rowid AS r FROM c) ---- project ├── columns: r:3!null - └── scan c - └── columns: x:1 y:2 rowid:3!null + └── project + ├── columns: x:1 y:2 rowid:3!null + └── scan c + └── columns: x:1 y:2 rowid:3!null crdb_internal_mvcc_timestamp:4 build SELECT rowid::string FROM b ---- project - ├── columns: rowid:4!null + ├── columns: rowid:5!null ├── scan b - │ └── columns: x:1 y:2 b.rowid:3!null + │ └── columns: x:1 y:2 b.rowid:3!null crdb_internal_mvcc_timestamp:4 └── projections - └── b.rowid:3::STRING [as=rowid:4] + └── b.rowid:3::STRING [as=rowid:5] build SELECT (x, y)::timestamp FROM b @@ -217,18 +227,18 @@ SELECT t.kv.k FROM kv project ├── columns: k:1!null └── scan t.public.kv - └── columns: t.public.kv.k:1!null t.public.kv.v:2 + └── columns: t.public.kv.k:1!null t.public.kv.v:2 t.public.kv.crdb_internal_mvcc_timestamp:3 # Check that tuple type includes labels. build SELECT x FROM (SELECT (row(v,v,v) AS a,b,c) AS x FROM kv) ---- project - ├── columns: x:3 + ├── columns: x:4 ├── scan kv - │ └── columns: k:1!null v:2 + │ └── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 └── projections - └── ((v:2, v:2, v:2) AS a, b, c) [as=x:3] + └── ((v:2, v:2, v:2) AS a, b, c) [as=x:4] # Regression test for #48179. Star expansion of un-labeled tuple must project # all columns from the tuple. diff --git a/pkg/sql/opt/optbuilder/testdata/projection-reuse b/pkg/sql/opt/optbuilder/testdata/projection-reuse index f19fc617dd64..15eb3d109266 100644 --- a/pkg/sql/opt/optbuilder/testdata/projection-reuse +++ b/pkg/sql/opt/optbuilder/testdata/projection-reuse @@ -9,45 +9,45 @@ build SELECT a+b, a+b FROM ab ---- project - ├── columns: "?column?":4 "?column?":4 + ├── columns: "?column?":5 "?column?":5 ├── scan ab - │ └── columns: a:1 b:2 rowid:3!null + │ └── columns: a:1 b:2 rowid:3!null crdb_internal_mvcc_timestamp:4 └── projections - └── a:1 + b:2 [as="?column?":4] + └── a:1 + b:2 [as="?column?":5] # Ensure whitespace differences don't prevent deduplication. build SELECT a+b, a + b FROM ab ---- project - ├── columns: "?column?":4 "?column?":4 + ├── columns: "?column?":5 "?column?":5 ├── scan ab - │ └── columns: a:1 b:2 rowid:3!null + │ └── columns: a:1 b:2 rowid:3!null crdb_internal_mvcc_timestamp:4 └── projections - └── a:1 + b:2 [as="?column?":4] + └── a:1 + b:2 [as="?column?":5] # Side-effecting expressions are not deduplicated. build SELECT random(), random() FROM ab ---- project - ├── columns: random:4 random:5 + ├── columns: random:5 random:6 ├── scan ab - │ └── columns: a:1 b:2 rowid:3!null + │ └── columns: a:1 b:2 rowid:3!null crdb_internal_mvcc_timestamp:4 └── projections - ├── random() [as=random:4] - └── random() [as=random:5] + ├── random() [as=random:5] + └── random() [as=random:6] build SELECT random(), random() FROM ab ---- project - ├── columns: random:4 random:5 + ├── columns: random:5 random:6 ├── scan ab - │ └── columns: a:1 b:2 rowid:3!null + │ └── columns: a:1 b:2 rowid:3!null crdb_internal_mvcc_timestamp:4 └── projections - ├── random() [as=random:4] - └── random() [as=random:5] + ├── random() [as=random:5] + └── random() [as=random:6] # ORDER BY does not add a new projection if the same expression is projected # already, regardless of side-effects. @@ -55,15 +55,15 @@ build SELECT a, b, random(), random() FROM ab ORDER BY random() ---- sort - ├── columns: a:1 b:2 random:4 random:5 - ├── ordering: +4 + ├── columns: a:1 b:2 random:5 random:6 + ├── ordering: +5 └── project - ├── columns: random:4 random:5 a:1 b:2 + ├── columns: random:5 random:6 a:1 b:2 ├── scan ab - │ └── columns: a:1 b:2 rowid:3!null + │ └── columns: a:1 b:2 rowid:3!null crdb_internal_mvcc_timestamp:4 └── projections - ├── random() [as=random:4] - └── random() [as=random:5] + ├── random() [as=random:5] + └── random() [as=random:6] # With GROUP BY, expressions identical to a grouping column are always # collapsed into a single value. @@ -71,14 +71,14 @@ build SELECT random(), random() FROM ab GROUP BY random() ---- group-by - ├── columns: random:4 random:4 - ├── grouping columns: column4:4 + ├── columns: random:5 random:5 + ├── grouping columns: column5:5 └── project - ├── columns: column4:4 + ├── columns: column5:5 ├── scan ab - │ └── columns: a:1 b:2 rowid:3!null + │ └── columns: a:1 b:2 rowid:3!null crdb_internal_mvcc_timestamp:4 └── projections - └── random() [as=column4:4] + └── random() [as=column5:5] build INSERT INTO ab VALUES (random(), random()) @@ -86,16 +86,16 @@ INSERT INTO ab VALUES (random(), random()) insert ab ├── columns: ├── insert-mapping: - │ ├── column1:4 => a:1 - │ ├── column2:5 => b:2 - │ └── column6:6 => rowid:3 + │ ├── column1:5 => a:1 + │ ├── column2:6 => b:2 + │ └── column7:7 => rowid:3 └── project - ├── columns: column6:6 column1:4 column2:5 + ├── columns: column7:7 column1:5 column2:6 ├── values - │ ├── columns: column1:4 column2:5 + │ ├── columns: column1:5 column2:6 │ └── (random(), random()) └── projections - └── unique_rowid() [as=column6:6] + └── unique_rowid() [as=column7:7] # Make sure impure default expressions are not deduplicated. exec-ddl @@ -108,20 +108,20 @@ INSERT INTO abcd VALUES (1, 1) insert abcd ├── columns: ├── insert-mapping: - │ ├── column1:6 => a:1 - │ ├── column2:7 => b:2 - │ ├── column8:8 => c:3 - │ ├── column9:9 => d:4 - │ └── column10:10 => rowid:5 + │ ├── column1:7 => a:1 + │ ├── column2:8 => b:2 + │ ├── column9:9 => c:3 + │ ├── column10:10 => d:4 + │ └── column11:11 => rowid:5 └── project - ├── columns: column8:8 column9:9 column10:10 column1:6!null column2:7!null + ├── columns: column9:9 column10:10 column11:11 column1:7!null column2:8!null ├── values - │ ├── columns: column1:6!null column2:7!null + │ ├── columns: column1:7!null column2:8!null │ └── (1.0, 1.0) └── projections - ├── random() [as=column8:8] ├── random() [as=column9:9] - └── unique_rowid() [as=column10:10] + ├── random() [as=column10:10] + └── unique_rowid() [as=column11:11] build INSERT INTO abcd VALUES (random(), random()) @@ -129,20 +129,20 @@ INSERT INTO abcd VALUES (random(), random()) insert abcd ├── columns: ├── insert-mapping: - │ ├── column1:6 => a:1 - │ ├── column2:7 => b:2 - │ ├── column8:8 => c:3 - │ ├── column9:9 => d:4 - │ └── column10:10 => rowid:5 + │ ├── column1:7 => a:1 + │ ├── column2:8 => b:2 + │ ├── column9:9 => c:3 + │ ├── column10:10 => d:4 + │ └── column11:11 => rowid:5 └── project - ├── columns: column8:8 column9:9 column10:10 column1:6 column2:7 + ├── columns: column9:9 column10:10 column11:11 column1:7 column2:8 ├── values - │ ├── columns: column1:6 column2:7 + │ ├── columns: column1:7 column2:8 │ └── (random(), random()) └── projections - ├── random() [as=column8:8] ├── random() [as=column9:9] - └── unique_rowid() [as=column10:10] + ├── random() [as=column10:10] + └── unique_rowid() [as=column11:11] build UPSERT INTO abcd VALUES (1, 1) @@ -150,20 +150,20 @@ UPSERT INTO abcd VALUES (1, 1) upsert abcd ├── columns: ├── upsert-mapping: - │ ├── column1:6 => a:1 - │ ├── column2:7 => b:2 - │ ├── column8:8 => c:3 - │ ├── column9:9 => d:4 - │ └── column10:10 => rowid:5 + │ ├── column1:7 => a:1 + │ ├── column2:8 => b:2 + │ ├── column9:9 => c:3 + │ ├── column10:10 => d:4 + │ └── column11:11 => rowid:5 └── project - ├── columns: column8:8 column9:9 column10:10 column1:6!null column2:7!null + ├── columns: column9:9 column10:10 column11:11 column1:7!null column2:8!null ├── values - │ ├── columns: column1:6!null column2:7!null + │ ├── columns: column1:7!null column2:8!null │ └── (1.0, 1.0) └── projections - ├── random() [as=column8:8] ├── random() [as=column9:9] - └── unique_rowid() [as=column10:10] + ├── random() [as=column10:10] + └── unique_rowid() [as=column11:11] build UPSERT INTO abcd VALUES (random(), random()) @@ -171,38 +171,38 @@ UPSERT INTO abcd VALUES (random(), random()) upsert abcd ├── columns: ├── upsert-mapping: - │ ├── column1:6 => a:1 - │ ├── column2:7 => b:2 - │ ├── column8:8 => c:3 - │ ├── column9:9 => d:4 - │ └── column10:10 => rowid:5 + │ ├── column1:7 => a:1 + │ ├── column2:8 => b:2 + │ ├── column9:9 => c:3 + │ ├── column10:10 => d:4 + │ └── column11:11 => rowid:5 └── project - ├── columns: column8:8 column9:9 column10:10 column1:6 column2:7 + ├── columns: column9:9 column10:10 column11:11 column1:7 column2:8 ├── values - │ ├── columns: column1:6 column2:7 + │ ├── columns: column1:7 column2:8 │ └── (random(), random()) └── projections - ├── random() [as=column8:8] ├── random() [as=column9:9] - └── unique_rowid() [as=column10:10] + ├── random() [as=column10:10] + └── unique_rowid() [as=column11:11] build UPDATE abcd SET a = random(), b = random() WHERE a=1 ---- update abcd ├── columns: - ├── fetch columns: a:6 b:7 c:8 d:9 rowid:10 + ├── fetch columns: a:7 b:8 c:9 d:10 rowid:11 ├── update-mapping: - │ ├── a_new:11 => a:1 - │ └── b_new:12 => b:2 + │ ├── a_new:13 => a:1 + │ └── b_new:14 => b:2 └── project - ├── columns: a_new:11 b_new:12 a:6!null b:7 c:8 d:9 rowid:10!null + ├── columns: a_new:13 b_new:14 a:7!null b:8 c:9 d:10 rowid:11!null crdb_internal_mvcc_timestamp:12 ├── select - │ ├── columns: a:6!null b:7 c:8 d:9 rowid:10!null + │ ├── columns: a:7!null b:8 c:9 d:10 rowid:11!null crdb_internal_mvcc_timestamp:12 │ ├── scan abcd - │ │ └── columns: a:6 b:7 c:8 d:9 rowid:10!null + │ │ └── columns: a:7 b:8 c:9 d:10 rowid:11!null crdb_internal_mvcc_timestamp:12 │ └── filters - │ └── a:6 = 1.0 + │ └── a:7 = 1.0 └── projections - ├── random() [as=a_new:11] - └── random() [as=b_new:12] + ├── random() [as=a_new:13] + └── random() [as=b_new:14] diff --git a/pkg/sql/opt/optbuilder/testdata/scalar b/pkg/sql/opt/optbuilder/testdata/scalar index 79695ede9af1..f65c3e9d884c 100644 --- a/pkg/sql/opt/optbuilder/testdata/scalar +++ b/pkg/sql/opt/optbuilder/testdata/scalar @@ -903,30 +903,35 @@ build SELECT b, ARRAY(SELECT a FROM x WHERE x.a = y.b) FROM y ---- project - ├── columns: b:1!null array:3 + ├── columns: b:1!null array:5 ├── scan y - │ └── columns: b:1!null + │ └── columns: b:1!null y.crdb_internal_mvcc_timestamp:2 └── projections - └── array-flatten [as=array:3] - └── select - ├── columns: a:2!null - ├── scan x - │ └── columns: a:2!null - └── filters - └── a:2 = b:1 + └── array-flatten [as=array:5] + └── project + ├── columns: a:3!null + └── select + ├── columns: a:3!null x.crdb_internal_mvcc_timestamp:4 + ├── scan x + │ └── columns: a:3!null x.crdb_internal_mvcc_timestamp:4 + └── filters + └── a:3 = b:1 build SELECT b, ARRAY(SELECT a FROM x ORDER BY a) FROM y ---- project - ├── columns: b:1!null array:3 + ├── columns: b:1!null array:5 ├── scan y - │ └── columns: b:1!null + │ └── columns: b:1!null y.crdb_internal_mvcc_timestamp:2 └── projections - └── array-flatten [as=array:3] - └── scan x - ├── columns: a:2!null - └── ordering: +2 + └── array-flatten [as=array:5] + └── project + ├── columns: a:3!null + ├── ordering: +3 + └── scan x + ├── columns: a:3!null x.crdb_internal_mvcc_timestamp:4 + └── ordering: +3 build SELECT ARRAY(VALUES ('foo'), ('bar'), ('baz')) @@ -1149,18 +1154,18 @@ build SELECT ARRAY(SELECT x FROM u ORDER BY x) ---- project - ├── columns: array:3 + ├── columns: array:4 ├── values │ └── () └── projections - └── array-flatten [as=array:3] + └── array-flatten [as=array:4] └── sort ├── columns: x:1 ├── ordering: +1 └── project ├── columns: x:1 └── scan u - └── columns: x:1 rowid:2!null + └── columns: x:1 rowid:2!null crdb_internal_mvcc_timestamp:3 build SELECT * FROM v WHERE y = ARRAY(SELECT x FROM u ORDER BY x) @@ -1168,20 +1173,20 @@ SELECT * FROM v WHERE y = ARRAY(SELECT x FROM u ORDER BY x) project ├── columns: y:1!null └── select - ├── columns: y:1!null v.rowid:2!null + ├── columns: y:1!null v.rowid:2!null v.crdb_internal_mvcc_timestamp:3 ├── scan v - │ └── columns: y:1 v.rowid:2!null + │ └── columns: y:1 v.rowid:2!null v.crdb_internal_mvcc_timestamp:3 └── filters └── eq ├── y:1 └── array-flatten └── sort - ├── columns: x:3 - ├── ordering: +3 + ├── columns: x:4 + ├── ordering: +4 └── project - ├── columns: x:3 + ├── columns: x:4 └── scan u - └── columns: x:3 u.rowid:4!null + └── columns: x:4 u.rowid:5!null u.crdb_internal_mvcc_timestamp:6 build SELECT ARRAY(SELECT (y, 2) FROM u ORDER BY x) FROM v diff --git a/pkg/sql/opt/optbuilder/testdata/select b/pkg/sql/opt/optbuilder/testdata/select index 95d8e194a536..790f61c1ad91 100644 --- a/pkg/sql/opt/optbuilder/testdata/select +++ b/pkg/sql/opt/optbuilder/testdata/select @@ -73,52 +73,62 @@ error (22P02): could not parse "hello" as type bool: invalid bool value build SELECT * FROM abc ---- -scan abc - └── columns: a:1!null b:2 c:3 +project + ├── columns: a:1!null b:2 c:3 + └── scan abc + └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 build SELECT NULL AS r, * FROM abc ---- project - ├── columns: r:4 a:1!null b:2 c:3 + ├── columns: r:5 a:1!null b:2 c:3 ├── scan abc - │ └── columns: a:1!null b:2 c:3 + │ └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 └── projections - └── NULL [as=r:4] + └── NULL [as=r:5] # synonym for SELECT * FROM abc build TABLE abc ---- -scan abc - └── columns: a:1!null b:2 c:3 +project + ├── columns: a:1!null b:2 c:3 + └── scan abc + └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 build SELECT * FROM abc WHERE NULL ---- -select +project ├── columns: a:1!null b:2 c:3 - ├── scan abc - │ └── columns: a:1!null b:2 c:3 - └── filters - └── NULL::BOOL + └── select + ├── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 + ├── scan abc + │ └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 + └── filters + └── NULL::BOOL build SELECT * FROM abc WHERE a = NULL ---- -select +project ├── columns: a:1!null b:2 c:3 - ├── scan abc - │ └── columns: a:1!null b:2 c:3 - └── filters - └── NULL::BOOL + └── select + ├── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 + ├── scan abc + │ └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 + └── filters + └── NULL::BOOL build SELECT *,* FROM abc ---- -scan abc - └── columns: a:1!null b:2 c:3 a:1!null b:2 c:3 +project + ├── columns: a:1!null b:2 c:3 a:1!null b:2 c:3 + └── scan abc + └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 build SELECT a,a,a,a FROM abc @@ -126,7 +136,7 @@ SELECT a,a,a,a FROM abc project ├── columns: a:1!null a:1!null a:1!null a:1!null └── scan abc - └── columns: a:1!null b:2 c:3 + └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 build SELECT a,c FROM abc @@ -134,17 +144,17 @@ SELECT a,c FROM abc project ├── columns: a:1!null c:3 └── scan abc - └── columns: a:1!null b:2 c:3 + └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 build SELECT a+b+c AS foo FROM abc ---- project - ├── columns: foo:4 + ├── columns: foo:5 ├── scan abc - │ └── columns: a:1!null b:2 c:3 + │ └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 └── projections - └── (a:1 + b:2) + c:3 [as=foo:4] + └── (a:1 + b:2) + c:3 [as=foo:5] build SELECT a,b FROM abc WHERE CASE WHEN a != 0 THEN b/a > 1.5 ELSE false END @@ -152,9 +162,9 @@ SELECT a,b FROM abc WHERE CASE WHEN a != 0 THEN b/a > 1.5 ELSE false END project ├── columns: a:1!null b:2 └── select - ├── columns: a:1!null b:2 c:3 + ├── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 ├── scan abc - │ └── columns: a:1!null b:2 c:3 + │ └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 └── filters └── CASE WHEN a:1 != 0 THEN (b:2 / a:1) > 1.5 ELSE false END @@ -167,34 +177,38 @@ CREATE TABLE kv (k CHAR PRIMARY KEY, v CHAR) build SELECT * FROM kv ---- -scan kv - └── columns: k:1!null v:2 +project + ├── columns: k:1!null v:2 + └── scan kv + └── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 build SELECT k,v FROM kv ---- -scan kv - └── columns: k:1!null v:2 +project + ├── columns: k:1!null v:2 + └── scan kv + └── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 build SELECT v||'foo' AS r FROM kv ---- project - ├── columns: r:3 + ├── columns: r:4 ├── scan kv - │ └── columns: k:1!null v:2 + │ └── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 └── projections - └── v:2::STRING || 'foo' [as=r:3] + └── v:2::STRING || 'foo' [as=r:4] build SELECT lower(v) FROM kv ---- project - ├── columns: lower:3 + ├── columns: lower:4 ├── scan kv - │ └── columns: k:1!null v:2 + │ └── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 └── projections - └── lower(v:2) [as=lower:3] + └── lower(v:2) [as=lower:4] build SELECT k FROM kv @@ -202,29 +216,33 @@ SELECT k FROM kv project ├── columns: k:1!null └── scan kv - └── columns: k:1!null v:2 + └── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 build SELECT kv.K,KV.v FROM kv ---- -scan kv - └── columns: k:1!null v:2 +project + ├── columns: k:1!null v:2 + └── scan kv + └── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 build SELECT kv.* FROM kv ---- -scan kv - └── columns: k:1!null v:2 +project + ├── columns: k:1!null v:2 + └── scan kv + └── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 build SELECT (kv.*) AS r FROM kv ---- project - ├── columns: r:3 + ├── columns: r:4 ├── scan kv - │ └── columns: k:1!null v:2 + │ └── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 └── projections - └── ((k:1, v:2) AS k, v) [as=r:3] + └── ((k:1, v:2) AS k, v) [as=r:4] build SELECT (SELECT t.*) FROM (VALUES (1)) AS t(x) @@ -282,9 +300,9 @@ SELECT FOO.k FROM kv AS foo WHERE foo.k = 'a' project ├── columns: k:1!null └── select - ├── columns: k:1!null v:2 + ├── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 ├── scan foo - │ └── columns: k:1!null v:2 + │ └── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 └── filters └── k:1 = 'a' @@ -294,9 +312,9 @@ SELECT "foo"."v" FROM kv AS foo WHERE foo.k = 'a' project ├── columns: v:2 └── select - ├── columns: k:1!null v:2 + ├── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 ├── scan foo - │ └── columns: k:1!null v:2 + │ └── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 └── filters └── k:1 = 'a' @@ -307,8 +325,10 @@ CREATE TABLE kw ("from" INT PRIMARY KEY) build SELECT *, "from", kw."from" FROM kw ---- -scan kw - └── columns: from:1!null from:1!null from:1!null +project + ├── columns: from:1!null from:1!null from:1!null + └── scan kw + └── columns: from:1!null crdb_internal_mvcc_timestamp:2 exec-ddl CREATE TABLE xyzw ( @@ -325,37 +345,47 @@ CREATE TABLE xyzw ( build SELECT * FROM xyzw@primary ---- -scan xyzw +project ├── columns: x:1!null y:2 z:3 w:4 - └── flags: force-index=primary + └── scan xyzw + ├── columns: x:1!null y:2 z:3 w:4 crdb_internal_mvcc_timestamp:5 + └── flags: force-index=primary build SELECT * FROM xyzw@foo ---- -scan xyzw +project ├── columns: x:1!null y:2 z:3 w:4 - └── flags: force-index=foo + └── scan xyzw + ├── columns: x:1!null y:2 z:3 w:4 crdb_internal_mvcc_timestamp:5 + └── flags: force-index=foo build SELECT * FROM xyzw@{FORCE_INDEX=foo,ASC} ---- -scan xyzw +project ├── columns: x:1!null y:2 z:3 w:4 - └── flags: force-index=foo,fwd + └── scan xyzw + ├── columns: x:1!null y:2 z:3 w:4 crdb_internal_mvcc_timestamp:5 + └── flags: force-index=foo,fwd build SELECT * FROM xyzw@{FORCE_INDEX=foo,DESC} ---- -scan xyzw,rev +project ├── columns: x:1!null y:2 z:3 w:4 - └── flags: force-index=foo,rev + └── scan xyzw,rev + ├── columns: x:1!null y:2 z:3 w:4 crdb_internal_mvcc_timestamp:5 + └── flags: force-index=foo,rev build SELECT * FROM xyzw@{NO_INDEX_JOIN} ---- -scan xyzw +project ├── columns: x:1!null y:2 z:3 w:4 - └── flags: no-index-join + └── scan xyzw + ├── columns: x:1!null y:2 z:3 w:4 crdb_internal_mvcc_timestamp:5 + └── flags: no-index-join build SELECT * FROM xyzw LIMIT x @@ -379,10 +409,14 @@ limit ├── columns: x:1!null y:2 z:3 w:4 ├── internal-ordering: +1 ├── ordering: +1 - ├── scan xyzw + ├── project │ ├── columns: x:1!null y:2 z:3 w:4 │ ├── ordering: +1 - │ └── limit hint: 1.00 + │ ├── limit hint: 1.00 + │ └── scan xyzw + │ ├── columns: x:1!null y:2 z:3 w:4 crdb_internal_mvcc_timestamp:5 + │ ├── ordering: +1 + │ └── limit hint: 1.00 └── 1 build @@ -396,9 +430,12 @@ SELECT * FROM xyzw LIMIT -100 ---- limit ├── columns: x:1!null y:2 z:3 w:4 - ├── scan xyzw + ├── project │ ├── columns: x:1!null y:2 z:3 w:4 - │ └── limit hint: 1.00 + │ ├── limit hint: 1.00 + │ └── scan xyzw + │ ├── columns: x:1!null y:2 z:3 w:4 crdb_internal_mvcc_timestamp:5 + │ └── limit hint: 1.00 └── -100 # At execution time, this will cause the error: negative value for OFFSET @@ -407,8 +444,10 @@ SELECT * FROM xyzw OFFSET -100 ---- offset ├── columns: x:1!null y:2 z:3 w:4 - ├── scan xyzw - │ └── columns: x:1!null y:2 z:3 w:4 + ├── project + │ ├── columns: x:1!null y:2 z:3 w:4 + │ └── scan xyzw + │ └── columns: x:1!null y:2 z:3 w:4 crdb_internal_mvcc_timestamp:5 └── -100 build @@ -418,29 +457,35 @@ offset ├── columns: x:1!null y:2 z:3 w:4 ├── internal-ordering: +1 ├── ordering: +1 - ├── scan xyzw + ├── project │ ├── columns: x:1!null y:2 z:3 w:4 - │ └── ordering: +1 + │ ├── ordering: +1 + │ └── scan xyzw + │ ├── columns: x:1!null y:2 z:3 w:4 crdb_internal_mvcc_timestamp:5 + │ └── ordering: +1 └── 1 + 0 build SELECT (x,y) AS r FROM xyzw ---- project - ├── columns: r:5 + ├── columns: r:6 ├── scan xyzw - │ └── columns: x:1!null y:2 z:3 w:4 + │ └── columns: x:1!null y:2 z:3 w:4 crdb_internal_mvcc_timestamp:5 └── projections - └── (x:1, y:2) [as=r:5] + └── (x:1, y:2) [as=r:6] build SELECT * FROM xyzw LIMIT 0 ---- limit ├── columns: x:1!null y:2 z:3 w:4 - ├── scan xyzw + ├── project │ ├── columns: x:1!null y:2 z:3 w:4 - │ └── limit hint: 1.00 + │ ├── limit hint: 1.00 + │ └── scan xyzw + │ ├── columns: x:1!null y:2 z:3 w:4 crdb_internal_mvcc_timestamp:5 + │ └── limit hint: 1.00 └── 0 build @@ -450,10 +495,14 @@ limit ├── columns: x:1!null y:2 z:3 w:4 ├── internal-ordering: +1 ├── ordering: +1 - ├── scan xyzw + ├── project │ ├── columns: x:1!null y:2 z:3 w:4 │ ├── ordering: +1 - │ └── limit hint: 1.00 + │ ├── limit hint: 1.00 + │ └── scan xyzw + │ ├── columns: x:1!null y:2 z:3 w:4 crdb_internal_mvcc_timestamp:5 + │ ├── ordering: +1 + │ └── limit hint: 1.00 └── 1 build @@ -468,10 +517,14 @@ limit │ ├── internal-ordering: +1 │ ├── ordering: +1 │ ├── limit hint: 1.00 - │ ├── scan xyzw + │ ├── project │ │ ├── columns: x:1!null y:2 z:3 w:4 │ │ ├── ordering: +1 - │ │ └── limit hint: 2.00 + │ │ ├── limit hint: 2.00 + │ │ └── scan xyzw + │ │ ├── columns: x:1!null y:2 z:3 w:4 crdb_internal_mvcc_timestamp:5 + │ │ ├── ordering: +1 + │ │ └── limit hint: 2.00 │ └── 1 └── 1 @@ -485,8 +538,10 @@ offset ├── sort │ ├── columns: x:1!null y:2 z:3 w:4 │ ├── ordering: +2 - │ └── scan xyzw - │ └── columns: x:1!null y:2 z:3 w:4 + │ └── project + │ ├── columns: x:1!null y:2 z:3 w:4 + │ └── scan xyzw + │ └── columns: x:1!null y:2 z:3 w:4 crdb_internal_mvcc_timestamp:5 └── 1 build @@ -505,8 +560,10 @@ limit │ │ ├── columns: x:1!null y:2 z:3 w:4 │ │ ├── ordering: +2 │ │ ├── limit hint: 2.00 - │ │ └── scan xyzw - │ │ └── columns: x:1!null y:2 z:3 w:4 + │ │ └── project + │ │ ├── columns: x:1!null y:2 z:3 w:4 + │ │ └── scan xyzw + │ │ └── columns: x:1!null y:2 z:3 w:4 crdb_internal_mvcc_timestamp:5 │ └── 1 └── 1 @@ -517,31 +574,33 @@ limit ├── columns: x:1!null y:2 z:3 w:4 ├── offset │ ├── columns: x:1!null y:2 z:3 w:4 - │ ├── scan xyzw - │ │ └── columns: x:1!null y:2 z:3 w:4 + │ ├── project + │ │ ├── columns: x:1!null y:2 z:3 w:4 + │ │ └── scan xyzw + │ │ └── columns: x:1!null y:2 z:3 w:4 xyzw.crdb_internal_mvcc_timestamp:5 │ └── mult │ ├── subquery │ │ └── max1-row - │ │ ├── columns: count_rows:8!null + │ │ ├── columns: count_rows:10!null │ │ └── scalar-group-by - │ │ ├── columns: count_rows:8!null + │ │ ├── columns: count_rows:10!null │ │ ├── project │ │ │ └── scan abc - │ │ │ └── columns: a:5!null b:6 c:7 + │ │ │ └── columns: a:6!null b:7 c:8 abc.crdb_internal_mvcc_timestamp:9 │ │ └── aggregations - │ │ └── count-rows [as=count_rows:8] + │ │ └── count-rows [as=count_rows:10] │ └── 3 └── mult ├── subquery │ └── max1-row - │ ├── columns: count_rows:12!null + │ ├── columns: count_rows:15!null │ └── scalar-group-by - │ ├── columns: count_rows:12!null + │ ├── columns: count_rows:15!null │ ├── project │ │ └── scan abc - │ │ └── columns: a:9!null b:10 c:11 + │ │ └── columns: a:11!null b:12 c:13 abc.crdb_internal_mvcc_timestamp:14 │ └── aggregations - │ └── count-rows [as=count_rows:12] + │ └── count-rows [as=count_rows:15] └── 2 build @@ -556,18 +615,23 @@ offset ├── columns: x:1!null y:2 z:3 w:4 ├── limit │ ├── columns: x:1!null y:2 z:3 w:4 - │ ├── scan xyzw + │ ├── project │ │ ├── columns: x:1!null y:2 z:3 w:4 - │ │ └── limit hint: 5.00 + │ │ ├── limit hint: 5.00 + │ │ └── scan xyzw + │ │ ├── columns: x:1!null y:2 z:3 w:4 crdb_internal_mvcc_timestamp:5 + │ │ └── limit hint: 5.00 │ └── 5 └── 5 build SELECT * FROM xyzw@foo ---- -scan xyzw +project ├── columns: x:1!null y:2 z:3 w:4 - └── flags: force-index=foo + └── scan xyzw + ├── columns: x:1!null y:2 z:3 w:4 crdb_internal_mvcc_timestamp:5 + └── flags: force-index=foo exec-ddl CREATE TABLE boolean_table ( @@ -582,7 +646,7 @@ SELECT value FROM boolean_table project ├── columns: value:2 └── scan boolean_table - └── columns: id:1!null value:2 + └── columns: id:1!null value:2 crdb_internal_mvcc_timestamp:3 build SELECT CASE WHEN NULL THEN 1 ELSE 2 END @@ -598,13 +662,13 @@ build SELECT 0 * b AS r, b % 1 AS s, 0 % b AS t from abc ---- project - ├── columns: r:4 s:5 t:6 + ├── columns: r:5 s:6 t:7 ├── scan abc - │ └── columns: a:1!null b:2 c:3 + │ └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 └── projections - ├── 0 * b:2 [as=r:4] - ├── b:2 % 1 [as=s:5] - └── 0 % b:2 [as=t:6] + ├── 0 * b:2 [as=r:5] + ├── b:2 % 1 [as=s:6] + └── 0 % b:2 [as=t:7] # Regression tests for #22670. build @@ -888,65 +952,77 @@ CREATE TABLE a (x INT PRIMARY KEY, y FLOAT) build SELECT * FROM a ---- -scan a - └── columns: x:1!null y:2 +project + ├── columns: x:1!null y:2 + └── scan a + └── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 build SELECT * FROM a WHERE x > 10 ---- -select +project ├── columns: x:1!null y:2 - ├── scan a - │ └── columns: x:1!null y:2 - └── filters - └── x:1 > 10 + └── select + ├── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 + ├── scan a + │ └── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 + └── filters + └── x:1 > 10 build SELECT * FROM a WHERE (x > 10 AND (x < 20 AND x != 13)) ---- -select +project ├── columns: x:1!null y:2 - ├── scan a - │ └── columns: x:1!null y:2 - └── filters - └── (x:1 > 10) AND ((x:1 < 20) AND (x:1 != 13)) + └── select + ├── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 + ├── scan a + │ └── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 + └── filters + └── (x:1 > 10) AND ((x:1 < 20) AND (x:1 != 13)) build SELECT * FROM a WHERE x IN (1, 2, 3) ---- -select +project ├── columns: x:1!null y:2 - ├── scan a - │ └── columns: x:1!null y:2 - └── filters - └── x:1 IN (1, 2, 3) + └── select + ├── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 + ├── scan a + │ └── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 + └── filters + └── x:1 IN (1, 2, 3) build SELECT * FROM a AS A(X, Y) ---- -scan a - └── columns: x:1!null y:2 +project + ├── columns: x:1!null y:2 + └── scan a + └── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 build SELECT @1 AS r, @2 AS s FROM a ---- project - ├── columns: r:3!null s:4 + ├── columns: r:4!null s:5 ├── scan a - │ └── columns: x:1!null y:2 + │ └── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 └── projections - ├── x:1 [as=r:3] - └── y:2 [as=s:4] + ├── x:1 [as=r:4] + └── y:2 [as=s:5] build SELECT * FROM a WHERE (x > 10)::bool ---- -select +project ├── columns: x:1!null y:2 - ├── scan a - │ └── columns: x:1!null y:2 - └── filters - └── x:1 > 10 + └── select + ├── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 + ├── scan a + │ └── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 + └── filters + └── x:1 > 10 build SELECT * FROM a WHERE (x > 10)::INT[] @@ -956,12 +1032,14 @@ error (42846): invalid cast: bool -> int[] build SELECT * FROM a WHERE x = $1 ---- -select +project ├── columns: x:1!null y:2 - ├── scan a - │ └── columns: x:1!null y:2 - └── filters - └── x:1 = $1 + └── select + ├── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 + ├── scan a + │ └── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 + └── filters + └── x:1 = $1 # This is slightly funky, because the AS OF SYSTEM TIME timestamp only gets # interpreted by the executor, which obviously is not at play in these tests. @@ -1000,7 +1078,7 @@ SELECT (((x, y) AS x, y)).x FROM a project ├── columns: x:1!null └── scan a - └── columns: x:1!null y:2 + └── columns: x:1!null y:2 crdb_internal_mvcc_timestamp:3 # Numeric Reference Tests @@ -1011,8 +1089,10 @@ project build SELECT * FROM [53 AS t] ---- -scan t - └── columns: a:1!null y:2 b:3 c:4 +project + ├── columns: a:1!null y:2 b:3 c:4 + └── scan t + └── columns: a:1!null y:2 b:3 c:4 crdb_internal_mvcc_timestamp:5 build SELECT * FROM [53(1) AS t] @@ -1035,7 +1115,9 @@ scan t build SELECT * FROM [53(5) AS t] ---- -error (42703): column [5] does not exist +project + └── scan t + └── columns: crdb_internal_mvcc_timestamp:5 build SELECT * FROM [53(2,4) AS t] @@ -1061,22 +1143,26 @@ SELECT * FROM [54 AS t] project ├── columns: x:1 y:2 └── scan t - └── columns: x:1 y:2 rowid:3!null + └── columns: x:1 y:2 rowid:3!null crdb_internal_mvcc_timestamp:4 # Verify that we force the given index. build SELECT * FROM [53 AS t]@[1] ---- -scan t +project ├── columns: a:1!null y:2 b:3 c:4 - └── flags: force-index=primary + └── scan t + ├── columns: a:1!null y:2 b:3 c:4 crdb_internal_mvcc_timestamp:5 + └── flags: force-index=primary build SELECT * FROM [53 AS t]@[2] ---- -scan t +project ├── columns: a:1!null y:2 b:3 c:4 - └── flags: force-index=bc + └── scan t + ├── columns: a:1!null y:2 b:3 c:4 crdb_internal_mvcc_timestamp:5 + └── flags: force-index=bc # Test that hidden columns are not presented. build @@ -1131,7 +1217,7 @@ SELECT * FROM no_cols_table ---- project └── scan no_cols_table - └── columns: rowid:1!null + └── columns: rowid:1!null crdb_internal_mvcc_timestamp:2 build SELECT * FROM [54(3) as t] @@ -1145,73 +1231,77 @@ build WITH cte AS (SELECT b FROM [INSERT INTO abc VALUES (1) RETURNING *] LIMIT 1) SELECT * FROM abc ---- with &1 - ├── columns: a:9!null b:10 c:11 + ├── columns: a:10!null b:11 c:12 ├── insert abc │ ├── columns: abc.a:1!null abc.b:2 abc.c:3 │ ├── insert-mapping: - │ │ ├── column1:4 => abc.a:1 - │ │ ├── column5:5 => abc.b:2 - │ │ └── column5:5 => abc.c:3 + │ │ ├── column1:5 => abc.a:1 + │ │ ├── column6:6 => abc.b:2 + │ │ └── column6:6 => abc.c:3 │ └── project - │ ├── columns: column5:5 column1:4!null + │ ├── columns: column6:6 column1:5!null │ ├── values - │ │ ├── columns: column1:4!null + │ │ ├── columns: column1:5!null │ │ └── (1,) │ └── projections - │ └── NULL::INT8 [as=column5:5] + │ └── NULL::INT8 [as=column6:6] └── with &2 (cte) - ├── columns: abc.a:9!null abc.b:10 abc.c:11 + ├── columns: abc.a:10!null abc.b:11 abc.c:12 ├── limit - │ ├── columns: b:7 + │ ├── columns: b:8 │ ├── project - │ │ ├── columns: b:7 + │ │ ├── columns: b:8 │ │ ├── limit hint: 1.00 │ │ └── with-scan &1 - │ │ ├── columns: a:6!null b:7 c:8 + │ │ ├── columns: a:7!null b:8 c:9 │ │ ├── mapping: - │ │ │ ├── abc.a:1 => a:6 - │ │ │ ├── abc.b:2 => b:7 - │ │ │ └── abc.c:3 => c:8 + │ │ │ ├── abc.a:1 => a:7 + │ │ │ ├── abc.b:2 => b:8 + │ │ │ └── abc.c:3 => c:9 │ │ └── limit hint: 1.00 │ └── 1 - └── scan abc - └── columns: abc.a:9!null abc.b:10 abc.c:11 + └── project + ├── columns: abc.a:10!null abc.b:11 abc.c:12 + └── scan abc + └── columns: abc.a:10!null abc.b:11 abc.c:12 crdb_internal_mvcc_timestamp:13 # Tests for the square bracket syntax. build SELECT * FROM [SELECT * FROM abc] ---- with &1 - ├── columns: a:4!null b:5 c:6 - ├── scan abc - │ └── columns: abc.a:1!null abc.b:2 abc.c:3 + ├── columns: a:5!null b:6 c:7 + ├── project + │ ├── columns: abc.a:1!null abc.b:2 abc.c:3 + │ └── scan abc + │ └── columns: abc.a:1!null abc.b:2 abc.c:3 crdb_internal_mvcc_timestamp:4 └── with-scan &1 - ├── columns: a:4!null b:5 c:6 + ├── columns: a:5!null b:6 c:7 └── mapping: - ├── abc.a:1 => a:4 - ├── abc.b:2 => b:5 - └── abc.c:3 => c:6 + ├── abc.a:1 => a:5 + ├── abc.b:2 => b:6 + └── abc.c:3 => c:7 build SELECT * FROM [INSERT INTO abc VALUES (1, 2, 3) RETURNING a] ---- with &1 - ├── columns: a:7!null + ├── columns: a:8!null ├── project │ ├── columns: abc.a:1!null │ └── insert abc │ ├── columns: abc.a:1!null b:2!null c:3!null │ ├── insert-mapping: - │ │ ├── column1:4 => abc.a:1 - │ │ ├── column2:5 => b:2 - │ │ └── column3:6 => c:3 + │ │ ├── column1:5 => abc.a:1 + │ │ ├── column2:6 => b:2 + │ │ └── column3:7 => c:3 │ └── values - │ ├── columns: column1:4!null column2:5!null column3:6!null + │ ├── columns: column1:5!null column2:6!null column3:7!null │ └── (1, 2, 3) └── with-scan &1 - ├── columns: a:7!null + ├── columns: a:8!null └── mapping: - └── abc.a:1 => a:7 + └── abc.a:1 => a:8 # Statement inside brackets cannot refer to outer column. build @@ -1236,7 +1326,7 @@ build WITH cte AS (SELECT 1) SELECT 1 + (SELECT * FROM cte) FROM [SELECT * from xyzw] ---- with &1 (cte) - ├── columns: "?column?":11 + ├── columns: "?column?":12 ├── project │ ├── columns: "?column?":1!null │ ├── values @@ -1244,55 +1334,59 @@ with &1 (cte) │ └── projections │ └── 1 [as="?column?":1] └── with &2 - ├── columns: "?column?":11 - ├── scan xyzw - │ └── columns: xyzw.x:2!null xyzw.y:3 xyzw.z:4 xyzw.w:5 + ├── columns: "?column?":12 + ├── project + │ ├── columns: xyzw.x:2!null xyzw.y:3 xyzw.z:4 xyzw.w:5 + │ └── scan xyzw + │ └── columns: xyzw.x:2!null xyzw.y:3 xyzw.z:4 xyzw.w:5 crdb_internal_mvcc_timestamp:6 └── project - ├── columns: "?column?":11 + ├── columns: "?column?":12 ├── with-scan &2 - │ ├── columns: x:6!null y:7 z:8 w:9 + │ ├── columns: x:7!null y:8 z:9 w:10 │ └── mapping: - │ ├── xyzw.x:2 => x:6 - │ ├── xyzw.y:3 => y:7 - │ ├── xyzw.z:4 => z:8 - │ └── xyzw.w:5 => w:9 + │ ├── xyzw.x:2 => x:7 + │ ├── xyzw.y:3 => y:8 + │ ├── xyzw.z:4 => z:9 + │ └── xyzw.w:5 => w:10 └── projections - └── plus [as="?column?":11] + └── plus [as="?column?":12] ├── 1 └── subquery └── max1-row - ├── columns: "?column?":10!null + ├── columns: "?column?":11!null └── with-scan &1 (cte) - ├── columns: "?column?":10!null + ├── columns: "?column?":11!null └── mapping: - └── "?column?":1 => "?column?":10 + └── "?column?":1 => "?column?":11 build SELECT a, (SELECT a+x FROM [SELECT * from xyzw]) FROM abc ---- with &1 - ├── columns: a:1!null "?column?":13 - ├── scan xyzw - │ └── columns: xyzw.x:4!null xyzw.y:5 xyzw.z:6 xyzw.w:7 + ├── columns: a:1!null "?column?":15 + ├── project + │ ├── columns: xyzw.x:5!null xyzw.y:6 xyzw.z:7 xyzw.w:8 + │ └── scan xyzw + │ └── columns: xyzw.x:5!null xyzw.y:6 xyzw.z:7 xyzw.w:8 xyzw.crdb_internal_mvcc_timestamp:9 └── project - ├── columns: "?column?":13 a:1!null + ├── columns: "?column?":15 a:1!null ├── scan abc - │ └── columns: a:1!null b:2 c:3 + │ └── columns: a:1!null b:2 c:3 abc.crdb_internal_mvcc_timestamp:4 └── projections - └── subquery [as="?column?":13] + └── subquery [as="?column?":15] └── max1-row - ├── columns: "?column?":12 + ├── columns: "?column?":14 └── project - ├── columns: "?column?":12 + ├── columns: "?column?":14 ├── with-scan &1 - │ ├── columns: x:8!null y:9 z:10 w:11 + │ ├── columns: x:10!null y:11 z:12 w:13 │ └── mapping: - │ ├── xyzw.x:4 => x:8 - │ ├── xyzw.y:5 => y:9 - │ ├── xyzw.z:6 => z:10 - │ └── xyzw.w:7 => w:11 + │ ├── xyzw.x:5 => x:10 + │ ├── xyzw.y:6 => y:11 + │ ├── xyzw.z:7 => z:12 + │ └── xyzw.w:8 => w:13 └── projections - └── a:1 + x:8 [as="?column?":12] + └── a:1 + x:10 [as="?column?":14] # Populates table metadata with partial index predicates. exec-ddl @@ -1312,7 +1406,7 @@ SELECT k FROM partial_index project ├── columns: k:1!null └── scan partial_index - ├── columns: k:1!null u:2 v:3 + ├── columns: k:1!null u:2 v:3 crdb_internal_mvcc_timestamp:4 └── partial index predicates ├── u: filters │ └── u:2 = 1 diff --git a/pkg/sql/opt/optbuilder/testdata/select_for_update b/pkg/sql/opt/optbuilder/testdata/select_for_update index 2f661f5f10ba..c318da6601fd 100644 --- a/pkg/sql/opt/optbuilder/testdata/select_for_update +++ b/pkg/sql/opt/optbuilder/testdata/select_for_update @@ -17,58 +17,74 @@ CREATE VIEW v AS SELECT a FROM t AS t2 build SELECT * FROM t FOR UPDATE ---- -scan t +project ├── columns: a:1!null b:2 - └── locking: for-update + └── scan t + ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 + └── locking: for-update build SELECT * FROM t FOR NO KEY UPDATE ---- -scan t +project ├── columns: a:1!null b:2 - └── locking: for-no-key-update + └── scan t + ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 + └── locking: for-no-key-update build SELECT * FROM t FOR SHARE ---- -scan t +project ├── columns: a:1!null b:2 - └── locking: for-share + └── scan t + ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 + └── locking: for-share build SELECT * FROM t FOR KEY SHARE ---- -scan t +project ├── columns: a:1!null b:2 - └── locking: for-key-share + └── scan t + ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 + └── locking: for-key-share build SELECT * FROM t FOR KEY SHARE FOR SHARE ---- -scan t +project ├── columns: a:1!null b:2 - └── locking: for-share + └── scan t + ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 + └── locking: for-share build SELECT * FROM t FOR KEY SHARE FOR SHARE FOR NO KEY UPDATE ---- -scan t +project ├── columns: a:1!null b:2 - └── locking: for-no-key-update + └── scan t + ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 + └── locking: for-no-key-update build SELECT * FROM t FOR KEY SHARE FOR SHARE FOR NO KEY UPDATE FOR UPDATE ---- -scan t +project ├── columns: a:1!null b:2 - └── locking: for-update + └── scan t + ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 + └── locking: for-update build SELECT * FROM t FOR UPDATE OF t ---- -scan t +project ├── columns: a:1!null b:2 - └── locking: for-update + └── scan t + ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 + └── locking: for-update build SELECT * FROM t FOR UPDATE OF t2 @@ -79,12 +95,12 @@ build SELECT 1 FROM t FOR UPDATE OF t ---- project - ├── columns: "?column?":3!null + ├── columns: "?column?":4!null ├── scan t - │ ├── columns: a:1!null b:2 + │ ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 │ └── locking: for-update └── projections - └── 1 [as="?column?":3] + └── 1 [as="?column?":4] # ------------------------------------------------------------------------------ # Tests with table aliases. @@ -93,9 +109,11 @@ project build SELECT * FROM t AS t2 FOR UPDATE ---- -scan t2 +project ├── columns: a:1!null b:2 - └── locking: for-update + └── scan t2 + ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 + └── locking: for-update build SELECT * FROM t AS t2 FOR UPDATE OF t @@ -105,9 +123,11 @@ error (42P01): relation "t" in FOR UPDATE clause not found in FROM clause build SELECT * FROM t AS t2 FOR UPDATE OF t2 ---- -scan t2 +project ├── columns: a:1!null b:2 - └── locking: for-update + └── scan t2 + ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 + └── locking: for-update # ------------------------------------------------------------------------------ # Tests with numeric table references. @@ -117,16 +137,20 @@ scan t2 build SELECT * FROM [53 AS t] FOR UPDATE ---- -scan t +project ├── columns: a:1!null b:2 - └── locking: for-update + └── scan t + ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 + └── locking: for-update build SELECT * FROM [53 AS t] FOR UPDATE OF t ---- -scan t +project ├── columns: a:1!null b:2 - └── locking: for-update + └── scan t + ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 + └── locking: for-update build SELECT * FROM [53 AS t] FOR UPDATE OF t2 @@ -143,7 +167,7 @@ SELECT * FROM v FOR UPDATE project ├── columns: a:1!null └── scan t2 - ├── columns: a:1!null b:2 + ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 └── locking: for-update build @@ -152,7 +176,7 @@ SELECT * FROM v FOR UPDATE OF v project ├── columns: a:1!null └── scan t2 - ├── columns: a:1!null b:2 + ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 └── locking: for-update build @@ -180,7 +204,7 @@ SELECT * FROM v AS v2 FOR UPDATE project ├── columns: a:1!null └── scan t2 - ├── columns: a:1!null b:2 + ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 └── locking: for-update build @@ -194,7 +218,7 @@ SELECT * FROM v AS v2 FOR UPDATE OF v2 project ├── columns: a:1!null └── scan t2 - ├── columns: a:1!null b:2 + ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 └── locking: for-update # ------------------------------------------------------------------------------ @@ -211,7 +235,7 @@ SELECT * FROM (SELECT a FROM t) FOR UPDATE project ├── columns: a:1!null └── scan t - ├── columns: a:1!null b:2 + ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 └── locking: for-update build @@ -220,7 +244,7 @@ SELECT * FROM (SELECT a FROM t FOR UPDATE) project ├── columns: a:1!null └── scan t - ├── columns: a:1!null b:2 + ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 └── locking: for-update build @@ -229,7 +253,7 @@ SELECT * FROM (SELECT a FROM t FOR NO KEY UPDATE) FOR KEY SHARE project ├── columns: a:1!null └── scan t - ├── columns: a:1!null b:2 + ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 └── locking: for-no-key-update build @@ -238,7 +262,7 @@ SELECT * FROM (SELECT a FROM t FOR KEY SHARE) FOR NO KEY UPDATE project ├── columns: a:1!null └── scan t - ├── columns: a:1!null b:2 + ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 └── locking: for-no-key-update build @@ -252,7 +276,7 @@ SELECT * FROM (SELECT a FROM t FOR UPDATE OF t) project ├── columns: a:1!null └── scan t - ├── columns: a:1!null b:2 + ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 └── locking: for-update build @@ -261,7 +285,7 @@ SELECT * FROM (SELECT a FROM t) AS r FOR UPDATE project ├── columns: a:1!null └── scan t - ├── columns: a:1!null b:2 + ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 └── locking: for-update build @@ -270,7 +294,7 @@ SELECT * FROM (SELECT a FROM t FOR UPDATE) AS r project ├── columns: a:1!null └── scan t - ├── columns: a:1!null b:2 + ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 └── locking: for-update build @@ -284,40 +308,40 @@ SELECT * FROM (SELECT a FROM t FOR UPDATE OF t) AS r project ├── columns: a:1!null └── scan t - ├── columns: a:1!null b:2 + ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 └── locking: for-update build SELECT (SELECT a FROM t) FOR UPDATE ---- project - ├── columns: a:3 + ├── columns: a:4 ├── values │ └── () └── projections - └── subquery [as=a:3] + └── subquery [as=a:4] └── max1-row ├── columns: t.a:1!null └── project ├── columns: t.a:1!null └── scan t - └── columns: t.a:1!null b:2 + └── columns: t.a:1!null b:2 crdb_internal_mvcc_timestamp:3 build SELECT (SELECT a FROM t FOR UPDATE) ---- project - ├── columns: a:3 + ├── columns: a:4 ├── values │ └── () └── projections - └── subquery [as=a:3] + └── subquery [as=a:4] └── max1-row ├── columns: t.a:1!null └── project ├── columns: t.a:1!null └── scan t - ├── columns: t.a:1!null b:2 + ├── columns: t.a:1!null b:2 crdb_internal_mvcc_timestamp:3 └── locking: for-update build @@ -329,50 +353,50 @@ build SELECT (SELECT a FROM t FOR UPDATE OF t) ---- project - ├── columns: a:3 + ├── columns: a:4 ├── values │ └── () └── projections - └── subquery [as=a:3] + └── subquery [as=a:4] └── max1-row ├── columns: t.a:1!null └── project ├── columns: t.a:1!null └── scan t - ├── columns: t.a:1!null b:2 + ├── columns: t.a:1!null b:2 crdb_internal_mvcc_timestamp:3 └── locking: for-update build SELECT (SELECT a FROM t) AS r FOR UPDATE ---- project - ├── columns: r:3 + ├── columns: r:4 ├── values │ └── () └── projections - └── subquery [as=r:3] + └── subquery [as=r:4] └── max1-row ├── columns: a:1!null └── project ├── columns: a:1!null └── scan t - └── columns: a:1!null b:2 + └── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 build SELECT (SELECT a FROM t FOR UPDATE) AS r ---- project - ├── columns: r:3 + ├── columns: r:4 ├── values │ └── () └── projections - └── subquery [as=r:3] + └── subquery [as=r:4] └── max1-row ├── columns: a:1!null └── project ├── columns: a:1!null └── scan t - ├── columns: a:1!null b:2 + ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 └── locking: for-update build @@ -384,82 +408,90 @@ build SELECT (SELECT a FROM t FOR UPDATE OF t) AS r ---- project - ├── columns: r:3 + ├── columns: r:4 ├── values │ └── () └── projections - └── subquery [as=r:3] + └── subquery [as=r:4] └── max1-row ├── columns: a:1!null └── project ├── columns: a:1!null └── scan t - ├── columns: a:1!null b:2 + ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 └── locking: for-update build SELECT * FROM t WHERE a IN (SELECT a FROM t) FOR UPDATE ---- -select +project ├── columns: a:1!null b:2 - ├── scan t - │ ├── columns: a:1!null b:2 - │ └── locking: for-update - └── filters - └── any: eq - ├── project - │ ├── columns: a:3!null - │ └── scan t - │ └── columns: a:3!null b:4 - └── a:1 + └── select + ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 + ├── scan t + │ ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 + │ └── locking: for-update + └── filters + └── any: eq + ├── project + │ ├── columns: a:4!null + │ └── scan t + │ └── columns: a:4!null b:5 crdb_internal_mvcc_timestamp:6 + └── a:1 build SELECT * FROM t WHERE a IN (SELECT a FROM t FOR UPDATE) ---- -select +project ├── columns: a:1!null b:2 - ├── scan t - │ └── columns: a:1!null b:2 - └── filters - └── any: eq - ├── project - │ ├── columns: a:3!null - │ └── scan t - │ ├── columns: a:3!null b:4 - │ └── locking: for-update - └── a:1 + └── select + ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 + ├── scan t + │ └── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 + └── filters + └── any: eq + ├── project + │ ├── columns: a:4!null + │ └── scan t + │ ├── columns: a:4!null b:5 crdb_internal_mvcc_timestamp:6 + │ └── locking: for-update + └── a:1 build SELECT * FROM t WHERE a IN (SELECT a FROM t) FOR UPDATE OF t ---- -select +project ├── columns: a:1!null b:2 - ├── scan t - │ ├── columns: a:1!null b:2 - │ └── locking: for-update - └── filters - └── any: eq - ├── project - │ ├── columns: a:3!null - │ └── scan t - │ └── columns: a:3!null b:4 - └── a:1 + └── select + ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 + ├── scan t + │ ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 + │ └── locking: for-update + └── filters + └── any: eq + ├── project + │ ├── columns: a:4!null + │ └── scan t + │ └── columns: a:4!null b:5 crdb_internal_mvcc_timestamp:6 + └── a:1 build SELECT * FROM t WHERE a IN (SELECT a FROM t FOR UPDATE OF t) ---- -select +project ├── columns: a:1!null b:2 - ├── scan t - │ └── columns: a:1!null b:2 - └── filters - └── any: eq - ├── project - │ ├── columns: a:3!null - │ └── scan t - │ ├── columns: a:3!null b:4 - │ └── locking: for-update - └── a:1 + └── select + ├── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 + ├── scan t + │ └── columns: a:1!null b:2 crdb_internal_mvcc_timestamp:3 + └── filters + └── any: eq + ├── project + │ ├── columns: a:4!null + │ └── scan t + │ ├── columns: a:4!null b:5 crdb_internal_mvcc_timestamp:6 + │ └── locking: for-update + └── a:1 # ------------------------------------------------------------------------------ # Tests with common-table expressions. @@ -473,59 +505,59 @@ build SELECT * FROM [SELECT a FROM t] FOR UPDATE ---- with &1 - ├── columns: a:3!null + ├── columns: a:4!null ├── project │ ├── columns: t.a:1!null │ └── scan t - │ └── columns: t.a:1!null b:2 + │ └── columns: t.a:1!null b:2 crdb_internal_mvcc_timestamp:3 └── with-scan &1 - ├── columns: a:3!null + ├── columns: a:4!null └── mapping: - └── t.a:1 => a:3 + └── t.a:1 => a:4 build WITH cte AS (SELECT a FROM t) SELECT * FROM cte FOR UPDATE ---- with &1 (cte) - ├── columns: a:3!null + ├── columns: a:4!null ├── project │ ├── columns: t.a:1!null │ └── scan t - │ └── columns: t.a:1!null b:2 + │ └── columns: t.a:1!null b:2 crdb_internal_mvcc_timestamp:3 └── with-scan &1 (cte) - ├── columns: a:3!null + ├── columns: a:4!null └── mapping: - └── t.a:1 => a:3 + └── t.a:1 => a:4 build SELECT * FROM [SELECT a FROM t FOR UPDATE] ---- with &1 - ├── columns: a:3!null + ├── columns: a:4!null ├── project │ ├── columns: t.a:1!null │ └── scan t - │ ├── columns: t.a:1!null b:2 + │ ├── columns: t.a:1!null b:2 crdb_internal_mvcc_timestamp:3 │ └── locking: for-update └── with-scan &1 - ├── columns: a:3!null + ├── columns: a:4!null └── mapping: - └── t.a:1 => a:3 + └── t.a:1 => a:4 build WITH cte AS (SELECT a FROM t FOR UPDATE) SELECT * FROM cte ---- with &1 (cte) - ├── columns: a:3!null + ├── columns: a:4!null ├── project │ ├── columns: t.a:1!null │ └── scan t - │ ├── columns: t.a:1!null b:2 + │ ├── columns: t.a:1!null b:2 crdb_internal_mvcc_timestamp:3 │ └── locking: for-update └── with-scan &1 (cte) - ├── columns: a:3!null + ├── columns: a:4!null └── mapping: - └── t.a:1 => a:3 + └── t.a:1 => a:4 # ------------------------------------------------------------------------------ # Tests with joins. @@ -535,79 +567,79 @@ build SELECT * FROM t JOIN u USING (a) FOR UPDATE ---- project - ├── columns: a:1!null b:2 c:4 + ├── columns: a:1!null b:2 c:5 └── inner-join (hash) - ├── columns: t.a:1!null b:2 u.a:3!null c:4 + ├── columns: t.a:1!null b:2 t.crdb_internal_mvcc_timestamp:3 u.a:4!null c:5 u.crdb_internal_mvcc_timestamp:6 ├── scan t - │ ├── columns: t.a:1!null b:2 + │ ├── columns: t.a:1!null b:2 t.crdb_internal_mvcc_timestamp:3 │ └── locking: for-update ├── scan u - │ ├── columns: u.a:3!null c:4 + │ ├── columns: u.a:4!null c:5 u.crdb_internal_mvcc_timestamp:6 │ └── locking: for-update └── filters - └── t.a:1 = u.a:3 + └── t.a:1 = u.a:4 build SELECT * FROM t JOIN u USING (a) FOR UPDATE OF t ---- project - ├── columns: a:1!null b:2 c:4 + ├── columns: a:1!null b:2 c:5 └── inner-join (hash) - ├── columns: t.a:1!null b:2 u.a:3!null c:4 + ├── columns: t.a:1!null b:2 t.crdb_internal_mvcc_timestamp:3 u.a:4!null c:5 u.crdb_internal_mvcc_timestamp:6 ├── scan t - │ ├── columns: t.a:1!null b:2 + │ ├── columns: t.a:1!null b:2 t.crdb_internal_mvcc_timestamp:3 │ └── locking: for-update ├── scan u - │ └── columns: u.a:3!null c:4 + │ └── columns: u.a:4!null c:5 u.crdb_internal_mvcc_timestamp:6 └── filters - └── t.a:1 = u.a:3 + └── t.a:1 = u.a:4 build SELECT * FROM t JOIN u USING (a) FOR UPDATE OF u ---- project - ├── columns: a:1!null b:2 c:4 + ├── columns: a:1!null b:2 c:5 └── inner-join (hash) - ├── columns: t.a:1!null b:2 u.a:3!null c:4 + ├── columns: t.a:1!null b:2 t.crdb_internal_mvcc_timestamp:3 u.a:4!null c:5 u.crdb_internal_mvcc_timestamp:6 ├── scan t - │ └── columns: t.a:1!null b:2 + │ └── columns: t.a:1!null b:2 t.crdb_internal_mvcc_timestamp:3 ├── scan u - │ ├── columns: u.a:3!null c:4 + │ ├── columns: u.a:4!null c:5 u.crdb_internal_mvcc_timestamp:6 │ └── locking: for-update └── filters - └── t.a:1 = u.a:3 + └── t.a:1 = u.a:4 build SELECT * FROM t JOIN u USING (a) FOR UPDATE OF t, u ---- project - ├── columns: a:1!null b:2 c:4 + ├── columns: a:1!null b:2 c:5 └── inner-join (hash) - ├── columns: t.a:1!null b:2 u.a:3!null c:4 + ├── columns: t.a:1!null b:2 t.crdb_internal_mvcc_timestamp:3 u.a:4!null c:5 u.crdb_internal_mvcc_timestamp:6 ├── scan t - │ ├── columns: t.a:1!null b:2 + │ ├── columns: t.a:1!null b:2 t.crdb_internal_mvcc_timestamp:3 │ └── locking: for-update ├── scan u - │ ├── columns: u.a:3!null c:4 + │ ├── columns: u.a:4!null c:5 u.crdb_internal_mvcc_timestamp:6 │ └── locking: for-update └── filters - └── t.a:1 = u.a:3 + └── t.a:1 = u.a:4 build SELECT * FROM t JOIN u USING (a) FOR UPDATE OF t FOR SHARE OF u ---- project - ├── columns: a:1!null b:2 c:4 + ├── columns: a:1!null b:2 c:5 └── inner-join (hash) - ├── columns: t.a:1!null b:2 u.a:3!null c:4 + ├── columns: t.a:1!null b:2 t.crdb_internal_mvcc_timestamp:3 u.a:4!null c:5 u.crdb_internal_mvcc_timestamp:6 ├── scan t - │ ├── columns: t.a:1!null b:2 + │ ├── columns: t.a:1!null b:2 t.crdb_internal_mvcc_timestamp:3 │ └── locking: for-update ├── scan u - │ ├── columns: u.a:3!null c:4 + │ ├── columns: u.a:4!null c:5 u.crdb_internal_mvcc_timestamp:6 │ └── locking: for-share └── filters - └── t.a:1 = u.a:3 + └── t.a:1 = u.a:4 build SELECT * FROM t JOIN u USING (a) FOR UPDATE OF t2 FOR SHARE OF u2 @@ -618,65 +650,65 @@ build SELECT * FROM t AS t2 JOIN u AS u2 USING (a) FOR UPDATE OF t2 FOR SHARE OF u2 ---- project - ├── columns: a:1!null b:2 c:4 + ├── columns: a:1!null b:2 c:5 └── inner-join (hash) - ├── columns: t2.a:1!null b:2 u2.a:3!null c:4 + ├── columns: t2.a:1!null b:2 t2.crdb_internal_mvcc_timestamp:3 u2.a:4!null c:5 u2.crdb_internal_mvcc_timestamp:6 ├── scan t2 - │ ├── columns: t2.a:1!null b:2 + │ ├── columns: t2.a:1!null b:2 t2.crdb_internal_mvcc_timestamp:3 │ └── locking: for-update ├── scan u2 - │ ├── columns: u2.a:3!null c:4 + │ ├── columns: u2.a:4!null c:5 u2.crdb_internal_mvcc_timestamp:6 │ └── locking: for-share └── filters - └── t2.a:1 = u2.a:3 + └── t2.a:1 = u2.a:4 build SELECT * FROM t JOIN u USING (a) FOR KEY SHARE FOR UPDATE ---- project - ├── columns: a:1!null b:2 c:4 + ├── columns: a:1!null b:2 c:5 └── inner-join (hash) - ├── columns: t.a:1!null b:2 u.a:3!null c:4 + ├── columns: t.a:1!null b:2 t.crdb_internal_mvcc_timestamp:3 u.a:4!null c:5 u.crdb_internal_mvcc_timestamp:6 ├── scan t - │ ├── columns: t.a:1!null b:2 + │ ├── columns: t.a:1!null b:2 t.crdb_internal_mvcc_timestamp:3 │ └── locking: for-update ├── scan u - │ ├── columns: u.a:3!null c:4 + │ ├── columns: u.a:4!null c:5 u.crdb_internal_mvcc_timestamp:6 │ └── locking: for-update └── filters - └── t.a:1 = u.a:3 + └── t.a:1 = u.a:4 build SELECT * FROM t JOIN u USING (a) FOR KEY SHARE FOR NO KEY UPDATE OF t ---- project - ├── columns: a:1!null b:2 c:4 + ├── columns: a:1!null b:2 c:5 └── inner-join (hash) - ├── columns: t.a:1!null b:2 u.a:3!null c:4 + ├── columns: t.a:1!null b:2 t.crdb_internal_mvcc_timestamp:3 u.a:4!null c:5 u.crdb_internal_mvcc_timestamp:6 ├── scan t - │ ├── columns: t.a:1!null b:2 + │ ├── columns: t.a:1!null b:2 t.crdb_internal_mvcc_timestamp:3 │ └── locking: for-no-key-update ├── scan u - │ ├── columns: u.a:3!null c:4 + │ ├── columns: u.a:4!null c:5 u.crdb_internal_mvcc_timestamp:6 │ └── locking: for-key-share └── filters - └── t.a:1 = u.a:3 + └── t.a:1 = u.a:4 build SELECT * FROM t JOIN u USING (a) FOR SHARE FOR NO KEY UPDATE OF t FOR UPDATE OF u ---- project - ├── columns: a:1!null b:2 c:4 + ├── columns: a:1!null b:2 c:5 └── inner-join (hash) - ├── columns: t.a:1!null b:2 u.a:3!null c:4 + ├── columns: t.a:1!null b:2 t.crdb_internal_mvcc_timestamp:3 u.a:4!null c:5 u.crdb_internal_mvcc_timestamp:6 ├── scan t - │ ├── columns: t.a:1!null b:2 + │ ├── columns: t.a:1!null b:2 t.crdb_internal_mvcc_timestamp:3 │ └── locking: for-no-key-update ├── scan u - │ ├── columns: u.a:3!null c:4 + │ ├── columns: u.a:4!null c:5 u.crdb_internal_mvcc_timestamp:6 │ └── locking: for-update └── filters - └── t.a:1 = u.a:3 + └── t.a:1 = u.a:4 # ------------------------------------------------------------------------------ # Tests with joins of aliased tables and aliased joins. @@ -686,17 +718,17 @@ build SELECT * FROM t AS t2 JOIN u AS u2 USING (a) FOR UPDATE ---- project - ├── columns: a:1!null b:2 c:4 + ├── columns: a:1!null b:2 c:5 └── inner-join (hash) - ├── columns: t2.a:1!null b:2 u2.a:3!null c:4 + ├── columns: t2.a:1!null b:2 t2.crdb_internal_mvcc_timestamp:3 u2.a:4!null c:5 u2.crdb_internal_mvcc_timestamp:6 ├── scan t2 - │ ├── columns: t2.a:1!null b:2 + │ ├── columns: t2.a:1!null b:2 t2.crdb_internal_mvcc_timestamp:3 │ └── locking: for-update ├── scan u2 - │ ├── columns: u2.a:3!null c:4 + │ ├── columns: u2.a:4!null c:5 u2.crdb_internal_mvcc_timestamp:6 │ └── locking: for-update └── filters - └── t2.a:1 = u2.a:3 + └── t2.a:1 = u2.a:4 build SELECT * FROM t AS t2 JOIN u AS u2 USING (a) FOR UPDATE OF t @@ -717,47 +749,47 @@ build SELECT * FROM t AS t2 JOIN u AS u2 USING (a) FOR UPDATE OF t2 ---- project - ├── columns: a:1!null b:2 c:4 + ├── columns: a:1!null b:2 c:5 └── inner-join (hash) - ├── columns: t2.a:1!null b:2 u2.a:3!null c:4 + ├── columns: t2.a:1!null b:2 t2.crdb_internal_mvcc_timestamp:3 u2.a:4!null c:5 u2.crdb_internal_mvcc_timestamp:6 ├── scan t2 - │ ├── columns: t2.a:1!null b:2 + │ ├── columns: t2.a:1!null b:2 t2.crdb_internal_mvcc_timestamp:3 │ └── locking: for-update ├── scan u2 - │ └── columns: u2.a:3!null c:4 + │ └── columns: u2.a:4!null c:5 u2.crdb_internal_mvcc_timestamp:6 └── filters - └── t2.a:1 = u2.a:3 + └── t2.a:1 = u2.a:4 build SELECT * FROM t AS t2 JOIN u AS u2 USING (a) FOR UPDATE OF u2 ---- project - ├── columns: a:1!null b:2 c:4 + ├── columns: a:1!null b:2 c:5 └── inner-join (hash) - ├── columns: t2.a:1!null b:2 u2.a:3!null c:4 + ├── columns: t2.a:1!null b:2 t2.crdb_internal_mvcc_timestamp:3 u2.a:4!null c:5 u2.crdb_internal_mvcc_timestamp:6 ├── scan t2 - │ └── columns: t2.a:1!null b:2 + │ └── columns: t2.a:1!null b:2 t2.crdb_internal_mvcc_timestamp:3 ├── scan u2 - │ ├── columns: u2.a:3!null c:4 + │ ├── columns: u2.a:4!null c:5 u2.crdb_internal_mvcc_timestamp:6 │ └── locking: for-update └── filters - └── t2.a:1 = u2.a:3 + └── t2.a:1 = u2.a:4 build SELECT * FROM t AS t2 JOIN u AS u2 USING (a) FOR UPDATE OF t2, u2 ---- project - ├── columns: a:1!null b:2 c:4 + ├── columns: a:1!null b:2 c:5 └── inner-join (hash) - ├── columns: t2.a:1!null b:2 u2.a:3!null c:4 + ├── columns: t2.a:1!null b:2 t2.crdb_internal_mvcc_timestamp:3 u2.a:4!null c:5 u2.crdb_internal_mvcc_timestamp:6 ├── scan t2 - │ ├── columns: t2.a:1!null b:2 + │ ├── columns: t2.a:1!null b:2 t2.crdb_internal_mvcc_timestamp:3 │ └── locking: for-update ├── scan u2 - │ ├── columns: u2.a:3!null c:4 + │ ├── columns: u2.a:4!null c:5 u2.crdb_internal_mvcc_timestamp:6 │ └── locking: for-update └── filters - └── t2.a:1 = u2.a:3 + └── t2.a:1 = u2.a:4 # Postgres doesn't support applying locking clauses to joins. The following @@ -768,17 +800,17 @@ build SELECT * FROM (t JOIN u AS u2 USING (a)) j FOR UPDATE ---- project - ├── columns: a:1!null b:2 c:4 + ├── columns: a:1!null b:2 c:5 └── inner-join (hash) - ├── columns: t.a:1!null b:2 u2.a:3!null c:4 + ├── columns: t.a:1!null b:2 t.crdb_internal_mvcc_timestamp:3 u2.a:4!null c:5 u2.crdb_internal_mvcc_timestamp:6 ├── scan t - │ ├── columns: t.a:1!null b:2 + │ ├── columns: t.a:1!null b:2 t.crdb_internal_mvcc_timestamp:3 │ └── locking: for-update ├── scan u2 - │ ├── columns: u2.a:3!null c:4 + │ ├── columns: u2.a:4!null c:5 u2.crdb_internal_mvcc_timestamp:6 │ └── locking: for-update └── filters - └── t.a:1 = u2.a:3 + └── t.a:1 = u2.a:4 build SELECT * FROM (t JOIN u AS u2 USING (a)) j FOR UPDATE OF t @@ -799,17 +831,17 @@ build SELECT * FROM (t JOIN u AS u2 USING (a)) j FOR UPDATE OF j ---- project - ├── columns: a:1!null b:2 c:4 + ├── columns: a:1!null b:2 c:5 └── inner-join (hash) - ├── columns: t.a:1!null b:2 u2.a:3!null c:4 + ├── columns: t.a:1!null b:2 t.crdb_internal_mvcc_timestamp:3 u2.a:4!null c:5 u2.crdb_internal_mvcc_timestamp:6 ├── scan t - │ ├── columns: t.a:1!null b:2 + │ ├── columns: t.a:1!null b:2 t.crdb_internal_mvcc_timestamp:3 │ └── locking: for-update ├── scan u2 - │ ├── columns: u2.a:3!null c:4 + │ ├── columns: u2.a:4!null c:5 u2.crdb_internal_mvcc_timestamp:6 │ └── locking: for-update └── filters - └── t.a:1 = u2.a:3 + └── t.a:1 = u2.a:4 # ------------------------------------------------------------------------------ # Tests with lateral joins. @@ -818,53 +850,63 @@ project build SELECT * FROM t, u FOR UPDATE ---- -inner-join (cross) - ├── columns: a:1!null b:2 a:3!null c:4 - ├── scan t - │ ├── columns: t.a:1!null b:2 - │ └── locking: for-update - ├── scan u - │ ├── columns: u.a:3!null c:4 - │ └── locking: for-update - └── filters (true) +project + ├── columns: a:1!null b:2 a:4!null c:5 + └── inner-join (cross) + ├── columns: t.a:1!null b:2 t.crdb_internal_mvcc_timestamp:3 u.a:4!null c:5 u.crdb_internal_mvcc_timestamp:6 + ├── scan t + │ ├── columns: t.a:1!null b:2 t.crdb_internal_mvcc_timestamp:3 + │ └── locking: for-update + ├── scan u + │ ├── columns: u.a:4!null c:5 u.crdb_internal_mvcc_timestamp:6 + │ └── locking: for-update + └── filters (true) build SELECT * FROM t, u FOR UPDATE OF t ---- -inner-join (cross) - ├── columns: a:1!null b:2 a:3!null c:4 - ├── scan t - │ ├── columns: t.a:1!null b:2 - │ └── locking: for-update - ├── scan u - │ └── columns: u.a:3!null c:4 - └── filters (true) +project + ├── columns: a:1!null b:2 a:4!null c:5 + └── inner-join (cross) + ├── columns: t.a:1!null b:2 t.crdb_internal_mvcc_timestamp:3 u.a:4!null c:5 u.crdb_internal_mvcc_timestamp:6 + ├── scan t + │ ├── columns: t.a:1!null b:2 t.crdb_internal_mvcc_timestamp:3 + │ └── locking: for-update + ├── scan u + │ └── columns: u.a:4!null c:5 u.crdb_internal_mvcc_timestamp:6 + └── filters (true) build SELECT * FROM t, u FOR SHARE OF t FOR UPDATE OF u ---- -inner-join (cross) - ├── columns: a:1!null b:2 a:3!null c:4 - ├── scan t - │ ├── columns: t.a:1!null b:2 - │ └── locking: for-share - ├── scan u - │ ├── columns: u.a:3!null c:4 - │ └── locking: for-update - └── filters (true) +project + ├── columns: a:1!null b:2 a:4!null c:5 + └── inner-join (cross) + ├── columns: t.a:1!null b:2 t.crdb_internal_mvcc_timestamp:3 u.a:4!null c:5 u.crdb_internal_mvcc_timestamp:6 + ├── scan t + │ ├── columns: t.a:1!null b:2 t.crdb_internal_mvcc_timestamp:3 + │ └── locking: for-share + ├── scan u + │ ├── columns: u.a:4!null c:5 u.crdb_internal_mvcc_timestamp:6 + │ └── locking: for-update + └── filters (true) build SELECT * FROM t, LATERAL (SELECT * FROM u) sub FOR UPDATE ---- -inner-join-apply - ├── columns: a:1!null b:2 a:3!null c:4 - ├── scan t - │ ├── columns: t.a:1!null b:2 - │ └── locking: for-update - ├── scan sub - │ ├── columns: sub.a:3!null c:4 - │ └── locking: for-update - └── filters (true) +project + ├── columns: a:1!null b:2 a:4!null c:5 + └── inner-join-apply + ├── columns: t.a:1!null b:2 t.crdb_internal_mvcc_timestamp:3 u.a:4!null c:5 + ├── scan t + │ ├── columns: t.a:1!null b:2 t.crdb_internal_mvcc_timestamp:3 + │ └── locking: for-update + ├── project + │ ├── columns: u.a:4!null c:5 + │ └── scan u + │ ├── columns: u.a:4!null c:5 u.crdb_internal_mvcc_timestamp:6 + │ └── locking: for-update + └── filters (true) build SELECT * FROM t, LATERAL (SELECT * FROM u) sub FOR UPDATE OF u @@ -874,14 +916,18 @@ error (42P01): relation "u" in FOR UPDATE clause not found in FROM clause build SELECT * FROM t, LATERAL (SELECT * FROM u) sub FOR UPDATE OF sub ---- -inner-join-apply - ├── columns: a:1!null b:2 a:3!null c:4 - ├── scan t - │ └── columns: t.a:1!null b:2 - ├── scan sub - │ ├── columns: sub.a:3!null c:4 - │ └── locking: for-update - └── filters (true) +project + ├── columns: a:1!null b:2 a:4!null c:5 + └── inner-join-apply + ├── columns: t.a:1!null b:2 t.crdb_internal_mvcc_timestamp:3 u.a:4!null c:5 + ├── scan t + │ └── columns: t.a:1!null b:2 t.crdb_internal_mvcc_timestamp:3 + ├── project + │ ├── columns: u.a:4!null c:5 + │ └── scan u + │ ├── columns: u.a:4!null c:5 u.crdb_internal_mvcc_timestamp:6 + │ └── locking: for-update + └── filters (true) # ------------------------------------------------------------------------------ # Tests with virtual tables. diff --git a/pkg/sql/opt/optbuilder/testdata/srfs b/pkg/sql/opt/optbuilder/testdata/srfs index c7b41b12d580..8d501bd7e7d6 100644 --- a/pkg/sql/opt/optbuilder/testdata/srfs +++ b/pkg/sql/opt/optbuilder/testdata/srfs @@ -93,27 +93,27 @@ build SELECT t.*, u.*, a.*, b.* FROM t, u, generate_series(1, 2) AS a, generate_series(3, 4) AS b ---- project - ├── columns: a:1 b:3 a:5 b:6 + ├── columns: a:1 b:4 a:7 b:8 └── inner-join-apply - ├── columns: a:1 t.rowid:2!null b:3 u.rowid:4!null generate_series:5 generate_series:6 + ├── columns: a:1 t.rowid:2!null t.crdb_internal_mvcc_timestamp:3 b:4 u.rowid:5!null u.crdb_internal_mvcc_timestamp:6 generate_series:7 generate_series:8 ├── inner-join-apply - │ ├── columns: a:1 t.rowid:2!null b:3 u.rowid:4!null generate_series:5 + │ ├── columns: a:1 t.rowid:2!null t.crdb_internal_mvcc_timestamp:3 b:4 u.rowid:5!null u.crdb_internal_mvcc_timestamp:6 generate_series:7 │ ├── inner-join-apply - │ │ ├── columns: a:1 t.rowid:2!null b:3 u.rowid:4!null + │ │ ├── columns: a:1 t.rowid:2!null t.crdb_internal_mvcc_timestamp:3 b:4 u.rowid:5!null u.crdb_internal_mvcc_timestamp:6 │ │ ├── scan t - │ │ │ └── columns: a:1 t.rowid:2!null + │ │ │ └── columns: a:1 t.rowid:2!null t.crdb_internal_mvcc_timestamp:3 │ │ ├── scan u - │ │ │ └── columns: b:3 u.rowid:4!null + │ │ │ └── columns: b:4 u.rowid:5!null u.crdb_internal_mvcc_timestamp:6 │ │ └── filters (true) │ ├── project-set - │ │ ├── columns: generate_series:5 + │ │ ├── columns: generate_series:7 │ │ ├── values │ │ │ └── () │ │ └── zip │ │ └── generate_series(1, 2) │ └── filters (true) ├── project-set - │ ├── columns: generate_series:6 + │ ├── columns: generate_series:8 │ ├── values │ │ └── () │ └── zip @@ -715,70 +715,70 @@ build SELECT max(a) FROM t HAVING max(a::int) > (SELECT generate_series(0, b::int) FROM u limit 1) ---- project - ├── columns: max:3 + ├── columns: max:4 └── select - ├── columns: max:3 max:5!null + ├── columns: max:4 max:6!null ├── scalar-group-by - │ ├── columns: max:3 max:5 + │ ├── columns: max:4 max:6 │ ├── project - │ │ ├── columns: column4:4 a:1 + │ │ ├── columns: column5:5 a:1 │ │ ├── scan t - │ │ │ └── columns: a:1 t.rowid:2!null + │ │ │ └── columns: a:1 t.rowid:2!null t.crdb_internal_mvcc_timestamp:3 │ │ └── projections - │ │ └── a:1::INT8 [as=column4:4] + │ │ └── a:1::INT8 [as=column5:5] │ └── aggregations - │ ├── max [as=max:3] + │ ├── max [as=max:4] │ │ └── a:1 - │ └── max [as=max:5] - │ └── column4:4 + │ └── max [as=max:6] + │ └── column5:5 └── filters └── gt - ├── max:5 + ├── max:6 └── subquery └── max1-row - ├── columns: generate_series:8 + ├── columns: generate_series:10 └── limit - ├── columns: generate_series:8 + ├── columns: generate_series:10 ├── project - │ ├── columns: generate_series:8 + │ ├── columns: generate_series:10 │ ├── limit hint: 1.00 │ └── project-set - │ ├── columns: b:6 u.rowid:7!null generate_series:8 + │ ├── columns: b:7 u.rowid:8!null u.crdb_internal_mvcc_timestamp:9 generate_series:10 │ ├── limit hint: 1.00 │ ├── scan u - │ │ ├── columns: b:6 u.rowid:7!null + │ │ ├── columns: b:7 u.rowid:8!null u.crdb_internal_mvcc_timestamp:9 │ │ └── limit hint: 1.00 │ └── zip - │ └── generate_series(0, b:6::INT8) + │ └── generate_series(0, b:7::INT8) └── 1 build SELECT generate_series((SELECT generate_subscripts(ARRAY[a, a||b]) FROM t, u), 100) FROM t ---- project - ├── columns: generate_series:8 + ├── columns: generate_series:11 └── project-set - ├── columns: a:1 t.rowid:2!null generate_series:8 + ├── columns: a:1 t.rowid:2!null t.crdb_internal_mvcc_timestamp:3 generate_series:11 ├── scan t - │ └── columns: a:1 t.rowid:2!null + │ └── columns: a:1 t.rowid:2!null t.crdb_internal_mvcc_timestamp:3 └── zip └── function: generate_series ├── subquery │ └── max1-row - │ ├── columns: generate_subscripts:7 + │ ├── columns: generate_subscripts:10 │ └── project - │ ├── columns: generate_subscripts:7 + │ ├── columns: generate_subscripts:10 │ └── project-set - │ ├── columns: a:3 t.rowid:4!null b:5 u.rowid:6!null generate_subscripts:7 + │ ├── columns: a:4 t.rowid:5!null t.crdb_internal_mvcc_timestamp:6 b:7 u.rowid:8!null u.crdb_internal_mvcc_timestamp:9 generate_subscripts:10 │ ├── inner-join (cross) - │ │ ├── columns: a:3 t.rowid:4!null b:5 u.rowid:6!null + │ │ ├── columns: a:4 t.rowid:5!null t.crdb_internal_mvcc_timestamp:6 b:7 u.rowid:8!null u.crdb_internal_mvcc_timestamp:9 │ │ ├── scan t - │ │ │ └── columns: a:3 t.rowid:4!null + │ │ │ └── columns: a:4 t.rowid:5!null t.crdb_internal_mvcc_timestamp:6 │ │ ├── scan u - │ │ │ └── columns: b:5 u.rowid:6!null + │ │ │ └── columns: b:7 u.rowid:8!null u.crdb_internal_mvcc_timestamp:9 │ │ └── filters (true) │ └── zip - │ └── generate_subscripts(ARRAY[a:3, a:3 || b:5]) + │ └── generate_subscripts(ARRAY[a:4, a:4 || b:7]) └── 100 exec-ddl @@ -794,56 +794,56 @@ SELECT FROM a ---- project - ├── columns: json_array_elements:6 jsonb_each:19 jsonb_object_keys:20 generate_series:21 + ├── columns: json_array_elements:7 jsonb_each:21 jsonb_object_keys:22 generate_series:23 ├── project-set - │ ├── columns: x:1!null j:2 k:3 m:4 n:5 json_array_elements:6 + │ ├── columns: x:1!null j:2 k:3 m:4 n:5 crdb_internal_mvcc_timestamp:6 json_array_elements:7 │ ├── scan a - │ │ └── columns: x:1!null j:2 k:3 m:4 n:5 + │ │ └── columns: x:1!null j:2 k:3 m:4 n:5 crdb_internal_mvcc_timestamp:6 │ └── zip │ └── json_array_elements(j:2) └── projections - ├── subquery [as=jsonb_each:19] + ├── subquery [as=jsonb_each:21] │ └── max1-row - │ ├── columns: jsonb_each:9 + │ ├── columns: jsonb_each:10 │ └── project - │ ├── columns: jsonb_each:9 + │ ├── columns: jsonb_each:10 │ ├── project-set - │ │ ├── columns: key:7 value:8 + │ │ ├── columns: key:8 value:9 │ │ ├── values │ │ │ └── () │ │ └── zip │ │ └── jsonb_each(k:3) │ └── projections - │ └── ((key:7, value:8) AS key, value) [as=jsonb_each:9] - ├── subquery [as=jsonb_object_keys:20] + │ └── ((key:8, value:9) AS key, value) [as=jsonb_each:10] + ├── subquery [as=jsonb_object_keys:22] │ └── max1-row - │ ├── columns: jsonb_object_keys:15 + │ ├── columns: jsonb_object_keys:17 │ └── project - │ ├── columns: jsonb_object_keys:15 + │ ├── columns: jsonb_object_keys:17 │ └── project-set - │ ├── columns: x:10!null j:11 k:12 m:13 n:14 jsonb_object_keys:15 + │ ├── columns: x:11!null j:12 k:13 m:14 n:15 crdb_internal_mvcc_timestamp:16 jsonb_object_keys:17 │ ├── scan a - │ │ └── columns: x:10!null j:11 k:12 m:13 n:14 + │ │ └── columns: x:11!null j:12 k:13 m:14 n:15 crdb_internal_mvcc_timestamp:16 │ └── zip - │ └── jsonb_object_keys(m:13) - └── subquery [as=generate_series:21] + │ └── jsonb_object_keys(m:14) + └── subquery [as=generate_series:23] └── max1-row - ├── columns: generate_series:18 + ├── columns: generate_series:20 └── project-set - ├── columns: generate_series:18 + ├── columns: generate_series:20 ├── values │ └── () └── zip └── function: generate_series ├── subquery │ └── max1-row - │ ├── columns: generate_series:17 + │ ├── columns: generate_series:19 │ └── project - │ ├── columns: generate_series:17 + │ ├── columns: generate_series:19 │ └── project-set - │ ├── columns: value:16 generate_series:17 + │ ├── columns: value:18 generate_series:19 │ ├── project-set - │ │ ├── columns: value:16 + │ │ ├── columns: value:18 │ │ ├── values │ │ │ └── () │ │ └── zip @@ -877,7 +877,7 @@ build SELECT * FROM ROWS FROM (CAST((SELECT a FROM tab31755 LIMIT 1) AS SERIAL2[])) AS ident ---- project-set - ├── columns: ident:3 + ├── columns: ident:4 ├── values │ └── () └── zip @@ -891,6 +891,6 @@ project-set │ ├── columns: tab31755.a:1 │ ├── limit hint: 1.00 │ └── scan tab31755 - │ ├── columns: tab31755.a:1 rowid:2!null + │ ├── columns: tab31755.a:1 rowid:2!null crdb_internal_mvcc_timestamp:3 │ └── limit hint: 1.00 └── 1 diff --git a/pkg/sql/opt/optbuilder/testdata/subquery b/pkg/sql/opt/optbuilder/testdata/subquery index 2fa039894d38..41c3885b60ac 100644 --- a/pkg/sql/opt/optbuilder/testdata/subquery +++ b/pkg/sql/opt/optbuilder/testdata/subquery @@ -589,42 +589,42 @@ build SELECT (1, 2) IN (SELECT a, b FROM abc) AS r ---- project - ├── columns: r:5 + ├── columns: r:6 ├── values │ └── () └── projections - └── any: eq [as=r:5] + └── any: eq [as=r:6] ├── project - │ ├── columns: column4:4 + │ ├── columns: column5:5 │ ├── project │ │ ├── columns: a:1!null b:2 │ │ └── scan abc - │ │ └── columns: a:1!null b:2 c:3 + │ │ └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 │ └── projections - │ └── (a:1, b:2) [as=column4:4] + │ └── (a:1, b:2) [as=column5:5] └── (1, 2) build SELECT (1, 2) IN (SELECT a, b FROM abc WHERE false) AS r ---- project - ├── columns: r:5 + ├── columns: r:6 ├── values │ └── () └── projections - └── any: eq [as=r:5] + └── any: eq [as=r:6] ├── project - │ ├── columns: column4:4 + │ ├── columns: column5:5 │ ├── project │ │ ├── columns: a:1!null b:2 │ │ └── select - │ │ ├── columns: a:1!null b:2 c:3 + │ │ ├── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 │ │ ├── scan abc - │ │ │ └── columns: a:1!null b:2 c:3 + │ │ │ └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 │ │ └── filters │ │ └── false │ └── projections - │ └── (a:1, b:2) [as=column4:4] + │ └── (a:1, b:2) [as=column5:5] └── (1, 2) build @@ -636,31 +636,31 @@ build SELECT (SELECT a FROM abc) ---- project - ├── columns: a:4 + ├── columns: a:5 ├── values │ └── () └── projections - └── subquery [as=a:4] + └── subquery [as=a:5] └── max1-row ├── columns: abc.a:1!null └── project ├── columns: abc.a:1!null └── scan abc - └── columns: abc.a:1!null b:2 c:3 + └── columns: abc.a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 build SELECT EXISTS (SELECT a FROM abc) ---- project - ├── columns: exists:4 + ├── columns: exists:5 ├── values │ └── () └── projections - └── exists [as=exists:4] + └── exists [as=exists:5] └── project ├── columns: a:1!null └── scan abc - └── columns: a:1!null b:2 c:3 + └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 build SELECT true = EXISTS (SELECT 1) @@ -684,19 +684,19 @@ build SELECT (SELECT a FROM abc WHERE false) ---- project - ├── columns: a:4 + ├── columns: a:5 ├── values │ └── () └── projections - └── subquery [as=a:4] + └── subquery [as=a:5] └── max1-row ├── columns: abc.a:1!null └── project ├── columns: abc.a:1!null └── select - ├── columns: abc.a:1!null b:2 c:3 + ├── columns: abc.a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 ├── scan abc - │ └── columns: abc.a:1!null b:2 c:3 + │ └── columns: abc.a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 └── filters └── false @@ -729,12 +729,14 @@ values build SELECT * FROM abc WHERE a = 7 ---- -select +project ├── columns: a:1!null b:2 c:3 - ├── scan abc - │ └── columns: a:1!null b:2 c:3 - └── filters - └── a:1 = 7 + └── select + ├── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 + ├── scan abc + │ └── columns: a:1!null b:2 c:3 crdb_internal_mvcc_timestamp:4 + └── filters + └── a:1 = 7 exec-ddl CREATE TABLE xyz (x INT PRIMARY KEY, y INT, z INT) @@ -743,69 +745,75 @@ CREATE TABLE xyz (x INT PRIMARY KEY, y INT, z INT) build SELECT * FROM xyz ---- -scan xyz - └── columns: x:1!null y:2 z:3 +project + ├── columns: x:1!null y:2 z:3 + └── scan xyz + └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 build SELECT 1 IN (SELECT x FROM xyz ORDER BY x DESC) AS r ---- project - ├── columns: r:4 + ├── columns: r:5 ├── values │ └── () └── projections - └── any: eq [as=r:4] + └── any: eq [as=r:5] ├── project │ ├── columns: x:1!null │ └── scan xyz - │ └── columns: x:1!null y:2 z:3 + │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 └── 1 build SELECT * FROM xyz WHERE x = (SELECT min(x) FROM xyz) ---- -select +project ├── columns: x:1!null y:2 z:3 - ├── scan xyz - │ └── columns: x:1!null y:2 z:3 - └── filters - └── eq - ├── x:1 - └── subquery - └── max1-row - ├── columns: min:7 - └── scalar-group-by - ├── columns: min:7 - ├── project - │ ├── columns: x:4!null - │ └── scan xyz - │ └── columns: x:4!null y:5 z:6 - └── aggregations - └── min [as=min:7] - └── x:4 + └── select + ├── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 + ├── scan xyz + │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 + └── filters + └── eq + ├── x:1 + └── subquery + └── max1-row + ├── columns: min:9 + └── scalar-group-by + ├── columns: min:9 + ├── project + │ ├── columns: x:5!null + │ └── scan xyz + │ └── columns: x:5!null y:6 z:7 crdb_internal_mvcc_timestamp:8 + └── aggregations + └── min [as=min:9] + └── x:5 build SELECT * FROM xyz WHERE x = (SELECT max(x) FROM xyz) ---- -select +project ├── columns: x:1!null y:2 z:3 - ├── scan xyz - │ └── columns: x:1!null y:2 z:3 - └── filters - └── eq - ├── x:1 - └── subquery - └── max1-row - ├── columns: max:7 - └── scalar-group-by - ├── columns: max:7 - ├── project - │ ├── columns: x:4!null - │ └── scan xyz - │ └── columns: x:4!null y:5 z:6 - └── aggregations - └── max [as=max:7] - └── x:4 + └── select + ├── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 + ├── scan xyz + │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 + └── filters + └── eq + ├── x:1 + └── subquery + └── max1-row + ├── columns: max:9 + └── scalar-group-by + ├── columns: max:9 + ├── project + │ ├── columns: x:5!null + │ └── scan xyz + │ └── columns: x:5!null y:6 z:7 crdb_internal_mvcc_timestamp:8 + └── aggregations + └── max [as=max:9] + └── x:5 # Drop previous table with same name, but different schema. exec-ddl @@ -819,64 +827,66 @@ CREATE TABLE kv (k INT PRIMARY KEY, v STRING) build SELECT * FROM kv WHERE k = (SELECT k FROM kv WHERE (k, v) = (1, 'one')) ---- -select +project ├── columns: k:1!null v:2 - ├── scan kv - │ └── columns: k:1!null v:2 - └── filters - └── eq - ├── k:1 - └── subquery - └── max1-row - ├── columns: k:3!null - └── project - ├── columns: k:3!null - └── select - ├── columns: k:3!null v:4 - ├── scan kv - │ └── columns: k:3!null v:4 - └── filters - └── (k:3, v:4) = (1, 'one') + └── select + ├── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 + ├── scan kv + │ └── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 + └── filters + └── eq + ├── k:1 + └── subquery + └── max1-row + ├── columns: k:4!null + └── project + ├── columns: k:4!null + └── select + ├── columns: k:4!null v:5 crdb_internal_mvcc_timestamp:6 + ├── scan kv + │ └── columns: k:4!null v:5 crdb_internal_mvcc_timestamp:6 + └── filters + └── (k:4, v:5) = (1, 'one') build SELECT EXISTS(SELECT 1 r FROM kv AS x WHERE x.k = 1) ---- project - ├── columns: exists:4 + ├── columns: exists:5 ├── values │ └── () └── projections - └── exists [as=exists:4] + └── exists [as=exists:5] └── project - ├── columns: r:3!null + ├── columns: r:4!null ├── select - │ ├── columns: k:1!null v:2 + │ ├── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 │ ├── scan x - │ │ └── columns: k:1!null v:2 + │ │ └── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 │ └── filters │ └── k:1 = 1 └── projections - └── 1 [as=r:3] + └── 1 [as=r:4] build SELECT EXISTS(SELECT 1 r FROM kv WHERE k = 2) ---- project - ├── columns: exists:4 + ├── columns: exists:5 ├── values │ └── () └── projections - └── exists [as=exists:4] + └── exists [as=exists:5] └── project - ├── columns: r:3!null + ├── columns: r:4!null ├── select - │ ├── columns: k:1!null v:2 + │ ├── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 │ ├── scan kv - │ │ └── columns: k:1!null v:2 + │ │ └── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 │ └── filters │ └── k:1 = 2 └── projections - └── 1 [as=r:3] + └── 1 [as=r:4] # Tests for subquery in the FROM part of a SELECT @@ -933,8 +943,10 @@ SELECT * FROM (SELECT * FROM xyz) AS foo WHERE x < 7 ---- select ├── columns: x:1!null y:2 z:3 - ├── scan foo - │ └── columns: x:1!null y:2 z:3 + ├── project + │ ├── columns: x:1!null y:2 z:3 + │ └── scan xyz + │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 └── filters └── x:1 < 7 @@ -943,30 +955,36 @@ SELECT * FROM (SELECT * FROM xyz) AS foo (foo1) WHERE foo1 < 7 ---- select ├── columns: foo1:1!null y:2 z:3 - ├── scan foo - │ └── columns: foo1:1!null y:2 z:3 + ├── project + │ ├── columns: x:1!null y:2 z:3 + │ └── scan xyz + │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 └── filters - └── foo1:1 < 7 + └── x:1 < 7 build SELECT * FROM (SELECT * FROM xyz AS moo (moo1, moo2, moo3)) as foo (foo1) WHERE foo1 < 7 ---- select ├── columns: foo1:1!null moo2:2 moo3:3 - ├── scan foo - │ └── columns: foo1:1!null moo2:2 moo3:3 + ├── project + │ ├── columns: moo1:1!null moo2:2 moo3:3 + │ └── scan moo + │ └── columns: moo1:1!null moo2:2 moo3:3 crdb_internal_mvcc_timestamp:4 └── filters - └── foo1:1 < 7 + └── moo1:1 < 7 build SELECT * FROM (SELECT * FROM xyz AS moo (moo1, moo2, moo3) ORDER BY moo1) as foo (foo1) WHERE foo1 < 7 ---- select ├── columns: foo1:1!null moo2:2 moo3:3 - ├── scan foo - │ └── columns: foo1:1!null moo2:2 moo3:3 + ├── project + │ ├── columns: moo1:1!null moo2:2 moo3:3 + │ └── scan moo + │ └── columns: moo1:1!null moo2:2 moo3:3 crdb_internal_mvcc_timestamp:4 └── filters - └── foo1:1 < 7 + └── moo1:1 < 7 build SELECT * FROM (SELECT * FROM xyz AS moo (moo1, moo2, moo3) ORDER BY moo1) as foo (foo1) WHERE foo1 < 7 ORDER BY moo2 DESC @@ -975,11 +993,13 @@ sort ├── columns: foo1:1!null moo2:2 moo3:3 ├── ordering: -2 └── select - ├── columns: foo1:1!null moo2:2 moo3:3 - ├── scan foo - │ └── columns: foo1:1!null moo2:2 moo3:3 + ├── columns: moo1:1!null moo2:2 moo3:3 + ├── project + │ ├── columns: moo1:1!null moo2:2 moo3:3 + │ └── scan moo + │ └── columns: moo1:1!null moo2:2 moo3:3 crdb_internal_mvcc_timestamp:4 └── filters - └── foo1:1 < 7 + └── moo1:1 < 7 build SELECT * FROM (SELECT * FROM (VALUES (1, 2, 3), (4, 5, 6)) AS moo (moo1, moo2, moo3) WHERE moo1 = 4) as foo (foo1) @@ -1044,19 +1064,19 @@ SELECT x FROM xyz WHERE x IN (SELECT x FROM xyz WHERE x = 7) project ├── columns: x:1!null └── select - ├── columns: x:1!null y:2 z:3 + ├── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 ├── scan xyz - │ └── columns: x:1!null y:2 z:3 + │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 └── filters └── any: eq ├── project - │ ├── columns: x:4!null + │ ├── columns: x:5!null │ └── select - │ ├── columns: x:4!null y:5 z:6 + │ ├── columns: x:5!null y:6 z:7 crdb_internal_mvcc_timestamp:8 │ ├── scan xyz - │ │ └── columns: x:4!null y:5 z:6 + │ │ └── columns: x:5!null y:6 z:7 crdb_internal_mvcc_timestamp:8 │ └── filters - │ └── x:4 = 7 + │ └── x:5 = 7 └── x:1 build @@ -1067,22 +1087,22 @@ limit ├── project │ ├── columns: x:1!null │ └── select - │ ├── columns: x:1!null y:2 z:3 + │ ├── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 │ ├── scan xyz - │ │ └── columns: x:1!null y:2 z:3 + │ │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 │ └── filters │ └── x:1 = 7 └── subquery └── max1-row - ├── columns: x:4!null + ├── columns: x:5!null └── project - ├── columns: x:4!null + ├── columns: x:5!null └── select - ├── columns: x:4!null y:5 z:6 + ├── columns: x:5!null y:6 z:7 crdb_internal_mvcc_timestamp:8 ├── scan xyz - │ └── columns: x:4!null y:5 z:6 + │ └── columns: x:5!null y:6 z:7 crdb_internal_mvcc_timestamp:8 └── filters - └── x:4 = 1 + └── x:5 = 1 build SELECT x FROM xyz ORDER BY x OFFSET (SELECT x FROM xyz WHERE x = 1) @@ -1095,19 +1115,19 @@ offset │ ├── columns: x:1!null │ ├── ordering: +1 │ └── scan xyz - │ ├── columns: x:1!null y:2 z:3 + │ ├── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 │ └── ordering: +1 └── subquery └── max1-row - ├── columns: x:4!null + ├── columns: x:5!null └── project - ├── columns: x:4!null + ├── columns: x:5!null └── select - ├── columns: x:4!null y:5 z:6 + ├── columns: x:5!null y:6 z:7 crdb_internal_mvcc_timestamp:8 ├── scan xyz - │ └── columns: x:4!null y:5 z:6 + │ └── columns: x:5!null y:6 z:7 crdb_internal_mvcc_timestamp:8 └── filters - └── x:4 = 1 + └── x:5 = 1 # check that residual filters are not expanded twice build @@ -1116,15 +1136,15 @@ SELECT x FROM xyz WHERE x IN (SELECT x FROM xyz) project ├── columns: x:1!null └── select - ├── columns: x:1!null y:2 z:3 + ├── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 ├── scan xyz - │ └── columns: x:1!null y:2 z:3 + │ └── columns: x:1!null y:2 z:3 crdb_internal_mvcc_timestamp:4 └── filters └── any: eq ├── project - │ ├── columns: x:4!null + │ ├── columns: x:5!null │ └── scan xyz - │ └── columns: x:4!null y:5 z:6 + │ └── columns: x:5!null y:6 z:7 crdb_internal_mvcc_timestamp:8 └── x:1 # This test checks that the double sub-query plan expansion caused by a @@ -1140,22 +1160,22 @@ SELECT col0 FROM tab4 WHERE (col0 <= 0 AND col4 <= 5.38) OR (col4 IN (SELECT col project ├── columns: col0:1!null └── select - ├── columns: col0:1!null col1:2 col3:3 col4:4!null rowid:5!null + ├── columns: col0:1!null col1:2 col3:3 col4:4!null rowid:5!null crdb_internal_mvcc_timestamp:6 ├── scan tab4 - │ └── columns: col0:1 col1:2 col3:3 col4:4 rowid:5!null + │ └── columns: col0:1 col1:2 col3:3 col4:4 rowid:5!null crdb_internal_mvcc_timestamp:6 └── filters └── or ├── (col0:1 <= 0) AND (col4:4 <= 5.38) └── and ├── any: eq │ ├── project - │ │ ├── columns: col1:7!null + │ │ ├── columns: col1:8!null │ │ └── select - │ │ ├── columns: col0:6 col1:7!null col3:8 col4:9 rowid:10!null + │ │ ├── columns: col0:7 col1:8!null col3:9 col4:10 rowid:11!null crdb_internal_mvcc_timestamp:12 │ │ ├── scan tab4 - │ │ │ └── columns: col0:6 col1:7 col3:8 col4:9 rowid:10!null + │ │ │ └── columns: col0:7 col1:8 col3:9 col4:10 rowid:11!null crdb_internal_mvcc_timestamp:12 │ │ └── filters - │ │ └── col1:7 > 8.27 + │ │ └── col1:8 > 8.27 │ └── col4:4 └── (col3:3 <= 5) AND ((col3:3 >= 7) AND (col3:3 <= 9)) @@ -1192,11 +1212,11 @@ build SELECT (SELECT 1 a), (SELECT a FROM abc), (SELECT 1 a), (SELECT a FROM abc) ---- project - ├── columns: a:9 a:10 a:9 a:10 + ├── columns: a:11 a:12 a:11 a:12 ├── values │ └── () └── projections - ├── subquery [as=a:9] + ├── subquery [as=a:11] │ └── max1-row │ ├── columns: a:1!null │ └── project @@ -1205,13 +1225,13 @@ project │ │ └── () │ └── projections │ └── 1 [as=a:1] - └── subquery [as=a:10] + └── subquery [as=a:12] └── max1-row ├── columns: abc.a:2!null └── project ├── columns: abc.a:2!null └── scan abc - └── columns: abc.a:2!null b:3 c:4 + └── columns: abc.a:2!null b:3 c:4 crdb_internal_mvcc_timestamp:5 # Multiple nested subqueries in same column list. build @@ -1254,19 +1274,19 @@ build SELECT (SELECT akv.k) FROM kv akv ---- project - ├── columns: k:4 + ├── columns: k:5 ├── scan akv - │ └── columns: akv.k:1!null v:2 + │ └── columns: akv.k:1!null v:2 crdb_internal_mvcc_timestamp:3 └── projections - └── subquery [as=k:4] + └── subquery [as=k:5] └── max1-row - ├── columns: k:3 + ├── columns: k:4 └── project - ├── columns: k:3 + ├── columns: k:4 ├── values │ └── () └── projections - └── akv.k:1 [as=k:3] + └── akv.k:1 [as=k:4] exec-ddl CREATE TABLE db1.kv (k INT PRIMARY KEY, v INT) @@ -1276,24 +1296,24 @@ build fully-qualify-names SELECT (SELECT t.kv.k) FROM db1.kv, kv ---- project - ├── columns: k:6 + ├── columns: k:8 ├── inner-join (cross) - │ ├── columns: db1.public.kv.k:1!null db1.public.kv.v:2 t.public.kv.k:3!null t.public.kv.v:4 + │ ├── columns: db1.public.kv.k:1!null db1.public.kv.v:2 db1.public.kv.crdb_internal_mvcc_timestamp:3 t.public.kv.k:4!null t.public.kv.v:5 t.public.kv.crdb_internal_mvcc_timestamp:6 │ ├── scan db1.public.kv - │ │ └── columns: db1.public.kv.k:1!null db1.public.kv.v:2 + │ │ └── columns: db1.public.kv.k:1!null db1.public.kv.v:2 db1.public.kv.crdb_internal_mvcc_timestamp:3 │ ├── scan t.public.kv - │ │ └── columns: t.public.kv.k:3!null t.public.kv.v:4 + │ │ └── columns: t.public.kv.k:4!null t.public.kv.v:5 t.public.kv.crdb_internal_mvcc_timestamp:6 │ └── filters (true) └── projections - └── subquery [as=k:6] + └── subquery [as=k:8] └── max1-row - ├── columns: k:5 + ├── columns: k:7 └── project - ├── columns: k:5 + ├── columns: k:7 ├── values │ └── () └── projections - └── t.public.kv.k:3 [as=k:5] + └── t.public.kv.k:4 [as=k:7] # Ambiguity in parent scope. build fully-qualify-names @@ -1311,24 +1331,24 @@ build fully-qualify-names SELECT (SELECT kv1.k) FROM db1.kv AS kv1, kv ---- project - ├── columns: k:6 + ├── columns: k:8 ├── inner-join (cross) - │ ├── columns: db1.public.kv.k:1!null db1.public.kv.v:2 t.public.kv.k:3!null t.public.kv.v:4 + │ ├── columns: db1.public.kv.k:1!null db1.public.kv.v:2 db1.public.kv.crdb_internal_mvcc_timestamp:3 t.public.kv.k:4!null t.public.kv.v:5 t.public.kv.crdb_internal_mvcc_timestamp:6 │ ├── scan db1.public.kv - │ │ └── columns: db1.public.kv.k:1!null db1.public.kv.v:2 + │ │ └── columns: db1.public.kv.k:1!null db1.public.kv.v:2 db1.public.kv.crdb_internal_mvcc_timestamp:3 │ ├── scan t.public.kv - │ │ └── columns: t.public.kv.k:3!null t.public.kv.v:4 + │ │ └── columns: t.public.kv.k:4!null t.public.kv.v:5 t.public.kv.crdb_internal_mvcc_timestamp:6 │ └── filters (true) └── projections - └── subquery [as=k:6] + └── subquery [as=k:8] └── max1-row - ├── columns: k:5 + ├── columns: k:7 └── project - ├── columns: k:5 + ├── columns: k:7 ├── values │ └── () └── projections - └── db1.public.kv.k:1 [as=k:5] + └── db1.public.kv.k:1 [as=k:7] # Check that the inner kv is chosen when there are matching names in both # scopes. @@ -1336,60 +1356,62 @@ build fully-qualify-names SELECT (SELECT kv.k FROM db1.kv) FROM kv ---- project - ├── columns: k:5 + ├── columns: k:7 ├── scan t.public.kv - │ └── columns: t.public.kv.k:1!null t.public.kv.v:2 + │ └── columns: t.public.kv.k:1!null t.public.kv.v:2 t.public.kv.crdb_internal_mvcc_timestamp:3 └── projections - └── subquery [as=k:5] + └── subquery [as=k:7] └── max1-row - ├── columns: db1.public.kv.k:3!null + ├── columns: db1.public.kv.k:4!null └── project - ├── columns: db1.public.kv.k:3!null + ├── columns: db1.public.kv.k:4!null └── scan db1.public.kv - └── columns: db1.public.kv.k:3!null db1.public.kv.v:4 + └── columns: db1.public.kv.k:4!null db1.public.kv.v:5 db1.public.kv.crdb_internal_mvcc_timestamp:6 # 2 nested scopes, mixed scope references. build fully-qualify-names SELECT (SELECT (SELECT t.kv.k + k AS r) FROM db1.kv) FROM kv ---- project - ├── columns: r:7 + ├── columns: r:9 ├── scan t.public.kv - │ └── columns: t.public.kv.k:1!null t.public.kv.v:2 + │ └── columns: t.public.kv.k:1!null t.public.kv.v:2 t.public.kv.crdb_internal_mvcc_timestamp:3 └── projections - └── subquery [as=r:7] + └── subquery [as=r:9] └── max1-row - ├── columns: r:6 + ├── columns: r:8 └── project - ├── columns: r:6 + ├── columns: r:8 ├── scan db1.public.kv - │ └── columns: db1.public.kv.k:3!null db1.public.kv.v:4 + │ └── columns: db1.public.kv.k:4!null db1.public.kv.v:5 db1.public.kv.crdb_internal_mvcc_timestamp:6 └── projections - └── subquery [as=r:6] + └── subquery [as=r:8] └── max1-row - ├── columns: r:5 + ├── columns: r:7 └── project - ├── columns: r:5 + ├── columns: r:7 ├── values │ └── () └── projections - └── t.public.kv.k:1 + db1.public.kv.k:3 [as=r:5] + └── t.public.kv.k:1 + db1.public.kv.k:4 [as=r:7] build SELECT (SELECT k FROM kv ORDER BY v) ---- project - ├── columns: k:3 + ├── columns: k:4 ├── values │ └── () └── projections - └── subquery [as=k:3] + └── subquery [as=k:4] └── max1-row ├── columns: kv.k:1!null └── project ├── columns: kv.k:1!null - └── scan kv - └── columns: kv.k:1!null v:2 + └── project + ├── columns: kv.k:1!null v:2 + └── scan kv + └── columns: kv.k:1!null v:2 crdb_internal_mvcc_timestamp:3 exec-ddl CREATE TABLE t1 (a INT, b INT) @@ -1407,206 +1429,206 @@ build SELECT (SELECT (SELECT DISTINCT t3.a FROM t1) FROM t2) FROM t3 ---- project - ├── columns: a:12 + ├── columns: a:15 ├── scan t3 - │ └── columns: t3.a:1 t3.b:2 t3.rowid:3!null + │ └── columns: t3.a:1 t3.b:2 t3.rowid:3!null t3.crdb_internal_mvcc_timestamp:4 └── projections - └── subquery [as=a:12] + └── subquery [as=a:15] └── max1-row - ├── columns: a:11 + ├── columns: a:14 └── project - ├── columns: a:11 + ├── columns: a:14 ├── scan t2 - │ └── columns: t2.a:4 t2.b:5 t2.rowid:6!null + │ └── columns: t2.a:5 t2.b:6 t2.rowid:7!null t2.crdb_internal_mvcc_timestamp:8 └── projections - └── subquery [as=a:11] + └── subquery [as=a:14] └── max1-row - ├── columns: a:10 + ├── columns: a:13 └── distinct-on - ├── columns: a:10 - ├── grouping columns: a:10 + ├── columns: a:13 + ├── grouping columns: a:13 └── project - ├── columns: a:10 + ├── columns: a:13 ├── scan t1 - │ └── columns: t1.a:7 t1.b:8 t1.rowid:9!null + │ └── columns: t1.a:9 t1.b:10 t1.rowid:11!null t1.crdb_internal_mvcc_timestamp:12 └── projections - └── t3.a:1 [as=a:10] + └── t3.a:1 [as=a:13] build SELECT (SELECT (SELECT count(*) FROM t1 GROUP BY t3.a) FROM t2) FROM t3 ---- project - ├── columns: count:13 + ├── columns: count:16 ├── scan t3 - │ └── columns: t3.a:1 t3.b:2 t3.rowid:3!null + │ └── columns: t3.a:1 t3.b:2 t3.rowid:3!null t3.crdb_internal_mvcc_timestamp:4 └── projections - └── subquery [as=count:13] + └── subquery [as=count:16] └── max1-row - ├── columns: count:12 + ├── columns: count:15 └── project - ├── columns: count:12 + ├── columns: count:15 ├── scan t2 - │ └── columns: t2.a:4 t2.b:5 t2.rowid:6!null + │ └── columns: t2.a:5 t2.b:6 t2.rowid:7!null t2.crdb_internal_mvcc_timestamp:8 └── projections - └── subquery [as=count:12] + └── subquery [as=count:15] └── max1-row - ├── columns: count_rows:10!null + ├── columns: count_rows:13!null └── project - ├── columns: count_rows:10!null + ├── columns: count_rows:13!null └── group-by - ├── columns: count_rows:10!null a:11 - ├── grouping columns: a:11 + ├── columns: count_rows:13!null a:14 + ├── grouping columns: a:14 ├── project - │ ├── columns: a:11 + │ ├── columns: a:14 │ ├── scan t1 - │ │ └── columns: t1.a:7 t1.b:8 t1.rowid:9!null + │ │ └── columns: t1.a:9 t1.b:10 t1.rowid:11!null t1.crdb_internal_mvcc_timestamp:12 │ └── projections - │ └── t3.a:1 [as=a:11] + │ └── t3.a:1 [as=a:14] └── aggregations - └── count-rows [as=count_rows:10] + └── count-rows [as=count_rows:13] build SELECT (SELECT (SELECT t2.a + t3.a AS x FROM t1 GROUP BY t2.a + t3.a) AS y FROM t2) AS z FROM t3 ---- project - ├── columns: z:12 + ├── columns: z:15 ├── scan t3 - │ └── columns: t3.a:1 t3.b:2 t3.rowid:3!null + │ └── columns: t3.a:1 t3.b:2 t3.rowid:3!null t3.crdb_internal_mvcc_timestamp:4 └── projections - └── subquery [as=z:12] + └── subquery [as=z:15] └── max1-row - ├── columns: y:11 + ├── columns: y:14 └── project - ├── columns: y:11 + ├── columns: y:14 ├── scan t2 - │ └── columns: t2.a:4 t2.b:5 t2.rowid:6!null + │ └── columns: t2.a:5 t2.b:6 t2.rowid:7!null t2.crdb_internal_mvcc_timestamp:8 └── projections - └── subquery [as=y:11] + └── subquery [as=y:14] └── max1-row - ├── columns: column10:10 + ├── columns: column13:13 └── group-by - ├── columns: column10:10 - ├── grouping columns: column10:10 + ├── columns: column13:13 + ├── grouping columns: column13:13 └── project - ├── columns: column10:10 + ├── columns: column13:13 ├── scan t1 - │ └── columns: t1.a:7 t1.b:8 t1.rowid:9!null + │ └── columns: t1.a:9 t1.b:10 t1.rowid:11!null t1.crdb_internal_mvcc_timestamp:12 └── projections - └── t2.a:4 + t3.a:1 [as=column10:10] + └── t2.a:5 + t3.a:1 [as=column13:13] build SELECT (SELECT (SELECT t2.a + t3.a AS r FROM t1 GROUP BY t2.a, t3.a HAVING t2.a > t3.a) FROM t2) FROM t3 ---- project - ├── columns: r:14 + ├── columns: r:17 ├── scan t3 - │ └── columns: t3.a:1 t3.b:2 t3.rowid:3!null + │ └── columns: t3.a:1 t3.b:2 t3.rowid:3!null t3.crdb_internal_mvcc_timestamp:4 └── projections - └── subquery [as=r:14] + └── subquery [as=r:17] └── max1-row - ├── columns: r:13 + ├── columns: r:16 └── project - ├── columns: r:13 + ├── columns: r:16 ├── scan t2 - │ └── columns: t2.a:4 t2.b:5 t2.rowid:6!null + │ └── columns: t2.a:5 t2.b:6 t2.rowid:7!null t2.crdb_internal_mvcc_timestamp:8 └── projections - └── subquery [as=r:13] + └── subquery [as=r:16] └── max1-row - ├── columns: r:12!null + ├── columns: r:15!null └── project - ├── columns: r:12!null + ├── columns: r:15!null ├── select - │ ├── columns: a:10!null a:11!null + │ ├── columns: a:13!null a:14!null │ ├── group-by - │ │ ├── columns: a:10 a:11 - │ │ ├── grouping columns: a:10 a:11 + │ │ ├── columns: a:13 a:14 + │ │ ├── grouping columns: a:13 a:14 │ │ └── project - │ │ ├── columns: a:10 a:11 + │ │ ├── columns: a:13 a:14 │ │ ├── scan t1 - │ │ │ └── columns: t1.a:7 t1.b:8 t1.rowid:9!null + │ │ │ └── columns: t1.a:9 t1.b:10 t1.rowid:11!null t1.crdb_internal_mvcc_timestamp:12 │ │ └── projections - │ │ ├── t2.a:4 [as=a:10] - │ │ └── t3.a:1 [as=a:11] + │ │ ├── t2.a:5 [as=a:13] + │ │ └── t3.a:1 [as=a:14] │ └── filters - │ └── a:10 > a:11 + │ └── a:13 > a:14 └── projections - └── a:10 + a:11 [as=r:12] + └── a:13 + a:14 [as=r:15] build SELECT (SELECT ARRAY[count(*), t1.a, t2.a] FROM t1 GROUP BY t1.a, t2.a HAVING t2.a > 5) FROM t2 ---- project - ├── columns: array:10 + ├── columns: array:12 ├── scan t2 - │ └── columns: t2.a:1 t2.b:2 t2.rowid:3!null + │ └── columns: t2.a:1 t2.b:2 t2.rowid:3!null t2.crdb_internal_mvcc_timestamp:4 └── projections - └── subquery [as=array:10] + └── subquery [as=array:12] └── max1-row - ├── columns: array:9 + ├── columns: array:11 └── project - ├── columns: array:9 + ├── columns: array:11 ├── select - │ ├── columns: t1.a:4 count_rows:7!null a:8!null + │ ├── columns: t1.a:5 count_rows:9!null a:10!null │ ├── group-by - │ │ ├── columns: t1.a:4 count_rows:7!null a:8 - │ │ ├── grouping columns: t1.a:4 a:8 + │ │ ├── columns: t1.a:5 count_rows:9!null a:10 + │ │ ├── grouping columns: t1.a:5 a:10 │ │ ├── project - │ │ │ ├── columns: a:8 t1.a:4 + │ │ │ ├── columns: a:10 t1.a:5 │ │ │ ├── scan t1 - │ │ │ │ └── columns: t1.a:4 t1.b:5 t1.rowid:6!null + │ │ │ │ └── columns: t1.a:5 t1.b:6 t1.rowid:7!null t1.crdb_internal_mvcc_timestamp:8 │ │ │ └── projections - │ │ │ └── t2.a:1 [as=a:8] + │ │ │ └── t2.a:1 [as=a:10] │ │ └── aggregations - │ │ └── count-rows [as=count_rows:7] + │ │ └── count-rows [as=count_rows:9] │ └── filters - │ └── a:8 > 5 + │ └── a:10 > 5 └── projections - └── ARRAY[count_rows:7, t1.a:4, a:8] [as=array:9] + └── ARRAY[count_rows:9, t1.a:5, a:10] [as=array:11] build SELECT (SELECT (SELECT max(t3.a) / min(t3.a) AS r FROM t1 GROUP BY t2.a) FROM t2) FROM t3 ---- project - ├── columns: r:17 + ├── columns: r:20 ├── scalar-group-by - │ ├── columns: max:11 min:13 + │ ├── columns: max:14 min:16 │ ├── project - │ │ ├── columns: a:10 a:12 + │ │ ├── columns: a:13 a:15 │ │ ├── scan t3 - │ │ │ └── columns: t3.a:1 t3.b:2 t3.rowid:3!null + │ │ │ └── columns: t3.a:1 t3.b:2 t3.rowid:3!null t3.crdb_internal_mvcc_timestamp:4 │ │ └── projections - │ │ ├── t3.a:1 [as=a:10] - │ │ └── t3.a:1 [as=a:12] + │ │ ├── t3.a:1 [as=a:13] + │ │ └── t3.a:1 [as=a:15] │ └── aggregations - │ ├── max [as=max:11] - │ │ └── a:10 - │ └── min [as=min:13] - │ └── a:12 + │ ├── max [as=max:14] + │ │ └── a:13 + │ └── min [as=min:16] + │ └── a:15 └── projections - └── subquery [as=r:17] + └── subquery [as=r:20] └── max1-row - ├── columns: r:16 + ├── columns: r:19 └── project - ├── columns: r:16 + ├── columns: r:19 ├── scan t2 - │ └── columns: t2.a:4 t2.b:5 t2.rowid:6!null + │ └── columns: t2.a:5 t2.b:6 t2.rowid:7!null t2.crdb_internal_mvcc_timestamp:8 └── projections - └── subquery [as=r:16] + └── subquery [as=r:19] └── max1-row - ├── columns: r:15 + ├── columns: r:18 └── project - ├── columns: r:15 + ├── columns: r:18 ├── group-by - │ ├── columns: a:14 - │ ├── grouping columns: a:14 + │ ├── columns: a:17 + │ ├── grouping columns: a:17 │ └── project - │ ├── columns: a:14 + │ ├── columns: a:17 │ ├── scan t1 - │ │ └── columns: t1.a:7 t1.b:8 t1.rowid:9!null + │ │ └── columns: t1.a:9 t1.b:10 t1.rowid:11!null t1.crdb_internal_mvcc_timestamp:12 │ └── projections - │ └── t2.a:4 [as=a:14] + │ └── t2.a:5 [as=a:17] └── projections - └── max:11 / min:13 [as=r:15] + └── max:14 / min:16 [as=r:18] exec-ddl CREATE TABLE a (k INT PRIMARY KEY, i INT, f FLOAT, s STRING, j JSON) @@ -1619,36 +1641,38 @@ SELECT * FROM a WHERE 'bar'=(SELECT max(s) FROM a GROUP BY i ORDER BY i LIMIT 1) ---- -select +project ├── columns: k:1!null i:2 f:3 s:4 j:5 - ├── scan a - │ └── columns: k:1!null i:2 f:3 s:4 j:5 - └── filters - └── eq - ├── 'bar' - └── subquery - └── max1-row - ├── columns: max:11 - └── project - ├── columns: max:11 - └── limit - ├── columns: i:7 max:11 - ├── internal-ordering: +7 - ├── sort - │ ├── columns: i:7 max:11 - │ ├── ordering: +7 - │ ├── limit hint: 1.00 - │ └── group-by - │ ├── columns: i:7 max:11 - │ ├── grouping columns: i:7 - │ ├── project - │ │ ├── columns: i:7 s:9 - │ │ └── scan a - │ │ └── columns: k:6!null i:7 f:8 s:9 j:10 - │ └── aggregations - │ └── max [as=max:11] - │ └── s:9 - └── 1 + └── select + ├── columns: k:1!null i:2 f:3 s:4 j:5 crdb_internal_mvcc_timestamp:6 + ├── scan a + │ └── columns: k:1!null i:2 f:3 s:4 j:5 crdb_internal_mvcc_timestamp:6 + └── filters + └── eq + ├── 'bar' + └── subquery + └── max1-row + ├── columns: max:13 + └── project + ├── columns: max:13 + └── limit + ├── columns: i:8 max:13 + ├── internal-ordering: +8 + ├── sort + │ ├── columns: i:8 max:13 + │ ├── ordering: +8 + │ ├── limit hint: 1.00 + │ └── group-by + │ ├── columns: i:8 max:13 + │ ├── grouping columns: i:8 + │ ├── project + │ │ ├── columns: i:8 s:10 + │ │ └── scan a + │ │ └── columns: k:7!null i:8 f:9 s:10 j:11 crdb_internal_mvcc_timestamp:12 + │ └── aggregations + │ └── max [as=max:13] + │ └── s:10 + └── 1 exec-ddl CREATE TABLE t (a string) @@ -1664,45 +1688,45 @@ build SELECT max(a) FROM t HAVING max(a) < ANY(SELECT b FROM u) ---- select - ├── columns: max:3 + ├── columns: max:4 ├── scalar-group-by - │ ├── columns: max:3 + │ ├── columns: max:4 │ ├── project │ │ ├── columns: a:1 │ │ └── scan t - │ │ └── columns: a:1 t.rowid:2!null + │ │ └── columns: a:1 t.rowid:2!null t.crdb_internal_mvcc_timestamp:3 │ └── aggregations - │ └── max [as=max:3] + │ └── max [as=max:4] │ └── a:1 └── filters └── any: lt ├── project - │ ├── columns: b:4 + │ ├── columns: b:5 │ └── scan u - │ └── columns: b:4 u.rowid:5!null - └── max:3 + │ └── columns: b:5 u.rowid:6!null u.crdb_internal_mvcc_timestamp:7 + └── max:4 build SELECT min(a) IN (SELECT b FROM u) FROM t ---- project - ├── columns: "?column?":6 + ├── columns: "?column?":8 ├── scalar-group-by - │ ├── columns: min:3 + │ ├── columns: min:4 │ ├── project │ │ ├── columns: a:1 │ │ └── scan t - │ │ └── columns: a:1 t.rowid:2!null + │ │ └── columns: a:1 t.rowid:2!null t.crdb_internal_mvcc_timestamp:3 │ └── aggregations - │ └── min [as=min:3] + │ └── min [as=min:4] │ └── a:1 └── projections - └── any: eq [as="?column?":6] + └── any: eq [as="?column?":8] ├── project - │ ├── columns: b:4 + │ ├── columns: b:5 │ └── scan u - │ └── columns: b:4 u.rowid:5!null - └── min:3 + │ └── columns: b:5 u.rowid:6!null u.crdb_internal_mvcc_timestamp:7 + └── min:4 # Regression test for #28240. Make sure that the tuple labels are stripped from # the subquery. @@ -1734,36 +1758,36 @@ build SELECT (SELECT (SELECT max(t3.a) FROM t1) FROM t2) FROM t3 ---- project - ├── columns: max:14 + ├── columns: max:17 ├── scalar-group-by - │ ├── columns: max:11 + │ ├── columns: max:14 │ ├── project - │ │ ├── columns: a:10 + │ │ ├── columns: a:13 │ │ ├── scan t3 - │ │ │ └── columns: t3.a:1 t3.b:2 t3.rowid:3!null + │ │ │ └── columns: t3.a:1 t3.b:2 t3.rowid:3!null t3.crdb_internal_mvcc_timestamp:4 │ │ └── projections - │ │ └── t3.a:1 [as=a:10] + │ │ └── t3.a:1 [as=a:13] │ └── aggregations - │ └── max [as=max:11] - │ └── a:10 + │ └── max [as=max:14] + │ └── a:13 └── projections - └── subquery [as=max:14] + └── subquery [as=max:17] └── max1-row - ├── columns: max:13 + ├── columns: max:16 └── project - ├── columns: max:13 + ├── columns: max:16 ├── scan t2 - │ └── columns: t2.a:4 t2.b:5 t2.rowid:6!null + │ └── columns: t2.a:5 t2.b:6 t2.rowid:7!null t2.crdb_internal_mvcc_timestamp:8 └── projections - └── subquery [as=max:13] + └── subquery [as=max:16] └── max1-row - ├── columns: max:12 + ├── columns: max:15 └── project - ├── columns: max:12 + ├── columns: max:15 ├── scan t1 - │ └── columns: t1.a:7 t1.b:8 t1.rowid:9!null + │ └── columns: t1.a:9 t1.b:10 t1.rowid:11!null t1.crdb_internal_mvcc_timestamp:12 └── projections - └── max:11 [as=max:12] + └── max:14 [as=max:15] build SELECT ( @@ -1773,47 +1797,47 @@ SELECT ( FROM t3; ---- project - ├── columns: row:17 + ├── columns: row:20 ├── scan t3 - │ └── columns: t3.a:1 t3.b:2 t3.rowid:3!null + │ └── columns: t3.a:1 t3.b:2 t3.rowid:3!null t3.crdb_internal_mvcc_timestamp:4 └── projections - └── subquery [as=row:17] + └── subquery [as=row:20] └── max1-row - ├── columns: row:16 + ├── columns: row:19 └── project - ├── columns: row:16 + ├── columns: row:19 ├── scalar-group-by - │ ├── columns: max:12 + │ ├── columns: max:15 │ ├── project - │ │ ├── columns: a:11 + │ │ ├── columns: a:14 │ │ ├── scan t2 - │ │ │ └── columns: t2.a:4 t2.b:5 t2.rowid:6!null + │ │ │ └── columns: t2.a:5 t2.b:6 t2.rowid:7!null t2.crdb_internal_mvcc_timestamp:8 │ │ └── projections - │ │ └── t2.a:4 [as=a:11] + │ │ └── t2.a:5 [as=a:14] │ └── aggregations - │ └── max [as=max:12] - │ └── a:11 + │ └── max [as=max:15] + │ └── a:14 └── projections - └── subquery [as=row:16] + └── subquery [as=row:19] └── max1-row - ├── columns: row:15 + ├── columns: row:18 └── project - ├── columns: row:15 + ├── columns: row:18 ├── scalar-group-by - │ ├── columns: max:10 max:14 + │ ├── columns: max:13 max:17 │ ├── project - │ │ ├── columns: column13:13 t1.a:7 + │ │ ├── columns: column16:16 t1.a:9 │ │ ├── scan t1 - │ │ │ └── columns: t1.a:7 t1.b:8 t1.rowid:9!null + │ │ │ └── columns: t1.a:9 t1.b:10 t1.rowid:11!null t1.crdb_internal_mvcc_timestamp:12 │ │ └── projections - │ │ └── t1.a:7 + t3.a:1 [as=column13:13] + │ │ └── t1.a:9 + t3.a:1 [as=column16:16] │ └── aggregations - │ ├── max [as=max:10] - │ │ └── t1.a:7 - │ └── max [as=max:14] - │ └── column13:13 + │ ├── max [as=max:13] + │ │ └── t1.a:9 + │ └── max [as=max:17] + │ └── column16:16 └── projections - └── (max:10, max:12, max:14) [as=row:15] + └── (max:13, max:15, max:17) [as=row:18] build SELECT (SELECT row(max(t1.a), max(t2.a), max(t1.a + t2.a)) FROM t1) FROM t2; @@ -1824,40 +1848,40 @@ build SELECT (SELECT row(max(t1.a), max(t2.a), max(t1.a + t2.a)) FROM t1) FROM t2 GROUP BY t2.a; ---- project - ├── columns: row:13 + ├── columns: row:15 ├── group-by - │ ├── columns: t2.a:1 max:9 + │ ├── columns: t2.a:1 max:11 │ ├── grouping columns: t2.a:1 │ ├── project - │ │ ├── columns: a:8 t2.a:1 + │ │ ├── columns: a:10 t2.a:1 │ │ ├── scan t2 - │ │ │ └── columns: t2.a:1 t2.b:2 t2.rowid:3!null + │ │ │ └── columns: t2.a:1 t2.b:2 t2.rowid:3!null t2.crdb_internal_mvcc_timestamp:4 │ │ └── projections - │ │ └── t2.a:1 [as=a:8] + │ │ └── t2.a:1 [as=a:10] │ └── aggregations - │ └── max [as=max:9] - │ └── a:8 + │ └── max [as=max:11] + │ └── a:10 └── projections - └── subquery [as=row:13] + └── subquery [as=row:15] └── max1-row - ├── columns: row:12 + ├── columns: row:14 └── project - ├── columns: row:12 + ├── columns: row:14 ├── scalar-group-by - │ ├── columns: max:7 max:11 + │ ├── columns: max:9 max:13 │ ├── project - │ │ ├── columns: column10:10 t1.a:4 + │ │ ├── columns: column12:12 t1.a:5 │ │ ├── scan t1 - │ │ │ └── columns: t1.a:4 t1.b:5 t1.rowid:6!null + │ │ │ └── columns: t1.a:5 t1.b:6 t1.rowid:7!null t1.crdb_internal_mvcc_timestamp:8 │ │ └── projections - │ │ └── t1.a:4 + t2.a:1 [as=column10:10] + │ │ └── t1.a:5 + t2.a:1 [as=column12:12] │ └── aggregations - │ ├── max [as=max:7] - │ │ └── t1.a:4 - │ └── max [as=max:11] - │ └── column10:10 + │ ├── max [as=max:9] + │ │ └── t1.a:5 + │ └── max [as=max:13] + │ └── column12:12 └── projections - └── (max:7, max:9, max:11) [as=row:12] + └── (max:9, max:11, max:13) [as=row:14] build SELECT @@ -1894,48 +1918,48 @@ FROM t2; ---- project - ├── columns: max:14 max:15 + ├── columns: max:17 max:18 ├── scalar-group-by - │ ├── columns: max:12 + │ ├── columns: max:15 │ ├── project - │ │ ├── columns: a:11 + │ │ ├── columns: a:14 │ │ ├── scan t2 - │ │ │ └── columns: t2.a:1 t2.b:2 t2.rowid:3!null + │ │ │ └── columns: t2.a:1 t2.b:2 t2.rowid:3!null t2.crdb_internal_mvcc_timestamp:4 │ │ └── projections - │ │ └── t2.a:1 [as=a:11] + │ │ └── t2.a:1 [as=a:14] │ └── aggregations - │ └── max [as=max:12] - │ └── a:11 + │ └── max [as=max:15] + │ └── a:14 └── projections - ├── subquery [as=max:14] + ├── subquery [as=max:17] │ └── max1-row - │ ├── columns: max:7 + │ ├── columns: max:9 │ └── project - │ ├── columns: max:7 + │ ├── columns: max:9 │ └── group-by - │ ├── columns: t1.b:5 max:7 - │ ├── grouping columns: t1.b:5 + │ ├── columns: t1.b:6 max:9 + │ ├── grouping columns: t1.b:6 │ ├── project - │ │ ├── columns: t1.a:4 t1.b:5 + │ │ ├── columns: t1.a:5 t1.b:6 │ │ └── scan t1 - │ │ └── columns: t1.a:4 t1.b:5 t1.rowid:6!null + │ │ └── columns: t1.a:5 t1.b:6 t1.rowid:7!null t1.crdb_internal_mvcc_timestamp:8 │ └── aggregations - │ └── max [as=max:7] - │ └── t1.a:4 - └── subquery [as=max:15] + │ └── max [as=max:9] + │ └── t1.a:5 + └── subquery [as=max:18] └── max1-row - ├── columns: max:13 + ├── columns: max:16 └── project - ├── columns: max:13 + ├── columns: max:16 ├── group-by - │ ├── columns: t1.b:9 - │ ├── grouping columns: t1.b:9 + │ ├── columns: t1.b:11 + │ ├── grouping columns: t1.b:11 │ └── project - │ ├── columns: t1.b:9 + │ ├── columns: t1.b:11 │ └── scan t1 - │ └── columns: t1.a:8 t1.b:9 t1.rowid:10!null + │ └── columns: t1.a:10 t1.b:11 t1.rowid:12!null t1.crdb_internal_mvcc_timestamp:13 └── projections - └── max:12 [as=max:13] + └── max:15 [as=max:16] build SELECT @@ -1946,37 +1970,37 @@ FROM GROUP BY t2.b; ---- project - ├── columns: array:14 array:15 + ├── columns: array:17 array:18 ├── group-by - │ ├── columns: t2.b:2 max:12 + │ ├── columns: t2.b:2 max:15 │ ├── grouping columns: t2.b:2 │ ├── project - │ │ ├── columns: a:11 t2.b:2 + │ │ ├── columns: a:14 t2.b:2 │ │ ├── scan t2 - │ │ │ └── columns: t2.a:1 t2.b:2 t2.rowid:3!null + │ │ │ └── columns: t2.a:1 t2.b:2 t2.rowid:3!null t2.crdb_internal_mvcc_timestamp:4 │ │ └── projections - │ │ └── t2.a:1 [as=a:11] + │ │ └── t2.a:1 [as=a:14] │ └── aggregations - │ └── max [as=max:12] - │ └── a:11 + │ └── max [as=max:15] + │ └── a:14 └── projections - ├── array-flatten [as=array:14] + ├── array-flatten [as=array:17] │ └── scalar-group-by - │ ├── columns: max:7 + │ ├── columns: max:9 │ ├── project - │ │ ├── columns: t1.a:4 + │ │ ├── columns: t1.a:5 │ │ └── scan t1 - │ │ └── columns: t1.a:4 t1.b:5 t1.rowid:6!null + │ │ └── columns: t1.a:5 t1.b:6 t1.rowid:7!null t1.crdb_internal_mvcc_timestamp:8 │ └── aggregations - │ └── max [as=max:7] - │ └── t1.a:4 - └── array-flatten [as=array:15] + │ └── max [as=max:9] + │ └── t1.a:5 + └── array-flatten [as=array:18] └── project - ├── columns: max:13 + ├── columns: max:16 ├── scan t1 - │ └── columns: t1.a:8 t1.b:9 t1.rowid:10!null + │ └── columns: t1.a:10 t1.b:11 t1.rowid:12!null t1.crdb_internal_mvcc_timestamp:13 └── projections - └── max:12 [as=max:13] + └── max:15 [as=max:16] build SELECT @@ -1988,65 +2012,65 @@ FROM GROUP BY t2.a, t2.b; ---- project - ├── columns: array:22 array:23 array:24 + ├── columns: array:26 array:27 array:28 ├── group-by - │ ├── columns: t2.a:1 t2.b:2 max:13 + │ ├── columns: t2.a:1 t2.b:2 max:16 │ ├── grouping columns: t2.a:1 t2.b:2 │ ├── project - │ │ ├── columns: a:12 t2.a:1 t2.b:2 + │ │ ├── columns: a:15 t2.a:1 t2.b:2 │ │ ├── scan t2 - │ │ │ └── columns: t2.a:1 t2.b:2 t2.rowid:3!null + │ │ │ └── columns: t2.a:1 t2.b:2 t2.rowid:3!null t2.crdb_internal_mvcc_timestamp:4 │ │ └── projections - │ │ └── t2.a:1 [as=a:12] + │ │ └── t2.a:1 [as=a:15] │ └── aggregations - │ └── max [as=max:13] - │ └── a:12 + │ └── max [as=max:16] + │ └── a:15 └── projections - ├── array-flatten [as=array:22] + ├── array-flatten [as=array:26] │ └── project - │ ├── columns: max:7 + │ ├── columns: max:9 │ └── group-by - │ ├── columns: max:7 b:8 - │ ├── grouping columns: b:8 + │ ├── columns: max:9 b:10 + │ ├── grouping columns: b:10 │ ├── project - │ │ ├── columns: b:8 t1.a:4 + │ │ ├── columns: b:10 t1.a:5 │ │ ├── scan t1 - │ │ │ └── columns: t1.a:4 t1.b:5 t1.rowid:6!null + │ │ │ └── columns: t1.a:5 t1.b:6 t1.rowid:7!null t1.crdb_internal_mvcc_timestamp:8 │ │ └── projections - │ │ └── t2.b:2 [as=b:8] + │ │ └── t2.b:2 [as=b:10] │ └── aggregations - │ └── max [as=max:7] - │ └── t1.a:4 - ├── array-flatten [as=array:23] + │ └── max [as=max:9] + │ └── t1.a:5 + ├── array-flatten [as=array:27] │ └── project - │ ├── columns: max:15 + │ ├── columns: max:18 │ ├── group-by - │ │ ├── columns: b:14 - │ │ ├── grouping columns: b:14 + │ │ ├── columns: b:17 + │ │ ├── grouping columns: b:17 │ │ └── project - │ │ ├── columns: b:14 + │ │ ├── columns: b:17 │ │ ├── scan t1 - │ │ │ └── columns: t1.a:9 t1.b:10 t1.rowid:11!null + │ │ │ └── columns: t1.a:11 t1.b:12 t1.rowid:13!null t1.crdb_internal_mvcc_timestamp:14 │ │ └── projections - │ │ └── t2.b:2 [as=b:14] + │ │ └── t2.b:2 [as=b:17] │ └── projections - │ └── max:13 [as=max:15] - └── array-flatten [as=array:24] + │ └── max:16 [as=max:18] + └── array-flatten [as=array:28] └── project - ├── columns: max:20 + ├── columns: max:24 └── group-by - ├── columns: max:20 b:21 - ├── grouping columns: b:21 + ├── columns: max:24 b:25 + ├── grouping columns: b:25 ├── project - │ ├── columns: column19:19 b:21 + │ ├── columns: column23:23 b:25 │ ├── scan t1 - │ │ └── columns: t1.a:16 t1.b:17 t1.rowid:18!null + │ │ └── columns: t1.a:19 t1.b:20 t1.rowid:21!null t1.crdb_internal_mvcc_timestamp:22 │ └── projections - │ ├── t1.a:16 + t2.a:1 [as=column19:19] - │ └── t2.b:2 [as=b:21] + │ ├── t1.a:19 + t2.a:1 [as=column23:23] + │ └── t2.b:2 [as=b:25] └── aggregations - └── max [as=max:20] - └── column19:19 + └── max [as=max:24] + └── column23:23 build SELECT @@ -2068,57 +2092,57 @@ FROM t3; ---- project - ├── columns: "?column?":21 + ├── columns: "?column?":25 ├── scalar-group-by - │ ├── columns: max:8 max:18 + │ ├── columns: max:10 max:22 │ ├── project - │ │ ├── columns: a:7 column17:17 + │ │ ├── columns: a:9 column21:21 │ │ ├── scan t3 - │ │ │ └── columns: t3.a:1 t3.b:2 t3.rowid:3!null + │ │ │ └── columns: t3.a:1 t3.b:2 t3.rowid:3!null t3.crdb_internal_mvcc_timestamp:4 │ │ └── projections - │ │ ├── t3.a:1 [as=a:7] - │ │ └── subquery [as=column17:17] + │ │ ├── t3.a:1 [as=a:9] + │ │ └── subquery [as=column21:21] │ │ └── max1-row - │ │ ├── columns: max:15 + │ │ ├── columns: max:19 │ │ └── project - │ │ ├── columns: max:15 + │ │ ├── columns: max:19 │ │ └── group-by - │ │ ├── columns: max:15 b:16 - │ │ ├── grouping columns: b:16 + │ │ ├── columns: max:19 b:20 + │ │ ├── grouping columns: b:20 │ │ ├── project - │ │ │ ├── columns: b:16 t1.a:12 + │ │ │ ├── columns: b:20 t1.a:15 │ │ │ ├── scan t1 - │ │ │ │ └── columns: t1.a:12 t1.b:13 t1.rowid:14!null + │ │ │ │ └── columns: t1.a:15 t1.b:16 t1.rowid:17!null t1.crdb_internal_mvcc_timestamp:18 │ │ │ └── projections - │ │ │ └── t3.b:2 [as=b:16] + │ │ │ └── t3.b:2 [as=b:20] │ │ └── aggregations - │ │ └── max [as=max:15] - │ │ └── t1.a:12 + │ │ └── max [as=max:19] + │ │ └── t1.a:15 │ └── aggregations - │ ├── max [as=max:8] - │ │ └── a:7 - │ └── max [as=max:18] - │ └── column17:17 + │ ├── max [as=max:10] + │ │ └── a:9 + │ └── max [as=max:22] + │ └── column21:21 └── projections - └── subquery [as="?column?":21] + └── subquery [as="?column?":25] └── max1-row - ├── columns: "?column?":20 + ├── columns: "?column?":24 └── project - ├── columns: "?column?":20 + ├── columns: "?column?":24 ├── scan t2 - │ └── columns: t2.a:4 t2.b:5 t2.rowid:6!null + │ └── columns: t2.a:5 t2.b:6 t2.rowid:7!null t2.crdb_internal_mvcc_timestamp:8 └── projections - └── tuple [as="?column?":20] + └── tuple [as="?column?":24] ├── not │ └── any: ge │ ├── project - │ │ ├── columns: max:19 + │ │ ├── columns: max:23 │ │ ├── scan t1 - │ │ │ └── columns: t1.a:9 t1.b:10 t1.rowid:11!null + │ │ │ └── columns: t1.a:11 t1.b:12 t1.rowid:13!null t1.crdb_internal_mvcc_timestamp:14 │ │ └── projections - │ │ └── max:18 [as=max:19] - │ └── t2.a:4 - └── max:8 + │ │ └── max:22 [as=max:23] + │ └── t2.a:5 + └── max:10 exec-ddl CREATE TABLE v (x INT) @@ -2135,20 +2159,20 @@ SELECT * FROM w WHERE y = ARRAY(SELECT x FROM v ORDER BY x) project ├── columns: y:1!null └── select - ├── columns: y:1!null w.rowid:2!null + ├── columns: y:1!null w.rowid:2!null w.crdb_internal_mvcc_timestamp:3 ├── scan w - │ └── columns: y:1 w.rowid:2!null + │ └── columns: y:1 w.rowid:2!null w.crdb_internal_mvcc_timestamp:3 └── filters └── eq ├── y:1 └── array-flatten └── sort - ├── columns: x:3 - ├── ordering: +3 + ├── columns: x:4 + ├── ordering: +4 └── project - ├── columns: x:3 + ├── columns: x:4 └── scan v - └── columns: x:3 v.rowid:4!null + └── columns: x:4 v.rowid:5!null v.crdb_internal_mvcc_timestamp:6 # Regression test for #30424. Aggregate function in the WHERE subquery # is aggregated in the outer scope, so it's not allowed. @@ -2168,26 +2192,26 @@ group-by └── project ├── columns: s:4 └── select - ├── columns: k:1!null i:2 f:3 s:4 j:5 + ├── columns: k:1!null i:2 f:3 s:4 j:5 crdb_internal_mvcc_timestamp:6 ├── scan a - │ └── columns: k:1!null i:2 f:3 s:4 j:5 + │ └── columns: k:1!null i:2 f:3 s:4 j:5 crdb_internal_mvcc_timestamp:6 └── filters └── subquery └── max1-row - ├── columns: "?column?":12!null + ├── columns: "?column?":14!null └── project - ├── columns: "?column?":12!null + ├── columns: "?column?":14!null ├── scalar-group-by - │ ├── columns: count:11!null + │ ├── columns: count:13!null │ ├── project - │ │ ├── columns: i:7 + │ │ ├── columns: i:8 │ │ └── scan a - │ │ └── columns: k:6!null i:7 f:8 s:9 j:10 + │ │ └── columns: k:7!null i:8 f:9 s:10 j:11 crdb_internal_mvcc_timestamp:12 │ └── aggregations - │ └── count [as=count:11] - │ └── i:7 + │ └── count [as=count:13] + │ └── i:8 └── projections - └── count:11 >= 100 [as="?column?":12] + └── count:13 >= 100 [as="?column?":14] exec-ddl CREATE TABLE xyzs (x INT PRIMARY KEY, y INT, z FLOAT NOT NULL, s STRING, UNIQUE (s DESC, z)); diff --git a/pkg/sql/opt/optbuilder/testdata/union b/pkg/sql/opt/optbuilder/testdata/union index 94fabb724a98..cadb97740dfa 100644 --- a/pkg/sql/opt/optbuilder/testdata/union +++ b/pkg/sql/opt/optbuilder/testdata/union @@ -302,49 +302,49 @@ build SELECT v FROM uniontest WHERE k = 1 UNION SELECT v FROM uniontest WHERE k = 2 ---- union - ├── columns: v:7 + ├── columns: v:9 ├── left columns: uniontest.v:2 - ├── right columns: uniontest.v:5 + ├── right columns: uniontest.v:6 ├── project │ ├── columns: uniontest.v:2 │ └── select - │ ├── columns: k:1!null uniontest.v:2 rowid:3!null + │ ├── columns: k:1!null uniontest.v:2 rowid:3!null crdb_internal_mvcc_timestamp:4 │ ├── scan uniontest - │ │ └── columns: k:1 uniontest.v:2 rowid:3!null + │ │ └── columns: k:1 uniontest.v:2 rowid:3!null crdb_internal_mvcc_timestamp:4 │ └── filters │ └── k:1 = 1 └── project - ├── columns: uniontest.v:5 + ├── columns: uniontest.v:6 └── select - ├── columns: k:4!null uniontest.v:5 rowid:6!null + ├── columns: k:5!null uniontest.v:6 rowid:7!null crdb_internal_mvcc_timestamp:8 ├── scan uniontest - │ └── columns: k:4 uniontest.v:5 rowid:6!null + │ └── columns: k:5 uniontest.v:6 rowid:7!null crdb_internal_mvcc_timestamp:8 └── filters - └── k:4 = 2 + └── k:5 = 2 build SELECT v FROM uniontest WHERE k = 1 UNION ALL SELECT v FROM uniontest WHERE k = 2 ---- union-all - ├── columns: v:7 + ├── columns: v:9 ├── left columns: uniontest.v:2 - ├── right columns: uniontest.v:5 + ├── right columns: uniontest.v:6 ├── project │ ├── columns: uniontest.v:2 │ └── select - │ ├── columns: k:1!null uniontest.v:2 rowid:3!null + │ ├── columns: k:1!null uniontest.v:2 rowid:3!null crdb_internal_mvcc_timestamp:4 │ ├── scan uniontest - │ │ └── columns: k:1 uniontest.v:2 rowid:3!null + │ │ └── columns: k:1 uniontest.v:2 rowid:3!null crdb_internal_mvcc_timestamp:4 │ └── filters │ └── k:1 = 1 └── project - ├── columns: uniontest.v:5 + ├── columns: uniontest.v:6 └── select - ├── columns: k:4!null uniontest.v:5 rowid:6!null + ├── columns: k:5!null uniontest.v:6 rowid:7!null crdb_internal_mvcc_timestamp:8 ├── scan uniontest - │ └── columns: k:4 uniontest.v:5 rowid:6!null + │ └── columns: k:5 uniontest.v:6 rowid:7!null crdb_internal_mvcc_timestamp:8 └── filters - └── k:4 = 2 + └── k:5 = 2 build SELECT v FROM uniontest WHERE k = 1 INTERSECT SELECT v FROM uniontest WHERE k = 2 @@ -352,23 +352,23 @@ SELECT v FROM uniontest WHERE k = 1 INTERSECT SELECT v FROM uniontest WHERE k = intersect ├── columns: v:2 ├── left columns: v:2 - ├── right columns: v:5 + ├── right columns: v:6 ├── project │ ├── columns: v:2 │ └── select - │ ├── columns: k:1!null v:2 rowid:3!null + │ ├── columns: k:1!null v:2 rowid:3!null crdb_internal_mvcc_timestamp:4 │ ├── scan uniontest - │ │ └── columns: k:1 v:2 rowid:3!null + │ │ └── columns: k:1 v:2 rowid:3!null crdb_internal_mvcc_timestamp:4 │ └── filters │ └── k:1 = 1 └── project - ├── columns: v:5 + ├── columns: v:6 └── select - ├── columns: k:4!null v:5 rowid:6!null + ├── columns: k:5!null v:6 rowid:7!null crdb_internal_mvcc_timestamp:8 ├── scan uniontest - │ └── columns: k:4 v:5 rowid:6!null + │ └── columns: k:5 v:6 rowid:7!null crdb_internal_mvcc_timestamp:8 └── filters - └── k:4 = 2 + └── k:5 = 2 build SELECT v FROM uniontest WHERE k = 1 INTERSECT ALL SELECT v FROM uniontest WHERE k = 2 @@ -376,23 +376,23 @@ SELECT v FROM uniontest WHERE k = 1 INTERSECT ALL SELECT v FROM uniontest WHERE intersect-all ├── columns: v:2 ├── left columns: v:2 - ├── right columns: v:5 + ├── right columns: v:6 ├── project │ ├── columns: v:2 │ └── select - │ ├── columns: k:1!null v:2 rowid:3!null + │ ├── columns: k:1!null v:2 rowid:3!null crdb_internal_mvcc_timestamp:4 │ ├── scan uniontest - │ │ └── columns: k:1 v:2 rowid:3!null + │ │ └── columns: k:1 v:2 rowid:3!null crdb_internal_mvcc_timestamp:4 │ └── filters │ └── k:1 = 1 └── project - ├── columns: v:5 + ├── columns: v:6 └── select - ├── columns: k:4!null v:5 rowid:6!null + ├── columns: k:5!null v:6 rowid:7!null crdb_internal_mvcc_timestamp:8 ├── scan uniontest - │ └── columns: k:4 v:5 rowid:6!null + │ └── columns: k:5 v:6 rowid:7!null crdb_internal_mvcc_timestamp:8 └── filters - └── k:4 = 2 + └── k:5 = 2 build SELECT v FROM uniontest WHERE k = 1 EXCEPT SELECT v FROM uniontest WHERE k = 2 @@ -400,23 +400,23 @@ SELECT v FROM uniontest WHERE k = 1 EXCEPT SELECT v FROM uniontest WHERE k = 2 except ├── columns: v:2 ├── left columns: v:2 - ├── right columns: v:5 + ├── right columns: v:6 ├── project │ ├── columns: v:2 │ └── select - │ ├── columns: k:1!null v:2 rowid:3!null + │ ├── columns: k:1!null v:2 rowid:3!null crdb_internal_mvcc_timestamp:4 │ ├── scan uniontest - │ │ └── columns: k:1 v:2 rowid:3!null + │ │ └── columns: k:1 v:2 rowid:3!null crdb_internal_mvcc_timestamp:4 │ └── filters │ └── k:1 = 1 └── project - ├── columns: v:5 + ├── columns: v:6 └── select - ├── columns: k:4!null v:5 rowid:6!null + ├── columns: k:5!null v:6 rowid:7!null crdb_internal_mvcc_timestamp:8 ├── scan uniontest - │ └── columns: k:4 v:5 rowid:6!null + │ └── columns: k:5 v:6 rowid:7!null crdb_internal_mvcc_timestamp:8 └── filters - └── k:4 = 2 + └── k:5 = 2 build SELECT v FROM uniontest WHERE k = 1 EXCEPT ALL SELECT v FROM uniontest WHERE k = 2 @@ -424,55 +424,55 @@ SELECT v FROM uniontest WHERE k = 1 EXCEPT ALL SELECT v FROM uniontest WHERE k = except-all ├── columns: v:2 ├── left columns: v:2 - ├── right columns: v:5 + ├── right columns: v:6 ├── project │ ├── columns: v:2 │ └── select - │ ├── columns: k:1!null v:2 rowid:3!null + │ ├── columns: k:1!null v:2 rowid:3!null crdb_internal_mvcc_timestamp:4 │ ├── scan uniontest - │ │ └── columns: k:1 v:2 rowid:3!null + │ │ └── columns: k:1 v:2 rowid:3!null crdb_internal_mvcc_timestamp:4 │ └── filters │ └── k:1 = 1 └── project - ├── columns: v:5 + ├── columns: v:6 └── select - ├── columns: k:4!null v:5 rowid:6!null + ├── columns: k:5!null v:6 rowid:7!null crdb_internal_mvcc_timestamp:8 ├── scan uniontest - │ └── columns: k:4 v:5 rowid:6!null + │ └── columns: k:5 v:6 rowid:7!null crdb_internal_mvcc_timestamp:8 └── filters - └── k:4 = 2 + └── k:5 = 2 build (SELECT v FROM uniontest WHERE k = 1 UNION ALL SELECT v FROM uniontest WHERE k = 2) ORDER BY 1 DESC LIMIT 2 ---- limit - ├── columns: v:7 - ├── internal-ordering: -7 - ├── ordering: -7 + ├── columns: v:9 + ├── internal-ordering: -9 + ├── ordering: -9 ├── sort - │ ├── columns: v:7 - │ ├── ordering: -7 + │ ├── columns: v:9 + │ ├── ordering: -9 │ ├── limit hint: 2.00 │ └── union-all - │ ├── columns: v:7 + │ ├── columns: v:9 │ ├── left columns: uniontest.v:2 - │ ├── right columns: uniontest.v:5 + │ ├── right columns: uniontest.v:6 │ ├── project │ │ ├── columns: uniontest.v:2 │ │ └── select - │ │ ├── columns: k:1!null uniontest.v:2 rowid:3!null + │ │ ├── columns: k:1!null uniontest.v:2 rowid:3!null crdb_internal_mvcc_timestamp:4 │ │ ├── scan uniontest - │ │ │ └── columns: k:1 uniontest.v:2 rowid:3!null + │ │ │ └── columns: k:1 uniontest.v:2 rowid:3!null crdb_internal_mvcc_timestamp:4 │ │ └── filters │ │ └── k:1 = 1 │ └── project - │ ├── columns: uniontest.v:5 + │ ├── columns: uniontest.v:6 │ └── select - │ ├── columns: k:4!null uniontest.v:5 rowid:6!null + │ ├── columns: k:5!null uniontest.v:6 rowid:7!null crdb_internal_mvcc_timestamp:8 │ ├── scan uniontest - │ │ └── columns: k:4 uniontest.v:5 rowid:6!null + │ │ └── columns: k:5 uniontest.v:6 rowid:7!null crdb_internal_mvcc_timestamp:8 │ └── filters - │ └── k:4 = 2 + │ └── k:5 = 2 └── 2 # The ORDER BY and LIMIT apply to the UNION, not the last SELECT. @@ -480,66 +480,66 @@ build SELECT v FROM uniontest WHERE k = 1 UNION ALL SELECT v FROM uniontest WHERE k = 2 ORDER BY 1 DESC LIMIT 2 ---- limit - ├── columns: v:7 - ├── internal-ordering: -7 - ├── ordering: -7 + ├── columns: v:9 + ├── internal-ordering: -9 + ├── ordering: -9 ├── sort - │ ├── columns: v:7 - │ ├── ordering: -7 + │ ├── columns: v:9 + │ ├── ordering: -9 │ ├── limit hint: 2.00 │ └── union-all - │ ├── columns: v:7 + │ ├── columns: v:9 │ ├── left columns: uniontest.v:2 - │ ├── right columns: uniontest.v:5 + │ ├── right columns: uniontest.v:6 │ ├── project │ │ ├── columns: uniontest.v:2 │ │ └── select - │ │ ├── columns: k:1!null uniontest.v:2 rowid:3!null + │ │ ├── columns: k:1!null uniontest.v:2 rowid:3!null crdb_internal_mvcc_timestamp:4 │ │ ├── scan uniontest - │ │ │ └── columns: k:1 uniontest.v:2 rowid:3!null + │ │ │ └── columns: k:1 uniontest.v:2 rowid:3!null crdb_internal_mvcc_timestamp:4 │ │ └── filters │ │ └── k:1 = 1 │ └── project - │ ├── columns: uniontest.v:5 + │ ├── columns: uniontest.v:6 │ └── select - │ ├── columns: k:4!null uniontest.v:5 rowid:6!null + │ ├── columns: k:5!null uniontest.v:6 rowid:7!null crdb_internal_mvcc_timestamp:8 │ ├── scan uniontest - │ │ └── columns: k:4 uniontest.v:5 rowid:6!null + │ │ └── columns: k:5 uniontest.v:6 rowid:7!null crdb_internal_mvcc_timestamp:8 │ └── filters - │ └── k:4 = 2 + │ └── k:5 = 2 └── 2 build SELECT v FROM uniontest UNION SELECT k FROM uniontest ---- union - ├── columns: v:7 + ├── columns: v:9 ├── left columns: uniontest.v:2 - ├── right columns: k:4 + ├── right columns: k:5 ├── project │ ├── columns: uniontest.v:2 │ └── scan uniontest - │ └── columns: k:1 uniontest.v:2 rowid:3!null + │ └── columns: k:1 uniontest.v:2 rowid:3!null crdb_internal_mvcc_timestamp:4 └── project - ├── columns: k:4 + ├── columns: k:5 └── scan uniontest - └── columns: k:4 uniontest.v:5 rowid:6!null + └── columns: k:5 uniontest.v:6 rowid:7!null crdb_internal_mvcc_timestamp:8 build SELECT v FROM uniontest UNION ALL SELECT k FROM uniontest ---- union-all - ├── columns: v:7 + ├── columns: v:9 ├── left columns: uniontest.v:2 - ├── right columns: k:4 + ├── right columns: k:5 ├── project │ ├── columns: uniontest.v:2 │ └── scan uniontest - │ └── columns: k:1 uniontest.v:2 rowid:3!null + │ └── columns: k:1 uniontest.v:2 rowid:3!null crdb_internal_mvcc_timestamp:4 └── project - ├── columns: k:4 + ├── columns: k:5 └── scan uniontest - └── columns: k:4 uniontest.v:5 rowid:6!null + └── columns: k:5 uniontest.v:6 rowid:7!null crdb_internal_mvcc_timestamp:8 build SELECT * FROM (SELECT * FROM (VALUES (1)) a LEFT JOIN (VALUES (1) UNION VALUES (2)) b on a.column1 = b.column1); @@ -694,55 +694,55 @@ build (SELECT x, x, y FROM xy) UNION (SELECT a, b, c FROM abc) ---- union - ├── columns: x:8 x:9!null y:10!null + ├── columns: x:10 x:11!null y:12!null ├── left columns: xy.x:1 xy.x:1 xy.y:2 - ├── right columns: a:4 b:5 c:6 + ├── right columns: a:5 b:6 c:7 ├── project │ ├── columns: xy.x:1!null xy.y:2!null │ └── scan xy - │ └── columns: xy.x:1!null xy.y:2!null xy.rowid:3!null + │ └── columns: xy.x:1!null xy.y:2!null xy.rowid:3!null xy.crdb_internal_mvcc_timestamp:4 └── project - ├── columns: a:4 b:5!null c:6!null + ├── columns: a:5 b:6!null c:7!null └── scan abc - └── columns: a:4 b:5!null c:6!null abc.rowid:7!null + └── columns: a:5 b:6!null c:7!null abc.rowid:8!null abc.crdb_internal_mvcc_timestamp:9 build (SELECT a FROM abc ORDER BY b) UNION ALL (SELECT b FROM abc) ORDER BY a ---- sort - ├── columns: a:9 - ├── ordering: +9 + ├── columns: a:11 + ├── ordering: +11 └── union-all - ├── columns: a:9 + ├── columns: a:11 ├── left columns: abc.a:1 - ├── right columns: b:6 + ├── right columns: b:7 ├── project │ ├── columns: abc.a:1 b:2!null │ └── scan abc - │ └── columns: abc.a:1 b:2!null c:3!null rowid:4!null + │ └── columns: abc.a:1 b:2!null c:3!null rowid:4!null crdb_internal_mvcc_timestamp:5 └── project - ├── columns: b:6!null + ├── columns: b:7!null └── scan abc - └── columns: abc.a:5 b:6!null c:7!null rowid:8!null + └── columns: abc.a:6 b:7!null c:8!null rowid:9!null crdb_internal_mvcc_timestamp:10 build (SELECT a FROM abc ORDER BY b) UNION ALL (SELECT a FROM abc ORDER BY c) ORDER BY a ---- sort - ├── columns: a:9 - ├── ordering: +9 + ├── columns: a:11 + ├── ordering: +11 └── union-all - ├── columns: a:9 + ├── columns: a:11 ├── left columns: abc.a:1 - ├── right columns: abc.a:5 + ├── right columns: abc.a:6 ├── project │ ├── columns: abc.a:1 b:2!null │ └── scan abc - │ └── columns: abc.a:1 b:2!null c:3!null rowid:4!null + │ └── columns: abc.a:1 b:2!null c:3!null rowid:4!null crdb_internal_mvcc_timestamp:5 └── project - ├── columns: abc.a:5 c:7!null + ├── columns: abc.a:6 c:8!null └── scan abc - └── columns: abc.a:5 b:6!null c:7!null rowid:8!null + └── columns: abc.a:6 b:7!null c:8!null rowid:9!null crdb_internal_mvcc_timestamp:10 build (SELECT a FROM abc ORDER BY b) EXCEPT (SELECT b FROM abc ORDER BY c, b, a) @@ -750,15 +750,15 @@ build except ├── columns: a:1 ├── left columns: a:1 - ├── right columns: b:6 + ├── right columns: b:7 ├── project │ ├── columns: a:1 b:2!null │ └── scan abc - │ └── columns: a:1 b:2!null c:3!null rowid:4!null + │ └── columns: a:1 b:2!null c:3!null rowid:4!null crdb_internal_mvcc_timestamp:5 └── project - ├── columns: a:5 b:6!null c:7!null + ├── columns: a:6 b:7!null c:8!null └── scan abc - └── columns: a:5 b:6!null c:7!null rowid:8!null + └── columns: a:6 b:7!null c:8!null rowid:9!null crdb_internal_mvcc_timestamp:10 # Tests for type propagation. @@ -843,34 +843,34 @@ build (SELECT NULL FROM a) EXCEPT (VALUES((SELECT 1 FROM a LIMIT 1)), (1)) ---- except - ├── columns: "?column?":6 - ├── left columns: "?column?":6 - ├── right columns: column1:5 + ├── columns: "?column?":8 + ├── left columns: "?column?":8 + ├── right columns: column1:7 ├── project - │ ├── columns: "?column?":6 + │ ├── columns: "?column?":8 │ ├── project - │ │ ├── columns: "?column?":2 + │ │ ├── columns: "?column?":3 │ │ ├── scan a - │ │ │ └── columns: a:1!null + │ │ │ └── columns: a:1!null crdb_internal_mvcc_timestamp:2 │ │ └── projections - │ │ └── NULL [as="?column?":2] + │ │ └── NULL [as="?column?":3] │ └── projections - │ └── "?column?":2::INT8 [as="?column?":6] + │ └── "?column?":3::INT8 [as="?column?":8] └── values - ├── columns: column1:5 + ├── columns: column1:7 ├── tuple │ └── subquery │ └── max1-row - │ ├── columns: "?column?":4!null + │ ├── columns: "?column?":6!null │ └── limit - │ ├── columns: "?column?":4!null + │ ├── columns: "?column?":6!null │ ├── project - │ │ ├── columns: "?column?":4!null + │ │ ├── columns: "?column?":6!null │ │ ├── limit hint: 1.00 │ │ ├── scan a - │ │ │ ├── columns: a:3!null + │ │ │ ├── columns: a:4!null crdb_internal_mvcc_timestamp:5 │ │ │ └── limit hint: 1.00 │ │ └── projections - │ │ └── 1 [as="?column?":4] + │ │ └── 1 [as="?column?":6] │ └── 1 └── (1,) diff --git a/pkg/sql/opt/optbuilder/testdata/update b/pkg/sql/opt/optbuilder/testdata/update index 788db465d60a..bf42b4bdf2f7 100644 --- a/pkg/sql/opt/optbuilder/testdata/update +++ b/pkg/sql/opt/optbuilder/testdata/update @@ -63,30 +63,30 @@ UPDATE abcde SET a=2 WHERE a=1 ---- update abcde ├── columns: - ├── fetch columns: a:7 b:8 c:9 d:10 e:11 rowid:12 + ├── fetch columns: a:8 b:9 c:10 d:11 e:12 rowid:13 ├── update-mapping: - │ ├── a_new:13 => a:1 - │ ├── column14:14 => d:4 - │ └── a_new:13 => e:5 + │ ├── a_new:15 => a:1 + │ ├── column16:16 => d:4 + │ └── a_new:15 => e:5 └── project - ├── columns: column14:14 a:7!null b:8 c:9 d:10 e:11 rowid:12!null a_new:13!null + ├── columns: column16:16 a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 a_new:15!null ├── project - │ ├── columns: a_new:13!null a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ ├── columns: a_new:15!null a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ ├── select - │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ ├── scan abcde - │ │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ │ └── computed column expressions - │ │ │ ├── d:10 - │ │ │ │ └── (b:8 + c:9) + 1 - │ │ │ └── e:11 - │ │ │ └── a:7 + │ │ │ ├── d:11 + │ │ │ │ └── (b:9 + c:10) + 1 + │ │ │ └── e:12 + │ │ │ └── a:8 │ │ └── filters - │ │ └── a:7 = 1 + │ │ └── a:8 = 1 │ └── projections - │ └── 2 [as=a_new:13] + │ └── 2 [as=a_new:15] └── projections - └── (b:8 + c:9) + 1 [as=column14:14] + └── (b:9 + c:10) + 1 [as=column16:16] # Set all non-computed columns. build @@ -94,32 +94,32 @@ UPDATE abcde SET a=1, b=2, c=3, rowid=4 ---- update abcde ├── columns: - ├── fetch columns: a:7 b:8 c:9 d:10 e:11 rowid:12 + ├── fetch columns: a:8 b:9 c:10 d:11 e:12 rowid:13 ├── update-mapping: - │ ├── a_new:13 => a:1 - │ ├── b_new:14 => b:2 - │ ├── c_new:15 => c:3 - │ ├── column17:17 => d:4 - │ ├── a_new:13 => e:5 - │ └── rowid_new:16 => rowid:6 + │ ├── a_new:15 => a:1 + │ ├── b_new:16 => b:2 + │ ├── c_new:17 => c:3 + │ ├── column19:19 => d:4 + │ ├── a_new:15 => e:5 + │ └── rowid_new:18 => rowid:6 └── project - ├── columns: column17:17!null a:7!null b:8 c:9 d:10 e:11 rowid:12!null a_new:13!null b_new:14!null c_new:15!null rowid_new:16!null + ├── columns: column19:19!null a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 a_new:15!null b_new:16!null c_new:17!null rowid_new:18!null ├── project - │ ├── columns: a_new:13!null b_new:14!null c_new:15!null rowid_new:16!null a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ ├── columns: a_new:15!null b_new:16!null c_new:17!null rowid_new:18!null a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ ├── scan abcde - │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ └── computed column expressions - │ │ ├── d:10 - │ │ │ └── (b:8 + c:9) + 1 - │ │ └── e:11 - │ │ └── a:7 + │ │ ├── d:11 + │ │ │ └── (b:9 + c:10) + 1 + │ │ └── e:12 + │ │ └── a:8 │ └── projections - │ ├── 1 [as=a_new:13] - │ ├── 2 [as=b_new:14] - │ ├── 3 [as=c_new:15] - │ └── 4 [as=rowid_new:16] + │ ├── 1 [as=a_new:15] + │ ├── 2 [as=b_new:16] + │ ├── 3 [as=c_new:17] + │ └── 4 [as=rowid_new:18] └── projections - └── (b_new:14 + c_new:15) + 1 [as=column17:17] + └── (b_new:16 + c_new:17) + 1 [as=column19:19] # Set all non-computed columns in reverse order. build @@ -127,32 +127,32 @@ UPDATE abcde SET rowid=1, c=2, b=3, a=4 ---- update abcde ├── columns: - ├── fetch columns: a:7 b:8 c:9 d:10 e:11 rowid:12 + ├── fetch columns: a:8 b:9 c:10 d:11 e:12 rowid:13 ├── update-mapping: - │ ├── a_new:16 => a:1 - │ ├── b_new:15 => b:2 - │ ├── c_new:14 => c:3 - │ ├── column17:17 => d:4 - │ ├── a_new:16 => e:5 - │ └── rowid_new:13 => rowid:6 + │ ├── a_new:18 => a:1 + │ ├── b_new:17 => b:2 + │ ├── c_new:16 => c:3 + │ ├── column19:19 => d:4 + │ ├── a_new:18 => e:5 + │ └── rowid_new:15 => rowid:6 └── project - ├── columns: column17:17!null a:7!null b:8 c:9 d:10 e:11 rowid:12!null rowid_new:13!null c_new:14!null b_new:15!null a_new:16!null + ├── columns: column19:19!null a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 rowid_new:15!null c_new:16!null b_new:17!null a_new:18!null ├── project - │ ├── columns: rowid_new:13!null c_new:14!null b_new:15!null a_new:16!null a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ ├── columns: rowid_new:15!null c_new:16!null b_new:17!null a_new:18!null a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ ├── scan abcde - │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ └── computed column expressions - │ │ ├── d:10 - │ │ │ └── (b:8 + c:9) + 1 - │ │ └── e:11 - │ │ └── a:7 + │ │ ├── d:11 + │ │ │ └── (b:9 + c:10) + 1 + │ │ └── e:12 + │ │ └── a:8 │ └── projections - │ ├── 1 [as=rowid_new:13] - │ ├── 2 [as=c_new:14] - │ ├── 3 [as=b_new:15] - │ └── 4 [as=a_new:16] + │ ├── 1 [as=rowid_new:15] + │ ├── 2 [as=c_new:16] + │ ├── 3 [as=b_new:17] + │ └── 4 [as=a_new:18] └── projections - └── (b_new:15 + c_new:14) + 1 [as=column17:17] + └── (b_new:17 + c_new:16) + 1 [as=column19:19] # Set all non-computed columns to NULL. build @@ -160,29 +160,29 @@ UPDATE abcde SET a=NULL, b=NULL, c=NULL, rowid=NULL ---- update abcde ├── columns: - ├── fetch columns: a:7 b:8 c:9 d:10 e:11 rowid:12 + ├── fetch columns: a:8 b:9 c:10 d:11 e:12 rowid:13 ├── update-mapping: - │ ├── a_new:13 => a:1 - │ ├── a_new:13 => b:2 - │ ├── a_new:13 => c:3 - │ ├── column14:14 => d:4 - │ ├── a_new:13 => e:5 - │ └── a_new:13 => rowid:6 + │ ├── a_new:15 => a:1 + │ ├── a_new:15 => b:2 + │ ├── a_new:15 => c:3 + │ ├── column16:16 => d:4 + │ ├── a_new:15 => e:5 + │ └── a_new:15 => rowid:6 └── project - ├── columns: column14:14 a:7!null b:8 c:9 d:10 e:11 rowid:12!null a_new:13 + ├── columns: column16:16 a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 a_new:15 ├── project - │ ├── columns: a_new:13 a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ ├── columns: a_new:15 a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ ├── scan abcde - │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ └── computed column expressions - │ │ ├── d:10 - │ │ │ └── (b:8 + c:9) + 1 - │ │ └── e:11 - │ │ └── a:7 + │ │ ├── d:11 + │ │ │ └── (b:9 + c:10) + 1 + │ │ └── e:12 + │ │ └── a:8 │ └── projections - │ └── NULL::INT8 [as=a_new:13] + │ └── NULL::INT8 [as=a_new:15] └── projections - └── (a_new:13 + a_new:13) + 1 [as=column14:14] + └── (a_new:15 + a_new:15) + 1 [as=column16:16] # Set columns using variable expressions. build @@ -190,32 +190,32 @@ UPDATE abcde SET a=a+1, b=b*c WHERE b>e ---- update abcde ├── columns: - ├── fetch columns: a:7 b:8 c:9 d:10 e:11 rowid:12 + ├── fetch columns: a:8 b:9 c:10 d:11 e:12 rowid:13 ├── update-mapping: - │ ├── a_new:13 => a:1 - │ ├── b_new:14 => b:2 - │ ├── column15:15 => d:4 - │ └── a_new:13 => e:5 + │ ├── a_new:15 => a:1 + │ ├── b_new:16 => b:2 + │ ├── column17:17 => d:4 + │ └── a_new:15 => e:5 └── project - ├── columns: column15:15 a:7!null b:8!null c:9 d:10 e:11!null rowid:12!null a_new:13!null b_new:14 + ├── columns: column17:17 a:8!null b:9!null c:10 d:11 e:12!null rowid:13!null crdb_internal_mvcc_timestamp:14 a_new:15!null b_new:16 ├── project - │ ├── columns: a_new:13!null b_new:14 a:7!null b:8!null c:9 d:10 e:11!null rowid:12!null + │ ├── columns: a_new:15!null b_new:16 a:8!null b:9!null c:10 d:11 e:12!null rowid:13!null crdb_internal_mvcc_timestamp:14 │ ├── select - │ │ ├── columns: a:7!null b:8!null c:9 d:10 e:11!null rowid:12!null + │ │ ├── columns: a:8!null b:9!null c:10 d:11 e:12!null rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ ├── scan abcde - │ │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ │ └── computed column expressions - │ │ │ ├── d:10 - │ │ │ │ └── (b:8 + c:9) + 1 - │ │ │ └── e:11 - │ │ │ └── a:7 + │ │ │ ├── d:11 + │ │ │ │ └── (b:9 + c:10) + 1 + │ │ │ └── e:12 + │ │ │ └── a:8 │ │ └── filters - │ │ └── b:8 > e:11 + │ │ └── b:9 > e:12 │ └── projections - │ ├── a:7 + 1 [as=a_new:13] - │ └── b:8 * c:9 [as=b_new:14] + │ ├── a:8 + 1 [as=a_new:15] + │ └── b:9 * c:10 [as=b_new:16] └── projections - └── (b_new:14 + c:9) + 1 [as=column15:15] + └── (b_new:16 + c:10) + 1 [as=column17:17] # Set columns using aliased expressions. build @@ -223,23 +223,23 @@ UPDATE abcde AS foo SET a=foo.b, b=foo.c ---- update foo ├── columns: - ├── fetch columns: a:7 b:8 c:9 d:10 e:11 rowid:12 + ├── fetch columns: a:8 b:9 c:10 d:11 e:12 rowid:13 ├── update-mapping: - │ ├── b:8 => a:1 - │ ├── c:9 => b:2 - │ ├── column13:13 => d:4 - │ └── b:8 => e:5 + │ ├── b:9 => a:1 + │ ├── c:10 => b:2 + │ ├── column15:15 => d:4 + │ └── b:9 => e:5 └── project - ├── columns: column13:13 a:7!null b:8 c:9 d:10 e:11 rowid:12!null + ├── columns: column15:15 a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 ├── scan foo - │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ └── computed column expressions - │ ├── d:10 - │ │ └── (b:8 + c:9) + 1 - │ └── e:11 - │ └── a:7 + │ ├── d:11 + │ │ └── (b:9 + c:10) + 1 + │ └── e:12 + │ └── a:8 └── projections - └── (c:9 + c:9) + 1 [as=column13:13] + └── (c:10 + c:10) + 1 [as=column15:15] # Use WHERE, ORDER BY, LIMIT. build @@ -247,38 +247,38 @@ UPDATE abcde SET b=1 WHERE a>0 ORDER BY a LIMIT 10 ---- update abcde ├── columns: - ├── fetch columns: a:7 b:8 c:9 d:10 e:11 rowid:12 + ├── fetch columns: a:8 b:9 c:10 d:11 e:12 rowid:13 ├── update-mapping: - │ ├── b_new:13 => b:2 - │ ├── column14:14 => d:4 - │ └── a:7 => e:5 + │ ├── b_new:15 => b:2 + │ ├── column16:16 => d:4 + │ └── a:8 => e:5 └── project - ├── columns: column14:14 a:7!null b:8 c:9 d:10 e:11 rowid:12!null b_new:13!null + ├── columns: column16:16 a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 b_new:15!null ├── project - │ ├── columns: b_new:13!null a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ ├── columns: b_new:15!null a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ ├── limit - │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null - │ │ ├── internal-ordering: +7 + │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 + │ │ ├── internal-ordering: +8 │ │ ├── sort - │ │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null - │ │ │ ├── ordering: +7 + │ │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 + │ │ │ ├── ordering: +8 │ │ │ ├── limit hint: 10.00 │ │ │ └── select - │ │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ │ ├── scan abcde - │ │ │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ │ │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ │ │ └── computed column expressions - │ │ │ │ ├── d:10 - │ │ │ │ │ └── (b:8 + c:9) + 1 - │ │ │ │ └── e:11 - │ │ │ │ └── a:7 + │ │ │ │ ├── d:11 + │ │ │ │ │ └── (b:9 + c:10) + 1 + │ │ │ │ └── e:12 + │ │ │ │ └── a:8 │ │ │ └── filters - │ │ │ └── a:7 > 0 + │ │ │ └── a:8 > 0 │ │ └── 10 │ └── projections - │ └── 1 [as=b_new:13] + │ └── 1 [as=b_new:15] └── projections - └── (b_new:13 + c:9) + 1 [as=column14:14] + └── (b_new:15 + c:10) + 1 [as=column16:16] # UPDATE with index hints. exec-ddl @@ -296,100 +296,100 @@ UPDATE xyzw@primary SET x=2 WHERE z=1 ---- update xyzw ├── columns: - ├── fetch columns: x:5 y:6 z:7 w:8 + ├── fetch columns: x:6 y:7 z:8 w:9 ├── update-mapping: - │ └── x_new:9 => x:1 + │ └── x_new:11 => x:1 └── project - ├── columns: x_new:9!null x:5!null y:6 z:7!null w:8 + ├── columns: x_new:11!null x:6!null y:7 z:8!null w:9 crdb_internal_mvcc_timestamp:10 ├── select - │ ├── columns: x:5!null y:6 z:7!null w:8 + │ ├── columns: x:6!null y:7 z:8!null w:9 crdb_internal_mvcc_timestamp:10 │ ├── scan xyzw - │ │ ├── columns: x:5!null y:6 z:7 w:8 + │ │ ├── columns: x:6!null y:7 z:8 w:9 crdb_internal_mvcc_timestamp:10 │ │ └── flags: force-index=primary │ └── filters - │ └── z:7 = 1 + │ └── z:8 = 1 └── projections - └── 2 [as=x_new:9] + └── 2 [as=x_new:11] build UPDATE xyzw@foo SET x=2 WHERE z=1 ---- update xyzw ├── columns: - ├── fetch columns: x:5 y:6 z:7 w:8 + ├── fetch columns: x:6 y:7 z:8 w:9 ├── update-mapping: - │ └── x_new:9 => x:1 + │ └── x_new:11 => x:1 └── project - ├── columns: x_new:9!null x:5!null y:6 z:7!null w:8 + ├── columns: x_new:11!null x:6!null y:7 z:8!null w:9 crdb_internal_mvcc_timestamp:10 ├── select - │ ├── columns: x:5!null y:6 z:7!null w:8 + │ ├── columns: x:6!null y:7 z:8!null w:9 crdb_internal_mvcc_timestamp:10 │ ├── scan xyzw - │ │ ├── columns: x:5!null y:6 z:7 w:8 + │ │ ├── columns: x:6!null y:7 z:8 w:9 crdb_internal_mvcc_timestamp:10 │ │ └── flags: force-index=foo │ └── filters - │ └── z:7 = 1 + │ └── z:8 = 1 └── projections - └── 2 [as=x_new:9] + └── 2 [as=x_new:11] build UPDATE xyzw@{FORCE_INDEX=foo,ASC} SET x=2 WHERE z=1 ---- update xyzw ├── columns: - ├── fetch columns: x:5 y:6 z:7 w:8 + ├── fetch columns: x:6 y:7 z:8 w:9 ├── update-mapping: - │ └── x_new:9 => x:1 + │ └── x_new:11 => x:1 └── project - ├── columns: x_new:9!null x:5!null y:6 z:7!null w:8 + ├── columns: x_new:11!null x:6!null y:7 z:8!null w:9 crdb_internal_mvcc_timestamp:10 ├── select - │ ├── columns: x:5!null y:6 z:7!null w:8 + │ ├── columns: x:6!null y:7 z:8!null w:9 crdb_internal_mvcc_timestamp:10 │ ├── scan xyzw - │ │ ├── columns: x:5!null y:6 z:7 w:8 + │ │ ├── columns: x:6!null y:7 z:8 w:9 crdb_internal_mvcc_timestamp:10 │ │ └── flags: force-index=foo,fwd │ └── filters - │ └── z:7 = 1 + │ └── z:8 = 1 └── projections - └── 2 [as=x_new:9] + └── 2 [as=x_new:11] build UPDATE xyzw@{FORCE_INDEX=foo,DESC} SET x=2 WHERE z=1 ---- update xyzw ├── columns: - ├── fetch columns: x:5 y:6 z:7 w:8 + ├── fetch columns: x:6 y:7 z:8 w:9 ├── update-mapping: - │ └── x_new:9 => x:1 + │ └── x_new:11 => x:1 └── project - ├── columns: x_new:9!null x:5!null y:6 z:7!null w:8 + ├── columns: x_new:11!null x:6!null y:7 z:8!null w:9 crdb_internal_mvcc_timestamp:10 ├── select - │ ├── columns: x:5!null y:6 z:7!null w:8 + │ ├── columns: x:6!null y:7 z:8!null w:9 crdb_internal_mvcc_timestamp:10 │ ├── scan xyzw,rev - │ │ ├── columns: x:5!null y:6 z:7 w:8 + │ │ ├── columns: x:6!null y:7 z:8 w:9 crdb_internal_mvcc_timestamp:10 │ │ └── flags: force-index=foo,rev │ └── filters - │ └── z:7 = 1 + │ └── z:8 = 1 └── projections - └── 2 [as=x_new:9] + └── 2 [as=x_new:11] build UPDATE xyzw@{NO_INDEX_JOIN} SET x=2 WHERE z=1 ---- update xyzw ├── columns: - ├── fetch columns: x:5 y:6 z:7 w:8 + ├── fetch columns: x:6 y:7 z:8 w:9 ├── update-mapping: - │ └── x_new:9 => x:1 + │ └── x_new:11 => x:1 └── project - ├── columns: x_new:9!null x:5!null y:6 z:7!null w:8 + ├── columns: x_new:11!null x:6!null y:7 z:8!null w:9 crdb_internal_mvcc_timestamp:10 ├── select - │ ├── columns: x:5!null y:6 z:7!null w:8 + │ ├── columns: x:6!null y:7 z:8!null w:9 crdb_internal_mvcc_timestamp:10 │ ├── scan xyzw - │ │ ├── columns: x:5!null y:6 z:7 w:8 + │ │ ├── columns: x:6!null y:7 z:8 w:9 crdb_internal_mvcc_timestamp:10 │ │ └── flags: no-index-join │ └── filters - │ └── z:7 = 1 + │ └── z:8 = 1 └── projections - └── 2 [as=x_new:9] + └── 2 [as=x_new:11] build UPDATE xyzw@bad_idx SET x=2 WHERE z=1 @@ -403,17 +403,17 @@ UPDATE xyz SET y=1, z=1 ---- update xyz ├── columns: - ├── fetch columns: x:4 y:5 z:6 + ├── fetch columns: x:5 y:6 z:7 ├── update-mapping: - │ ├── y_new:7 => y:2 - │ └── z_new:8 => z:3 + │ ├── y_new:9 => y:2 + │ └── z_new:10 => z:3 └── project - ├── columns: y_new:7!null z_new:8!null x:4!null y:5 z:6 + ├── columns: y_new:9!null z_new:10!null x:5!null y:6 z:7 crdb_internal_mvcc_timestamp:8 ├── scan xyz - │ └── columns: x:4!null y:5 z:6 + │ └── columns: x:5!null y:6 z:7 crdb_internal_mvcc_timestamp:8 └── projections - ├── 1 [as=y_new:7] - └── 1.0 [as=z_new:8] + ├── 1 [as=y_new:9] + └── 1.0 [as=z_new:10] # Use placeholders. build @@ -421,19 +421,19 @@ UPDATE xyz SET x=$1, y=$2, z=$3 ---- update xyz ├── columns: - ├── fetch columns: x:4 y:5 z:6 + ├── fetch columns: x:5 y:6 z:7 ├── update-mapping: - │ ├── x_new:7 => x:1 - │ ├── y_new:8 => y:2 - │ └── z_new:9 => z:3 + │ ├── x_new:9 => x:1 + │ ├── y_new:10 => y:2 + │ └── z_new:11 => z:3 └── project - ├── columns: x_new:7 y_new:8 z_new:9 x:4!null y:5 z:6 + ├── columns: x_new:9 y_new:10 z_new:11 x:5!null y:6 z:7 crdb_internal_mvcc_timestamp:8 ├── scan xyz - │ └── columns: x:4!null y:5 z:6 + │ └── columns: x:5!null y:6 z:7 crdb_internal_mvcc_timestamp:8 └── projections - ├── $1 [as=x_new:7] - ├── $2 [as=y_new:8] - └── $3 [as=z_new:9] + ├── $1 [as=x_new:9] + ├── $2 [as=y_new:10] + └── $3 [as=z_new:11] # Duplicate expressions with placeholders. build @@ -441,31 +441,31 @@ UPDATE abcde SET a=$1 + 1, b=$1 + 1 WHERE c=10 ---- update abcde ├── columns: - ├── fetch columns: a:7 b:8 c:9 d:10 e:11 rowid:12 + ├── fetch columns: a:8 b:9 c:10 d:11 e:12 rowid:13 ├── update-mapping: - │ ├── a_new:13 => a:1 - │ ├── a_new:13 => b:2 - │ ├── column14:14 => d:4 - │ └── a_new:13 => e:5 + │ ├── a_new:15 => a:1 + │ ├── a_new:15 => b:2 + │ ├── column16:16 => d:4 + │ └── a_new:15 => e:5 └── project - ├── columns: column14:14 a:7!null b:8 c:9!null d:10 e:11 rowid:12!null a_new:13 + ├── columns: column16:16 a:8!null b:9 c:10!null d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 a_new:15 ├── project - │ ├── columns: a_new:13 a:7!null b:8 c:9!null d:10 e:11 rowid:12!null + │ ├── columns: a_new:15 a:8!null b:9 c:10!null d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ ├── select - │ │ ├── columns: a:7!null b:8 c:9!null d:10 e:11 rowid:12!null + │ │ ├── columns: a:8!null b:9 c:10!null d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ ├── scan abcde - │ │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ │ └── computed column expressions - │ │ │ ├── d:10 - │ │ │ │ └── (b:8 + c:9) + 1 - │ │ │ └── e:11 - │ │ │ └── a:7 + │ │ │ ├── d:11 + │ │ │ │ └── (b:9 + c:10) + 1 + │ │ │ └── e:12 + │ │ │ └── a:8 │ │ └── filters - │ │ └── c:9 = 10 + │ │ └── c:10 = 10 │ └── projections - │ └── $1 + 1 [as=a_new:13] + │ └── $1 + 1 [as=a_new:15] └── projections - └── (a_new:13 + c:9) + 1 [as=column14:14] + └── (a_new:15 + c:10) + 1 [as=column16:16] # Unknown target table. @@ -501,51 +501,51 @@ with &1 │ ├── columns: abcde.a:1!null abcde.b:2 abcde.c:3 abcde.d:4 abcde.e:5 │ └── update abcde │ ├── columns: abcde.a:1!null abcde.b:2 abcde.c:3 abcde.d:4 abcde.e:5 rowid:6!null - │ ├── fetch columns: abcde.a:7 abcde.b:8 abcde.c:9 abcde.d:10 abcde.e:11 rowid:12 + │ ├── fetch columns: abcde.a:8 abcde.b:9 abcde.c:10 abcde.d:11 abcde.e:12 rowid:13 │ ├── update-mapping: - │ │ ├── abcde.b:8 => abcde.a:1 - │ │ ├── column13:13 => abcde.d:4 - │ │ └── abcde.b:8 => abcde.e:5 + │ │ ├── abcde.b:9 => abcde.a:1 + │ │ ├── column15:15 => abcde.d:4 + │ │ └── abcde.b:9 => abcde.e:5 │ └── project - │ ├── columns: column13:13 abcde.a:7!null abcde.b:8 abcde.c:9 abcde.d:10 abcde.e:11 rowid:12!null + │ ├── columns: column15:15 abcde.a:8!null abcde.b:9 abcde.c:10 abcde.d:11 abcde.e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ ├── scan abcde - │ │ ├── columns: abcde.a:7!null abcde.b:8 abcde.c:9 abcde.d:10 abcde.e:11 rowid:12!null + │ │ ├── columns: abcde.a:8!null abcde.b:9 abcde.c:10 abcde.d:11 abcde.e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ └── computed column expressions - │ │ ├── abcde.d:10 - │ │ │ └── (abcde.b:8 + abcde.c:9) + 1 - │ │ └── abcde.e:11 - │ │ └── abcde.a:7 + │ │ ├── abcde.d:11 + │ │ │ └── (abcde.b:9 + abcde.c:10) + 1 + │ │ └── abcde.e:12 + │ │ └── abcde.a:8 │ └── projections - │ └── (abcde.b:8 + abcde.c:9) + 1 [as=column13:13] + │ └── (abcde.b:9 + abcde.c:10) + 1 [as=column15:15] └── with &2 (cte) ├── project - │ ├── columns: b:15 + │ ├── columns: b:17 │ └── with-scan &1 - │ ├── columns: a:14!null b:15 c:16 d:17 e:18 + │ ├── columns: a:16!null b:17 c:18 d:19 e:20 │ └── mapping: - │ ├── abcde.a:1 => a:14 - │ ├── abcde.b:2 => b:15 - │ ├── abcde.c:3 => c:16 - │ ├── abcde.d:4 => d:17 - │ └── abcde.e:5 => e:18 + │ ├── abcde.a:1 => a:16 + │ ├── abcde.b:2 => b:17 + │ ├── abcde.c:3 => c:18 + │ ├── abcde.d:4 => d:19 + │ └── abcde.e:5 => e:20 └── update abcde ├── columns: - ├── fetch columns: abcde.a:25 abcde.b:26 abcde.c:27 abcde.d:28 abcde.e:29 rowid:30 + ├── fetch columns: abcde.a:28 abcde.b:29 abcde.c:30 abcde.d:31 abcde.e:32 rowid:33 ├── update-mapping: - │ ├── abcde.b:26 => abcde.a:19 - │ ├── column31:31 => abcde.d:22 - │ └── abcde.b:26 => abcde.e:23 + │ ├── abcde.b:29 => abcde.a:21 + │ ├── column35:35 => abcde.d:24 + │ └── abcde.b:29 => abcde.e:25 └── project - ├── columns: column31:31 abcde.a:25!null abcde.b:26 abcde.c:27 abcde.d:28 abcde.e:29 rowid:30!null + ├── columns: column35:35 abcde.a:28!null abcde.b:29 abcde.c:30 abcde.d:31 abcde.e:32 rowid:33!null crdb_internal_mvcc_timestamp:34 ├── scan abcde - │ ├── columns: abcde.a:25!null abcde.b:26 abcde.c:27 abcde.d:28 abcde.e:29 rowid:30!null + │ ├── columns: abcde.a:28!null abcde.b:29 abcde.c:30 abcde.d:31 abcde.e:32 rowid:33!null crdb_internal_mvcc_timestamp:34 │ └── computed column expressions - │ ├── abcde.d:28 - │ │ └── (abcde.b:26 + abcde.c:27) + 1 - │ └── abcde.e:29 - │ └── abcde.a:25 + │ ├── abcde.d:31 + │ │ └── (abcde.b:29 + abcde.c:30) + 1 + │ └── abcde.e:32 + │ └── abcde.a:28 └── projections - └── (abcde.b:26 + abcde.c:27) + 1 [as=column31:31] + └── (abcde.b:29 + abcde.c:30) + 1 [as=column35:35] # With alias, original table name should be inaccessible. build @@ -571,119 +571,119 @@ project ├── columns: a:1!null b:2 c:3 d:4 e:5!null └── update abcde ├── columns: a:1!null b:2 c:3 d:4 e:5!null rowid:6!null - ├── fetch columns: a:7 b:8 c:9 d:10 e:11 rowid:12 + ├── fetch columns: a:8 b:9 c:10 d:11 e:12 rowid:13 ├── update-mapping: - │ ├── a_new:13 => a:1 - │ ├── column14:14 => d:4 - │ └── a_new:13 => e:5 + │ ├── a_new:15 => a:1 + │ ├── column16:16 => d:4 + │ └── a_new:15 => e:5 └── project - ├── columns: column14:14 a:7!null b:8 c:9 d:10 e:11 rowid:12!null a_new:13!null + ├── columns: column16:16 a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 a_new:15!null ├── project - │ ├── columns: a_new:13!null a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ ├── columns: a_new:15!null a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ ├── select - │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ ├── scan abcde - │ │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ │ └── computed column expressions - │ │ │ ├── d:10 - │ │ │ │ └── (b:8 + c:9) + 1 - │ │ │ └── e:11 - │ │ │ └── a:7 + │ │ │ ├── d:11 + │ │ │ │ └── (b:9 + c:10) + 1 + │ │ │ └── e:12 + │ │ │ └── a:8 │ │ └── filters - │ │ └── a:7 = 1 + │ │ └── a:8 = 1 │ └── projections - │ └── 2 [as=a_new:13] + │ └── 2 [as=a_new:15] └── projections - └── (b:8 + c:9) + 1 [as=column14:14] + └── (b:9 + c:10) + 1 [as=column16:16] # Return values from aliased table. build UPDATE abcde AS foo SET a=2 WHERE a=1 RETURNING foo.a+1, foo.b * foo.d ---- project - ├── columns: "?column?":15!null "?column?":16 + ├── columns: "?column?":17!null "?column?":18 ├── update foo │ ├── columns: a:1!null b:2 c:3 d:4 e:5!null rowid:6!null - │ ├── fetch columns: a:7 b:8 c:9 d:10 e:11 rowid:12 + │ ├── fetch columns: a:8 b:9 c:10 d:11 e:12 rowid:13 │ ├── update-mapping: - │ │ ├── a_new:13 => a:1 - │ │ ├── column14:14 => d:4 - │ │ └── a_new:13 => e:5 + │ │ ├── a_new:15 => a:1 + │ │ ├── column16:16 => d:4 + │ │ └── a_new:15 => e:5 │ └── project - │ ├── columns: column14:14 a:7!null b:8 c:9 d:10 e:11 rowid:12!null a_new:13!null + │ ├── columns: column16:16 a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 a_new:15!null │ ├── project - │ │ ├── columns: a_new:13!null a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ │ ├── columns: a_new:15!null a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ ├── select - │ │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ │ ├── scan foo - │ │ │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ │ │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ │ │ └── computed column expressions - │ │ │ │ ├── d:10 - │ │ │ │ │ └── (b:8 + c:9) + 1 - │ │ │ │ └── e:11 - │ │ │ │ └── a:7 + │ │ │ │ ├── d:11 + │ │ │ │ │ └── (b:9 + c:10) + 1 + │ │ │ │ └── e:12 + │ │ │ │ └── a:8 │ │ │ └── filters - │ │ │ └── a:7 = 1 + │ │ │ └── a:8 = 1 │ │ └── projections - │ │ └── 2 [as=a_new:13] + │ │ └── 2 [as=a_new:15] │ └── projections - │ └── (b:8 + c:9) + 1 [as=column14:14] + │ └── (b:9 + c:10) + 1 [as=column16:16] └── projections - ├── a:1 + 1 [as="?column?":15] - └── b:2 * d:4 [as="?column?":16] + ├── a:1 + 1 [as="?column?":17] + └── b:2 * d:4 [as="?column?":18] # Use returning UPDATE as a FROM expression. build SELECT a, d FROM [UPDATE abcde SET a=2 WHERE a>0 ORDER BY b LIMIT 10 RETURNING *] ---- with &1 - ├── columns: a:15!null d:18 + ├── columns: a:17!null d:20 ├── project │ ├── columns: abcde.a:1!null abcde.b:2 abcde.c:3 abcde.d:4 abcde.e:5!null │ └── update abcde │ ├── columns: abcde.a:1!null abcde.b:2 abcde.c:3 abcde.d:4 abcde.e:5!null rowid:6!null - │ ├── fetch columns: abcde.a:7 abcde.b:8 abcde.c:9 abcde.d:10 abcde.e:11 rowid:12 + │ ├── fetch columns: abcde.a:8 abcde.b:9 abcde.c:10 abcde.d:11 abcde.e:12 rowid:13 │ ├── update-mapping: - │ │ ├── a_new:13 => abcde.a:1 - │ │ ├── column14:14 => abcde.d:4 - │ │ └── a_new:13 => abcde.e:5 + │ │ ├── a_new:15 => abcde.a:1 + │ │ ├── column16:16 => abcde.d:4 + │ │ └── a_new:15 => abcde.e:5 │ └── project - │ ├── columns: column14:14 abcde.a:7!null abcde.b:8 abcde.c:9 abcde.d:10 abcde.e:11 rowid:12!null a_new:13!null + │ ├── columns: column16:16 abcde.a:8!null abcde.b:9 abcde.c:10 abcde.d:11 abcde.e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 a_new:15!null │ ├── project - │ │ ├── columns: a_new:13!null abcde.a:7!null abcde.b:8 abcde.c:9 abcde.d:10 abcde.e:11 rowid:12!null + │ │ ├── columns: a_new:15!null abcde.a:8!null abcde.b:9 abcde.c:10 abcde.d:11 abcde.e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ ├── limit - │ │ │ ├── columns: abcde.a:7!null abcde.b:8 abcde.c:9 abcde.d:10 abcde.e:11 rowid:12!null - │ │ │ ├── internal-ordering: +8 + │ │ │ ├── columns: abcde.a:8!null abcde.b:9 abcde.c:10 abcde.d:11 abcde.e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 + │ │ │ ├── internal-ordering: +9 │ │ │ ├── sort - │ │ │ │ ├── columns: abcde.a:7!null abcde.b:8 abcde.c:9 abcde.d:10 abcde.e:11 rowid:12!null - │ │ │ │ ├── ordering: +8 + │ │ │ │ ├── columns: abcde.a:8!null abcde.b:9 abcde.c:10 abcde.d:11 abcde.e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 + │ │ │ │ ├── ordering: +9 │ │ │ │ ├── limit hint: 10.00 │ │ │ │ └── select - │ │ │ │ ├── columns: abcde.a:7!null abcde.b:8 abcde.c:9 abcde.d:10 abcde.e:11 rowid:12!null + │ │ │ │ ├── columns: abcde.a:8!null abcde.b:9 abcde.c:10 abcde.d:11 abcde.e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ │ │ ├── scan abcde - │ │ │ │ │ ├── columns: abcde.a:7!null abcde.b:8 abcde.c:9 abcde.d:10 abcde.e:11 rowid:12!null + │ │ │ │ │ ├── columns: abcde.a:8!null abcde.b:9 abcde.c:10 abcde.d:11 abcde.e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ │ │ │ └── computed column expressions - │ │ │ │ │ ├── abcde.d:10 - │ │ │ │ │ │ └── (abcde.b:8 + abcde.c:9) + 1 - │ │ │ │ │ └── abcde.e:11 - │ │ │ │ │ └── abcde.a:7 + │ │ │ │ │ ├── abcde.d:11 + │ │ │ │ │ │ └── (abcde.b:9 + abcde.c:10) + 1 + │ │ │ │ │ └── abcde.e:12 + │ │ │ │ │ └── abcde.a:8 │ │ │ │ └── filters - │ │ │ │ └── abcde.a:7 > 0 + │ │ │ │ └── abcde.a:8 > 0 │ │ │ └── 10 │ │ └── projections - │ │ └── 2 [as=a_new:13] + │ │ └── 2 [as=a_new:15] │ └── projections - │ └── (abcde.b:8 + abcde.c:9) + 1 [as=column14:14] + │ └── (abcde.b:9 + abcde.c:10) + 1 [as=column16:16] └── project - ├── columns: a:15!null d:18 + ├── columns: a:17!null d:20 └── with-scan &1 - ├── columns: a:15!null b:16 c:17 d:18 e:19!null + ├── columns: a:17!null b:18 c:19 d:20 e:21!null └── mapping: - ├── abcde.a:1 => a:15 - ├── abcde.b:2 => b:16 - ├── abcde.c:3 => c:17 - ├── abcde.d:4 => d:18 - └── abcde.e:5 => e:19 + ├── abcde.a:1 => a:17 + ├── abcde.b:2 => b:18 + ├── abcde.c:3 => c:19 + ├── abcde.d:4 => d:20 + └── abcde.e:5 => e:21 # Return hidden column. build @@ -693,26 +693,26 @@ project ├── columns: rowid:6!null └── update abcde ├── columns: a:1!null b:2 c:3 d:4 e:5!null rowid:6!null - ├── fetch columns: a:7 b:8 c:9 d:10 e:11 rowid:12 + ├── fetch columns: a:8 b:9 c:10 d:11 e:12 rowid:13 ├── update-mapping: - │ ├── column14:14 => d:4 - │ ├── a:7 => e:5 - │ └── rowid_new:13 => rowid:6 + │ ├── column16:16 => d:4 + │ ├── a:8 => e:5 + │ └── rowid_new:15 => rowid:6 └── project - ├── columns: column14:14 a:7!null b:8 c:9 d:10 e:11 rowid:12!null rowid_new:13!null + ├── columns: column16:16 a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 rowid_new:15!null ├── project - │ ├── columns: rowid_new:13!null a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ ├── columns: rowid_new:15!null a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ ├── scan abcde - │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ └── computed column expressions - │ │ ├── d:10 - │ │ │ └── (b:8 + c:9) + 1 - │ │ └── e:11 - │ │ └── a:7 + │ │ ├── d:11 + │ │ │ └── (b:9 + c:10) + 1 + │ │ └── e:12 + │ │ └── a:8 │ └── projections - │ └── rowid:12 + 1 [as=rowid_new:13] + │ └── rowid:13 + 1 [as=rowid_new:15] └── projections - └── (b:8 + c:9) + 1 [as=column14:14] + └── (b:9 + c:10) + 1 [as=column16:16] # Try to use aggregate function in RETURNING clause. build @@ -736,28 +736,28 @@ UPDATE abcde SET b=DEFAULT, c=DEFAULT ---- update abcde ├── columns: - ├── fetch columns: a:7 b:8 c:9 d:10 e:11 rowid:12 + ├── fetch columns: a:8 b:9 c:10 d:11 e:12 rowid:13 ├── update-mapping: - │ ├── b_new:13 => b:2 - │ ├── c_new:14 => c:3 - │ ├── column15:15 => d:4 - │ └── a:7 => e:5 + │ ├── b_new:15 => b:2 + │ ├── c_new:16 => c:3 + │ ├── column17:17 => d:4 + │ └── a:8 => e:5 └── project - ├── columns: column15:15 a:7!null b:8 c:9 d:10 e:11 rowid:12!null b_new:13 c_new:14!null + ├── columns: column17:17 a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 b_new:15 c_new:16!null ├── project - │ ├── columns: b_new:13 c_new:14!null a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ ├── columns: b_new:15 c_new:16!null a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ ├── scan abcde - │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ └── computed column expressions - │ │ ├── d:10 - │ │ │ └── (b:8 + c:9) + 1 - │ │ └── e:11 - │ │ └── a:7 + │ │ ├── d:11 + │ │ │ └── (b:9 + c:10) + 1 + │ │ └── e:12 + │ │ └── a:8 │ └── projections - │ ├── NULL::INT8 [as=b_new:13] - │ └── 10 [as=c_new:14] + │ ├── NULL::INT8 [as=b_new:15] + │ └── 10 [as=c_new:16] └── projections - └── (b_new:13 + c_new:14) + 1 [as=column15:15] + └── (b_new:15 + c_new:16) + 1 [as=column17:17] # Allow not-null column to be updated with NULL DEFAULT value (would fail at # runtime if there are any rows to update). @@ -766,26 +766,26 @@ UPDATE abcde SET a=DEFAULT ---- update abcde ├── columns: - ├── fetch columns: a:7 b:8 c:9 d:10 e:11 rowid:12 + ├── fetch columns: a:8 b:9 c:10 d:11 e:12 rowid:13 ├── update-mapping: - │ ├── a_new:13 => a:1 - │ ├── column14:14 => d:4 - │ └── a_new:13 => e:5 + │ ├── a_new:15 => a:1 + │ ├── column16:16 => d:4 + │ └── a_new:15 => e:5 └── project - ├── columns: column14:14 a:7!null b:8 c:9 d:10 e:11 rowid:12!null a_new:13 + ├── columns: column16:16 a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 a_new:15 ├── project - │ ├── columns: a_new:13 a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ ├── columns: a_new:15 a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ ├── scan abcde - │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ └── computed column expressions - │ │ ├── d:10 - │ │ │ └── (b:8 + c:9) + 1 - │ │ └── e:11 - │ │ └── a:7 + │ │ ├── d:11 + │ │ │ └── (b:9 + c:10) + 1 + │ │ └── e:12 + │ │ └── a:8 │ └── projections - │ └── NULL::INT8 [as=a_new:13] + │ └── NULL::INT8 [as=a_new:15] └── projections - └── (b:8 + c:9) + 1 [as=column14:14] + └── (b:9 + c:10) + 1 [as=column16:16] build UPDATE abcde SET c=1+DEFAULT @@ -801,60 +801,60 @@ UPDATE abcde SET (a, b, c) = (1, 2, 3) ---- update abcde ├── columns: - ├── fetch columns: a:7 b:8 c:9 d:10 e:11 rowid:12 + ├── fetch columns: a:8 b:9 c:10 d:11 e:12 rowid:13 ├── update-mapping: - │ ├── a_new:13 => a:1 - │ ├── b_new:14 => b:2 - │ ├── c_new:15 => c:3 - │ ├── column16:16 => d:4 - │ └── a_new:13 => e:5 + │ ├── a_new:15 => a:1 + │ ├── b_new:16 => b:2 + │ ├── c_new:17 => c:3 + │ ├── column18:18 => d:4 + │ └── a_new:15 => e:5 └── project - ├── columns: column16:16!null a:7!null b:8 c:9 d:10 e:11 rowid:12!null a_new:13!null b_new:14!null c_new:15!null + ├── columns: column18:18!null a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 a_new:15!null b_new:16!null c_new:17!null ├── project - │ ├── columns: a_new:13!null b_new:14!null c_new:15!null a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ ├── columns: a_new:15!null b_new:16!null c_new:17!null a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ ├── scan abcde - │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ └── computed column expressions - │ │ ├── d:10 - │ │ │ └── (b:8 + c:9) + 1 - │ │ └── e:11 - │ │ └── a:7 + │ │ ├── d:11 + │ │ │ └── (b:9 + c:10) + 1 + │ │ └── e:12 + │ │ └── a:8 │ └── projections - │ ├── 1 [as=a_new:13] - │ ├── 2 [as=b_new:14] - │ └── 3 [as=c_new:15] + │ ├── 1 [as=a_new:15] + │ ├── 2 [as=b_new:16] + │ └── 3 [as=c_new:17] └── projections - └── (b_new:14 + c_new:15) + 1 [as=column16:16] + └── (b_new:16 + c_new:17) + 1 [as=column18:18] build UPDATE abcde SET (c) = (NULL), (b, a) = (1, 2) ---- update abcde ├── columns: - ├── fetch columns: a:7 b:8 c:9 d:10 e:11 rowid:12 + ├── fetch columns: a:8 b:9 c:10 d:11 e:12 rowid:13 ├── update-mapping: - │ ├── a_new:15 => a:1 - │ ├── b_new:14 => b:2 - │ ├── c_new:13 => c:3 - │ ├── column16:16 => d:4 - │ └── a_new:15 => e:5 + │ ├── a_new:17 => a:1 + │ ├── b_new:16 => b:2 + │ ├── c_new:15 => c:3 + │ ├── column18:18 => d:4 + │ └── a_new:17 => e:5 └── project - ├── columns: column16:16 a:7!null b:8 c:9 d:10 e:11 rowid:12!null c_new:13 b_new:14!null a_new:15!null + ├── columns: column18:18 a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 c_new:15 b_new:16!null a_new:17!null ├── project - │ ├── columns: c_new:13 b_new:14!null a_new:15!null a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ ├── columns: c_new:15 b_new:16!null a_new:17!null a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ ├── scan abcde - │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ └── computed column expressions - │ │ ├── d:10 - │ │ │ └── (b:8 + c:9) + 1 - │ │ └── e:11 - │ │ └── a:7 + │ │ ├── d:11 + │ │ │ └── (b:9 + c:10) + 1 + │ │ └── e:12 + │ │ └── a:8 │ └── projections - │ ├── NULL::INT8 [as=c_new:13] - │ ├── 1 [as=b_new:14] - │ └── 2 [as=a_new:15] + │ ├── NULL::INT8 [as=c_new:15] + │ ├── 1 [as=b_new:16] + │ └── 2 [as=a_new:17] └── projections - └── (b_new:14 + c_new:13) + 1 [as=column16:16] + └── (b_new:16 + c_new:15) + 1 [as=column18:18] # Tuples + DEFAULT. build @@ -862,28 +862,28 @@ UPDATE abcde SET (b, c)=(DEFAULT, DEFAULT) ---- update abcde ├── columns: - ├── fetch columns: a:7 b:8 c:9 d:10 e:11 rowid:12 + ├── fetch columns: a:8 b:9 c:10 d:11 e:12 rowid:13 ├── update-mapping: - │ ├── b_new:13 => b:2 - │ ├── c_new:14 => c:3 - │ ├── column15:15 => d:4 - │ └── a:7 => e:5 + │ ├── b_new:15 => b:2 + │ ├── c_new:16 => c:3 + │ ├── column17:17 => d:4 + │ └── a:8 => e:5 └── project - ├── columns: column15:15 a:7!null b:8 c:9 d:10 e:11 rowid:12!null b_new:13 c_new:14!null + ├── columns: column17:17 a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 b_new:15 c_new:16!null ├── project - │ ├── columns: b_new:13 c_new:14!null a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ ├── columns: b_new:15 c_new:16!null a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ ├── scan abcde - │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ └── computed column expressions - │ │ ├── d:10 - │ │ │ └── (b:8 + c:9) + 1 - │ │ └── e:11 - │ │ └── a:7 + │ │ ├── d:11 + │ │ │ └── (b:9 + c:10) + 1 + │ │ └── e:12 + │ │ └── a:8 │ └── projections - │ ├── NULL::INT8 [as=b_new:13] - │ └── 10 [as=c_new:14] + │ ├── NULL::INT8 [as=b_new:15] + │ └── 10 [as=c_new:16] └── projections - └── (b_new:13 + c_new:14) + 1 [as=column15:15] + └── (b_new:15 + c_new:16) + 1 [as=column17:17] # Tuples + non-null DEFAULT. build @@ -891,27 +891,27 @@ UPDATE abcde SET (a, b)=(DEFAULT, DEFAULT) ---- update abcde ├── columns: - ├── fetch columns: a:7 b:8 c:9 d:10 e:11 rowid:12 + ├── fetch columns: a:8 b:9 c:10 d:11 e:12 rowid:13 ├── update-mapping: - │ ├── a_new:13 => a:1 - │ ├── a_new:13 => b:2 - │ ├── column14:14 => d:4 - │ └── a_new:13 => e:5 + │ ├── a_new:15 => a:1 + │ ├── a_new:15 => b:2 + │ ├── column16:16 => d:4 + │ └── a_new:15 => e:5 └── project - ├── columns: column14:14 a:7!null b:8 c:9 d:10 e:11 rowid:12!null a_new:13 + ├── columns: column16:16 a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 a_new:15 ├── project - │ ├── columns: a_new:13 a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ ├── columns: a_new:15 a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ ├── scan abcde - │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ └── computed column expressions - │ │ ├── d:10 - │ │ │ └── (b:8 + c:9) + 1 - │ │ └── e:11 - │ │ └── a:7 + │ │ ├── d:11 + │ │ │ └── (b:9 + c:10) + 1 + │ │ └── e:12 + │ │ └── a:8 │ └── projections - │ └── NULL::INT8 [as=a_new:13] + │ └── NULL::INT8 [as=a_new:15] └── projections - └── (a_new:13 + c:9) + 1 [as=column14:14] + └── (a_new:15 + c:10) + 1 [as=column16:16] build UPDATE abcde SET (a, b)=(1, 2, 3) @@ -943,33 +943,33 @@ UPDATE abcde SET (a)=(SELECT 1 AS one) ---- update abcde ├── columns: - ├── fetch columns: a:7 b:8 c:9 d:10 e:11 rowid:12 + ├── fetch columns: a:8 b:9 c:10 d:11 e:12 rowid:13 ├── update-mapping: - │ ├── one:13 => a:1 - │ ├── column14:14 => d:4 - │ └── one:13 => e:5 + │ ├── one:15 => a:1 + │ ├── column16:16 => d:4 + │ └── one:15 => e:5 └── project - ├── columns: column14:14 a:7!null b:8 c:9 d:10 e:11 rowid:12!null one:13 + ├── columns: column16:16 a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 one:15 ├── left-join-apply - │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null one:13 + │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 one:15 │ ├── scan abcde - │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ └── computed column expressions - │ │ ├── d:10 - │ │ │ └── (b:8 + c:9) + 1 - │ │ └── e:11 - │ │ └── a:7 + │ │ ├── d:11 + │ │ │ └── (b:9 + c:10) + 1 + │ │ └── e:12 + │ │ └── a:8 │ ├── max1-row - │ │ ├── columns: one:13!null + │ │ ├── columns: one:15!null │ │ └── project - │ │ ├── columns: one:13!null + │ │ ├── columns: one:15!null │ │ ├── values │ │ │ └── () │ │ └── projections - │ │ └── 1 [as=one:13] + │ │ └── 1 [as=one:15] │ └── filters (true) └── projections - └── (b:8 + c:9) + 1 [as=column14:14] + └── (b:9 + c:10) + 1 [as=column16:16] # Update all updatable columns. build @@ -977,38 +977,38 @@ UPDATE abcde SET (a, b, c, rowid)=(SELECT x::int, y, z::int, y+1 AS y1 FROM xyz) ---- update abcde ├── columns: - ├── fetch columns: a:7 b:8 c:9 d:10 e:11 rowid:12 + ├── fetch columns: a:8 b:9 c:10 d:11 e:12 rowid:13 ├── update-mapping: - │ ├── x:16 => a:1 - │ ├── y:14 => b:2 - │ ├── z:17 => c:3 - │ ├── column19:19 => d:4 - │ ├── x:16 => e:5 - │ └── y1:18 => rowid:6 + │ ├── x:19 => a:1 + │ ├── y:16 => b:2 + │ ├── z:20 => c:3 + │ ├── column22:22 => d:4 + │ ├── x:19 => e:5 + │ └── y1:21 => rowid:6 └── project - ├── columns: column19:19 a:7!null b:8 c:9 d:10 e:11 rowid:12!null y:14 x:16 z:17 y1:18 + ├── columns: column22:22 a:8!null b:9 c:10 d:11 e:12 rowid:13!null abcde.crdb_internal_mvcc_timestamp:14 y:16 x:19 z:20 y1:21 ├── left-join-apply - │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null y:14 x:16 z:17 y1:18 + │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null abcde.crdb_internal_mvcc_timestamp:14 y:16 x:19 z:20 y1:21 │ ├── scan abcde - │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null abcde.crdb_internal_mvcc_timestamp:14 │ │ └── computed column expressions - │ │ ├── d:10 - │ │ │ └── (b:8 + c:9) + 1 - │ │ └── e:11 - │ │ └── a:7 + │ │ ├── d:11 + │ │ │ └── (b:9 + c:10) + 1 + │ │ └── e:12 + │ │ └── a:8 │ ├── max1-row - │ │ ├── columns: y:14 x:16!null z:17 y1:18 + │ │ ├── columns: y:16 x:19!null z:20 y1:21 │ │ └── project - │ │ ├── columns: x:16!null z:17 y1:18 y:14 + │ │ ├── columns: x:19!null z:20 y1:21 y:16 │ │ ├── scan xyz - │ │ │ └── columns: xyz.x:13!null y:14 xyz.z:15 + │ │ │ └── columns: xyz.x:15!null y:16 xyz.z:17 xyz.crdb_internal_mvcc_timestamp:18 │ │ └── projections - │ │ ├── xyz.x:13::INT8 [as=x:16] - │ │ ├── xyz.z:15::INT8 [as=z:17] - │ │ └── y:14 + 1 [as=y1:18] + │ │ ├── xyz.x:15::INT8 [as=x:19] + │ │ ├── xyz.z:17::INT8 [as=z:20] + │ │ └── y:16 + 1 [as=y1:21] │ └── filters (true) └── projections - └── (y:14 + z:17) + 1 [as=column19:19] + └── (y:16 + z:20) + 1 [as=column22:22] # Update using combination of subquery and tuple SET expressions. build @@ -1016,41 +1016,41 @@ UPDATE abcde SET (a, b)=(SELECT y, y+1 AS y1 FROM xyz), (c, rowid)=(1, 2) ---- update abcde ├── columns: - ├── fetch columns: a:7 b:8 c:9 d:10 e:11 rowid:12 + ├── fetch columns: a:8 b:9 c:10 d:11 e:12 rowid:13 ├── update-mapping: - │ ├── y:14 => a:1 - │ ├── y1:16 => b:2 - │ ├── c_new:17 => c:3 - │ ├── column19:19 => d:4 - │ ├── y:14 => e:5 - │ └── rowid_new:18 => rowid:6 + │ ├── y:16 => a:1 + │ ├── y1:19 => b:2 + │ ├── c_new:20 => c:3 + │ ├── column22:22 => d:4 + │ ├── y:16 => e:5 + │ └── rowid_new:21 => rowid:6 └── project - ├── columns: column19:19 a:7!null b:8 c:9 d:10 e:11 rowid:12!null y:14 y1:16 c_new:17!null rowid_new:18!null + ├── columns: column22:22 a:8!null b:9 c:10 d:11 e:12 rowid:13!null abcde.crdb_internal_mvcc_timestamp:14 y:16 y1:19 c_new:20!null rowid_new:21!null ├── project - │ ├── columns: c_new:17!null rowid_new:18!null a:7!null b:8 c:9 d:10 e:11 rowid:12!null y:14 y1:16 + │ ├── columns: c_new:20!null rowid_new:21!null a:8!null b:9 c:10 d:11 e:12 rowid:13!null abcde.crdb_internal_mvcc_timestamp:14 y:16 y1:19 │ ├── left-join-apply - │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null y:14 y1:16 + │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null abcde.crdb_internal_mvcc_timestamp:14 y:16 y1:19 │ │ ├── scan abcde - │ │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null abcde.crdb_internal_mvcc_timestamp:14 │ │ │ └── computed column expressions - │ │ │ ├── d:10 - │ │ │ │ └── (b:8 + c:9) + 1 - │ │ │ └── e:11 - │ │ │ └── a:7 + │ │ │ ├── d:11 + │ │ │ │ └── (b:9 + c:10) + 1 + │ │ │ └── e:12 + │ │ │ └── a:8 │ │ ├── max1-row - │ │ │ ├── columns: y:14 y1:16 + │ │ │ ├── columns: y:16 y1:19 │ │ │ └── project - │ │ │ ├── columns: y1:16 y:14 + │ │ │ ├── columns: y1:19 y:16 │ │ │ ├── scan xyz - │ │ │ │ └── columns: x:13!null y:14 z:15 + │ │ │ │ └── columns: x:15!null y:16 z:17 xyz.crdb_internal_mvcc_timestamp:18 │ │ │ └── projections - │ │ │ └── y:14 + 1 [as=y1:16] + │ │ │ └── y:16 + 1 [as=y1:19] │ │ └── filters (true) │ └── projections - │ ├── 1 [as=c_new:17] - │ └── 2 [as=rowid_new:18] + │ ├── 1 [as=c_new:20] + │ └── 2 [as=rowid_new:21] └── projections - └── (y1:16 + c_new:17) + 1 [as=column19:19] + └── (y1:19 + c_new:20) + 1 [as=column22:22] # Use subquery SET expression after other expressions. build @@ -1058,41 +1058,41 @@ UPDATE abcde SET a=1, (b)=(2), (c, rowid)=(SELECT y, y+1 AS y1 FROM xyz) ---- update abcde ├── columns: - ├── fetch columns: a:7 b:8 c:9 d:10 e:11 rowid:12 + ├── fetch columns: a:8 b:9 c:10 d:11 e:12 rowid:13 ├── update-mapping: - │ ├── a_new:17 => a:1 - │ ├── b_new:18 => b:2 - │ ├── y:14 => c:3 - │ ├── column19:19 => d:4 - │ ├── a_new:17 => e:5 - │ └── y1:16 => rowid:6 + │ ├── a_new:20 => a:1 + │ ├── b_new:21 => b:2 + │ ├── y:16 => c:3 + │ ├── column22:22 => d:4 + │ ├── a_new:20 => e:5 + │ └── y1:19 => rowid:6 └── project - ├── columns: column19:19 a:7!null b:8 c:9 d:10 e:11 rowid:12!null y:14 y1:16 a_new:17!null b_new:18!null + ├── columns: column22:22 a:8!null b:9 c:10 d:11 e:12 rowid:13!null abcde.crdb_internal_mvcc_timestamp:14 y:16 y1:19 a_new:20!null b_new:21!null ├── project - │ ├── columns: a_new:17!null b_new:18!null a:7!null b:8 c:9 d:10 e:11 rowid:12!null y:14 y1:16 + │ ├── columns: a_new:20!null b_new:21!null a:8!null b:9 c:10 d:11 e:12 rowid:13!null abcde.crdb_internal_mvcc_timestamp:14 y:16 y1:19 │ ├── left-join-apply - │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null y:14 y1:16 + │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null abcde.crdb_internal_mvcc_timestamp:14 y:16 y1:19 │ │ ├── scan abcde - │ │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null abcde.crdb_internal_mvcc_timestamp:14 │ │ │ └── computed column expressions - │ │ │ ├── d:10 - │ │ │ │ └── (b:8 + c:9) + 1 - │ │ │ └── e:11 - │ │ │ └── a:7 + │ │ │ ├── d:11 + │ │ │ │ └── (b:9 + c:10) + 1 + │ │ │ └── e:12 + │ │ │ └── a:8 │ │ ├── max1-row - │ │ │ ├── columns: y:14 y1:16 + │ │ │ ├── columns: y:16 y1:19 │ │ │ └── project - │ │ │ ├── columns: y1:16 y:14 + │ │ │ ├── columns: y1:19 y:16 │ │ │ ├── scan xyz - │ │ │ │ └── columns: x:13!null y:14 z:15 + │ │ │ │ └── columns: x:15!null y:16 z:17 xyz.crdb_internal_mvcc_timestamp:18 │ │ │ └── projections - │ │ │ └── y:14 + 1 [as=y1:16] + │ │ │ └── y:16 + 1 [as=y1:19] │ │ └── filters (true) │ └── projections - │ ├── 1 [as=a_new:17] - │ └── 2 [as=b_new:18] + │ ├── 1 [as=a_new:20] + │ └── 2 [as=b_new:21] └── projections - └── (b_new:18 + y:14) + 1 [as=column19:19] + └── (b_new:21 + y:16) + 1 [as=column22:22] # Multiple subqueries in SET expressions. build @@ -1100,48 +1100,48 @@ UPDATE abcde SET (b, a)=(SELECT y, y+1 AS y1 FROM xyz), (c, rowid)=(SELECT 1 AS ---- update abcde ├── columns: - ├── fetch columns: a:7 b:8 c:9 d:10 e:11 rowid:12 + ├── fetch columns: a:8 b:9 c:10 d:11 e:12 rowid:13 ├── update-mapping: - │ ├── y1:16 => a:1 - │ ├── y:14 => b:2 - │ ├── one:17 => c:3 - │ ├── column19:19 => d:4 - │ ├── y1:16 => e:5 - │ └── two:18 => rowid:6 + │ ├── y1:19 => a:1 + │ ├── y:16 => b:2 + │ ├── one:20 => c:3 + │ ├── column22:22 => d:4 + │ ├── y1:19 => e:5 + │ └── two:21 => rowid:6 └── project - ├── columns: column19:19 a:7!null b:8 c:9 d:10 e:11 rowid:12!null y:14 y1:16 one:17 two:18 + ├── columns: column22:22 a:8!null b:9 c:10 d:11 e:12 rowid:13!null abcde.crdb_internal_mvcc_timestamp:14 y:16 y1:19 one:20 two:21 ├── left-join-apply - │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null y:14 y1:16 one:17 two:18 + │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null abcde.crdb_internal_mvcc_timestamp:14 y:16 y1:19 one:20 two:21 │ ├── left-join-apply - │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null y:14 y1:16 + │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null abcde.crdb_internal_mvcc_timestamp:14 y:16 y1:19 │ │ ├── scan abcde - │ │ │ ├── columns: a:7!null b:8 c:9 d:10 e:11 rowid:12!null + │ │ │ ├── columns: a:8!null b:9 c:10 d:11 e:12 rowid:13!null abcde.crdb_internal_mvcc_timestamp:14 │ │ │ └── computed column expressions - │ │ │ ├── d:10 - │ │ │ │ └── (b:8 + c:9) + 1 - │ │ │ └── e:11 - │ │ │ └── a:7 + │ │ │ ├── d:11 + │ │ │ │ └── (b:9 + c:10) + 1 + │ │ │ └── e:12 + │ │ │ └── a:8 │ │ ├── max1-row - │ │ │ ├── columns: y:14 y1:16 + │ │ │ ├── columns: y:16 y1:19 │ │ │ └── project - │ │ │ ├── columns: y1:16 y:14 + │ │ │ ├── columns: y1:19 y:16 │ │ │ ├── scan xyz - │ │ │ │ └── columns: x:13!null y:14 z:15 + │ │ │ │ └── columns: x:15!null y:16 z:17 xyz.crdb_internal_mvcc_timestamp:18 │ │ │ └── projections - │ │ │ └── y:14 + 1 [as=y1:16] + │ │ │ └── y:16 + 1 [as=y1:19] │ │ └── filters (true) │ ├── max1-row - │ │ ├── columns: one:17!null two:18!null + │ │ ├── columns: one:20!null two:21!null │ │ └── project - │ │ ├── columns: one:17!null two:18!null + │ │ ├── columns: one:20!null two:21!null │ │ ├── values │ │ │ └── () │ │ └── projections - │ │ ├── 1 [as=one:17] - │ │ └── 2 [as=two:18] + │ │ ├── 1 [as=one:20] + │ │ └── 2 [as=two:21] │ └── filters (true) └── projections - └── (y:14 + one:17) + 1 [as=column19:19] + └── (y:16 + one:20) + 1 [as=column22:22] # Incorporate desired types when compiling subquery. build @@ -1149,29 +1149,29 @@ UPDATE xyz SET x='foo', (z, y)=(SELECT 2 AS two, 3 AS three) ---- update xyz ├── columns: - ├── fetch columns: x:4 y:5 z:6 + ├── fetch columns: x:5 y:6 z:7 ├── update-mapping: - │ ├── x_new:9 => x:1 - │ ├── three:8 => y:2 - │ └── two:7 => z:3 + │ ├── x_new:11 => x:1 + │ ├── three:10 => y:2 + │ └── two:9 => z:3 └── project - ├── columns: x_new:9!null x:4!null y:5 z:6 two:7 three:8 + ├── columns: x_new:11!null x:5!null y:6 z:7 crdb_internal_mvcc_timestamp:8 two:9 three:10 ├── left-join-apply - │ ├── columns: x:4!null y:5 z:6 two:7 three:8 + │ ├── columns: x:5!null y:6 z:7 crdb_internal_mvcc_timestamp:8 two:9 three:10 │ ├── scan xyz - │ │ └── columns: x:4!null y:5 z:6 + │ │ └── columns: x:5!null y:6 z:7 crdb_internal_mvcc_timestamp:8 │ ├── max1-row - │ │ ├── columns: two:7!null three:8!null + │ │ ├── columns: two:9!null three:10!null │ │ └── project - │ │ ├── columns: two:7!null three:8!null + │ │ ├── columns: two:9!null three:10!null │ │ ├── values │ │ │ └── () │ │ └── projections - │ │ ├── 2.0 [as=two:7] - │ │ └── 3 [as=three:8] + │ │ ├── 2.0 [as=two:9] + │ │ └── 3 [as=three:10] │ └── filters (true) └── projections - └── 'foo' [as=x_new:9] + └── 'foo' [as=x_new:11] # SET expression contains correlated subquery + alias. build @@ -1181,41 +1181,41 @@ project ├── columns: a:1!null b:2 c:3 d:4 e:5!null └── update abcde1 ├── columns: abcde1.a:1!null abcde1.b:2 abcde1.c:3 abcde1.d:4 abcde1.e:5!null abcde1.rowid:6!null - ├── fetch columns: abcde1.a:7 abcde1.b:8 abcde1.c:9 abcde1.d:10 abcde1.e:11 abcde1.rowid:12 + ├── fetch columns: abcde1.a:8 abcde1.b:9 abcde1.c:10 abcde1.d:11 abcde1.e:12 abcde1.rowid:13 ├── update-mapping: - │ ├── b_new:19 => abcde1.b:2 - │ ├── column20:20 => abcde1.d:4 - │ └── abcde1.a:7 => abcde1.e:5 + │ ├── b_new:22 => abcde1.b:2 + │ ├── column23:23 => abcde1.d:4 + │ └── abcde1.a:8 => abcde1.e:5 └── project - ├── columns: column20:20 abcde1.a:7!null abcde1.b:8 abcde1.c:9 abcde1.d:10 abcde1.e:11 abcde1.rowid:12!null b_new:19 + ├── columns: column23:23 abcde1.a:8!null abcde1.b:9 abcde1.c:10 abcde1.d:11 abcde1.e:12 abcde1.rowid:13!null abcde1.crdb_internal_mvcc_timestamp:14 b_new:22 ├── project - │ ├── columns: b_new:19 abcde1.a:7!null abcde1.b:8 abcde1.c:9 abcde1.d:10 abcde1.e:11 abcde1.rowid:12!null + │ ├── columns: b_new:22 abcde1.a:8!null abcde1.b:9 abcde1.c:10 abcde1.d:11 abcde1.e:12 abcde1.rowid:13!null abcde1.crdb_internal_mvcc_timestamp:14 │ ├── scan abcde1 - │ │ ├── columns: abcde1.a:7!null abcde1.b:8 abcde1.c:9 abcde1.d:10 abcde1.e:11 abcde1.rowid:12!null + │ │ ├── columns: abcde1.a:8!null abcde1.b:9 abcde1.c:10 abcde1.d:11 abcde1.e:12 abcde1.rowid:13!null abcde1.crdb_internal_mvcc_timestamp:14 │ │ └── computed column expressions - │ │ ├── abcde1.d:10 - │ │ │ └── (abcde1.b:8 + abcde1.c:9) + 1 - │ │ └── abcde1.e:11 - │ │ └── abcde1.a:7 + │ │ ├── abcde1.d:11 + │ │ │ └── (abcde1.b:9 + abcde1.c:10) + 1 + │ │ └── abcde1.e:12 + │ │ └── abcde1.a:8 │ └── projections - │ └── subquery [as=b_new:19] + │ └── subquery [as=b_new:22] │ └── max1-row - │ ├── columns: abcde2.b:14 + │ ├── columns: abcde2.b:16 │ └── project - │ ├── columns: abcde2.b:14 + │ ├── columns: abcde2.b:16 │ └── select - │ ├── columns: abcde2.a:13!null abcde2.b:14 abcde2.c:15 abcde2.d:16 abcde2.e:17 abcde2.rowid:18!null + │ ├── columns: abcde2.a:15!null abcde2.b:16 abcde2.c:17 abcde2.d:18 abcde2.e:19 abcde2.rowid:20!null abcde2.crdb_internal_mvcc_timestamp:21 │ ├── scan abcde2 - │ │ ├── columns: abcde2.a:13!null abcde2.b:14 abcde2.c:15 abcde2.d:16 abcde2.e:17 abcde2.rowid:18!null + │ │ ├── columns: abcde2.a:15!null abcde2.b:16 abcde2.c:17 abcde2.d:18 abcde2.e:19 abcde2.rowid:20!null abcde2.crdb_internal_mvcc_timestamp:21 │ │ └── computed column expressions - │ │ ├── abcde2.d:16 - │ │ │ └── (abcde2.b:14 + abcde2.c:15) + 1 - │ │ └── abcde2.e:17 - │ │ └── abcde2.a:13 + │ │ ├── abcde2.d:18 + │ │ │ └── (abcde2.b:16 + abcde2.c:17) + 1 + │ │ └── abcde2.e:19 + │ │ └── abcde2.a:15 │ └── filters - │ └── abcde2.rowid:18 = abcde1.a:7 + │ └── abcde2.rowid:20 = abcde1.a:8 └── projections - └── (b_new:19 + abcde1.c:9) + 1 [as=column20:20] + └── (b_new:22 + abcde1.c:10) + 1 [as=column23:23] # Too many values. build @@ -1253,33 +1253,33 @@ with &1 (cte) ├── project │ ├── columns: xyz.x:1!null │ └── scan xyz - │ └── columns: xyz.x:1!null y:2 z:3 + │ └── columns: xyz.x:1!null y:2 z:3 xyz.crdb_internal_mvcc_timestamp:4 └── update abcde ├── columns: - ├── fetch columns: a:10 b:11 c:12 d:13 e:14 rowid:15 + ├── fetch columns: a:12 b:13 c:14 d:15 e:16 rowid:17 ├── update-mapping: - │ ├── b:11 => a:4 - │ ├── column17:17 => d:7 - │ └── b:11 => e:8 + │ ├── b:13 => a:5 + │ ├── column20:20 => d:8 + │ └── b:13 => e:9 └── project - ├── columns: column17:17 a:10!null b:11 c:12 d:13 e:14 rowid:15!null + ├── columns: column20:20 a:12!null b:13 c:14 d:15 e:16 rowid:17!null abcde.crdb_internal_mvcc_timestamp:18 ├── select - │ ├── columns: a:10!null b:11 c:12 d:13 e:14 rowid:15!null + │ ├── columns: a:12!null b:13 c:14 d:15 e:16 rowid:17!null abcde.crdb_internal_mvcc_timestamp:18 │ ├── scan abcde - │ │ ├── columns: a:10!null b:11 c:12 d:13 e:14 rowid:15!null + │ │ ├── columns: a:12!null b:13 c:14 d:15 e:16 rowid:17!null abcde.crdb_internal_mvcc_timestamp:18 │ │ └── computed column expressions - │ │ ├── d:13 - │ │ │ └── (b:11 + c:12) + 1 - │ │ └── e:14 - │ │ └── a:10 + │ │ ├── d:15 + │ │ │ └── (b:13 + c:14) + 1 + │ │ └── e:16 + │ │ └── a:12 │ └── filters │ └── exists │ └── with-scan &1 (cte) - │ ├── columns: x:16!null + │ ├── columns: x:19!null │ └── mapping: - │ └── xyz.x:1 => x:16 + │ └── xyz.x:1 => x:19 └── projections - └── (b:11 + c:12) + 1 [as=column17:17] + └── (b:13 + c:14) + 1 [as=column20:20] # Use CTE within SET expression. build @@ -1287,40 +1287,40 @@ WITH a AS (SELECT y, y+1 AS y1 FROM xyz) UPDATE abcde SET (a, b) = (SELECT * FRO ---- with &1 (a) ├── project - │ ├── columns: y1:4 xyz.y:2 + │ ├── columns: y1:5 xyz.y:2 │ ├── scan xyz - │ │ └── columns: x:1!null xyz.y:2 z:3 + │ │ └── columns: x:1!null xyz.y:2 z:3 xyz.crdb_internal_mvcc_timestamp:4 │ └── projections - │ └── xyz.y:2 + 1 [as=y1:4] + │ └── xyz.y:2 + 1 [as=y1:5] └── update abcde ├── columns: - ├── fetch columns: a:11 b:12 c:13 d:14 e:15 rowid:16 + ├── fetch columns: a:13 b:14 c:15 d:16 e:17 rowid:18 ├── update-mapping: - │ ├── y:17 => a:5 - │ ├── y1:18 => b:6 - │ ├── column19:19 => d:8 - │ └── y:17 => e:9 + │ ├── y:20 => a:6 + │ ├── y1:21 => b:7 + │ ├── column22:22 => d:9 + │ └── y:20 => e:10 └── project - ├── columns: column19:19 a:11!null b:12 c:13 d:14 e:15 rowid:16!null y:17 y1:18 + ├── columns: column22:22 a:13!null b:14 c:15 d:16 e:17 rowid:18!null abcde.crdb_internal_mvcc_timestamp:19 y:20 y1:21 ├── left-join-apply - │ ├── columns: a:11!null b:12 c:13 d:14 e:15 rowid:16!null y:17 y1:18 + │ ├── columns: a:13!null b:14 c:15 d:16 e:17 rowid:18!null abcde.crdb_internal_mvcc_timestamp:19 y:20 y1:21 │ ├── scan abcde - │ │ ├── columns: a:11!null b:12 c:13 d:14 e:15 rowid:16!null + │ │ ├── columns: a:13!null b:14 c:15 d:16 e:17 rowid:18!null abcde.crdb_internal_mvcc_timestamp:19 │ │ └── computed column expressions - │ │ ├── d:14 - │ │ │ └── (b:12 + c:13) + 1 - │ │ └── e:15 - │ │ └── a:11 + │ │ ├── d:16 + │ │ │ └── (b:14 + c:15) + 1 + │ │ └── e:17 + │ │ └── a:13 │ ├── max1-row - │ │ ├── columns: y:17 y1:18 + │ │ ├── columns: y:20 y1:21 │ │ └── with-scan &1 (a) - │ │ ├── columns: y:17 y1:18 + │ │ ├── columns: y:20 y1:21 │ │ └── mapping: - │ │ ├── xyz.y:2 => y:17 - │ │ └── y1:4 => y1:18 + │ │ ├── xyz.y:2 => y:20 + │ │ └── y1:5 => y1:21 │ └── filters (true) └── projections - └── (y1:18 + c:13) + 1 [as=column19:19] + └── (y1:21 + c:15) + 1 [as=column22:22] # ------------------------------------------------------------------------------ # Tests with mutations. @@ -1332,32 +1332,32 @@ UPDATE mutation SET m=1 ---- update mutation ├── columns: - ├── fetch columns: m:6 n:7 o:8 p:9 q:10 + ├── fetch columns: m:7 n:8 o:9 p:10 q:11 ├── update-mapping: - │ ├── m_new:11 => m:1 - │ ├── column12:12 => o:3 - │ └── column13:13 => p:4 - ├── check columns: check1:14 + │ ├── m_new:13 => m:1 + │ ├── column14:14 => o:3 + │ └── column15:15 => p:4 + ├── check columns: check1:16 └── project - ├── columns: check1:14!null m:6!null n:7 o:8 p:9 q:10 m_new:11!null column12:12!null column13:13 + ├── columns: check1:16!null m:7!null n:8 o:9 p:10 q:11 crdb_internal_mvcc_timestamp:12 m_new:13!null column14:14!null column15:15 ├── project - │ ├── columns: column13:13 m:6!null n:7 o:8 p:9 q:10 m_new:11!null column12:12!null + │ ├── columns: column15:15 m:7!null n:8 o:9 p:10 q:11 crdb_internal_mvcc_timestamp:12 m_new:13!null column14:14!null │ ├── project - │ │ ├── columns: column12:12!null m:6!null n:7 o:8 p:9 q:10 m_new:11!null + │ │ ├── columns: column14:14!null m:7!null n:8 o:9 p:10 q:11 crdb_internal_mvcc_timestamp:12 m_new:13!null │ │ ├── project - │ │ │ ├── columns: m_new:11!null m:6!null n:7 o:8 p:9 q:10 + │ │ │ ├── columns: m_new:13!null m:7!null n:8 o:9 p:10 q:11 crdb_internal_mvcc_timestamp:12 │ │ │ ├── scan mutation - │ │ │ │ ├── columns: m:6!null n:7 o:8 p:9 q:10 + │ │ │ │ ├── columns: m:7!null n:8 o:9 p:10 q:11 crdb_internal_mvcc_timestamp:12 │ │ │ │ └── check constraint expressions - │ │ │ │ └── m:6 > 0 + │ │ │ │ └── m:7 > 0 │ │ │ └── projections - │ │ │ └── 1 [as=m_new:11] + │ │ │ └── 1 [as=m_new:13] │ │ └── projections - │ │ └── 10 [as=column12:12] + │ │ └── 10 [as=column14:14] │ └── projections - │ └── column12:12 + n:7 [as=column13:13] + │ └── column14:14 + n:8 [as=column15:15] └── projections - └── m_new:11 > 0 [as=check1:14] + └── m_new:13 > 0 [as=check1:16] # Update column that mutation column depends upon. build @@ -1365,34 +1365,34 @@ UPDATE mutation SET m=1, n=2 ---- update mutation ├── columns: - ├── fetch columns: m:6 n:7 o:8 p:9 q:10 + ├── fetch columns: m:7 n:8 o:9 p:10 q:11 ├── update-mapping: - │ ├── m_new:11 => m:1 - │ ├── n_new:12 => n:2 - │ ├── column13:13 => o:3 - │ └── column14:14 => p:4 - ├── check columns: check1:15 + │ ├── m_new:13 => m:1 + │ ├── n_new:14 => n:2 + │ ├── column15:15 => o:3 + │ └── column16:16 => p:4 + ├── check columns: check1:17 └── project - ├── columns: check1:15!null m:6!null n:7 o:8 p:9 q:10 m_new:11!null n_new:12!null column13:13!null column14:14!null + ├── columns: check1:17!null m:7!null n:8 o:9 p:10 q:11 crdb_internal_mvcc_timestamp:12 m_new:13!null n_new:14!null column15:15!null column16:16!null ├── project - │ ├── columns: column14:14!null m:6!null n:7 o:8 p:9 q:10 m_new:11!null n_new:12!null column13:13!null + │ ├── columns: column16:16!null m:7!null n:8 o:9 p:10 q:11 crdb_internal_mvcc_timestamp:12 m_new:13!null n_new:14!null column15:15!null │ ├── project - │ │ ├── columns: column13:13!null m:6!null n:7 o:8 p:9 q:10 m_new:11!null n_new:12!null + │ │ ├── columns: column15:15!null m:7!null n:8 o:9 p:10 q:11 crdb_internal_mvcc_timestamp:12 m_new:13!null n_new:14!null │ │ ├── project - │ │ │ ├── columns: m_new:11!null n_new:12!null m:6!null n:7 o:8 p:9 q:10 + │ │ │ ├── columns: m_new:13!null n_new:14!null m:7!null n:8 o:9 p:10 q:11 crdb_internal_mvcc_timestamp:12 │ │ │ ├── scan mutation - │ │ │ │ ├── columns: m:6!null n:7 o:8 p:9 q:10 + │ │ │ │ ├── columns: m:7!null n:8 o:9 p:10 q:11 crdb_internal_mvcc_timestamp:12 │ │ │ │ └── check constraint expressions - │ │ │ │ └── m:6 > 0 + │ │ │ │ └── m:7 > 0 │ │ │ └── projections - │ │ │ ├── 1 [as=m_new:11] - │ │ │ └── 2 [as=n_new:12] + │ │ │ ├── 1 [as=m_new:13] + │ │ │ └── 2 [as=n_new:14] │ │ └── projections - │ │ └── 10 [as=column13:13] + │ │ └── 10 [as=column15:15] │ └── projections - │ └── column13:13 + n_new:12 [as=column14:14] + │ └── column15:15 + n_new:14 [as=column16:16] └── projections - └── m_new:11 > 0 [as=check1:15] + └── m_new:13 > 0 [as=check1:17] # Ensure that ORDER BY wildcard does not select mutation columns. build @@ -1400,41 +1400,41 @@ UPDATE mutation SET m=1 ORDER BY mutation.* LIMIT 10 ---- update mutation ├── columns: - ├── fetch columns: m:6 n:7 o:8 p:9 q:10 + ├── fetch columns: m:7 n:8 o:9 p:10 q:11 ├── update-mapping: - │ ├── m_new:11 => m:1 - │ ├── column12:12 => o:3 - │ └── column13:13 => p:4 - ├── check columns: check1:14 + │ ├── m_new:13 => m:1 + │ ├── column14:14 => o:3 + │ └── column15:15 => p:4 + ├── check columns: check1:16 └── project - ├── columns: check1:14!null m:6!null n:7 o:8 p:9 q:10 m_new:11!null column12:12!null column13:13 + ├── columns: check1:16!null m:7!null n:8 o:9 p:10 q:11 crdb_internal_mvcc_timestamp:12 m_new:13!null column14:14!null column15:15 ├── project - │ ├── columns: column13:13 m:6!null n:7 o:8 p:9 q:10 m_new:11!null column12:12!null + │ ├── columns: column15:15 m:7!null n:8 o:9 p:10 q:11 crdb_internal_mvcc_timestamp:12 m_new:13!null column14:14!null │ ├── project - │ │ ├── columns: column12:12!null m:6!null n:7 o:8 p:9 q:10 m_new:11!null + │ │ ├── columns: column14:14!null m:7!null n:8 o:9 p:10 q:11 crdb_internal_mvcc_timestamp:12 m_new:13!null │ │ ├── project - │ │ │ ├── columns: m_new:11!null m:6!null n:7 o:8 p:9 q:10 + │ │ │ ├── columns: m_new:13!null m:7!null n:8 o:9 p:10 q:11 crdb_internal_mvcc_timestamp:12 │ │ │ ├── limit - │ │ │ │ ├── columns: m:6!null n:7 o:8 p:9 q:10 - │ │ │ │ ├── internal-ordering: +6,+7 + │ │ │ │ ├── columns: m:7!null n:8 o:9 p:10 q:11 crdb_internal_mvcc_timestamp:12 + │ │ │ │ ├── internal-ordering: +7,+8 │ │ │ │ ├── sort (segmented) - │ │ │ │ │ ├── columns: m:6!null n:7 o:8 p:9 q:10 - │ │ │ │ │ ├── ordering: +6,+7 + │ │ │ │ │ ├── columns: m:7!null n:8 o:9 p:10 q:11 crdb_internal_mvcc_timestamp:12 + │ │ │ │ │ ├── ordering: +7,+8 │ │ │ │ │ ├── limit hint: 10.00 │ │ │ │ │ └── scan mutation - │ │ │ │ │ ├── columns: m:6!null n:7 o:8 p:9 q:10 + │ │ │ │ │ ├── columns: m:7!null n:8 o:9 p:10 q:11 crdb_internal_mvcc_timestamp:12 │ │ │ │ │ ├── check constraint expressions - │ │ │ │ │ │ └── m:6 > 0 - │ │ │ │ │ └── ordering: +6 + │ │ │ │ │ │ └── m:7 > 0 + │ │ │ │ │ └── ordering: +7 │ │ │ │ └── 10 │ │ │ └── projections - │ │ │ └── 1 [as=m_new:11] + │ │ │ └── 1 [as=m_new:13] │ │ └── projections - │ │ └── 10 [as=column12:12] + │ │ └── 10 [as=column14:14] │ └── projections - │ └── column12:12 + n:7 [as=column13:13] + │ └── column14:14 + n:8 [as=column15:15] └── projections - └── m_new:11 > 0 [as=check1:14] + └── m_new:13 > 0 [as=check1:16] # Try to return a mutation column. build @@ -1476,35 +1476,35 @@ UPDATE checks SET a=1, b=2, c=3 ---- update checks ├── columns: - ├── fetch columns: a:5 b:6 c:7 d:8 + ├── fetch columns: a:6 b:7 c:8 d:9 ├── update-mapping: - │ ├── a_new:9 => a:1 - │ ├── b_new:10 => b:2 - │ ├── c_new:11 => c:3 - │ └── column12:12 => d:4 - ├── check columns: check1:13 check2:14 + │ ├── a_new:11 => a:1 + │ ├── b_new:12 => b:2 + │ ├── c_new:13 => c:3 + │ └── column14:14 => d:4 + ├── check columns: check1:15 check2:16 └── project - ├── columns: check1:13!null check2:14!null a:5!null b:6 c:7 d:8 a_new:9!null b_new:10!null c_new:11!null column12:12!null + ├── columns: check1:15!null check2:16!null a:6!null b:7 c:8 d:9 crdb_internal_mvcc_timestamp:10 a_new:11!null b_new:12!null c_new:13!null column14:14!null ├── project - │ ├── columns: column12:12!null a:5!null b:6 c:7 d:8 a_new:9!null b_new:10!null c_new:11!null + │ ├── columns: column14:14!null a:6!null b:7 c:8 d:9 crdb_internal_mvcc_timestamp:10 a_new:11!null b_new:12!null c_new:13!null │ ├── project - │ │ ├── columns: a_new:9!null b_new:10!null c_new:11!null a:5!null b:6 c:7 d:8 + │ │ ├── columns: a_new:11!null b_new:12!null c_new:13!null a:6!null b:7 c:8 d:9 crdb_internal_mvcc_timestamp:10 │ │ ├── scan checks - │ │ │ ├── columns: a:5!null b:6 c:7 d:8 + │ │ │ ├── columns: a:6!null b:7 c:8 d:9 crdb_internal_mvcc_timestamp:10 │ │ │ ├── check constraint expressions - │ │ │ │ └── a:5 > 0 + │ │ │ │ └── a:6 > 0 │ │ │ └── computed column expressions - │ │ │ └── d:8 - │ │ │ └── c:7 + 1 + │ │ │ └── d:9 + │ │ │ └── c:8 + 1 │ │ └── projections - │ │ ├── 1 [as=a_new:9] - │ │ ├── 2 [as=b_new:10] - │ │ └── 3 [as=c_new:11] + │ │ ├── 1 [as=a_new:11] + │ │ ├── 2 [as=b_new:12] + │ │ └── 3 [as=c_new:13] │ └── projections - │ └── c_new:11 + 1 [as=column12:12] + │ └── c_new:13 + 1 [as=column14:14] └── projections - ├── b_new:10 < column12:12 [as=check1:13] - └── a_new:9 > 0 [as=check2:14] + ├── b_new:12 < column14:14 [as=check1:15] + └── a_new:11 > 0 [as=check2:16] # Do not update columns for one of the constraints. build @@ -1512,31 +1512,31 @@ UPDATE checks SET a=1 ---- update checks ├── columns: - ├── fetch columns: a:5 b:6 c:7 d:8 + ├── fetch columns: a:6 b:7 c:8 d:9 ├── update-mapping: - │ ├── a_new:9 => a:1 - │ └── column10:10 => d:4 - ├── check columns: check1:11 check2:12 + │ ├── a_new:11 => a:1 + │ └── column12:12 => d:4 + ├── check columns: check1:13 check2:14 └── project - ├── columns: check1:11 check2:12!null a:5!null b:6 c:7 d:8 a_new:9!null column10:10 + ├── columns: check1:13 check2:14!null a:6!null b:7 c:8 d:9 crdb_internal_mvcc_timestamp:10 a_new:11!null column12:12 ├── project - │ ├── columns: column10:10 a:5!null b:6 c:7 d:8 a_new:9!null + │ ├── columns: column12:12 a:6!null b:7 c:8 d:9 crdb_internal_mvcc_timestamp:10 a_new:11!null │ ├── project - │ │ ├── columns: a_new:9!null a:5!null b:6 c:7 d:8 + │ │ ├── columns: a_new:11!null a:6!null b:7 c:8 d:9 crdb_internal_mvcc_timestamp:10 │ │ ├── scan checks - │ │ │ ├── columns: a:5!null b:6 c:7 d:8 + │ │ │ ├── columns: a:6!null b:7 c:8 d:9 crdb_internal_mvcc_timestamp:10 │ │ │ ├── check constraint expressions - │ │ │ │ └── a:5 > 0 + │ │ │ │ └── a:6 > 0 │ │ │ └── computed column expressions - │ │ │ └── d:8 - │ │ │ └── c:7 + 1 + │ │ │ └── d:9 + │ │ │ └── c:8 + 1 │ │ └── projections - │ │ └── 1 [as=a_new:9] + │ │ └── 1 [as=a_new:11] │ └── projections - │ └── c:7 + 1 [as=column10:10] + │ └── c:8 + 1 [as=column12:12] └── projections - ├── b:6 < column10:10 [as=check1:11] - └── a_new:9 > 0 [as=check2:12] + ├── b:7 < column12:12 [as=check1:13] + └── a_new:11 > 0 [as=check2:14] # Update one column in constraint, but not the other. build @@ -1544,31 +1544,31 @@ UPDATE checks SET b=2 ---- update checks ├── columns: - ├── fetch columns: a:5 b:6 c:7 d:8 + ├── fetch columns: a:6 b:7 c:8 d:9 ├── update-mapping: - │ ├── b_new:9 => b:2 - │ └── column10:10 => d:4 - ├── check columns: check1:11 check2:12 + │ ├── b_new:11 => b:2 + │ └── column12:12 => d:4 + ├── check columns: check1:13 check2:14 └── project - ├── columns: check1:11 check2:12!null a:5!null b:6 c:7 d:8 b_new:9!null column10:10 + ├── columns: check1:13 check2:14!null a:6!null b:7 c:8 d:9 crdb_internal_mvcc_timestamp:10 b_new:11!null column12:12 ├── project - │ ├── columns: column10:10 a:5!null b:6 c:7 d:8 b_new:9!null + │ ├── columns: column12:12 a:6!null b:7 c:8 d:9 crdb_internal_mvcc_timestamp:10 b_new:11!null │ ├── project - │ │ ├── columns: b_new:9!null a:5!null b:6 c:7 d:8 + │ │ ├── columns: b_new:11!null a:6!null b:7 c:8 d:9 crdb_internal_mvcc_timestamp:10 │ │ ├── scan checks - │ │ │ ├── columns: a:5!null b:6 c:7 d:8 + │ │ │ ├── columns: a:6!null b:7 c:8 d:9 crdb_internal_mvcc_timestamp:10 │ │ │ ├── check constraint expressions - │ │ │ │ └── a:5 > 0 + │ │ │ │ └── a:6 > 0 │ │ │ └── computed column expressions - │ │ │ └── d:8 - │ │ │ └── c:7 + 1 + │ │ │ └── d:9 + │ │ │ └── c:8 + 1 │ │ └── projections - │ │ └── 2 [as=b_new:9] + │ │ └── 2 [as=b_new:11] │ └── projections - │ └── c:7 + 1 [as=column10:10] + │ └── c:8 + 1 [as=column12:12] └── projections - ├── b_new:9 < column10:10 [as=check1:11] - └── a:5 > 0 [as=check2:12] + ├── b_new:11 < column12:12 [as=check1:13] + └── a:6 > 0 [as=check2:14] # Update using tuple and subquery. build @@ -1576,46 +1576,46 @@ UPDATE checks SET (a, b)=(SELECT a, b FROM abcde WHERE abcde.a=checks.a) ---- update checks ├── columns: - ├── fetch columns: checks.a:5 checks.b:6 checks.c:7 checks.d:8 + ├── fetch columns: checks.a:6 checks.b:7 checks.c:8 checks.d:9 ├── update-mapping: - │ ├── abcde.a:9 => checks.a:1 - │ ├── abcde.b:10 => checks.b:2 - │ └── column15:15 => checks.d:4 - ├── check columns: check1:16 check2:17 + │ ├── abcde.a:11 => checks.a:1 + │ ├── abcde.b:12 => checks.b:2 + │ └── column18:18 => checks.d:4 + ├── check columns: check1:19 check2:20 └── project - ├── columns: check1:16 check2:17 checks.a:5!null checks.b:6 checks.c:7 checks.d:8 abcde.a:9 abcde.b:10 column15:15 + ├── columns: check1:19 check2:20 checks.a:6!null checks.b:7 checks.c:8 checks.d:9 checks.crdb_internal_mvcc_timestamp:10 abcde.a:11 abcde.b:12 column18:18 ├── project - │ ├── columns: column15:15 checks.a:5!null checks.b:6 checks.c:7 checks.d:8 abcde.a:9 abcde.b:10 + │ ├── columns: column18:18 checks.a:6!null checks.b:7 checks.c:8 checks.d:9 checks.crdb_internal_mvcc_timestamp:10 abcde.a:11 abcde.b:12 │ ├── left-join-apply - │ │ ├── columns: checks.a:5!null checks.b:6 checks.c:7 checks.d:8 abcde.a:9 abcde.b:10 + │ │ ├── columns: checks.a:6!null checks.b:7 checks.c:8 checks.d:9 checks.crdb_internal_mvcc_timestamp:10 abcde.a:11 abcde.b:12 │ │ ├── scan checks - │ │ │ ├── columns: checks.a:5!null checks.b:6 checks.c:7 checks.d:8 + │ │ │ ├── columns: checks.a:6!null checks.b:7 checks.c:8 checks.d:9 checks.crdb_internal_mvcc_timestamp:10 │ │ │ ├── check constraint expressions - │ │ │ │ └── checks.a:5 > 0 + │ │ │ │ └── checks.a:6 > 0 │ │ │ └── computed column expressions - │ │ │ └── checks.d:8 - │ │ │ └── checks.c:7 + 1 + │ │ │ └── checks.d:9 + │ │ │ └── checks.c:8 + 1 │ │ ├── max1-row - │ │ │ ├── columns: abcde.a:9!null abcde.b:10 + │ │ │ ├── columns: abcde.a:11!null abcde.b:12 │ │ │ └── project - │ │ │ ├── columns: abcde.a:9!null abcde.b:10 + │ │ │ ├── columns: abcde.a:11!null abcde.b:12 │ │ │ └── select - │ │ │ ├── columns: abcde.a:9!null abcde.b:10 abcde.c:11 abcde.d:12 e:13 rowid:14!null + │ │ │ ├── columns: abcde.a:11!null abcde.b:12 abcde.c:13 abcde.d:14 e:15 rowid:16!null abcde.crdb_internal_mvcc_timestamp:17 │ │ │ ├── scan abcde - │ │ │ │ ├── columns: abcde.a:9!null abcde.b:10 abcde.c:11 abcde.d:12 e:13 rowid:14!null + │ │ │ │ ├── columns: abcde.a:11!null abcde.b:12 abcde.c:13 abcde.d:14 e:15 rowid:16!null abcde.crdb_internal_mvcc_timestamp:17 │ │ │ │ └── computed column expressions - │ │ │ │ ├── abcde.d:12 - │ │ │ │ │ └── (abcde.b:10 + abcde.c:11) + 1 - │ │ │ │ └── e:13 - │ │ │ │ └── abcde.a:9 + │ │ │ │ ├── abcde.d:14 + │ │ │ │ │ └── (abcde.b:12 + abcde.c:13) + 1 + │ │ │ │ └── e:15 + │ │ │ │ └── abcde.a:11 │ │ │ └── filters - │ │ │ └── abcde.a:9 = checks.a:5 + │ │ │ └── abcde.a:11 = checks.a:6 │ │ └── filters (true) │ └── projections - │ └── checks.c:7 + 1 [as=column15:15] + │ └── checks.c:8 + 1 [as=column18:18] └── projections - ├── abcde.b:10 < column15:15 [as=check1:16] - └── abcde.a:9 > 0 [as=check2:17] + ├── abcde.b:12 < column18:18 [as=check1:19] + └── abcde.a:11 > 0 [as=check2:20] # ------------------------------------------------------------------------------ # Test decimal column truncation. @@ -1626,40 +1626,40 @@ UPDATE decimals SET a=1.1, b=ARRAY[0.95, NULL, 15] ---- update decimals ├── columns: - ├── fetch columns: decimals.a:5 decimals.b:6 c:7 decimals.d:8 + ├── fetch columns: decimals.a:6 decimals.b:7 c:8 decimals.d:9 ├── update-mapping: - │ ├── a:11 => decimals.a:1 - │ ├── b:12 => decimals.b:2 - │ └── d:14 => decimals.d:4 - ├── check columns: check1:15 check2:16 + │ ├── a:13 => decimals.a:1 + │ ├── b:14 => decimals.b:2 + │ └── d:16 => decimals.d:4 + ├── check columns: check1:17 check2:18 └── project - ├── columns: check1:15 check2:16 decimals.a:5!null decimals.b:6 c:7 decimals.d:8 a:11 b:12 d:14 + ├── columns: check1:17 check2:18 decimals.a:6!null decimals.b:7 c:8 decimals.d:9 crdb_internal_mvcc_timestamp:10 a:13 b:14 d:16 ├── project - │ ├── columns: d:14 decimals.a:5!null decimals.b:6 c:7 decimals.d:8 a:11 b:12 + │ ├── columns: d:16 decimals.a:6!null decimals.b:7 c:8 decimals.d:9 crdb_internal_mvcc_timestamp:10 a:13 b:14 │ ├── project - │ │ ├── columns: column13:13 decimals.a:5!null decimals.b:6 c:7 decimals.d:8 a:11 b:12 + │ │ ├── columns: column15:15 decimals.a:6!null decimals.b:7 c:8 decimals.d:9 crdb_internal_mvcc_timestamp:10 a:13 b:14 │ │ ├── project - │ │ │ ├── columns: a:11 b:12 decimals.a:5!null decimals.b:6 c:7 decimals.d:8 + │ │ │ ├── columns: a:13 b:14 decimals.a:6!null decimals.b:7 c:8 decimals.d:9 crdb_internal_mvcc_timestamp:10 │ │ │ ├── project - │ │ │ │ ├── columns: a_new:9!null b_new:10 decimals.a:5!null decimals.b:6 c:7 decimals.d:8 + │ │ │ │ ├── columns: a_new:11!null b_new:12 decimals.a:6!null decimals.b:7 c:8 decimals.d:9 crdb_internal_mvcc_timestamp:10 │ │ │ │ ├── scan decimals - │ │ │ │ │ ├── columns: decimals.a:5!null decimals.b:6 c:7 decimals.d:8 + │ │ │ │ │ ├── columns: decimals.a:6!null decimals.b:7 c:8 decimals.d:9 crdb_internal_mvcc_timestamp:10 │ │ │ │ │ └── computed column expressions - │ │ │ │ │ └── decimals.d:8 - │ │ │ │ │ └── decimals.a:5::DECIMAL + c:7::DECIMAL + │ │ │ │ │ └── decimals.d:9 + │ │ │ │ │ └── decimals.a:6::DECIMAL + c:8::DECIMAL │ │ │ │ └── projections - │ │ │ │ ├── 1.1 [as=a_new:9] - │ │ │ │ └── ARRAY[0.95,NULL,15] [as=b_new:10] + │ │ │ │ ├── 1.1 [as=a_new:11] + │ │ │ │ └── ARRAY[0.95,NULL,15] [as=b_new:12] │ │ │ └── projections - │ │ │ ├── crdb_internal.round_decimal_values(a_new:9, 0) [as=a:11] - │ │ │ └── crdb_internal.round_decimal_values(b_new:10, 1) [as=b:12] + │ │ │ ├── crdb_internal.round_decimal_values(a_new:11, 0) [as=a:13] + │ │ │ └── crdb_internal.round_decimal_values(b_new:12, 1) [as=b:14] │ │ └── projections - │ │ └── a:11 + c:7::DECIMAL [as=column13:13] + │ │ └── a:13 + c:8::DECIMAL [as=column15:15] │ └── projections - │ └── crdb_internal.round_decimal_values(column13:13, 1) [as=d:14] + │ └── crdb_internal.round_decimal_values(column15:15, 1) [as=d:16] └── projections - ├── round(a:11) = a:11 [as=check1:15] - └── b:12[0] > 1 [as=check2:16] + ├── round(a:13) = a:13 [as=check1:17] + └── b:14[0] > 1 [as=check2:18] # ------------------------------------------------------------------------------ # Test partial index column values. @@ -1683,31 +1683,31 @@ UPDATE partial_indexes SET a = 1 ---- update partial_indexes ├── columns: - ├── fetch columns: a:4 b:5 c:6 + ├── fetch columns: a:5 b:6 c:7 ├── update-mapping: - │ └── a_new:11 => a:1 - ├── partial index put columns: partial_index_del1:7 partial_index_put2:12 partial_index_del3:9 partial_index_del4:10 - ├── partial index del columns: partial_index_del1:7 partial_index_del2:8 partial_index_del3:9 partial_index_del4:10 + │ └── a_new:13 => a:1 + ├── partial index put columns: partial_index_del1:9 partial_index_put2:14 partial_index_del3:11 partial_index_del4:12 + ├── partial index del columns: partial_index_del1:9 partial_index_del2:10 partial_index_del3:11 partial_index_del4:12 └── project - ├── columns: partial_index_put2:12 a:4!null b:5 c:6 partial_index_del1:7 partial_index_del2:8 partial_index_del3:9 partial_index_del4:10 a_new:11!null + ├── columns: partial_index_put2:14 a:5!null b:6 c:7 crdb_internal_mvcc_timestamp:8 partial_index_del1:9 partial_index_del2:10 partial_index_del3:11 partial_index_del4:12 a_new:13!null ├── project - │ ├── columns: a_new:11!null a:4!null b:5 c:6 partial_index_del1:7 partial_index_del2:8 partial_index_del3:9 partial_index_del4:10 + │ ├── columns: a_new:13!null a:5!null b:6 c:7 crdb_internal_mvcc_timestamp:8 partial_index_del1:9 partial_index_del2:10 partial_index_del3:11 partial_index_del4:12 │ ├── project - │ │ ├── columns: partial_index_del1:7 partial_index_del2:8 partial_index_del3:9 partial_index_del4:10 a:4!null b:5 c:6 + │ │ ├── columns: partial_index_del1:9 partial_index_del2:10 partial_index_del3:11 partial_index_del4:12 a:5!null b:6 c:7 crdb_internal_mvcc_timestamp:8 │ │ ├── scan partial_indexes - │ │ │ ├── columns: a:4!null b:5 c:6 + │ │ │ ├── columns: a:5!null b:6 c:7 crdb_internal_mvcc_timestamp:8 │ │ │ └── partial index predicates │ │ │ ├── secondary: filters - │ │ │ │ └── c:6 = 'foo' + │ │ │ │ └── c:7 = 'foo' │ │ │ └── secondary: filters - │ │ │ ├── a:4 > b:5 - │ │ │ └── c:6 = 'bar' + │ │ │ ├── a:5 > b:6 + │ │ │ └── c:7 = 'bar' │ │ └── projections - │ │ ├── c:6 = 'foo' [as=partial_index_del1:7] - │ │ ├── (a:4 > b:5) AND (c:6 = 'bar') [as=partial_index_del2:8] - │ │ ├── c:6 = 'delete-only' [as=partial_index_del3:9] - │ │ └── c:6 = 'write-only' [as=partial_index_del4:10] + │ │ ├── c:7 = 'foo' [as=partial_index_del1:9] + │ │ ├── (a:5 > b:6) AND (c:7 = 'bar') [as=partial_index_del2:10] + │ │ ├── c:7 = 'delete-only' [as=partial_index_del3:11] + │ │ └── c:7 = 'write-only' [as=partial_index_del4:12] │ └── projections - │ └── 1 [as=a_new:11] + │ └── 1 [as=a_new:13] └── projections - └── (a_new:11 > b:5) AND (c:6 = 'bar') [as=partial_index_put2:12] + └── (a_new:13 > b:6) AND (c:7 = 'bar') [as=partial_index_put2:14] diff --git a/pkg/sql/opt/optbuilder/testdata/update_from b/pkg/sql/opt/optbuilder/testdata/update_from index 8b113e9e3e34..89db59bdff45 100644 --- a/pkg/sql/opt/optbuilder/testdata/update_from +++ b/pkg/sql/opt/optbuilder/testdata/update_from @@ -12,26 +12,26 @@ UPDATE abc SET b = other.b + 1, c = other.c + 1 FROM abc AS other WHERE abc.a = ---- update abc ├── columns: - ├── fetch columns: abc.a:4 abc.b:5 abc.c:6 + ├── fetch columns: abc.a:5 abc.b:6 abc.c:7 ├── update-mapping: - │ ├── b_new:10 => abc.b:2 - │ └── c_new:11 => abc.c:3 + │ ├── b_new:13 => abc.b:2 + │ └── c_new:14 => abc.c:3 └── project - ├── columns: b_new:10 c_new:11 abc.a:4!null abc.b:5 abc.c:6 other.a:7!null other.b:8 other.c:9 + ├── columns: b_new:13 c_new:14 abc.a:5!null abc.b:6 abc.c:7 other.a:9!null other.b:10 other.c:11 other.crdb_internal_mvcc_timestamp:12 ├── inner-join (merge) - │ ├── columns: abc.a:4!null abc.b:5 abc.c:6 other.a:7!null other.b:8 other.c:9 - │ ├── left ordering: +4 - │ ├── right ordering: +7 + │ ├── columns: abc.a:5!null abc.b:6 abc.c:7 other.a:9!null other.b:10 other.c:11 other.crdb_internal_mvcc_timestamp:12 + │ ├── left ordering: +5 + │ ├── right ordering: +9 │ ├── scan abc - │ │ ├── columns: abc.a:4!null abc.b:5 abc.c:6 - │ │ └── ordering: +4 + │ │ ├── columns: abc.a:5!null abc.b:6 abc.c:7 + │ │ └── ordering: +5 │ ├── scan other - │ │ ├── columns: other.a:7!null other.b:8 other.c:9 - │ │ └── ordering: +7 + │ │ ├── columns: other.a:9!null other.b:10 other.c:11 other.crdb_internal_mvcc_timestamp:12 + │ │ └── ordering: +9 │ └── filters (true) └── projections - ├── other.b:8 + 1 [as=b_new:10] - └── other.c:9 + 1 [as=c_new:11] + ├── other.b:10 + 1 [as=b_new:13] + └── other.c:11 + 1 [as=c_new:14] # Test when Update uses multiple tables. opt @@ -39,34 +39,36 @@ UPDATE abc SET b = other.b, c = other.c FROM new_abc AS other WHERE abc.a = othe ---- update abc ├── columns: - ├── fetch columns: abc.a:4 abc.b:5 abc.c:6 + ├── fetch columns: abc.a:5 abc.b:6 abc.c:7 ├── update-mapping: - │ ├── other.b:8 => abc.b:2 - │ └── other.c:9 => abc.c:3 + │ ├── other.b:10 => abc.b:2 + │ └── other.c:11 => abc.c:3 └── distinct-on - ├── columns: abc.a:4!null abc.b:5 abc.c:6 other.a:7!null other.b:8 other.c:9 rowid:10!null - ├── grouping columns: abc.a:4!null + ├── columns: abc.a:5!null abc.b:6 abc.c:7 other.a:9!null other.b:10 other.c:11 rowid:12!null other.crdb_internal_mvcc_timestamp:13 + ├── grouping columns: abc.a:5!null ├── inner-join (hash) - │ ├── columns: abc.a:4!null abc.b:5 abc.c:6 other.a:7!null other.b:8 other.c:9 rowid:10!null + │ ├── columns: abc.a:5!null abc.b:6 abc.c:7 other.a:9!null other.b:10 other.c:11 rowid:12!null other.crdb_internal_mvcc_timestamp:13 │ ├── scan abc - │ │ └── columns: abc.a:4!null abc.b:5 abc.c:6 + │ │ └── columns: abc.a:5!null abc.b:6 abc.c:7 │ ├── scan other - │ │ └── columns: other.a:7 other.b:8 other.c:9 rowid:10!null + │ │ └── columns: other.a:9 other.b:10 other.c:11 rowid:12!null other.crdb_internal_mvcc_timestamp:13 │ └── filters - │ └── abc.a:4 = other.a:7 + │ └── abc.a:5 = other.a:9 └── aggregations - ├── first-agg [as=abc.b:5] - │ └── abc.b:5 - ├── first-agg [as=abc.c:6] - │ └── abc.c:6 - ├── first-agg [as=other.a:7] - │ └── other.a:7 - ├── first-agg [as=other.b:8] - │ └── other.b:8 - ├── first-agg [as=other.c:9] - │ └── other.c:9 - └── first-agg [as=rowid:10] - └── rowid:10 + ├── first-agg [as=abc.b:6] + │ └── abc.b:6 + ├── first-agg [as=abc.c:7] + │ └── abc.c:7 + ├── first-agg [as=other.a:9] + │ └── other.a:9 + ├── first-agg [as=other.b:10] + │ └── other.b:10 + ├── first-agg [as=other.c:11] + │ └── other.c:11 + ├── first-agg [as=rowid:12] + │ └── rowid:12 + └── first-agg [as=other.crdb_internal_mvcc_timestamp:13] + └── other.crdb_internal_mvcc_timestamp:13 # Check if UPDATE FROM works well with RETURNING expressions that reference the FROM tables. opt @@ -81,54 +83,54 @@ RETURNING abc.a, abc.b AS new_b, old.b as old_b, abc.c as new_c, old.c as old_c ---- update abc - ├── columns: a:1!null new_b:2 old_b:8 new_c:3 old_c:9 - ├── fetch columns: abc.a:4 abc.b:5 abc.c:6 + ├── columns: a:1!null new_b:2 old_b:10 new_c:3 old_c:11 + ├── fetch columns: abc.a:5 abc.b:6 abc.c:7 ├── update-mapping: - │ ├── b_new:10 => abc.b:2 - │ └── c_new:11 => abc.c:3 + │ ├── b_new:13 => abc.b:2 + │ └── c_new:14 => abc.c:3 └── project - ├── columns: b_new:10 c_new:11 abc.a:4!null abc.b:5 abc.c:6 old.b:8 old.c:9 + ├── columns: b_new:13 c_new:14 abc.a:5!null abc.b:6 abc.c:7 old.b:10 old.c:11 ├── inner-join (merge) - │ ├── columns: abc.a:4!null abc.b:5 abc.c:6 old.a:7!null old.b:8 old.c:9 - │ ├── left ordering: +4 - │ ├── right ordering: +7 + │ ├── columns: abc.a:5!null abc.b:6 abc.c:7 old.a:9!null old.b:10 old.c:11 + │ ├── left ordering: +5 + │ ├── right ordering: +9 │ ├── scan abc - │ │ ├── columns: abc.a:4!null abc.b:5 abc.c:6 - │ │ └── ordering: +4 + │ │ ├── columns: abc.a:5!null abc.b:6 abc.c:7 + │ │ └── ordering: +5 │ ├── scan old - │ │ ├── columns: old.a:7!null old.b:8 old.c:9 - │ │ └── ordering: +7 + │ │ ├── columns: old.a:9!null old.b:10 old.c:11 + │ │ └── ordering: +9 │ └── filters (true) └── projections - ├── old.b:8 + 1 [as=b_new:10] - └── old.c:9 + 2 [as=c_new:11] + ├── old.b:10 + 1 [as=b_new:13] + └── old.c:11 + 2 [as=c_new:14] # Check if RETURNING * returns everything opt UPDATE abc SET b = old.b + 1, c = old.c + 2 FROM abc AS old WHERE abc.a = old.a RETURNING * ---- update abc - ├── columns: a:1!null b:2 c:3 a:7 b:8 c:9 - ├── fetch columns: abc.a:4 abc.b:5 abc.c:6 + ├── columns: a:1!null b:2 c:3 a:9 b:10 c:11 + ├── fetch columns: abc.a:5 abc.b:6 abc.c:7 ├── update-mapping: - │ ├── b_new:10 => abc.b:2 - │ └── c_new:11 => abc.c:3 + │ ├── b_new:13 => abc.b:2 + │ └── c_new:14 => abc.c:3 └── project - ├── columns: b_new:10 c_new:11 abc.a:4!null abc.b:5 abc.c:6 old.a:7!null old.b:8 old.c:9 + ├── columns: b_new:13 c_new:14 abc.a:5!null abc.b:6 abc.c:7 old.a:9!null old.b:10 old.c:11 ├── inner-join (merge) - │ ├── columns: abc.a:4!null abc.b:5 abc.c:6 old.a:7!null old.b:8 old.c:9 - │ ├── left ordering: +4 - │ ├── right ordering: +7 + │ ├── columns: abc.a:5!null abc.b:6 abc.c:7 old.a:9!null old.b:10 old.c:11 + │ ├── left ordering: +5 + │ ├── right ordering: +9 │ ├── scan abc - │ │ ├── columns: abc.a:4!null abc.b:5 abc.c:6 - │ │ └── ordering: +4 + │ │ ├── columns: abc.a:5!null abc.b:6 abc.c:7 + │ │ └── ordering: +5 │ ├── scan old - │ │ ├── columns: old.a:7!null old.b:8 old.c:9 - │ │ └── ordering: +7 + │ │ ├── columns: old.a:9!null old.b:10 old.c:11 + │ │ └── ordering: +9 │ └── filters (true) └── projections - ├── old.b:8 + 1 [as=b_new:10] - └── old.c:9 + 2 [as=c_new:11] + ├── old.b:10 + 1 [as=b_new:13] + └── old.c:11 + 2 [as=c_new:14] # Check if the joins are optimized (check if the filters are pushed down). opt @@ -136,24 +138,24 @@ UPDATE abc SET b = old.b + 1, c = old.c + 2 FROM abc AS old WHERE abc.a = old.a ---- update abc ├── columns: - ├── fetch columns: abc.a:4 abc.b:5 abc.c:6 + ├── fetch columns: abc.a:5 abc.b:6 abc.c:7 ├── update-mapping: - │ ├── b_new:10 => abc.b:2 - │ └── c_new:11 => abc.c:3 + │ ├── b_new:13 => abc.b:2 + │ └── c_new:14 => abc.c:3 └── project - ├── columns: b_new:10 c_new:11 abc.a:4!null abc.b:5 abc.c:6 old.a:7!null old.b:8 old.c:9 + ├── columns: b_new:13 c_new:14 abc.a:5!null abc.b:6 abc.c:7 old.a:9!null old.b:10 old.c:11 old.crdb_internal_mvcc_timestamp:12 ├── inner-join (cross) - │ ├── columns: abc.a:4!null abc.b:5 abc.c:6 old.a:7!null old.b:8 old.c:9 + │ ├── columns: abc.a:5!null abc.b:6 abc.c:7 old.a:9!null old.b:10 old.c:11 old.crdb_internal_mvcc_timestamp:12 │ ├── scan abc - │ │ ├── columns: abc.a:4!null abc.b:5 abc.c:6 - │ │ └── constraint: /4: [/2 - /2] + │ │ ├── columns: abc.a:5!null abc.b:6 abc.c:7 + │ │ └── constraint: /5: [/2 - /2] │ ├── scan old - │ │ ├── columns: old.a:7!null old.b:8 old.c:9 - │ │ └── constraint: /7: [/2 - /2] + │ │ ├── columns: old.a:9!null old.b:10 old.c:11 old.crdb_internal_mvcc_timestamp:12 + │ │ └── constraint: /9: [/2 - /2] │ └── filters (true) └── projections - ├── old.b:8 + 1 [as=b_new:10] - └── old.c:9 + 2 [as=c_new:11] + ├── old.b:10 + 1 [as=b_new:13] + └── old.c:11 + 2 [as=c_new:14] # Update values of table from values expression opt @@ -161,33 +163,33 @@ UPDATE abc SET b = other.b, c = other.c FROM (values (1, 2, 3), (2, 3, 4)) as ot ---- update abc ├── columns: - ├── fetch columns: a:4 b:5 c:6 + ├── fetch columns: a:5 b:6 c:7 ├── update-mapping: - │ ├── column2:8 => b:2 - │ └── column3:9 => c:3 + │ ├── column2:10 => b:2 + │ └── column3:11 => c:3 └── distinct-on - ├── columns: a:4!null b:5 c:6 column1:7!null column2:8!null column3:9!null - ├── grouping columns: a:4!null + ├── columns: a:5!null b:6 c:7 column1:9!null column2:10!null column3:11!null + ├── grouping columns: a:5!null ├── inner-join (lookup abc) - │ ├── columns: a:4!null b:5 c:6 column1:7!null column2:8!null column3:9!null - │ ├── key columns: [7] = [4] + │ ├── columns: a:5!null b:6 c:7 column1:9!null column2:10!null column3:11!null + │ ├── key columns: [9] = [5] │ ├── lookup columns are key │ ├── values - │ │ ├── columns: column1:7!null column2:8!null column3:9!null + │ │ ├── columns: column1:9!null column2:10!null column3:11!null │ │ ├── (1, 2, 3) │ │ └── (2, 3, 4) │ └── filters (true) └── aggregations - ├── first-agg [as=b:5] - │ └── b:5 - ├── first-agg [as=c:6] - │ └── c:6 - ├── first-agg [as=column1:7] - │ └── column1:7 - ├── first-agg [as=column2:8] - │ └── column2:8 - └── first-agg [as=column3:9] - └── column3:9 + ├── first-agg [as=b:6] + │ └── b:6 + ├── first-agg [as=c:7] + │ └── c:7 + ├── first-agg [as=column1:9] + │ └── column1:9 + ├── first-agg [as=column2:10] + │ └── column2:10 + └── first-agg [as=column3:11] + └── column3:11 # Check if UPDATE ... FROM works with multiple tables. exec-ddl @@ -203,44 +205,48 @@ UPDATE abc SET b = ab.b, c = ac.c FROM ab, ac WHERE abc.a = ab.a AND abc.a = ac. ---- update abc ├── columns: - ├── fetch columns: abc.a:4 abc.b:5 abc.c:6 + ├── fetch columns: abc.a:5 abc.b:6 abc.c:7 ├── update-mapping: - │ ├── ab.b:8 => abc.b:2 - │ └── ac.c:11 => abc.c:3 + │ ├── ab.b:10 => abc.b:2 + │ └── ac.c:14 => abc.c:3 └── distinct-on - ├── columns: abc.a:4!null abc.b:5 abc.c:6 ab.a:7!null ab.b:8 ab.rowid:9!null ac.a:10!null ac.c:11 ac.rowid:12!null - ├── grouping columns: abc.a:4!null + ├── columns: abc.a:5!null abc.b:6 abc.c:7 ab.a:9!null ab.b:10 ab.rowid:11!null ab.crdb_internal_mvcc_timestamp:12 ac.a:13!null ac.c:14 ac.rowid:15!null ac.crdb_internal_mvcc_timestamp:16 + ├── grouping columns: abc.a:5!null ├── inner-join (hash) - │ ├── columns: abc.a:4!null abc.b:5 abc.c:6 ab.a:7!null ab.b:8 ab.rowid:9!null ac.a:10!null ac.c:11 ac.rowid:12!null + │ ├── columns: abc.a:5!null abc.b:6 abc.c:7 ab.a:9!null ab.b:10 ab.rowid:11!null ab.crdb_internal_mvcc_timestamp:12 ac.a:13!null ac.c:14 ac.rowid:15!null ac.crdb_internal_mvcc_timestamp:16 │ ├── scan ab - │ │ └── columns: ab.a:7 ab.b:8 ab.rowid:9!null + │ │ └── columns: ab.a:9 ab.b:10 ab.rowid:11!null ab.crdb_internal_mvcc_timestamp:12 │ ├── inner-join (hash) - │ │ ├── columns: abc.a:4!null abc.b:5 abc.c:6 ac.a:10!null ac.c:11 ac.rowid:12!null + │ │ ├── columns: abc.a:5!null abc.b:6 abc.c:7 ac.a:13!null ac.c:14 ac.rowid:15!null ac.crdb_internal_mvcc_timestamp:16 │ │ ├── scan abc - │ │ │ └── columns: abc.a:4!null abc.b:5 abc.c:6 + │ │ │ └── columns: abc.a:5!null abc.b:6 abc.c:7 │ │ ├── scan ac - │ │ │ └── columns: ac.a:10 ac.c:11 ac.rowid:12!null + │ │ │ └── columns: ac.a:13 ac.c:14 ac.rowid:15!null ac.crdb_internal_mvcc_timestamp:16 │ │ └── filters - │ │ └── abc.a:4 = ac.a:10 + │ │ └── abc.a:5 = ac.a:13 │ └── filters - │ └── abc.a:4 = ab.a:7 + │ └── abc.a:5 = ab.a:9 └── aggregations - ├── first-agg [as=abc.b:5] - │ └── abc.b:5 - ├── first-agg [as=abc.c:6] - │ └── abc.c:6 - ├── first-agg [as=ab.a:7] - │ └── ab.a:7 - ├── first-agg [as=ab.b:8] - │ └── ab.b:8 - ├── first-agg [as=ab.rowid:9] - │ └── ab.rowid:9 - ├── first-agg [as=ac.a:10] - │ └── ac.a:10 - ├── first-agg [as=ac.c:11] - │ └── ac.c:11 - └── first-agg [as=ac.rowid:12] - └── ac.rowid:12 + ├── first-agg [as=abc.b:6] + │ └── abc.b:6 + ├── first-agg [as=abc.c:7] + │ └── abc.c:7 + ├── first-agg [as=ab.a:9] + │ └── ab.a:9 + ├── first-agg [as=ab.b:10] + │ └── ab.b:10 + ├── first-agg [as=ab.rowid:11] + │ └── ab.rowid:11 + ├── first-agg [as=ab.crdb_internal_mvcc_timestamp:12] + │ └── ab.crdb_internal_mvcc_timestamp:12 + ├── first-agg [as=ac.a:13] + │ └── ac.a:13 + ├── first-agg [as=ac.c:14] + │ └── ac.c:14 + ├── first-agg [as=ac.rowid:15] + │ └── ac.rowid:15 + └── first-agg [as=ac.crdb_internal_mvcc_timestamp:16] + └── ac.crdb_internal_mvcc_timestamp:16 # Make sure UPDATE ... FROM works with LATERAL. opt @@ -256,41 +262,41 @@ RETURNING * ---- update abc - ├── columns: a:1!null b:2 c:3 a:7 b:8 a:10 c:11 - ├── fetch columns: abc.a:4 abc.b:5 abc.c:6 + ├── columns: a:1!null b:2 c:3 a:9 b:10 a:13 c:14 + ├── fetch columns: abc.a:5 abc.b:6 abc.c:7 ├── update-mapping: - │ ├── ab.b:8 => abc.b:2 - │ └── ac.c:11 => abc.c:3 + │ ├── ab.b:10 => abc.b:2 + │ └── ac.c:14 => abc.c:3 └── distinct-on - ├── columns: abc.a:4!null abc.b:5 abc.c:6 ab.a:7!null ab.b:8 ac.a:10!null ac.c:11 - ├── grouping columns: abc.a:4!null + ├── columns: abc.a:5!null abc.b:6 abc.c:7 ab.a:9!null ab.b:10 ac.a:13!null ac.c:14 + ├── grouping columns: abc.a:5!null ├── inner-join (hash) - │ ├── columns: abc.a:4!null abc.b:5 abc.c:6 ab.a:7!null ab.b:8 ac.a:10!null ac.c:11 + │ ├── columns: abc.a:5!null abc.b:6 abc.c:7 ab.a:9!null ab.b:10 ac.a:13!null ac.c:14 │ ├── scan ab - │ │ └── columns: ab.a:7 ab.b:8 + │ │ └── columns: ab.a:9 ab.b:10 │ ├── inner-join (hash) - │ │ ├── columns: abc.a:4!null abc.b:5 abc.c:6 ac.a:10!null ac.c:11 + │ │ ├── columns: abc.a:5!null abc.b:6 abc.c:7 ac.a:13!null ac.c:14 │ │ ├── scan abc - │ │ │ └── columns: abc.a:4!null abc.b:5 abc.c:6 + │ │ │ └── columns: abc.a:5!null abc.b:6 abc.c:7 │ │ ├── scan ac - │ │ │ └── columns: ac.a:10 ac.c:11 + │ │ │ └── columns: ac.a:13 ac.c:14 │ │ └── filters - │ │ └── abc.a:4 = ac.a:10 + │ │ └── abc.a:5 = ac.a:13 │ └── filters - │ └── ab.a:7 = ac.a:10 + │ └── ab.a:9 = ac.a:13 └── aggregations - ├── first-agg [as=abc.b:5] - │ └── abc.b:5 - ├── first-agg [as=abc.c:6] - │ └── abc.c:6 - ├── first-agg [as=ab.a:7] - │ └── ab.a:7 - ├── first-agg [as=ab.b:8] - │ └── ab.b:8 - ├── first-agg [as=ac.a:10] - │ └── ac.a:10 - └── first-agg [as=ac.c:11] - └── ac.c:11 + ├── first-agg [as=abc.b:6] + │ └── abc.b:6 + ├── first-agg [as=abc.c:7] + │ └── abc.c:7 + ├── first-agg [as=ab.a:9] + │ └── ab.a:9 + ├── first-agg [as=ab.b:10] + │ └── ab.b:10 + ├── first-agg [as=ac.a:13] + │ └── ac.a:13 + └── first-agg [as=ac.c:14] + └── ac.c:14 # Make sure UPDATE ... FROM can return hidden columns. opt @@ -305,42 +311,42 @@ RETURNING *, ab.rowid, ac.rowid ---- update abc - ├── columns: a:1!null b:2 c:3 a:7 b:8 a:10 c:11 rowid:9 rowid:12 - ├── fetch columns: abc.a:4 abc.b:5 abc.c:6 + ├── columns: a:1!null b:2 c:3 a:9 b:10 a:13 c:14 rowid:11 rowid:15 + ├── fetch columns: abc.a:5 abc.b:6 abc.c:7 ├── update-mapping: - │ ├── ab.b:8 => abc.b:2 - │ └── ac.c:11 => abc.c:3 + │ ├── ab.b:10 => abc.b:2 + │ └── ac.c:14 => abc.c:3 └── distinct-on - ├── columns: abc.a:4!null abc.b:5 abc.c:6 ab.a:7!null ab.b:8 ab.rowid:9!null ac.a:10!null ac.c:11 ac.rowid:12!null - ├── grouping columns: abc.a:4!null + ├── columns: abc.a:5!null abc.b:6 abc.c:7 ab.a:9!null ab.b:10 ab.rowid:11!null ac.a:13!null ac.c:14 ac.rowid:15!null + ├── grouping columns: abc.a:5!null ├── inner-join (hash) - │ ├── columns: abc.a:4!null abc.b:5 abc.c:6 ab.a:7!null ab.b:8 ab.rowid:9!null ac.a:10!null ac.c:11 ac.rowid:12!null + │ ├── columns: abc.a:5!null abc.b:6 abc.c:7 ab.a:9!null ab.b:10 ab.rowid:11!null ac.a:13!null ac.c:14 ac.rowid:15!null │ ├── scan ab - │ │ └── columns: ab.a:7 ab.b:8 ab.rowid:9!null + │ │ └── columns: ab.a:9 ab.b:10 ab.rowid:11!null │ ├── inner-join (hash) - │ │ ├── columns: abc.a:4!null abc.b:5 abc.c:6 ac.a:10!null ac.c:11 ac.rowid:12!null + │ │ ├── columns: abc.a:5!null abc.b:6 abc.c:7 ac.a:13!null ac.c:14 ac.rowid:15!null │ │ ├── scan abc - │ │ │ └── columns: abc.a:4!null abc.b:5 abc.c:6 + │ │ │ └── columns: abc.a:5!null abc.b:6 abc.c:7 │ │ ├── scan ac - │ │ │ └── columns: ac.a:10 ac.c:11 ac.rowid:12!null + │ │ │ └── columns: ac.a:13 ac.c:14 ac.rowid:15!null │ │ └── filters - │ │ └── abc.a:4 = ac.a:10 + │ │ └── abc.a:5 = ac.a:13 │ └── filters - │ └── abc.a:4 = ab.a:7 + │ └── abc.a:5 = ab.a:9 └── aggregations - ├── first-agg [as=abc.b:5] - │ └── abc.b:5 - ├── first-agg [as=abc.c:6] - │ └── abc.c:6 - ├── first-agg [as=ab.a:7] - │ └── ab.a:7 - ├── first-agg [as=ab.b:8] - │ └── ab.b:8 - ├── first-agg [as=ab.rowid:9] - │ └── ab.rowid:9 - ├── first-agg [as=ac.a:10] - │ └── ac.a:10 - ├── first-agg [as=ac.c:11] - │ └── ac.c:11 - └── first-agg [as=ac.rowid:12] - └── ac.rowid:12 + ├── first-agg [as=abc.b:6] + │ └── abc.b:6 + ├── first-agg [as=abc.c:7] + │ └── abc.c:7 + ├── first-agg [as=ab.a:9] + │ └── ab.a:9 + ├── first-agg [as=ab.b:10] + │ └── ab.b:10 + ├── first-agg [as=ab.rowid:11] + │ └── ab.rowid:11 + ├── first-agg [as=ac.a:13] + │ └── ac.a:13 + ├── first-agg [as=ac.c:14] + │ └── ac.c:14 + └── first-agg [as=ac.rowid:15] + └── ac.rowid:15 diff --git a/pkg/sql/opt/optbuilder/testdata/upsert b/pkg/sql/opt/optbuilder/testdata/upsert index 2fa6dbc09996..47e691c96c49 100644 --- a/pkg/sql/opt/optbuilder/testdata/upsert +++ b/pkg/sql/opt/optbuilder/testdata/upsert @@ -78,62 +78,62 @@ UPDATE SET a=5 ---- upsert abc ├── columns: - ├── canary column: 13 - ├── fetch columns: a:10 b:11 c:12 rowid:13 + ├── canary column: 15 + ├── fetch columns: a:12 b:13 c:14 rowid:15 ├── insert-mapping: - │ ├── x:5 => a:1 - │ ├── y:6 => b:2 - │ ├── column9:9 => c:3 - │ └── column8:8 => rowid:4 + │ ├── x:6 => a:1 + │ ├── y:7 => b:2 + │ ├── column11:11 => c:3 + │ └── column10:10 => rowid:4 ├── update-mapping: - │ ├── upsert_a:16 => a:1 - │ └── upsert_c:18 => c:3 + │ ├── upsert_a:19 => a:1 + │ └── upsert_c:21 => c:3 └── project - ├── columns: upsert_a:16!null upsert_b:17 upsert_c:18 upsert_rowid:19 x:5!null y:6 column8:8 column9:9 a:10 b:11 c:12 rowid:13 a_new:14!null column15:15 + ├── columns: upsert_a:19!null upsert_b:20 upsert_c:21 upsert_rowid:22 x:6!null y:7 column10:10 column11:11 a:12 b:13 c:14 rowid:15 abc.crdb_internal_mvcc_timestamp:16 a_new:17!null column18:18 ├── project - │ ├── columns: column15:15 x:5!null y:6 column8:8 column9:9 a:10 b:11 c:12 rowid:13 a_new:14!null + │ ├── columns: column18:18 x:6!null y:7 column10:10 column11:11 a:12 b:13 c:14 rowid:15 abc.crdb_internal_mvcc_timestamp:16 a_new:17!null │ ├── project - │ │ ├── columns: a_new:14!null x:5!null y:6 column8:8 column9:9 a:10 b:11 c:12 rowid:13 + │ │ ├── columns: a_new:17!null x:6!null y:7 column10:10 column11:11 a:12 b:13 c:14 rowid:15 abc.crdb_internal_mvcc_timestamp:16 │ │ ├── left-join (hash) - │ │ │ ├── columns: x:5!null y:6 column8:8 column9:9 a:10 b:11 c:12 rowid:13 + │ │ │ ├── columns: x:6!null y:7 column10:10 column11:11 a:12 b:13 c:14 rowid:15 abc.crdb_internal_mvcc_timestamp:16 │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ ├── columns: x:5!null y:6 column8:8 column9:9 - │ │ │ │ ├── grouping columns: x:5!null + │ │ │ │ ├── columns: x:6!null y:7 column10:10 column11:11 + │ │ │ │ ├── grouping columns: x:6!null │ │ │ │ ├── project - │ │ │ │ │ ├── columns: column9:9 x:5!null y:6 column8:8 + │ │ │ │ │ ├── columns: column11:11 x:6!null y:7 column10:10 │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: column8:8 x:5!null y:6 + │ │ │ │ │ │ ├── columns: column10:10 x:6!null y:7 │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ ├── columns: x:5!null y:6 + │ │ │ │ │ │ │ ├── columns: x:6!null y:7 │ │ │ │ │ │ │ └── scan xyz - │ │ │ │ │ │ │ └── columns: x:5!null y:6 z:7 + │ │ │ │ │ │ │ └── columns: x:6!null y:7 z:8 xyz.crdb_internal_mvcc_timestamp:9 │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── unique_rowid() [as=column8:8] + │ │ │ │ │ │ └── unique_rowid() [as=column10:10] │ │ │ │ │ └── projections - │ │ │ │ │ └── y:6 + 1 [as=column9:9] + │ │ │ │ │ └── y:7 + 1 [as=column11:11] │ │ │ │ └── aggregations - │ │ │ │ ├── first-agg [as=y:6] - │ │ │ │ │ └── y:6 - │ │ │ │ ├── first-agg [as=column8:8] - │ │ │ │ │ └── column8:8 - │ │ │ │ └── first-agg [as=column9:9] - │ │ │ │ └── column9:9 + │ │ │ │ ├── first-agg [as=y:7] + │ │ │ │ │ └── y:7 + │ │ │ │ ├── first-agg [as=column10:10] + │ │ │ │ │ └── column10:10 + │ │ │ │ └── first-agg [as=column11:11] + │ │ │ │ └── column11:11 │ │ │ ├── scan abc - │ │ │ │ ├── columns: a:10!null b:11 c:12 rowid:13!null + │ │ │ │ ├── columns: a:12!null b:13 c:14 rowid:15!null abc.crdb_internal_mvcc_timestamp:16 │ │ │ │ └── computed column expressions - │ │ │ │ └── c:12 - │ │ │ │ └── b:11 + 1 + │ │ │ │ └── c:14 + │ │ │ │ └── b:13 + 1 │ │ │ └── filters - │ │ │ └── x:5 = a:10 + │ │ │ └── x:6 = a:12 │ │ └── projections - │ │ └── 5 [as=a_new:14] + │ │ └── 5 [as=a_new:17] │ └── projections - │ └── b:11 + 1 [as=column15:15] + │ └── b:13 + 1 [as=column18:18] └── projections - ├── CASE WHEN rowid:13 IS NULL THEN x:5 ELSE a_new:14 END [as=upsert_a:16] - ├── CASE WHEN rowid:13 IS NULL THEN y:6 ELSE b:11 END [as=upsert_b:17] - ├── CASE WHEN rowid:13 IS NULL THEN column9:9 ELSE column15:15 END [as=upsert_c:18] - └── CASE WHEN rowid:13 IS NULL THEN column8:8 ELSE rowid:13 END [as=upsert_rowid:19] + ├── CASE WHEN rowid:15 IS NULL THEN x:6 ELSE a_new:17 END [as=upsert_a:19] + ├── CASE WHEN rowid:15 IS NULL THEN y:7 ELSE b:13 END [as=upsert_b:20] + ├── CASE WHEN rowid:15 IS NULL THEN column11:11 ELSE column18:18 END [as=upsert_c:21] + └── CASE WHEN rowid:15 IS NULL THEN column10:10 ELSE rowid:15 END [as=upsert_rowid:22] # Set all columns, multi-column conflict. build @@ -147,64 +147,66 @@ project ├── columns: a:1!null b:2 c:3 └── upsert abc ├── columns: a:1!null b:2 c:3 rowid:4!null - ├── canary column: 12 - ├── fetch columns: a:9 b:10 c:11 rowid:12 + ├── canary column: 14 + ├── fetch columns: a:11 b:12 c:13 rowid:14 ├── insert-mapping: - │ ├── x:5 => a:1 - │ ├── y:6 => b:2 - │ ├── column8:8 => c:3 - │ └── z:7 => rowid:4 + │ ├── x:6 => a:1 + │ ├── y:7 => b:2 + │ ├── column10:10 => c:3 + │ └── z:8 => rowid:4 ├── update-mapping: - │ ├── upsert_a:17 => a:1 - │ ├── upsert_b:18 => b:2 - │ ├── upsert_c:19 => c:3 - │ └── upsert_rowid:20 => rowid:4 + │ ├── upsert_a:20 => a:1 + │ ├── upsert_b:21 => b:2 + │ ├── upsert_c:22 => c:3 + │ └── upsert_rowid:23 => rowid:4 ├── return-mapping: - │ ├── upsert_a:17 => a:1 - │ ├── upsert_b:18 => b:2 - │ ├── upsert_c:19 => c:3 - │ └── upsert_rowid:20 => rowid:4 + │ ├── upsert_a:20 => a:1 + │ ├── upsert_b:21 => b:2 + │ ├── upsert_c:22 => c:3 + │ └── upsert_rowid:23 => rowid:4 └── project - ├── columns: upsert_a:17!null upsert_b:18 upsert_c:19 upsert_rowid:20 x:5!null y:6 z:7 column8:8 a:9 b:10 c:11 rowid:12 a_new:13!null b_new:14!null rowid_new:15!null column16:16!null + ├── columns: upsert_a:20!null upsert_b:21 upsert_c:22 upsert_rowid:23 x:6!null y:7 z:8 column10:10 a:11 b:12 c:13 rowid:14 abc.crdb_internal_mvcc_timestamp:15 a_new:16!null b_new:17!null rowid_new:18!null column19:19!null ├── project - │ ├── columns: column16:16!null x:5!null y:6 z:7 column8:8 a:9 b:10 c:11 rowid:12 a_new:13!null b_new:14!null rowid_new:15!null + │ ├── columns: column19:19!null x:6!null y:7 z:8 column10:10 a:11 b:12 c:13 rowid:14 abc.crdb_internal_mvcc_timestamp:15 a_new:16!null b_new:17!null rowid_new:18!null │ ├── project - │ │ ├── columns: a_new:13!null b_new:14!null rowid_new:15!null x:5!null y:6 z:7 column8:8 a:9 b:10 c:11 rowid:12 + │ │ ├── columns: a_new:16!null b_new:17!null rowid_new:18!null x:6!null y:7 z:8 column10:10 a:11 b:12 c:13 rowid:14 abc.crdb_internal_mvcc_timestamp:15 │ │ ├── left-join (hash) - │ │ │ ├── columns: x:5!null y:6 z:7 column8:8 a:9 b:10 c:11 rowid:12 + │ │ │ ├── columns: x:6!null y:7 z:8 column10:10 a:11 b:12 c:13 rowid:14 abc.crdb_internal_mvcc_timestamp:15 │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ ├── columns: x:5!null y:6 z:7 column8:8 - │ │ │ │ ├── grouping columns: y:6 column8:8 + │ │ │ │ ├── columns: x:6!null y:7 z:8 column10:10 + │ │ │ │ ├── grouping columns: y:7 column10:10 │ │ │ │ ├── project - │ │ │ │ │ ├── columns: column8:8 x:5!null y:6 z:7 - │ │ │ │ │ ├── scan xyz - │ │ │ │ │ │ └── columns: x:5!null y:6 z:7 + │ │ │ │ │ ├── columns: column10:10 x:6!null y:7 z:8 + │ │ │ │ │ ├── project + │ │ │ │ │ │ ├── columns: x:6!null y:7 z:8 + │ │ │ │ │ │ └── scan xyz + │ │ │ │ │ │ └── columns: x:6!null y:7 z:8 xyz.crdb_internal_mvcc_timestamp:9 │ │ │ │ │ └── projections - │ │ │ │ │ └── y:6 + 1 [as=column8:8] + │ │ │ │ │ └── y:7 + 1 [as=column10:10] │ │ │ │ └── aggregations - │ │ │ │ ├── first-agg [as=x:5] - │ │ │ │ │ └── x:5 - │ │ │ │ └── first-agg [as=z:7] - │ │ │ │ └── z:7 + │ │ │ │ ├── first-agg [as=x:6] + │ │ │ │ │ └── x:6 + │ │ │ │ └── first-agg [as=z:8] + │ │ │ │ └── z:8 │ │ │ ├── scan abc - │ │ │ │ ├── columns: a:9!null b:10 c:11 rowid:12!null + │ │ │ │ ├── columns: a:11!null b:12 c:13 rowid:14!null abc.crdb_internal_mvcc_timestamp:15 │ │ │ │ └── computed column expressions - │ │ │ │ └── c:11 - │ │ │ │ └── b:10 + 1 + │ │ │ │ └── c:13 + │ │ │ │ └── b:12 + 1 │ │ │ └── filters - │ │ │ ├── y:6 = b:10 - │ │ │ └── column8:8 = c:11 + │ │ │ ├── y:7 = b:12 + │ │ │ └── column10:10 = c:13 │ │ └── projections - │ │ ├── 1 [as=a_new:13] - │ │ ├── 2 [as=b_new:14] - │ │ └── 3 [as=rowid_new:15] + │ │ ├── 1 [as=a_new:16] + │ │ ├── 2 [as=b_new:17] + │ │ └── 3 [as=rowid_new:18] │ └── projections - │ └── b_new:14 + 1 [as=column16:16] + │ └── b_new:17 + 1 [as=column19:19] └── projections - ├── CASE WHEN rowid:12 IS NULL THEN x:5 ELSE a_new:13 END [as=upsert_a:17] - ├── CASE WHEN rowid:12 IS NULL THEN y:6 ELSE b_new:14 END [as=upsert_b:18] - ├── CASE WHEN rowid:12 IS NULL THEN column8:8 ELSE column16:16 END [as=upsert_c:19] - └── CASE WHEN rowid:12 IS NULL THEN z:7 ELSE rowid_new:15 END [as=upsert_rowid:20] + ├── CASE WHEN rowid:14 IS NULL THEN x:6 ELSE a_new:16 END [as=upsert_a:20] + ├── CASE WHEN rowid:14 IS NULL THEN y:7 ELSE b_new:17 END [as=upsert_b:21] + ├── CASE WHEN rowid:14 IS NULL THEN column10:10 ELSE column19:19 END [as=upsert_c:22] + └── CASE WHEN rowid:14 IS NULL THEN z:8 ELSE rowid_new:18 END [as=upsert_rowid:23] # UPDATE + WHERE clause. build @@ -216,66 +218,66 @@ WHERE abc.a>0 ---- upsert abc ├── columns: - ├── canary column: 13 - ├── fetch columns: a:10 b:11 c:12 rowid:13 + ├── canary column: 15 + ├── fetch columns: a:12 b:13 c:14 rowid:15 ├── insert-mapping: - │ ├── x:5 => a:1 - │ ├── y:6 => b:2 - │ ├── column9:9 => c:3 - │ └── column8:8 => rowid:4 + │ ├── x:6 => a:1 + │ ├── y:7 => b:2 + │ ├── column11:11 => c:3 + │ └── column10:10 => rowid:4 ├── update-mapping: - │ ├── upsert_b:17 => b:2 - │ └── upsert_c:18 => c:3 + │ ├── upsert_b:20 => b:2 + │ └── upsert_c:21 => c:3 └── project - ├── columns: upsert_a:16 upsert_b:17 upsert_c:18 upsert_rowid:19 x:5!null y:6 column8:8 column9:9 a:10 b:11 c:12 rowid:13 b_new:14!null column15:15!null + ├── columns: upsert_a:19 upsert_b:20 upsert_c:21 upsert_rowid:22 x:6!null y:7 column10:10 column11:11 a:12 b:13 c:14 rowid:15 abc.crdb_internal_mvcc_timestamp:16 b_new:17!null column18:18!null ├── project - │ ├── columns: column15:15!null x:5!null y:6 column8:8 column9:9 a:10 b:11 c:12 rowid:13 b_new:14!null + │ ├── columns: column18:18!null x:6!null y:7 column10:10 column11:11 a:12 b:13 c:14 rowid:15 abc.crdb_internal_mvcc_timestamp:16 b_new:17!null │ ├── project - │ │ ├── columns: b_new:14!null x:5!null y:6 column8:8 column9:9 a:10 b:11 c:12 rowid:13 + │ │ ├── columns: b_new:17!null x:6!null y:7 column10:10 column11:11 a:12 b:13 c:14 rowid:15 abc.crdb_internal_mvcc_timestamp:16 │ │ ├── select - │ │ │ ├── columns: x:5!null y:6 column8:8 column9:9 a:10 b:11 c:12 rowid:13 + │ │ │ ├── columns: x:6!null y:7 column10:10 column11:11 a:12 b:13 c:14 rowid:15 abc.crdb_internal_mvcc_timestamp:16 │ │ │ ├── left-join (hash) - │ │ │ │ ├── columns: x:5!null y:6 column8:8 column9:9 a:10 b:11 c:12 rowid:13 + │ │ │ │ ├── columns: x:6!null y:7 column10:10 column11:11 a:12 b:13 c:14 rowid:15 abc.crdb_internal_mvcc_timestamp:16 │ │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ │ ├── columns: x:5!null y:6 column8:8 column9:9 - │ │ │ │ │ ├── grouping columns: x:5!null + │ │ │ │ │ ├── columns: x:6!null y:7 column10:10 column11:11 + │ │ │ │ │ ├── grouping columns: x:6!null │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: column9:9 x:5!null y:6 column8:8 + │ │ │ │ │ │ ├── columns: column11:11 x:6!null y:7 column10:10 │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ ├── columns: column8:8 x:5!null y:6 + │ │ │ │ │ │ │ ├── columns: column10:10 x:6!null y:7 │ │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ │ ├── columns: x:5!null y:6 + │ │ │ │ │ │ │ │ ├── columns: x:6!null y:7 │ │ │ │ │ │ │ │ └── scan xyz - │ │ │ │ │ │ │ │ └── columns: x:5!null y:6 z:7 + │ │ │ │ │ │ │ │ └── columns: x:6!null y:7 z:8 xyz.crdb_internal_mvcc_timestamp:9 │ │ │ │ │ │ │ └── projections - │ │ │ │ │ │ │ └── unique_rowid() [as=column8:8] + │ │ │ │ │ │ │ └── unique_rowid() [as=column10:10] │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── y:6 + 1 [as=column9:9] + │ │ │ │ │ │ └── y:7 + 1 [as=column11:11] │ │ │ │ │ └── aggregations - │ │ │ │ │ ├── first-agg [as=y:6] - │ │ │ │ │ │ └── y:6 - │ │ │ │ │ ├── first-agg [as=column8:8] - │ │ │ │ │ │ └── column8:8 - │ │ │ │ │ └── first-agg [as=column9:9] - │ │ │ │ │ └── column9:9 + │ │ │ │ │ ├── first-agg [as=y:7] + │ │ │ │ │ │ └── y:7 + │ │ │ │ │ ├── first-agg [as=column10:10] + │ │ │ │ │ │ └── column10:10 + │ │ │ │ │ └── first-agg [as=column11:11] + │ │ │ │ │ └── column11:11 │ │ │ │ ├── scan abc - │ │ │ │ │ ├── columns: a:10!null b:11 c:12 rowid:13!null + │ │ │ │ │ ├── columns: a:12!null b:13 c:14 rowid:15!null abc.crdb_internal_mvcc_timestamp:16 │ │ │ │ │ └── computed column expressions - │ │ │ │ │ └── c:12 - │ │ │ │ │ └── b:11 + 1 + │ │ │ │ │ └── c:14 + │ │ │ │ │ └── b:13 + 1 │ │ │ │ └── filters - │ │ │ │ └── x:5 = a:10 + │ │ │ │ └── x:6 = a:12 │ │ │ └── filters - │ │ │ └── (rowid:13 IS NULL) OR (a:10 > 0) + │ │ │ └── (rowid:15 IS NULL) OR (a:12 > 0) │ │ └── projections - │ │ └── 10 [as=b_new:14] + │ │ └── 10 [as=b_new:17] │ └── projections - │ └── b_new:14 + 1 [as=column15:15] + │ └── b_new:17 + 1 [as=column18:18] └── projections - ├── CASE WHEN rowid:13 IS NULL THEN x:5 ELSE a:10 END [as=upsert_a:16] - ├── CASE WHEN rowid:13 IS NULL THEN y:6 ELSE b_new:14 END [as=upsert_b:17] - ├── CASE WHEN rowid:13 IS NULL THEN column9:9 ELSE column15:15 END [as=upsert_c:18] - └── CASE WHEN rowid:13 IS NULL THEN column8:8 ELSE rowid:13 END [as=upsert_rowid:19] + ├── CASE WHEN rowid:15 IS NULL THEN x:6 ELSE a:12 END [as=upsert_a:19] + ├── CASE WHEN rowid:15 IS NULL THEN y:7 ELSE b_new:17 END [as=upsert_b:20] + ├── CASE WHEN rowid:15 IS NULL THEN column11:11 ELSE column18:18 END [as=upsert_c:21] + └── CASE WHEN rowid:15 IS NULL THEN column10:10 ELSE rowid:15 END [as=upsert_rowid:22] # Use RETURNING INSERT..ON CONFLICT as a FROM clause. build @@ -284,81 +286,81 @@ FROM [INSERT INTO abc (a, b) VALUES (1,2), (3,4) ON CONFLICT (a) DO UPDATE SET b ORDER BY a, b DESC ---- sort - ├── columns: a:19!null b:20!null c:21!null - ├── ordering: +19,-20 + ├── columns: a:21!null b:22!null c:23!null + ├── ordering: +21,-22 └── with &1 - ├── columns: a:19!null b:20!null c:21!null + ├── columns: a:21!null b:22!null c:23!null ├── project │ ├── columns: abc.a:1!null abc.b:2!null abc.c:3!null │ └── upsert abc │ ├── columns: abc.a:1!null abc.b:2!null abc.c:3!null rowid:4!null - │ ├── canary column: 12 - │ ├── fetch columns: abc.a:9 abc.b:10 abc.c:11 rowid:12 + │ ├── canary column: 13 + │ ├── fetch columns: abc.a:10 abc.b:11 abc.c:12 rowid:13 │ ├── insert-mapping: - │ │ ├── column1:5 => abc.a:1 - │ │ ├── column2:6 => abc.b:2 - │ │ ├── column8:8 => abc.c:3 - │ │ └── column7:7 => rowid:4 + │ │ ├── column1:6 => abc.a:1 + │ │ ├── column2:7 => abc.b:2 + │ │ ├── column9:9 => abc.c:3 + │ │ └── column8:8 => rowid:4 │ ├── update-mapping: - │ │ ├── upsert_b:16 => abc.b:2 - │ │ └── upsert_c:17 => abc.c:3 + │ │ ├── upsert_b:18 => abc.b:2 + │ │ └── upsert_c:19 => abc.c:3 │ ├── return-mapping: - │ │ ├── upsert_a:15 => abc.a:1 - │ │ ├── upsert_b:16 => abc.b:2 - │ │ ├── upsert_c:17 => abc.c:3 - │ │ └── upsert_rowid:18 => rowid:4 + │ │ ├── upsert_a:17 => abc.a:1 + │ │ ├── upsert_b:18 => abc.b:2 + │ │ ├── upsert_c:19 => abc.c:3 + │ │ └── upsert_rowid:20 => rowid:4 │ └── project - │ ├── columns: upsert_a:15 upsert_b:16!null upsert_c:17!null upsert_rowid:18 column1:5!null column2:6!null column7:7 column8:8!null abc.a:9 abc.b:10 abc.c:11 rowid:12 b_new:13!null column14:14!null + │ ├── columns: upsert_a:17 upsert_b:18!null upsert_c:19!null upsert_rowid:20 column1:6!null column2:7!null column8:8 column9:9!null abc.a:10 abc.b:11 abc.c:12 rowid:13 crdb_internal_mvcc_timestamp:14 b_new:15!null column16:16!null │ ├── project - │ │ ├── columns: column14:14!null column1:5!null column2:6!null column7:7 column8:8!null abc.a:9 abc.b:10 abc.c:11 rowid:12 b_new:13!null + │ │ ├── columns: column16:16!null column1:6!null column2:7!null column8:8 column9:9!null abc.a:10 abc.b:11 abc.c:12 rowid:13 crdb_internal_mvcc_timestamp:14 b_new:15!null │ │ ├── project - │ │ │ ├── columns: b_new:13!null column1:5!null column2:6!null column7:7 column8:8!null abc.a:9 abc.b:10 abc.c:11 rowid:12 + │ │ │ ├── columns: b_new:15!null column1:6!null column2:7!null column8:8 column9:9!null abc.a:10 abc.b:11 abc.c:12 rowid:13 crdb_internal_mvcc_timestamp:14 │ │ │ ├── left-join (hash) - │ │ │ │ ├── columns: column1:5!null column2:6!null column7:7 column8:8!null abc.a:9 abc.b:10 abc.c:11 rowid:12 + │ │ │ │ ├── columns: column1:6!null column2:7!null column8:8 column9:9!null abc.a:10 abc.b:11 abc.c:12 rowid:13 crdb_internal_mvcc_timestamp:14 │ │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ │ ├── columns: column1:5!null column2:6!null column7:7 column8:8!null - │ │ │ │ │ ├── grouping columns: column1:5!null + │ │ │ │ │ ├── columns: column1:6!null column2:7!null column8:8 column9:9!null + │ │ │ │ │ ├── grouping columns: column1:6!null │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: column8:8!null column1:5!null column2:6!null column7:7 + │ │ │ │ │ │ ├── columns: column9:9!null column1:6!null column2:7!null column8:8 │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ ├── columns: column7:7 column1:5!null column2:6!null + │ │ │ │ │ │ │ ├── columns: column8:8 column1:6!null column2:7!null │ │ │ │ │ │ │ ├── values - │ │ │ │ │ │ │ │ ├── columns: column1:5!null column2:6!null + │ │ │ │ │ │ │ │ ├── columns: column1:6!null column2:7!null │ │ │ │ │ │ │ │ ├── (1, 2) │ │ │ │ │ │ │ │ └── (3, 4) │ │ │ │ │ │ │ └── projections - │ │ │ │ │ │ │ └── unique_rowid() [as=column7:7] + │ │ │ │ │ │ │ └── unique_rowid() [as=column8:8] │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── column2:6 + 1 [as=column8:8] + │ │ │ │ │ │ └── column2:7 + 1 [as=column9:9] │ │ │ │ │ └── aggregations - │ │ │ │ │ ├── first-agg [as=column2:6] - │ │ │ │ │ │ └── column2:6 - │ │ │ │ │ ├── first-agg [as=column7:7] - │ │ │ │ │ │ └── column7:7 - │ │ │ │ │ └── first-agg [as=column8:8] - │ │ │ │ │ └── column8:8 + │ │ │ │ │ ├── first-agg [as=column2:7] + │ │ │ │ │ │ └── column2:7 + │ │ │ │ │ ├── first-agg [as=column8:8] + │ │ │ │ │ │ └── column8:8 + │ │ │ │ │ └── first-agg [as=column9:9] + │ │ │ │ │ └── column9:9 │ │ │ │ ├── scan abc - │ │ │ │ │ ├── columns: abc.a:9!null abc.b:10 abc.c:11 rowid:12!null + │ │ │ │ │ ├── columns: abc.a:10!null abc.b:11 abc.c:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ │ │ │ └── computed column expressions - │ │ │ │ │ └── abc.c:11 - │ │ │ │ │ └── abc.b:10 + 1 + │ │ │ │ │ └── abc.c:12 + │ │ │ │ │ └── abc.b:11 + 1 │ │ │ │ └── filters - │ │ │ │ └── column1:5 = abc.a:9 + │ │ │ │ └── column1:6 = abc.a:10 │ │ │ └── projections - │ │ │ └── 1 [as=b_new:13] + │ │ │ └── 1 [as=b_new:15] │ │ └── projections - │ │ └── b_new:13 + 1 [as=column14:14] + │ │ └── b_new:15 + 1 [as=column16:16] │ └── projections - │ ├── CASE WHEN rowid:12 IS NULL THEN column1:5 ELSE abc.a:9 END [as=upsert_a:15] - │ ├── CASE WHEN rowid:12 IS NULL THEN column2:6 ELSE b_new:13 END [as=upsert_b:16] - │ ├── CASE WHEN rowid:12 IS NULL THEN column8:8 ELSE column14:14 END [as=upsert_c:17] - │ └── CASE WHEN rowid:12 IS NULL THEN column7:7 ELSE rowid:12 END [as=upsert_rowid:18] + │ ├── CASE WHEN rowid:13 IS NULL THEN column1:6 ELSE abc.a:10 END [as=upsert_a:17] + │ ├── CASE WHEN rowid:13 IS NULL THEN column2:7 ELSE b_new:15 END [as=upsert_b:18] + │ ├── CASE WHEN rowid:13 IS NULL THEN column9:9 ELSE column16:16 END [as=upsert_c:19] + │ └── CASE WHEN rowid:13 IS NULL THEN column8:8 ELSE rowid:13 END [as=upsert_rowid:20] └── with-scan &1 - ├── columns: a:19!null b:20!null c:21!null + ├── columns: a:21!null b:22!null c:23!null └── mapping: - ├── abc.a:1 => a:19 - ├── abc.b:2 => b:20 - └── abc.c:3 => c:21 + ├── abc.a:1 => a:21 + ├── abc.b:2 => b:22 + └── abc.c:3 => c:23 # Use table alias. build @@ -369,61 +371,61 @@ UPDATE SET a=tab.a*excluded.a ---- upsert tab ├── columns: - ├── canary column: 12 - ├── fetch columns: a:9 b:10 c:11 rowid:12 + ├── canary column: 13 + ├── fetch columns: a:10 b:11 c:12 rowid:13 ├── insert-mapping: - │ ├── column1:5 => a:1 - │ ├── column2:6 => b:2 - │ ├── column8:8 => c:3 - │ └── column7:7 => rowid:4 + │ ├── column1:6 => a:1 + │ ├── column2:7 => b:2 + │ ├── column9:9 => c:3 + │ └── column8:8 => rowid:4 ├── update-mapping: - │ ├── upsert_a:15 => a:1 - │ └── upsert_c:17 => c:3 + │ ├── upsert_a:17 => a:1 + │ └── upsert_c:19 => c:3 └── project - ├── columns: upsert_a:15 upsert_b:16 upsert_c:17 upsert_rowid:18 column1:5!null column2:6!null column7:7 column8:8!null a:9 b:10 c:11 rowid:12 a_new:13 column14:14 + ├── columns: upsert_a:17 upsert_b:18 upsert_c:19 upsert_rowid:20 column1:6!null column2:7!null column8:8 column9:9!null a:10 b:11 c:12 rowid:13 crdb_internal_mvcc_timestamp:14 a_new:15 column16:16 ├── project - │ ├── columns: column14:14 column1:5!null column2:6!null column7:7 column8:8!null a:9 b:10 c:11 rowid:12 a_new:13 + │ ├── columns: column16:16 column1:6!null column2:7!null column8:8 column9:9!null a:10 b:11 c:12 rowid:13 crdb_internal_mvcc_timestamp:14 a_new:15 │ ├── project - │ │ ├── columns: a_new:13 column1:5!null column2:6!null column7:7 column8:8!null a:9 b:10 c:11 rowid:12 + │ │ ├── columns: a_new:15 column1:6!null column2:7!null column8:8 column9:9!null a:10 b:11 c:12 rowid:13 crdb_internal_mvcc_timestamp:14 │ │ ├── left-join (hash) - │ │ │ ├── columns: column1:5!null column2:6!null column7:7 column8:8!null a:9 b:10 c:11 rowid:12 + │ │ │ ├── columns: column1:6!null column2:7!null column8:8 column9:9!null a:10 b:11 c:12 rowid:13 crdb_internal_mvcc_timestamp:14 │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ ├── columns: column1:5!null column2:6!null column7:7 column8:8!null - │ │ │ │ ├── grouping columns: column1:5!null + │ │ │ │ ├── columns: column1:6!null column2:7!null column8:8 column9:9!null + │ │ │ │ ├── grouping columns: column1:6!null │ │ │ │ ├── project - │ │ │ │ │ ├── columns: column8:8!null column1:5!null column2:6!null column7:7 + │ │ │ │ │ ├── columns: column9:9!null column1:6!null column2:7!null column8:8 │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: column7:7 column1:5!null column2:6!null + │ │ │ │ │ │ ├── columns: column8:8 column1:6!null column2:7!null │ │ │ │ │ │ ├── values - │ │ │ │ │ │ │ ├── columns: column1:5!null column2:6!null + │ │ │ │ │ │ │ ├── columns: column1:6!null column2:7!null │ │ │ │ │ │ │ └── (1, 2) │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── unique_rowid() [as=column7:7] + │ │ │ │ │ │ └── unique_rowid() [as=column8:8] │ │ │ │ │ └── projections - │ │ │ │ │ └── column2:6 + 1 [as=column8:8] + │ │ │ │ │ └── column2:7 + 1 [as=column9:9] │ │ │ │ └── aggregations - │ │ │ │ ├── first-agg [as=column2:6] - │ │ │ │ │ └── column2:6 - │ │ │ │ ├── first-agg [as=column7:7] - │ │ │ │ │ └── column7:7 - │ │ │ │ └── first-agg [as=column8:8] - │ │ │ │ └── column8:8 + │ │ │ │ ├── first-agg [as=column2:7] + │ │ │ │ │ └── column2:7 + │ │ │ │ ├── first-agg [as=column8:8] + │ │ │ │ │ └── column8:8 + │ │ │ │ └── first-agg [as=column9:9] + │ │ │ │ └── column9:9 │ │ │ ├── scan tab - │ │ │ │ ├── columns: a:9!null b:10 c:11 rowid:12!null + │ │ │ │ ├── columns: a:10!null b:11 c:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ │ │ └── computed column expressions - │ │ │ │ └── c:11 - │ │ │ │ └── b:10 + 1 + │ │ │ │ └── c:12 + │ │ │ │ └── b:11 + 1 │ │ │ └── filters - │ │ │ └── column1:5 = a:9 + │ │ │ └── column1:6 = a:10 │ │ └── projections - │ │ └── a:9 * column1:5 [as=a_new:13] + │ │ └── a:10 * column1:6 [as=a_new:15] │ └── projections - │ └── b:10 + 1 [as=column14:14] + │ └── b:11 + 1 [as=column16:16] └── projections - ├── CASE WHEN rowid:12 IS NULL THEN column1:5 ELSE a_new:13 END [as=upsert_a:15] - ├── CASE WHEN rowid:12 IS NULL THEN column2:6 ELSE b:10 END [as=upsert_b:16] - ├── CASE WHEN rowid:12 IS NULL THEN column8:8 ELSE column14:14 END [as=upsert_c:17] - └── CASE WHEN rowid:12 IS NULL THEN column7:7 ELSE rowid:12 END [as=upsert_rowid:18] + ├── CASE WHEN rowid:13 IS NULL THEN column1:6 ELSE a_new:15 END [as=upsert_a:17] + ├── CASE WHEN rowid:13 IS NULL THEN column2:7 ELSE b:11 END [as=upsert_b:18] + ├── CASE WHEN rowid:13 IS NULL THEN column9:9 ELSE column16:16 END [as=upsert_c:19] + └── CASE WHEN rowid:13 IS NULL THEN column8:8 ELSE rowid:13 END [as=upsert_rowid:20] # Conflict columns are in different order than index key columns. build @@ -434,60 +436,60 @@ UPDATE SET a=5 ---- upsert abc ├── columns: - ├── canary column: 12 - ├── fetch columns: a:9 b:10 c:11 rowid:12 + ├── canary column: 13 + ├── fetch columns: a:10 b:11 c:12 rowid:13 ├── insert-mapping: - │ ├── column1:5 => a:1 - │ ├── column2:6 => b:2 - │ ├── column8:8 => c:3 - │ └── column7:7 => rowid:4 + │ ├── column1:6 => a:1 + │ ├── column2:7 => b:2 + │ ├── column9:9 => c:3 + │ └── column8:8 => rowid:4 ├── update-mapping: - │ ├── upsert_a:15 => a:1 - │ └── upsert_c:17 => c:3 + │ ├── upsert_a:17 => a:1 + │ └── upsert_c:19 => c:3 └── project - ├── columns: upsert_a:15!null upsert_b:16 upsert_c:17 upsert_rowid:18 column1:5!null column2:6!null column7:7 column8:8!null a:9 b:10 c:11 rowid:12 a_new:13!null column14:14 + ├── columns: upsert_a:17!null upsert_b:18 upsert_c:19 upsert_rowid:20 column1:6!null column2:7!null column8:8 column9:9!null a:10 b:11 c:12 rowid:13 crdb_internal_mvcc_timestamp:14 a_new:15!null column16:16 ├── project - │ ├── columns: column14:14 column1:5!null column2:6!null column7:7 column8:8!null a:9 b:10 c:11 rowid:12 a_new:13!null + │ ├── columns: column16:16 column1:6!null column2:7!null column8:8 column9:9!null a:10 b:11 c:12 rowid:13 crdb_internal_mvcc_timestamp:14 a_new:15!null │ ├── project - │ │ ├── columns: a_new:13!null column1:5!null column2:6!null column7:7 column8:8!null a:9 b:10 c:11 rowid:12 + │ │ ├── columns: a_new:15!null column1:6!null column2:7!null column8:8 column9:9!null a:10 b:11 c:12 rowid:13 crdb_internal_mvcc_timestamp:14 │ │ ├── left-join (hash) - │ │ │ ├── columns: column1:5!null column2:6!null column7:7 column8:8!null a:9 b:10 c:11 rowid:12 + │ │ │ ├── columns: column1:6!null column2:7!null column8:8 column9:9!null a:10 b:11 c:12 rowid:13 crdb_internal_mvcc_timestamp:14 │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ ├── columns: column1:5!null column2:6!null column7:7 column8:8!null - │ │ │ │ ├── grouping columns: column2:6!null column8:8!null + │ │ │ │ ├── columns: column1:6!null column2:7!null column8:8 column9:9!null + │ │ │ │ ├── grouping columns: column2:7!null column9:9!null │ │ │ │ ├── project - │ │ │ │ │ ├── columns: column8:8!null column1:5!null column2:6!null column7:7 + │ │ │ │ │ ├── columns: column9:9!null column1:6!null column2:7!null column8:8 │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: column7:7 column1:5!null column2:6!null + │ │ │ │ │ │ ├── columns: column8:8 column1:6!null column2:7!null │ │ │ │ │ │ ├── values - │ │ │ │ │ │ │ ├── columns: column1:5!null column2:6!null + │ │ │ │ │ │ │ ├── columns: column1:6!null column2:7!null │ │ │ │ │ │ │ └── (1, 2) │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── unique_rowid() [as=column7:7] + │ │ │ │ │ │ └── unique_rowid() [as=column8:8] │ │ │ │ │ └── projections - │ │ │ │ │ └── column2:6 + 1 [as=column8:8] + │ │ │ │ │ └── column2:7 + 1 [as=column9:9] │ │ │ │ └── aggregations - │ │ │ │ ├── first-agg [as=column1:5] - │ │ │ │ │ └── column1:5 - │ │ │ │ └── first-agg [as=column7:7] - │ │ │ │ └── column7:7 + │ │ │ │ ├── first-agg [as=column1:6] + │ │ │ │ │ └── column1:6 + │ │ │ │ └── first-agg [as=column8:8] + │ │ │ │ └── column8:8 │ │ │ ├── scan abc - │ │ │ │ ├── columns: a:9!null b:10 c:11 rowid:12!null + │ │ │ │ ├── columns: a:10!null b:11 c:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ │ │ └── computed column expressions - │ │ │ │ └── c:11 - │ │ │ │ └── b:10 + 1 + │ │ │ │ └── c:12 + │ │ │ │ └── b:11 + 1 │ │ │ └── filters - │ │ │ ├── column2:6 = b:10 - │ │ │ └── column8:8 = c:11 + │ │ │ ├── column2:7 = b:11 + │ │ │ └── column9:9 = c:12 │ │ └── projections - │ │ └── 5 [as=a_new:13] + │ │ └── 5 [as=a_new:15] │ └── projections - │ └── b:10 + 1 [as=column14:14] + │ └── b:11 + 1 [as=column16:16] └── projections - ├── CASE WHEN rowid:12 IS NULL THEN column1:5 ELSE a_new:13 END [as=upsert_a:15] - ├── CASE WHEN rowid:12 IS NULL THEN column2:6 ELSE b:10 END [as=upsert_b:16] - ├── CASE WHEN rowid:12 IS NULL THEN column8:8 ELSE column14:14 END [as=upsert_c:17] - └── CASE WHEN rowid:12 IS NULL THEN column7:7 ELSE rowid:12 END [as=upsert_rowid:18] + ├── CASE WHEN rowid:13 IS NULL THEN column1:6 ELSE a_new:15 END [as=upsert_a:17] + ├── CASE WHEN rowid:13 IS NULL THEN column2:7 ELSE b:11 END [as=upsert_b:18] + ├── CASE WHEN rowid:13 IS NULL THEN column9:9 ELSE column16:16 END [as=upsert_c:19] + └── CASE WHEN rowid:13 IS NULL THEN column8:8 ELSE rowid:13 END [as=upsert_rowid:20] # Conflict columns don't match unique index (too few columns). build @@ -529,71 +531,71 @@ ON CONFLICT DO NOTHING insert xyz ├── columns: ├── insert-mapping: - │ ├── column1:4 => x:1 - │ ├── column2:5 => y:2 - │ └── column3:6 => z:3 + │ ├── column1:5 => x:1 + │ ├── column2:6 => y:2 + │ └── column3:7 => z:3 └── upsert-distinct-on - ├── columns: column1:4!null column2:5!null column3:6!null - ├── grouping columns: column2:5!null column3:6!null + ├── columns: column1:5!null column2:6!null column3:7!null + ├── grouping columns: column2:6!null column3:7!null ├── project - │ ├── columns: column1:4!null column2:5!null column3:6!null + │ ├── columns: column1:5!null column2:6!null column3:7!null │ └── select - │ ├── columns: column1:4!null column2:5!null column3:6!null x:13 y:14 z:15 + │ ├── columns: column1:5!null column2:6!null column3:7!null x:16 y:17 z:18 crdb_internal_mvcc_timestamp:19 │ ├── left-join (hash) - │ │ ├── columns: column1:4!null column2:5!null column3:6!null x:13 y:14 z:15 + │ │ ├── columns: column1:5!null column2:6!null column3:7!null x:16 y:17 z:18 crdb_internal_mvcc_timestamp:19 │ │ ├── upsert-distinct-on - │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null - │ │ │ ├── grouping columns: column2:5!null column3:6!null + │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null + │ │ │ ├── grouping columns: column2:6!null column3:7!null │ │ │ ├── project - │ │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null + │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null │ │ │ │ └── select - │ │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null x:10 y:11 z:12 + │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null x:12 y:13 z:14 crdb_internal_mvcc_timestamp:15 │ │ │ │ ├── left-join (hash) - │ │ │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null x:10 y:11 z:12 + │ │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null x:12 y:13 z:14 crdb_internal_mvcc_timestamp:15 │ │ │ │ │ ├── upsert-distinct-on - │ │ │ │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null - │ │ │ │ │ │ ├── grouping columns: column1:4!null + │ │ │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null + │ │ │ │ │ │ ├── grouping columns: column1:5!null │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null + │ │ │ │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null │ │ │ │ │ │ │ └── select - │ │ │ │ │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null x:7 y:8 z:9 + │ │ │ │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null x:8 y:9 z:10 crdb_internal_mvcc_timestamp:11 │ │ │ │ │ │ │ ├── left-join (hash) - │ │ │ │ │ │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null x:7 y:8 z:9 + │ │ │ │ │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null x:8 y:9 z:10 crdb_internal_mvcc_timestamp:11 │ │ │ │ │ │ │ │ ├── values - │ │ │ │ │ │ │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null + │ │ │ │ │ │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null │ │ │ │ │ │ │ │ │ ├── (1, 2, 3) │ │ │ │ │ │ │ │ │ └── (4, 5, 6) │ │ │ │ │ │ │ │ ├── scan xyz - │ │ │ │ │ │ │ │ │ └── columns: x:7!null y:8 z:9 + │ │ │ │ │ │ │ │ │ └── columns: x:8!null y:9 z:10 crdb_internal_mvcc_timestamp:11 │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ └── column1:4 = x:7 + │ │ │ │ │ │ │ │ └── column1:5 = x:8 │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── x:7 IS NULL + │ │ │ │ │ │ │ └── x:8 IS NULL │ │ │ │ │ │ └── aggregations - │ │ │ │ │ │ ├── first-agg [as=column2:5] - │ │ │ │ │ │ │ └── column2:5 - │ │ │ │ │ │ └── first-agg [as=column3:6] - │ │ │ │ │ │ └── column3:6 + │ │ │ │ │ │ ├── first-agg [as=column2:6] + │ │ │ │ │ │ │ └── column2:6 + │ │ │ │ │ │ └── first-agg [as=column3:7] + │ │ │ │ │ │ └── column3:7 │ │ │ │ │ ├── scan xyz - │ │ │ │ │ │ └── columns: x:10!null y:11 z:12 + │ │ │ │ │ │ └── columns: x:12!null y:13 z:14 crdb_internal_mvcc_timestamp:15 │ │ │ │ │ └── filters - │ │ │ │ │ ├── column2:5 = y:11 - │ │ │ │ │ └── column3:6 = z:12 + │ │ │ │ │ ├── column2:6 = y:13 + │ │ │ │ │ └── column3:7 = z:14 │ │ │ │ └── filters - │ │ │ │ └── x:10 IS NULL + │ │ │ │ └── x:12 IS NULL │ │ │ └── aggregations - │ │ │ └── first-agg [as=column1:4] - │ │ │ └── column1:4 + │ │ │ └── first-agg [as=column1:5] + │ │ │ └── column1:5 │ │ ├── scan xyz - │ │ │ └── columns: x:13!null y:14 z:15 + │ │ │ └── columns: x:16!null y:17 z:18 crdb_internal_mvcc_timestamp:19 │ │ └── filters - │ │ ├── column3:6 = z:15 - │ │ └── column2:5 = y:14 + │ │ ├── column3:7 = z:18 + │ │ └── column2:6 = y:17 │ └── filters - │ └── x:13 IS NULL + │ └── x:16 IS NULL └── aggregations - └── first-agg [as=column1:4] - └── column1:4 + └── first-agg [as=column1:5] + └── column1:5 # Conflict columns are explicitly specified. build @@ -604,32 +606,32 @@ ON CONFLICT (y, z) DO NOTHING insert xyz ├── columns: ├── insert-mapping: - │ ├── column1:4 => x:1 - │ ├── column2:5 => y:2 - │ └── column3:6 => z:3 + │ ├── column1:5 => x:1 + │ ├── column2:6 => y:2 + │ └── column3:7 => z:3 └── upsert-distinct-on - ├── columns: column1:4!null column2:5!null column3:6!null - ├── grouping columns: column2:5!null column3:6!null + ├── columns: column1:5!null column2:6!null column3:7!null + ├── grouping columns: column2:6!null column3:7!null ├── project - │ ├── columns: column1:4!null column2:5!null column3:6!null + │ ├── columns: column1:5!null column2:6!null column3:7!null │ └── select - │ ├── columns: column1:4!null column2:5!null column3:6!null x:7 y:8 z:9 + │ ├── columns: column1:5!null column2:6!null column3:7!null x:8 y:9 z:10 crdb_internal_mvcc_timestamp:11 │ ├── left-join (hash) - │ │ ├── columns: column1:4!null column2:5!null column3:6!null x:7 y:8 z:9 + │ │ ├── columns: column1:5!null column2:6!null column3:7!null x:8 y:9 z:10 crdb_internal_mvcc_timestamp:11 │ │ ├── values - │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null + │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null │ │ │ ├── (1, 2, 3) │ │ │ └── (4, 5, 6) │ │ ├── scan xyz - │ │ │ └── columns: x:7!null y:8 z:9 + │ │ │ └── columns: x:8!null y:9 z:10 crdb_internal_mvcc_timestamp:11 │ │ └── filters - │ │ ├── column2:5 = y:8 - │ │ └── column3:6 = z:9 + │ │ ├── column2:6 = y:9 + │ │ └── column3:7 = z:10 │ └── filters - │ └── x:7 IS NULL + │ └── x:8 IS NULL └── aggregations - └── first-agg [as=column1:4] - └── column1:4 + └── first-agg [as=column1:5] + └── column1:5 # ------------------------------------------------------------------------------ # Test excluded columns. @@ -644,59 +646,59 @@ WHERE excluded.y>xyz.y RETURNING xyz.x*2, y+z ---- project - ├── columns: "?column?":16!null "?column?":17 + ├── columns: "?column?":18!null "?column?":19 ├── upsert xyz │ ├── columns: x:1!null y:2 z:3!null - │ ├── canary column: 7 - │ ├── fetch columns: x:7 y:8 z:9 + │ ├── canary column: 8 + │ ├── fetch columns: x:8 y:9 z:10 │ ├── insert-mapping: - │ │ ├── column1:4 => x:1 - │ │ ├── column2:5 => y:2 - │ │ └── column3:6 => z:3 + │ │ ├── column1:5 => x:1 + │ │ ├── column2:6 => y:2 + │ │ └── column3:7 => z:3 │ ├── update-mapping: - │ │ ├── upsert_x:13 => x:1 - │ │ ├── upsert_y:14 => y:2 - │ │ └── upsert_z:15 => z:3 + │ │ ├── upsert_x:15 => x:1 + │ │ ├── upsert_y:16 => y:2 + │ │ └── upsert_z:17 => z:3 │ ├── return-mapping: - │ │ ├── upsert_x:13 => x:1 - │ │ ├── upsert_y:14 => y:2 - │ │ └── upsert_z:15 => z:3 + │ │ ├── upsert_x:15 => x:1 + │ │ ├── upsert_y:16 => y:2 + │ │ └── upsert_z:17 => z:3 │ └── project - │ ├── columns: upsert_x:13!null upsert_y:14 upsert_z:15!null column1:4!null column2:5!null column3:6!null x:7 y:8 z:9 x_new:10!null y_new:11 z_new:12!null + │ ├── columns: upsert_x:15!null upsert_y:16 upsert_z:17!null column1:5!null column2:6!null column3:7!null x:8 y:9 z:10 crdb_internal_mvcc_timestamp:11 x_new:12!null y_new:13 z_new:14!null │ ├── project - │ │ ├── columns: x_new:10!null y_new:11 z_new:12!null column1:4!null column2:5!null column3:6!null x:7 y:8 z:9 + │ │ ├── columns: x_new:12!null y_new:13 z_new:14!null column1:5!null column2:6!null column3:7!null x:8 y:9 z:10 crdb_internal_mvcc_timestamp:11 │ │ ├── select - │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null x:7 y:8 z:9 + │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null x:8 y:9 z:10 crdb_internal_mvcc_timestamp:11 │ │ │ ├── left-join (hash) - │ │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null x:7 y:8 z:9 + │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null x:8 y:9 z:10 crdb_internal_mvcc_timestamp:11 │ │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null - │ │ │ │ │ ├── grouping columns: column2:5!null column3:6!null + │ │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null + │ │ │ │ │ ├── grouping columns: column2:6!null column3:7!null │ │ │ │ │ ├── values - │ │ │ │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null + │ │ │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null │ │ │ │ │ │ ├── (1, 2, 3) │ │ │ │ │ │ └── (-1, -1, -1) │ │ │ │ │ └── aggregations - │ │ │ │ │ └── first-agg [as=column1:4] - │ │ │ │ │ └── column1:4 + │ │ │ │ │ └── first-agg [as=column1:5] + │ │ │ │ │ └── column1:5 │ │ │ │ ├── scan xyz - │ │ │ │ │ └── columns: x:7!null y:8 z:9 + │ │ │ │ │ └── columns: x:8!null y:9 z:10 crdb_internal_mvcc_timestamp:11 │ │ │ │ └── filters - │ │ │ │ ├── column2:5 = y:8 - │ │ │ │ └── column3:6 = z:9 + │ │ │ │ ├── column2:6 = y:9 + │ │ │ │ └── column3:7 = z:10 │ │ │ └── filters - │ │ │ └── (x:7 IS NULL) OR (column2:5 > y:8) + │ │ │ └── (x:8 IS NULL) OR (column2:6 > y:9) │ │ └── projections - │ │ ├── column1:4 + 1 [as=x_new:10] - │ │ ├── column2:5 * y:8 [as=y_new:11] - │ │ └── column1:4 - column3:6 [as=z_new:12] + │ │ ├── column1:5 + 1 [as=x_new:12] + │ │ ├── column2:6 * y:9 [as=y_new:13] + │ │ └── column1:5 - column3:7 [as=z_new:14] │ └── projections - │ ├── CASE WHEN x:7 IS NULL THEN column1:4 ELSE x_new:10 END [as=upsert_x:13] - │ ├── CASE WHEN x:7 IS NULL THEN column2:5 ELSE y_new:11 END [as=upsert_y:14] - │ └── CASE WHEN x:7 IS NULL THEN column3:6 ELSE z_new:12 END [as=upsert_z:15] + │ ├── CASE WHEN x:8 IS NULL THEN column1:5 ELSE x_new:12 END [as=upsert_x:15] + │ ├── CASE WHEN x:8 IS NULL THEN column2:6 ELSE y_new:13 END [as=upsert_y:16] + │ └── CASE WHEN x:8 IS NULL THEN column3:7 ELSE z_new:14 END [as=upsert_z:17] └── projections - ├── x:1 * 2 [as="?column?":16] - └── y:2 + z:3 [as="?column?":17] + ├── x:1 * 2 [as="?column?":18] + └── y:2 + z:3 [as="?column?":19] # Try to use excluded in RETURNING. build @@ -730,73 +732,73 @@ UPDATE SET (b, a)=(SELECT x, y+excluded.b FROM xyz WHERE x=excluded.a) ---- upsert abc ├── columns: - ├── canary column: 12 - ├── fetch columns: a:9 b:10 c:11 rowid:12 + ├── canary column: 13 + ├── fetch columns: a:10 b:11 c:12 rowid:13 ├── insert-mapping: - │ ├── column1:5 => a:1 - │ ├── column2:6 => b:2 - │ ├── column8:8 => c:3 - │ └── column7:7 => rowid:4 + │ ├── column1:6 => a:1 + │ ├── column2:7 => b:2 + │ ├── column9:9 => c:3 + │ └── column8:8 => rowid:4 ├── update-mapping: - │ ├── upsert_a:18 => a:1 - │ ├── upsert_b:19 => b:2 - │ └── upsert_c:20 => c:3 + │ ├── upsert_a:21 => a:1 + │ ├── upsert_b:22 => b:2 + │ └── upsert_c:23 => c:3 └── project - ├── columns: upsert_a:18 upsert_b:19 upsert_c:20 upsert_rowid:21 column1:5!null column2:6!null column7:7 column8:8!null a:9 b:10 c:11 rowid:12 x:13 "?column?":16 column17:17 + ├── columns: upsert_a:21 upsert_b:22 upsert_c:23 upsert_rowid:24 column1:6!null column2:7!null column8:8 column9:9!null a:10 b:11 c:12 rowid:13 abc.crdb_internal_mvcc_timestamp:14 x:15 "?column?":19 column20:20 ├── project - │ ├── columns: column17:17 column1:5!null column2:6!null column7:7 column8:8!null a:9 b:10 c:11 rowid:12 x:13 "?column?":16 + │ ├── columns: column20:20 column1:6!null column2:7!null column8:8 column9:9!null a:10 b:11 c:12 rowid:13 abc.crdb_internal_mvcc_timestamp:14 x:15 "?column?":19 │ ├── left-join-apply - │ │ ├── columns: column1:5!null column2:6!null column7:7 column8:8!null a:9 b:10 c:11 rowid:12 x:13 "?column?":16 + │ │ ├── columns: column1:6!null column2:7!null column8:8 column9:9!null a:10 b:11 c:12 rowid:13 abc.crdb_internal_mvcc_timestamp:14 x:15 "?column?":19 │ │ ├── left-join (hash) - │ │ │ ├── columns: column1:5!null column2:6!null column7:7 column8:8!null a:9 b:10 c:11 rowid:12 + │ │ │ ├── columns: column1:6!null column2:7!null column8:8 column9:9!null a:10 b:11 c:12 rowid:13 abc.crdb_internal_mvcc_timestamp:14 │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ ├── columns: column1:5!null column2:6!null column7:7 column8:8!null - │ │ │ │ ├── grouping columns: column1:5!null + │ │ │ │ ├── columns: column1:6!null column2:7!null column8:8 column9:9!null + │ │ │ │ ├── grouping columns: column1:6!null │ │ │ │ ├── project - │ │ │ │ │ ├── columns: column8:8!null column1:5!null column2:6!null column7:7 + │ │ │ │ │ ├── columns: column9:9!null column1:6!null column2:7!null column8:8 │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: column7:7 column1:5!null column2:6!null + │ │ │ │ │ │ ├── columns: column8:8 column1:6!null column2:7!null │ │ │ │ │ │ ├── values - │ │ │ │ │ │ │ ├── columns: column1:5!null column2:6!null + │ │ │ │ │ │ │ ├── columns: column1:6!null column2:7!null │ │ │ │ │ │ │ └── (1, 2) │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── unique_rowid() [as=column7:7] + │ │ │ │ │ │ └── unique_rowid() [as=column8:8] │ │ │ │ │ └── projections - │ │ │ │ │ └── column2:6 + 1 [as=column8:8] + │ │ │ │ │ └── column2:7 + 1 [as=column9:9] │ │ │ │ └── aggregations - │ │ │ │ ├── first-agg [as=column2:6] - │ │ │ │ │ └── column2:6 - │ │ │ │ ├── first-agg [as=column7:7] - │ │ │ │ │ └── column7:7 - │ │ │ │ └── first-agg [as=column8:8] - │ │ │ │ └── column8:8 + │ │ │ │ ├── first-agg [as=column2:7] + │ │ │ │ │ └── column2:7 + │ │ │ │ ├── first-agg [as=column8:8] + │ │ │ │ │ └── column8:8 + │ │ │ │ └── first-agg [as=column9:9] + │ │ │ │ └── column9:9 │ │ │ ├── scan abc - │ │ │ │ ├── columns: a:9!null b:10 c:11 rowid:12!null + │ │ │ │ ├── columns: a:10!null b:11 c:12 rowid:13!null abc.crdb_internal_mvcc_timestamp:14 │ │ │ │ └── computed column expressions - │ │ │ │ └── c:11 - │ │ │ │ └── b:10 + 1 + │ │ │ │ └── c:12 + │ │ │ │ └── b:11 + 1 │ │ │ └── filters - │ │ │ └── column1:5 = a:9 + │ │ │ └── column1:6 = a:10 │ │ ├── max1-row - │ │ │ ├── columns: x:13!null "?column?":16 + │ │ │ ├── columns: x:15!null "?column?":19 │ │ │ └── project - │ │ │ ├── columns: "?column?":16 x:13!null + │ │ │ ├── columns: "?column?":19 x:15!null │ │ │ ├── select - │ │ │ │ ├── columns: x:13!null y:14 z:15 + │ │ │ │ ├── columns: x:15!null y:16 z:17 xyz.crdb_internal_mvcc_timestamp:18 │ │ │ │ ├── scan xyz - │ │ │ │ │ └── columns: x:13!null y:14 z:15 + │ │ │ │ │ └── columns: x:15!null y:16 z:17 xyz.crdb_internal_mvcc_timestamp:18 │ │ │ │ └── filters - │ │ │ │ └── x:13 = column1:5 + │ │ │ │ └── x:15 = column1:6 │ │ │ └── projections - │ │ │ └── y:14 + column2:6 [as="?column?":16] + │ │ │ └── y:16 + column2:7 [as="?column?":19] │ │ └── filters (true) │ └── projections - │ └── x:13 + 1 [as=column17:17] + │ └── x:15 + 1 [as=column20:20] └── projections - ├── CASE WHEN rowid:12 IS NULL THEN column1:5 ELSE "?column?":16 END [as=upsert_a:18] - ├── CASE WHEN rowid:12 IS NULL THEN column2:6 ELSE x:13 END [as=upsert_b:19] - ├── CASE WHEN rowid:12 IS NULL THEN column8:8 ELSE column17:17 END [as=upsert_c:20] - └── CASE WHEN rowid:12 IS NULL THEN column7:7 ELSE rowid:12 END [as=upsert_rowid:21] + ├── CASE WHEN rowid:13 IS NULL THEN column1:6 ELSE "?column?":19 END [as=upsert_a:21] + ├── CASE WHEN rowid:13 IS NULL THEN column2:7 ELSE x:15 END [as=upsert_b:22] + ├── CASE WHEN rowid:13 IS NULL THEN column9:9 ELSE column20:20 END [as=upsert_c:23] + └── CASE WHEN rowid:13 IS NULL THEN column8:8 ELSE rowid:13 END [as=upsert_rowid:24] # Default expressions. build @@ -807,63 +809,63 @@ UPDATE SET a=DEFAULT, b=DEFAULT ---- upsert abc ├── columns: - ├── canary column: 12 - ├── fetch columns: a:9 b:10 c:11 rowid:12 + ├── canary column: 13 + ├── fetch columns: a:10 b:11 c:12 rowid:13 ├── insert-mapping: - │ ├── column1:5 => a:1 - │ ├── column2:6 => b:2 - │ ├── column8:8 => c:3 - │ └── column7:7 => rowid:4 + │ ├── column1:6 => a:1 + │ ├── column2:7 => b:2 + │ ├── column9:9 => c:3 + │ └── column8:8 => rowid:4 ├── update-mapping: - │ ├── upsert_a:16 => a:1 - │ ├── upsert_b:17 => b:2 - │ └── upsert_c:18 => c:3 + │ ├── upsert_a:18 => a:1 + │ ├── upsert_b:19 => b:2 + │ └── upsert_c:20 => c:3 └── project - ├── columns: upsert_a:16 upsert_b:17!null upsert_c:18!null upsert_rowid:19 column1:5!null column2:6!null column7:7 column8:8!null a:9 b:10 c:11 rowid:12 a_new:13 b_new:14!null column15:15!null + ├── columns: upsert_a:18 upsert_b:19!null upsert_c:20!null upsert_rowid:21 column1:6!null column2:7!null column8:8 column9:9!null a:10 b:11 c:12 rowid:13 crdb_internal_mvcc_timestamp:14 a_new:15 b_new:16!null column17:17!null ├── project - │ ├── columns: column15:15!null column1:5!null column2:6!null column7:7 column8:8!null a:9 b:10 c:11 rowid:12 a_new:13 b_new:14!null + │ ├── columns: column17:17!null column1:6!null column2:7!null column8:8 column9:9!null a:10 b:11 c:12 rowid:13 crdb_internal_mvcc_timestamp:14 a_new:15 b_new:16!null │ ├── project - │ │ ├── columns: a_new:13 b_new:14!null column1:5!null column2:6!null column7:7 column8:8!null a:9 b:10 c:11 rowid:12 + │ │ ├── columns: a_new:15 b_new:16!null column1:6!null column2:7!null column8:8 column9:9!null a:10 b:11 c:12 rowid:13 crdb_internal_mvcc_timestamp:14 │ │ ├── left-join (hash) - │ │ │ ├── columns: column1:5!null column2:6!null column7:7 column8:8!null a:9 b:10 c:11 rowid:12 + │ │ │ ├── columns: column1:6!null column2:7!null column8:8 column9:9!null a:10 b:11 c:12 rowid:13 crdb_internal_mvcc_timestamp:14 │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ ├── columns: column1:5!null column2:6!null column7:7 column8:8!null - │ │ │ │ ├── grouping columns: column1:5!null + │ │ │ │ ├── columns: column1:6!null column2:7!null column8:8 column9:9!null + │ │ │ │ ├── grouping columns: column1:6!null │ │ │ │ ├── project - │ │ │ │ │ ├── columns: column8:8!null column1:5!null column2:6!null column7:7 + │ │ │ │ │ ├── columns: column9:9!null column1:6!null column2:7!null column8:8 │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: column7:7 column1:5!null column2:6!null + │ │ │ │ │ │ ├── columns: column8:8 column1:6!null column2:7!null │ │ │ │ │ │ ├── values - │ │ │ │ │ │ │ ├── columns: column1:5!null column2:6!null + │ │ │ │ │ │ │ ├── columns: column1:6!null column2:7!null │ │ │ │ │ │ │ └── (1, 2) │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── unique_rowid() [as=column7:7] + │ │ │ │ │ │ └── unique_rowid() [as=column8:8] │ │ │ │ │ └── projections - │ │ │ │ │ └── column2:6 + 1 [as=column8:8] + │ │ │ │ │ └── column2:7 + 1 [as=column9:9] │ │ │ │ └── aggregations - │ │ │ │ ├── first-agg [as=column2:6] - │ │ │ │ │ └── column2:6 - │ │ │ │ ├── first-agg [as=column7:7] - │ │ │ │ │ └── column7:7 - │ │ │ │ └── first-agg [as=column8:8] - │ │ │ │ └── column8:8 + │ │ │ │ ├── first-agg [as=column2:7] + │ │ │ │ │ └── column2:7 + │ │ │ │ ├── first-agg [as=column8:8] + │ │ │ │ │ └── column8:8 + │ │ │ │ └── first-agg [as=column9:9] + │ │ │ │ └── column9:9 │ │ │ ├── scan abc - │ │ │ │ ├── columns: a:9!null b:10 c:11 rowid:12!null + │ │ │ │ ├── columns: a:10!null b:11 c:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ │ │ └── computed column expressions - │ │ │ │ └── c:11 - │ │ │ │ └── b:10 + 1 + │ │ │ │ └── c:12 + │ │ │ │ └── b:11 + 1 │ │ │ └── filters - │ │ │ └── column1:5 = a:9 + │ │ │ └── column1:6 = a:10 │ │ └── projections - │ │ ├── NULL::INT8 [as=a_new:13] - │ │ └── 10 [as=b_new:14] + │ │ ├── NULL::INT8 [as=a_new:15] + │ │ └── 10 [as=b_new:16] │ └── projections - │ └── b_new:14 + 1 [as=column15:15] + │ └── b_new:16 + 1 [as=column17:17] └── projections - ├── CASE WHEN rowid:12 IS NULL THEN column1:5 ELSE a_new:13 END [as=upsert_a:16] - ├── CASE WHEN rowid:12 IS NULL THEN column2:6 ELSE b_new:14 END [as=upsert_b:17] - ├── CASE WHEN rowid:12 IS NULL THEN column8:8 ELSE column15:15 END [as=upsert_c:18] - └── CASE WHEN rowid:12 IS NULL THEN column7:7 ELSE rowid:12 END [as=upsert_rowid:19] + ├── CASE WHEN rowid:13 IS NULL THEN column1:6 ELSE a_new:15 END [as=upsert_a:18] + ├── CASE WHEN rowid:13 IS NULL THEN column2:7 ELSE b_new:16 END [as=upsert_b:19] + ├── CASE WHEN rowid:13 IS NULL THEN column9:9 ELSE column17:17 END [as=upsert_c:20] + └── CASE WHEN rowid:13 IS NULL THEN column8:8 ELSE rowid:13 END [as=upsert_rowid:21] # ------------------------------------------------------------------------------ # Test mutation columns. @@ -877,65 +879,65 @@ UPDATE SET m=mutation.m+1 ---- upsert mutation ├── columns: - ├── canary column: 10 - ├── fetch columns: m:10 n:11 o:12 p:13 q:14 + ├── canary column: 11 + ├── fetch columns: m:11 n:12 o:13 p:14 q:15 ├── insert-mapping: - │ ├── column1:6 => m:1 - │ ├── column2:7 => n:2 - │ ├── column8:8 => o:3 - │ └── column9:9 => p:4 + │ ├── column1:7 => m:1 + │ ├── column2:8 => n:2 + │ ├── column9:9 => o:3 + │ └── column10:10 => p:4 ├── update-mapping: - │ ├── upsert_m:17 => m:1 - │ ├── column8:8 => o:3 - │ └── upsert_p:19 => p:4 - ├── check columns: check1:20 + │ ├── upsert_m:19 => m:1 + │ ├── column9:9 => o:3 + │ └── upsert_p:21 => p:4 + ├── check columns: check1:22 └── project - ├── columns: check1:20 column1:6!null column2:7!null column8:8!null column9:9!null m:10 n:11 o:12 p:13 q:14 m_new:15 column16:16 upsert_m:17 upsert_n:18 upsert_p:19 + ├── columns: check1:22 column1:7!null column2:8!null column9:9!null column10:10!null m:11 n:12 o:13 p:14 q:15 crdb_internal_mvcc_timestamp:16 m_new:17 column18:18 upsert_m:19 upsert_n:20 upsert_p:21 ├── project - │ ├── columns: upsert_m:17 upsert_n:18 upsert_p:19 column1:6!null column2:7!null column8:8!null column9:9!null m:10 n:11 o:12 p:13 q:14 m_new:15 column16:16 + │ ├── columns: upsert_m:19 upsert_n:20 upsert_p:21 column1:7!null column2:8!null column9:9!null column10:10!null m:11 n:12 o:13 p:14 q:15 crdb_internal_mvcc_timestamp:16 m_new:17 column18:18 │ ├── project - │ │ ├── columns: column16:16 column1:6!null column2:7!null column8:8!null column9:9!null m:10 n:11 o:12 p:13 q:14 m_new:15 + │ │ ├── columns: column18:18 column1:7!null column2:8!null column9:9!null column10:10!null m:11 n:12 o:13 p:14 q:15 crdb_internal_mvcc_timestamp:16 m_new:17 │ │ ├── project - │ │ │ ├── columns: m_new:15 column1:6!null column2:7!null column8:8!null column9:9!null m:10 n:11 o:12 p:13 q:14 + │ │ │ ├── columns: m_new:17 column1:7!null column2:8!null column9:9!null column10:10!null m:11 n:12 o:13 p:14 q:15 crdb_internal_mvcc_timestamp:16 │ │ │ ├── left-join (hash) - │ │ │ │ ├── columns: column1:6!null column2:7!null column8:8!null column9:9!null m:10 n:11 o:12 p:13 q:14 + │ │ │ │ ├── columns: column1:7!null column2:8!null column9:9!null column10:10!null m:11 n:12 o:13 p:14 q:15 crdb_internal_mvcc_timestamp:16 │ │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ │ ├── columns: column1:6!null column2:7!null column8:8!null column9:9!null - │ │ │ │ │ ├── grouping columns: column1:6!null + │ │ │ │ │ ├── columns: column1:7!null column2:8!null column9:9!null column10:10!null + │ │ │ │ │ ├── grouping columns: column1:7!null │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: column9:9!null column1:6!null column2:7!null column8:8!null + │ │ │ │ │ │ ├── columns: column10:10!null column1:7!null column2:8!null column9:9!null │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ ├── columns: column8:8!null column1:6!null column2:7!null + │ │ │ │ │ │ │ ├── columns: column9:9!null column1:7!null column2:8!null │ │ │ │ │ │ │ ├── values - │ │ │ │ │ │ │ │ ├── columns: column1:6!null column2:7!null + │ │ │ │ │ │ │ │ ├── columns: column1:7!null column2:8!null │ │ │ │ │ │ │ │ └── (1, 2) │ │ │ │ │ │ │ └── projections - │ │ │ │ │ │ │ └── 10 [as=column8:8] + │ │ │ │ │ │ │ └── 10 [as=column9:9] │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── column8:8 + column2:7 [as=column9:9] + │ │ │ │ │ │ └── column9:9 + column2:8 [as=column10:10] │ │ │ │ │ └── aggregations - │ │ │ │ │ ├── first-agg [as=column2:7] - │ │ │ │ │ │ └── column2:7 - │ │ │ │ │ ├── first-agg [as=column8:8] - │ │ │ │ │ │ └── column8:8 - │ │ │ │ │ └── first-agg [as=column9:9] - │ │ │ │ │ └── column9:9 + │ │ │ │ │ ├── first-agg [as=column2:8] + │ │ │ │ │ │ └── column2:8 + │ │ │ │ │ ├── first-agg [as=column9:9] + │ │ │ │ │ │ └── column9:9 + │ │ │ │ │ └── first-agg [as=column10:10] + │ │ │ │ │ └── column10:10 │ │ │ │ ├── scan mutation - │ │ │ │ │ ├── columns: m:10!null n:11 o:12 p:13 q:14 + │ │ │ │ │ ├── columns: m:11!null n:12 o:13 p:14 q:15 crdb_internal_mvcc_timestamp:16 │ │ │ │ │ └── check constraint expressions - │ │ │ │ │ └── m:10 > 0 + │ │ │ │ │ └── m:11 > 0 │ │ │ │ └── filters - │ │ │ │ └── column1:6 = m:10 + │ │ │ │ └── column1:7 = m:11 │ │ │ └── projections - │ │ │ └── m:10 + 1 [as=m_new:15] + │ │ │ └── m:11 + 1 [as=m_new:17] │ │ └── projections - │ │ └── column8:8 + n:11 [as=column16:16] + │ │ └── column9:9 + n:12 [as=column18:18] │ └── projections - │ ├── CASE WHEN m:10 IS NULL THEN column1:6 ELSE m_new:15 END [as=upsert_m:17] - │ ├── CASE WHEN m:10 IS NULL THEN column2:7 ELSE n:11 END [as=upsert_n:18] - │ └── CASE WHEN m:10 IS NULL THEN column9:9 ELSE column16:16 END [as=upsert_p:19] + │ ├── CASE WHEN m:11 IS NULL THEN column1:7 ELSE m_new:17 END [as=upsert_m:19] + │ ├── CASE WHEN m:11 IS NULL THEN column2:8 ELSE n:12 END [as=upsert_n:20] + │ └── CASE WHEN m:11 IS NULL THEN column10:10 ELSE column18:18 END [as=upsert_p:21] └── projections - └── upsert_m:17 > 0 [as=check1:20] + └── upsert_m:19 > 0 [as=check1:22] # ------------------------------------------------------------------------------ # Test UPSERT. @@ -947,38 +949,38 @@ UPSERT INTO xyz VALUES (1) ---- upsert xyz ├── columns: - ├── canary column: 6 - ├── fetch columns: x:6 y:7 z:8 + ├── canary column: 7 + ├── fetch columns: x:7 y:8 z:9 ├── insert-mapping: - │ ├── column1:4 => x:1 - │ ├── column5:5 => y:2 - │ └── column5:5 => z:3 + │ ├── column1:5 => x:1 + │ ├── column6:6 => y:2 + │ └── column6:6 => z:3 ├── update-mapping: - │ ├── column5:5 => y:2 - │ └── column5:5 => z:3 + │ ├── column6:6 => y:2 + │ └── column6:6 => z:3 └── project - ├── columns: upsert_x:9 column1:4!null column5:5 x:6 y:7 z:8 + ├── columns: upsert_x:11 column1:5!null column6:6 x:7 y:8 z:9 crdb_internal_mvcc_timestamp:10 ├── left-join (hash) - │ ├── columns: column1:4!null column5:5 x:6 y:7 z:8 + │ ├── columns: column1:5!null column6:6 x:7 y:8 z:9 crdb_internal_mvcc_timestamp:10 │ ├── ensure-upsert-distinct-on - │ │ ├── columns: column1:4!null column5:5 - │ │ ├── grouping columns: column1:4!null + │ │ ├── columns: column1:5!null column6:6 + │ │ ├── grouping columns: column1:5!null │ │ ├── project - │ │ │ ├── columns: column5:5 column1:4!null + │ │ │ ├── columns: column6:6 column1:5!null │ │ │ ├── values - │ │ │ │ ├── columns: column1:4!null + │ │ │ │ ├── columns: column1:5!null │ │ │ │ └── (1,) │ │ │ └── projections - │ │ │ └── NULL::INT8 [as=column5:5] + │ │ │ └── NULL::INT8 [as=column6:6] │ │ └── aggregations - │ │ └── first-agg [as=column5:5] - │ │ └── column5:5 + │ │ └── first-agg [as=column6:6] + │ │ └── column6:6 │ ├── scan xyz - │ │ └── columns: x:6!null y:7 z:8 + │ │ └── columns: x:7!null y:8 z:9 crdb_internal_mvcc_timestamp:10 │ └── filters - │ └── column1:4 = x:6 + │ └── column1:5 = x:7 └── projections - └── CASE WHEN x:6 IS NULL THEN column1:4 ELSE x:6 END [as=upsert_x:9] + └── CASE WHEN x:7 IS NULL THEN column1:5 ELSE x:7 END [as=upsert_x:11] # Test multi-column primary key that contains all columns in table. build @@ -987,10 +989,10 @@ UPSERT INTO uv VALUES (1, 2) RETURNING * upsert uv ├── columns: u:1!null v:2!null ├── upsert-mapping: - │ ├── column1:3 => u:1 - │ └── column2:4 => v:2 + │ ├── column1:4 => u:1 + │ └── column2:5 => v:2 └── values - ├── columns: column1:3!null column2:4!null + ├── columns: column1:4!null column2:5!null └── (1, 2) # Use returning UPSERT as a FROM expression. @@ -998,67 +1000,67 @@ build SELECT * FROM [UPSERT INTO abc VALUES (1, 2) RETURNING *] ---- with &1 - ├── columns: a:14!null b:15!null c:16!null + ├── columns: a:16!null b:17!null c:18!null ├── project │ ├── columns: abc.a:1!null abc.b:2!null abc.c:3!null │ └── upsert abc │ ├── columns: abc.a:1!null abc.b:2!null abc.c:3!null rowid:4!null - │ ├── canary column: 12 - │ ├── fetch columns: abc.a:9 abc.b:10 abc.c:11 rowid:12 + │ ├── canary column: 13 + │ ├── fetch columns: abc.a:10 abc.b:11 abc.c:12 rowid:13 │ ├── insert-mapping: - │ │ ├── column1:5 => abc.a:1 - │ │ ├── column2:6 => abc.b:2 - │ │ ├── column8:8 => abc.c:3 - │ │ └── column7:7 => rowid:4 + │ │ ├── column1:6 => abc.a:1 + │ │ ├── column2:7 => abc.b:2 + │ │ ├── column9:9 => abc.c:3 + │ │ └── column8:8 => rowid:4 │ ├── update-mapping: - │ │ ├── column1:5 => abc.a:1 - │ │ ├── column2:6 => abc.b:2 - │ │ └── column8:8 => abc.c:3 + │ │ ├── column1:6 => abc.a:1 + │ │ ├── column2:7 => abc.b:2 + │ │ └── column9:9 => abc.c:3 │ ├── return-mapping: - │ │ ├── column1:5 => abc.a:1 - │ │ ├── column2:6 => abc.b:2 - │ │ ├── column8:8 => abc.c:3 - │ │ └── upsert_rowid:13 => rowid:4 + │ │ ├── column1:6 => abc.a:1 + │ │ ├── column2:7 => abc.b:2 + │ │ ├── column9:9 => abc.c:3 + │ │ └── upsert_rowid:15 => rowid:4 │ └── project - │ ├── columns: upsert_rowid:13 column1:5!null column2:6!null column7:7 column8:8!null abc.a:9 abc.b:10 abc.c:11 rowid:12 + │ ├── columns: upsert_rowid:15 column1:6!null column2:7!null column8:8 column9:9!null abc.a:10 abc.b:11 abc.c:12 rowid:13 crdb_internal_mvcc_timestamp:14 │ ├── left-join (hash) - │ │ ├── columns: column1:5!null column2:6!null column7:7 column8:8!null abc.a:9 abc.b:10 abc.c:11 rowid:12 + │ │ ├── columns: column1:6!null column2:7!null column8:8 column9:9!null abc.a:10 abc.b:11 abc.c:12 rowid:13 crdb_internal_mvcc_timestamp:14 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: column1:5!null column2:6!null column7:7 column8:8!null - │ │ │ ├── grouping columns: column7:7 + │ │ │ ├── columns: column1:6!null column2:7!null column8:8 column9:9!null + │ │ │ ├── grouping columns: column8:8 │ │ │ ├── project - │ │ │ │ ├── columns: column8:8!null column1:5!null column2:6!null column7:7 + │ │ │ │ ├── columns: column9:9!null column1:6!null column2:7!null column8:8 │ │ │ │ ├── project - │ │ │ │ │ ├── columns: column7:7 column1:5!null column2:6!null + │ │ │ │ │ ├── columns: column8:8 column1:6!null column2:7!null │ │ │ │ │ ├── values - │ │ │ │ │ │ ├── columns: column1:5!null column2:6!null + │ │ │ │ │ │ ├── columns: column1:6!null column2:7!null │ │ │ │ │ │ └── (1, 2) │ │ │ │ │ └── projections - │ │ │ │ │ └── unique_rowid() [as=column7:7] + │ │ │ │ │ └── unique_rowid() [as=column8:8] │ │ │ │ └── projections - │ │ │ │ └── column2:6 + 1 [as=column8:8] + │ │ │ │ └── column2:7 + 1 [as=column9:9] │ │ │ └── aggregations - │ │ │ ├── first-agg [as=column1:5] - │ │ │ │ └── column1:5 - │ │ │ ├── first-agg [as=column2:6] - │ │ │ │ └── column2:6 - │ │ │ └── first-agg [as=column8:8] - │ │ │ └── column8:8 + │ │ │ ├── first-agg [as=column1:6] + │ │ │ │ └── column1:6 + │ │ │ ├── first-agg [as=column2:7] + │ │ │ │ └── column2:7 + │ │ │ └── first-agg [as=column9:9] + │ │ │ └── column9:9 │ │ ├── scan abc - │ │ │ ├── columns: abc.a:9!null abc.b:10 abc.c:11 rowid:12!null + │ │ │ ├── columns: abc.a:10!null abc.b:11 abc.c:12 rowid:13!null crdb_internal_mvcc_timestamp:14 │ │ │ └── computed column expressions - │ │ │ └── abc.c:11 - │ │ │ └── abc.b:10 + 1 + │ │ │ └── abc.c:12 + │ │ │ └── abc.b:11 + 1 │ │ └── filters - │ │ └── column7:7 = rowid:12 + │ │ └── column8:8 = rowid:13 │ └── projections - │ └── CASE WHEN rowid:12 IS NULL THEN column7:7 ELSE rowid:12 END [as=upsert_rowid:13] + │ └── CASE WHEN rowid:13 IS NULL THEN column8:8 ELSE rowid:13 END [as=upsert_rowid:15] └── with-scan &1 - ├── columns: a:14!null b:15!null c:16!null + ├── columns: a:16!null b:17!null c:18!null └── mapping: - ├── abc.a:1 => a:14 - ├── abc.b:2 => b:15 - └── abc.c:3 => c:16 + ├── abc.a:1 => a:16 + ├── abc.b:2 => b:17 + └── abc.c:3 => c:18 # Use explicitly specified column names with secondary indexes present. Existing # values of other columns need to be fetched to delete existing index rows. @@ -1067,36 +1069,36 @@ UPSERT INTO xyz (z, x, y) VALUES (1, 2, 3) ---- upsert xyz ├── columns: - ├── canary column: 7 - ├── fetch columns: x:7 y:8 z:9 + ├── canary column: 8 + ├── fetch columns: x:8 y:9 z:10 ├── insert-mapping: - │ ├── column2:5 => x:1 - │ ├── column3:6 => y:2 - │ └── column1:4 => z:3 + │ ├── column2:6 => x:1 + │ ├── column3:7 => y:2 + │ └── column1:5 => z:3 ├── update-mapping: - │ ├── column3:6 => y:2 - │ └── column1:4 => z:3 + │ ├── column3:7 => y:2 + │ └── column1:5 => z:3 └── project - ├── columns: upsert_x:10 column1:4!null column2:5!null column3:6!null x:7 y:8 z:9 + ├── columns: upsert_x:12 column1:5!null column2:6!null column3:7!null x:8 y:9 z:10 crdb_internal_mvcc_timestamp:11 ├── left-join (hash) - │ ├── columns: column1:4!null column2:5!null column3:6!null x:7 y:8 z:9 + │ ├── columns: column1:5!null column2:6!null column3:7!null x:8 y:9 z:10 crdb_internal_mvcc_timestamp:11 │ ├── ensure-upsert-distinct-on - │ │ ├── columns: column1:4!null column2:5!null column3:6!null - │ │ ├── grouping columns: column2:5!null + │ │ ├── columns: column1:5!null column2:6!null column3:7!null + │ │ ├── grouping columns: column2:6!null │ │ ├── values - │ │ │ ├── columns: column1:4!null column2:5!null column3:6!null + │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null │ │ │ └── (1, 2, 3) │ │ └── aggregations - │ │ ├── first-agg [as=column1:4] - │ │ │ └── column1:4 - │ │ └── first-agg [as=column3:6] - │ │ └── column3:6 + │ │ ├── first-agg [as=column1:5] + │ │ │ └── column1:5 + │ │ └── first-agg [as=column3:7] + │ │ └── column3:7 │ ├── scan xyz - │ │ └── columns: x:7!null y:8 z:9 + │ │ └── columns: x:8!null y:9 z:10 crdb_internal_mvcc_timestamp:11 │ └── filters - │ └── column2:5 = x:7 + │ └── column2:6 = x:8 └── projections - └── CASE WHEN x:7 IS NULL THEN column2:5 ELSE x:7 END [as=upsert_x:10] + └── CASE WHEN x:8 IS NULL THEN column2:6 ELSE x:8 END [as=upsert_x:12] # Use explicitly specified column names with no secondary indexes present. # Upsert implemented with blind Puts is possible. @@ -1106,11 +1108,11 @@ UPSERT INTO noindex (x, y, z) VALUES (1, 2, 3) upsert noindex ├── columns: ├── upsert-mapping: - │ ├── column1:4 => x:1 - │ ├── column2:5 => y:2 - │ └── column3:6 => z:3 + │ ├── column1:5 => x:1 + │ ├── column2:6 => y:2 + │ └── column3:7 => z:3 └── values - ├── columns: column1:4!null column2:5!null column3:6!null + ├── columns: column1:5!null column2:6!null column3:7!null └── (1, 2, 3) # Use subset of explicitly specified column names with no secondary indexes @@ -1121,55 +1123,55 @@ UPSERT INTO checks (a, b, c) VALUES (1, 2, 3) ---- upsert checks ├── columns: - ├── canary column: 9 - ├── fetch columns: a:9 b:10 c:11 d:12 + ├── canary column: 10 + ├── fetch columns: a:10 b:11 c:12 d:13 ├── insert-mapping: - │ ├── column1:5 => a:1 - │ ├── column2:6 => b:2 - │ ├── column3:7 => c:3 - │ └── column8:8 => d:4 + │ ├── column1:6 => a:1 + │ ├── column2:7 => b:2 + │ ├── column3:8 => c:3 + │ └── column9:9 => d:4 ├── update-mapping: - │ ├── column2:6 => b:2 - │ ├── column3:7 => c:3 - │ └── column8:8 => d:4 - ├── check columns: check1:14 check2:15 + │ ├── column2:7 => b:2 + │ ├── column3:8 => c:3 + │ └── column9:9 => d:4 + ├── check columns: check1:16 check2:17 └── project - ├── columns: check1:14!null check2:15 column1:5!null column2:6!null column3:7!null column8:8!null a:9 b:10 c:11 d:12 upsert_a:13 + ├── columns: check1:16!null check2:17 column1:6!null column2:7!null column3:8!null column9:9!null a:10 b:11 c:12 d:13 crdb_internal_mvcc_timestamp:14 upsert_a:15 ├── project - │ ├── columns: upsert_a:13 column1:5!null column2:6!null column3:7!null column8:8!null a:9 b:10 c:11 d:12 + │ ├── columns: upsert_a:15 column1:6!null column2:7!null column3:8!null column9:9!null a:10 b:11 c:12 d:13 crdb_internal_mvcc_timestamp:14 │ ├── left-join (hash) - │ │ ├── columns: column1:5!null column2:6!null column3:7!null column8:8!null a:9 b:10 c:11 d:12 + │ │ ├── columns: column1:6!null column2:7!null column3:8!null column9:9!null a:10 b:11 c:12 d:13 crdb_internal_mvcc_timestamp:14 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null column8:8!null - │ │ │ ├── grouping columns: column1:5!null + │ │ │ ├── columns: column1:6!null column2:7!null column3:8!null column9:9!null + │ │ │ ├── grouping columns: column1:6!null │ │ │ ├── project - │ │ │ │ ├── columns: column8:8!null column1:5!null column2:6!null column3:7!null + │ │ │ │ ├── columns: column9:9!null column1:6!null column2:7!null column3:8!null │ │ │ │ ├── values - │ │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null + │ │ │ │ │ ├── columns: column1:6!null column2:7!null column3:8!null │ │ │ │ │ └── (1, 2, 3) │ │ │ │ └── projections - │ │ │ │ └── column3:7 + 1 [as=column8:8] + │ │ │ │ └── column3:8 + 1 [as=column9:9] │ │ │ └── aggregations - │ │ │ ├── first-agg [as=column2:6] - │ │ │ │ └── column2:6 - │ │ │ ├── first-agg [as=column3:7] - │ │ │ │ └── column3:7 - │ │ │ └── first-agg [as=column8:8] - │ │ │ └── column8:8 + │ │ │ ├── first-agg [as=column2:7] + │ │ │ │ └── column2:7 + │ │ │ ├── first-agg [as=column3:8] + │ │ │ │ └── column3:8 + │ │ │ └── first-agg [as=column9:9] + │ │ │ └── column9:9 │ │ ├── scan checks - │ │ │ ├── columns: a:9!null b:10 c:11 d:12 + │ │ │ ├── columns: a:10!null b:11 c:12 d:13 crdb_internal_mvcc_timestamp:14 │ │ │ ├── check constraint expressions - │ │ │ │ └── a:9 > 0 + │ │ │ │ └── a:10 > 0 │ │ │ └── computed column expressions - │ │ │ └── d:12 - │ │ │ └── c:11 + 1 + │ │ │ └── d:13 + │ │ │ └── c:12 + 1 │ │ └── filters - │ │ └── column1:5 = a:9 + │ │ └── column1:6 = a:10 │ └── projections - │ └── CASE WHEN a:9 IS NULL THEN column1:5 ELSE a:9 END [as=upsert_a:13] + │ └── CASE WHEN a:10 IS NULL THEN column1:6 ELSE a:10 END [as=upsert_a:15] └── projections - ├── column2:6 < column8:8 [as=check1:14] - └── upsert_a:13 > 0 [as=check2:15] + ├── column2:7 < column9:9 [as=check1:16] + └── upsert_a:15 > 0 [as=check2:17] # Ensure that mutation columns are set by the insert and update. Use explicit # target columns. @@ -1178,55 +1180,55 @@ UPSERT INTO mutation (m, n) VALUES (1, 2) ---- upsert mutation ├── columns: - ├── canary column: 10 - ├── fetch columns: m:10 n:11 o:12 p:13 q:14 + ├── canary column: 11 + ├── fetch columns: m:11 n:12 o:13 p:14 q:15 ├── insert-mapping: - │ ├── column1:6 => m:1 - │ ├── column2:7 => n:2 - │ ├── column8:8 => o:3 - │ └── column9:9 => p:4 + │ ├── column1:7 => m:1 + │ ├── column2:8 => n:2 + │ ├── column9:9 => o:3 + │ └── column10:10 => p:4 ├── update-mapping: - │ ├── column2:7 => n:2 - │ ├── column8:8 => o:3 - │ └── column9:9 => p:4 - ├── check columns: check1:16 + │ ├── column2:8 => n:2 + │ ├── column9:9 => o:3 + │ └── column10:10 => p:4 + ├── check columns: check1:18 └── project - ├── columns: check1:16 column1:6!null column2:7!null column8:8!null column9:9!null m:10 n:11 o:12 p:13 q:14 upsert_m:15 + ├── columns: check1:18 column1:7!null column2:8!null column9:9!null column10:10!null m:11 n:12 o:13 p:14 q:15 crdb_internal_mvcc_timestamp:16 upsert_m:17 ├── project - │ ├── columns: upsert_m:15 column1:6!null column2:7!null column8:8!null column9:9!null m:10 n:11 o:12 p:13 q:14 + │ ├── columns: upsert_m:17 column1:7!null column2:8!null column9:9!null column10:10!null m:11 n:12 o:13 p:14 q:15 crdb_internal_mvcc_timestamp:16 │ ├── left-join (hash) - │ │ ├── columns: column1:6!null column2:7!null column8:8!null column9:9!null m:10 n:11 o:12 p:13 q:14 + │ │ ├── columns: column1:7!null column2:8!null column9:9!null column10:10!null m:11 n:12 o:13 p:14 q:15 crdb_internal_mvcc_timestamp:16 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: column1:6!null column2:7!null column8:8!null column9:9!null - │ │ │ ├── grouping columns: column1:6!null + │ │ │ ├── columns: column1:7!null column2:8!null column9:9!null column10:10!null + │ │ │ ├── grouping columns: column1:7!null │ │ │ ├── project - │ │ │ │ ├── columns: column9:9!null column1:6!null column2:7!null column8:8!null + │ │ │ │ ├── columns: column10:10!null column1:7!null column2:8!null column9:9!null │ │ │ │ ├── project - │ │ │ │ │ ├── columns: column8:8!null column1:6!null column2:7!null + │ │ │ │ │ ├── columns: column9:9!null column1:7!null column2:8!null │ │ │ │ │ ├── values - │ │ │ │ │ │ ├── columns: column1:6!null column2:7!null + │ │ │ │ │ │ ├── columns: column1:7!null column2:8!null │ │ │ │ │ │ └── (1, 2) │ │ │ │ │ └── projections - │ │ │ │ │ └── 10 [as=column8:8] + │ │ │ │ │ └── 10 [as=column9:9] │ │ │ │ └── projections - │ │ │ │ └── column8:8 + column2:7 [as=column9:9] + │ │ │ │ └── column9:9 + column2:8 [as=column10:10] │ │ │ └── aggregations - │ │ │ ├── first-agg [as=column2:7] - │ │ │ │ └── column2:7 - │ │ │ ├── first-agg [as=column8:8] - │ │ │ │ └── column8:8 - │ │ │ └── first-agg [as=column9:9] - │ │ │ └── column9:9 + │ │ │ ├── first-agg [as=column2:8] + │ │ │ │ └── column2:8 + │ │ │ ├── first-agg [as=column9:9] + │ │ │ │ └── column9:9 + │ │ │ └── first-agg [as=column10:10] + │ │ │ └── column10:10 │ │ ├── scan mutation - │ │ │ ├── columns: m:10!null n:11 o:12 p:13 q:14 + │ │ │ ├── columns: m:11!null n:12 o:13 p:14 q:15 crdb_internal_mvcc_timestamp:16 │ │ │ └── check constraint expressions - │ │ │ └── m:10 > 0 + │ │ │ └── m:11 > 0 │ │ └── filters - │ │ └── column1:6 = m:10 + │ │ └── column1:7 = m:11 │ └── projections - │ └── CASE WHEN m:10 IS NULL THEN column1:6 ELSE m:10 END [as=upsert_m:15] + │ └── CASE WHEN m:11 IS NULL THEN column1:7 ELSE m:11 END [as=upsert_m:17] └── projections - └── upsert_m:15 > 0 [as=check1:16] + └── upsert_m:17 > 0 [as=check1:18] # Don't directly update mutation columns. However, computed columns do need to # be updated. Use implicit target columns. @@ -1235,55 +1237,55 @@ UPSERT INTO mutation VALUES (1, 2) ---- upsert mutation ├── columns: - ├── canary column: 10 - ├── fetch columns: m:10 n:11 o:12 p:13 q:14 + ├── canary column: 11 + ├── fetch columns: m:11 n:12 o:13 p:14 q:15 ├── insert-mapping: - │ ├── column1:6 => m:1 - │ ├── column2:7 => n:2 - │ ├── column8:8 => o:3 - │ └── column9:9 => p:4 + │ ├── column1:7 => m:1 + │ ├── column2:8 => n:2 + │ ├── column9:9 => o:3 + │ └── column10:10 => p:4 ├── update-mapping: - │ ├── column2:7 => n:2 - │ ├── column8:8 => o:3 - │ └── column9:9 => p:4 - ├── check columns: check1:16 + │ ├── column2:8 => n:2 + │ ├── column9:9 => o:3 + │ └── column10:10 => p:4 + ├── check columns: check1:18 └── project - ├── columns: check1:16 column1:6!null column2:7!null column8:8!null column9:9!null m:10 n:11 o:12 p:13 q:14 upsert_m:15 + ├── columns: check1:18 column1:7!null column2:8!null column9:9!null column10:10!null m:11 n:12 o:13 p:14 q:15 crdb_internal_mvcc_timestamp:16 upsert_m:17 ├── project - │ ├── columns: upsert_m:15 column1:6!null column2:7!null column8:8!null column9:9!null m:10 n:11 o:12 p:13 q:14 + │ ├── columns: upsert_m:17 column1:7!null column2:8!null column9:9!null column10:10!null m:11 n:12 o:13 p:14 q:15 crdb_internal_mvcc_timestamp:16 │ ├── left-join (hash) - │ │ ├── columns: column1:6!null column2:7!null column8:8!null column9:9!null m:10 n:11 o:12 p:13 q:14 + │ │ ├── columns: column1:7!null column2:8!null column9:9!null column10:10!null m:11 n:12 o:13 p:14 q:15 crdb_internal_mvcc_timestamp:16 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: column1:6!null column2:7!null column8:8!null column9:9!null - │ │ │ ├── grouping columns: column1:6!null + │ │ │ ├── columns: column1:7!null column2:8!null column9:9!null column10:10!null + │ │ │ ├── grouping columns: column1:7!null │ │ │ ├── project - │ │ │ │ ├── columns: column9:9!null column1:6!null column2:7!null column8:8!null + │ │ │ │ ├── columns: column10:10!null column1:7!null column2:8!null column9:9!null │ │ │ │ ├── project - │ │ │ │ │ ├── columns: column8:8!null column1:6!null column2:7!null + │ │ │ │ │ ├── columns: column9:9!null column1:7!null column2:8!null │ │ │ │ │ ├── values - │ │ │ │ │ │ ├── columns: column1:6!null column2:7!null + │ │ │ │ │ │ ├── columns: column1:7!null column2:8!null │ │ │ │ │ │ └── (1, 2) │ │ │ │ │ └── projections - │ │ │ │ │ └── 10 [as=column8:8] + │ │ │ │ │ └── 10 [as=column9:9] │ │ │ │ └── projections - │ │ │ │ └── column8:8 + column2:7 [as=column9:9] + │ │ │ │ └── column9:9 + column2:8 [as=column10:10] │ │ │ └── aggregations - │ │ │ ├── first-agg [as=column2:7] - │ │ │ │ └── column2:7 - │ │ │ ├── first-agg [as=column8:8] - │ │ │ │ └── column8:8 - │ │ │ └── first-agg [as=column9:9] - │ │ │ └── column9:9 + │ │ │ ├── first-agg [as=column2:8] + │ │ │ │ └── column2:8 + │ │ │ ├── first-agg [as=column9:9] + │ │ │ │ └── column9:9 + │ │ │ └── first-agg [as=column10:10] + │ │ │ └── column10:10 │ │ ├── scan mutation - │ │ │ ├── columns: m:10!null n:11 o:12 p:13 q:14 + │ │ │ ├── columns: m:11!null n:12 o:13 p:14 q:15 crdb_internal_mvcc_timestamp:16 │ │ │ └── check constraint expressions - │ │ │ └── m:10 > 0 + │ │ │ └── m:11 > 0 │ │ └── filters - │ │ └── column1:6 = m:10 + │ │ └── column1:7 = m:11 │ └── projections - │ └── CASE WHEN m:10 IS NULL THEN column1:6 ELSE m:10 END [as=upsert_m:15] + │ └── CASE WHEN m:11 IS NULL THEN column1:7 ELSE m:11 END [as=upsert_m:17] └── projections - └── upsert_m:15 > 0 [as=check1:16] + └── upsert_m:17 > 0 [as=check1:18] # Use unknown name in upsert column list. build @@ -1301,71 +1303,71 @@ INSERT INTO checks (a, b) VALUES (1, 2) ON CONFLICT (a) DO UPDATE SET b=3, c=4 ---- upsert checks ├── columns: - ├── canary column: 9 - ├── fetch columns: a:9 b:10 c:11 d:12 + ├── canary column: 10 + ├── fetch columns: a:10 b:11 c:12 d:13 ├── insert-mapping: - │ ├── column1:5 => a:1 - │ ├── column2:6 => b:2 - │ ├── column7:7 => c:3 - │ └── column8:8 => d:4 + │ ├── column1:6 => a:1 + │ ├── column2:7 => b:2 + │ ├── column8:8 => c:3 + │ └── column9:9 => d:4 ├── update-mapping: - │ ├── upsert_b:17 => b:2 - │ ├── upsert_c:18 => c:3 - │ └── upsert_d:19 => d:4 - ├── check columns: check1:20 check2:21 + │ ├── upsert_b:19 => b:2 + │ ├── upsert_c:20 => c:3 + │ └── upsert_d:21 => d:4 + ├── check columns: check1:22 check2:23 └── project - ├── columns: check1:20 check2:21 column1:5!null column2:6!null column7:7 column8:8 a:9 b:10 c:11 d:12 b_new:13!null c_new:14!null column15:15!null upsert_a:16 upsert_b:17!null upsert_c:18 upsert_d:19 + ├── columns: check1:22 check2:23 column1:6!null column2:7!null column8:8 column9:9 a:10 b:11 c:12 d:13 crdb_internal_mvcc_timestamp:14 b_new:15!null c_new:16!null column17:17!null upsert_a:18 upsert_b:19!null upsert_c:20 upsert_d:21 ├── project - │ ├── columns: upsert_a:16 upsert_b:17!null upsert_c:18 upsert_d:19 column1:5!null column2:6!null column7:7 column8:8 a:9 b:10 c:11 d:12 b_new:13!null c_new:14!null column15:15!null + │ ├── columns: upsert_a:18 upsert_b:19!null upsert_c:20 upsert_d:21 column1:6!null column2:7!null column8:8 column9:9 a:10 b:11 c:12 d:13 crdb_internal_mvcc_timestamp:14 b_new:15!null c_new:16!null column17:17!null │ ├── project - │ │ ├── columns: column15:15!null column1:5!null column2:6!null column7:7 column8:8 a:9 b:10 c:11 d:12 b_new:13!null c_new:14!null + │ │ ├── columns: column17:17!null column1:6!null column2:7!null column8:8 column9:9 a:10 b:11 c:12 d:13 crdb_internal_mvcc_timestamp:14 b_new:15!null c_new:16!null │ │ ├── project - │ │ │ ├── columns: b_new:13!null c_new:14!null column1:5!null column2:6!null column7:7 column8:8 a:9 b:10 c:11 d:12 + │ │ │ ├── columns: b_new:15!null c_new:16!null column1:6!null column2:7!null column8:8 column9:9 a:10 b:11 c:12 d:13 crdb_internal_mvcc_timestamp:14 │ │ │ ├── left-join (hash) - │ │ │ │ ├── columns: column1:5!null column2:6!null column7:7 column8:8 a:9 b:10 c:11 d:12 + │ │ │ │ ├── columns: column1:6!null column2:7!null column8:8 column9:9 a:10 b:11 c:12 d:13 crdb_internal_mvcc_timestamp:14 │ │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ │ ├── columns: column1:5!null column2:6!null column7:7 column8:8 - │ │ │ │ │ ├── grouping columns: column1:5!null + │ │ │ │ │ ├── columns: column1:6!null column2:7!null column8:8 column9:9 + │ │ │ │ │ ├── grouping columns: column1:6!null │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: column8:8 column1:5!null column2:6!null column7:7 + │ │ │ │ │ │ ├── columns: column9:9 column1:6!null column2:7!null column8:8 │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ ├── columns: column7:7 column1:5!null column2:6!null + │ │ │ │ │ │ │ ├── columns: column8:8 column1:6!null column2:7!null │ │ │ │ │ │ │ ├── values - │ │ │ │ │ │ │ │ ├── columns: column1:5!null column2:6!null + │ │ │ │ │ │ │ │ ├── columns: column1:6!null column2:7!null │ │ │ │ │ │ │ │ └── (1, 2) │ │ │ │ │ │ │ └── projections - │ │ │ │ │ │ │ └── NULL::INT8 [as=column7:7] + │ │ │ │ │ │ │ └── NULL::INT8 [as=column8:8] │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── column7:7 + 1 [as=column8:8] + │ │ │ │ │ │ └── column8:8 + 1 [as=column9:9] │ │ │ │ │ └── aggregations - │ │ │ │ │ ├── first-agg [as=column2:6] - │ │ │ │ │ │ └── column2:6 - │ │ │ │ │ ├── first-agg [as=column7:7] - │ │ │ │ │ │ └── column7:7 - │ │ │ │ │ └── first-agg [as=column8:8] - │ │ │ │ │ └── column8:8 + │ │ │ │ │ ├── first-agg [as=column2:7] + │ │ │ │ │ │ └── column2:7 + │ │ │ │ │ ├── first-agg [as=column8:8] + │ │ │ │ │ │ └── column8:8 + │ │ │ │ │ └── first-agg [as=column9:9] + │ │ │ │ │ └── column9:9 │ │ │ │ ├── scan checks - │ │ │ │ │ ├── columns: a:9!null b:10 c:11 d:12 + │ │ │ │ │ ├── columns: a:10!null b:11 c:12 d:13 crdb_internal_mvcc_timestamp:14 │ │ │ │ │ ├── check constraint expressions - │ │ │ │ │ │ └── a:9 > 0 + │ │ │ │ │ │ └── a:10 > 0 │ │ │ │ │ └── computed column expressions - │ │ │ │ │ └── d:12 - │ │ │ │ │ └── c:11 + 1 + │ │ │ │ │ └── d:13 + │ │ │ │ │ └── c:12 + 1 │ │ │ │ └── filters - │ │ │ │ └── column1:5 = a:9 + │ │ │ │ └── column1:6 = a:10 │ │ │ └── projections - │ │ │ ├── 3 [as=b_new:13] - │ │ │ └── 4 [as=c_new:14] + │ │ │ ├── 3 [as=b_new:15] + │ │ │ └── 4 [as=c_new:16] │ │ └── projections - │ │ └── c_new:14 + 1 [as=column15:15] + │ │ └── c_new:16 + 1 [as=column17:17] │ └── projections - │ ├── CASE WHEN a:9 IS NULL THEN column1:5 ELSE a:9 END [as=upsert_a:16] - │ ├── CASE WHEN a:9 IS NULL THEN column2:6 ELSE b_new:13 END [as=upsert_b:17] - │ ├── CASE WHEN a:9 IS NULL THEN column7:7 ELSE c_new:14 END [as=upsert_c:18] - │ └── CASE WHEN a:9 IS NULL THEN column8:8 ELSE column15:15 END [as=upsert_d:19] + │ ├── CASE WHEN a:10 IS NULL THEN column1:6 ELSE a:10 END [as=upsert_a:18] + │ ├── CASE WHEN a:10 IS NULL THEN column2:7 ELSE b_new:15 END [as=upsert_b:19] + │ ├── CASE WHEN a:10 IS NULL THEN column8:8 ELSE c_new:16 END [as=upsert_c:20] + │ └── CASE WHEN a:10 IS NULL THEN column9:9 ELSE column17:17 END [as=upsert_d:21] └── projections - ├── upsert_b:17 < upsert_d:19 [as=check1:20] - └── upsert_a:16 > 0 [as=check2:21] + ├── upsert_b:19 < upsert_d:21 [as=check1:22] + └── upsert_a:18 > 0 [as=check2:23] # INSERT..ON CONFLICT DO NOTHING build @@ -1374,54 +1376,54 @@ INSERT INTO checks (a, b) VALUES (1, 2) ON CONFLICT (a) DO NOTHING insert checks ├── columns: ├── insert-mapping: - │ ├── column1:5 => a:1 - │ ├── column2:6 => b:2 - │ ├── column7:7 => c:3 - │ └── column8:8 => d:4 - ├── check columns: check1:13 check2:14 + │ ├── column1:6 => a:1 + │ ├── column2:7 => b:2 + │ ├── column8:8 => c:3 + │ └── column9:9 => d:4 + ├── check columns: check1:15 check2:16 └── project - ├── columns: check1:13 check2:14!null column1:5!null column2:6!null column7:7 column8:8 + ├── columns: check1:15 check2:16!null column1:6!null column2:7!null column8:8 column9:9 ├── upsert-distinct-on - │ ├── columns: column1:5!null column2:6!null column7:7 column8:8 - │ ├── grouping columns: column1:5!null + │ ├── columns: column1:6!null column2:7!null column8:8 column9:9 + │ ├── grouping columns: column1:6!null │ ├── project - │ │ ├── columns: column1:5!null column2:6!null column7:7 column8:8 + │ │ ├── columns: column1:6!null column2:7!null column8:8 column9:9 │ │ └── select - │ │ ├── columns: column1:5!null column2:6!null column7:7 column8:8 a:9 b:10 c:11 d:12 + │ │ ├── columns: column1:6!null column2:7!null column8:8 column9:9 a:10 b:11 c:12 d:13 crdb_internal_mvcc_timestamp:14 │ │ ├── left-join (hash) - │ │ │ ├── columns: column1:5!null column2:6!null column7:7 column8:8 a:9 b:10 c:11 d:12 + │ │ │ ├── columns: column1:6!null column2:7!null column8:8 column9:9 a:10 b:11 c:12 d:13 crdb_internal_mvcc_timestamp:14 │ │ │ ├── project - │ │ │ │ ├── columns: column8:8 column1:5!null column2:6!null column7:7 + │ │ │ │ ├── columns: column9:9 column1:6!null column2:7!null column8:8 │ │ │ │ ├── project - │ │ │ │ │ ├── columns: column7:7 column1:5!null column2:6!null + │ │ │ │ │ ├── columns: column8:8 column1:6!null column2:7!null │ │ │ │ │ ├── values - │ │ │ │ │ │ ├── columns: column1:5!null column2:6!null + │ │ │ │ │ │ ├── columns: column1:6!null column2:7!null │ │ │ │ │ │ └── (1, 2) │ │ │ │ │ └── projections - │ │ │ │ │ └── NULL::INT8 [as=column7:7] + │ │ │ │ │ └── NULL::INT8 [as=column8:8] │ │ │ │ └── projections - │ │ │ │ └── column7:7 + 1 [as=column8:8] + │ │ │ │ └── column8:8 + 1 [as=column9:9] │ │ │ ├── scan checks - │ │ │ │ ├── columns: a:9!null b:10 c:11 d:12 + │ │ │ │ ├── columns: a:10!null b:11 c:12 d:13 crdb_internal_mvcc_timestamp:14 │ │ │ │ ├── check constraint expressions - │ │ │ │ │ └── a:9 > 0 + │ │ │ │ │ └── a:10 > 0 │ │ │ │ └── computed column expressions - │ │ │ │ └── d:12 - │ │ │ │ └── c:11 + 1 + │ │ │ │ └── d:13 + │ │ │ │ └── c:12 + 1 │ │ │ └── filters - │ │ │ └── column1:5 = a:9 + │ │ │ └── column1:6 = a:10 │ │ └── filters - │ │ └── a:9 IS NULL + │ │ └── a:10 IS NULL │ └── aggregations - │ ├── first-agg [as=column2:6] - │ │ └── column2:6 - │ ├── first-agg [as=column7:7] - │ │ └── column7:7 - │ └── first-agg [as=column8:8] - │ └── column8:8 + │ ├── first-agg [as=column2:7] + │ │ └── column2:7 + │ ├── first-agg [as=column8:8] + │ │ └── column8:8 + │ └── first-agg [as=column9:9] + │ └── column9:9 └── projections - ├── column2:6 < column8:8 [as=check1:13] - └── column1:5 > 0 [as=check2:14] + ├── column2:7 < column9:9 [as=check1:15] + └── column1:6 > 0 [as=check2:16] # UPSERT build @@ -1429,64 +1431,64 @@ UPSERT INTO checks (a, b) VALUES (1, 2) ---- upsert checks ├── columns: - ├── canary column: 9 - ├── fetch columns: a:9 b:10 c:11 d:12 + ├── canary column: 10 + ├── fetch columns: a:10 b:11 c:12 d:13 ├── insert-mapping: - │ ├── column1:5 => a:1 - │ ├── column2:6 => b:2 - │ ├── column7:7 => c:3 - │ └── column8:8 => d:4 + │ ├── column1:6 => a:1 + │ ├── column2:7 => b:2 + │ ├── column8:8 => c:3 + │ └── column9:9 => d:4 ├── update-mapping: - │ ├── column2:6 => b:2 - │ └── upsert_d:16 => d:4 - ├── check columns: check1:17 check2:18 + │ ├── column2:7 => b:2 + │ └── upsert_d:18 => d:4 + ├── check columns: check1:19 check2:20 └── project - ├── columns: check1:17 check2:18 column1:5!null column2:6!null column7:7 column8:8 a:9 b:10 c:11 d:12 column13:13 upsert_a:14 upsert_c:15 upsert_d:16 + ├── columns: check1:19 check2:20 column1:6!null column2:7!null column8:8 column9:9 a:10 b:11 c:12 d:13 crdb_internal_mvcc_timestamp:14 column15:15 upsert_a:16 upsert_c:17 upsert_d:18 ├── project - │ ├── columns: upsert_a:14 upsert_c:15 upsert_d:16 column1:5!null column2:6!null column7:7 column8:8 a:9 b:10 c:11 d:12 column13:13 + │ ├── columns: upsert_a:16 upsert_c:17 upsert_d:18 column1:6!null column2:7!null column8:8 column9:9 a:10 b:11 c:12 d:13 crdb_internal_mvcc_timestamp:14 column15:15 │ ├── project - │ │ ├── columns: column13:13 column1:5!null column2:6!null column7:7 column8:8 a:9 b:10 c:11 d:12 + │ │ ├── columns: column15:15 column1:6!null column2:7!null column8:8 column9:9 a:10 b:11 c:12 d:13 crdb_internal_mvcc_timestamp:14 │ │ ├── left-join (hash) - │ │ │ ├── columns: column1:5!null column2:6!null column7:7 column8:8 a:9 b:10 c:11 d:12 + │ │ │ ├── columns: column1:6!null column2:7!null column8:8 column9:9 a:10 b:11 c:12 d:13 crdb_internal_mvcc_timestamp:14 │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ ├── columns: column1:5!null column2:6!null column7:7 column8:8 - │ │ │ │ ├── grouping columns: column1:5!null + │ │ │ │ ├── columns: column1:6!null column2:7!null column8:8 column9:9 + │ │ │ │ ├── grouping columns: column1:6!null │ │ │ │ ├── project - │ │ │ │ │ ├── columns: column8:8 column1:5!null column2:6!null column7:7 + │ │ │ │ │ ├── columns: column9:9 column1:6!null column2:7!null column8:8 │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: column7:7 column1:5!null column2:6!null + │ │ │ │ │ │ ├── columns: column8:8 column1:6!null column2:7!null │ │ │ │ │ │ ├── values - │ │ │ │ │ │ │ ├── columns: column1:5!null column2:6!null + │ │ │ │ │ │ │ ├── columns: column1:6!null column2:7!null │ │ │ │ │ │ │ └── (1, 2) │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── NULL::INT8 [as=column7:7] + │ │ │ │ │ │ └── NULL::INT8 [as=column8:8] │ │ │ │ │ └── projections - │ │ │ │ │ └── column7:7 + 1 [as=column8:8] + │ │ │ │ │ └── column8:8 + 1 [as=column9:9] │ │ │ │ └── aggregations - │ │ │ │ ├── first-agg [as=column2:6] - │ │ │ │ │ └── column2:6 - │ │ │ │ ├── first-agg [as=column7:7] - │ │ │ │ │ └── column7:7 - │ │ │ │ └── first-agg [as=column8:8] - │ │ │ │ └── column8:8 + │ │ │ │ ├── first-agg [as=column2:7] + │ │ │ │ │ └── column2:7 + │ │ │ │ ├── first-agg [as=column8:8] + │ │ │ │ │ └── column8:8 + │ │ │ │ └── first-agg [as=column9:9] + │ │ │ │ └── column9:9 │ │ │ ├── scan checks - │ │ │ │ ├── columns: a:9!null b:10 c:11 d:12 + │ │ │ │ ├── columns: a:10!null b:11 c:12 d:13 crdb_internal_mvcc_timestamp:14 │ │ │ │ ├── check constraint expressions - │ │ │ │ │ └── a:9 > 0 + │ │ │ │ │ └── a:10 > 0 │ │ │ │ └── computed column expressions - │ │ │ │ └── d:12 - │ │ │ │ └── c:11 + 1 + │ │ │ │ └── d:13 + │ │ │ │ └── c:12 + 1 │ │ │ └── filters - │ │ │ └── column1:5 = a:9 + │ │ │ └── column1:6 = a:10 │ │ └── projections - │ │ └── c:11 + 1 [as=column13:13] + │ │ └── c:12 + 1 [as=column15:15] │ └── projections - │ ├── CASE WHEN a:9 IS NULL THEN column1:5 ELSE a:9 END [as=upsert_a:14] - │ ├── CASE WHEN a:9 IS NULL THEN column7:7 ELSE c:11 END [as=upsert_c:15] - │ └── CASE WHEN a:9 IS NULL THEN column8:8 ELSE column13:13 END [as=upsert_d:16] + │ ├── CASE WHEN a:10 IS NULL THEN column1:6 ELSE a:10 END [as=upsert_a:16] + │ ├── CASE WHEN a:10 IS NULL THEN column8:8 ELSE c:12 END [as=upsert_c:17] + │ └── CASE WHEN a:10 IS NULL THEN column9:9 ELSE column15:15 END [as=upsert_d:18] └── projections - ├── column2:6 < upsert_d:16 [as=check1:17] - └── upsert_a:14 > 0 [as=check2:18] + ├── column2:7 < upsert_d:18 [as=check1:19] + └── upsert_a:16 > 0 [as=check2:20] # Use subqueries and excluded. build @@ -1496,83 +1498,83 @@ ON CONFLICT (a) DO UPDATE SET a=excluded.a, b=(SELECT x FROM xyz WHERE x=checks. ---- upsert checks ├── columns: - ├── canary column: 11 - ├── fetch columns: checks.a:11 checks.b:12 checks.c:13 d:14 + ├── canary column: 13 + ├── fetch columns: checks.a:13 checks.b:14 checks.c:15 d:16 ├── insert-mapping: - │ ├── abc.a:5 => checks.a:1 - │ ├── abc.b:6 => checks.b:2 - │ ├── column9:9 => checks.c:3 - │ └── column10:10 => d:4 + │ ├── abc.a:6 => checks.a:1 + │ ├── abc.b:7 => checks.b:2 + │ ├── column11:11 => checks.c:3 + │ └── column12:12 => d:4 ├── update-mapping: - │ ├── abc.a:5 => checks.a:1 - │ ├── upsert_b:20 => checks.b:2 - │ └── upsert_d:22 => d:4 - ├── check columns: check1:23 check2:24 + │ ├── abc.a:6 => checks.a:1 + │ ├── upsert_b:24 => checks.b:2 + │ └── upsert_d:26 => d:4 + ├── check columns: check1:27 check2:28 └── project - ├── columns: check1:23 check2:24!null abc.a:5!null abc.b:6 column9:9 column10:10 checks.a:11 checks.b:12 checks.c:13 d:14 b_new:18 column19:19 upsert_b:20 upsert_c:21 upsert_d:22 + ├── columns: check1:27 check2:28!null abc.a:6!null abc.b:7 column11:11 column12:12 checks.a:13 checks.b:14 checks.c:15 d:16 checks.crdb_internal_mvcc_timestamp:17 b_new:22 column23:23 upsert_b:24 upsert_c:25 upsert_d:26 ├── project - │ ├── columns: upsert_b:20 upsert_c:21 upsert_d:22 abc.a:5!null abc.b:6 column9:9 column10:10 checks.a:11 checks.b:12 checks.c:13 d:14 b_new:18 column19:19 + │ ├── columns: upsert_b:24 upsert_c:25 upsert_d:26 abc.a:6!null abc.b:7 column11:11 column12:12 checks.a:13 checks.b:14 checks.c:15 d:16 checks.crdb_internal_mvcc_timestamp:17 b_new:22 column23:23 │ ├── project - │ │ ├── columns: column19:19 abc.a:5!null abc.b:6 column9:9 column10:10 checks.a:11 checks.b:12 checks.c:13 d:14 b_new:18 + │ │ ├── columns: column23:23 abc.a:6!null abc.b:7 column11:11 column12:12 checks.a:13 checks.b:14 checks.c:15 d:16 checks.crdb_internal_mvcc_timestamp:17 b_new:22 │ │ ├── project - │ │ │ ├── columns: b_new:18 abc.a:5!null abc.b:6 column9:9 column10:10 checks.a:11 checks.b:12 checks.c:13 d:14 + │ │ │ ├── columns: b_new:22 abc.a:6!null abc.b:7 column11:11 column12:12 checks.a:13 checks.b:14 checks.c:15 d:16 checks.crdb_internal_mvcc_timestamp:17 │ │ │ ├── left-join (hash) - │ │ │ │ ├── columns: abc.a:5!null abc.b:6 column9:9 column10:10 checks.a:11 checks.b:12 checks.c:13 d:14 + │ │ │ │ ├── columns: abc.a:6!null abc.b:7 column11:11 column12:12 checks.a:13 checks.b:14 checks.c:15 d:16 checks.crdb_internal_mvcc_timestamp:17 │ │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ │ ├── columns: abc.a:5!null abc.b:6 column9:9 column10:10 - │ │ │ │ │ ├── grouping columns: abc.a:5!null + │ │ │ │ │ ├── columns: abc.a:6!null abc.b:7 column11:11 column12:12 + │ │ │ │ │ ├── grouping columns: abc.a:6!null │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: column10:10 abc.a:5!null abc.b:6 column9:9 + │ │ │ │ │ │ ├── columns: column12:12 abc.a:6!null abc.b:7 column11:11 │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ ├── columns: column9:9 abc.a:5!null abc.b:6 + │ │ │ │ │ │ │ ├── columns: column11:11 abc.a:6!null abc.b:7 │ │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ │ ├── columns: abc.a:5!null abc.b:6 + │ │ │ │ │ │ │ │ ├── columns: abc.a:6!null abc.b:7 │ │ │ │ │ │ │ │ └── scan abc - │ │ │ │ │ │ │ │ ├── columns: abc.a:5!null abc.b:6 abc.c:7 rowid:8!null + │ │ │ │ │ │ │ │ ├── columns: abc.a:6!null abc.b:7 abc.c:8 rowid:9!null abc.crdb_internal_mvcc_timestamp:10 │ │ │ │ │ │ │ │ └── computed column expressions - │ │ │ │ │ │ │ │ └── abc.c:7 - │ │ │ │ │ │ │ │ └── abc.b:6 + 1 + │ │ │ │ │ │ │ │ └── abc.c:8 + │ │ │ │ │ │ │ │ └── abc.b:7 + 1 │ │ │ │ │ │ │ └── projections - │ │ │ │ │ │ │ └── NULL::INT8 [as=column9:9] + │ │ │ │ │ │ │ └── NULL::INT8 [as=column11:11] │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── column9:9 + 1 [as=column10:10] + │ │ │ │ │ │ └── column11:11 + 1 [as=column12:12] │ │ │ │ │ └── aggregations - │ │ │ │ │ ├── first-agg [as=abc.b:6] - │ │ │ │ │ │ └── abc.b:6 - │ │ │ │ │ ├── first-agg [as=column9:9] - │ │ │ │ │ │ └── column9:9 - │ │ │ │ │ └── first-agg [as=column10:10] - │ │ │ │ │ └── column10:10 + │ │ │ │ │ ├── first-agg [as=abc.b:7] + │ │ │ │ │ │ └── abc.b:7 + │ │ │ │ │ ├── first-agg [as=column11:11] + │ │ │ │ │ │ └── column11:11 + │ │ │ │ │ └── first-agg [as=column12:12] + │ │ │ │ │ └── column12:12 │ │ │ │ ├── scan checks - │ │ │ │ │ ├── columns: checks.a:11!null checks.b:12 checks.c:13 d:14 + │ │ │ │ │ ├── columns: checks.a:13!null checks.b:14 checks.c:15 d:16 checks.crdb_internal_mvcc_timestamp:17 │ │ │ │ │ ├── check constraint expressions - │ │ │ │ │ │ └── checks.a:11 > 0 + │ │ │ │ │ │ └── checks.a:13 > 0 │ │ │ │ │ └── computed column expressions - │ │ │ │ │ └── d:14 - │ │ │ │ │ └── checks.c:13 + 1 + │ │ │ │ │ └── d:16 + │ │ │ │ │ └── checks.c:15 + 1 │ │ │ │ └── filters - │ │ │ │ └── abc.a:5 = checks.a:11 + │ │ │ │ └── abc.a:6 = checks.a:13 │ │ │ └── projections - │ │ │ └── subquery [as=b_new:18] + │ │ │ └── subquery [as=b_new:22] │ │ │ └── max1-row - │ │ │ ├── columns: x:15!null + │ │ │ ├── columns: x:18!null │ │ │ └── project - │ │ │ ├── columns: x:15!null + │ │ │ ├── columns: x:18!null │ │ │ └── select - │ │ │ ├── columns: x:15!null y:16 z:17 + │ │ │ ├── columns: x:18!null y:19 z:20 xyz.crdb_internal_mvcc_timestamp:21 │ │ │ ├── scan xyz - │ │ │ │ └── columns: x:15!null y:16 z:17 + │ │ │ │ └── columns: x:18!null y:19 z:20 xyz.crdb_internal_mvcc_timestamp:21 │ │ │ └── filters - │ │ │ └── x:15 = checks.a:11 + │ │ │ └── x:18 = checks.a:13 │ │ └── projections - │ │ └── checks.c:13 + 1 [as=column19:19] + │ │ └── checks.c:15 + 1 [as=column23:23] │ └── projections - │ ├── CASE WHEN checks.a:11 IS NULL THEN abc.b:6 ELSE b_new:18 END [as=upsert_b:20] - │ ├── CASE WHEN checks.a:11 IS NULL THEN column9:9 ELSE checks.c:13 END [as=upsert_c:21] - │ └── CASE WHEN checks.a:11 IS NULL THEN column10:10 ELSE column19:19 END [as=upsert_d:22] + │ ├── CASE WHEN checks.a:13 IS NULL THEN abc.b:7 ELSE b_new:22 END [as=upsert_b:24] + │ ├── CASE WHEN checks.a:13 IS NULL THEN column11:11 ELSE checks.c:15 END [as=upsert_c:25] + │ └── CASE WHEN checks.a:13 IS NULL THEN column12:12 ELSE column23:23 END [as=upsert_d:26] └── projections - ├── upsert_b:20 < upsert_d:22 [as=check1:23] - └── abc.a:5 > 0 [as=check2:24] + ├── upsert_b:24 < upsert_d:26 [as=check1:27] + └── abc.a:6 > 0 [as=check2:28] # Use ORDER BY in upsert input (should be ignored and not cause error). build @@ -1582,44 +1584,44 @@ ON CONFLICT (z, y) DO UPDATE SET y=5 ---- upsert xyz ├── columns: - ├── canary column: 8 - ├── fetch columns: x:8 y:9 z:10 + ├── canary column: 10 + ├── fetch columns: x:10 y:11 z:12 ├── insert-mapping: - │ ├── a:4 => x:1 - │ ├── b:5 => y:2 - │ └── c:6 => z:3 + │ ├── a:5 => x:1 + │ ├── b:6 => y:2 + │ └── c:7 => z:3 ├── update-mapping: - │ └── upsert_y:13 => y:2 + │ └── upsert_y:16 => y:2 └── project - ├── columns: upsert_x:12 upsert_y:13 upsert_z:14 a:4!null b:5 c:6 x:8 y:9 z:10 y_new:11!null + ├── columns: upsert_x:15 upsert_y:16 upsert_z:17 a:5!null b:6 c:7 x:10 y:11 z:12 xyz.crdb_internal_mvcc_timestamp:13 y_new:14!null ├── project - │ ├── columns: y_new:11!null a:4!null b:5 c:6 x:8 y:9 z:10 + │ ├── columns: y_new:14!null a:5!null b:6 c:7 x:10 y:11 z:12 xyz.crdb_internal_mvcc_timestamp:13 │ ├── left-join (hash) - │ │ ├── columns: a:4!null b:5 c:6 x:8 y:9 z:10 + │ │ ├── columns: a:5!null b:6 c:7 x:10 y:11 z:12 xyz.crdb_internal_mvcc_timestamp:13 │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: a:4!null b:5 c:6 - │ │ │ ├── grouping columns: b:5 c:6 + │ │ │ ├── columns: a:5!null b:6 c:7 + │ │ │ ├── grouping columns: b:6 c:7 │ │ │ ├── project - │ │ │ │ ├── columns: a:4!null b:5 c:6 + │ │ │ │ ├── columns: a:5!null b:6 c:7 │ │ │ │ └── scan abc - │ │ │ │ ├── columns: a:4!null b:5 c:6 rowid:7!null + │ │ │ │ ├── columns: a:5!null b:6 c:7 rowid:8!null abc.crdb_internal_mvcc_timestamp:9 │ │ │ │ └── computed column expressions - │ │ │ │ └── c:6 - │ │ │ │ └── b:5 + 1 + │ │ │ │ └── c:7 + │ │ │ │ └── b:6 + 1 │ │ │ └── aggregations - │ │ │ └── first-agg [as=a:4] - │ │ │ └── a:4 + │ │ │ └── first-agg [as=a:5] + │ │ │ └── a:5 │ │ ├── scan xyz - │ │ │ └── columns: x:8!null y:9 z:10 + │ │ │ └── columns: x:10!null y:11 z:12 xyz.crdb_internal_mvcc_timestamp:13 │ │ └── filters - │ │ ├── b:5 = y:9 - │ │ └── c:6 = z:10 + │ │ ├── b:6 = y:11 + │ │ └── c:7 = z:12 │ └── projections - │ └── 5 [as=y_new:11] + │ └── 5 [as=y_new:14] └── projections - ├── CASE WHEN x:8 IS NULL THEN a:4 ELSE x:8 END [as=upsert_x:12] - ├── CASE WHEN x:8 IS NULL THEN b:5 ELSE y_new:11 END [as=upsert_y:13] - └── CASE WHEN x:8 IS NULL THEN c:6 ELSE z:10 END [as=upsert_z:14] + ├── CASE WHEN x:10 IS NULL THEN a:5 ELSE x:10 END [as=upsert_x:15] + ├── CASE WHEN x:10 IS NULL THEN b:6 ELSE y_new:14 END [as=upsert_y:16] + └── CASE WHEN x:10 IS NULL THEN c:7 ELSE z:12 END [as=upsert_z:17] # ------------------------------------------------------------------------------ # Test decimal column truncation. @@ -1631,76 +1633,76 @@ UPSERT INTO decimals (a, b) VALUES (1.1, ARRAY[0.95]) ---- upsert decimals ├── columns: - ├── canary column: 13 - ├── fetch columns: decimals.a:13 decimals.b:14 decimals.c:15 decimals.d:16 + ├── canary column: 14 + ├── fetch columns: decimals.a:14 decimals.b:15 decimals.c:16 decimals.d:17 ├── insert-mapping: - │ ├── a:8 => decimals.a:1 - │ ├── b:9 => decimals.b:2 - │ ├── c:10 => decimals.c:3 - │ └── d:12 => decimals.d:4 + │ ├── a:9 => decimals.a:1 + │ ├── b:10 => decimals.b:2 + │ ├── c:11 => decimals.c:3 + │ └── d:13 => decimals.d:4 ├── update-mapping: - │ ├── b:9 => decimals.b:2 - │ └── upsert_d:21 => decimals.d:4 - ├── check columns: check1:22 check2:23 + │ ├── b:10 => decimals.b:2 + │ └── upsert_d:23 => decimals.d:4 + ├── check columns: check1:24 check2:25 └── project - ├── columns: check1:22 check2:23 a:8 b:9 c:10 d:12 decimals.a:13 decimals.b:14 decimals.c:15 decimals.d:16 d:18 upsert_a:19 upsert_c:20 upsert_d:21 + ├── columns: check1:24 check2:25 a:9 b:10 c:11 d:13 decimals.a:14 decimals.b:15 decimals.c:16 decimals.d:17 crdb_internal_mvcc_timestamp:18 d:20 upsert_a:21 upsert_c:22 upsert_d:23 ├── project - │ ├── columns: upsert_a:19 upsert_c:20 upsert_d:21 a:8 b:9 c:10 d:12 decimals.a:13 decimals.b:14 decimals.c:15 decimals.d:16 d:18 + │ ├── columns: upsert_a:21 upsert_c:22 upsert_d:23 a:9 b:10 c:11 d:13 decimals.a:14 decimals.b:15 decimals.c:16 decimals.d:17 crdb_internal_mvcc_timestamp:18 d:20 │ ├── project - │ │ ├── columns: d:18 a:8 b:9 c:10 d:12 decimals.a:13 decimals.b:14 decimals.c:15 decimals.d:16 + │ │ ├── columns: d:20 a:9 b:10 c:11 d:13 decimals.a:14 decimals.b:15 decimals.c:16 decimals.d:17 crdb_internal_mvcc_timestamp:18 │ │ ├── project - │ │ │ ├── columns: column17:17 a:8 b:9 c:10 d:12 decimals.a:13 decimals.b:14 decimals.c:15 decimals.d:16 + │ │ │ ├── columns: column19:19 a:9 b:10 c:11 d:13 decimals.a:14 decimals.b:15 decimals.c:16 decimals.d:17 crdb_internal_mvcc_timestamp:18 │ │ │ ├── left-join (hash) - │ │ │ │ ├── columns: a:8 b:9 c:10 d:12 decimals.a:13 decimals.b:14 decimals.c:15 decimals.d:16 + │ │ │ │ ├── columns: a:9 b:10 c:11 d:13 decimals.a:14 decimals.b:15 decimals.c:16 decimals.d:17 crdb_internal_mvcc_timestamp:18 │ │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ │ ├── columns: a:8 b:9 c:10 d:12 - │ │ │ │ │ ├── grouping columns: a:8 + │ │ │ │ │ ├── columns: a:9 b:10 c:11 d:13 + │ │ │ │ │ ├── grouping columns: a:9 │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: d:12 a:8 b:9 c:10 + │ │ │ │ │ │ ├── columns: d:13 a:9 b:10 c:11 │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ ├── columns: column11:11 a:8 b:9 c:10 + │ │ │ │ │ │ │ ├── columns: column12:12 a:9 b:10 c:11 │ │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ │ ├── columns: a:8 b:9 c:10 + │ │ │ │ │ │ │ │ ├── columns: a:9 b:10 c:11 │ │ │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ │ │ ├── columns: column7:7!null column1:5!null column2:6 + │ │ │ │ │ │ │ │ │ ├── columns: column8:8!null column1:6!null column2:7 │ │ │ │ │ │ │ │ │ ├── values - │ │ │ │ │ │ │ │ │ │ ├── columns: column1:5!null column2:6 + │ │ │ │ │ │ │ │ │ │ ├── columns: column1:6!null column2:7 │ │ │ │ │ │ │ │ │ │ └── (1.1, ARRAY[0.95]) │ │ │ │ │ │ │ │ │ └── projections - │ │ │ │ │ │ │ │ │ └── 1.23 [as=column7:7] + │ │ │ │ │ │ │ │ │ └── 1.23 [as=column8:8] │ │ │ │ │ │ │ │ └── projections - │ │ │ │ │ │ │ │ ├── crdb_internal.round_decimal_values(column1:5, 0) [as=a:8] - │ │ │ │ │ │ │ │ ├── crdb_internal.round_decimal_values(column2:6, 1) [as=b:9] - │ │ │ │ │ │ │ │ └── crdb_internal.round_decimal_values(column7:7, 1) [as=c:10] + │ │ │ │ │ │ │ │ ├── crdb_internal.round_decimal_values(column1:6, 0) [as=a:9] + │ │ │ │ │ │ │ │ ├── crdb_internal.round_decimal_values(column2:7, 1) [as=b:10] + │ │ │ │ │ │ │ │ └── crdb_internal.round_decimal_values(column8:8, 1) [as=c:11] │ │ │ │ │ │ │ └── projections - │ │ │ │ │ │ │ └── a:8 + c:10 [as=column11:11] + │ │ │ │ │ │ │ └── a:9 + c:11 [as=column12:12] │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── crdb_internal.round_decimal_values(column11:11, 1) [as=d:12] + │ │ │ │ │ │ └── crdb_internal.round_decimal_values(column12:12, 1) [as=d:13] │ │ │ │ │ └── aggregations - │ │ │ │ │ ├── first-agg [as=b:9] - │ │ │ │ │ │ └── b:9 - │ │ │ │ │ ├── first-agg [as=c:10] - │ │ │ │ │ │ └── c:10 - │ │ │ │ │ └── first-agg [as=d:12] - │ │ │ │ │ └── d:12 + │ │ │ │ │ ├── first-agg [as=b:10] + │ │ │ │ │ │ └── b:10 + │ │ │ │ │ ├── first-agg [as=c:11] + │ │ │ │ │ │ └── c:11 + │ │ │ │ │ └── first-agg [as=d:13] + │ │ │ │ │ └── d:13 │ │ │ │ ├── scan decimals - │ │ │ │ │ ├── columns: decimals.a:13!null decimals.b:14 decimals.c:15 decimals.d:16 + │ │ │ │ │ ├── columns: decimals.a:14!null decimals.b:15 decimals.c:16 decimals.d:17 crdb_internal_mvcc_timestamp:18 │ │ │ │ │ └── computed column expressions - │ │ │ │ │ └── decimals.d:16 - │ │ │ │ │ └── decimals.a:13::DECIMAL + decimals.c:15::DECIMAL + │ │ │ │ │ └── decimals.d:17 + │ │ │ │ │ └── decimals.a:14::DECIMAL + decimals.c:16::DECIMAL │ │ │ │ └── filters - │ │ │ │ └── a:8 = decimals.a:13 + │ │ │ │ └── a:9 = decimals.a:14 │ │ │ └── projections - │ │ │ └── decimals.a:13::DECIMAL + decimals.c:15::DECIMAL [as=column17:17] + │ │ │ └── decimals.a:14::DECIMAL + decimals.c:16::DECIMAL [as=column19:19] │ │ └── projections - │ │ └── crdb_internal.round_decimal_values(column17:17, 1) [as=d:18] + │ │ └── crdb_internal.round_decimal_values(column19:19, 1) [as=d:20] │ └── projections - │ ├── CASE WHEN decimals.a:13 IS NULL THEN a:8 ELSE decimals.a:13 END [as=upsert_a:19] - │ ├── CASE WHEN decimals.a:13 IS NULL THEN c:10 ELSE decimals.c:15 END [as=upsert_c:20] - │ └── CASE WHEN decimals.a:13 IS NULL THEN d:12 ELSE d:18 END [as=upsert_d:21] + │ ├── CASE WHEN decimals.a:14 IS NULL THEN a:9 ELSE decimals.a:14 END [as=upsert_a:21] + │ ├── CASE WHEN decimals.a:14 IS NULL THEN c:11 ELSE decimals.c:16 END [as=upsert_c:22] + │ └── CASE WHEN decimals.a:14 IS NULL THEN d:13 ELSE d:20 END [as=upsert_d:23] └── projections - ├── round(upsert_a:19) = upsert_a:19 [as=check1:22] - └── b:9[0] > 1 [as=check2:23] + ├── round(upsert_a:21) = upsert_a:21 [as=check1:24] + └── b:10[0] > 1 [as=check2:25] # Regular UPSERT case. build @@ -1708,77 +1710,77 @@ UPSERT INTO decimals (a) VALUES (1.1) ---- upsert decimals ├── columns: - ├── canary column: 13 - ├── fetch columns: decimals.a:13 decimals.b:14 decimals.c:15 decimals.d:16 + ├── canary column: 14 + ├── fetch columns: decimals.a:14 decimals.b:15 decimals.c:16 decimals.d:17 ├── insert-mapping: - │ ├── a:8 => decimals.a:1 - │ ├── b:9 => decimals.b:2 - │ ├── c:10 => decimals.c:3 - │ └── d:12 => decimals.d:4 + │ ├── a:9 => decimals.a:1 + │ ├── b:10 => decimals.b:2 + │ ├── c:11 => decimals.c:3 + │ └── d:13 => decimals.d:4 ├── update-mapping: - │ └── upsert_d:22 => decimals.d:4 - ├── check columns: check1:23 check2:24 + │ └── upsert_d:24 => decimals.d:4 + ├── check columns: check1:25 check2:26 └── project - ├── columns: check1:23 check2:24 a:8 b:9 c:10 d:12 decimals.a:13 decimals.b:14 decimals.c:15 decimals.d:16 d:18 upsert_a:19 upsert_b:20 upsert_c:21 upsert_d:22 + ├── columns: check1:25 check2:26 a:9 b:10 c:11 d:13 decimals.a:14 decimals.b:15 decimals.c:16 decimals.d:17 crdb_internal_mvcc_timestamp:18 d:20 upsert_a:21 upsert_b:22 upsert_c:23 upsert_d:24 ├── project - │ ├── columns: upsert_a:19 upsert_b:20 upsert_c:21 upsert_d:22 a:8 b:9 c:10 d:12 decimals.a:13 decimals.b:14 decimals.c:15 decimals.d:16 d:18 + │ ├── columns: upsert_a:21 upsert_b:22 upsert_c:23 upsert_d:24 a:9 b:10 c:11 d:13 decimals.a:14 decimals.b:15 decimals.c:16 decimals.d:17 crdb_internal_mvcc_timestamp:18 d:20 │ ├── project - │ │ ├── columns: d:18 a:8 b:9 c:10 d:12 decimals.a:13 decimals.b:14 decimals.c:15 decimals.d:16 + │ │ ├── columns: d:20 a:9 b:10 c:11 d:13 decimals.a:14 decimals.b:15 decimals.c:16 decimals.d:17 crdb_internal_mvcc_timestamp:18 │ │ ├── project - │ │ │ ├── columns: column17:17 a:8 b:9 c:10 d:12 decimals.a:13 decimals.b:14 decimals.c:15 decimals.d:16 + │ │ │ ├── columns: column19:19 a:9 b:10 c:11 d:13 decimals.a:14 decimals.b:15 decimals.c:16 decimals.d:17 crdb_internal_mvcc_timestamp:18 │ │ │ ├── left-join (hash) - │ │ │ │ ├── columns: a:8 b:9 c:10 d:12 decimals.a:13 decimals.b:14 decimals.c:15 decimals.d:16 + │ │ │ │ ├── columns: a:9 b:10 c:11 d:13 decimals.a:14 decimals.b:15 decimals.c:16 decimals.d:17 crdb_internal_mvcc_timestamp:18 │ │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ │ ├── columns: a:8 b:9 c:10 d:12 - │ │ │ │ │ ├── grouping columns: a:8 + │ │ │ │ │ ├── columns: a:9 b:10 c:11 d:13 + │ │ │ │ │ ├── grouping columns: a:9 │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: d:12 a:8 b:9 c:10 + │ │ │ │ │ │ ├── columns: d:13 a:9 b:10 c:11 │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ ├── columns: column11:11 a:8 b:9 c:10 + │ │ │ │ │ │ │ ├── columns: column12:12 a:9 b:10 c:11 │ │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ │ ├── columns: a:8 b:9 c:10 + │ │ │ │ │ │ │ │ ├── columns: a:9 b:10 c:11 │ │ │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ │ │ ├── columns: column6:6 column7:7!null column1:5!null + │ │ │ │ │ │ │ │ │ ├── columns: column7:7 column8:8!null column1:6!null │ │ │ │ │ │ │ │ │ ├── values - │ │ │ │ │ │ │ │ │ │ ├── columns: column1:5!null + │ │ │ │ │ │ │ │ │ │ ├── columns: column1:6!null │ │ │ │ │ │ │ │ │ │ └── (1.1,) │ │ │ │ │ │ │ │ │ └── projections - │ │ │ │ │ │ │ │ │ ├── NULL::DECIMAL(5,1)[] [as=column6:6] - │ │ │ │ │ │ │ │ │ └── 1.23 [as=column7:7] + │ │ │ │ │ │ │ │ │ ├── NULL::DECIMAL(5,1)[] [as=column7:7] + │ │ │ │ │ │ │ │ │ └── 1.23 [as=column8:8] │ │ │ │ │ │ │ │ └── projections - │ │ │ │ │ │ │ │ ├── crdb_internal.round_decimal_values(column1:5, 0) [as=a:8] - │ │ │ │ │ │ │ │ ├── crdb_internal.round_decimal_values(column6:6, 1) [as=b:9] - │ │ │ │ │ │ │ │ └── crdb_internal.round_decimal_values(column7:7, 1) [as=c:10] + │ │ │ │ │ │ │ │ ├── crdb_internal.round_decimal_values(column1:6, 0) [as=a:9] + │ │ │ │ │ │ │ │ ├── crdb_internal.round_decimal_values(column7:7, 1) [as=b:10] + │ │ │ │ │ │ │ │ └── crdb_internal.round_decimal_values(column8:8, 1) [as=c:11] │ │ │ │ │ │ │ └── projections - │ │ │ │ │ │ │ └── a:8 + c:10 [as=column11:11] + │ │ │ │ │ │ │ └── a:9 + c:11 [as=column12:12] │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── crdb_internal.round_decimal_values(column11:11, 1) [as=d:12] + │ │ │ │ │ │ └── crdb_internal.round_decimal_values(column12:12, 1) [as=d:13] │ │ │ │ │ └── aggregations - │ │ │ │ │ ├── first-agg [as=b:9] - │ │ │ │ │ │ └── b:9 - │ │ │ │ │ ├── first-agg [as=c:10] - │ │ │ │ │ │ └── c:10 - │ │ │ │ │ └── first-agg [as=d:12] - │ │ │ │ │ └── d:12 + │ │ │ │ │ ├── first-agg [as=b:10] + │ │ │ │ │ │ └── b:10 + │ │ │ │ │ ├── first-agg [as=c:11] + │ │ │ │ │ │ └── c:11 + │ │ │ │ │ └── first-agg [as=d:13] + │ │ │ │ │ └── d:13 │ │ │ │ ├── scan decimals - │ │ │ │ │ ├── columns: decimals.a:13!null decimals.b:14 decimals.c:15 decimals.d:16 + │ │ │ │ │ ├── columns: decimals.a:14!null decimals.b:15 decimals.c:16 decimals.d:17 crdb_internal_mvcc_timestamp:18 │ │ │ │ │ └── computed column expressions - │ │ │ │ │ └── decimals.d:16 - │ │ │ │ │ └── decimals.a:13::DECIMAL + decimals.c:15::DECIMAL + │ │ │ │ │ └── decimals.d:17 + │ │ │ │ │ └── decimals.a:14::DECIMAL + decimals.c:16::DECIMAL │ │ │ │ └── filters - │ │ │ │ └── a:8 = decimals.a:13 + │ │ │ │ └── a:9 = decimals.a:14 │ │ │ └── projections - │ │ │ └── decimals.a:13::DECIMAL + decimals.c:15::DECIMAL [as=column17:17] + │ │ │ └── decimals.a:14::DECIMAL + decimals.c:16::DECIMAL [as=column19:19] │ │ └── projections - │ │ └── crdb_internal.round_decimal_values(column17:17, 1) [as=d:18] + │ │ └── crdb_internal.round_decimal_values(column19:19, 1) [as=d:20] │ └── projections - │ ├── CASE WHEN decimals.a:13 IS NULL THEN a:8 ELSE decimals.a:13 END [as=upsert_a:19] - │ ├── CASE WHEN decimals.a:13 IS NULL THEN b:9 ELSE decimals.b:14 END [as=upsert_b:20] - │ ├── CASE WHEN decimals.a:13 IS NULL THEN c:10 ELSE decimals.c:15 END [as=upsert_c:21] - │ └── CASE WHEN decimals.a:13 IS NULL THEN d:12 ELSE d:18 END [as=upsert_d:22] + │ ├── CASE WHEN decimals.a:14 IS NULL THEN a:9 ELSE decimals.a:14 END [as=upsert_a:21] + │ ├── CASE WHEN decimals.a:14 IS NULL THEN b:10 ELSE decimals.b:15 END [as=upsert_b:22] + │ ├── CASE WHEN decimals.a:14 IS NULL THEN c:11 ELSE decimals.c:16 END [as=upsert_c:23] + │ └── CASE WHEN decimals.a:14 IS NULL THEN d:13 ELSE d:20 END [as=upsert_d:24] └── projections - ├── round(upsert_a:19) = upsert_a:19 [as=check1:23] - └── upsert_b:20[0] > 1 [as=check2:24] + ├── round(upsert_a:21) = upsert_a:21 [as=check1:25] + └── upsert_b:22[0] > 1 [as=check2:26] # INSERT...ON CONFLICT case. build @@ -1788,82 +1790,82 @@ DO UPDATE SET b=ARRAY[0.99] ---- upsert decimals ├── columns: - ├── canary column: 13 - ├── fetch columns: decimals.a:13 decimals.b:14 decimals.c:15 decimals.d:16 + ├── canary column: 14 + ├── fetch columns: decimals.a:14 decimals.b:15 decimals.c:16 decimals.d:17 ├── insert-mapping: - │ ├── a:8 => decimals.a:1 - │ ├── b:9 => decimals.b:2 - │ ├── c:10 => decimals.c:3 - │ └── d:12 => decimals.d:4 + │ ├── a:9 => decimals.a:1 + │ ├── b:10 => decimals.b:2 + │ ├── c:11 => decimals.c:3 + │ └── d:13 => decimals.d:4 ├── update-mapping: - │ ├── upsert_b:22 => decimals.b:2 - │ └── upsert_d:24 => decimals.d:4 - ├── check columns: check1:25 check2:26 + │ ├── upsert_b:24 => decimals.b:2 + │ └── upsert_d:26 => decimals.d:4 + ├── check columns: check1:27 check2:28 └── project - ├── columns: check1:25 check2:26 a:8 b:9 c:10 d:12 decimals.a:13 decimals.b:14 decimals.c:15 decimals.d:16 b:18 d:20 upsert_a:21 upsert_b:22 upsert_c:23 upsert_d:24 + ├── columns: check1:27 check2:28 a:9 b:10 c:11 d:13 decimals.a:14 decimals.b:15 decimals.c:16 decimals.d:17 crdb_internal_mvcc_timestamp:18 b:20 d:22 upsert_a:23 upsert_b:24 upsert_c:25 upsert_d:26 ├── project - │ ├── columns: upsert_a:21 upsert_b:22 upsert_c:23 upsert_d:24 a:8 b:9 c:10 d:12 decimals.a:13 decimals.b:14 decimals.c:15 decimals.d:16 b:18 d:20 + │ ├── columns: upsert_a:23 upsert_b:24 upsert_c:25 upsert_d:26 a:9 b:10 c:11 d:13 decimals.a:14 decimals.b:15 decimals.c:16 decimals.d:17 crdb_internal_mvcc_timestamp:18 b:20 d:22 │ ├── project - │ │ ├── columns: d:20 a:8 b:9 c:10 d:12 decimals.a:13 decimals.b:14 decimals.c:15 decimals.d:16 b:18 + │ │ ├── columns: d:22 a:9 b:10 c:11 d:13 decimals.a:14 decimals.b:15 decimals.c:16 decimals.d:17 crdb_internal_mvcc_timestamp:18 b:20 │ │ ├── project - │ │ │ ├── columns: column19:19 a:8 b:9 c:10 d:12 decimals.a:13 decimals.b:14 decimals.c:15 decimals.d:16 b:18 + │ │ │ ├── columns: column21:21 a:9 b:10 c:11 d:13 decimals.a:14 decimals.b:15 decimals.c:16 decimals.d:17 crdb_internal_mvcc_timestamp:18 b:20 │ │ │ ├── project - │ │ │ │ ├── columns: b:18 a:8 b:9 c:10 d:12 decimals.a:13 decimals.b:14 decimals.c:15 decimals.d:16 + │ │ │ │ ├── columns: b:20 a:9 b:10 c:11 d:13 decimals.a:14 decimals.b:15 decimals.c:16 decimals.d:17 crdb_internal_mvcc_timestamp:18 │ │ │ │ ├── project - │ │ │ │ │ ├── columns: b_new:17!null a:8 b:9 c:10 d:12 decimals.a:13 decimals.b:14 decimals.c:15 decimals.d:16 + │ │ │ │ │ ├── columns: b_new:19!null a:9 b:10 c:11 d:13 decimals.a:14 decimals.b:15 decimals.c:16 decimals.d:17 crdb_internal_mvcc_timestamp:18 │ │ │ │ │ ├── left-join (hash) - │ │ │ │ │ │ ├── columns: a:8 b:9 c:10 d:12 decimals.a:13 decimals.b:14 decimals.c:15 decimals.d:16 + │ │ │ │ │ │ ├── columns: a:9 b:10 c:11 d:13 decimals.a:14 decimals.b:15 decimals.c:16 decimals.d:17 crdb_internal_mvcc_timestamp:18 │ │ │ │ │ │ ├── ensure-upsert-distinct-on - │ │ │ │ │ │ │ ├── columns: a:8 b:9 c:10 d:12 - │ │ │ │ │ │ │ ├── grouping columns: a:8 + │ │ │ │ │ │ │ ├── columns: a:9 b:10 c:11 d:13 + │ │ │ │ │ │ │ ├── grouping columns: a:9 │ │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ │ ├── columns: d:12 a:8 b:9 c:10 + │ │ │ │ │ │ │ │ ├── columns: d:13 a:9 b:10 c:11 │ │ │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ │ │ ├── columns: column11:11 a:8 b:9 c:10 + │ │ │ │ │ │ │ │ │ ├── columns: column12:12 a:9 b:10 c:11 │ │ │ │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ │ │ │ ├── columns: a:8 b:9 c:10 + │ │ │ │ │ │ │ │ │ │ ├── columns: a:9 b:10 c:11 │ │ │ │ │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ │ │ │ │ ├── columns: column7:7!null column1:5!null column2:6 + │ │ │ │ │ │ │ │ │ │ │ ├── columns: column8:8!null column1:6!null column2:7 │ │ │ │ │ │ │ │ │ │ │ ├── values - │ │ │ │ │ │ │ │ │ │ │ │ ├── columns: column1:5!null column2:6 + │ │ │ │ │ │ │ │ │ │ │ │ ├── columns: column1:6!null column2:7 │ │ │ │ │ │ │ │ │ │ │ │ └── (1.1, ARRAY[0.95]) │ │ │ │ │ │ │ │ │ │ │ └── projections - │ │ │ │ │ │ │ │ │ │ │ └── 1.23 [as=column7:7] + │ │ │ │ │ │ │ │ │ │ │ └── 1.23 [as=column8:8] │ │ │ │ │ │ │ │ │ │ └── projections - │ │ │ │ │ │ │ │ │ │ ├── crdb_internal.round_decimal_values(column1:5, 0) [as=a:8] - │ │ │ │ │ │ │ │ │ │ ├── crdb_internal.round_decimal_values(column2:6, 1) [as=b:9] - │ │ │ │ │ │ │ │ │ │ └── crdb_internal.round_decimal_values(column7:7, 1) [as=c:10] + │ │ │ │ │ │ │ │ │ │ ├── crdb_internal.round_decimal_values(column1:6, 0) [as=a:9] + │ │ │ │ │ │ │ │ │ │ ├── crdb_internal.round_decimal_values(column2:7, 1) [as=b:10] + │ │ │ │ │ │ │ │ │ │ └── crdb_internal.round_decimal_values(column8:8, 1) [as=c:11] │ │ │ │ │ │ │ │ │ └── projections - │ │ │ │ │ │ │ │ │ └── a:8 + c:10 [as=column11:11] + │ │ │ │ │ │ │ │ │ └── a:9 + c:11 [as=column12:12] │ │ │ │ │ │ │ │ └── projections - │ │ │ │ │ │ │ │ └── crdb_internal.round_decimal_values(column11:11, 1) [as=d:12] + │ │ │ │ │ │ │ │ └── crdb_internal.round_decimal_values(column12:12, 1) [as=d:13] │ │ │ │ │ │ │ └── aggregations - │ │ │ │ │ │ │ ├── first-agg [as=b:9] - │ │ │ │ │ │ │ │ └── b:9 - │ │ │ │ │ │ │ ├── first-agg [as=c:10] - │ │ │ │ │ │ │ │ └── c:10 - │ │ │ │ │ │ │ └── first-agg [as=d:12] - │ │ │ │ │ │ │ └── d:12 + │ │ │ │ │ │ │ ├── first-agg [as=b:10] + │ │ │ │ │ │ │ │ └── b:10 + │ │ │ │ │ │ │ ├── first-agg [as=c:11] + │ │ │ │ │ │ │ │ └── c:11 + │ │ │ │ │ │ │ └── first-agg [as=d:13] + │ │ │ │ │ │ │ └── d:13 │ │ │ │ │ │ ├── scan decimals - │ │ │ │ │ │ │ ├── columns: decimals.a:13!null decimals.b:14 decimals.c:15 decimals.d:16 + │ │ │ │ │ │ │ ├── columns: decimals.a:14!null decimals.b:15 decimals.c:16 decimals.d:17 crdb_internal_mvcc_timestamp:18 │ │ │ │ │ │ │ └── computed column expressions - │ │ │ │ │ │ │ └── decimals.d:16 - │ │ │ │ │ │ │ └── decimals.a:13::DECIMAL + decimals.c:15::DECIMAL + │ │ │ │ │ │ │ └── decimals.d:17 + │ │ │ │ │ │ │ └── decimals.a:14::DECIMAL + decimals.c:16::DECIMAL │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── a:8 = decimals.a:13 + │ │ │ │ │ │ └── a:9 = decimals.a:14 │ │ │ │ │ └── projections - │ │ │ │ │ └── ARRAY[0.99] [as=b_new:17] + │ │ │ │ │ └── ARRAY[0.99] [as=b_new:19] │ │ │ │ └── projections - │ │ │ │ └── crdb_internal.round_decimal_values(b_new:17, 1) [as=b:18] + │ │ │ │ └── crdb_internal.round_decimal_values(b_new:19, 1) [as=b:20] │ │ │ └── projections - │ │ │ └── decimals.a:13::DECIMAL + decimals.c:15::DECIMAL [as=column19:19] + │ │ │ └── decimals.a:14::DECIMAL + decimals.c:16::DECIMAL [as=column21:21] │ │ └── projections - │ │ └── crdb_internal.round_decimal_values(column19:19, 1) [as=d:20] + │ │ └── crdb_internal.round_decimal_values(column21:21, 1) [as=d:22] │ └── projections - │ ├── CASE WHEN decimals.a:13 IS NULL THEN a:8 ELSE decimals.a:13 END [as=upsert_a:21] - │ ├── CASE WHEN decimals.a:13 IS NULL THEN b:9 ELSE b:18 END [as=upsert_b:22] - │ ├── CASE WHEN decimals.a:13 IS NULL THEN c:10 ELSE decimals.c:15 END [as=upsert_c:23] - │ └── CASE WHEN decimals.a:13 IS NULL THEN d:12 ELSE d:20 END [as=upsert_d:24] + │ ├── CASE WHEN decimals.a:14 IS NULL THEN a:9 ELSE decimals.a:14 END [as=upsert_a:23] + │ ├── CASE WHEN decimals.a:14 IS NULL THEN b:10 ELSE b:20 END [as=upsert_b:24] + │ ├── CASE WHEN decimals.a:14 IS NULL THEN c:11 ELSE decimals.c:16 END [as=upsert_c:25] + │ └── CASE WHEN decimals.a:14 IS NULL THEN d:13 ELSE d:22 END [as=upsert_d:26] └── projections - ├── round(upsert_a:21) = upsert_a:21 [as=check1:25] - └── upsert_b:22[0] > 1 [as=check2:26] + ├── round(upsert_a:23) = upsert_a:23 [as=check1:27] + └── upsert_b:24[0] > 1 [as=check2:28] diff --git a/pkg/sql/opt/optbuilder/testdata/view b/pkg/sql/opt/optbuilder/testdata/view index 4caeb4a27c04..faaf2f4df570 100644 --- a/pkg/sql/opt/optbuilder/testdata/view +++ b/pkg/sql/opt/optbuilder/testdata/view @@ -12,7 +12,7 @@ SELECT * FROM av project ├── columns: k:1!null i:2 s:4 └── scan a - └── columns: k:1!null i:2 f:3 s:4 j:5 + └── columns: k:1!null i:2 f:3 s:4 j:5 crdb_internal_mvcc_timestamp:6 build SELECT av.i, s, t.public.av.s AS s2 FROM t.av @@ -22,29 +22,29 @@ project └── project ├── columns: k:1!null i:2 s:4 └── scan a - └── columns: k:1!null i:2 f:3 s:4 j:5 + └── columns: k:1!null i:2 f:3 s:4 j:5 crdb_internal_mvcc_timestamp:6 # Self view join (multiple references to view). build SELECT av.k, av2.s FROM av, av AS av2 WHERE av.k=av2.k ---- project - ├── columns: k:1!null s:9 + ├── columns: k:1!null s:10 └── select - ├── columns: k:1!null i:2 s:4 k:6!null i:7 s:9 + ├── columns: k:1!null i:2 s:4 k:7!null i:8 s:10 ├── inner-join (cross) - │ ├── columns: k:1!null i:2 s:4 k:6!null i:7 s:9 + │ ├── columns: k:1!null i:2 s:4 k:7!null i:8 s:10 │ ├── project │ │ ├── columns: k:1!null i:2 s:4 │ │ └── scan a - │ │ └── columns: k:1!null i:2 f:3 s:4 j:5 + │ │ └── columns: k:1!null i:2 f:3 s:4 j:5 crdb_internal_mvcc_timestamp:6 │ ├── project - │ │ ├── columns: k:6!null i:7 s:9 + │ │ ├── columns: k:7!null i:8 s:10 │ │ └── scan a - │ │ └── columns: k:6!null i:7 f:8 s:9 j:10 + │ │ └── columns: k:7!null i:8 f:9 s:10 j:11 crdb_internal_mvcc_timestamp:12 │ └── filters (true) └── filters - └── k:1 = k:6 + └── k:1 = k:7 # View with aliased column names, filter, and ORDER BY. exec-ddl @@ -60,9 +60,9 @@ project └── project ├── columns: k:1!null f:3 s:4 └── select - ├── columns: k:1!null i:2!null f:3 s:4 j:5 + ├── columns: k:1!null i:2!null f:3 s:4 j:5 crdb_internal_mvcc_timestamp:6 ├── scan a - │ └── columns: k:1!null i:2 f:3 s:4 j:5 + │ └── columns: k:1!null i:2 f:3 s:4 j:5 crdb_internal_mvcc_timestamp:6 └── filters └── i:2 = 10 @@ -71,7 +71,7 @@ build SELECT array_agg(y) FROM av2 ---- scalar-group-by - ├── columns: array_agg:6 + ├── columns: array_agg:7 ├── internal-ordering: +4 ├── sort │ ├── columns: f:3 s:4 @@ -81,13 +81,13 @@ scalar-group-by │ └── project │ ├── columns: k:1!null f:3 s:4 │ └── select - │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 + │ ├── columns: k:1!null i:2!null f:3 s:4 j:5 crdb_internal_mvcc_timestamp:6 │ ├── scan a - │ │ └── columns: k:1!null i:2 f:3 s:4 j:5 + │ │ └── columns: k:1!null i:2 f:3 s:4 j:5 crdb_internal_mvcc_timestamp:6 │ └── filters │ └── i:2 = 10 └── aggregations - └── array-agg [as=array_agg:6] + └── array-agg [as=array_agg:7] └── f:3 # Verify that an outer table is visible from a subquery that uses diff --git a/pkg/sql/opt/optbuilder/testdata/where b/pkg/sql/opt/optbuilder/testdata/where index 511f6a0798cf..30024be16a86 100644 --- a/pkg/sql/opt/optbuilder/testdata/where +++ b/pkg/sql/opt/optbuilder/testdata/where @@ -17,54 +17,64 @@ CREATE TABLE kvString ( build SELECT * FROM kv WHERE k IN (1, 3) ---- -select +project ├── columns: k:1!null v:2 - ├── scan kv - │ └── columns: k:1!null v:2 - └── filters - └── k:1 IN (1, 3) + └── select + ├── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 + ├── scan kv + │ └── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 + └── filters + └── k:1 IN (1, 3) build SELECT * FROM kv WHERE v IN (6) ---- -select +project ├── columns: k:1!null v:2!null - ├── scan kv - │ └── columns: k:1!null v:2 - └── filters - └── v:2 IN (6,) + └── select + ├── columns: k:1!null v:2!null crdb_internal_mvcc_timestamp:3 + ├── scan kv + │ └── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 + └── filters + └── v:2 IN (6,) build SELECT * FROM kv WHERE k IN (SELECT k FROM kv) ---- -select +project ├── columns: k:1!null v:2 - ├── scan kv - │ └── columns: k:1!null v:2 - └── filters - └── any: eq - ├── project - │ ├── columns: k:3!null - │ └── scan kv - │ └── columns: k:3!null v:4 - └── k:1 + └── select + ├── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 + ├── scan kv + │ └── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 + └── filters + └── any: eq + ├── project + │ ├── columns: k:4!null + │ └── scan kv + │ └── columns: k:4!null v:5 crdb_internal_mvcc_timestamp:6 + └── k:1 build SELECT * FROM kv WHERE (k,v) IN (SELECT * FROM kv) ---- -select +project ├── columns: k:1!null v:2 - ├── scan kv - │ └── columns: k:1!null v:2 - └── filters - └── any: eq - ├── project - │ ├── columns: column5:5 - │ ├── scan kv - │ │ └── columns: k:3!null v:4 - │ └── projections - │ └── (k:3, v:4) [as=column5:5] - └── (k:1, v:2) + └── select + ├── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 + ├── scan kv + │ └── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 + └── filters + └── any: eq + ├── project + │ ├── columns: column7:7 + │ ├── project + │ │ ├── columns: k:4!null v:5 + │ │ └── scan kv + │ │ └── columns: k:4!null v:5 crdb_internal_mvcc_timestamp:6 + │ └── projections + │ └── (k:4, v:5) [as=column7:7] + └── (k:1, v:2) build SELECT * FROM kv WHERE nonexistent = 1 @@ -75,64 +85,66 @@ build SELECT 'hello' LIKE v AS r FROM kvString WHERE k LIKE 'like%' ORDER BY k ---- project - ├── columns: r:3 [hidden: k:1!null] + ├── columns: r:4 [hidden: k:1!null] ├── ordering: +1 ├── select - │ ├── columns: k:1!null v:2 + │ ├── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 │ ├── ordering: +1 │ ├── scan kvstring - │ │ ├── columns: k:1!null v:2 + │ │ ├── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 │ │ └── ordering: +1 │ └── filters │ └── k:1 LIKE 'like%' └── projections - └── 'hello' LIKE v:2 [as=r:3] + └── 'hello' LIKE v:2 [as=r:4] build SELECT 'hello' SIMILAR TO v AS r FROM kvString WHERE k SIMILAR TO 'like[1-2]' ORDER BY k ---- project - ├── columns: r:3 [hidden: k:1!null] + ├── columns: r:4 [hidden: k:1!null] ├── ordering: +1 ├── select - │ ├── columns: k:1!null v:2 + │ ├── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 │ ├── ordering: +1 │ ├── scan kvstring - │ │ ├── columns: k:1!null v:2 + │ │ ├── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 │ │ └── ordering: +1 │ └── filters │ └── k:1 SIMILAR TO 'like[1-2]' └── projections - └── 'hello' SIMILAR TO v:2 [as=r:3] + └── 'hello' SIMILAR TO v:2 [as=r:4] build SELECT 'hello' ~ replace(v, '%', '.*') AS r FROM kvString WHERE k ~ 'like[1-2]' ORDER BY k ---- project - ├── columns: r:3 [hidden: k:1!null] + ├── columns: r:4 [hidden: k:1!null] ├── ordering: +1 ├── select - │ ├── columns: k:1!null v:2 + │ ├── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 │ ├── ordering: +1 │ ├── scan kvstring - │ │ ├── columns: k:1!null v:2 + │ │ ├── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 │ │ └── ordering: +1 │ └── filters │ └── k:1 ~ 'like[1-2]' └── projections - └── 'hello' ~ replace(v:2, '%', '.*') [as=r:3] + └── 'hello' ~ replace(v:2, '%', '.*') [as=r:4] # Test mixed type tuple comparison. build SELECT * FROM kv WHERE k IN (1, 5.0, 9) ---- -select +project ├── columns: k:1!null v:2 - ├── scan kv - │ └── columns: k:1!null v:2 - └── filters - └── k:1 IN (1, 5, 9) + └── select + ├── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 + ├── scan kv + │ └── columns: k:1!null v:2 crdb_internal_mvcc_timestamp:3 + └── filters + └── k:1 IN (1, 5, 9) # Regression tests for #22670. exec-ddl @@ -145,9 +157,9 @@ SELECT * FROM ab WHERE a IN (1, 3, 4) project ├── columns: a:1!null b:2 └── select - ├── columns: a:1!null b:2 rowid:3!null + ├── columns: a:1!null b:2 rowid:3!null crdb_internal_mvcc_timestamp:4 ├── scan ab - │ └── columns: a:1 b:2 rowid:3!null + │ └── columns: a:1 b:2 rowid:3!null crdb_internal_mvcc_timestamp:4 └── filters └── a:1 IN (1, 3, 4) @@ -157,9 +169,9 @@ SELECT * FROM ab WHERE a IN (1, 3, 4, NULL) project ├── columns: a:1!null b:2 └── select - ├── columns: a:1!null b:2 rowid:3!null + ├── columns: a:1!null b:2 rowid:3!null crdb_internal_mvcc_timestamp:4 ├── scan ab - │ └── columns: a:1 b:2 rowid:3!null + │ └── columns: a:1 b:2 rowid:3!null crdb_internal_mvcc_timestamp:4 └── filters └── a:1 IN (1, 3, 4, NULL) @@ -169,9 +181,9 @@ SELECT * FROM ab WHERE (a, b) IN ((1, 10), (3, 30), (4, 40)) project ├── columns: a:1!null b:2!null └── select - ├── columns: a:1!null b:2!null rowid:3!null + ├── columns: a:1!null b:2!null rowid:3!null crdb_internal_mvcc_timestamp:4 ├── scan ab - │ └── columns: a:1 b:2 rowid:3!null + │ └── columns: a:1 b:2 rowid:3!null crdb_internal_mvcc_timestamp:4 └── filters └── (a:1, b:2) IN ((1, 10), (3, 30), (4, 40)) @@ -181,9 +193,9 @@ SELECT * FROM ab WHERE (a, b) IN ((1, 10), (4, NULL), (NULL, 50)) project ├── columns: a:1!null b:2!null └── select - ├── columns: a:1!null b:2!null rowid:3!null + ├── columns: a:1!null b:2!null rowid:3!null crdb_internal_mvcc_timestamp:4 ├── scan ab - │ └── columns: a:1 b:2 rowid:3!null + │ └── columns: a:1 b:2 rowid:3!null crdb_internal_mvcc_timestamp:4 └── filters └── (a:1, b:2) IN ((1, 10), (4, NULL), (NULL, 50)) diff --git a/pkg/sql/opt/optbuilder/testdata/window b/pkg/sql/opt/optbuilder/testdata/window index 0add854dc89a..713958ebc572 100644 --- a/pkg/sql/opt/optbuilder/testdata/window +++ b/pkg/sql/opt/optbuilder/testdata/window @@ -79,94 +79,94 @@ build SELECT lag('foo'::string) OVER (), lag(1) OVER () FROM kv ---- project - ├── columns: lag:8 lag:9 + ├── columns: lag:9 lag:10 └── window partition=() - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 lag:8 lag:9 lag_1_arg1:10!null lag_1_arg2:11!null lag_1_arg3:12 lag_2_arg3:13 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 lag:9 lag:10 lag_1_arg1:11!null lag_1_arg2:12!null lag_1_arg3:13 lag_2_arg3:14 ├── project - │ ├── columns: lag_1_arg1:10!null lag_1_arg2:11!null lag_1_arg3:12 lag_2_arg3:13 k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ ├── columns: lag_1_arg1:11!null lag_1_arg2:12!null lag_1_arg3:13 lag_2_arg3:14 k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ └── projections - │ ├── 'foo' [as=lag_1_arg1:10] - │ ├── 1 [as=lag_1_arg2:11] - │ ├── NULL::STRING [as=lag_1_arg3:12] - │ └── NULL::INT8 [as=lag_2_arg3:13] + │ ├── 'foo' [as=lag_1_arg1:11] + │ ├── 1 [as=lag_1_arg2:12] + │ ├── NULL::STRING [as=lag_1_arg3:13] + │ └── NULL::INT8 [as=lag_2_arg3:14] └── windows - ├── lag [as=lag:8] - │ ├── lag_1_arg1:10 - │ ├── lag_1_arg2:11 - │ └── lag_1_arg3:12 - └── lag [as=lag:9] - ├── lag_1_arg2:11 - ├── lag_1_arg2:11 - └── lag_2_arg3:13 + ├── lag [as=lag:9] + │ ├── lag_1_arg1:11 + │ ├── lag_1_arg2:12 + │ └── lag_1_arg3:13 + └── lag [as=lag:10] + ├── lag_1_arg2:12 + ├── lag_1_arg2:12 + └── lag_2_arg3:14 build SELECT count(*) OVER () FROM kv ---- project - ├── columns: count:8 + ├── columns: count:9 └── window partition=() - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 count_rows:8 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 count_rows:9 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - └── count-rows [as=count_rows:8] + └── count-rows [as=count_rows:9] build SELECT lag((SELECT k FROM kv kv2 WHERE kv2.k = kv.k)) OVER () FROM kv ---- project - ├── columns: lag:15 + ├── columns: lag:17 └── window partition=() - ├── columns: kv.k:1!null kv.v:2 kv.w:3 kv.f:4 kv.d:5 kv.s:6 kv.b:7 lag:15 lag_1_arg1:16 lag_1_arg2:17!null lag_1_arg3:18 + ├── columns: kv.k:1!null kv.v:2 kv.w:3 kv.f:4 kv.d:5 kv.s:6 kv.b:7 kv.crdb_internal_mvcc_timestamp:8 lag:17 lag_1_arg1:18 lag_1_arg2:19!null lag_1_arg3:20 ├── project - │ ├── columns: lag_1_arg1:16 lag_1_arg2:17!null lag_1_arg3:18 kv.k:1!null kv.v:2 kv.w:3 kv.f:4 kv.d:5 kv.s:6 kv.b:7 + │ ├── columns: lag_1_arg1:18 lag_1_arg2:19!null lag_1_arg3:20 kv.k:1!null kv.v:2 kv.w:3 kv.f:4 kv.d:5 kv.s:6 kv.b:7 kv.crdb_internal_mvcc_timestamp:8 │ ├── scan kv - │ │ └── columns: kv.k:1!null kv.v:2 kv.w:3 kv.f:4 kv.d:5 kv.s:6 kv.b:7 + │ │ └── columns: kv.k:1!null kv.v:2 kv.w:3 kv.f:4 kv.d:5 kv.s:6 kv.b:7 kv.crdb_internal_mvcc_timestamp:8 │ └── projections - │ ├── subquery [as=lag_1_arg1:16] + │ ├── subquery [as=lag_1_arg1:18] │ │ └── max1-row - │ │ ├── columns: kv2.k:8!null + │ │ ├── columns: kv2.k:9!null │ │ └── project - │ │ ├── columns: kv2.k:8!null + │ │ ├── columns: kv2.k:9!null │ │ └── select - │ │ ├── columns: kv2.k:8!null kv2.v:9 kv2.w:10 kv2.f:11 kv2.d:12 kv2.s:13 kv2.b:14 + │ │ ├── columns: kv2.k:9!null kv2.v:10 kv2.w:11 kv2.f:12 kv2.d:13 kv2.s:14 kv2.b:15 kv2.crdb_internal_mvcc_timestamp:16 │ │ ├── scan kv2 - │ │ │ └── columns: kv2.k:8!null kv2.v:9 kv2.w:10 kv2.f:11 kv2.d:12 kv2.s:13 kv2.b:14 + │ │ │ └── columns: kv2.k:9!null kv2.v:10 kv2.w:11 kv2.f:12 kv2.d:13 kv2.s:14 kv2.b:15 kv2.crdb_internal_mvcc_timestamp:16 │ │ └── filters - │ │ └── kv2.k:8 = kv.k:1 - │ ├── 1 [as=lag_1_arg2:17] - │ └── NULL::INT8 [as=lag_1_arg3:18] + │ │ └── kv2.k:9 = kv.k:1 + │ ├── 1 [as=lag_1_arg2:19] + │ └── NULL::INT8 [as=lag_1_arg3:20] └── windows - └── lag [as=lag:15] - ├── lag_1_arg1:16 - ├── lag_1_arg2:17 - └── lag_1_arg3:18 + └── lag [as=lag:17] + ├── lag_1_arg1:18 + ├── lag_1_arg2:19 + └── lag_1_arg3:20 build SELECT lag(1) OVER (), lead(1) OVER () FROM kv ---- project - ├── columns: lag:8 lead:9 + ├── columns: lag:9 lead:10 └── window partition=() - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 lag:8 lead:9 lag_1_arg1:10!null lag_1_arg3:11 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 lag:9 lead:10 lag_1_arg1:11!null lag_1_arg3:12 ├── project - │ ├── columns: lag_1_arg1:10!null lag_1_arg3:11 k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ ├── columns: lag_1_arg1:11!null lag_1_arg3:12 k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ └── projections - │ ├── 1 [as=lag_1_arg1:10] - │ └── NULL::INT8 [as=lag_1_arg3:11] + │ ├── 1 [as=lag_1_arg1:11] + │ └── NULL::INT8 [as=lag_1_arg3:12] └── windows - ├── lag [as=lag:8] - │ ├── lag_1_arg1:10 - │ ├── lag_1_arg1:10 - │ └── lag_1_arg3:11 - └── lead [as=lead:9] - ├── lag_1_arg1:10 - ├── lag_1_arg1:10 - └── lag_1_arg3:11 + ├── lag [as=lag:9] + │ ├── lag_1_arg1:11 + │ ├── lag_1_arg1:11 + │ └── lag_1_arg3:12 + └── lead [as=lead:10] + ├── lag_1_arg1:11 + ├── lag_1_arg1:11 + └── lag_1_arg3:12 build SELECT @@ -178,43 +178,43 @@ SELECT FROM kv ---- project - ├── columns: lag:8 lag:9 lag:10 lead:11 lead:12 + ├── columns: lag:9 lag:10 lag:11 lead:12 lead:13 └── window partition=(2) - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 lag:8 lag:9 lag:10 lead:11 lead:12 lag_1_arg1:13!null lag_1_arg3:14 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 lag:9 lag:10 lag:11 lead:12 lead:13 lag_1_arg1:14!null lag_1_arg3:15 ├── window partition=(1) - │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 lag:8 lag:9 lead:11 lag_1_arg1:13!null lag_1_arg3:14 + │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 lag:9 lag:10 lead:12 lag_1_arg1:14!null lag_1_arg3:15 │ ├── window partition=(1,2) - │ │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 lag:8 lag_1_arg1:13!null lag_1_arg3:14 + │ │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 lag:9 lag_1_arg1:14!null lag_1_arg3:15 │ │ ├── project - │ │ │ ├── columns: lag_1_arg1:13!null lag_1_arg3:14 k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ │ │ ├── columns: lag_1_arg1:14!null lag_1_arg3:15 k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ │ │ ├── scan kv - │ │ │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ │ │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ │ │ └── projections - │ │ │ ├── 1 [as=lag_1_arg1:13] - │ │ │ └── NULL::INT8 [as=lag_1_arg3:14] + │ │ │ ├── 1 [as=lag_1_arg1:14] + │ │ │ └── NULL::INT8 [as=lag_1_arg3:15] │ │ └── windows - │ │ └── lag [as=lag:8] - │ │ ├── lag_1_arg1:13 - │ │ ├── lag_1_arg1:13 - │ │ └── lag_1_arg3:14 + │ │ └── lag [as=lag:9] + │ │ ├── lag_1_arg1:14 + │ │ ├── lag_1_arg1:14 + │ │ └── lag_1_arg3:15 │ └── windows - │ ├── lag [as=lag:9] - │ │ ├── lag_1_arg1:13 - │ │ ├── lag_1_arg1:13 - │ │ └── lag_1_arg3:14 - │ └── lead [as=lead:11] - │ ├── lag_1_arg1:13 - │ ├── lag_1_arg1:13 - │ └── lag_1_arg3:14 + │ ├── lag [as=lag:10] + │ │ ├── lag_1_arg1:14 + │ │ ├── lag_1_arg1:14 + │ │ └── lag_1_arg3:15 + │ └── lead [as=lead:12] + │ ├── lag_1_arg1:14 + │ ├── lag_1_arg1:14 + │ └── lag_1_arg3:15 └── windows - ├── lag [as=lag:10] - │ ├── lag_1_arg1:13 - │ ├── lag_1_arg1:13 - │ └── lag_1_arg3:14 - └── lead [as=lead:12] - ├── lag_1_arg1:13 - ├── lag_1_arg1:13 - └── lag_1_arg3:14 + ├── lag [as=lag:11] + │ ├── lag_1_arg1:14 + │ ├── lag_1_arg1:14 + │ └── lag_1_arg3:15 + └── lead [as=lead:13] + ├── lag_1_arg1:14 + ├── lag_1_arg1:14 + └── lag_1_arg3:15 build SELECT @@ -226,121 +226,121 @@ SELECT FROM kv ---- project - ├── columns: lag:8 lag:9 lag:10 lead:11 lead:12 + ├── columns: lag:9 lag:10 lag:11 lead:12 lead:13 └── window partition=(2) - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 lag:8 lag:9 lag:10 lead:11 lead:12 lag_1_arg1:13!null lag_1_arg3:14 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 lag:9 lag:10 lag:11 lead:12 lead:13 lag_1_arg1:14!null lag_1_arg3:15 ├── window partition=(2) ordering=+4 - │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 lag:8 lag:9 lag:10 lead:11 lag_1_arg1:13!null lag_1_arg3:14 + │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 lag:9 lag:10 lag:11 lead:12 lag_1_arg1:14!null lag_1_arg3:15 │ ├── window partition=(1) ordering=+2 - │ │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 lag:8 lag:9 lead:11 lag_1_arg1:13!null lag_1_arg3:14 + │ │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 lag:9 lag:10 lead:12 lag_1_arg1:14!null lag_1_arg3:15 │ │ ├── window partition=(1,2) - │ │ │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 lag:8 lag_1_arg1:13!null lag_1_arg3:14 + │ │ │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 lag:9 lag_1_arg1:14!null lag_1_arg3:15 │ │ │ ├── project - │ │ │ │ ├── columns: lag_1_arg1:13!null lag_1_arg3:14 k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ │ │ │ ├── columns: lag_1_arg1:14!null lag_1_arg3:15 k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ │ │ │ ├── scan kv - │ │ │ │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ │ │ │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ │ │ │ └── projections - │ │ │ │ ├── 1 [as=lag_1_arg1:13] - │ │ │ │ └── NULL::INT8 [as=lag_1_arg3:14] + │ │ │ │ ├── 1 [as=lag_1_arg1:14] + │ │ │ │ └── NULL::INT8 [as=lag_1_arg3:15] │ │ │ └── windows - │ │ │ └── lag [as=lag:8] - │ │ │ ├── lag_1_arg1:13 - │ │ │ ├── lag_1_arg1:13 - │ │ │ └── lag_1_arg3:14 + │ │ │ └── lag [as=lag:9] + │ │ │ ├── lag_1_arg1:14 + │ │ │ ├── lag_1_arg1:14 + │ │ │ └── lag_1_arg3:15 │ │ └── windows - │ │ ├── lag [as=lag:9] - │ │ │ ├── lag_1_arg1:13 - │ │ │ ├── lag_1_arg1:13 - │ │ │ └── lag_1_arg3:14 - │ │ └── lead [as=lead:11] - │ │ ├── lag_1_arg1:13 - │ │ ├── lag_1_arg1:13 - │ │ └── lag_1_arg3:14 + │ │ ├── lag [as=lag:10] + │ │ │ ├── lag_1_arg1:14 + │ │ │ ├── lag_1_arg1:14 + │ │ │ └── lag_1_arg3:15 + │ │ └── lead [as=lead:12] + │ │ ├── lag_1_arg1:14 + │ │ ├── lag_1_arg1:14 + │ │ └── lag_1_arg3:15 │ └── windows - │ └── lag [as=lag:10] - │ ├── lag_1_arg1:13 - │ ├── lag_1_arg1:13 - │ └── lag_1_arg3:14 + │ └── lag [as=lag:11] + │ ├── lag_1_arg1:14 + │ ├── lag_1_arg1:14 + │ └── lag_1_arg3:15 └── windows - └── lead [as=lead:12] - ├── lag_1_arg1:13 - ├── lag_1_arg1:13 - └── lag_1_arg3:14 + └── lead [as=lead:13] + ├── lag_1_arg1:14 + ├── lag_1_arg1:14 + └── lag_1_arg3:15 build SELECT lag(1, 2) OVER (), lead(1, 2) OVER () FROM kv ---- project - ├── columns: lag:8 lead:9 + ├── columns: lag:9 lead:10 └── window partition=() - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 lag:8 lead:9 lag_1_arg1:10!null lag_1_arg2:11!null lag_1_arg3:12 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 lag:9 lead:10 lag_1_arg1:11!null lag_1_arg2:12!null lag_1_arg3:13 ├── project - │ ├── columns: lag_1_arg1:10!null lag_1_arg2:11!null lag_1_arg3:12 k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ ├── columns: lag_1_arg1:11!null lag_1_arg2:12!null lag_1_arg3:13 k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ └── projections - │ ├── 1 [as=lag_1_arg1:10] - │ ├── 2 [as=lag_1_arg2:11] - │ └── NULL::INT8 [as=lag_1_arg3:12] + │ ├── 1 [as=lag_1_arg1:11] + │ ├── 2 [as=lag_1_arg2:12] + │ └── NULL::INT8 [as=lag_1_arg3:13] └── windows - ├── lag [as=lag:8] - │ ├── lag_1_arg1:10 - │ ├── lag_1_arg2:11 - │ └── lag_1_arg3:12 - └── lead [as=lead:9] - ├── lag_1_arg1:10 - ├── lag_1_arg2:11 - └── lag_1_arg3:12 + ├── lag [as=lag:9] + │ ├── lag_1_arg1:11 + │ ├── lag_1_arg2:12 + │ └── lag_1_arg3:13 + └── lead [as=lead:10] + ├── lag_1_arg1:11 + ├── lag_1_arg2:12 + └── lag_1_arg3:13 build SELECT lag(1, 2, 3) OVER (), lead(1, 2, 3) OVER () FROM kv ---- project - ├── columns: lag:8 lead:9 + ├── columns: lag:9 lead:10 └── window partition=() - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 lag:8 lead:9 lag_1_arg1:10!null lag_1_arg2:11!null lag_1_arg3:12!null + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 lag:9 lead:10 lag_1_arg1:11!null lag_1_arg2:12!null lag_1_arg3:13!null ├── project - │ ├── columns: lag_1_arg1:10!null lag_1_arg2:11!null lag_1_arg3:12!null k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ ├── columns: lag_1_arg1:11!null lag_1_arg2:12!null lag_1_arg3:13!null k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ └── projections - │ ├── 1 [as=lag_1_arg1:10] - │ ├── 2 [as=lag_1_arg2:11] - │ └── 3 [as=lag_1_arg3:12] + │ ├── 1 [as=lag_1_arg1:11] + │ ├── 2 [as=lag_1_arg2:12] + │ └── 3 [as=lag_1_arg3:13] └── windows - ├── lag [as=lag:8] - │ ├── lag_1_arg1:10 - │ ├── lag_1_arg2:11 - │ └── lag_1_arg3:12 - └── lead [as=lead:9] - ├── lag_1_arg1:10 - ├── lag_1_arg2:11 - └── lag_1_arg3:12 + ├── lag [as=lag:9] + │ ├── lag_1_arg1:11 + │ ├── lag_1_arg2:12 + │ └── lag_1_arg3:13 + └── lead [as=lead:10] + ├── lag_1_arg1:11 + ├── lag_1_arg2:12 + └── lag_1_arg3:13 build SELECT avg(k) OVER () FROM kv ---- project - ├── columns: avg:8 + ├── columns: avg:9 └── window partition=() - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 avg:8 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 avg:9 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - └── avg [as=avg:8] + └── avg [as=avg:9] └── k:1 build SELECT x FROM (SELECT avg(k) OVER () AS x FROM kv) ---- project - ├── columns: x:8 + ├── columns: x:9 └── window partition=() - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 avg:8 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 avg:9 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - └── avg [as=avg:8] + └── avg [as=avg:9] └── k:1 build @@ -352,20 +352,20 @@ build SELECT avg(k) FILTER (WHERE k > 5) OVER () FROM kv ---- project - ├── columns: avg:8 + ├── columns: avg:9 └── window partition=() - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 avg:8 avg_1_filter:9!null + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 avg:9 avg_1_filter:10!null ├── project - │ ├── columns: avg_1_filter:9!null k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ ├── columns: avg_1_filter:10!null k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ └── projections - │ └── k:1 > 5 [as=avg_1_filter:9] + │ └── k:1 > 5 [as=avg_1_filter:10] └── windows - └── agg-filter [as=avg:8] + └── agg-filter [as=avg:9] ├── avg │ └── k:1 - └── avg_1_filter:9 + └── avg_1_filter:10 build SELECT avg(k), max(v) OVER () FROM kv ORDER BY 1 @@ -376,228 +376,228 @@ build SELECT avg(k) OVER () FROM kv ORDER BY 1 ---- sort - ├── columns: avg:8 - ├── ordering: +8 + ├── columns: avg:9 + ├── ordering: +9 └── project - ├── columns: avg:8 + ├── columns: avg:9 └── window partition=() - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 avg:8 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 avg:9 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - └── avg [as=avg:8] + └── avg [as=avg:9] └── k:1 build SELECT k, v, rank() OVER w FROM kv WINDOW w AS () ---- project - ├── columns: k:1!null v:2 rank:8 + ├── columns: k:1!null v:2 rank:9 └── window partition=() - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 rank:8 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 rank:9 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - └── rank [as=rank:8] + └── rank [as=rank:9] build SELECT k, v, first_value(v) OVER () FROM kv ---- project - ├── columns: k:1!null v:2 first_value:8 + ├── columns: k:1!null v:2 first_value:9 └── window partition=() - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 first_value:8 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 first_value:9 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - └── first-value [as=first_value:8] + └── first-value [as=first_value:9] └── v:2 build SELECT avg(k), max(v), min(w), 2 + row_number() OVER () FROM kv ORDER BY 1 ---- project - ├── columns: avg:8 max:9 min:10 "?column?":12 - ├── ordering: +8 + ├── columns: avg:9 max:10 min:11 "?column?":13 + ├── ordering: +9 ├── window partition=() - │ ├── columns: avg:8 max:9 min:10 row_number:11 + │ ├── columns: avg:9 max:10 min:11 row_number:12 │ ├── scalar-group-by - │ │ ├── columns: avg:8 max:9 min:10 + │ │ ├── columns: avg:9 max:10 min:11 │ │ ├── project │ │ │ ├── columns: k:1!null v:2 w:3 │ │ │ └── scan kv - │ │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ │ └── aggregations - │ │ ├── avg [as=avg:8] + │ │ ├── avg [as=avg:9] │ │ │ └── k:1 - │ │ ├── max [as=max:9] + │ │ ├── max [as=max:10] │ │ │ └── v:2 - │ │ └── min [as=min:10] + │ │ └── min [as=min:11] │ │ └── w:3 │ └── windows - │ └── row-number [as=row_number:11] + │ └── row-number [as=row_number:12] └── projections - └── 2 + row_number:11 [as="?column?":12] + └── 2 + row_number:12 [as="?column?":13] build SELECT k, rank() OVER (), dense_rank() OVER (), percent_rank() OVER (), cume_dist() OVER () FROM kv ---- project - ├── columns: k:1!null rank:8 dense_rank:9 percent_rank:10 cume_dist:11 + ├── columns: k:1!null rank:9 dense_rank:10 percent_rank:11 cume_dist:12 └── window partition=() - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 rank:8 dense_rank:9 percent_rank:10 cume_dist:11 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 rank:9 dense_rank:10 percent_rank:11 cume_dist:12 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - ├── rank [as=rank:8] - ├── dense-rank [as=dense_rank:9] - ├── percent-rank [as=percent_rank:10] - └── cume-dist [as=cume_dist:11] + ├── rank [as=rank:9] + ├── dense-rank [as=dense_rank:10] + ├── percent-rank [as=percent_rank:11] + └── cume-dist [as=cume_dist:12] build SELECT k, rank() OVER (), rank() OVER () FROM kv ---- project - ├── columns: k:1!null rank:8 rank:8 + ├── columns: k:1!null rank:9 rank:9 └── window partition=() - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 rank:8 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 rank:9 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - └── rank [as=rank:8] + └── rank [as=rank:9] build SELECT k, rank() OVER (), row_number() OVER () FROM kv ---- project - ├── columns: k:1!null rank:8 row_number:9 + ├── columns: k:1!null rank:9 row_number:10 └── window partition=() - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 rank:8 row_number:9 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 rank:9 row_number:10 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - ├── rank [as=rank:8] - └── row-number [as=row_number:9] + ├── rank [as=rank:9] + └── row-number [as=row_number:10] build SELECT k, rank() OVER (), row_number() OVER () FROM kv ORDER BY 1 ---- sort - ├── columns: k:1!null rank:8 row_number:9 + ├── columns: k:1!null rank:9 row_number:10 ├── ordering: +1 └── project - ├── columns: k:1!null rank:8 row_number:9 + ├── columns: k:1!null rank:9 row_number:10 └── window partition=() - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 rank:8 row_number:9 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 rank:9 row_number:10 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - ├── rank [as=rank:8] - └── row-number [as=row_number:9] + ├── rank [as=rank:9] + └── row-number [as=row_number:10] build SELECT k, v, rank() OVER (PARTITION BY v) FROM kv ORDER BY 1 ---- sort - ├── columns: k:1!null v:2 rank:8 + ├── columns: k:1!null v:2 rank:9 ├── ordering: +1 └── project - ├── columns: k:1!null v:2 rank:8 + ├── columns: k:1!null v:2 rank:9 └── window partition=(2) - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 rank:8 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 rank:9 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - └── rank [as=rank:8] + └── rank [as=rank:9] build SELECT k, row_number() OVER (PARTITION BY v), rank() OVER (PARTITION BY v) FROM kv ORDER BY 1 ---- sort - ├── columns: k:1!null row_number:8 rank:9 + ├── columns: k:1!null row_number:9 rank:10 ├── ordering: +1 └── project - ├── columns: k:1!null row_number:8 rank:9 + ├── columns: k:1!null row_number:9 rank:10 └── window partition=(2) - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 row_number:8 rank:9 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 row_number:9 rank:10 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - ├── row-number [as=row_number:8] - └── rank [as=rank:9] + ├── row-number [as=row_number:9] + └── rank [as=rank:10] build SELECT k, v, ntile(1) OVER () FROM kv ---- project - ├── columns: k:1!null v:2 ntile:8 + ├── columns: k:1!null v:2 ntile:9 └── window partition=() - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 ntile:8 ntile_1_arg1:9!null + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 ntile:9 ntile_1_arg1:10!null ├── project - │ ├── columns: ntile_1_arg1:9!null k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ ├── columns: ntile_1_arg1:10!null k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ └── projections - │ └── 1 [as=ntile_1_arg1:9] + │ └── 1 [as=ntile_1_arg1:10] └── windows - └── ntile [as=ntile:8] - └── ntile_1_arg1:9 + └── ntile [as=ntile:9] + └── ntile_1_arg1:10 build SELECT k, v, ntile(1) OVER (), ntile(50) OVER () FROM kv ---- project - ├── columns: k:1!null v:2 ntile:8 ntile:9 + ├── columns: k:1!null v:2 ntile:9 ntile:10 └── window partition=() - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 ntile:8 ntile:9 ntile_1_arg1:10!null ntile_2_arg1:11!null + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 ntile:9 ntile:10 ntile_1_arg1:11!null ntile_2_arg1:12!null ├── project - │ ├── columns: ntile_1_arg1:10!null ntile_2_arg1:11!null k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ ├── columns: ntile_1_arg1:11!null ntile_2_arg1:12!null k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ └── projections - │ ├── 1 [as=ntile_1_arg1:10] - │ └── 50 [as=ntile_2_arg1:11] + │ ├── 1 [as=ntile_1_arg1:11] + │ └── 50 [as=ntile_2_arg1:12] └── windows - ├── ntile [as=ntile:8] - │ └── ntile_1_arg1:10 - └── ntile [as=ntile:9] - └── ntile_2_arg1:11 + ├── ntile [as=ntile:9] + │ └── ntile_1_arg1:11 + └── ntile [as=ntile:10] + └── ntile_2_arg1:12 build SELECT k, v, nth_value('foo', 1) OVER () FROM kv ---- project - ├── columns: k:1!null v:2 nth_value:8 + ├── columns: k:1!null v:2 nth_value:9 └── window partition=() - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 nth_value:8 nth_value_1_arg1:9!null nth_value_1_arg2:10!null + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 nth_value:9 nth_value_1_arg1:10!null nth_value_1_arg2:11!null ├── project - │ ├── columns: nth_value_1_arg1:9!null nth_value_1_arg2:10!null k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ ├── columns: nth_value_1_arg1:10!null nth_value_1_arg2:11!null k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ └── projections - │ ├── 'foo' [as=nth_value_1_arg1:9] - │ └── 1 [as=nth_value_1_arg2:10] + │ ├── 'foo' [as=nth_value_1_arg1:10] + │ └── 1 [as=nth_value_1_arg2:11] └── windows - └── nth-value [as=nth_value:8] - ├── nth_value_1_arg1:9 - └── nth_value_1_arg2:10 + └── nth-value [as=nth_value:9] + ├── nth_value_1_arg1:10 + └── nth_value_1_arg2:11 build SELECT k, v, nth_value(1, k) OVER () FROM kv ---- project - ├── columns: k:1!null v:2 nth_value:8 + ├── columns: k:1!null v:2 nth_value:9 └── window partition=() - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 nth_value:8 nth_value_1_arg1:9!null + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 nth_value:9 nth_value_1_arg1:10!null ├── project - │ ├── columns: nth_value_1_arg1:9!null k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ ├── columns: nth_value_1_arg1:10!null k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ └── projections - │ └── 1 [as=nth_value_1_arg1:9] + │ └── 1 [as=nth_value_1_arg1:10] └── windows - └── nth-value [as=nth_value:8] - ├── nth_value_1_arg1:9 + └── nth-value [as=nth_value:9] + ├── nth_value_1_arg1:10 └── k:1 # Partitions @@ -616,102 +616,102 @@ build SELECT row_number() OVER (PARTITION BY v) FROM kv ---- project - ├── columns: row_number:8 + ├── columns: row_number:9 └── window partition=(2) - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 row_number:8 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 row_number:9 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - └── row-number [as=row_number:8] + └── row-number [as=row_number:9] build SELECT v, row_number() OVER (PARTITION BY v) FROM kv ---- project - ├── columns: v:2 row_number:8 + ├── columns: v:2 row_number:9 └── window partition=(2) - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 row_number:8 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 row_number:9 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - └── row-number [as=row_number:8] + └── row-number [as=row_number:9] build SELECT v, row_number() OVER (PARTITION BY v+1) FROM kv ---- project - ├── columns: v:2 row_number:8 - └── window partition=(9) - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 row_number:8 row_number_1_partition_1:9 + ├── columns: v:2 row_number:9 + └── window partition=(10) + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 row_number:9 row_number_1_partition_1:10 ├── project - │ ├── columns: row_number_1_partition_1:9 k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ ├── columns: row_number_1_partition_1:10 k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ └── projections - │ └── v:2 + 1 [as=row_number_1_partition_1:9] + │ └── v:2 + 1 [as=row_number_1_partition_1:10] └── windows - └── row-number [as=row_number:8] + └── row-number [as=row_number:9] build SELECT v, row_number() OVER (PARTITION BY avg(k)) FROM kv GROUP BY v ---- project - ├── columns: v:2 row_number:9 - └── window partition=(8) - ├── columns: v:2 avg:8!null row_number:9 + ├── columns: v:2 row_number:10 + └── window partition=(9) + ├── columns: v:2 avg:9!null row_number:10 ├── group-by - │ ├── columns: v:2 avg:8!null + │ ├── columns: v:2 avg:9!null │ ├── grouping columns: v:2 │ ├── project │ │ ├── columns: k:1!null v:2 │ │ └── scan kv - │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ └── aggregations - │ └── avg [as=avg:8] + │ └── avg [as=avg:9] │ └── k:1 └── windows - └── row-number [as=row_number:9] + └── row-number [as=row_number:10] # TODO(justin): expand these tuples. build SELECT k, row_number() OVER (PARTITION BY (k, v)) FROM kv ---- project - ├── columns: k:1!null row_number:8 + ├── columns: k:1!null row_number:9 └── window partition=(1,2) - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 row_number:8 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 row_number:9 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - └── row-number [as=row_number:8] + └── row-number [as=row_number:9] build SELECT k, row_number() OVER (PARTITION BY kv.*) FROM kv ---- project - ├── columns: k:1!null row_number:8 + ├── columns: k:1!null row_number:9 └── window partition=(1-7) - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 row_number:8 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 row_number:9 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - └── row-number [as=row_number:8] + └── row-number [as=row_number:9] build SELECT row_number() OVER (PARTITION BY v), rank() OVER (PARTITION BY v, f) FROM kv ---- project - ├── columns: row_number:8 rank:9 + ├── columns: row_number:9 rank:10 └── window partition=(2,4) - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 row_number:8 rank:9 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 row_number:9 rank:10 ├── window partition=(2) - │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 row_number:8 + │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 row_number:9 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ └── windows - │ └── row-number [as=row_number:8] + │ └── row-number [as=row_number:9] └── windows - └── rank [as=rank:9] + └── rank [as=rank:10] # Ordering @@ -719,28 +719,28 @@ build SELECT row_number() OVER (ORDER BY v) FROM kv ---- project - ├── columns: row_number:8 + ├── columns: row_number:9 └── window partition=() ordering=+2 - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 row_number:8 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 row_number:9 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - └── row-number [as=row_number:8] + └── row-number [as=row_number:9] build SELECT k, v, rank() OVER (ORDER BY k) FROM kv ORDER BY 1 ---- sort - ├── columns: k:1!null v:2 rank:8 + ├── columns: k:1!null v:2 rank:9 ├── ordering: +1 └── project - ├── columns: k:1!null v:2 rank:8 + ├── columns: k:1!null v:2 rank:9 └── window partition=() ordering=+1 - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 rank:8 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 rank:9 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - └── rank [as=rank:8] + └── rank [as=rank:9] # Ensure tuples in orderings get expanded. @@ -754,41 +754,41 @@ SELECT FROM kv ORDER BY 1 ---- sort - ├── columns: k:1!null v:2 rank:8 row_number:9 dense_rank:10 + ├── columns: k:1!null v:2 rank:9 row_number:10 dense_rank:11 ├── ordering: +1 └── project - ├── columns: k:1!null v:2 rank:8 row_number:9 dense_rank:10 + ├── columns: k:1!null v:2 rank:9 row_number:10 dense_rank:11 └── window partition=() ordering=+4,+2,+3 - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 rank:8 row_number:9 dense_rank:10 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 rank:9 row_number:10 dense_rank:11 ├── window partition=() ordering=+2,+3,+4 - │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 rank:8 row_number:9 + │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 rank:9 row_number:10 │ ├── window partition=() ordering=+2,+3 - │ │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 rank:8 + │ │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 rank:9 │ │ ├── scan kv - │ │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ │ └── windows - │ │ └── rank [as=rank:8] + │ │ └── rank [as=rank:9] │ └── windows - │ └── row-number [as=row_number:9] + │ └── row-number [as=row_number:10] └── windows - └── dense-rank [as=dense_rank:10] + └── dense-rank [as=dense_rank:11] build SELECT k, v, w, v - w + 2 + row_number() OVER (PARTITION BY v, k ORDER BY w) FROM kv ORDER BY 1 ---- sort - ├── columns: k:1!null v:2 w:3 "?column?":9 + ├── columns: k:1!null v:2 w:3 "?column?":10 ├── ordering: +1 └── project - ├── columns: "?column?":9 k:1!null v:2 w:3 + ├── columns: "?column?":10 k:1!null v:2 w:3 ├── window partition=(1,2) ordering=+3 - │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 row_number:8 + │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 row_number:9 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ └── windows - │ └── row-number [as=row_number:8] + │ └── row-number [as=row_number:9] └── projections - └── ((v:2 - w:3) + 2) + row_number:8 [as="?column?":9] + └── ((v:2 - w:3) + 2) + row_number:9 [as="?column?":10] build SELECT @@ -799,20 +799,20 @@ ORDER BY row_number() OVER (ORDER BY (k, v)) DESC ---- distinct-on - ├── columns: w:3 [hidden: row_number:8] - ├── grouping columns: row_number:8 - ├── ordering: -8 + ├── columns: w:3 [hidden: row_number:9] + ├── grouping columns: row_number:9 + ├── ordering: -9 ├── sort - │ ├── columns: w:3 row_number:8 - │ ├── ordering: -8 + │ ├── columns: w:3 row_number:9 + │ ├── ordering: -9 │ └── project - │ ├── columns: w:3 row_number:8 + │ ├── columns: w:3 row_number:9 │ └── window partition=() ordering=+1,+2 - │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 row_number:8 + │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 row_number:9 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ └── windows - │ └── row-number [as=row_number:8] + │ └── row-number [as=row_number:9] └── aggregations └── first-agg [as=w:3] └── w:3 @@ -824,13 +824,13 @@ build SELECT avg(k) OVER () FROM kv ---- project - ├── columns: avg:8 + ├── columns: avg:9 └── window partition=() - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 avg:8 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 avg:9 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - └── avg [as=avg:8] + └── avg [as=avg:9] └── k:1 build @@ -852,19 +852,19 @@ SELECT FROM kv ---- project - ├── columns: avg:8 avg:9 avg:10 avg:11 + ├── columns: avg:9 avg:10 avg:11 avg:12 └── window partition=() - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 avg:8 avg:9 avg:10 avg:11 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 avg:9 avg:10 avg:11 avg:12 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - ├── avg [as=avg:8, frame="range from unbounded to unbounded"] + ├── avg [as=avg:9, frame="range from unbounded to unbounded"] │ └── k:1 - ├── avg [as=avg:9] + ├── avg [as=avg:10] │ └── k:1 - ├── avg [as=avg:10, frame="range from current-row to unbounded"] + ├── avg [as=avg:11, frame="range from current-row to unbounded"] │ └── k:1 - └── avg [as=avg:11, frame="range from current-row to current-row"] + └── avg [as=avg:12, frame="range from current-row to current-row"] └── k:1 build @@ -876,19 +876,19 @@ SELECT FROM kv ---- project - ├── columns: avg:8 avg:9 avg:10 avg:11 + ├── columns: avg:9 avg:10 avg:11 avg:12 └── window partition=() ordering=+2 - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 avg:8 avg:9 avg:10 avg:11 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 avg:9 avg:10 avg:11 avg:12 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - ├── avg [as=avg:8, frame="range from unbounded to unbounded"] + ├── avg [as=avg:9, frame="range from unbounded to unbounded"] │ └── k:1 - ├── avg [as=avg:9] + ├── avg [as=avg:10] │ └── k:1 - ├── avg [as=avg:10, frame="range from current-row to unbounded"] + ├── avg [as=avg:11, frame="range from current-row to unbounded"] │ └── k:1 - └── avg [as=avg:11, frame="range from current-row to current-row"] + └── avg [as=avg:12, frame="range from current-row to current-row"] └── k:1 build @@ -898,15 +898,15 @@ SELECT FROM kv ---- project - ├── columns: avg:8 avg:9 + ├── columns: avg:9 avg:10 └── window partition=() - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 avg:8 avg:9 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 avg:9 avg:10 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - ├── avg [as=avg:8] + ├── avg [as=avg:9] │ └── k:1 - └── avg [as=avg:9, frame="range from current-row to current-row"] + └── avg [as=avg:10, frame="range from current-row to current-row"] └── k:1 # TODO(justin): add support for offset expressions. @@ -919,13 +919,13 @@ build SELECT avg(k) OVER (ORDER BY v RANGE BETWEEN UNBOUNDED PRECEDING AND 10 FOLLOWING) FROM kv ---- project - ├── columns: avg:8 + ├── columns: avg:9 └── window partition=() ordering=+2 - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 avg:8 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 avg:9 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - └── window-to-offset [as=avg:8, frame="range from unbounded to offset"] + └── window-to-offset [as=avg:9, frame="range from unbounded to offset"] ├── avg │ └── k:1 └── 10 @@ -936,13 +936,13 @@ SELECT FROM kv ---- project - ├── columns: avg_price:8 + ├── columns: avg_price:9 └── window partition=(3) ordering=+4 - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 avg:8 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 avg:9 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - └── window-to-offset [as=avg:8, frame="range from offset to offset"] + └── window-to-offset [as=avg:9, frame="range from offset to offset"] ├── window-from-offset │ ├── avg │ │ └── v:2 @@ -955,13 +955,13 @@ SELECT FROM kv ---- project - ├── columns: avg_price:8 + ├── columns: avg_price:9 └── window partition=(3) - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 avg:8 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 avg:9 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - └── window-to-offset [as=avg:8, frame="rows from offset to offset"] + └── window-to-offset [as=avg:9, frame="rows from offset to offset"] ├── window-from-offset │ ├── avg │ │ └── v:2 @@ -980,26 +980,26 @@ FROM kv ---- project - ├── columns: avg:16 + ├── columns: avg:18 └── window partition=(3) - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 avg:16 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 avg:18 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - └── window-to-offset [as=avg:16, frame="rows from offset to offset"] + └── window-to-offset [as=avg:18, frame="rows from offset to offset"] ├── window-from-offset │ ├── avg │ │ └── v:2 │ └── subquery │ └── max1-row - │ ├── columns: count_rows:15!null + │ ├── columns: count_rows:17!null │ └── scalar-group-by - │ ├── columns: count_rows:15!null + │ ├── columns: count_rows:17!null │ ├── project │ │ └── scan kv - │ │ └── columns: k:8!null v:9 w:10 f:11 d:12 s:13 b:14 + │ │ └── columns: k:9!null v:10 w:11 f:12 d:13 s:14 b:15 crdb_internal_mvcc_timestamp:16 │ └── aggregations - │ └── count-rows [as=count_rows:15] + │ └── count-rows [as=count_rows:17] └── 1 # TODO(justin): consider adding rules to hoist correlated subqueries in the @@ -1023,13 +1023,13 @@ SELECT FROM kv ---- project - ├── columns: avg:8 + ├── columns: avg:9 └── window partition=() ordering=+2 - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 avg:8 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 avg:9 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - └── window-to-offset [as=avg:8, frame="groups from offset to offset"] + └── window-to-offset [as=avg:9, frame="groups from offset to offset"] ├── window-from-offset │ ├── avg │ │ └── v:2 @@ -1042,13 +1042,13 @@ SELECT FROM kv ---- project - ├── columns: avg:8 + ├── columns: avg:9 └── window partition=() ordering=+1 - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 avg:8 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 avg:9 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - └── window-from-offset [as=avg:8, frame="groups from offset to unbounded"] + └── window-from-offset [as=avg:9, frame="groups from offset to unbounded"] ├── avg │ └── v:2 └── 1 @@ -1057,13 +1057,13 @@ build SELECT avg(v) OVER (ORDER BY f RANGE BETWEEN 1 FOLLOWING AND UNBOUNDED FOLLOWING) FROM kv ---- project - ├── columns: avg:8 + ├── columns: avg:9 └── window partition=() ordering=+4 - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 avg:8 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 avg:9 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - └── window-from-offset [as=avg:8, frame="range from offset to unbounded"] + └── window-from-offset [as=avg:9, frame="range from offset to unbounded"] ├── avg │ └── v:2 └── 1.0 @@ -1075,20 +1075,20 @@ SELECT FROM kv ---- project - ├── columns: avg:8 avg:9 + ├── columns: avg:9 avg:10 └── window partition=() ordering=+5 - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 avg:8 avg:9 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 avg:9 avg:10 ├── window partition=() ordering=+4 - │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 avg:8 + │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 avg:9 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ └── windows - │ └── window-from-offset [as=avg:8, frame="range from offset to unbounded"] + │ └── window-from-offset [as=avg:9, frame="range from offset to unbounded"] │ ├── avg │ │ └── v:2 │ └── 1.0 └── windows - └── window-from-offset [as=avg:9, frame="range from offset to unbounded"] + └── window-from-offset [as=avg:10, frame="range from offset to unbounded"] ├── avg │ └── v:2 └── 2 @@ -1125,41 +1125,40 @@ FROM kv ---- project - ├── columns: avg:8 avg:9 avg:10 avg:11 avg:12 avg:13 avg:14 avg:15 avg:16 avg:17 avg:18 avg:19 "?column?":20 "?column?":21 "?column?":22 "?column?":23 "?column?":24 "?column?":25 "?column?":26 "?column?":27 "?column?":28 "?column?":29 "?column?":30 "?column?":31 + ├── columns: avg:9 avg:10 avg:11 avg:12 avg:13 avg:14 avg:15 avg:16 avg:17 avg:18 avg:19 avg:20 "?column?":21 "?column?":22 "?column?":23 "?column?":24 "?column?":25 "?column?":26 "?column?":27 "?column?":28 "?column?":29 "?column?":30 "?column?":31 "?column?":32 ├── window partition=() ordering=+2 - │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 avg:8 avg:9 avg:10 avg:11 avg:12 avg:13 avg:14 avg:15 avg:16 avg:17 avg:18 avg:19 + │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 avg:9 avg:10 avg:11 avg:12 avg:13 avg:14 avg:15 avg:16 avg:17 avg:18 avg:19 avg:20 │ ├── window partition=() - │ │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 avg:8 avg:9 avg:10 avg:11 avg:12 avg:13 avg:14 avg:15 + │ │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 avg:9 avg:10 avg:11 avg:12 avg:13 avg:14 avg:15 avg:16 │ │ ├── scan kv - │ │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ │ └── windows - │ │ ├── avg [as=avg:8, frame="range from unbounded to unbounded"] + │ │ ├── avg [as=avg:9, frame="range from unbounded to unbounded"] │ │ │ └── k:1 - │ │ ├── avg [as=avg:9] + │ │ ├── avg [as=avg:10] │ │ │ └── k:1 - │ │ ├── avg [as=avg:10, frame="range from current-row to unbounded"] + │ │ ├── avg [as=avg:11, frame="range from current-row to unbounded"] │ │ │ └── k:1 - │ │ ├── avg [as=avg:11, frame="range from current-row to current-row"] + │ │ ├── avg [as=avg:12, frame="range from current-row to current-row"] │ │ │ └── k:1 - │ │ ├── avg [as=avg:12, frame="rows from unbounded to unbounded"] + │ │ ├── avg [as=avg:13, frame="rows from unbounded to unbounded"] │ │ │ └── k:1 - │ │ ├── avg [as=avg:13, frame="rows from unbounded to current-row"] + │ │ ├── avg [as=avg:14, frame="rows from unbounded to current-row"] │ │ │ └── k:1 - │ │ ├── avg [as=avg:14, frame="rows from current-row to unbounded"] + │ │ ├── avg [as=avg:15, frame="rows from current-row to unbounded"] │ │ │ └── k:1 - │ │ └── avg [as=avg:15, frame="rows from current-row to current-row"] + │ │ └── avg [as=avg:16, frame="rows from current-row to current-row"] │ │ └── k:1 │ └── windows - │ ├── avg [as=avg:16, frame="groups from unbounded to unbounded"] + │ ├── avg [as=avg:17, frame="groups from unbounded to unbounded"] │ │ └── k:1 - │ ├── avg [as=avg:17, frame="groups from unbounded to current-row"] + │ ├── avg [as=avg:18, frame="groups from unbounded to current-row"] │ │ └── k:1 - │ ├── avg [as=avg:18, frame="groups from current-row to unbounded"] + │ ├── avg [as=avg:19, frame="groups from current-row to unbounded"] │ │ └── k:1 - │ └── avg [as=avg:19, frame="groups from current-row to current-row"] + │ └── avg [as=avg:20, frame="groups from current-row to current-row"] │ └── k:1 └── projections - ├── 1 + avg:8 [as="?column?":20] ├── 1 + avg:9 [as="?column?":21] ├── 1 + avg:10 [as="?column?":22] ├── 1 + avg:11 [as="?column?":23] @@ -1170,7 +1169,8 @@ project ├── 1 + avg:16 [as="?column?":28] ├── 1 + avg:17 [as="?column?":29] ├── 1 + avg:18 [as="?column?":30] - └── 1 + avg:19 [as="?column?":31] + ├── 1 + avg:19 [as="?column?":31] + └── 1 + avg:20 [as="?column?":32] build SELECT @@ -1186,33 +1186,33 @@ FROM kv ---- project - ├── columns: avg:8 avg:9 avg:10 avg:11 "?column?":12 "?column?":13 "?column?":14 "?column?":15 + ├── columns: avg:9 avg:10 avg:11 avg:12 "?column?":13 "?column?":14 "?column?":15 "?column?":16 ├── window partition=() ordering=+2 - │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 avg:8 avg:9 avg:10 avg:11 + │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 avg:9 avg:10 avg:11 avg:12 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ └── windows - │ ├── avg [as=avg:8, frame="range from unbounded to unbounded"] + │ ├── avg [as=avg:9, frame="range from unbounded to unbounded"] │ │ └── k:1 - │ ├── window-from-offset [as=avg:9, frame="range from offset to unbounded"] + │ ├── window-from-offset [as=avg:10, frame="range from offset to unbounded"] │ │ ├── avg │ │ │ └── k:1 │ │ └── 1 - │ ├── window-to-offset [as=avg:10, frame="range from offset to offset"] + │ ├── window-to-offset [as=avg:11, frame="range from offset to offset"] │ │ ├── window-from-offset │ │ │ ├── avg │ │ │ │ └── k:1 │ │ │ └── 1 │ │ └── 1 - │ └── window-to-offset [as=avg:11, frame="range from unbounded to offset"] + │ └── window-to-offset [as=avg:12, frame="range from unbounded to offset"] │ ├── avg │ │ └── k:1 │ └── 1 └── projections - ├── 1 + avg:8 [as="?column?":12] ├── 1 + avg:9 [as="?column?":13] ├── 1 + avg:10 [as="?column?":14] - └── 1 + avg:11 [as="?column?":15] + ├── 1 + avg:11 [as="?column?":15] + └── 1 + avg:12 [as="?column?":16] # Filter. @@ -1225,44 +1225,44 @@ build SELECT avg(f) FILTER (WHERE k = 1) OVER () FROM kv ---- project - ├── columns: avg:8 + ├── columns: avg:9 └── window partition=() - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 avg:8 avg_1_filter:9!null + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 avg:9 avg_1_filter:10!null ├── project - │ ├── columns: avg_1_filter:9!null k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ ├── columns: avg_1_filter:10!null k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ └── projections - │ └── k:1 = 1 [as=avg_1_filter:9] + │ └── k:1 = 1 [as=avg_1_filter:10] └── windows - └── agg-filter [as=avg:8] + └── agg-filter [as=avg:9] ├── avg │ └── f:4 - └── avg_1_filter:9 + └── avg_1_filter:10 build SELECT count(*) FILTER (WHERE true) OVER (), count(*) FILTER (WHERE false) OVER () FROM kv ---- project - ├── columns: count:8 count:9 + ├── columns: count:9 count:10 └── window partition=() - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 count:8 count:9 count_1_arg1:10!null count_2_filter:11!null + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 count:9 count:10 count_1_arg1:11!null count_2_filter:12!null ├── project - │ ├── columns: count_1_arg1:10!null count_2_filter:11!null k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ ├── columns: count_1_arg1:11!null count_2_filter:12!null k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ └── projections - │ ├── true [as=count_1_arg1:10] - │ └── false [as=count_2_filter:11] + │ ├── true [as=count_1_arg1:11] + │ └── false [as=count_2_filter:12] └── windows - ├── agg-filter [as=count:8] + ├── agg-filter [as=count:9] │ ├── count - │ │ └── count_1_arg1:10 - │ └── count_1_arg1:10 - └── agg-filter [as=count:9] + │ │ └── count_1_arg1:11 + │ └── count_1_arg1:11 + └── agg-filter [as=count:10] ├── count - │ └── count_1_arg1:10 - └── count_2_filter:11 + │ └── count_1_arg1:11 + └── count_2_filter:12 build SELECT @@ -1273,25 +1273,25 @@ FROM kv ---- project - ├── columns: count:8 count:8 avg:9 + ├── columns: count:9 count:9 avg:10 └── window partition=() - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 count:8 avg:9 count_1_arg1:10!null count_1_filter:11!null + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 count:9 avg:10 count_1_arg1:11!null count_1_filter:12!null ├── project - │ ├── columns: count_1_arg1:10!null count_1_filter:11!null k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ ├── columns: count_1_arg1:11!null count_1_filter:12!null k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ ├── scan kv - │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ └── projections - │ ├── true [as=count_1_arg1:10] - │ └── false [as=count_1_filter:11] + │ ├── true [as=count_1_arg1:11] + │ └── false [as=count_1_filter:12] └── windows - ├── agg-filter [as=count:8] + ├── agg-filter [as=count:9] │ ├── count - │ │ └── count_1_arg1:10 - │ └── count_1_filter:11 - └── agg-filter [as=avg:9] + │ │ └── count_1_arg1:11 + │ └── count_1_filter:12 + └── agg-filter [as=avg:10] ├── avg │ └── k:1 - └── count_1_filter:11 + └── count_1_filter:12 # Named windows. @@ -1301,13 +1301,13 @@ SELECT FROM kv WINDOW w as (ORDER BY v) ---- project - ├── columns: rank:8 + ├── columns: rank:9 └── window partition=() ordering=+2 - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 rank:8 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 rank:9 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - └── rank [as=rank:8] + └── rank [as=rank:9] build SELECT @@ -1315,13 +1315,13 @@ SELECT FROM kv WINDOW w as (ORDER BY v) ---- project - ├── columns: rank:8 + ├── columns: rank:9 └── window partition=() ordering=+2 - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 rank:8 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 rank:9 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - └── rank [as=rank:8] + └── rank [as=rank:9] build SELECT @@ -1333,23 +1333,23 @@ SELECT FROM kv WINDOW w as (ORDER BY v), w2 as (PARTITION BY v) ---- project - ├── columns: rank:8 rank:9 row_number:10 row_number:11 row_number:12 + ├── columns: rank:9 rank:10 row_number:11 row_number:12 row_number:13 └── window partition=(2) ordering=+4 - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 rank:8 rank:9 row_number:10 row_number:11 row_number:12 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 rank:9 rank:10 row_number:11 row_number:12 row_number:13 ├── window partition=(2) - │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 rank:8 rank:9 row_number:10 row_number:12 + │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 rank:9 rank:10 row_number:11 row_number:13 │ ├── window partition=() ordering=+2 - │ │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 rank:8 rank:9 + │ │ ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 rank:9 rank:10 │ │ ├── scan kv - │ │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ │ │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 │ │ └── windows - │ │ ├── rank [as=rank:8] - │ │ └── rank [as=rank:9] + │ │ ├── rank [as=rank:9] + │ │ └── rank [as=rank:10] │ └── windows - │ ├── row-number [as=row_number:10] - │ └── row-number [as=row_number:12] + │ ├── row-number [as=row_number:11] + │ └── row-number [as=row_number:13] └── windows - └── row-number [as=row_number:11] + └── row-number [as=row_number:12] build SELECT @@ -1357,13 +1357,13 @@ SELECT FROM kv WINDOW w as (ORDER BY v), w2 as (PARTITION BY v) ---- project - ├── columns: rank:8 + ├── columns: rank:9 └── window partition=() ordering=+2 - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 rank:8 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 rank:9 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - └── rank [as=rank:8] + └── rank [as=rank:9] build SELECT @@ -1372,16 +1372,16 @@ FROM kv WINDOW w as (ORDER BY v) ORDER BY rank() OVER w ---- sort - ├── columns: rank:8 - ├── ordering: +8 + ├── columns: rank:9 + ├── ordering: +9 └── project - ├── columns: rank:8 + ├── columns: rank:9 └── window partition=() ordering=+2 - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 rank:8 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 rank:9 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - └── rank [as=rank:8] + └── rank [as=rank:9] build SELECT @@ -1438,16 +1438,16 @@ SELECT FROM kv ---- project - ├── columns: rank:8 rank:9 rank:10 rank:11 + ├── columns: rank:9 rank:10 rank:11 rank:12 └── window partition=() - ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 rank:8 rank:9 rank:10 rank:11 + ├── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 rank:9 rank:10 rank:11 rank:12 ├── scan kv - │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 + │ └── columns: k:1!null v:2 w:3 f:4 d:5 s:6 b:7 crdb_internal_mvcc_timestamp:8 └── windows - ├── rank [as=rank:8, frame="range from unbounded to current-row exclude current row"] - ├── rank [as=rank:9, frame="range from unbounded to current-row exclude group"] - ├── rank [as=rank:10, frame="range from unbounded to current-row exclude ties"] - └── rank [as=rank:11] + ├── rank [as=rank:9, frame="range from unbounded to current-row exclude current row"] + ├── rank [as=rank:10, frame="range from unbounded to current-row exclude group"] + ├── rank [as=rank:11, frame="range from unbounded to current-row exclude ties"] + └── rank [as=rank:12] exec-ddl CREATE TABLE table1 (col5 CHAR, col8 INT2); diff --git a/pkg/sql/opt/optbuilder/testdata/with b/pkg/sql/opt/optbuilder/testdata/with index 264407de804b..a765b7a97519 100644 --- a/pkg/sql/opt/optbuilder/testdata/with +++ b/pkg/sql/opt/optbuilder/testdata/with @@ -11,27 +11,27 @@ WITH t AS (SELECT a FROM y WHERE a < 3) SELECT * FROM x NATURAL JOIN t ---- with &1 (t) - ├── columns: a:3!null b:4 + ├── columns: a:4!null b:5 ├── project │ ├── columns: y.a:1!null │ └── select - │ ├── columns: y.a:1!null y.rowid:2!null + │ ├── columns: y.a:1!null y.rowid:2!null y.crdb_internal_mvcc_timestamp:3 │ ├── scan y - │ │ └── columns: y.a:1 y.rowid:2!null + │ │ └── columns: y.a:1 y.rowid:2!null y.crdb_internal_mvcc_timestamp:3 │ └── filters │ └── y.a:1 < 3 └── project - ├── columns: x.a:3!null b:4 + ├── columns: x.a:4!null b:5 └── inner-join (hash) - ├── columns: x.a:3!null b:4 x.rowid:5!null a:6!null + ├── columns: x.a:4!null b:5 x.rowid:6!null x.crdb_internal_mvcc_timestamp:7 a:8!null ├── scan x - │ └── columns: x.a:3 b:4 x.rowid:5!null + │ └── columns: x.a:4 b:5 x.rowid:6!null x.crdb_internal_mvcc_timestamp:7 ├── with-scan &1 (t) - │ ├── columns: a:6!null + │ ├── columns: a:8!null │ └── mapping: - │ └── y.a:1 => a:6 + │ └── y.a:1 => a:8 └── filters - └── x.a:3 = a:6 + └── x.a:4 = a:8 build EXPLAIN @@ -39,29 +39,29 @@ EXPLAIN SELECT * FROM x NATURAL JOIN t ---- explain - ├── columns: tree:7 field:8 description:9 + ├── columns: tree:9 field:10 description:11 └── with &1 (t) - ├── columns: a:3!null b:4 + ├── columns: a:4!null b:5 ├── project │ ├── columns: y.a:1!null │ └── select - │ ├── columns: y.a:1!null y.rowid:2!null + │ ├── columns: y.a:1!null y.rowid:2!null y.crdb_internal_mvcc_timestamp:3 │ ├── scan y - │ │ └── columns: y.a:1 y.rowid:2!null + │ │ └── columns: y.a:1 y.rowid:2!null y.crdb_internal_mvcc_timestamp:3 │ └── filters │ └── y.a:1 < 3 └── project - ├── columns: x.a:3!null b:4 + ├── columns: x.a:4!null b:5 └── inner-join (hash) - ├── columns: x.a:3!null b:4 x.rowid:5!null a:6!null + ├── columns: x.a:4!null b:5 x.rowid:6!null x.crdb_internal_mvcc_timestamp:7 a:8!null ├── scan x - │ └── columns: x.a:3 b:4 x.rowid:5!null + │ └── columns: x.a:4 b:5 x.rowid:6!null x.crdb_internal_mvcc_timestamp:7 ├── with-scan &1 (t) - │ ├── columns: a:6!null + │ ├── columns: a:8!null │ └── mapping: - │ └── y.a:1 => a:6 + │ └── y.a:1 => a:8 └── filters - └── x.a:3 = a:6 + └── x.a:4 = a:8 build WITH @@ -91,7 +91,7 @@ CREATE VIEW v1 AS ---- create-view t.public.v1 ├── WITH t AS (SELECT a FROM t.public.y WHERE a < 3) SELECT 1 FROM t.public.x NATURAL JOIN t - ├── columns: "?column?":7 + ├── columns: "?column?":9 └── dependencies ├── y [columns: a] └── x [columns: a] @@ -104,52 +104,52 @@ CREATE TABLE t1 AS create-table ├── CREATE TABLE t1 AS WITH t AS (SELECT a FROM t.public.y WHERE a < 3) SELECT 1 FROM t.public.x NATURAL JOIN t └── project - ├── columns: rowid:8 "?column?":7!null + ├── columns: rowid:10 "?column?":9!null ├── with &1 (t) - │ ├── columns: "?column?":7!null + │ ├── columns: "?column?":9!null │ ├── project │ │ ├── columns: y.a:1!null │ │ └── select - │ │ ├── columns: y.a:1!null y.rowid:2!null + │ │ ├── columns: y.a:1!null y.rowid:2!null y.crdb_internal_mvcc_timestamp:3 │ │ ├── scan y - │ │ │ └── columns: y.a:1 y.rowid:2!null + │ │ │ └── columns: y.a:1 y.rowid:2!null y.crdb_internal_mvcc_timestamp:3 │ │ └── filters │ │ └── y.a:1 < 3 │ └── project - │ ├── columns: "?column?":7!null + │ ├── columns: "?column?":9!null │ ├── inner-join (hash) - │ │ ├── columns: x.a:3!null b:4 x.rowid:5!null a:6!null + │ │ ├── columns: x.a:4!null b:5 x.rowid:6!null x.crdb_internal_mvcc_timestamp:7 a:8!null │ │ ├── scan x - │ │ │ └── columns: x.a:3 b:4 x.rowid:5!null + │ │ │ └── columns: x.a:4 b:5 x.rowid:6!null x.crdb_internal_mvcc_timestamp:7 │ │ ├── with-scan &1 (t) - │ │ │ ├── columns: a:6!null + │ │ │ ├── columns: a:8!null │ │ │ └── mapping: - │ │ │ └── y.a:1 => a:6 + │ │ │ └── y.a:1 => a:8 │ │ └── filters - │ │ └── x.a:3 = a:6 + │ │ └── x.a:4 = a:8 │ └── projections - │ └── 1 [as="?column?":7] + │ └── 1 [as="?column?":9] └── projections - └── unique_rowid() [as=rowid:8] + └── unique_rowid() [as=rowid:10] build WITH t AS (SELECT a FROM y WHERE a < 3) SELECT * FROM t ---- with &1 (t) - ├── columns: a:3!null + ├── columns: a:4!null ├── project │ ├── columns: y.a:1!null │ └── select - │ ├── columns: y.a:1!null rowid:2!null + │ ├── columns: y.a:1!null rowid:2!null crdb_internal_mvcc_timestamp:3 │ ├── scan y - │ │ └── columns: y.a:1 rowid:2!null + │ │ └── columns: y.a:1 rowid:2!null crdb_internal_mvcc_timestamp:3 │ └── filters │ └── y.a:1 < 3 └── with-scan &1 (t) - ├── columns: a:3!null + ├── columns: a:4!null └── mapping: - └── y.a:1 => a:3 + └── y.a:1 => a:4 # Chaining multiple CTEs. build @@ -159,29 +159,29 @@ WITH SELECT * FROM t2 ---- with &1 (t1) - ├── columns: a:4!null + ├── columns: a:5!null ├── project │ ├── columns: y.a:1!null │ └── select - │ ├── columns: y.a:1!null rowid:2!null + │ ├── columns: y.a:1!null rowid:2!null crdb_internal_mvcc_timestamp:3 │ ├── scan y - │ │ └── columns: y.a:1 rowid:2!null + │ │ └── columns: y.a:1 rowid:2!null crdb_internal_mvcc_timestamp:3 │ └── filters │ └── y.a:1 < 3 └── with &2 (t2) - ├── columns: a:4!null + ├── columns: a:5!null ├── select - │ ├── columns: a:3!null + │ ├── columns: a:4!null │ ├── with-scan &1 (t1) - │ │ ├── columns: a:3!null + │ │ ├── columns: a:4!null │ │ └── mapping: - │ │ └── y.a:1 => a:3 + │ │ └── y.a:1 => a:4 │ └── filters - │ └── a:3 > 1 + │ └── a:4 > 1 └── with-scan &2 (t2) - ├── columns: a:4!null + ├── columns: a:5!null └── mapping: - └── a:3 => a:4 + └── a:4 => a:5 build WITH @@ -191,39 +191,39 @@ WITH SELECT * FROM t3 ---- with &1 (t1) - ├── columns: a:5!null + ├── columns: a:6!null ├── project │ ├── columns: y.a:1!null │ └── select - │ ├── columns: y.a:1!null rowid:2!null + │ ├── columns: y.a:1!null rowid:2!null crdb_internal_mvcc_timestamp:3 │ ├── scan y - │ │ └── columns: y.a:1 rowid:2!null + │ │ └── columns: y.a:1 rowid:2!null crdb_internal_mvcc_timestamp:3 │ └── filters │ └── y.a:1 < 3 └── with &2 (t2) - ├── columns: a:5!null + ├── columns: a:6!null ├── select - │ ├── columns: a:3!null + │ ├── columns: a:4!null │ ├── with-scan &1 (t1) - │ │ ├── columns: a:3!null + │ │ ├── columns: a:4!null │ │ └── mapping: - │ │ └── y.a:1 => a:3 + │ │ └── y.a:1 => a:4 │ └── filters - │ └── a:3 > 1 + │ └── a:4 > 1 └── with &3 (t3) - ├── columns: a:5!null + ├── columns: a:6!null ├── select - │ ├── columns: a:4!null + │ ├── columns: a:5!null │ ├── with-scan &2 (t2) - │ │ ├── columns: a:4!null + │ │ ├── columns: a:5!null │ │ └── mapping: - │ │ └── a:3 => a:4 + │ │ └── a:4 => a:5 │ └── filters - │ └── a:4 = 2 + │ └── a:5 = 2 └── with-scan &3 (t3) - ├── columns: a:5!null + ├── columns: a:6!null └── mapping: - └── a:4 => a:5 + └── a:5 => a:6 build WITH @@ -234,59 +234,59 @@ WITH SELECT * FROM t3 NATURAL JOIN t4 ---- with &1 (t1) - ├── columns: a:7!null + ├── columns: a:9!null ├── project │ ├── columns: y.a:1!null │ └── select - │ ├── columns: y.a:1!null rowid:2!null + │ ├── columns: y.a:1!null rowid:2!null crdb_internal_mvcc_timestamp:3 │ ├── scan y - │ │ └── columns: y.a:1 rowid:2!null + │ │ └── columns: y.a:1 rowid:2!null crdb_internal_mvcc_timestamp:3 │ └── filters │ └── y.a:1 < 3 └── with &2 (t2) - ├── columns: a:7!null + ├── columns: a:9!null ├── project - │ ├── columns: y.a:3!null + │ ├── columns: y.a:4!null │ └── select - │ ├── columns: y.a:3!null rowid:4!null + │ ├── columns: y.a:4!null rowid:5!null crdb_internal_mvcc_timestamp:6 │ ├── scan y - │ │ └── columns: y.a:3 rowid:4!null + │ │ └── columns: y.a:4 rowid:5!null crdb_internal_mvcc_timestamp:6 │ └── filters - │ └── y.a:3 > 1 + │ └── y.a:4 > 1 └── with &3 (t3) - ├── columns: a:7!null + ├── columns: a:9!null ├── select - │ ├── columns: a:5!null + │ ├── columns: a:7!null │ ├── with-scan &1 (t1) - │ │ ├── columns: a:5!null + │ │ ├── columns: a:7!null │ │ └── mapping: - │ │ └── y.a:1 => a:5 + │ │ └── y.a:1 => a:7 │ └── filters - │ └── a:5 < 4 + │ └── a:7 < 4 └── with &4 (t4) - ├── columns: a:7!null + ├── columns: a:9!null ├── select - │ ├── columns: a:6!null + │ ├── columns: a:8!null │ ├── with-scan &2 (t2) - │ │ ├── columns: a:6!null + │ │ ├── columns: a:8!null │ │ └── mapping: - │ │ └── y.a:3 => a:6 + │ │ └── y.a:4 => a:8 │ └── filters - │ └── a:6 > 3 + │ └── a:8 > 3 └── project - ├── columns: a:7!null + ├── columns: a:9!null └── inner-join (hash) - ├── columns: a:7!null a:8!null + ├── columns: a:9!null a:10!null ├── with-scan &3 (t3) - │ ├── columns: a:7!null + │ ├── columns: a:9!null │ └── mapping: - │ └── a:5 => a:7 + │ └── a:7 => a:9 ├── with-scan &4 (t4) - │ ├── columns: a:8!null + │ ├── columns: a:10!null │ └── mapping: - │ └── a:6 => a:8 + │ └── a:8 => a:10 └── filters - └── a:7 = a:8 + └── a:9 = a:10 # Make sure they scope properly. build @@ -364,41 +364,41 @@ WITH SELECT * FROM t2 NATURAL JOIN x ---- with &1 (t1) - ├── columns: a:9!null b:10!null + ├── columns: a:11!null b:12!null ├── project │ ├── columns: x.a:1 x.b:2 │ └── scan x - │ └── columns: x.a:1 x.b:2 rowid:3!null + │ └── columns: x.a:1 x.b:2 rowid:3!null crdb_internal_mvcc_timestamp:4 └── with &2 (t2) - ├── columns: a:9!null b:10!null + ├── columns: a:11!null b:12!null ├── project - │ ├── columns: x.a:4!null x.b:5!null + │ ├── columns: x.a:5!null x.b:6!null │ └── inner-join (hash) - │ ├── columns: x.a:4!null x.b:5!null rowid:6!null a:7!null b:8!null + │ ├── columns: x.a:5!null x.b:6!null rowid:7!null crdb_internal_mvcc_timestamp:8 a:9!null b:10!null │ ├── scan x - │ │ └── columns: x.a:4 x.b:5 rowid:6!null + │ │ └── columns: x.a:5 x.b:6 rowid:7!null crdb_internal_mvcc_timestamp:8 │ ├── with-scan &1 (t1) - │ │ ├── columns: a:7 b:8 + │ │ ├── columns: a:9 b:10 │ │ └── mapping: - │ │ ├── x.a:1 => a:7 - │ │ └── x.b:2 => b:8 + │ │ ├── x.a:1 => a:9 + │ │ └── x.b:2 => b:10 │ └── filters - │ ├── x.a:4 = a:7 - │ └── x.b:5 = b:8 + │ ├── x.a:5 = a:9 + │ └── x.b:6 = b:10 └── project - ├── columns: a:9!null b:10!null + ├── columns: a:11!null b:12!null └── inner-join (hash) - ├── columns: a:9!null b:10!null x.a:11!null x.b:12!null rowid:13!null + ├── columns: a:11!null b:12!null x.a:13!null x.b:14!null rowid:15!null crdb_internal_mvcc_timestamp:16 ├── with-scan &2 (t2) - │ ├── columns: a:9!null b:10!null + │ ├── columns: a:11!null b:12!null │ └── mapping: - │ ├── x.a:4 => a:9 - │ └── x.b:5 => b:10 + │ ├── x.a:5 => a:11 + │ └── x.b:6 => b:12 ├── scan x - │ └── columns: x.a:11 x.b:12 rowid:13!null + │ └── columns: x.a:13 x.b:14 rowid:15!null crdb_internal_mvcc_timestamp:16 └── filters - ├── a:9 = x.a:11 - └── b:10 = x.b:12 + ├── a:11 = x.a:13 + └── b:12 = x.b:14 build WITH t AS (SELECT a FROM y WHERE a < 3) @@ -411,15 +411,15 @@ WITH t(x) AS (SELECT a FROM x) SELECT x FROM (SELECT x FROM t) ---- with &1 (t) - ├── columns: x:4 + ├── columns: x:5 ├── project │ ├── columns: a:1 │ └── scan x - │ └── columns: a:1 b:2 rowid:3!null + │ └── columns: a:1 b:2 rowid:3!null crdb_internal_mvcc_timestamp:4 └── with-scan &1 (t) - ├── columns: x:4 + ├── columns: x:5 └── mapping: - └── a:1 => x:4 + └── a:1 => x:5 build WITH t(a, b) AS (SELECT true a, false b) @@ -464,48 +464,48 @@ WITH t AS (SELECT a FROM x) SELECT * FROM y WHERE a IN (SELECT * FROM t) ---- with &1 (t) - ├── columns: a:4 + ├── columns: a:5 ├── project │ ├── columns: x.a:1 │ └── scan x - │ └── columns: x.a:1 b:2 x.rowid:3!null + │ └── columns: x.a:1 b:2 x.rowid:3!null x.crdb_internal_mvcc_timestamp:4 └── project - ├── columns: y.a:4 + ├── columns: y.a:5 └── select - ├── columns: y.a:4 y.rowid:5!null + ├── columns: y.a:5 y.rowid:6!null y.crdb_internal_mvcc_timestamp:7 ├── scan y - │ └── columns: y.a:4 y.rowid:5!null + │ └── columns: y.a:5 y.rowid:6!null y.crdb_internal_mvcc_timestamp:7 └── filters └── any: eq ├── with-scan &1 (t) - │ ├── columns: a:6 + │ ├── columns: a:8 │ └── mapping: - │ └── x.a:1 => a:6 - └── y.a:4 + │ └── x.a:1 => a:8 + └── y.a:5 build WITH t(x) AS (SELECT a FROM x) SELECT * FROM y WHERE a IN (SELECT x FROM t) ---- with &1 (t) - ├── columns: a:4 + ├── columns: a:5 ├── project │ ├── columns: x.a:1 │ └── scan x - │ └── columns: x.a:1 b:2 x.rowid:3!null + │ └── columns: x.a:1 b:2 x.rowid:3!null x.crdb_internal_mvcc_timestamp:4 └── project - ├── columns: y.a:4 + ├── columns: y.a:5 └── select - ├── columns: y.a:4 y.rowid:5!null + ├── columns: y.a:5 y.rowid:6!null y.crdb_internal_mvcc_timestamp:7 ├── scan y - │ └── columns: y.a:4 y.rowid:5!null + │ └── columns: y.a:5 y.rowid:6!null y.crdb_internal_mvcc_timestamp:7 └── filters └── any: eq ├── with-scan &1 (t) - │ ├── columns: x:6 + │ ├── columns: x:8 │ └── mapping: - │ └── x.a:1 => x:6 - └── y.a:4 + │ └── x.a:1 => x:8 + └── y.a:5 # Using a subquery inside a CTE build @@ -515,25 +515,25 @@ SELECT * FROM x WHERE a IN with &1 (t) ├── columns: a:1 b:2 ├── project - │ ├── columns: y.a:4!null + │ ├── columns: y.a:5!null │ └── select - │ ├── columns: y.a:4!null y.rowid:5!null + │ ├── columns: y.a:5!null y.rowid:6!null y.crdb_internal_mvcc_timestamp:7 │ ├── scan y - │ │ └── columns: y.a:4 y.rowid:5!null + │ │ └── columns: y.a:5 y.rowid:6!null y.crdb_internal_mvcc_timestamp:7 │ └── filters - │ └── y.a:4 < 3 + │ └── y.a:5 < 3 └── project ├── columns: x.a:1 b:2 └── select - ├── columns: x.a:1 b:2 x.rowid:3!null + ├── columns: x.a:1 b:2 x.rowid:3!null x.crdb_internal_mvcc_timestamp:4 ├── scan x - │ └── columns: x.a:1 b:2 x.rowid:3!null + │ └── columns: x.a:1 b:2 x.rowid:3!null x.crdb_internal_mvcc_timestamp:4 └── filters └── any: eq ├── with-scan &1 (t) - │ ├── columns: a:6!null + │ ├── columns: a:8!null │ └── mapping: - │ └── y.a:4 => a:6 + │ └── y.a:5 => a:8 └── x.a:1 # Using a correlated subquery inside a CTE @@ -547,15 +547,15 @@ build WITH t(b) AS (SELECT a FROM x) SELECT b, t.b FROM t ---- with &1 (t) - ├── columns: b:4 b:4 + ├── columns: b:5 b:5 ├── project │ ├── columns: a:1 │ └── scan x - │ └── columns: a:1 x.b:2 rowid:3!null + │ └── columns: a:1 x.b:2 rowid:3!null crdb_internal_mvcc_timestamp:4 └── with-scan &1 (t) - ├── columns: b:4 + ├── columns: b:5 └── mapping: - └── a:1 => b:4 + └── a:1 => b:5 build WITH t(b, c) AS (SELECT a FROM x) SELECT b, t.b FROM t @@ -605,141 +605,141 @@ WITH one AS (SELECT a AS u FROM x), SELECT * FROM one JOIN two ON u = v ---- with &1 (one) - ├── columns: u:8!null v:9!null + ├── columns: u:10!null v:11!null ├── project │ ├── columns: x.a:1 │ └── scan x - │ └── columns: x.a:1 b:2 x.rowid:3!null + │ └── columns: x.a:1 b:2 x.rowid:3!null x.crdb_internal_mvcc_timestamp:4 └── with &2 (two) - ├── columns: u:8!null v:9!null + ├── columns: u:10!null v:11!null ├── union-all - │ ├── columns: a:7 - │ ├── left columns: y.a:4 - │ ├── right columns: u:6 + │ ├── columns: a:9 + │ ├── left columns: y.a:5 + │ ├── right columns: u:8 │ ├── project - │ │ ├── columns: y.a:4 + │ │ ├── columns: y.a:5 │ │ └── scan y - │ │ └── columns: y.a:4 y.rowid:5!null + │ │ └── columns: y.a:5 y.rowid:6!null y.crdb_internal_mvcc_timestamp:7 │ └── with-scan &1 (one) - │ ├── columns: u:6 + │ ├── columns: u:8 │ └── mapping: - │ └── x.a:1 => u:6 + │ └── x.a:1 => u:8 └── inner-join (hash) - ├── columns: u:8!null v:9!null + ├── columns: u:10!null v:11!null ├── with-scan &1 (one) - │ ├── columns: u:8 + │ ├── columns: u:10 │ └── mapping: - │ └── x.a:1 => u:8 + │ └── x.a:1 => u:10 ├── with-scan &2 (two) - │ ├── columns: v:9 + │ ├── columns: v:11 │ └── mapping: - │ └── a:7 => v:9 + │ └── a:9 => v:11 └── filters - └── u:8 = v:9 + └── u:10 = v:11 build WITH foo AS (SELECT x.a FROM x ORDER by x.a) SELECT * FROM foo ---- with &1 (foo) - ├── columns: a:4 + ├── columns: a:5 ├── project │ ├── columns: x.a:1 │ └── scan x - │ └── columns: x.a:1 b:2 rowid:3!null + │ └── columns: x.a:1 b:2 rowid:3!null crdb_internal_mvcc_timestamp:4 └── with-scan &1 (foo) - ├── columns: a:4 + ├── columns: a:5 └── mapping: - └── x.a:1 => a:4 + └── x.a:1 => a:5 # Mutations. build WITH t AS (SELECT a FROM x) INSERT INTO x SELECT a + 20 FROM t RETURNING * ---- with &1 (t) - ├── columns: a:4 b:5 + ├── columns: a:5 b:6 ├── project │ ├── columns: x.a:1 │ └── scan x - │ └── columns: x.a:1 b:2 rowid:3!null + │ └── columns: x.a:1 b:2 rowid:3!null crdb_internal_mvcc_timestamp:4 └── project - ├── columns: x.a:4 b:5 + ├── columns: x.a:5 b:6 └── insert x - ├── columns: x.a:4 b:5 rowid:6!null + ├── columns: x.a:5 b:6 rowid:7!null ├── insert-mapping: - │ ├── "?column?":8 => x.a:4 - │ ├── column9:9 => b:5 - │ └── column10:10 => rowid:6 + │ ├── "?column?":10 => x.a:5 + │ ├── column11:11 => b:6 + │ └── column12:12 => rowid:7 └── project - ├── columns: column9:9 column10:10 "?column?":8 + ├── columns: column11:11 column12:12 "?column?":10 ├── project - │ ├── columns: "?column?":8 + │ ├── columns: "?column?":10 │ ├── with-scan &1 (t) - │ │ ├── columns: a:7 + │ │ ├── columns: a:9 │ │ └── mapping: - │ │ └── x.a:1 => a:7 + │ │ └── x.a:1 => a:9 │ └── projections - │ └── a:7 + 20 [as="?column?":8] + │ └── a:9 + 20 [as="?column?":10] └── projections - ├── NULL::INT8 [as=column9:9] - └── unique_rowid() [as=column10:10] + ├── NULL::INT8 [as=column11:11] + └── unique_rowid() [as=column12:12] build WITH t AS (SELECT a FROM x) UPDATE x SET a = (SELECT * FROM t) RETURNING * ---- with &1 (t) - ├── columns: a:4 b:5 + ├── columns: a:5 b:6 ├── project │ ├── columns: x.a:1 │ └── scan x - │ └── columns: x.a:1 b:2 rowid:3!null + │ └── columns: x.a:1 b:2 rowid:3!null crdb_internal_mvcc_timestamp:4 └── project - ├── columns: x.a:4 b:5 + ├── columns: x.a:5 b:6 └── update x - ├── columns: x.a:4 b:5 rowid:6!null - ├── fetch columns: x.a:7 b:8 rowid:9 + ├── columns: x.a:5 b:6 rowid:7!null + ├── fetch columns: x.a:9 b:10 rowid:11 ├── update-mapping: - │ └── a_new:11 => x.a:4 + │ └── a_new:14 => x.a:5 └── project - ├── columns: a_new:11 x.a:7 b:8 rowid:9!null + ├── columns: a_new:14 x.a:9 b:10 rowid:11!null crdb_internal_mvcc_timestamp:12 ├── scan x - │ └── columns: x.a:7 b:8 rowid:9!null + │ └── columns: x.a:9 b:10 rowid:11!null crdb_internal_mvcc_timestamp:12 └── projections - └── subquery [as=a_new:11] + └── subquery [as=a_new:14] └── max1-row - ├── columns: a:10 + ├── columns: a:13 └── with-scan &1 (t) - ├── columns: a:10 + ├── columns: a:13 └── mapping: - └── x.a:1 => a:10 + └── x.a:1 => a:13 build WITH t AS (SELECT a FROM x) DELETE FROM x WHERE a = (SELECT * FROM t) RETURNING * ---- with &1 (t) - ├── columns: a:4!null b:5 + ├── columns: a:5!null b:6 ├── project │ ├── columns: x.a:1 │ └── scan x - │ └── columns: x.a:1 b:2 rowid:3!null + │ └── columns: x.a:1 b:2 rowid:3!null crdb_internal_mvcc_timestamp:4 └── project - ├── columns: x.a:4!null b:5 + ├── columns: x.a:5!null b:6 └── delete x - ├── columns: x.a:4!null b:5 rowid:6!null - ├── fetch columns: x.a:7 b:8 rowid:9 + ├── columns: x.a:5!null b:6 rowid:7!null + ├── fetch columns: x.a:9 b:10 rowid:11 └── select - ├── columns: x.a:7!null b:8 rowid:9!null + ├── columns: x.a:9!null b:10 rowid:11!null crdb_internal_mvcc_timestamp:12 ├── scan x - │ └── columns: x.a:7 b:8 rowid:9!null + │ └── columns: x.a:9 b:10 rowid:11!null crdb_internal_mvcc_timestamp:12 └── filters └── eq - ├── x.a:7 + ├── x.a:9 └── subquery └── max1-row - ├── columns: a:10 + ├── columns: a:13 └── with-scan &1 (t) - ├── columns: a:10 + ├── columns: a:13 └── mapping: - └── x.a:1 => a:10 + └── x.a:1 => a:13 # Correlated WITH is not allowed. @@ -774,37 +774,37 @@ with &1 (t) ├── project │ ├── columns: xy.x:1!null │ └── limit - │ ├── columns: xy.x:1!null v:6 - │ ├── internal-ordering: -6 + │ ├── columns: xy.x:1!null v:7 + │ ├── internal-ordering: -7 │ ├── sort - │ │ ├── columns: xy.x:1!null v:6 - │ │ ├── ordering: -6 + │ │ ├── columns: xy.x:1!null v:7 + │ │ ├── ordering: -7 │ │ ├── limit hint: 5.00 │ │ └── project - │ │ ├── columns: xy.x:1!null v:6 + │ │ ├── columns: xy.x:1!null v:7 │ │ └── inner-join (hash) - │ │ ├── columns: xy.x:1!null y:2 z:3 xy.rowid:4!null u:5!null v:6 uv.rowid:7!null + │ │ ├── columns: xy.x:1!null y:2 z:3 xy.rowid:4!null xy.crdb_internal_mvcc_timestamp:5 u:6!null v:7 uv.rowid:8!null uv.crdb_internal_mvcc_timestamp:9 │ │ ├── scan xy - │ │ │ └── columns: xy.x:1 y:2 z:3 xy.rowid:4!null + │ │ │ └── columns: xy.x:1 y:2 z:3 xy.rowid:4!null xy.crdb_internal_mvcc_timestamp:5 │ │ ├── scan uv - │ │ │ └── columns: u:5 v:6 uv.rowid:7!null + │ │ │ └── columns: u:6 v:7 uv.rowid:8!null uv.crdb_internal_mvcc_timestamp:9 │ │ └── filters - │ │ └── xy.x:1 = u:5 + │ │ └── xy.x:1 = u:6 │ └── 5 └── delete xy ├── columns: - ├── fetch columns: xy.x:12 y:13 z:14 xy.rowid:15 + ├── fetch columns: xy.x:15 y:16 z:17 xy.rowid:18 └── select - ├── columns: xy.x:12 y:13 z:14 xy.rowid:15!null + ├── columns: xy.x:15 y:16 z:17 xy.rowid:18!null xy.crdb_internal_mvcc_timestamp:19 ├── scan xy - │ └── columns: xy.x:12 y:13 z:14 xy.rowid:15!null + │ └── columns: xy.x:15 y:16 z:17 xy.rowid:18!null xy.crdb_internal_mvcc_timestamp:19 └── filters └── any: eq ├── with-scan &1 (t) - │ ├── columns: x:16!null + │ ├── columns: x:20!null │ └── mapping: - │ └── xy.x:1 => x:16 - └── xy.x:12 + │ └── xy.x:1 => x:20 + └── xy.x:15 # Check hidden column handling: level, node_type should not be output. build @@ -891,51 +891,51 @@ FROM included_parts GROUP BY sub_part ---- with &2 (included_parts) - ├── columns: sub_part:15 total_quantity:18 + ├── columns: sub_part:17 total_quantity:20 ├── recursive-c-t-e - │ ├── columns: sub_part:5 part:6 quantity:7 + │ ├── columns: sub_part:6 part:7 quantity:8 │ ├── working table binding: &1 │ ├── initial columns: parts.sub_part:2 parts.part:1 parts.quantity:3 - │ ├── recursive columns: p.sub_part:12 p.part:11 p.quantity:13 + │ ├── recursive columns: p.sub_part:13 p.part:12 p.quantity:14 │ ├── project │ │ ├── columns: parts.part:1!null parts.sub_part:2 parts.quantity:3 │ │ └── select - │ │ ├── columns: parts.part:1!null parts.sub_part:2 parts.quantity:3 parts.rowid:4!null + │ │ ├── columns: parts.part:1!null parts.sub_part:2 parts.quantity:3 parts.rowid:4!null parts.crdb_internal_mvcc_timestamp:5 │ │ ├── scan parts - │ │ │ └── columns: parts.part:1 parts.sub_part:2 parts.quantity:3 parts.rowid:4!null + │ │ │ └── columns: parts.part:1 parts.sub_part:2 parts.quantity:3 parts.rowid:4!null parts.crdb_internal_mvcc_timestamp:5 │ │ └── filters │ │ └── parts.part:1 = 'our_product' │ └── project - │ ├── columns: p.part:11!null p.sub_part:12 p.quantity:13 + │ ├── columns: p.part:12!null p.sub_part:13 p.quantity:14 │ └── select - │ ├── columns: sub_part:8!null part:9 quantity:10 p.part:11!null p.sub_part:12 p.quantity:13 p.rowid:14!null + │ ├── columns: sub_part:9!null part:10 quantity:11 p.part:12!null p.sub_part:13 p.quantity:14 p.rowid:15!null p.crdb_internal_mvcc_timestamp:16 │ ├── inner-join (cross) - │ │ ├── columns: sub_part:8 part:9 quantity:10 p.part:11 p.sub_part:12 p.quantity:13 p.rowid:14!null + │ │ ├── columns: sub_part:9 part:10 quantity:11 p.part:12 p.sub_part:13 p.quantity:14 p.rowid:15!null p.crdb_internal_mvcc_timestamp:16 │ │ ├── with-scan &1 (included_parts) - │ │ │ ├── columns: sub_part:8 part:9 quantity:10 + │ │ │ ├── columns: sub_part:9 part:10 quantity:11 │ │ │ └── mapping: - │ │ │ ├── sub_part:5 => sub_part:8 - │ │ │ ├── part:6 => part:9 - │ │ │ └── quantity:7 => quantity:10 + │ │ │ ├── sub_part:6 => sub_part:9 + │ │ │ ├── part:7 => part:10 + │ │ │ └── quantity:8 => quantity:11 │ │ ├── scan p - │ │ │ └── columns: p.part:11 p.sub_part:12 p.quantity:13 p.rowid:14!null + │ │ │ └── columns: p.part:12 p.sub_part:13 p.quantity:14 p.rowid:15!null p.crdb_internal_mvcc_timestamp:16 │ │ └── filters (true) │ └── filters - │ └── p.part:11 = sub_part:8 + │ └── p.part:12 = sub_part:9 └── group-by - ├── columns: sub_part:15 sum:18 - ├── grouping columns: sub_part:15 + ├── columns: sub_part:17 sum:20 + ├── grouping columns: sub_part:17 ├── project - │ ├── columns: sub_part:15 quantity:17 + │ ├── columns: sub_part:17 quantity:19 │ └── with-scan &2 (included_parts) - │ ├── columns: sub_part:15 part:16 quantity:17 + │ ├── columns: sub_part:17 part:18 quantity:19 │ └── mapping: - │ ├── sub_part:5 => sub_part:15 - │ ├── part:6 => part:16 - │ └── quantity:7 => quantity:17 + │ ├── sub_part:6 => sub_part:17 + │ ├── part:7 => part:18 + │ └── quantity:8 => quantity:19 └── aggregations - └── sum [as=sum:18] - └── quantity:17 + └── sum [as=sum:20] + └── quantity:19 exec-ddl @@ -958,53 +958,53 @@ WITH RECURSIVE search_graph(id, link, data, depth, path, cycle) AS ( SELECT * FROM search_graph ---- with &2 (search_graph) - ├── columns: id:25 link:26 data:27 depth:28 path:29 cycle:30 + ├── columns: id:27 link:28 data:29 depth:30 path:31 cycle:32 ├── recursive-c-t-e - │ ├── columns: id:7 link:8 data:9 depth:10 path:11 cycle:12 + │ ├── columns: id:8 link:9 data:10 depth:11 path:12 cycle:13 │ ├── working table binding: &1 - │ ├── initial columns: g.id:1 g.link:2 g.data:3 "?column?":4 array:5 bool:6 - │ ├── recursive columns: g.id:13 g.link:14 g.data:15 "?column?":22 "?column?":23 "?column?":24 + │ ├── initial columns: g.id:1 g.link:2 g.data:3 "?column?":5 array:6 bool:7 + │ ├── recursive columns: g.id:14 g.link:15 g.data:16 "?column?":24 "?column?":25 "?column?":26 │ ├── project - │ │ ├── columns: "?column?":4!null array:5!null bool:6!null g.id:1!null g.link:2 g.data:3 + │ │ ├── columns: "?column?":5!null array:6!null bool:7!null g.id:1!null g.link:2 g.data:3 │ │ ├── scan g - │ │ │ └── columns: g.id:1!null g.link:2 g.data:3 + │ │ │ └── columns: g.id:1!null g.link:2 g.data:3 crdb_internal_mvcc_timestamp:4 │ │ └── projections - │ │ ├── 1 [as="?column?":4] - │ │ ├── ARRAY[g.id:1] [as=array:5] - │ │ └── false [as=bool:6] + │ │ ├── 1 [as="?column?":5] + │ │ ├── ARRAY[g.id:1] [as=array:6] + │ │ └── false [as=bool:7] │ └── project - │ ├── columns: "?column?":22 "?column?":23 "?column?":24 g.id:13!null g.link:14 g.data:15 + │ ├── columns: "?column?":24 "?column?":25 "?column?":26 g.id:14!null g.link:15 g.data:16 │ ├── select - │ │ ├── columns: g.id:13!null g.link:14 g.data:15 id:16 link:17!null data:18 depth:19 path:20 cycle:21!null + │ │ ├── columns: g.id:14!null g.link:15 g.data:16 crdb_internal_mvcc_timestamp:17 id:18 link:19!null data:20 depth:21 path:22 cycle:23!null │ │ ├── inner-join (cross) - │ │ │ ├── columns: g.id:13!null g.link:14 g.data:15 id:16 link:17 data:18 depth:19 path:20 cycle:21 + │ │ │ ├── columns: g.id:14!null g.link:15 g.data:16 crdb_internal_mvcc_timestamp:17 id:18 link:19 data:20 depth:21 path:22 cycle:23 │ │ │ ├── scan g - │ │ │ │ └── columns: g.id:13!null g.link:14 g.data:15 + │ │ │ │ └── columns: g.id:14!null g.link:15 g.data:16 crdb_internal_mvcc_timestamp:17 │ │ │ ├── with-scan &1 (search_graph) - │ │ │ │ ├── columns: id:16 link:17 data:18 depth:19 path:20 cycle:21 + │ │ │ │ ├── columns: id:18 link:19 data:20 depth:21 path:22 cycle:23 │ │ │ │ └── mapping: - │ │ │ │ ├── id:7 => id:16 - │ │ │ │ ├── link:8 => link:17 - │ │ │ │ ├── data:9 => data:18 - │ │ │ │ ├── depth:10 => depth:19 - │ │ │ │ ├── path:11 => path:20 - │ │ │ │ └── cycle:12 => cycle:21 + │ │ │ │ ├── id:8 => id:18 + │ │ │ │ ├── link:9 => link:19 + │ │ │ │ ├── data:10 => data:20 + │ │ │ │ ├── depth:11 => depth:21 + │ │ │ │ ├── path:12 => path:22 + │ │ │ │ └── cycle:13 => cycle:23 │ │ │ └── filters (true) │ │ └── filters - │ │ └── (g.id:13 = link:17) AND (NOT cycle:21) + │ │ └── (g.id:14 = link:19) AND (NOT cycle:23) │ └── projections - │ ├── depth:19 + 1 [as="?column?":22] - │ ├── path:20 || g.id:13 [as="?column?":23] - │ └── g.id:13 = ANY path:20 [as="?column?":24] + │ ├── depth:21 + 1 [as="?column?":24] + │ ├── path:22 || g.id:14 [as="?column?":25] + │ └── g.id:14 = ANY path:22 [as="?column?":26] └── with-scan &2 (search_graph) - ├── columns: id:25 link:26 data:27 depth:28 path:29 cycle:30 + ├── columns: id:27 link:28 data:29 depth:30 path:31 cycle:32 └── mapping: - ├── id:7 => id:25 - ├── link:8 => link:26 - ├── data:9 => data:27 - ├── depth:10 => depth:28 - ├── path:11 => path:29 - └── cycle:12 => cycle:30 + ├── id:8 => id:27 + ├── link:9 => link:28 + ├── data:10 => data:29 + ├── depth:11 => depth:30 + ├── path:12 => path:31 + └── cycle:13 => cycle:32 # Test where initial query has duplicate columns. build @@ -1341,27 +1341,27 @@ build EXPLAIN WITH foo AS (INSERT INTO y VALUES (1) RETURNING *) SELECT * FROM foo ---- explain - ├── columns: tree:6 field:7 description:8 + ├── columns: tree:7 field:8 description:9 └── with &1 (foo) - ├── columns: a:5!null + ├── columns: a:6!null ├── project │ ├── columns: y.a:1!null │ └── insert y │ ├── columns: y.a:1!null rowid:2!null │ ├── insert-mapping: - │ │ ├── column1:3 => y.a:1 - │ │ └── column4:4 => rowid:2 + │ │ ├── column1:4 => y.a:1 + │ │ └── column5:5 => rowid:2 │ └── project - │ ├── columns: column4:4 column1:3!null + │ ├── columns: column5:5 column1:4!null │ ├── values - │ │ ├── columns: column1:3!null + │ │ ├── columns: column1:4!null │ │ └── (1,) │ └── projections - │ └── unique_rowid() [as=column4:4] + │ └── unique_rowid() [as=column5:5] └── with-scan &1 (foo) - ├── columns: a:5!null + ├── columns: a:6!null └── mapping: - └── y.a:1 => a:5 + └── y.a:1 => a:6 exec-ddl CREATE TABLE a(x INT); @@ -1373,27 +1373,27 @@ INSERT INTO a(x) ---- with &1 (b) ├── values - │ ├── columns: column1:3!null + │ ├── columns: column1:4!null │ ├── (1,) │ ├── (2,) │ └── (3,) └── insert a ├── columns: ├── insert-mapping: - │ ├── w:5 => x:1 - │ └── column6:6 => rowid:2 + │ ├── w:6 => x:1 + │ └── column7:7 => rowid:2 └── project - ├── columns: column6:6 w:5!null + ├── columns: column7:7 w:6!null ├── project - │ ├── columns: w:5!null + │ ├── columns: w:6!null │ ├── with-scan &1 (b) - │ │ ├── columns: z:4!null + │ │ ├── columns: z:5!null │ │ └── mapping: - │ │ └── column1:3 => z:4 + │ │ └── column1:4 => z:5 │ └── projections - │ └── z:4 + 1 [as=w:5] + │ └── z:5 + 1 [as=w:6] └── projections - └── unique_rowid() [as=column6:6] + └── unique_rowid() [as=column7:7] build SELECT @@ -1451,53 +1451,53 @@ WITH t AS MATERIALIZED (SELECT a FROM y WHERE a < 3) SELECT * FROM x NATURAL JOIN t ---- with &1 (t) - ├── columns: a:3!null b:4 + ├── columns: a:4!null b:5 ├── materialized ├── project │ ├── columns: y.a:1!null │ └── select - │ ├── columns: y.a:1!null y.rowid:2!null + │ ├── columns: y.a:1!null y.rowid:2!null y.crdb_internal_mvcc_timestamp:3 │ ├── scan y - │ │ └── columns: y.a:1 y.rowid:2!null + │ │ └── columns: y.a:1 y.rowid:2!null y.crdb_internal_mvcc_timestamp:3 │ └── filters │ └── y.a:1 < 3 └── project - ├── columns: x.a:3!null b:4 + ├── columns: x.a:4!null b:5 └── inner-join (hash) - ├── columns: x.a:3!null b:4 x.rowid:5!null a:6!null + ├── columns: x.a:4!null b:5 x.rowid:6!null x.crdb_internal_mvcc_timestamp:7 a:8!null ├── scan x - │ └── columns: x.a:3 b:4 x.rowid:5!null + │ └── columns: x.a:4 b:5 x.rowid:6!null x.crdb_internal_mvcc_timestamp:7 ├── with-scan &1 (t) - │ ├── columns: a:6!null + │ ├── columns: a:8!null │ └── mapping: - │ └── y.a:1 => a:6 + │ └── y.a:1 => a:8 └── filters - └── x.a:3 = a:6 + └── x.a:4 = a:8 build WITH t AS NOT MATERIALIZED (SELECT a FROM y WHERE a < 3) SELECT * FROM x NATURAL JOIN t ---- with &1 (t) - ├── columns: a:3!null b:4 + ├── columns: a:4!null b:5 ├── not-materialized ├── project │ ├── columns: y.a:1!null │ └── select - │ ├── columns: y.a:1!null y.rowid:2!null + │ ├── columns: y.a:1!null y.rowid:2!null y.crdb_internal_mvcc_timestamp:3 │ ├── scan y - │ │ └── columns: y.a:1 y.rowid:2!null + │ │ └── columns: y.a:1 y.rowid:2!null y.crdb_internal_mvcc_timestamp:3 │ └── filters │ └── y.a:1 < 3 └── project - ├── columns: x.a:3!null b:4 + ├── columns: x.a:4!null b:5 └── inner-join (hash) - ├── columns: x.a:3!null b:4 x.rowid:5!null a:6!null + ├── columns: x.a:4!null b:5 x.rowid:6!null x.crdb_internal_mvcc_timestamp:7 a:8!null ├── scan x - │ └── columns: x.a:3 b:4 x.rowid:5!null + │ └── columns: x.a:4 b:5 x.rowid:6!null x.crdb_internal_mvcc_timestamp:7 ├── with-scan &1 (t) - │ ├── columns: a:6!null + │ ├── columns: a:8!null │ └── mapping: - │ └── y.a:1 => a:6 + │ └── y.a:1 => a:8 └── filters - └── x.a:3 = a:6 + └── x.a:4 = a:8 diff --git a/pkg/sql/opt/optgen/exprgen/testdata/join b/pkg/sql/opt/optgen/exprgen/testdata/join index e397c0bace81..63bd7fc07536 100644 --- a/pkg/sql/opt/optgen/exprgen/testdata/join +++ b/pkg/sql/opt/optgen/exprgen/testdata/join @@ -15,11 +15,11 @@ expr ) ---- inner-join (hash) - ├── columns: t.public.abc.a:1(int!null) t.public.abc.b:2(int) t.public.abc.c:3(int) t.public.def.d:5(int!null) t.public.def.e:6(int) t.public.def.f:7(int) - ├── stats: [rows=9801, distinct(1)=99, null(1)=0, distinct(5)=99, null(5)=0] + ├── columns: t.public.abc.a:1(int!null) t.public.abc.b:2(int) t.public.abc.c:3(int) t.public.def.d:6(int!null) t.public.def.e:7(int) t.public.def.f:8(int) + ├── stats: [rows=9801, distinct(1)=99, null(1)=0, distinct(6)=99, null(6)=0] ├── cost: 2268.07 - ├── fd: (1)==(5), (5)==(1) - ├── prune: (2,3,6,7) + ├── fd: (1)==(6), (6)==(1) + ├── prune: (2,3,7,8) ├── interesting orderings: (+1,+2) ├── scan t.public.abc │ ├── columns: t.public.abc.a:1(int) t.public.abc.b:2(int) t.public.abc.c:3(int) @@ -27,17 +27,17 @@ inner-join (hash) │ ├── cost: 1070.02 │ ├── prune: (1-3) │ ├── interesting orderings: (+1,+2) - │ └── unfiltered-cols: (1-4) + │ └── unfiltered-cols: (1-5) ├── scan t.public.def - │ ├── columns: t.public.def.d:5(int) t.public.def.e:6(int) t.public.def.f:7(int) - │ ├── stats: [rows=1000, distinct(5)=100, null(5)=10] + │ ├── columns: t.public.def.d:6(int) t.public.def.e:7(int) t.public.def.f:8(int) + │ ├── stats: [rows=1000, distinct(6)=100, null(6)=10] │ ├── cost: 1070.02 - │ ├── prune: (5-7) - │ └── unfiltered-cols: (5-8) + │ ├── prune: (6-8) + │ └── unfiltered-cols: (6-10) └── filters - └── eq [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + └── eq [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] ├── variable: t.public.abc.a:1 [type=int] - └── variable: t.public.def.d:5 [type=int] + └── variable: t.public.def.d:6 [type=int] expr (MakeLookupJoin @@ -47,9 +47,9 @@ expr ) ---- left-join (lookup abc@ab) - ├── columns: t.public.abc.a:5(int) t.public.abc.b:6(int) - ├── key columns: [5] = [5] - ├── stats: [rows=3333.33333, distinct(5)=100, null(5)=33.3333333] + ├── columns: t.public.abc.a:6(int) t.public.abc.b:7(int) + ├── key columns: [6] = [6] + ├── stats: [rows=3333.33333, distinct(6)=100, null(6)=33.3333333] ├── cost: 41660.04 ├── scan t.public.def │ ├── columns: t.public.def.d:1(int) t.public.def.e:2(int) @@ -57,8 +57,8 @@ left-join (lookup abc@ab) │ ├── cost: 1060.02 │ └── prune: (1,2) └── filters - └── gt [type=bool, outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ])] - ├── variable: t.public.abc.a:5 [type=int] + └── gt [type=bool, outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + ├── variable: t.public.abc.a:6 [type=int] └── variable: t.public.def.e:2 [type=int] expr @@ -76,12 +76,12 @@ expr ) ---- inner-join (merge) - ├── columns: t.public.abc.a:1(int!null) t.public.abc.b:2(int) t.public.abc.c:3(int) t.public.def.d:5(int!null) t.public.def.e:6(int) t.public.def.f:7(int) + ├── columns: t.public.abc.a:1(int!null) t.public.abc.b:2(int) t.public.abc.c:3(int) t.public.def.d:6(int!null) t.public.def.e:7(int) t.public.def.f:8(int) ├── left ordering: +1 - ├── right ordering: +5 - ├── stats: [rows=9801, distinct(1)=99, null(1)=0, distinct(5)=99, null(5)=0] + ├── right ordering: +6 + ├── stats: [rows=9801, distinct(1)=99, null(1)=0, distinct(6)=99, null(6)=0] ├── cost: 2696.71137 - ├── fd: (1)==(5), (5)==(1) + ├── fd: (1)==(6), (6)==(1) ├── sort │ ├── columns: t.public.abc.a:1(int) t.public.abc.b:2(int) t.public.abc.c:3(int) │ ├── stats: [rows=1000, distinct(1)=100, null(1)=10] @@ -96,16 +96,16 @@ inner-join (merge) │ ├── prune: (1-3) │ └── interesting orderings: (+1,+2) ├── sort - │ ├── columns: t.public.def.d:5(int) t.public.def.e:6(int) t.public.def.f:7(int) - │ ├── stats: [rows=1000, distinct(5)=100, null(5)=10] + │ ├── columns: t.public.def.d:6(int) t.public.def.e:7(int) t.public.def.f:8(int) + │ ├── stats: [rows=1000, distinct(6)=100, null(6)=10] │ ├── cost: 1289.34569 - │ ├── ordering: +5 - │ ├── prune: (5-7) + │ ├── ordering: +6 + │ ├── prune: (6-8) │ └── scan t.public.def - │ ├── columns: t.public.def.d:5(int) t.public.def.e:6(int) t.public.def.f:7(int) - │ ├── stats: [rows=1000, distinct(5)=100, null(5)=10] + │ ├── columns: t.public.def.d:6(int) t.public.def.e:7(int) t.public.def.f:8(int) + │ ├── stats: [rows=1000, distinct(6)=100, null(6)=10] │ ├── cost: 1070.02 - │ └── prune: (5-7) + │ └── prune: (6-8) └── filters (true) expr @@ -120,11 +120,11 @@ expr ) ---- inner-join-apply - ├── columns: t.public.abc.a:1(int) t.public.abc.b:2(int) t.public.abc.c:3(int) t.public.def.d:5(int) t.public.def.e:6(int) t.public.def.f:7(int) + ├── columns: t.public.abc.a:1(int) t.public.abc.b:2(int) t.public.abc.c:3(int) t.public.def.d:6(int) t.public.def.e:7(int) t.public.def.f:8(int) ├── immutable ├── stats: [rows=333333.333] ├── cost: 5611.40451 - ├── prune: (7) + ├── prune: (8) ├── interesting orderings: (+1,+2) ├── sort │ ├── columns: t.public.abc.a:1(int) t.public.abc.b:2(int) t.public.abc.c:3(int) @@ -137,23 +137,23 @@ inner-join-apply │ ├── cost: 1070.02 │ └── interesting orderings: (+1,+2) ├── select - │ ├── columns: t.public.def.d:5(int) t.public.def.e:6(int) t.public.def.f:7(int) + │ ├── columns: t.public.def.d:6(int) t.public.def.e:7(int) t.public.def.f:8(int) │ ├── outer: (1) │ ├── immutable │ ├── stats: [rows=333.333333, distinct(1)=1, null(1)=0] │ ├── cost: 1080.04 - │ ├── prune: (7) + │ ├── prune: (8) │ ├── scan t.public.def - │ │ ├── columns: t.public.def.d:5(int) t.public.def.e:6(int) t.public.def.f:7(int) + │ │ ├── columns: t.public.def.d:6(int) t.public.def.e:7(int) t.public.def.f:8(int) │ │ ├── stats: [rows=1000] │ │ ├── cost: 1070.02 - │ │ └── prune: (5-7) + │ │ └── prune: (6-8) │ └── filters - │ └── eq [type=bool, outer=(1,5,6), immutable, constraints=(/1: (/NULL - ])] + │ └── eq [type=bool, outer=(1,6,7), immutable, constraints=(/1: (/NULL - ])] │ ├── variable: t.public.abc.a:1 [type=int] │ └── plus [type=int] - │ ├── variable: t.public.def.d:5 [type=int] - │ └── variable: t.public.def.e:6 [type=int] + │ ├── variable: t.public.def.d:6 [type=int] + │ └── variable: t.public.def.e:7 [type=int] └── filters (true) expr diff --git a/pkg/sql/opt/ordering_test.go b/pkg/sql/opt/ordering_test.go index 6d64873a12ef..cafb30de7b2b 100644 --- a/pkg/sql/opt/ordering_test.go +++ b/pkg/sql/opt/ordering_test.go @@ -160,12 +160,12 @@ func TestOrderingColumn_RemapColumn(t *testing.T) { t.Errorf("\ncol2 was changed: %s\n", col2.String()) } - expected = "+5" + expected = "+6" if remappedCol1.String() != expected { t.Errorf("\nexpected: %s\nactual: %s\n", expected, remappedCol1.String()) } - expected = "-8" + expected = "-9" if remappedCol2.String() != expected { t.Errorf("\nexpected: %s\nactual: %s\n", expected, remappedCol2.String()) } diff --git a/pkg/sql/opt/testutils/opttester/testdata/opt-steps b/pkg/sql/opt/testutils/opttester/testdata/opt-steps index bf0bc3c21b2b..1d1b111f0f03 100644 --- a/pkg/sql/opt/testutils/opttester/testdata/opt-steps +++ b/pkg/sql/opt/testutils/opttester/testdata/opt-steps @@ -55,20 +55,77 @@ SELECT * FROM ab WHERE b=1 ---- ================================================================================ Initial expression - Cost: 1050.04 + Cost: 1060.15 ================================================================================ - select + project ├── columns: a:1(int!null) b:2(int!null) ├── key: (1) ├── fd: ()-->(2) - ├── scan ab - │ ├── columns: a:1(int!null) b:2(int) - │ ├── key: (1) - │ └── fd: (1)-->(2) - └── filters - └── eq [type=bool, outer=(2), constraints=(/2: [/1 - /1]; tight), fd=()-->(2)] - ├── variable: b:2 [type=int] - └── const: 1 [type=int] + └── select + ├── columns: a:1(int!null) b:2(int!null) crdb_internal_mvcc_timestamp:3(decimal) + ├── key: (1) + ├── fd: ()-->(2), (1)-->(3) + ├── scan ab + │ ├── columns: a:1(int!null) b:2(int) crdb_internal_mvcc_timestamp:3(decimal) + │ ├── key: (1) + │ └── fd: (1)-->(2,3) + └── filters + └── eq [type=bool, outer=(2), constraints=(/2: [/1 - /1]; tight), fd=()-->(2)] + ├── variable: b:2 [type=int] + └── const: 1 [type=int] +================================================================================ +PruneSelectCols + Cost: 1050.15 +================================================================================ + project + ├── columns: a:1(int!null) b:2(int!null) + ├── key: (1) + ├── fd: ()-->(2) + └── select + - ├── columns: a:1(int!null) b:2(int!null) crdb_internal_mvcc_timestamp:3(decimal) + + ├── columns: a:1(int!null) b:2(int!null) + ├── key: (1) + - ├── fd: ()-->(2), (1)-->(3) + + ├── fd: ()-->(2) + ├── scan ab + - │ ├── columns: a:1(int!null) b:2(int) crdb_internal_mvcc_timestamp:3(decimal) + + │ ├── columns: a:1(int!null) b:2(int) + │ ├── key: (1) + - │ └── fd: (1)-->(2,3) + + │ └── fd: (1)-->(2) + └── filters + └── eq [type=bool, outer=(2), constraints=(/2: [/1 - /1]; tight), fd=()-->(2)] + ├── variable: b:2 [type=int] + └── const: 1 [type=int] +================================================================================ +EliminateProject + Cost: 1050.04 +================================================================================ + -project + +select + ├── columns: a:1(int!null) b:2(int!null) + ├── key: (1) + ├── fd: ()-->(2) + - └── select + - ├── columns: a:1(int!null) b:2(int!null) + - ├── key: (1) + - ├── fd: ()-->(2) + - ├── scan ab + - │ ├── columns: a:1(int!null) b:2(int) + - │ ├── key: (1) + - │ └── fd: (1)-->(2) + - └── filters + - └── eq [type=bool, outer=(2), constraints=(/2: [/1 - /1]; tight), fd=()-->(2)] + - ├── variable: b:2 [type=int] + - └── const: 1 [type=int] + + ├── scan ab + + │ ├── columns: a:1(int!null) b:2(int) + + │ ├── key: (1) + + │ └── fd: (1)-->(2) + + └── filters + + └── eq [type=bool, outer=(2), constraints=(/2: [/1 - /1]; tight), fd=()-->(2)] + + ├── variable: b:2 [type=int] + + └── const: 1 [type=int] -------------------------------------------------------------------------------- GenerateIndexScans (higher cost) -------------------------------------------------------------------------------- @@ -151,62 +208,195 @@ SELECT * FROM orders LEFT JOIN customers ON customer_id = customers.id ---- ================================================================================ Initial expression - Cost: 2160.06 + Cost: 2190.07 ================================================================================ - left-join (hash) - ├── columns: id:1(int!null) customer_id:2(int) status:3(string!null) id:4(int) name:5(string) address:6(string) - ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) + project + ├── columns: id:1(int!null) customer_id:2(int) status:3(string!null) id:5(int) name:6(string) address:7(string) ├── key: (1) - ├── fd: (1)-->(2-6), (4)-->(5,6) - ├── scan orders - │ ├── columns: orders.id:1(int!null) customer_id:2(int) status:3(string!null) - │ ├── check constraint expressions - │ │ └── in [type=bool, outer=(3), constraints=(/3: [/'cancelled' - /'cancelled'] [/'complete' - /'complete'] [/'open' - /'open']; tight)] - │ │ ├── variable: status:3 [type=string] - │ │ └── tuple [type=tuple{string, string, string}] - │ │ ├── const: 'open' [type=string] - │ │ ├── const: 'complete' [type=string] - │ │ └── const: 'cancelled' [type=string] - │ ├── key: (1) - │ └── fd: (1)-->(2,3) - ├── scan customers - │ ├── columns: customers.id:4(int!null) name:5(string!null) address:6(string) - │ ├── key: (4) - │ └── fd: (4)-->(5,6) - └── filters - └── eq [type=bool, outer=(2,4), constraints=(/2: (/NULL - ]; /4: (/NULL - ]), fd=(2)==(4), (4)==(2)] - ├── variable: customer_id:2 [type=int] - └── variable: customers.id:4 [type=int] + ├── fd: (1)-->(2,3,5-7), (5)-->(6,7) + └── left-join (hash) + ├── columns: orders.id:1(int!null) customer_id:2(int) status:3(string!null) orders.crdb_internal_mvcc_timestamp:4(decimal) customers.id:5(int) name:6(string) address:7(string) customers.crdb_internal_mvcc_timestamp:8(decimal) + ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) + ├── key: (1) + ├── fd: (1)-->(2-8), (5)-->(6-8) + ├── scan orders + │ ├── columns: orders.id:1(int!null) customer_id:2(int) status:3(string!null) orders.crdb_internal_mvcc_timestamp:4(decimal) + │ ├── check constraint expressions + │ │ └── in [type=bool, outer=(3), constraints=(/3: [/'cancelled' - /'cancelled'] [/'complete' - /'complete'] [/'open' - /'open']; tight)] + │ │ ├── variable: status:3 [type=string] + │ │ └── tuple [type=tuple{string, string, string}] + │ │ ├── const: 'open' [type=string] + │ │ ├── const: 'complete' [type=string] + │ │ └── const: 'cancelled' [type=string] + │ ├── key: (1) + │ └── fd: (1)-->(2-4) + ├── scan customers + │ ├── columns: customers.id:5(int!null) name:6(string!null) address:7(string) customers.crdb_internal_mvcc_timestamp:8(decimal) + │ ├── key: (5) + │ └── fd: (5)-->(6-8) + └── filters + └── eq [type=bool, outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ]), fd=(2)==(5), (5)==(2)] + ├── variable: customer_id:2 [type=int] + └── variable: customers.id:5 [type=int] ================================================================================ NormalizeInConst + Cost: 2190.07 +================================================================================ + project + ├── columns: id:1(int!null) customer_id:2(int) status:3(string!null) id:5(int) name:6(string) address:7(string) + ├── key: (1) + ├── fd: (1)-->(2,3,5-7), (5)-->(6,7) + └── left-join (hash) + ├── columns: orders.id:1(int!null) customer_id:2(int) status:3(string!null) orders.crdb_internal_mvcc_timestamp:4(decimal) customers.id:5(int) name:6(string) address:7(string) customers.crdb_internal_mvcc_timestamp:8(decimal) + ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) + ├── key: (1) + ├── fd: (1)-->(2-8), (5)-->(6-8) + ├── scan orders + │ ├── columns: orders.id:1(int!null) customer_id:2(int) status:3(string!null) orders.crdb_internal_mvcc_timestamp:4(decimal) + │ ├── check constraint expressions + │ │ └── in [type=bool, outer=(3), constraints=(/3: [/'cancelled' - /'cancelled'] [/'complete' - /'complete'] [/'open' - /'open']; tight)] + │ │ ├── variable: status:3 [type=string] + │ │ └── tuple [type=tuple{string, string, string}] + - │ │ ├── const: 'open' [type=string] + + │ │ ├── const: 'cancelled' [type=string] + │ │ ├── const: 'complete' [type=string] + - │ │ └── const: 'cancelled' [type=string] + + │ │ └── const: 'open' [type=string] + │ ├── key: (1) + │ └── fd: (1)-->(2-4) + ├── scan customers + │ ├── columns: customers.id:5(int!null) name:6(string!null) address:7(string) customers.crdb_internal_mvcc_timestamp:8(decimal) + │ ├── key: (5) + │ └── fd: (5)-->(6-8) + └── filters + └── eq [type=bool, outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ]), fd=(2)==(5), (5)==(2)] + ├── variable: customer_id:2 [type=int] + └── variable: customers.id:5 [type=int] +================================================================================ +PruneJoinLeftCols + Cost: 2180.07 +================================================================================ + project + ├── columns: id:1(int!null) customer_id:2(int) status:3(string!null) id:5(int) name:6(string) address:7(string) + ├── key: (1) + ├── fd: (1)-->(2,3,5-7), (5)-->(6,7) + └── left-join (hash) + - ├── columns: orders.id:1(int!null) customer_id:2(int) status:3(string!null) orders.crdb_internal_mvcc_timestamp:4(decimal) customers.id:5(int) name:6(string) address:7(string) customers.crdb_internal_mvcc_timestamp:8(decimal) + + ├── columns: orders.id:1(int!null) customer_id:2(int) status:3(string!null) customers.id:5(int) name:6(string) address:7(string) customers.crdb_internal_mvcc_timestamp:8(decimal) + ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) + ├── key: (1) + - ├── fd: (1)-->(2-8), (5)-->(6-8) + + ├── fd: (1)-->(2,3,5-8), (5)-->(6-8) + ├── scan orders + - │ ├── columns: orders.id:1(int!null) customer_id:2(int) status:3(string!null) orders.crdb_internal_mvcc_timestamp:4(decimal) + + │ ├── columns: orders.id:1(int!null) customer_id:2(int) status:3(string!null) + │ ├── check constraint expressions + │ │ └── in [type=bool, outer=(3), constraints=(/3: [/'cancelled' - /'cancelled'] [/'complete' - /'complete'] [/'open' - /'open']; tight)] + │ │ ├── variable: status:3 [type=string] + │ │ └── tuple [type=tuple{string, string, string}] + │ │ ├── const: 'cancelled' [type=string] + │ │ ├── const: 'complete' [type=string] + │ │ └── const: 'open' [type=string] + │ ├── key: (1) + - │ └── fd: (1)-->(2-4) + + │ └── fd: (1)-->(2,3) + ├── scan customers + │ ├── columns: customers.id:5(int!null) name:6(string!null) address:7(string) customers.crdb_internal_mvcc_timestamp:8(decimal) + │ ├── key: (5) + │ └── fd: (5)-->(6-8) + └── filters + └── eq [type=bool, outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ]), fd=(2)==(5), (5)==(2)] + ├── variable: customer_id:2 [type=int] + └── variable: customers.id:5 [type=int] +================================================================================ +PruneJoinRightCols + Cost: 2170.07 +================================================================================ + project + ├── columns: id:1(int!null) customer_id:2(int) status:3(string!null) id:5(int) name:6(string) address:7(string) + ├── key: (1) + ├── fd: (1)-->(2,3,5-7), (5)-->(6,7) + └── left-join (hash) + - ├── columns: orders.id:1(int!null) customer_id:2(int) status:3(string!null) customers.id:5(int) name:6(string) address:7(string) customers.crdb_internal_mvcc_timestamp:8(decimal) + + ├── columns: orders.id:1(int!null) customer_id:2(int) status:3(string!null) customers.id:5(int) name:6(string) address:7(string) + ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) + ├── key: (1) + - ├── fd: (1)-->(2,3,5-8), (5)-->(6-8) + + ├── fd: (1)-->(2,3,5-7), (5)-->(6,7) + ├── scan orders + │ ├── columns: orders.id:1(int!null) customer_id:2(int) status:3(string!null) + │ ├── check constraint expressions + │ │ └── in [type=bool, outer=(3), constraints=(/3: [/'cancelled' - /'cancelled'] [/'complete' - /'complete'] [/'open' - /'open']; tight)] + │ │ ├── variable: status:3 [type=string] + │ │ └── tuple [type=tuple{string, string, string}] + │ │ ├── const: 'cancelled' [type=string] + │ │ ├── const: 'complete' [type=string] + │ │ └── const: 'open' [type=string] + │ ├── key: (1) + │ └── fd: (1)-->(2,3) + ├── scan customers + - │ ├── columns: customers.id:5(int!null) name:6(string!null) address:7(string) customers.crdb_internal_mvcc_timestamp:8(decimal) + + │ ├── columns: customers.id:5(int!null) name:6(string!null) address:7(string) + │ ├── key: (5) + - │ └── fd: (5)-->(6-8) + + │ └── fd: (5)-->(6,7) + └── filters + └── eq [type=bool, outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ]), fd=(2)==(5), (5)==(2)] + ├── variable: customer_id:2 [type=int] + └── variable: customers.id:5 [type=int] +================================================================================ +EliminateProject Cost: 2160.06 ================================================================================ - left-join (hash) - ├── columns: id:1(int!null) customer_id:2(int) status:3(string!null) id:4(int) name:5(string) address:6(string) - ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) + -project + +left-join (hash) + ├── columns: id:1(int!null) customer_id:2(int) status:3(string!null) id:5(int) name:6(string) address:7(string) + + ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) ├── key: (1) - ├── fd: (1)-->(2-6), (4)-->(5,6) - ├── scan orders - │ ├── columns: orders.id:1(int!null) customer_id:2(int) status:3(string!null) - │ ├── check constraint expressions - │ │ └── in [type=bool, outer=(3), constraints=(/3: [/'cancelled' - /'cancelled'] [/'complete' - /'complete'] [/'open' - /'open']; tight)] - │ │ ├── variable: status:3 [type=string] - │ │ └── tuple [type=tuple{string, string, string}] - - │ │ ├── const: 'open' [type=string] + ├── fd: (1)-->(2,3,5-7), (5)-->(6,7) + - └── left-join (hash) + - ├── columns: orders.id:1(int!null) customer_id:2(int) status:3(string!null) customers.id:5(int) name:6(string) address:7(string) + - ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) + - ├── key: (1) + - ├── fd: (1)-->(2,3,5-7), (5)-->(6,7) + - ├── scan orders + - │ ├── columns: orders.id:1(int!null) customer_id:2(int) status:3(string!null) + - │ ├── check constraint expressions + - │ │ └── in [type=bool, outer=(3), constraints=(/3: [/'cancelled' - /'cancelled'] [/'complete' - /'complete'] [/'open' - /'open']; tight)] + - │ │ ├── variable: status:3 [type=string] + - │ │ └── tuple [type=tuple{string, string, string}] + - │ │ ├── const: 'cancelled' [type=string] + - │ │ ├── const: 'complete' [type=string] + - │ │ └── const: 'open' [type=string] + - │ ├── key: (1) + - │ └── fd: (1)-->(2,3) + - ├── scan customers + - │ ├── columns: customers.id:5(int!null) name:6(string!null) address:7(string) + - │ ├── key: (5) + - │ └── fd: (5)-->(6,7) + - └── filters + - └── eq [type=bool, outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ]), fd=(2)==(5), (5)==(2)] + - ├── variable: customer_id:2 [type=int] + - └── variable: customers.id:5 [type=int] + + ├── scan orders + + │ ├── columns: orders.id:1(int!null) customer_id:2(int) status:3(string!null) + + │ ├── check constraint expressions + + │ │ └── in [type=bool, outer=(3), constraints=(/3: [/'cancelled' - /'cancelled'] [/'complete' - /'complete'] [/'open' - /'open']; tight)] + + │ │ ├── variable: status:3 [type=string] + + │ │ └── tuple [type=tuple{string, string, string}] + │ │ ├── const: 'cancelled' [type=string] - │ │ ├── const: 'complete' [type=string] - - │ │ └── const: 'cancelled' [type=string] + + │ │ ├── const: 'complete' [type=string] + │ │ └── const: 'open' [type=string] - │ ├── key: (1) - │ └── fd: (1)-->(2,3) - ├── scan customers - │ ├── columns: customers.id:4(int!null) name:5(string!null) address:6(string) - │ ├── key: (4) - │ └── fd: (4)-->(5,6) - └── filters - └── eq [type=bool, outer=(2,4), constraints=(/2: (/NULL - ]; /4: (/NULL - ]), fd=(2)==(4), (4)==(2)] - ├── variable: customer_id:2 [type=int] - └── variable: customers.id:4 [type=int] + + │ ├── key: (1) + + │ └── fd: (1)-->(2,3) + + ├── scan customers + + │ ├── columns: customers.id:5(int!null) name:6(string!null) address:7(string) + + │ ├── key: (5) + + │ └── fd: (5)-->(6,7) + + └── filters + + └── eq [type=bool, outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ]), fd=(2)==(5), (5)==(2)] + + ├── variable: customer_id:2 [type=int] + + └── variable: customers.id:5 [type=int] -------------------------------------------------------------------------------- GenerateIndexScans (no changes) -------------------------------------------------------------------------------- @@ -218,14 +408,14 @@ CommuteLeftJoin (higher cost) -------------------------------------------------------------------------------- -left-join (hash) +right-join (hash) - ├── columns: id:1(int!null) customer_id:2(int) status:3(string!null) id:4(int) name:5(string) address:6(string) + ├── columns: id:1(int!null) customer_id:2(int) status:3(string!null) id:5(int) name:6(string) address:7(string) - ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) ├── key: (1) - ├── fd: (1)-->(2-6), (4)-->(5,6) + ├── fd: (1)-->(2,3,5-7), (5)-->(6,7) + ├── scan customers - + │ ├── columns: customers.id:4(int!null) name:5(string!null) address:6(string) - + │ ├── key: (4) - + │ └── fd: (4)-->(5,6) + + │ ├── columns: customers.id:5(int!null) name:6(string!null) address:7(string) + + │ ├── key: (5) + + │ └── fd: (5)-->(6,7) ├── scan orders │ ├── columns: orders.id:1(int!null) customer_id:2(int) status:3(string!null) │ ├── check constraint expressions @@ -238,13 +428,13 @@ CommuteLeftJoin (higher cost) │ ├── key: (1) │ └── fd: (1)-->(2,3) - ├── scan customers - - │ ├── columns: customers.id:4(int!null) name:5(string!null) address:6(string) - - │ ├── key: (4) - - │ └── fd: (4)-->(5,6) + - │ ├── columns: customers.id:5(int!null) name:6(string!null) address:7(string) + - │ ├── key: (5) + - │ └── fd: (5)-->(6,7) └── filters - └── eq [type=bool, outer=(2,4), constraints=(/2: (/NULL - ]; /4: (/NULL - ]), fd=(2)==(4), (4)==(2)] + └── eq [type=bool, outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ]), fd=(2)==(5), (5)==(2)] ├── variable: customer_id:2 [type=int] - └── variable: customers.id:4 [type=int] + └── variable: customers.id:5 [type=int] -------------------------------------------------------------------------------- GenerateMergeJoins (no changes) -------------------------------------------------------------------------------- @@ -253,12 +443,12 @@ GenerateLookupJoins (higher cost) -------------------------------------------------------------------------------- -left-join (hash) +left-join (lookup customers) - ├── columns: id:1(int!null) customer_id:2(int) status:3(string!null) id:4(int) name:5(string) address:6(string) + ├── columns: id:1(int!null) customer_id:2(int) status:3(string!null) id:5(int) name:6(string) address:7(string) - ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - + ├── key columns: [2] = [4] + + ├── key columns: [2] = [5] + ├── lookup columns are key ├── key: (1) - ├── fd: (1)-->(2-6), (4)-->(5,6) + ├── fd: (1)-->(2,3,5-7), (5)-->(6,7) ├── scan orders │ ├── columns: orders.id:1(int!null) customer_id:2(int) status:3(string!null) │ ├── check constraint expressions @@ -271,32 +461,32 @@ GenerateLookupJoins (higher cost) │ ├── key: (1) │ └── fd: (1)-->(2,3) - ├── scan customers - - │ ├── columns: customers.id:4(int!null) name:5(string!null) address:6(string) - - │ ├── key: (4) - - │ └── fd: (4)-->(5,6) + - │ ├── columns: customers.id:5(int!null) name:6(string!null) address:7(string) + - │ ├── key: (5) + - │ └── fd: (5)-->(6,7) - └── filters - - └── eq [type=bool, outer=(2,4), constraints=(/2: (/NULL - ]; /4: (/NULL - ]), fd=(2)==(4), (4)==(2)] + - └── eq [type=bool, outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ]), fd=(2)==(5), (5)==(2)] - ├── variable: customer_id:2 [type=int] - - └── variable: customers.id:4 [type=int] + - └── variable: customers.id:5 [type=int] + └── filters (true) -------------------------------------------------------------------------------- GenerateMergeJoins (higher cost) -------------------------------------------------------------------------------- -left-join (lookup customers) +right-join (merge) - ├── columns: id:1(int!null) customer_id:2(int) status:3(string!null) id:4(int) name:5(string) address:6(string) - - ├── key columns: [2] = [4] + ├── columns: id:1(int!null) customer_id:2(int) status:3(string!null) id:5(int) name:6(string) address:7(string) + - ├── key columns: [2] = [5] - ├── lookup columns are key - + ├── left ordering: +4 + + ├── left ordering: +5 + ├── right ordering: +2 ├── key: (1) - ├── fd: (1)-->(2-6), (4)-->(5,6) + ├── fd: (1)-->(2,3,5-7), (5)-->(6,7) - ├── scan orders + ├── scan customers - + │ ├── columns: customers.id:4(int!null) name:5(string!null) address:6(string) - + │ ├── key: (4) - + │ ├── fd: (4)-->(5,6) - + │ └── ordering: +4 + + │ ├── columns: customers.id:5(int!null) name:6(string!null) address:7(string) + + │ ├── key: (5) + + │ ├── fd: (5)-->(6,7) + + │ └── ordering: +5 + ├── sort │ ├── columns: orders.id:1(int!null) customer_id:2(int) status:3(string!null) - │ ├── check constraint expressions @@ -327,10 +517,10 @@ Final best expression Cost: 2160.06 ================================================================================ left-join (hash) - ├── columns: id:1(int!null) customer_id:2(int) status:3(string!null) id:4(int) name:5(string) address:6(string) + ├── columns: id:1(int!null) customer_id:2(int) status:3(string!null) id:5(int) name:6(string) address:7(string) ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) ├── key: (1) - ├── fd: (1)-->(2-6), (4)-->(5,6) + ├── fd: (1)-->(2,3,5-7), (5)-->(6,7) ├── scan orders │ ├── columns: orders.id:1(int!null) customer_id:2(int) status:3(string!null) │ ├── check constraint expressions @@ -343,13 +533,13 @@ Final best expression │ ├── key: (1) │ └── fd: (1)-->(2,3) ├── scan customers - │ ├── columns: customers.id:4(int!null) name:5(string!null) address:6(string) - │ ├── key: (4) - │ └── fd: (4)-->(5,6) + │ ├── columns: customers.id:5(int!null) name:6(string!null) address:7(string) + │ ├── key: (5) + │ └── fd: (5)-->(6,7) └── filters - └── eq [type=bool, outer=(2,4), constraints=(/2: (/NULL - ]; /4: (/NULL - ]), fd=(2)==(4), (4)==(2)] + └── eq [type=bool, outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ]), fd=(2)==(5), (5)==(2)] ├── variable: customer_id:2 [type=int] - └── variable: customers.id:4 [type=int] + └── variable: customers.id:5 [type=int] exec-ddl CREATE TABLE comp ( @@ -367,17 +557,17 @@ SELECT * FROM comp WHERE k=1 ---- ================================================================================ Initial expression - Cost: 1070.15 + Cost: 1080.15 ================================================================================ project ├── columns: k:1(int!null) c:2(bool) ├── fd: ()-->(1) └── select - ├── columns: k:1(int!null) c:2(bool) rowid:3(int!null) + ├── columns: k:1(int!null) c:2(bool) rowid:3(int!null) crdb_internal_mvcc_timestamp:4(decimal) ├── key: (3) - ├── fd: ()-->(1), (3)-->(2) + ├── fd: ()-->(1), (3)-->(2,4) ├── scan comp - │ ├── columns: k:1(int) c:2(bool) rowid:3(int!null) + │ ├── columns: k:1(int) c:2(bool) rowid:3(int!null) crdb_internal_mvcc_timestamp:4(decimal) │ ├── computed column expressions │ │ └── c:2 │ │ └── in [type=bool] @@ -387,24 +577,24 @@ Initial expression │ │ ├── const: 3 [type=int] │ │ └── const: 2 [type=int] │ ├── key: (3) - │ └── fd: (3)-->(1,2) + │ └── fd: (3)-->(1,2,4) └── filters └── eq [type=bool, outer=(1), constraints=(/1: [/1 - /1]; tight), fd=()-->(1)] ├── variable: k:1 [type=int] └── const: 1 [type=int] ================================================================================ NormalizeInConst - Cost: 1070.15 + Cost: 1080.15 ================================================================================ project ├── columns: k:1(int!null) c:2(bool) ├── fd: ()-->(1) └── select - ├── columns: k:1(int!null) c:2(bool) rowid:3(int!null) + ├── columns: k:1(int!null) c:2(bool) rowid:3(int!null) crdb_internal_mvcc_timestamp:4(decimal) ├── key: (3) - ├── fd: ()-->(1), (3)-->(2) + ├── fd: ()-->(1), (3)-->(2,4) ├── scan comp - │ ├── columns: k:1(int) c:2(bool) rowid:3(int!null) + │ ├── columns: k:1(int) c:2(bool) rowid:3(int!null) crdb_internal_mvcc_timestamp:4(decimal) │ ├── computed column expressions │ │ └── c:2 │ │ └── in [type=bool] @@ -416,7 +606,7 @@ NormalizeInConst + │ │ ├── const: 2 [type=int] + │ │ └── const: 3 [type=int] │ ├── key: (3) - │ └── fd: (3)-->(1,2) + │ └── fd: (3)-->(1,2,4) └── filters └── eq [type=bool, outer=(1), constraints=(/1: [/1 - /1]; tight), fd=()-->(1)] ├── variable: k:1 [type=int] @@ -429,13 +619,13 @@ PruneSelectCols ├── columns: k:1(int!null) c:2(bool) ├── fd: ()-->(1) └── select - - ├── columns: k:1(int!null) c:2(bool) rowid:3(int!null) + - ├── columns: k:1(int!null) c:2(bool) rowid:3(int!null) crdb_internal_mvcc_timestamp:4(decimal) - ├── key: (3) - - ├── fd: ()-->(1), (3)-->(2) + - ├── fd: ()-->(1), (3)-->(2,4) + ├── columns: k:1(int!null) c:2(bool) + ├── fd: ()-->(1) ├── scan comp - - │ ├── columns: k:1(int) c:2(bool) rowid:3(int!null) + - │ ├── columns: k:1(int) c:2(bool) rowid:3(int!null) crdb_internal_mvcc_timestamp:4(decimal) - │ ├── computed column expressions - │ │ └── c:2 - │ │ └── in [type=bool] @@ -445,7 +635,7 @@ PruneSelectCols - │ │ ├── const: 2 [type=int] - │ │ └── const: 3 [type=int] - │ ├── key: (3) - - │ └── fd: (3)-->(1,2) + - │ └── fd: (3)-->(1,2,4) + │ ├── columns: k:1(int) c:2(bool) + │ └── computed column expressions + │ └── c:2 diff --git a/pkg/sql/opt/testutils/testcat/create_table.go b/pkg/sql/opt/testutils/testcat/create_table.go index 7b9784f6e5a5..9f211997e4b3 100644 --- a/pkg/sql/opt/testutils/testcat/create_table.go +++ b/pkg/sql/opt/testutils/testcat/create_table.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/geo/geoindex" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util" ) @@ -113,6 +114,15 @@ func (tc *Catalog) CreateTable(stmt *tree.CreateTable) *Table { } } + // Add the MVCC timestamp system column. + tab.SystemColumns = append(tab.SystemColumns, &Column{ + Ordinal: len(tab.Columns), + Name: sqlbase.MVCCTimestampColumnName, + Type: sqlbase.MVCCTimestampColumnType, + Nullable: true, + Hidden: true, + }) + // Add the primary index. if hasPrimaryIndex { for _, def := range stmt.Defs { diff --git a/pkg/sql/opt/testutils/testcat/test_catalog.go b/pkg/sql/opt/testutils/testcat/test_catalog.go index b91710896416..6215d4db8a3f 100644 --- a/pkg/sql/opt/testutils/testcat/test_catalog.go +++ b/pkg/sql/opt/testutils/testcat/test_catalog.go @@ -527,16 +527,17 @@ func (tv *View) ColumnName(i int) tree.Name { // Table implements the cat.Table interface for testing purposes. type Table struct { - TabID cat.StableID - TabVersion int - TabName tree.TableName - Columns []*Column - Indexes []*Index - Stats TableStats - Checks []cat.CheckConstraint - Families []*Family - IsVirtual bool - Catalog cat.Catalog + TabID cat.StableID + TabVersion int + TabName tree.TableName + Columns []*Column + SystemColumns []*Column + Indexes []*Index + Stats TableStats + Checks []cat.CheckConstraint + Families []*Family + IsVirtual bool + Catalog cat.Catalog // If Revoked is true, then the user has had privileges on the table revoked. Revoked bool @@ -598,11 +599,14 @@ func (tt *Table) IsVirtualTable() bool { // ColumnCount is part of the cat.Table interface. func (tt *Table) ColumnCount() int { - return len(tt.Columns) + return len(tt.Columns) + len(tt.SystemColumns) } // Column is part of the cat.Table interface. func (tt *Table) Column(i int) cat.Column { + if i >= len(tt.Columns) { + return tt.SystemColumns[i-len(tt.Columns)] + } return tt.Columns[i] } @@ -615,8 +619,10 @@ func (tt *Table) ColumnKind(i int) cat.ColumnKind { return cat.Ordinary case i < writeOnlyEnd: return cat.WriteOnly - default: + case i < len(tt.Columns): return cat.DeleteOnly + default: + return cat.System } } diff --git a/pkg/sql/opt/testutils/testcat/testdata/foreign_keys b/pkg/sql/opt/testutils/testcat/testdata/foreign_keys index 58728fb560b6..1d7eadd7b525 100644 --- a/pkg/sql/opt/testutils/testcat/testdata/foreign_keys +++ b/pkg/sql/opt/testutils/testcat/testdata/foreign_keys @@ -11,6 +11,7 @@ SHOW CREATE parent ---- TABLE parent ├── p int not null + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── INDEX primary │ └── p int not null └── REFERENCED BY CONSTRAINT fk_p_ref_parent FOREIGN KEY child (p) REFERENCES parent (p) @@ -21,6 +22,7 @@ SHOW CREATE child TABLE child ├── c int not null ├── p int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── INDEX primary │ └── c int not null ├── INDEX child_auto_index_fk_p_ref_parent @@ -42,6 +44,7 @@ SHOW CREATE parent2 TABLE parent2 ├── p int ├── rowid int not null default (unique_rowid()) [hidden] + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── INDEX primary │ └── rowid int not null default (unique_rowid()) [hidden] ├── INDEX parent2_p_key @@ -55,6 +58,7 @@ SHOW CREATE child2 TABLE child2 ├── c int not null ├── p int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── INDEX primary │ └── c int not null ├── INDEX child2_auto_index_fk_p_ref_parent2 @@ -93,6 +97,7 @@ TABLE parent_multicol ├── p int not null ├── q int not null ├── r int not null + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── INDEX primary │ ├── p int not null │ ├── q int not null @@ -107,6 +112,7 @@ TABLE child_multicol ├── p int not null ├── q int not null ├── r int not null + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── INDEX primary │ ├── p int not null │ ├── q int not null @@ -120,6 +126,7 @@ TABLE child_multicol_full ├── p int not null ├── q int not null ├── r int not null + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── INDEX primary │ ├── p int not null │ ├── q int not null @@ -135,6 +142,7 @@ SHOW CREATE parent ---- TABLE parent ├── p int not null + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] └── INDEX primary └── p int not null @@ -149,6 +157,7 @@ TABLE parent_multicol ├── p int not null ├── q int not null ├── r int not null + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── INDEX primary │ ├── p int not null │ ├── q int not null @@ -166,6 +175,7 @@ TABLE parent_multicol ├── p int not null ├── q int not null ├── r int not null + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] └── INDEX primary ├── p int not null ├── q int not null diff --git a/pkg/sql/opt/testutils/testcat/testdata/table b/pkg/sql/opt/testutils/testcat/testdata/table index f21b91869e61..159d963079d4 100644 --- a/pkg/sql/opt/testutils/testcat/testdata/table +++ b/pkg/sql/opt/testutils/testcat/testdata/table @@ -11,6 +11,7 @@ SHOW CREATE kv TABLE kv ├── k int not null ├── v int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] └── INDEX primary └── k int not null @@ -36,6 +37,7 @@ TABLE abcdef ├── e int as (a) stored ├── f int ├── rowid int not null default (unique_rowid()) [hidden] + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── CHECK (f > 2) └── INDEX primary └── rowid int not null default (unique_rowid()) [hidden] @@ -64,6 +66,7 @@ TABLE uvwxy ├── w int ├── x int ├── y int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── FAMILY family1 (u, v, w) ├── FAMILY family2 (x) ├── FAMILY family3 (y) @@ -81,6 +84,7 @@ SHOW CREATE a TABLE a ├── a int ├── rowid int not null default (unique_rowid()) [hidden] + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── INDEX primary │ └── rowid int not null default (unique_rowid()) [hidden] └── INDEX a_a_key @@ -101,6 +105,7 @@ SHOW CREATE part1 TABLE part1 ├── a int not null ├── b int + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] └── INDEX primary ├── a int not null └── partition by list prefixes @@ -133,6 +138,7 @@ TABLE part2 ├── a string not null ├── b string not null ├── c int not null + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── INDEX primary │ ├── a string not null │ ├── b string not null diff --git a/pkg/sql/opt/testutils/testcat/testdata/zone b/pkg/sql/opt/testutils/testcat/testdata/zone index ec2bd9e9e521..357bb9671155 100644 --- a/pkg/sql/opt/testutils/testcat/testdata/zone +++ b/pkg/sql/opt/testutils/testcat/testdata/zone @@ -19,6 +19,7 @@ TABLE abc ├── a int not null ├── b int ├── c string + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── INDEX primary │ ├── a int not null │ └── ZONE @@ -43,6 +44,7 @@ TABLE abc ├── a int not null ├── b int ├── c string + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── INDEX primary │ ├── a int not null │ └── ZONE @@ -69,6 +71,7 @@ TABLE abc ├── a int not null ├── b int ├── c string + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── INDEX primary │ ├── a int not null │ └── ZONE @@ -97,6 +100,7 @@ TABLE abc ├── a int not null ├── b int ├── c string + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── INDEX primary │ ├── a int not null │ └── ZONE @@ -125,6 +129,7 @@ TABLE abc ├── a int not null ├── b int ├── c string + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── INDEX primary │ ├── a int not null │ └── ZONE @@ -153,6 +158,7 @@ TABLE abc ├── a int not null ├── b int ├── c string + ├── crdb_internal_mvcc_timestamp decimal [hidden] [system] ├── INDEX primary │ ├── a int not null │ └── ZONE diff --git a/pkg/sql/opt/xform/coster.go b/pkg/sql/opt/xform/coster.go index 8e4cbccbd488..ee0c20bc043c 100644 --- a/pkg/sql/opt/xform/coster.go +++ b/pkg/sql/opt/xform/coster.go @@ -1012,6 +1012,12 @@ func (c *coster) rowScanCost(tabID opt.TableID, idxOrd int, numScannedCols int) tab := md.Table(tabID) idx := tab.Index(idxOrd) numCols := idx.ColumnCount() + // Remove any system columns from numCols. + for i := 0; i < idx.ColumnCount(); i++ { + if cat.IsSystemColumn(tab, idx.Column(i).Ordinal) { + numCols-- + } + } // Adjust cost based on how well the current locality matches the index's // zone constraints. diff --git a/pkg/sql/opt/xform/testdata/coster/groupby b/pkg/sql/opt/xform/testdata/coster/groupby index 472c6a1c1546..456847b0048c 100644 --- a/pkg/sql/opt/xform/testdata/coster/groupby +++ b/pkg/sql/opt/xform/testdata/coster/groupby @@ -6,12 +6,12 @@ opt SELECT max(k), min(k), i, s FROM a GROUP BY i, s ---- group-by - ├── columns: max:5!null min:6!null i:2 s:3 + ├── columns: max:6!null min:7!null i:2 s:3 ├── grouping columns: i:2 s:3 ├── stats: [rows=1000, distinct(2,3)=1000, null(2,3)=0.1] ├── cost: 1130.04 ├── key: (2,3) - ├── fd: (2,3)-->(5,6) + ├── fd: (2,3)-->(6,7) ├── scan a │ ├── columns: k:1!null i:2 s:3 │ ├── stats: [rows=1000, distinct(2,3)=1000, null(2,3)=0.1] @@ -19,7 +19,7 @@ group-by │ ├── key: (1) │ └── fd: (1)-->(2,3) └── aggregations - ├── max [as=max:5, outer=(1)] + ├── max [as=max:6, outer=(1)] │ └── k:1 - └── min [as=min:6, outer=(1)] + └── min [as=min:7, outer=(1)] └── k:1 diff --git a/pkg/sql/opt/xform/testdata/coster/join b/pkg/sql/opt/xform/testdata/coster/join index 02e1a00862c3..9641590017c4 100644 --- a/pkg/sql/opt/xform/testdata/coster/join +++ b/pkg/sql/opt/xform/testdata/coster/join @@ -10,21 +10,21 @@ opt SELECT k, x FROM a INNER JOIN b ON k=x WHERE d=1.0 ---- project - ├── columns: k:1!null x:5!null + ├── columns: k:1!null x:6!null ├── immutable ├── stats: [rows=99] ├── cost: 2124.745 - ├── fd: (1)==(5), (5)==(1) + ├── fd: (1)==(6), (6)==(1) └── inner-join (hash) - ├── columns: k:1!null d:4!null x:5!null + ├── columns: k:1!null d:4!null x:6!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) ├── immutable - ├── stats: [rows=99, distinct(1)=10, null(1)=0, distinct(5)=10, null(5)=0] + ├── stats: [rows=99, distinct(1)=10, null(1)=0, distinct(6)=10, null(6)=0] ├── cost: 2123.745 - ├── fd: ()-->(4), (1)==(5), (5)==(1) + ├── fd: ()-->(4), (1)==(6), (6)==(1) ├── scan b - │ ├── columns: x:5 - │ ├── stats: [rows=1000, distinct(5)=100, null(5)=10] + │ ├── columns: x:6 + │ ├── stats: [rows=1000, distinct(6)=100, null(6)=10] │ └── cost: 1040.02 ├── select │ ├── columns: k:1!null d:4!null @@ -42,20 +42,20 @@ project │ └── filters │ └── d:4 = 1.0 [outer=(4), immutable, constraints=(/4: [/1.0 - /1.0]; tight), fd=()-->(4)] └── filters - └── k:1 = x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] # Verify that we pick merge join if we force it. opt SELECT k, x FROM a INNER MERGE JOIN b ON k=x ---- inner-join (merge) - ├── columns: k:1!null x:5!null + ├── columns: k:1!null x:6!null ├── flags: force merge join ├── left ordering: +1 - ├── right ordering: +5 - ├── stats: [rows=990, distinct(1)=99, null(1)=0, distinct(5)=99, null(5)=0] + ├── right ordering: +6 + ├── stats: [rows=990, distinct(1)=99, null(1)=0, distinct(6)=99, null(6)=0] ├── cost: 2339.27569 - ├── fd: (1)==(5), (5)==(1) + ├── fd: (1)==(6), (6)==(1) ├── scan a │ ├── columns: k:1!null │ ├── stats: [rows=1000, distinct(1)=1000, null(1)=0] @@ -63,13 +63,13 @@ inner-join (merge) │ ├── key: (1) │ └── ordering: +1 ├── sort - │ ├── columns: x:5 - │ ├── stats: [rows=1000, distinct(5)=100, null(5)=10] + │ ├── columns: x:6 + │ ├── stats: [rows=1000, distinct(6)=100, null(6)=10] │ ├── cost: 1259.34569 - │ ├── ordering: +5 + │ ├── ordering: +6 │ └── scan b - │ ├── columns: x:5 - │ ├── stats: [rows=1000, distinct(5)=100, null(5)=10] + │ ├── columns: x:6 + │ ├── stats: [rows=1000, distinct(6)=100, null(6)=10] │ └── cost: 1040.02 └── filters (true) @@ -79,13 +79,13 @@ opt SELECT k, x FROM b INNER LOOKUP JOIN a ON k=x ---- inner-join (lookup a) - ├── columns: k:4!null x:1!null + ├── columns: k:5!null x:1!null ├── flags: force lookup join (into right side) - ├── key columns: [1] = [4] + ├── key columns: [1] = [5] ├── lookup columns are key - ├── stats: [rows=990, distinct(1)=99, null(1)=0, distinct(4)=99, null(4)=0] + ├── stats: [rows=990, distinct(1)=99, null(1)=0, distinct(5)=99, null(5)=0] ├── cost: 7079.43 - ├── fd: (1)==(4), (4)==(1) + ├── fd: (1)==(5), (5)==(1) ├── scan b │ ├── columns: x:1 │ ├── stats: [rows=1000, distinct(1)=100, null(1)=10] @@ -99,23 +99,23 @@ opt SELECT k, x FROM a INNER LOOKUP JOIN b ON k=x ---- inner-join (hash) - ├── columns: k:1!null x:5!null + ├── columns: k:1!null x:6!null ├── flags: force lookup join (into right side) ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) - ├── stats: [rows=990, distinct(1)=99, null(1)=0, distinct(5)=99, null(5)=0] + ├── stats: [rows=990, distinct(1)=99, null(1)=0, distinct(6)=99, null(6)=0] ├── cost: 1e+100 - ├── fd: (1)==(5), (5)==(1) + ├── fd: (1)==(6), (6)==(1) ├── scan a │ ├── columns: k:1!null │ ├── stats: [rows=1000, distinct(1)=1000, null(1)=0] │ ├── cost: 1050.02 │ └── key: (1) ├── scan b - │ ├── columns: x:5 - │ ├── stats: [rows=1000, distinct(5)=100, null(5)=10] + │ ├── columns: x:6 + │ ├── stats: [rows=1000, distinct(6)=100, null(6)=10] │ └── cost: 1040.02 └── filters - └── k:1 = x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + └── k:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] exec-ddl ALTER TABLE a INJECT STATISTICS '[ @@ -144,22 +144,22 @@ opt SELECT * FROM a JOIN b ON k=z WHERE x > 0 AND x <= 5000 ---- inner-join (lookup a) - ├── columns: k:1!null i:2 s:3 d:4!null x:5!null z:6!null - ├── key columns: [6] = [1] + ├── columns: k:1!null i:2 s:3 d:4!null x:6!null z:7!null + ├── key columns: [7] = [1] ├── lookup columns are key - ├── stats: [rows=10000, distinct(1)=1000, null(1)=0, distinct(6)=1000, null(6)=0] + ├── stats: [rows=10000, distinct(1)=1000, null(1)=0, distinct(7)=1000, null(7)=0] ├── cost: 71500.05 - ├── fd: (1)-->(2-4), (1)==(6), (6)==(1) + ├── fd: (1)-->(2-4), (1)==(7), (7)==(1) ├── select - │ ├── columns: x:5!null z:6!null - │ ├── stats: [rows=10000, distinct(5)=1000, null(5)=0, distinct(6)=1000, null(6)=0] + │ ├── columns: x:6!null z:7!null + │ ├── stats: [rows=10000, distinct(6)=1000, null(6)=0, distinct(7)=1000, null(7)=0] │ ├── cost: 10600.04 │ ├── scan b - │ │ ├── columns: x:5 z:6!null - │ │ ├── stats: [rows=10000, distinct(5)=1000, null(5)=0, distinct(6)=1000, null(6)=0] + │ │ ├── columns: x:6 z:7!null + │ │ ├── stats: [rows=10000, distinct(6)=1000, null(6)=0, distinct(7)=1000, null(7)=0] │ │ └── cost: 10500.02 │ └── filters - │ └── (x:5 > 0) AND (x:5 <= 5000) [outer=(5), constraints=(/5: [/1 - /5000]; tight)] + │ └── (x:6 > 0) AND (x:6 <= 5000) [outer=(6), constraints=(/6: [/1 - /5000]; tight)] └── filters (true) # With the limit hint, the cost of the lookup join is reduced. @@ -167,31 +167,31 @@ opt SELECT * FROM a JOIN b ON k=z WHERE x > 0 AND x <= 5000 LIMIT 6000 ---- limit - ├── columns: k:1!null i:2 s:3 d:4!null x:5!null z:6!null + ├── columns: k:1!null i:2 s:3 d:4!null x:6!null z:7!null ├── cardinality: [0 - 6000] ├── stats: [rows=6000] ├── cost: 47200.06 - ├── fd: (1)-->(2-4), (1)==(6), (6)==(1) + ├── fd: (1)-->(2-4), (1)==(7), (7)==(1) ├── inner-join (lookup a) - │ ├── columns: k:1!null i:2 s:3 d:4!null x:5!null z:6!null - │ ├── key columns: [6] = [1] + │ ├── columns: k:1!null i:2 s:3 d:4!null x:6!null z:7!null + │ ├── key columns: [7] = [1] │ ├── lookup columns are key - │ ├── stats: [rows=10000, distinct(1)=1000, null(1)=0, distinct(6)=1000, null(6)=0] + │ ├── stats: [rows=10000, distinct(1)=1000, null(1)=0, distinct(7)=1000, null(7)=0] │ ├── cost: 47140.05 - │ ├── fd: (1)-->(2-4), (1)==(6), (6)==(1) + │ ├── fd: (1)-->(2-4), (1)==(7), (7)==(1) │ ├── limit hint: 6000.00 │ ├── select - │ │ ├── columns: x:5!null z:6!null - │ │ ├── stats: [rows=10000, distinct(5)=1000, null(5)=0, distinct(6)=1000, null(6)=0] + │ │ ├── columns: x:6!null z:7!null + │ │ ├── stats: [rows=10000, distinct(6)=1000, null(6)=0, distinct(7)=1000, null(7)=0] │ │ ├── cost: 10600.04 │ │ ├── limit hint: 6000.00 │ │ ├── scan b - │ │ │ ├── columns: x:5 z:6!null - │ │ │ ├── stats: [rows=10000, distinct(5)=1000, null(5)=0, distinct(6)=1000, null(6)=0] + │ │ │ ├── columns: x:6 z:7!null + │ │ │ ├── stats: [rows=10000, distinct(6)=1000, null(6)=0, distinct(7)=1000, null(7)=0] │ │ │ ├── cost: 10500.02 │ │ │ └── limit hint: 6000.00 │ │ └── filters - │ │ └── (x:5 > 0) AND (x:5 <= 5000) [outer=(5), constraints=(/5: [/1 - /5000]; tight)] + │ │ └── (x:6 > 0) AND (x:6 <= 5000) [outer=(6), constraints=(/6: [/1 - /5000]; tight)] │ └── filters (true) └── 6000 @@ -202,31 +202,31 @@ opt SELECT * FROM a JOIN b ON k=z WHERE x > 0 AND x <= 5000 LIMIT 5950 ---- limit - ├── columns: k:1!null i:2 s:3 d:4!null x:5!null z:6!null + ├── columns: k:1!null i:2 s:3 d:4!null x:6!null z:7!null ├── cardinality: [0 - 5950] ├── stats: [rows=5950] ├── cost: 47199.56 - ├── fd: (1)-->(2-4), (1)==(6), (6)==(1) + ├── fd: (1)-->(2-4), (1)==(7), (7)==(1) ├── inner-join (lookup a) - │ ├── columns: k:1!null i:2 s:3 d:4!null x:5!null z:6!null - │ ├── key columns: [6] = [1] + │ ├── columns: k:1!null i:2 s:3 d:4!null x:6!null z:7!null + │ ├── key columns: [7] = [1] │ ├── lookup columns are key - │ ├── stats: [rows=10000, distinct(1)=1000, null(1)=0, distinct(6)=1000, null(6)=0] + │ ├── stats: [rows=10000, distinct(1)=1000, null(1)=0, distinct(7)=1000, null(7)=0] │ ├── cost: 47140.05 - │ ├── fd: (1)-->(2-4), (1)==(6), (6)==(1) + │ ├── fd: (1)-->(2-4), (1)==(7), (7)==(1) │ ├── limit hint: 5950.00 │ ├── select - │ │ ├── columns: x:5!null z:6!null - │ │ ├── stats: [rows=10000, distinct(5)=1000, null(5)=0, distinct(6)=1000, null(6)=0] + │ │ ├── columns: x:6!null z:7!null + │ │ ├── stats: [rows=10000, distinct(6)=1000, null(6)=0, distinct(7)=1000, null(7)=0] │ │ ├── cost: 10600.04 │ │ ├── limit hint: 6000.00 │ │ ├── scan b - │ │ │ ├── columns: x:5 z:6!null - │ │ │ ├── stats: [rows=10000, distinct(5)=1000, null(5)=0, distinct(6)=1000, null(6)=0] + │ │ │ ├── columns: x:6 z:7!null + │ │ │ ├── stats: [rows=10000, distinct(6)=1000, null(6)=0, distinct(7)=1000, null(7)=0] │ │ │ ├── cost: 10500.02 │ │ │ └── limit hint: 6000.00 │ │ └── filters - │ │ └── (x:5 > 0) AND (x:5 <= 5000) [outer=(5), constraints=(/5: [/1 - /5000]; tight)] + │ │ └── (x:6 > 0) AND (x:6 <= 5000) [outer=(6), constraints=(/6: [/1 - /5000]; tight)] │ └── filters (true) └── 5950 @@ -279,29 +279,29 @@ JOIN wallet AS r on t.receiver_id = r.id limit 10; ---- limit - ├── columns: id:1!null sender_id:2!null receiver_id:3!null amount:4!null creation_date:5!null last_update:6 schedule_date:7 status:8 comment:9 linked_trans_id:10 c1:11 c2:12 c3:13 id:14!null name:15!null gender:16 email:17 first_name:18 last_name:19 creation_date:20!null situation:21 balance:22!null is_blocked:23 id:24!null name:25!null gender:26 email:27 first_name:28 last_name:29 creation_date:30!null situation:31 balance:32!null is_blocked:33 + ├── columns: id:1!null sender_id:2!null receiver_id:3!null amount:4!null creation_date:5!null last_update:6 schedule_date:7 status:8 comment:9 linked_trans_id:10 c1:11 c2:12 c3:13 id:15!null name:16!null gender:17 email:18 first_name:19 last_name:20 creation_date:21!null situation:22 balance:23!null is_blocked:24 id:26!null name:27!null gender:28 email:29 first_name:30 last_name:31 creation_date:32!null situation:33 balance:34!null is_blocked:35 ├── cardinality: [0 - 10] ├── stats: [rows=10] ├── cost: 2360.52 ├── key: (1) - ├── fd: (1)-->(2-13), (14)-->(15-23), (2)==(14), (14)==(2), (24)-->(25-33), (3)==(24), (24)==(3) + ├── fd: (1)-->(2-13), (15)-->(16-24), (2)==(15), (15)==(2), (26)-->(27-35), (3)==(26), (26)==(3) ├── inner-join (lookup wallet) - │ ├── columns: t.id:1!null sender_id:2!null receiver_id:3!null amount:4!null t.creation_date:5!null last_update:6 schedule_date:7 status:8 comment:9 linked_trans_id:10 c1:11 c2:12 c3:13 s.id:14!null s.name:15!null s.gender:16 s.email:17 s.first_name:18 s.last_name:19 s.creation_date:20!null s.situation:21 s.balance:22!null s.is_blocked:23 r.id:24!null r.name:25!null r.gender:26 r.email:27 r.first_name:28 r.last_name:29 r.creation_date:30!null r.situation:31 r.balance:32!null r.is_blocked:33 - │ ├── key columns: [3] = [24] + │ ├── columns: t.id:1!null sender_id:2!null receiver_id:3!null amount:4!null t.creation_date:5!null last_update:6 schedule_date:7 status:8 comment:9 linked_trans_id:10 c1:11 c2:12 c3:13 s.id:15!null s.name:16!null s.gender:17 s.email:18 s.first_name:19 s.last_name:20 s.creation_date:21!null s.situation:22 s.balance:23!null s.is_blocked:24 r.id:26!null r.name:27!null r.gender:28 r.email:29 r.first_name:30 r.last_name:31 r.creation_date:32!null r.situation:33 r.balance:34!null r.is_blocked:35 + │ ├── key columns: [3] = [26] │ ├── lookup columns are key - │ ├── stats: [rows=980.1, distinct(3)=98.9950071, null(3)=0, distinct(24)=98.9950071, null(24)=0] + │ ├── stats: [rows=980.1, distinct(3)=98.9950071, null(3)=0, distinct(26)=98.9950071, null(26)=0] │ ├── cost: 2360.41 │ ├── key: (1) - │ ├── fd: (1)-->(2-13), (14)-->(15-23), (2)==(14), (14)==(2), (24)-->(25-33), (3)==(24), (24)==(3) + │ ├── fd: (1)-->(2-13), (15)-->(16-24), (2)==(15), (15)==(2), (26)-->(27-35), (3)==(26), (26)==(3) │ ├── limit hint: 10.00 │ ├── inner-join (lookup wallet) - │ │ ├── columns: t.id:1!null sender_id:2!null receiver_id:3 amount:4!null t.creation_date:5!null last_update:6 schedule_date:7 status:8 comment:9 linked_trans_id:10 c1:11 c2:12 c3:13 s.id:14!null s.name:15!null s.gender:16 s.email:17 s.first_name:18 s.last_name:19 s.creation_date:20!null s.situation:21 s.balance:22!null s.is_blocked:23 - │ │ ├── key columns: [2] = [14] + │ │ ├── columns: t.id:1!null sender_id:2!null receiver_id:3 amount:4!null t.creation_date:5!null last_update:6 schedule_date:7 status:8 comment:9 linked_trans_id:10 c1:11 c2:12 c3:13 s.id:15!null s.name:16!null s.gender:17 s.email:18 s.first_name:19 s.last_name:20 s.creation_date:21!null s.situation:22 s.balance:23!null s.is_blocked:24 + │ │ ├── key columns: [2] = [15] │ │ ├── lookup columns are key - │ │ ├── stats: [rows=990, distinct(1)=628.605476, null(1)=0, distinct(2)=99, null(2)=0, distinct(3)=99.9950071, null(3)=9.9, distinct(4)=99.9950071, null(4)=0, distinct(5)=99.9950071, null(5)=0, distinct(14)=99, null(14)=0, distinct(15)=99.9950071, null(15)=0, distinct(20)=99.9950071, null(20)=0, distinct(22)=99.9950071, null(22)=0] + │ │ ├── stats: [rows=990, distinct(1)=628.605476, null(1)=0, distinct(2)=99, null(2)=0, distinct(3)=99.9950071, null(3)=9.9, distinct(4)=99.9950071, null(4)=0, distinct(5)=99.9950071, null(5)=0, distinct(15)=99, null(15)=0, distinct(16)=99.9950071, null(16)=0, distinct(21)=99.9950071, null(21)=0, distinct(23)=99.9950071, null(23)=0] │ │ ├── cost: 1741.61 │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2-13), (14)-->(15-23), (2)==(14), (14)==(2) + │ │ ├── fd: (1)-->(2-13), (15)-->(16-24), (2)==(15), (15)==(2) │ │ ├── limit hint: 100.00 │ │ ├── scan t │ │ │ ├── columns: t.id:1!null sender_id:2 receiver_id:3 amount:4!null t.creation_date:5!null last_update:6 schedule_date:7 status:8 comment:9 linked_trans_id:10 c1:11 c2:12 c3:13 @@ -430,11 +430,11 @@ project ├── cost: 3160.09489 ├── fd: ()-->(1,2) └── inner-join (lookup abcde@idx_abcd) - ├── columns: w:1!null x:2!null y:3!null z:4!null a:6!null b:7!null c:8!null - ├── key columns: [1 2 3] = [6 7 8] - ├── stats: [rows=500.488759, distinct(1)=1, null(1)=0, distinct(2)=1, null(2)=0, distinct(3)=25, null(3)=0, distinct(6)=1, null(6)=0, distinct(7)=1, null(7)=0, distinct(8)=25, null(8)=0] + ├── columns: w:1!null x:2!null y:3!null z:4!null a:7!null b:8!null c:9!null + ├── key columns: [1 2 3] = [7 8 9] + ├── stats: [rows=500.488759, distinct(1)=1, null(1)=0, distinct(2)=1, null(2)=0, distinct(3)=25, null(3)=0, distinct(7)=1, null(7)=0, distinct(8)=1, null(8)=0, distinct(9)=25, null(9)=0] ├── cost: 3155.08 - ├── fd: ()-->(1,2,6,7), (1)==(6), (6)==(1), (2)==(7), (7)==(2), (3)==(8), (8)==(3) + ├── fd: ()-->(1,2,7,8), (1)==(7), (7)==(1), (2)==(8), (8)==(2), (3)==(9), (9)==(3) ├── select │ ├── columns: w:1!null x:2!null y:3!null z:4!null │ ├── stats: [rows=100, distinct(1)=1, null(1)=0, distinct(2)=1, null(2)=0, distinct(3)=25, null(3)=0, distinct(4)=10, null(4)=0] @@ -448,8 +448,8 @@ project │ ├── w:1 = 'foo' [outer=(1), constraints=(/1: [/'foo' - /'foo']; tight), fd=()-->(1)] │ └── x:2 = '2ab23800-06b1-4e19-a3bb-df3768b808d2' [outer=(2), constraints=(/2: [/'2ab23800-06b1-4e19-a3bb-df3768b808d2' - /'2ab23800-06b1-4e19-a3bb-df3768b808d2']; tight), fd=()-->(2)] └── filters - ├── a:6 = 'foo' [outer=(6), constraints=(/6: [/'foo' - /'foo']; tight), fd=()-->(6)] - └── b:7 = '2ab23800-06b1-4e19-a3bb-df3768b808d2' [outer=(7), constraints=(/7: [/'2ab23800-06b1-4e19-a3bb-df3768b808d2' - /'2ab23800-06b1-4e19-a3bb-df3768b808d2']; tight), fd=()-->(7)] + ├── a:7 = 'foo' [outer=(7), constraints=(/7: [/'foo' - /'foo']; tight), fd=()-->(7)] + └── b:8 = '2ab23800-06b1-4e19-a3bb-df3768b808d2' [outer=(8), constraints=(/8: [/'2ab23800-06b1-4e19-a3bb-df3768b808d2' - /'2ab23800-06b1-4e19-a3bb-df3768b808d2']; tight), fd=()-->(8)] # Also for 34810: make sure the cost adjustment works when the estimated row # count is tiny. @@ -540,11 +540,11 @@ project ├── cost: 12227.6 ├── fd: ()-->(1,2) └── inner-join (lookup abcde@idx_abcd) - ├── columns: w:1!null x:2!null y:3!null z:4!null i:5!null j:6!null k:7!null l:8!null m:9!null n:10!null a:12!null b:13!null c:14!null - ├── key columns: [1 2 3] = [12 13 14] - ├── stats: [rows=4.50439883, distinct(1)=0.9, null(1)=0, distinct(2)=0.9, null(2)=0, distinct(3)=0.884031733, null(3)=0, distinct(12)=0.9, null(12)=0, distinct(13)=0.9, null(13)=0, distinct(14)=0.884031733, null(14)=0] + ├── columns: w:1!null x:2!null y:3!null z:4!null i:5!null j:6!null k:7!null l:8!null m:9!null n:10!null a:13!null b:14!null c:15!null + ├── key columns: [1 2 3] = [13 14 15] + ├── stats: [rows=4.50439883, distinct(1)=0.9, null(1)=0, distinct(2)=0.9, null(2)=0, distinct(3)=0.884031733, null(3)=0, distinct(13)=0.9, null(13)=0, distinct(14)=0.9, null(14)=0, distinct(15)=0.884031733, null(15)=0] ├── cost: 12227.545 - ├── fd: ()-->(1,2,5-10,12,13), (1)==(12), (12)==(1), (2)==(13), (13)==(2), (3)==(14), (14)==(3) + ├── fd: ()-->(1,2,5-10,13,14), (1)==(13), (13)==(1), (2)==(14), (14)==(2), (3)==(15), (15)==(3) ├── select │ ├── columns: w:1!null x:2!null y:3!null z:4!null i:5!null j:6!null k:7!null l:8!null m:9!null n:10!null │ ├── stats: [rows=0.9, distinct(1)=0.9, null(1)=0, distinct(2)=0.9, null(2)=0, distinct(3)=0.884031733, null(3)=0, distinct(4)=0.899635587, null(4)=0, distinct(5)=0.9, null(5)=0, distinct(6)=0.9, null(6)=0, distinct(7)=0.9, null(7)=0, distinct(8)=0.9, null(8)=0, distinct(9)=0.9, null(9)=0, distinct(10)=0.9, null(10)=0, distinct(5-10)=0.9, null(5-10)=0] @@ -564,8 +564,8 @@ project │ ├── m:9 = 5 [outer=(9), constraints=(/9: [/5 - /5]; tight), fd=()-->(9)] │ └── n:10 = 6 [outer=(10), constraints=(/10: [/6 - /6]; tight), fd=()-->(10)] └── filters - ├── a:12 = 'foo' [outer=(12), constraints=(/12: [/'foo' - /'foo']; tight), fd=()-->(12)] - └── b:13 = '2ab23800-06b1-4e19-a3bb-df3768b808d2' [outer=(13), constraints=(/13: [/'2ab23800-06b1-4e19-a3bb-df3768b808d2' - /'2ab23800-06b1-4e19-a3bb-df3768b808d2']; tight), fd=()-->(13)] + ├── a:13 = 'foo' [outer=(13), constraints=(/13: [/'foo' - /'foo']; tight), fd=()-->(13)] + └── b:14 = '2ab23800-06b1-4e19-a3bb-df3768b808d2' [outer=(14), constraints=(/14: [/'2ab23800-06b1-4e19-a3bb-df3768b808d2' - /'2ab23800-06b1-4e19-a3bb-df3768b808d2']; tight), fd=()-->(14)] exec-ddl DROP TABLE abcde @@ -657,40 +657,40 @@ ORDER BY d LIMIT 10 ---- project - ├── columns: w:1!null x:2!null y:3!null z:4!null [hidden: d:8!null] + ├── columns: w:1!null x:2!null y:3!null z:4!null [hidden: d:9!null] ├── cardinality: [0 - 10] ├── stats: [rows=10] ├── cost: 699.94172 - ├── key: (8) - ├── fd: ()-->(1,2), (3)-->(4,8), (8)-->(3,4) - ├── ordering: +8 opt(1,2) [actual: +8] + ├── key: (9) + ├── fd: ()-->(1,2), (3)-->(4,9), (9)-->(3,4) + ├── ordering: +9 opt(1,2) [actual: +9] └── limit - ├── columns: w:1!null x:2!null y:3!null z:4!null a:5!null b:6!null c:7!null d:8!null - ├── internal-ordering: +8 opt(1,2,5,6) + ├── columns: w:1!null x:2!null y:3!null z:4!null a:6!null b:7!null c:8!null d:9!null + ├── internal-ordering: +9 opt(1,2,6,7) ├── cardinality: [0 - 10] ├── stats: [rows=10] ├── cost: 699.83172 - ├── key: (7) - ├── fd: ()-->(1,2,5,6), (3)-->(4), (7)-->(8), (8)-->(7), (1)==(5), (5)==(1), (2)==(6), (6)==(2), (3)==(7), (7)==(3) - ├── ordering: +8 opt(1,2,5,6) [actual: +8] + ├── key: (8) + ├── fd: ()-->(1,2,6,7), (3)-->(4), (8)-->(9), (9)-->(8), (1)==(6), (6)==(1), (2)==(7), (7)==(2), (3)==(8), (8)==(3) + ├── ordering: +9 opt(1,2,6,7) [actual: +9] ├── inner-join (lookup wxyz) - │ ├── columns: w:1!null x:2!null y:3!null z:4!null a:5!null b:6!null c:7!null d:8!null - │ ├── key columns: [5 6 7] = [1 2 3] + │ ├── columns: w:1!null x:2!null y:3!null z:4!null a:6!null b:7!null c:8!null d:9!null + │ ├── key columns: [6 7 8] = [1 2 3] │ ├── lookup columns are key - │ ├── stats: [rows=50048.8759, distinct(1)=1, null(1)=0, distinct(2)=1, null(2)=0, distinct(3)=2500, null(3)=0, distinct(5)=1, null(5)=0, distinct(6)=1, null(6)=0, distinct(7)=2500, null(7)=0] + │ ├── stats: [rows=50048.8759, distinct(1)=1, null(1)=0, distinct(2)=1, null(2)=0, distinct(3)=2500, null(3)=0, distinct(6)=1, null(6)=0, distinct(7)=1, null(7)=0, distinct(8)=2500, null(8)=0] │ ├── cost: 699.72172 - │ ├── key: (7) - │ ├── fd: ()-->(1,2,5,6), (3)-->(4), (7)-->(8), (8)-->(7), (1)==(5), (5)==(1), (2)==(6), (6)==(2), (3)==(7), (7)==(3) - │ ├── ordering: +8 opt(1,2,5,6) [actual: +8] + │ ├── key: (8) + │ ├── fd: ()-->(1,2,6,7), (3)-->(4), (8)-->(9), (9)-->(8), (1)==(6), (6)==(1), (2)==(7), (7)==(2), (3)==(8), (8)==(3) + │ ├── ordering: +9 opt(1,2,6,7) [actual: +9] │ ├── limit hint: 10.00 │ ├── scan abcde@idx_abd - │ │ ├── columns: a:5!null b:6!null c:7!null d:8!null - │ │ ├── constraint: /5/6/8: [/'foo'/'2ab23800-06b1-4e19-a3bb-df3768b808d2' - /'foo'/'2ab23800-06b1-4e19-a3bb-df3768b808d2'] - │ │ ├── stats: [rows=125000, distinct(5)=1, null(5)=0, distinct(6)=1, null(6)=0, distinct(7)=24975.5859, null(7)=0, distinct(8)=93750, null(8)=0] + │ │ ├── columns: a:6!null b:7!null c:8!null d:9!null + │ │ ├── constraint: /6/7/9: [/'foo'/'2ab23800-06b1-4e19-a3bb-df3768b808d2' - /'foo'/'2ab23800-06b1-4e19-a3bb-df3768b808d2'] + │ │ ├── stats: [rows=125000, distinct(6)=1, null(6)=0, distinct(7)=1, null(7)=0, distinct(8)=24975.5859, null(8)=0, distinct(9)=93750, null(9)=0] │ │ ├── cost: 216.01 - │ │ ├── key: (7) - │ │ ├── fd: ()-->(5,6), (7)-->(8), (8)-->(7) - │ │ ├── ordering: +8 opt(5,6) [actual: +8] + │ │ ├── key: (8) + │ │ ├── fd: ()-->(6,7), (8)-->(9), (9)-->(8) + │ │ ├── ordering: +9 opt(6,7) [actual: +9] │ │ └── limit hint: 100.00 │ └── filters │ ├── w:1 = 'foo' [outer=(1), constraints=(/1: [/'foo' - /'foo']; tight), fd=()-->(1)] diff --git a/pkg/sql/opt/xform/testdata/coster/perturb-cost b/pkg/sql/opt/xform/testdata/coster/perturb-cost index c14fc3a3dd29..e063c97c3a40 100644 --- a/pkg/sql/opt/xform/testdata/coster/perturb-cost +++ b/pkg/sql/opt/xform/testdata/coster/perturb-cost @@ -20,19 +20,19 @@ norm perturb-cost=(0.5) SELECT * FROM a JOIN b ON a.x=b.x ORDER BY a.y ---- sort - ├── columns: x:1!null y:2 x:3!null - ├── stats: [rows=1000, distinct(1)=1000, null(1)=0, distinct(3)=1000, null(3)=0] + ├── columns: x:1!null y:2 x:4!null + ├── stats: [rows=1000, distinct(1)=1000, null(1)=0, distinct(4)=1000, null(4)=0] ├── cost: 2319.38569 - ├── key: (3) - ├── fd: (1)-->(2), (1)==(3), (3)==(1) + ├── key: (4) + ├── fd: (1)-->(2), (1)==(4), (4)==(1) ├── ordering: +2 └── inner-join (hash) - ├── columns: a.x:1!null y:2 b.x:3!null + ├── columns: a.x:1!null y:2 b.x:4!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-one) - ├── stats: [rows=1000, distinct(1)=1000, null(1)=0, distinct(3)=1000, null(3)=0] + ├── stats: [rows=1000, distinct(1)=1000, null(1)=0, distinct(4)=1000, null(4)=0] ├── cost: 2100.06 - ├── key: (3) - ├── fd: (1)-->(2), (1)==(3), (3)==(1) + ├── key: (4) + ├── fd: (1)-->(2), (1)==(4), (4)==(1) ├── scan a │ ├── columns: a.x:1!null y:2 │ ├── stats: [rows=1000, distinct(1)=1000, null(1)=0] @@ -40,12 +40,12 @@ sort │ ├── key: (1) │ └── fd: (1)-->(2) ├── scan b - │ ├── columns: b.x:3!null - │ ├── stats: [rows=1000, distinct(3)=1000, null(3)=0] + │ ├── columns: b.x:4!null + │ ├── stats: [rows=1000, distinct(4)=1000, null(4)=0] │ ├── cost: 1020.02 - │ └── key: (3) + │ └── key: (4) └── filters - └── a.x:1 = b.x:3 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] + └── a.x:1 = b.x:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] opt perturb-cost=(0.9) SELECT 1 diff --git a/pkg/sql/opt/xform/testdata/coster/project b/pkg/sql/opt/xform/testdata/coster/project index d3866dbe0eb2..e49b860cd4fb 100644 --- a/pkg/sql/opt/xform/testdata/coster/project +++ b/pkg/sql/opt/xform/testdata/coster/project @@ -6,12 +6,12 @@ opt SELECT k, i, s || 'foo' FROM a ---- project - ├── columns: k:1!null i:2 "?column?":5 + ├── columns: k:1!null i:2 "?column?":6 ├── immutable ├── stats: [rows=1000] ├── cost: 1090.03 ├── key: (1) - ├── fd: (1)-->(2,5) + ├── fd: (1)-->(2,6) ├── scan a │ ├── columns: k:1!null i:2 s:3 │ ├── stats: [rows=1000] @@ -19,18 +19,18 @@ project │ ├── key: (1) │ └── fd: (1)-->(2,3) └── projections - └── s:3 || 'foo' [as="?column?":5, outer=(3), immutable] + └── s:3 || 'foo' [as="?column?":6, outer=(3), immutable] opt SELECT k, k+2, i*d FROM a ---- project - ├── columns: k:1!null "?column?":5!null "?column?":6 + ├── columns: k:1!null "?column?":6!null "?column?":7 ├── immutable ├── stats: [rows=1000] ├── cost: 1100.03 ├── key: (1) - ├── fd: (1)-->(5,6) + ├── fd: (1)-->(6,7) ├── scan a │ ├── columns: k:1!null i:2 d:4!null │ ├── stats: [rows=1000] @@ -38,5 +38,5 @@ project │ ├── key: (1) │ └── fd: (1)-->(2,4) └── projections - ├── k:1 + 2 [as="?column?":5, outer=(1), immutable] - └── i:2 * d:4 [as="?column?":6, outer=(2,4), immutable] + ├── k:1 + 2 [as="?column?":6, outer=(1), immutable] + └── i:2 * d:4 [as="?column?":7, outer=(2,4), immutable] diff --git a/pkg/sql/opt/xform/testdata/coster/set b/pkg/sql/opt/xform/testdata/coster/set index 360d91746cb0..c4cd2e635064 100644 --- a/pkg/sql/opt/xform/testdata/coster/set +++ b/pkg/sql/opt/xform/testdata/coster/set @@ -10,12 +10,12 @@ opt SELECT k, i FROM a UNION SELECT * FROM b ---- union - ├── columns: k:8 i:9 + ├── columns: k:10 i:11 ├── left columns: a.k:1 a.i:2 - ├── right columns: x:5 z:6 - ├── stats: [rows=2000, distinct(8,9)=2000, null(8,9)=0] + ├── right columns: x:6 z:7 + ├── stats: [rows=2000, distinct(10,11)=2000, null(10,11)=0] ├── cost: 2150.05 - ├── key: (8,9) + ├── key: (10,11) ├── scan a │ ├── columns: a.k:1!null a.i:2 │ ├── stats: [rows=1000, distinct(1,2)=1000, null(1,2)=0] @@ -23,17 +23,17 @@ union │ ├── key: (1) │ └── fd: (1)-->(2) └── scan b - ├── columns: x:5 z:6!null - ├── stats: [rows=1000, distinct(5,6)=1000, null(5,6)=0] + ├── columns: x:6 z:7!null + ├── stats: [rows=1000, distinct(6,7)=1000, null(6,7)=0] └── cost: 1050.02 opt SELECT k, i FROM a UNION ALL SELECT * FROM b ---- union-all - ├── columns: k:8 i:9 + ├── columns: k:10 i:11 ├── left columns: a.k:1 a.i:2 - ├── right columns: x:5 z:6 + ├── right columns: x:6 z:7 ├── stats: [rows=2000] ├── cost: 2130.05 ├── scan a @@ -43,7 +43,7 @@ union-all │ ├── key: (1) │ └── fd: (1)-->(2) └── scan b - ├── columns: x:5 z:6!null + ├── columns: x:6 z:7!null ├── stats: [rows=1000] └── cost: 1050.02 @@ -53,7 +53,7 @@ SELECT k, i FROM a INTERSECT SELECT * FROM b intersect ├── columns: k:1 i:2 ├── left columns: k:1 i:2 - ├── right columns: x:5 z:6 + ├── right columns: x:6 z:7 ├── stats: [rows=1000, distinct(1,2)=1000, null(1,2)=0] ├── cost: 2140.05 ├── key: (1,2) @@ -64,8 +64,8 @@ intersect │ ├── key: (1) │ └── fd: (1)-->(2) └── scan b - ├── columns: x:5 z:6!null - ├── stats: [rows=1000, distinct(5,6)=1000, null(5,6)=0] + ├── columns: x:6 z:7!null + ├── stats: [rows=1000, distinct(6,7)=1000, null(6,7)=0] └── cost: 1050.02 opt @@ -74,7 +74,7 @@ SELECT k, i FROM a INTERSECT ALL SELECT * FROM b intersect-all ├── columns: k:1 i:2 ├── left columns: k:1 i:2 - ├── right columns: x:5 z:6 + ├── right columns: x:6 z:7 ├── stats: [rows=1000] ├── cost: 2140.05 ├── scan a @@ -84,7 +84,7 @@ intersect-all │ ├── key: (1) │ └── fd: (1)-->(2) └── scan b - ├── columns: x:5 z:6!null + ├── columns: x:6 z:7!null ├── stats: [rows=1000] └── cost: 1050.02 @@ -94,7 +94,7 @@ SELECT k, i FROM a EXCEPT SELECT * FROM b except ├── columns: k:1 i:2 ├── left columns: k:1 i:2 - ├── right columns: x:5 z:6 + ├── right columns: x:6 z:7 ├── stats: [rows=1000, distinct(1,2)=1000, null(1,2)=0] ├── cost: 2140.05 ├── key: (1,2) @@ -105,8 +105,8 @@ except │ ├── key: (1) │ └── fd: (1)-->(2) └── scan b - ├── columns: x:5 z:6!null - ├── stats: [rows=1000, distinct(5,6)=1000, null(5,6)=0] + ├── columns: x:6 z:7!null + ├── stats: [rows=1000, distinct(6,7)=1000, null(6,7)=0] └── cost: 1050.02 opt @@ -115,7 +115,7 @@ SELECT k, i FROM a EXCEPT ALL SELECT * FROM b except-all ├── columns: k:1 i:2 ├── left columns: k:1 i:2 - ├── right columns: x:5 z:6 + ├── right columns: x:6 z:7 ├── stats: [rows=1000] ├── cost: 2140.05 ├── scan a @@ -125,6 +125,6 @@ except-all │ ├── key: (1) │ └── fd: (1)-->(2) └── scan b - ├── columns: x:5 z:6!null + ├── columns: x:6 z:7!null ├── stats: [rows=1000] └── cost: 1050.02 diff --git a/pkg/sql/opt/xform/testdata/coster/zone b/pkg/sql/opt/xform/testdata/coster/zone index 61dfcf1a3a38..418f567c0376 100644 --- a/pkg/sql/opt/xform/testdata/coster/zone +++ b/pkg/sql/opt/xform/testdata/coster/zone @@ -363,15 +363,15 @@ opt format=show-all locality=(region=us,dc=west) SELECT * FROM abc INNER LOOKUP JOIN xy ON b=y WHERE b=1 ---- inner-join (lookup xy@y2) - ├── columns: a:1(int!null) b:2(int!null) c:3(string) x:4(int!null) y:5(int!null) + ├── columns: a:1(int!null) b:2(int!null) c:3(string) x:5(int!null) y:6(int!null) ├── flags: force lookup join (into right side) - ├── key columns: [2] = [5] - ├── stats: [rows=100, distinct(2)=1, null(2)=0, distinct(5)=1, null(5)=0] + ├── key columns: [2] = [6] + ├── stats: [rows=100, distinct(2)=1, null(2)=0, distinct(6)=1, null(6)=0] ├── cost: 413.58 - ├── key: (1,4) - ├── fd: ()-->(2,5), (1)-->(3), (2,3)~~>(1), (2)==(5), (5)==(2) - ├── prune: (1,3,4) - ├── interesting orderings: (+1) (+2,+3,+1) (+4) (+5,+4) + ├── key: (1,5) + ├── fd: ()-->(2,6), (1)-->(3), (2,3)~~>(1), (2)==(6), (6)==(2) + ├── prune: (1,3,5) + ├── interesting orderings: (+1) (+2,+3,+1) (+5) (+6,+5) ├── scan t.public.abc@bc2 │ ├── columns: t.public.abc.a:1(int!null) t.public.abc.b:2(int!null) t.public.abc.c:3(string) │ ├── constraint: /2/3: [/1 - /1] @@ -382,8 +382,8 @@ inner-join (lookup xy@y2) │ ├── prune: (1,3) │ └── interesting orderings: (+1) (+2,+3,+1) └── filters - └── eq [type=bool, outer=(5), constraints=(/5: [/1 - /1]; tight), fd=()-->(5)] - ├── variable: t.public.xy.y:5 [type=int] + └── eq [type=bool, outer=(6), constraints=(/6: [/1 - /1]; tight), fd=()-->(6)] + ├── variable: t.public.xy.y:6 [type=int] └── const: 1 [type=int] # Switch the data center for the target lookup join index. @@ -401,15 +401,15 @@ opt format=show-all locality=(region=us,dc=west) SELECT * FROM abc INNER LOOKUP JOIN xy ON b=y WHERE b=1 ---- inner-join (lookup xy@y1) - ├── columns: a:1(int!null) b:2(int!null) c:3(string) x:4(int!null) y:5(int!null) + ├── columns: a:1(int!null) b:2(int!null) c:3(string) x:5(int!null) y:6(int!null) ├── flags: force lookup join (into right side) - ├── key columns: [2] = [5] - ├── stats: [rows=100, distinct(2)=1, null(2)=0, distinct(5)=1, null(5)=0] + ├── key columns: [2] = [6] + ├── stats: [rows=100, distinct(2)=1, null(2)=0, distinct(6)=1, null(6)=0] ├── cost: 413.58 - ├── key: (1,4) - ├── fd: ()-->(2,5), (1)-->(3), (2,3)~~>(1), (2)==(5), (5)==(2) - ├── prune: (1,3,4) - ├── interesting orderings: (+1) (+2,+3,+1) (+4) (+5,+4) + ├── key: (1,5) + ├── fd: ()-->(2,6), (1)-->(3), (2,3)~~>(1), (2)==(6), (6)==(2) + ├── prune: (1,3,5) + ├── interesting orderings: (+1) (+2,+3,+1) (+5) (+6,+5) ├── scan t.public.abc@bc2 │ ├── columns: t.public.abc.a:1(int!null) t.public.abc.b:2(int!null) t.public.abc.c:3(string) │ ├── constraint: /2/3: [/1 - /1] @@ -420,8 +420,8 @@ inner-join (lookup xy@y1) │ ├── prune: (1,3) │ └── interesting orderings: (+1) (+2,+3,+1) └── filters - └── eq [type=bool, outer=(5), constraints=(/5: [/1 - /1]; tight), fd=()-->(5)] - ├── variable: t.public.xy.y:5 [type=int] + └── eq [type=bool, outer=(6), constraints=(/6: [/1 - /1]; tight), fd=()-->(6)] + ├── variable: t.public.xy.y:6 [type=int] └── const: 1 [type=int] # -------------------------------------------------- diff --git a/pkg/sql/opt/xform/testdata/external/activerecord b/pkg/sql/opt/xform/testdata/external/activerecord index d5e189e1578c..d6004ccc2476 100644 --- a/pkg/sql/opt/xform/testdata/external/activerecord +++ b/pkg/sql/opt/xform/testdata/external/activerecord @@ -115,40 +115,40 @@ AND a.attnum > 0 AND NOT a.attisdropped ORDER BY a.attnum ---- sort - ├── columns: attname:2!null format_type:67 pg_get_expr:68 attnotnull:13!null atttypid:3!null atttypmod:9!null collname:69 comment:70 [hidden: attnum:6!null] + ├── columns: attname:2!null format_type:71 pg_get_expr:72 attnotnull:13!null atttypid:3!null atttypmod:9!null collname:73 comment:74 [hidden: attnum:6!null] ├── stable ├── key: (6) - ├── fd: (6)-->(2,3,9,13,67-70), (2)-->(3,6,9,13,67-70), (3,9)-->(67) + ├── fd: (6)-->(2,3,9,13,71-74), (2)-->(3,6,9,13,71-74), (3,9)-->(71) ├── ordering: +6 └── project - ├── columns: format_type:67 pg_get_expr:68 collname:69 comment:70 attname:2!null atttypid:3!null attnum:6!null atttypmod:9!null attnotnull:13!null + ├── columns: format_type:71 pg_get_expr:72 collname:73 comment:74 attname:2!null atttypid:3!null attnum:6!null atttypmod:9!null attnotnull:13!null ├── stable ├── key: (6) - ├── fd: (6)-->(2,3,9,13,67-70), (2)-->(3,6,9,13,67-70), (3,9)-->(67) + ├── fd: (6)-->(2,3,9,13,71-74), (2)-->(3,6,9,13,71-74), (3,9)-->(71) ├── right-join (hash) - │ ├── columns: attrelid:1!null attname:2!null atttypid:3!null attnum:6!null atttypmod:9!null attnotnull:13!null attisdropped:17!null attcollation:20!null adrelid:25 adnum:26 adbin:27 c.oid:29 c.collname:30 t.oid:36 typcollation:63 + │ ├── columns: attrelid:1!null attname:2!null atttypid:3!null attnum:6!null atttypmod:9!null attnotnull:13!null attisdropped:17!null attcollation:20!null adrelid:26 adnum:27 adbin:28 c.oid:31 c.collname:32 t.oid:39 typcollation:66 │ ├── key: (6) - │ ├── fd: ()-->(1,17), (6)-->(2,3,9,13,20,25-27,29,30,36,63), (2)-->(3,6,9,13,20), (26)-->(27), (29)-->(30), (36)-->(63) + │ ├── fd: ()-->(1,17), (6)-->(2,3,9,13,20,26-28,31,32,39,66), (2)-->(3,6,9,13,20), (27)-->(28), (31)-->(32), (39)-->(66) │ ├── inner-join (cross) - │ │ ├── columns: c.oid:29!null c.collname:30!null t.oid:36!null typcollation:63!null - │ │ ├── key: (29,36) - │ │ ├── fd: (29)-->(30), (36)-->(63) + │ │ ├── columns: c.oid:31!null c.collname:32!null t.oid:39!null typcollation:66!null + │ │ ├── key: (31,39) + │ │ ├── fd: (31)-->(32), (39)-->(66) │ │ ├── scan c@pg_collation_name_enc_nsp_index - │ │ │ ├── columns: c.oid:29!null c.collname:30!null - │ │ │ ├── key: (29) - │ │ │ └── fd: (29)-->(30) + │ │ │ ├── columns: c.oid:31!null c.collname:32!null + │ │ │ ├── key: (31) + │ │ │ └── fd: (31)-->(32) │ │ ├── scan t - │ │ │ ├── columns: t.oid:36!null typcollation:63!null - │ │ │ ├── key: (36) - │ │ │ └── fd: (36)-->(63) + │ │ │ ├── columns: t.oid:39!null typcollation:66!null + │ │ │ ├── key: (39) + │ │ │ └── fd: (39)-->(66) │ │ └── filters - │ │ └── c.oid:29 != typcollation:63 [outer=(29,63), constraints=(/29: (/NULL - ]; /63: (/NULL - ])] + │ │ └── c.oid:31 != typcollation:66 [outer=(31,66), constraints=(/31: (/NULL - ]; /66: (/NULL - ])] │ ├── left-join (merge) - │ │ ├── columns: attrelid:1!null attname:2!null atttypid:3!null attnum:6!null atttypmod:9!null attnotnull:13!null attisdropped:17!null attcollation:20!null adrelid:25 adnum:26 adbin:27 + │ │ ├── columns: attrelid:1!null attname:2!null atttypid:3!null attnum:6!null atttypmod:9!null attnotnull:13!null attisdropped:17!null attcollation:20!null adrelid:26 adnum:27 adbin:28 │ │ ├── left ordering: +1,+6 - │ │ ├── right ordering: +25,+26 + │ │ ├── right ordering: +26,+27 │ │ ├── key: (6) - │ │ ├── fd: ()-->(1,17), (6)-->(2,3,9,13,20,25-27), (2)-->(3,6,9,13,20), (26)-->(27) + │ │ ├── fd: ()-->(1,17), (6)-->(2,3,9,13,20,26-28), (2)-->(3,6,9,13,20), (27)-->(28) │ │ ├── select │ │ │ ├── columns: attrelid:1!null attname:2!null atttypid:3!null attnum:6!null atttypmod:9!null attnotnull:13!null attisdropped:17!null attcollation:20!null │ │ │ ├── key: (6) @@ -163,22 +163,22 @@ sort │ │ │ └── filters │ │ │ └── NOT attisdropped:17 [outer=(17), constraints=(/17: [/false - /false]; tight), fd=()-->(17)] │ │ ├── index-join pg_attrdef - │ │ │ ├── columns: adrelid:25!null adnum:26!null adbin:27 - │ │ │ ├── key: (26) - │ │ │ ├── fd: ()-->(25), (26)-->(27) - │ │ │ ├── ordering: +26 opt(25) [actual: +26] + │ │ │ ├── columns: adrelid:26!null adnum:27!null adbin:28 + │ │ │ ├── key: (27) + │ │ │ ├── fd: ()-->(26), (27)-->(28) + │ │ │ ├── ordering: +27 opt(26) [actual: +27] │ │ │ └── scan d@pg_attrdef_adrelid_adnum_index - │ │ │ ├── columns: d.oid:24!null adrelid:25!null adnum:26!null - │ │ │ ├── constraint: /25/26: [/numbers/1 - /numbers] - │ │ │ ├── key: (24) - │ │ │ ├── fd: ()-->(25), (24)-->(26), (26)-->(24) - │ │ │ └── ordering: +26 opt(25) [actual: +26] + │ │ │ ├── columns: d.oid:25!null adrelid:26!null adnum:27!null + │ │ │ ├── constraint: /26/27: [/numbers/1 - /numbers] + │ │ │ ├── key: (25) + │ │ │ ├── fd: ()-->(26), (25)-->(27), (27)-->(25) + │ │ │ └── ordering: +27 opt(26) [actual: +27] │ │ └── filters (true) │ └── filters - │ ├── c.oid:29 = attcollation:20 [outer=(20,29), constraints=(/20: (/NULL - ]; /29: (/NULL - ]), fd=(20)==(29), (29)==(20)] - │ └── t.oid:36 = atttypid:3 [outer=(3,36), constraints=(/3: (/NULL - ]; /36: (/NULL - ]), fd=(3)==(36), (36)==(3)] + │ ├── c.oid:31 = attcollation:20 [outer=(20,31), constraints=(/20: (/NULL - ]; /31: (/NULL - ]), fd=(20)==(31), (31)==(20)] + │ └── t.oid:39 = atttypid:3 [outer=(3,39), constraints=(/3: (/NULL - ]; /39: (/NULL - ]), fd=(3)==(39), (39)==(3)] └── projections - ├── format_type(atttypid:3, atttypmod:9) [as=format_type:67, outer=(3,9), stable] - ├── pg_get_expr(adbin:27, adrelid:25) [as=pg_get_expr:68, outer=(25,27), stable] - ├── c.collname:30 [as=collname:69, outer=(30)] - └── col_description(attrelid:1, attnum:6) [as=comment:70, outer=(1,6), stable] + ├── format_type(atttypid:3, atttypmod:9) [as=format_type:71, outer=(3,9), stable] + ├── pg_get_expr(adbin:28, adrelid:26) [as=pg_get_expr:72, outer=(26,28), stable] + ├── c.collname:32 [as=collname:73, outer=(32)] + └── col_description(attrelid:1, attnum:6) [as=comment:74, outer=(1,6), stable] diff --git a/pkg/sql/opt/xform/testdata/external/customer b/pkg/sql/opt/xform/testdata/external/customer index 5eff094636c6..25724045dd99 100644 --- a/pkg/sql/opt/xform/testdata/external/customer +++ b/pkg/sql/opt/xform/testdata/external/customer @@ -37,17 +37,17 @@ opt select nodes.id,dst from nodes join edges on edges.dst=nodes.id ---- inner-join (merge) - ├── columns: id:1!null dst:4!null + ├── columns: id:1!null dst:5!null ├── left ordering: +1 - ├── right ordering: +4 - ├── fd: (1)==(4), (4)==(1) + ├── right ordering: +5 + ├── fd: (1)==(5), (5)==(1) ├── scan nodes │ ├── columns: id:1!null │ ├── key: (1) │ └── ordering: +1 ├── scan edges@edges_auto_index_fk_dst_ref_nodes - │ ├── columns: dst:4!null - │ └── ordering: +4 + │ ├── columns: dst:5!null + │ └── ordering: +5 └── filters (true) # ------------------------------------------------------------------------------ @@ -284,24 +284,24 @@ opt select v.owner_id, count(*) from rides r, vehicles v where v.id = r.vehicle_id group by v.owner_id ---- group-by - ├── columns: owner_id:12 count:16!null - ├── grouping columns: owner_id:12 - ├── key: (12) - ├── fd: (12)-->(16) + ├── columns: owner_id:13 count:18!null + ├── grouping columns: owner_id:13 + ├── key: (13) + ├── fd: (13)-->(18) ├── inner-join (merge) - │ ├── columns: vehicle_id:3!null v.id:9!null owner_id:12 + │ ├── columns: vehicle_id:3!null v.id:10!null owner_id:13 │ ├── left ordering: +3 - │ ├── right ordering: +9 - │ ├── fd: (3)==(9), (9)==(3) + │ ├── right ordering: +10 + │ ├── fd: (3)==(10), (10)==(3) │ ├── scan r@rides_vehicle_id_idx │ │ ├── columns: vehicle_id:3 │ │ └── ordering: +3 │ ├── scan v@vehicles_id_idx - │ │ ├── columns: v.id:9!null owner_id:12 - │ │ └── ordering: +9 + │ │ ├── columns: v.id:10!null owner_id:13 + │ │ └── ordering: +10 │ └── filters (true) └── aggregations - └── count-rows [as=count_rows:16] + └── count-rows [as=count_rows:18] # ------------------------------------------------------------------------------ # Github Issue 24415: Ensure the optimizer uses the covering index. @@ -328,16 +328,16 @@ opt SELECT id, sum(value) FROM data GROUP BY id ---- group-by - ├── columns: id:1 sum:14 + ├── columns: id:1 sum:15 ├── grouping columns: id:1 ├── internal-ordering: +1 ├── key: (1) - ├── fd: (1)-->(14) + ├── fd: (1)-->(15) ├── scan data@foo │ ├── columns: id:1 value:2 │ └── ordering: +1 └── aggregations - └── sum [as=sum:14, outer=(2)] + └── sum [as=sum:15, outer=(2)] └── value:2 # ------------------------------------------------------------------------------ @@ -426,7 +426,7 @@ opt EXPLAIN SELECT id FROM test ORDER BY id asc LIMIT 10 offset 10000; ---- explain - ├── columns: tree:4 field:5 description:6 + ├── columns: tree:5 field:6 description:7 └── offset ├── columns: id:1!null ├── internal-ordering: +1 @@ -496,25 +496,25 @@ opt SELECT count(l_orderkey) FROM orders, lineitem WHERE orders.o_orderkey = lineitem.l_orderkey ---- scalar-group-by - ├── columns: count:28!null + ├── columns: count:30!null ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(28) + ├── fd: ()-->(30) ├── inner-join (merge) - │ ├── columns: o_orderkey:1!null l_orderkey:11!null + │ ├── columns: o_orderkey:1!null l_orderkey:12!null │ ├── left ordering: +1 - │ ├── right ordering: +11 - │ ├── fd: (1)==(11), (11)==(1) + │ ├── right ordering: +12 + │ ├── fd: (1)==(12), (12)==(1) │ ├── scan orders@o_ok │ │ ├── columns: o_orderkey:1!null │ │ ├── key: (1) │ │ └── ordering: +1 │ ├── scan lineitem@l_ok - │ │ ├── columns: l_orderkey:11!null - │ │ └── ordering: +11 + │ │ ├── columns: l_orderkey:12!null + │ │ └── ordering: +12 │ └── filters (true) └── aggregations - └── count-rows [as=count:28] + └── count-rows [as=count:30] # ------------------------------------------------------------------------------ # Ensure we do a lookup join when one side comes from an SRF. @@ -543,33 +543,33 @@ WHERE secondary_id = (elem->>'secondary_id')::UUID ---- project - ├── columns: secondary_id:6 "?column?":7 + ├── columns: secondary_id:7 "?column?":8 ├── stable ├── inner-join (lookup idtable) - │ ├── columns: primary_id:1!null idtable.secondary_id:2!null data:3!null value:4!null column5:5!null + │ ├── columns: primary_id:1!null idtable.secondary_id:2!null data:3!null value:5!null column6:6!null │ ├── key columns: [1] = [1] │ ├── lookup columns are key │ ├── immutable - │ ├── fd: (1)-->(2,3), (4)-->(5), (2)==(5), (5)==(2) + │ ├── fd: (1)-->(2,3), (5)-->(6), (2)==(6), (6)==(2) │ ├── inner-join (lookup idtable@secondary_id) - │ │ ├── columns: primary_id:1!null idtable.secondary_id:2!null value:4!null column5:5!null - │ │ ├── key columns: [5] = [2] + │ │ ├── columns: primary_id:1!null idtable.secondary_id:2!null value:5!null column6:6!null + │ │ ├── key columns: [6] = [2] │ │ ├── immutable - │ │ ├── fd: (4)-->(5), (1)-->(2), (2)==(5), (5)==(2) + │ │ ├── fd: (5)-->(6), (1)-->(2), (2)==(6), (6)==(2) │ │ ├── project - │ │ │ ├── columns: column5:5 value:4!null + │ │ │ ├── columns: column6:6 value:5!null │ │ │ ├── cardinality: [2 - 2] │ │ │ ├── immutable - │ │ │ ├── fd: (4)-->(5) + │ │ │ ├── fd: (5)-->(6) │ │ │ ├── values - │ │ │ │ ├── columns: value:4!null + │ │ │ │ ├── columns: value:5!null │ │ │ │ ├── cardinality: [2 - 2] │ │ │ │ ├── ('{"person_id": "8e5dc104-9f38-4255-9283-fd080be16c57", "product_id": "a739c2d3-edec-413b-88d8-9c31d0414b1e"}',) │ │ │ │ └── ('{"person_id": "308686c4-7415-4c2d-92d5-25b39a1c84e2", "product_id": "3f12802d-5b0f-43d7-a0d0-12ac8e88cb18"}',) │ │ │ └── projections - │ │ │ └── (value:4->>'secondary_id')::UUID [as=column5:5, outer=(4), immutable] + │ │ │ └── (value:5->>'secondary_id')::UUID [as=column6:6, outer=(5), immutable] │ │ └── filters (true) │ └── filters (true) └── projections - ├── value:4->>'secondary_id' [as=secondary_id:6, outer=(4), immutable] - └── data:3 || jsonb_build_object('primary_id', primary_id:1) [as="?column?":7, outer=(1,3), stable] + ├── value:5->>'secondary_id' [as=secondary_id:7, outer=(5), immutable] + └── data:3 || jsonb_build_object('primary_id', primary_id:1) [as="?column?":8, outer=(1,3), stable] diff --git a/pkg/sql/opt/xform/testdata/external/hibernate b/pkg/sql/opt/xform/testdata/external/hibernate index 8f5c0a1d276e..8bfe0131239d 100644 --- a/pkg/sql/opt/xform/testdata/external/hibernate +++ b/pkg/sql/opt/xform/testdata/external/hibernate @@ -54,21 +54,21 @@ inner join Phone unidirecti1_ where phoneregis0_.phone_id=1; ---- project - ├── columns: phone_id1_2_0_:1!null person_i2_2_0_:2!null formula159_0_:11 id1_1_1_:3!null number2_1_1_:4 since3_1_1_:5 type4_1_1_:6 - ├── key: (3) - ├── fd: ()-->(1), (3)-->(4-6), (2)==(3), (3)==(2), (2)-->(11) + ├── columns: phone_id1_2_0_:1!null person_i2_2_0_:2!null formula159_0_:14 id1_1_1_:4!null number2_1_1_:5 since3_1_1_:6 type4_1_1_:7 + ├── key: (4) + ├── fd: ()-->(1), (4)-->(5-7), (2)==(4), (4)==(2), (2)-->(14) ├── inner-join (lookup phone) - │ ├── columns: phone_id:1!null person_id:2!null unidirecti1_.id:3!null unidirecti1_.number:4 unidirecti1_.since:5 unidirecti1_.type:6 a10.id:7!null a10.since:9 - │ ├── key columns: [2] = [7] + │ ├── columns: phone_id:1!null person_id:2!null unidirecti1_.id:4!null unidirecti1_.number:5 unidirecti1_.since:6 unidirecti1_.type:7 a10.id:9!null a10.since:11 + │ ├── key columns: [2] = [9] │ ├── lookup columns are key - │ ├── key: (7) - │ ├── fd: ()-->(1), (3)-->(4-6), (2)==(3,7), (3)==(2,7), (7)-->(9), (7)==(2,3) + │ ├── key: (9) + │ ├── fd: ()-->(1), (4)-->(5-7), (2)==(4,9), (4)==(2,9), (9)-->(11), (9)==(2,4) │ ├── inner-join (lookup phone) - │ │ ├── columns: phone_id:1!null person_id:2!null unidirecti1_.id:3!null unidirecti1_.number:4 unidirecti1_.since:5 unidirecti1_.type:6 - │ │ ├── key columns: [2] = [3] + │ │ ├── columns: phone_id:1!null person_id:2!null unidirecti1_.id:4!null unidirecti1_.number:5 unidirecti1_.since:6 unidirecti1_.type:7 + │ │ ├── key columns: [2] = [4] │ │ ├── lookup columns are key - │ │ ├── key: (3) - │ │ ├── fd: ()-->(1), (3)-->(4-6), (2)==(3), (3)==(2) + │ │ ├── key: (4) + │ │ ├── fd: ()-->(1), (4)-->(5-7), (2)==(4), (4)==(2) │ │ ├── scan phoneregis0_ │ │ │ ├── columns: phone_id:1!null person_id:2!null │ │ │ ├── constraint: /1/2: [/1 - /1] @@ -77,7 +77,7 @@ project │ │ └── filters (true) │ └── filters (true) └── projections - └── a10.since:9 [as=formula159_0_:11, outer=(9)] + └── a10.since:11 [as=formula159_0_:14, outer=(11)] exec-ddl drop table phone_register, Person, Phone; @@ -179,39 +179,39 @@ where ) ---- inner-join (hash) - ├── columns: id1_6_0_:1!null id1_4_1_:6!null phone_nu2_6_0_:2 person_i4_6_0_:4!null phone_ty3_6_0_:3 person_i1_5_0__:12!null addresse2_5_0__:13 addresse3_0__:14!null address2_4_1_:7 createdo3_4_1_:8 name4_4_1_:9 nickname5_4_1_:10 version6_4_1_:11!null person_i1_5_0__:12!null addresse2_5_0__:13 addresse3_0__:14!null - ├── key: (1,14) - ├── fd: (1)-->(2-4), (6)-->(7-11), (4)==(6,12), (6)==(4,12), (12,14)-->(13), (12)==(4,6) + ├── columns: id1_6_0_:1!null id1_4_1_:7!null phone_nu2_6_0_:2 person_i4_6_0_:4!null phone_ty3_6_0_:3 person_i1_5_0__:14!null addresse2_5_0__:15 addresse3_0__:16!null address2_4_1_:8 createdo3_4_1_:9 name4_4_1_:10 nickname5_4_1_:11 version6_4_1_:12!null person_i1_5_0__:14!null addresse2_5_0__:15 addresse3_0__:16!null + ├── key: (1,16) + ├── fd: (1)-->(2-4), (7)-->(8-12), (4)==(7,14), (7)==(4,14), (14,16)-->(15), (14)==(4,7) ├── scan addresses2_ - │ ├── columns: addresses2_.person_id:12!null addresses:13 addresses_key:14!null - │ ├── key: (12,14) - │ └── fd: (12,14)-->(13) + │ ├── columns: addresses2_.person_id:14!null addresses:15 addresses_key:16!null + │ ├── key: (14,16) + │ └── fd: (14,16)-->(15) ├── inner-join (lookup person) - │ ├── columns: phone0_.id:1!null phone_number:2 phone_type:3 phone0_.person_id:4!null person1_.id:6!null address:7 createdon:8 name:9 nickname:10 version:11!null - │ ├── key columns: [4] = [6] + │ ├── columns: phone0_.id:1!null phone_number:2 phone_type:3 phone0_.person_id:4!null person1_.id:7!null address:8 createdon:9 name:10 nickname:11 version:12!null + │ ├── key columns: [4] = [7] │ ├── lookup columns are key │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (6)-->(7-11), (4)==(6), (6)==(4) + │ ├── fd: (1)-->(2-4), (7)-->(8-12), (4)==(7), (7)==(4) │ ├── project │ │ ├── columns: phone0_.id:1!null phone_number:2 phone_type:3 phone0_.person_id:4 │ │ ├── key: (1) │ │ ├── fd: (1)-->(2-4) │ │ └── inner-join (lookup phone) - │ │ ├── columns: phone0_.id:1!null phone_number:2 phone_type:3 phone0_.person_id:4 phone_id:18!null - │ │ ├── key columns: [18] = [1] + │ │ ├── columns: phone0_.id:1!null phone_number:2 phone_type:3 phone0_.person_id:4 phone_id:21!null + │ │ ├── key columns: [21] = [1] │ │ ├── lookup columns are key - │ │ ├── key: (18) - │ │ ├── fd: (1)-->(2-4), (1)==(18), (18)==(1) + │ │ ├── key: (21) + │ │ ├── fd: (1)-->(2-4), (1)==(21), (21)==(1) │ │ ├── distinct-on - │ │ │ ├── columns: phone_id:18 - │ │ │ ├── grouping columns: phone_id:18 - │ │ │ ├── key: (18) + │ │ │ ├── columns: phone_id:21 + │ │ │ ├── grouping columns: phone_id:21 + │ │ │ ├── key: (21) │ │ │ └── scan calls3_ - │ │ │ └── columns: phone_id:18 + │ │ │ └── columns: phone_id:21 │ │ └── filters (true) │ └── filters (true) └── filters - └── person1_.id:6 = addresses2_.person_id:12 [outer=(6,12), constraints=(/6: (/NULL - ]; /12: (/NULL - ]), fd=(6)==(12), (12)==(6)] + └── person1_.id:7 = addresses2_.person_id:14 [outer=(7,14), constraints=(/7: (/NULL - ]; /14: (/NULL - ]), fd=(7)==(14), (14)==(7)] opt select @@ -241,36 +241,36 @@ project ├── columns: id1_6_:1!null phone_nu2_6_:2 person_i4_6_:4!null phone_ty3_6_:3 ├── fd: (1)-->(2-4) └── inner-join (hash) - ├── columns: phone0_.id:1!null phone_number:2 phone_type:3 phone0_.person_id:4!null person1_.id:6!null addresses2_.person_id:12!null - ├── fd: (1)-->(2-4), (4)==(6,12), (6)==(4,12), (12)==(4,6) + ├── columns: phone0_.id:1!null phone_number:2 phone_type:3 phone0_.person_id:4!null person1_.id:7!null addresses2_.person_id:14!null + ├── fd: (1)-->(2-4), (4)==(7,14), (7)==(4,14), (14)==(4,7) ├── scan addresses2_ - │ └── columns: addresses2_.person_id:12!null + │ └── columns: addresses2_.person_id:14!null ├── inner-join (lookup person) - │ ├── columns: phone0_.id:1!null phone_number:2 phone_type:3 phone0_.person_id:4!null person1_.id:6!null - │ ├── key columns: [4] = [6] + │ ├── columns: phone0_.id:1!null phone_number:2 phone_type:3 phone0_.person_id:4!null person1_.id:7!null + │ ├── key columns: [4] = [7] │ ├── lookup columns are key │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (4)==(6), (6)==(4) + │ ├── fd: (1)-->(2-4), (4)==(7), (7)==(4) │ ├── project │ │ ├── columns: phone0_.id:1!null phone_number:2 phone_type:3 phone0_.person_id:4 │ │ ├── key: (1) │ │ ├── fd: (1)-->(2-4) │ │ └── inner-join (lookup phone) - │ │ ├── columns: phone0_.id:1!null phone_number:2 phone_type:3 phone0_.person_id:4 phone_id:18!null - │ │ ├── key columns: [18] = [1] + │ │ ├── columns: phone0_.id:1!null phone_number:2 phone_type:3 phone0_.person_id:4 phone_id:21!null + │ │ ├── key columns: [21] = [1] │ │ ├── lookup columns are key - │ │ ├── key: (18) - │ │ ├── fd: (1)-->(2-4), (1)==(18), (18)==(1) + │ │ ├── key: (21) + │ │ ├── fd: (1)-->(2-4), (1)==(21), (21)==(1) │ │ ├── distinct-on - │ │ │ ├── columns: phone_id:18 - │ │ │ ├── grouping columns: phone_id:18 - │ │ │ ├── key: (18) + │ │ │ ├── columns: phone_id:21 + │ │ │ ├── grouping columns: phone_id:21 + │ │ │ ├── key: (21) │ │ │ └── scan calls3_ - │ │ │ └── columns: phone_id:18 + │ │ │ └── columns: phone_id:21 │ │ └── filters (true) │ └── filters (true) └── filters - └── person1_.id:6 = addresses2_.person_id:12 [outer=(6,12), constraints=(/6: (/NULL - ]; /12: (/NULL - ]), fd=(6)==(12), (12)==(6)] + └── person1_.id:7 = addresses2_.person_id:14 [outer=(7,14), constraints=(/7: (/NULL - ]; /14: (/NULL - ]), fd=(7)==(14), (14)==(7)] opt select @@ -305,42 +305,42 @@ where and partner1_.version=0 ---- inner-join (cross) - ├── columns: id1_4_0_:1!null id1_2_1_:7!null address2_4_0_:2!null createdo3_4_0_:3 name4_4_0_:4 nickname5_4_0_:5 version6_4_0_:6!null name2_2_1_:8!null version3_2_1_:9!null + ├── columns: id1_4_0_:1!null id1_2_1_:8!null address2_4_0_:2!null createdo3_4_0_:3 name4_4_0_:4 nickname5_4_0_:5 version6_4_0_:6!null name2_2_1_:9!null version3_2_1_:10!null ├── has-placeholder - ├── key: (1,7) - ├── fd: ()-->(9), (1)-->(2-6), (7)-->(8) + ├── key: (1,8) + ├── fd: ()-->(10), (1)-->(2-6), (8)-->(9) ├── project │ ├── columns: person0_.id:1!null address:2!null createdon:3 person0_.name:4 nickname:5 person0_.version:6!null │ ├── has-placeholder │ ├── key: (1) │ ├── fd: (1)-->(2-6) │ └── inner-join (lookup person) - │ ├── columns: person0_.id:1!null address:2!null createdon:3 person0_.name:4 nickname:5 person0_.version:6!null person_id:13!null - │ ├── key columns: [13] = [1] + │ ├── columns: person0_.id:1!null address:2!null createdon:3 person0_.name:4 nickname:5 person0_.version:6!null person_id:15!null + │ ├── key columns: [15] = [1] │ ├── lookup columns are key │ ├── has-placeholder - │ ├── key: (13) - │ ├── fd: (1)-->(2-6), (1)==(13), (13)==(1) + │ ├── key: (15) + │ ├── fd: (1)-->(2-6), (1)==(15), (15)==(1) │ ├── distinct-on - │ │ ├── columns: person_id:13 - │ │ ├── grouping columns: person_id:13 - │ │ ├── key: (13) + │ │ ├── columns: person_id:15 + │ │ ├── grouping columns: person_id:15 + │ │ ├── key: (15) │ │ └── scan phones2_ - │ │ └── columns: person_id:13 + │ │ └── columns: person_id:15 │ └── filters │ └── address:2 = $1 [outer=(2), constraints=(/2: (/NULL - ])] ├── select - │ ├── columns: partner1_.id:7!null partner1_.name:8!null partner1_.version:9!null + │ ├── columns: partner1_.id:8!null partner1_.name:9!null partner1_.version:10!null │ ├── has-placeholder - │ ├── key: (7) - │ ├── fd: ()-->(9), (7)-->(8) + │ ├── key: (8) + │ ├── fd: ()-->(10), (8)-->(9) │ ├── scan partner1_ - │ │ ├── columns: partner1_.id:7!null partner1_.name:8 partner1_.version:9!null - │ │ ├── key: (7) - │ │ └── fd: (7)-->(8,9) + │ │ ├── columns: partner1_.id:8!null partner1_.name:9 partner1_.version:10!null + │ │ ├── key: (8) + │ │ └── fd: (8)-->(9,10) │ └── filters - │ ├── partner1_.name:8 LIKE $2 [outer=(8), constraints=(/8: (/NULL - ])] - │ └── partner1_.version:9 = 0 [outer=(9), constraints=(/9: [/0 - /0]; tight), fd=()-->(9)] + │ ├── partner1_.name:9 LIKE $2 [outer=(9), constraints=(/9: (/NULL - ])] + │ └── partner1_.version:10 = 0 [outer=(10), constraints=(/10: [/0 - /0]; tight), fd=()-->(10)] └── filters (true) exec-ddl @@ -391,23 +391,23 @@ project ├── key: (1,2) ├── fd: (1,2)-->(3-6) └── select - ├── columns: defaultaud0_.id:1!null defaultaud0_.rev:2!null defaultaud0_.revtype:3!null defaultaud0_.created_on:4 defaultaud0_.firstname:5 defaultaud0_.lastname:6 max:13!null + ├── columns: defaultaud0_.id:1!null defaultaud0_.rev:2!null defaultaud0_.revtype:3!null defaultaud0_.created_on:4 defaultaud0_.firstname:5 defaultaud0_.lastname:6 max:15!null ├── has-placeholder ├── key: (1,2) - ├── fd: (1,2)-->(3-6,13), (2)==(13), (13)==(2) + ├── fd: (1,2)-->(3-6,15), (2)==(15), (15)==(2) ├── group-by - │ ├── columns: defaultaud0_.id:1!null defaultaud0_.rev:2!null defaultaud0_.revtype:3!null defaultaud0_.created_on:4 defaultaud0_.firstname:5 defaultaud0_.lastname:6 max:13!null + │ ├── columns: defaultaud0_.id:1!null defaultaud0_.rev:2!null defaultaud0_.revtype:3!null defaultaud0_.created_on:4 defaultaud0_.firstname:5 defaultaud0_.lastname:6 max:15!null │ ├── grouping columns: defaultaud0_.id:1!null defaultaud0_.rev:2!null │ ├── has-placeholder │ ├── key: (1,2) - │ ├── fd: (1,2)-->(3-6,13) + │ ├── fd: (1,2)-->(3-6,15) │ ├── inner-join (merge) - │ │ ├── columns: defaultaud0_.id:1!null defaultaud0_.rev:2!null defaultaud0_.revtype:3!null defaultaud0_.created_on:4 defaultaud0_.firstname:5 defaultaud0_.lastname:6 defaultaud1_.id:7!null defaultaud1_.rev:8!null + │ │ ├── columns: defaultaud0_.id:1!null defaultaud0_.rev:2!null defaultaud0_.revtype:3!null defaultaud0_.created_on:4 defaultaud0_.firstname:5 defaultaud0_.lastname:6 defaultaud1_.id:8!null defaultaud1_.rev:9!null │ │ ├── left ordering: +1 - │ │ ├── right ordering: +7 + │ │ ├── right ordering: +8 │ │ ├── has-placeholder - │ │ ├── key: (2,7,8) - │ │ ├── fd: (1,2)-->(3-6), (1)==(7), (7)==(1) + │ │ ├── key: (2,8,9) + │ │ ├── fd: (1,2)-->(3-6), (1)==(8), (8)==(1) │ │ ├── select │ │ │ ├── columns: defaultaud0_.id:1!null defaultaud0_.rev:2!null defaultaud0_.revtype:3!null defaultaud0_.created_on:4 defaultaud0_.firstname:5 defaultaud0_.lastname:6 │ │ │ ├── has-placeholder @@ -422,20 +422,20 @@ project │ │ │ └── filters │ │ │ └── defaultaud0_.revtype:3 != $2 [outer=(3), constraints=(/3: (/NULL - ])] │ │ ├── select - │ │ │ ├── columns: defaultaud1_.id:7!null defaultaud1_.rev:8!null + │ │ │ ├── columns: defaultaud1_.id:8!null defaultaud1_.rev:9!null │ │ │ ├── has-placeholder - │ │ │ ├── key: (7,8) - │ │ │ ├── ordering: +7 + │ │ │ ├── key: (8,9) + │ │ │ ├── ordering: +8 │ │ │ ├── scan defaultaud1_ - │ │ │ │ ├── columns: defaultaud1_.id:7!null defaultaud1_.rev:8!null - │ │ │ │ ├── key: (7,8) - │ │ │ │ └── ordering: +7 + │ │ │ │ ├── columns: defaultaud1_.id:8!null defaultaud1_.rev:9!null + │ │ │ │ ├── key: (8,9) + │ │ │ │ └── ordering: +8 │ │ │ └── filters - │ │ │ └── defaultaud1_.rev:8 <= $1 [outer=(8), constraints=(/8: (/NULL - ])] + │ │ │ └── defaultaud1_.rev:9 <= $1 [outer=(9), constraints=(/9: (/NULL - ])] │ │ └── filters (true) │ └── aggregations - │ ├── max [as=max:13, outer=(8)] - │ │ └── defaultaud1_.rev:8 + │ ├── max [as=max:15, outer=(9)] + │ │ └── defaultaud1_.rev:9 │ ├── const-agg [as=defaultaud0_.revtype:3, outer=(3)] │ │ └── defaultaud0_.revtype:3 │ ├── const-agg [as=defaultaud0_.created_on:4, outer=(4)] @@ -445,7 +445,7 @@ project │ └── const-agg [as=defaultaud0_.lastname:6, outer=(6)] │ └── defaultaud0_.lastname:6 └── filters - └── defaultaud0_.rev:2 = max:13 [outer=(2,13), constraints=(/2: (/NULL - ]; /13: (/NULL - ]), fd=(2)==(13), (13)==(2)] + └── defaultaud0_.rev:2 = max:15 [outer=(2,15), constraints=(/2: (/NULL - ]; /15: (/NULL - ]), fd=(2)==(15), (15)==(2)] exec-ddl drop table Customer_AUD; @@ -506,23 +506,23 @@ sort ├── key: (1,2) ├── fd: (1,2)-->(3-8) └── select - ├── columns: queryaudit0_.id:1!null queryaudit0_.rev:2!null queryaudit0_.revtype:3!null queryaudit0_.revend:4 queryaudit0_.created_on:5 queryaudit0_.firstname:6 queryaudit0_.lastname:7 queryaudit0_.address_id:8 max:17!null + ├── columns: queryaudit0_.id:1!null queryaudit0_.rev:2!null queryaudit0_.revtype:3!null queryaudit0_.revend:4 queryaudit0_.created_on:5 queryaudit0_.firstname:6 queryaudit0_.lastname:7 queryaudit0_.address_id:8 max:19!null ├── has-placeholder ├── key: (1,2) - ├── fd: (1,2)-->(3-8,17), (2)==(17), (17)==(2) + ├── fd: (1,2)-->(3-8,19), (2)==(19), (19)==(2) ├── group-by - │ ├── columns: queryaudit0_.id:1!null queryaudit0_.rev:2!null queryaudit0_.revtype:3!null queryaudit0_.revend:4 queryaudit0_.created_on:5 queryaudit0_.firstname:6 queryaudit0_.lastname:7 queryaudit0_.address_id:8 max:17!null + │ ├── columns: queryaudit0_.id:1!null queryaudit0_.rev:2!null queryaudit0_.revtype:3!null queryaudit0_.revend:4 queryaudit0_.created_on:5 queryaudit0_.firstname:6 queryaudit0_.lastname:7 queryaudit0_.address_id:8 max:19!null │ ├── grouping columns: queryaudit0_.id:1!null queryaudit0_.rev:2!null │ ├── has-placeholder │ ├── key: (1,2) - │ ├── fd: (1,2)-->(3-8,17) + │ ├── fd: (1,2)-->(3-8,19) │ ├── inner-join (merge) - │ │ ├── columns: queryaudit0_.id:1!null queryaudit0_.rev:2!null queryaudit0_.revtype:3!null queryaudit0_.revend:4 queryaudit0_.created_on:5 queryaudit0_.firstname:6 queryaudit0_.lastname:7 queryaudit0_.address_id:8 queryaudit1_.id:9!null queryaudit1_.rev:10!null + │ │ ├── columns: queryaudit0_.id:1!null queryaudit0_.rev:2!null queryaudit0_.revtype:3!null queryaudit0_.revend:4 queryaudit0_.created_on:5 queryaudit0_.firstname:6 queryaudit0_.lastname:7 queryaudit0_.address_id:8 queryaudit1_.id:10!null queryaudit1_.rev:11!null │ │ ├── left ordering: +1 - │ │ ├── right ordering: +9 + │ │ ├── right ordering: +10 │ │ ├── has-placeholder - │ │ ├── key: (2,9,10) - │ │ ├── fd: (1,2)-->(3-8), (1)==(9), (9)==(1) + │ │ ├── key: (2,10,11) + │ │ ├── fd: (1,2)-->(3-8), (1)==(10), (10)==(1) │ │ ├── select │ │ │ ├── columns: queryaudit0_.id:1!null queryaudit0_.rev:2!null queryaudit0_.revtype:3!null queryaudit0_.revend:4 queryaudit0_.created_on:5 queryaudit0_.firstname:6 queryaudit0_.lastname:7 queryaudit0_.address_id:8 │ │ │ ├── has-placeholder @@ -537,13 +537,13 @@ sort │ │ │ └── filters │ │ │ └── queryaudit0_.revtype:3 != $1 [outer=(3), constraints=(/3: (/NULL - ])] │ │ ├── scan queryaudit1_ - │ │ │ ├── columns: queryaudit1_.id:9!null queryaudit1_.rev:10!null - │ │ │ ├── key: (9,10) - │ │ │ └── ordering: +9 + │ │ │ ├── columns: queryaudit1_.id:10!null queryaudit1_.rev:11!null + │ │ │ ├── key: (10,11) + │ │ │ └── ordering: +10 │ │ └── filters (true) │ └── aggregations - │ ├── max [as=max:17, outer=(10)] - │ │ └── queryaudit1_.rev:10 + │ ├── max [as=max:19, outer=(11)] + │ │ └── queryaudit1_.rev:11 │ ├── const-agg [as=queryaudit0_.revtype:3, outer=(3)] │ │ └── queryaudit0_.revtype:3 │ ├── const-agg [as=queryaudit0_.revend:4, outer=(4)] @@ -557,7 +557,7 @@ sort │ └── const-agg [as=queryaudit0_.address_id:8, outer=(8)] │ └── queryaudit0_.address_id:8 └── filters - └── queryaudit0_.rev:2 = max:17 [outer=(2,17), constraints=(/2: (/NULL - ]; /17: (/NULL - ]), fd=(2)==(17), (17)==(2)] + └── queryaudit0_.rev:2 = max:19 [outer=(2,19), constraints=(/2: (/NULL - ]; /19: (/NULL - ]), fd=(2)==(19), (19)==(2)] exec-ddl drop table Customer_AUD; @@ -661,25 +661,25 @@ where ) ---- distinct-on - ├── columns: id1_2_:10!null address2_2_:11 createdo3_2_:12 name4_2_:13 nickname5_2_:14 version6_2_:15!null - ├── grouping columns: person2_.id:10!null - ├── key: (10) - ├── fd: (10)-->(11-15) + ├── columns: id1_2_:12!null address2_2_:13 createdo3_2_:14 name4_2_:15 nickname5_2_:16 version6_2_:17!null + ├── grouping columns: person2_.id:12!null + ├── key: (12) + ├── fd: (12)-->(13-17) ├── inner-join (hash) - │ ├── columns: phone0_.id:1!null person_id:4!null calls1_.phone_id:9!null person2_.id:10!null address:11 createdon:12 name:13 nickname:14 version:15!null + │ ├── columns: phone0_.id:1!null person_id:4!null calls1_.phone_id:10!null person2_.id:12!null address:13 createdon:14 name:15 nickname:16 version:17!null │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ ├── fd: (1)-->(4), (1)==(9), (9)==(1), (10)-->(11-15), (4)==(10), (10)==(4) + │ ├── fd: (1)-->(4), (1)==(10), (10)==(1), (12)-->(13-17), (4)==(12), (12)==(4) │ ├── scan calls1_ - │ │ └── columns: calls1_.phone_id:9 + │ │ └── columns: calls1_.phone_id:10 │ ├── inner-join (hash) - │ │ ├── columns: phone0_.id:1!null person_id:4!null person2_.id:10!null address:11 createdon:12 name:13 nickname:14 version:15!null + │ │ ├── columns: phone0_.id:1!null person_id:4!null person2_.id:12!null address:13 createdon:14 name:15 nickname:16 version:17!null │ │ ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) │ │ ├── key: (1) - │ │ ├── fd: (1)-->(4), (10)-->(11-15), (4)==(10), (10)==(4) + │ │ ├── fd: (1)-->(4), (12)-->(13-17), (4)==(12), (12)==(4) │ │ ├── scan person2_ - │ │ │ ├── columns: person2_.id:10!null address:11 createdon:12 name:13 nickname:14 version:15!null - │ │ │ ├── key: (10) - │ │ │ └── fd: (10)-->(11-15) + │ │ │ ├── columns: person2_.id:12!null address:13 createdon:14 name:15 nickname:16 version:17!null + │ │ │ ├── key: (12) + │ │ │ └── fd: (12)-->(13-17) │ │ ├── anti-join (hash) │ │ │ ├── columns: phone0_.id:1!null person_id:4 │ │ │ ├── key: (1) @@ -689,28 +689,28 @@ distinct-on │ │ │ │ ├── key: (1) │ │ │ │ └── fd: (1)-->(4) │ │ │ ├── select - │ │ │ │ ├── columns: call3_.duration:17!null call3_.phone_id:19 + │ │ │ │ ├── columns: call3_.duration:20!null call3_.phone_id:22 │ │ │ │ ├── scan call3_ - │ │ │ │ │ └── columns: call3_.duration:17!null call3_.phone_id:19 + │ │ │ │ │ └── columns: call3_.duration:20!null call3_.phone_id:22 │ │ │ │ └── filters - │ │ │ │ └── (call3_.duration:17 >= 50) IS NOT false [outer=(17)] + │ │ │ │ └── (call3_.duration:20 >= 50) IS NOT false [outer=(20)] │ │ │ └── filters - │ │ │ └── call3_.phone_id:19 = phone0_.id:1 [outer=(1,19), constraints=(/1: (/NULL - ]; /19: (/NULL - ]), fd=(1)==(19), (19)==(1)] + │ │ │ └── call3_.phone_id:22 = phone0_.id:1 [outer=(1,22), constraints=(/1: (/NULL - ]; /22: (/NULL - ]), fd=(1)==(22), (22)==(1)] │ │ └── filters - │ │ └── person_id:4 = person2_.id:10 [outer=(4,10), constraints=(/4: (/NULL - ]; /10: (/NULL - ]), fd=(4)==(10), (10)==(4)] + │ │ └── person_id:4 = person2_.id:12 [outer=(4,12), constraints=(/4: (/NULL - ]; /12: (/NULL - ]), fd=(4)==(12), (12)==(4)] │ └── filters - │ └── phone0_.id:1 = calls1_.phone_id:9 [outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ]), fd=(1)==(9), (9)==(1)] + │ └── phone0_.id:1 = calls1_.phone_id:10 [outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] └── aggregations - ├── const-agg [as=address:11, outer=(11)] - │ └── address:11 - ├── const-agg [as=createdon:12, outer=(12)] - │ └── createdon:12 - ├── const-agg [as=name:13, outer=(13)] - │ └── name:13 - ├── const-agg [as=nickname:14, outer=(14)] - │ └── nickname:14 - └── const-agg [as=version:15, outer=(15)] - └── version:15 + ├── const-agg [as=address:13, outer=(13)] + │ └── address:13 + ├── const-agg [as=createdon:14, outer=(14)] + │ └── createdon:14 + ├── const-agg [as=name:15, outer=(15)] + │ └── name:15 + ├── const-agg [as=nickname:16, outer=(16)] + │ └── nickname:16 + └── const-agg [as=version:17, outer=(17)] + └── version:17 opt select @@ -736,33 +736,33 @@ project ├── key: (1) ├── fd: (1)-->(2-4) └── select - ├── columns: phone0_.id:1!null phone_number:2 phone_type:3 person_id:4 max:10!null + ├── columns: phone0_.id:1!null phone_number:2 phone_type:3 person_id:4 max:12!null ├── has-placeholder ├── key: (1) - ├── fd: (1)-->(2-4,10) + ├── fd: (1)-->(2-4,12) ├── group-by - │ ├── columns: phone0_.id:1!null phone_number:2 phone_type:3 person_id:4 max:10!null + │ ├── columns: phone0_.id:1!null phone_number:2 phone_type:3 person_id:4 max:12!null │ ├── grouping columns: phone0_.id:1!null │ ├── key: (1) - │ ├── fd: (1)-->(2-4,10) + │ ├── fd: (1)-->(2-4,12) │ ├── inner-join (hash) - │ │ ├── columns: phone0_.id:1!null phone_number:2 phone_type:3 person_id:4 calls1_.id:6!null phone_id:9!null + │ │ ├── columns: phone0_.id:1!null phone_number:2 phone_type:3 person_id:4 calls1_.id:7!null phone_id:10!null │ │ ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) - │ │ ├── key: (6) - │ │ ├── fd: (1)-->(2-4), (6)-->(9), (1)==(9), (9)==(1) + │ │ ├── key: (7) + │ │ ├── fd: (1)-->(2-4), (7)-->(10), (1)==(10), (10)==(1) │ │ ├── scan phone0_ │ │ │ ├── columns: phone0_.id:1!null phone_number:2 phone_type:3 person_id:4 │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2-4) │ │ ├── scan calls1_ - │ │ │ ├── columns: calls1_.id:6!null phone_id:9 - │ │ │ ├── key: (6) - │ │ │ └── fd: (6)-->(9) + │ │ │ ├── columns: calls1_.id:7!null phone_id:10 + │ │ │ ├── key: (7) + │ │ │ └── fd: (7)-->(10) │ │ └── filters - │ │ └── phone0_.id:1 = phone_id:9 [outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ]), fd=(1)==(9), (9)==(1)] + │ │ └── phone0_.id:1 = phone_id:10 [outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] │ └── aggregations - │ ├── max [as=max:10, outer=(6)] - │ │ └── calls1_.id:6 + │ ├── max [as=max:12, outer=(7)] + │ │ └── calls1_.id:7 │ ├── const-agg [as=phone_number:2, outer=(2)] │ │ └── phone_number:2 │ ├── const-agg [as=phone_type:3, outer=(3)] @@ -770,7 +770,7 @@ project │ └── const-agg [as=person_id:4, outer=(4)] │ └── person_id:4 └── filters - └── max:10 = $1 [outer=(10), constraints=(/10: (/NULL - ])] + └── max:12 = $1 [outer=(12), constraints=(/12: (/NULL - ])] opt select @@ -797,16 +797,16 @@ project ├── key: (1) ├── fd: (1)-->(2-6) └── select - ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null count:12!null + ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null count:14!null ├── key: (1) - ├── fd: ()-->(12), (1)-->(2-6) + ├── fd: ()-->(14), (1)-->(2-6) ├── group-by - │ ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null count:12!null + │ ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null count:14!null │ ├── grouping columns: person0_.id:1!null │ ├── key: (1) - │ ├── fd: (1)-->(2-6,12) + │ ├── fd: (1)-->(2-6,14) │ ├── left-join (hash) - │ │ ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null person_id:10 + │ │ ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null person_id:11 │ │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) │ │ ├── fd: (1)-->(2-6) │ │ ├── scan person0_ @@ -814,12 +814,12 @@ project │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2-6) │ │ ├── scan phones1_ - │ │ │ └── columns: person_id:10 + │ │ │ └── columns: person_id:11 │ │ └── filters - │ │ └── person0_.id:1 = person_id:10 [outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] + │ │ └── person0_.id:1 = person_id:11 [outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ]), fd=(1)==(11), (11)==(1)] │ └── aggregations - │ ├── count [as=count:12, outer=(10)] - │ │ └── person_id:10 + │ ├── count [as=count:14, outer=(11)] + │ │ └── person_id:11 │ ├── const-agg [as=address:2, outer=(2)] │ │ └── address:2 │ ├── const-agg [as=createdon:3, outer=(3)] @@ -831,7 +831,7 @@ project │ └── const-agg [as=version:6, outer=(6)] │ └── version:6 └── filters - └── count:12 = 2 [outer=(12), constraints=(/12: [/2 - /2]; tight), fd=()-->(12)] + └── count:14 = 2 [outer=(14), constraints=(/14: [/2 - /2]; tight), fd=()-->(14)] opt select @@ -857,33 +857,33 @@ project ├── key: (1) ├── fd: (1)-->(2-4) └── select - ├── columns: phone0_.id:1!null phone_number:2 phone_type:3 person_id:4 min:10!null + ├── columns: phone0_.id:1!null phone_number:2 phone_type:3 person_id:4 min:12!null ├── has-placeholder ├── key: (1) - ├── fd: (1)-->(2-4,10) + ├── fd: (1)-->(2-4,12) ├── group-by - │ ├── columns: phone0_.id:1!null phone_number:2 phone_type:3 person_id:4 min:10!null + │ ├── columns: phone0_.id:1!null phone_number:2 phone_type:3 person_id:4 min:12!null │ ├── grouping columns: phone0_.id:1!null │ ├── key: (1) - │ ├── fd: (1)-->(2-4,10) + │ ├── fd: (1)-->(2-4,12) │ ├── inner-join (hash) - │ │ ├── columns: phone0_.id:1!null phone_number:2 phone_type:3 person_id:4 calls1_.id:6!null phone_id:9!null + │ │ ├── columns: phone0_.id:1!null phone_number:2 phone_type:3 person_id:4 calls1_.id:7!null phone_id:10!null │ │ ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) - │ │ ├── key: (6) - │ │ ├── fd: (1)-->(2-4), (6)-->(9), (1)==(9), (9)==(1) + │ │ ├── key: (7) + │ │ ├── fd: (1)-->(2-4), (7)-->(10), (1)==(10), (10)==(1) │ │ ├── scan phone0_ │ │ │ ├── columns: phone0_.id:1!null phone_number:2 phone_type:3 person_id:4 │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2-4) │ │ ├── scan calls1_ - │ │ │ ├── columns: calls1_.id:6!null phone_id:9 - │ │ │ ├── key: (6) - │ │ │ └── fd: (6)-->(9) + │ │ │ ├── columns: calls1_.id:7!null phone_id:10 + │ │ │ ├── key: (7) + │ │ │ └── fd: (7)-->(10) │ │ └── filters - │ │ └── phone0_.id:1 = phone_id:9 [outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ]), fd=(1)==(9), (9)==(1)] + │ │ └── phone0_.id:1 = phone_id:10 [outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] │ └── aggregations - │ ├── min [as=min:10, outer=(6)] - │ │ └── calls1_.id:6 + │ ├── min [as=min:12, outer=(7)] + │ │ └── calls1_.id:7 │ ├── const-agg [as=phone_number:2, outer=(2)] │ │ └── phone_number:2 │ ├── const-agg [as=phone_type:3, outer=(3)] @@ -891,7 +891,7 @@ project │ └── const-agg [as=person_id:4, outer=(4)] │ └── person_id:4 └── filters - └── min:10 = $1 [outer=(10), constraints=(/10: (/NULL - ])] + └── min:12 = $1 [outer=(12), constraints=(/12: (/NULL - ])] opt select @@ -918,33 +918,33 @@ project ├── key: (1) ├── fd: (1)-->(2-6) └── select - ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null max:12!null + ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null max:14!null ├── key: (1) - ├── fd: ()-->(12), (1)-->(2-6) + ├── fd: ()-->(14), (1)-->(2-6) ├── group-by - │ ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null max:12!null + │ ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null max:14!null │ ├── grouping columns: person0_.id:1!null │ ├── key: (1) - │ ├── fd: (1)-->(2-6,12) + │ ├── fd: (1)-->(2-6,14) │ ├── inner-join (hash) - │ │ ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null person_id:10!null order_id:11!null + │ │ ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null person_id:11!null order_id:12!null │ │ ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) - │ │ ├── fd: (1)-->(2-6), (1)==(10), (10)==(1) + │ │ ├── fd: (1)-->(2-6), (1)==(11), (11)==(1) │ │ ├── scan person0_ │ │ │ ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2-6) │ │ ├── select - │ │ │ ├── columns: person_id:10 order_id:11!null + │ │ │ ├── columns: person_id:11 order_id:12!null │ │ │ ├── scan phones1_ - │ │ │ │ └── columns: person_id:10 order_id:11 + │ │ │ │ └── columns: person_id:11 order_id:12 │ │ │ └── filters - │ │ │ └── order_id:11 IS NOT NULL [outer=(11), constraints=(/11: (/NULL - ]; tight)] + │ │ │ └── order_id:12 IS NOT NULL [outer=(12), constraints=(/12: (/NULL - ]; tight)] │ │ └── filters - │ │ └── person0_.id:1 = person_id:10 [outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] + │ │ └── person0_.id:1 = person_id:11 [outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ]), fd=(1)==(11), (11)==(1)] │ └── aggregations - │ ├── max [as=max:12, outer=(11)] - │ │ └── order_id:11 + │ ├── max [as=max:14, outer=(12)] + │ │ └── order_id:12 │ ├── const-agg [as=address:2, outer=(2)] │ │ └── address:2 │ ├── const-agg [as=createdon:3, outer=(3)] @@ -956,7 +956,7 @@ project │ └── const-agg [as=version:6, outer=(6)] │ └── version:6 └── filters - └── max:12 = 0 [outer=(12), constraints=(/12: [/0 - /0]; tight), fd=()-->(12)] + └── max:14 = 0 [outer=(14), constraints=(/14: [/0 - /0]; tight), fd=()-->(14)] opt select @@ -984,28 +984,28 @@ project ├── key: (1) ├── fd: (1)-->(2-6) └── inner-join (lookup person) - ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null person_id:10!null - ├── key columns: [10] = [1] + ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null person_id:11!null + ├── key columns: [11] = [1] ├── lookup columns are key ├── immutable, has-placeholder - ├── key: (10) - ├── fd: (1)-->(2-6), (1)==(10), (10)==(1) + ├── key: (11) + ├── fd: (1)-->(2-6), (1)==(11), (11)==(1) ├── distinct-on - │ ├── columns: person_id:10 - │ ├── grouping columns: person_id:10 + │ ├── columns: person_id:11 + │ ├── grouping columns: person_id:11 │ ├── immutable, has-placeholder - │ ├── key: (10) + │ ├── key: (11) │ └── select - │ ├── columns: phones1_.id:7!null person_id:10 + │ ├── columns: phones1_.id:8!null person_id:11 │ ├── immutable, has-placeholder - │ ├── key: (7) - │ ├── fd: (7)-->(10) + │ ├── key: (8) + │ ├── fd: (8)-->(11) │ ├── scan phones1_ - │ │ ├── columns: phones1_.id:7!null person_id:10 - │ │ ├── key: (7) - │ │ └── fd: (7)-->(10) + │ │ ├── columns: phones1_.id:8!null person_id:11 + │ │ ├── key: (8) + │ │ └── fd: (8)-->(11) │ └── filters - │ └── phones1_.id:7 = $1::INT8 [outer=(7), immutable, constraints=(/7: (/NULL - ])] + │ └── phones1_.id:8 = $1::INT8 [outer=(8), immutable, constraints=(/8: (/NULL - ])] └── filters (true) opt @@ -1034,28 +1034,28 @@ project ├── key: (1) ├── fd: (1)-->(2-6) └── inner-join (lookup person) - ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null person_id:10!null - ├── key columns: [10] = [1] + ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null person_id:11!null + ├── key columns: [11] = [1] ├── lookup columns are key ├── immutable, has-placeholder - ├── key: (10) - ├── fd: (1)-->(2-6), (1)==(10), (10)==(1) + ├── key: (11) + ├── fd: (1)-->(2-6), (1)==(11), (11)==(1) ├── distinct-on - │ ├── columns: person_id:10 - │ ├── grouping columns: person_id:10 + │ ├── columns: person_id:11 + │ ├── grouping columns: person_id:11 │ ├── immutable, has-placeholder - │ ├── key: (10) + │ ├── key: (11) │ └── select - │ ├── columns: phones1_.id:7!null person_id:10 + │ ├── columns: phones1_.id:8!null person_id:11 │ ├── immutable, has-placeholder - │ ├── key: (7) - │ ├── fd: (7)-->(10) + │ ├── key: (8) + │ ├── fd: (8)-->(11) │ ├── scan phones1_ - │ │ ├── columns: phones1_.id:7!null person_id:10 - │ │ ├── key: (7) - │ │ └── fd: (7)-->(10) + │ │ ├── columns: phones1_.id:8!null person_id:11 + │ │ ├── key: (8) + │ │ └── fd: (8)-->(11) │ └── filters - │ └── phones1_.id:7 = $1::INT8 [outer=(7), immutable, constraints=(/7: (/NULL - ])] + │ └── phones1_.id:8 = $1::INT8 [outer=(8), immutable, constraints=(/8: (/NULL - ])] └── filters (true) opt @@ -1083,17 +1083,17 @@ project ├── key: (1) ├── fd: (1)-->(2-6) └── inner-join (lookup person) - ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null person_id:10!null - ├── key columns: [10] = [1] + ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null person_id:11!null + ├── key columns: [11] = [1] ├── lookup columns are key - ├── key: (10) - ├── fd: (1)-->(2-6), (1)==(10), (10)==(1) + ├── key: (11) + ├── fd: (1)-->(2-6), (1)==(11), (11)==(1) ├── distinct-on - │ ├── columns: person_id:10 - │ ├── grouping columns: person_id:10 - │ ├── key: (10) + │ ├── columns: person_id:11 + │ ├── grouping columns: person_id:11 + │ ├── key: (11) │ └── scan phones1_ - │ └── columns: person_id:10 + │ └── columns: person_id:11 └── filters (true) opt @@ -1124,14 +1124,14 @@ anti-join (hash) │ ├── key: (1) │ └── fd: (1)-->(2-4) ├── select - │ ├── columns: phone_id:6!null repairtimestamps:7 + │ ├── columns: phone_id:7!null repairtimestamps:8 │ ├── stable, has-placeholder │ ├── scan repairtime1_ - │ │ └── columns: phone_id:6!null repairtimestamps:7 + │ │ └── columns: phone_id:7!null repairtimestamps:8 │ └── filters - │ └── (repairtimestamps:7 >= $1::DATE) IS NOT false [outer=(7), stable] + │ └── (repairtimestamps:8 >= $1::DATE) IS NOT false [outer=(8), stable] └── filters - └── id:1 = phone_id:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── id:1 = phone_id:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] opt select @@ -1158,22 +1158,22 @@ project ├── key: (1) ├── fd: (1)-->(2-6) └── inner-join (lookup person) - ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null person_id:10!null - ├── key columns: [10] = [1] + ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null person_id:11!null + ├── key columns: [11] = [1] ├── lookup columns are key - ├── key: (10) - ├── fd: (1)-->(2-6), (1)==(10), (10)==(1) + ├── key: (11) + ├── fd: (1)-->(2-6), (1)==(11), (11)==(1) ├── distinct-on - │ ├── columns: person_id:10 - │ ├── grouping columns: person_id:10 - │ ├── key: (10) + │ ├── columns: person_id:11 + │ ├── grouping columns: person_id:11 + │ ├── key: (11) │ └── select - │ ├── columns: person_id:10 order_id:11!null - │ ├── fd: ()-->(11) + │ ├── columns: person_id:11 order_id:12!null + │ ├── fd: ()-->(12) │ ├── scan phones1_ - │ │ └── columns: person_id:10 order_id:11 + │ │ └── columns: person_id:11 order_id:12 │ └── filters - │ └── order_id:11 = 1 [outer=(11), constraints=(/11: [/1 - /1]; tight), fd=()-->(11)] + │ └── order_id:12 = 1 [outer=(12), constraints=(/12: [/1 - /1]; tight), fd=()-->(12)] └── filters (true) opt @@ -1204,47 +1204,47 @@ project ├── columns: id1_2_:1!null address2_2_:2 createdo3_2_:3 name4_2_:4 nickname5_2_:5 version6_2_:6!null ├── fd: (1)-->(2-6) └── select - ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null phones2_.id:7!null phones2_.order_id:11!null max:17!null - ├── key: (7) - ├── fd: (1)-->(2-6), (7)-->(1-6,11,17), (11)==(17), (17)==(11) + ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null phones2_.id:8!null phones2_.order_id:12!null max:20!null + ├── key: (8) + ├── fd: (1)-->(2-6), (8)-->(1-6,12,20), (12)==(20), (20)==(12) ├── group-by - │ ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null phones2_.id:7!null phones2_.order_id:11 max:17!null - │ ├── grouping columns: phones2_.id:7!null - │ ├── key: (7) - │ ├── fd: (1)-->(2-6), (7)-->(1-6,11,17) + │ ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null phones2_.id:8!null phones2_.order_id:12 max:20!null + │ ├── grouping columns: phones2_.id:8!null + │ ├── key: (8) + │ ├── fd: (1)-->(2-6), (8)-->(1-6,12,20) │ ├── inner-join (hash) - │ │ ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null phones2_.id:7!null phones2_.phone_type:9!null phones2_.person_id:10!null phones2_.order_id:11 phones1_.person_id:15!null phones1_.order_id:16!null - │ │ ├── fd: ()-->(9), (1)-->(2-6), (7)-->(10,11), (1)==(10,15), (10)==(1,15), (15)==(1,10) + │ │ ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null phones2_.id:8!null phones2_.phone_type:10!null phones2_.person_id:11!null phones2_.order_id:12 phones1_.person_id:17!null phones1_.order_id:18!null + │ │ ├── fd: ()-->(10), (1)-->(2-6), (8)-->(11,12), (1)==(11,17), (11)==(1,17), (17)==(1,11) │ │ ├── select - │ │ │ ├── columns: phones1_.person_id:15 phones1_.order_id:16!null + │ │ │ ├── columns: phones1_.person_id:17 phones1_.order_id:18!null │ │ │ ├── scan phones1_ - │ │ │ │ └── columns: phones1_.person_id:15 phones1_.order_id:16 + │ │ │ │ └── columns: phones1_.person_id:17 phones1_.order_id:18 │ │ │ └── filters - │ │ │ └── phones1_.order_id:16 IS NOT NULL [outer=(16), constraints=(/16: (/NULL - ]; tight)] + │ │ │ └── phones1_.order_id:18 IS NOT NULL [outer=(18), constraints=(/18: (/NULL - ]; tight)] │ │ ├── inner-join (lookup person) - │ │ │ ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null phones2_.id:7!null phones2_.phone_type:9!null phones2_.person_id:10!null phones2_.order_id:11 - │ │ │ ├── key columns: [10] = [1] + │ │ │ ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null phones2_.id:8!null phones2_.phone_type:10!null phones2_.person_id:11!null phones2_.order_id:12 + │ │ │ ├── key columns: [11] = [1] │ │ │ ├── lookup columns are key - │ │ │ ├── key: (7) - │ │ │ ├── fd: ()-->(9), (1)-->(2-6), (7)-->(10,11), (1)==(10), (10)==(1) + │ │ │ ├── key: (8) + │ │ │ ├── fd: ()-->(10), (1)-->(2-6), (8)-->(11,12), (1)==(11), (11)==(1) │ │ │ ├── select - │ │ │ │ ├── columns: phones2_.id:7!null phones2_.phone_type:9!null phones2_.person_id:10 phones2_.order_id:11 - │ │ │ │ ├── key: (7) - │ │ │ │ ├── fd: ()-->(9), (7)-->(10,11) + │ │ │ │ ├── columns: phones2_.id:8!null phones2_.phone_type:10!null phones2_.person_id:11 phones2_.order_id:12 + │ │ │ │ ├── key: (8) + │ │ │ │ ├── fd: ()-->(10), (8)-->(11,12) │ │ │ │ ├── scan phones2_ - │ │ │ │ │ ├── columns: phones2_.id:7!null phones2_.phone_type:9 phones2_.person_id:10 phones2_.order_id:11 - │ │ │ │ │ ├── key: (7) - │ │ │ │ │ └── fd: (7)-->(9-11) + │ │ │ │ │ ├── columns: phones2_.id:8!null phones2_.phone_type:10 phones2_.person_id:11 phones2_.order_id:12 + │ │ │ │ │ ├── key: (8) + │ │ │ │ │ └── fd: (8)-->(10-12) │ │ │ │ └── filters - │ │ │ │ └── phones2_.phone_type:9 = 'LAND_LINE' [outer=(9), constraints=(/9: [/'LAND_LINE' - /'LAND_LINE']; tight), fd=()-->(9)] + │ │ │ │ └── phones2_.phone_type:10 = 'LAND_LINE' [outer=(10), constraints=(/10: [/'LAND_LINE' - /'LAND_LINE']; tight), fd=()-->(10)] │ │ │ └── filters (true) │ │ └── filters - │ │ └── person0_.id:1 = phones1_.person_id:15 [outer=(1,15), constraints=(/1: (/NULL - ]; /15: (/NULL - ]), fd=(1)==(15), (15)==(1)] + │ │ └── person0_.id:1 = phones1_.person_id:17 [outer=(1,17), constraints=(/1: (/NULL - ]; /17: (/NULL - ]), fd=(1)==(17), (17)==(1)] │ └── aggregations - │ ├── max [as=max:17, outer=(16)] - │ │ └── phones1_.order_id:16 - │ ├── const-agg [as=phones2_.order_id:11, outer=(11)] - │ │ └── phones2_.order_id:11 + │ ├── max [as=max:20, outer=(18)] + │ │ └── phones1_.order_id:18 + │ ├── const-agg [as=phones2_.order_id:12, outer=(12)] + │ │ └── phones2_.order_id:12 │ ├── const-agg [as=address:2, outer=(2)] │ │ └── address:2 │ ├── const-agg [as=createdon:3, outer=(3)] @@ -1258,7 +1258,7 @@ project │ └── const-agg [as=person0_.id:1, outer=(1)] │ └── person0_.id:1 └── filters - └── phones2_.order_id:11 = max:17 [outer=(11,17), constraints=(/11: (/NULL - ]; /17: (/NULL - ]), fd=(11)==(17), (17)==(11)] + └── phones2_.order_id:12 = max:20 [outer=(12,20), constraints=(/12: (/NULL - ]; /20: (/NULL - ]), fd=(12)==(20), (20)==(12)] opt select @@ -1287,9 +1287,9 @@ anti-join (hash) │ ├── key: (1) │ └── fd: (1)-->(2-6) ├── scan phones1_ - │ └── columns: person_id:10 + │ └── columns: person_id:11 └── filters - └── person0_.id:1 = person_id:10 [outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] + └── person0_.id:1 = person_id:11 [outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ]), fd=(1)==(11), (11)==(1)] opt select @@ -1316,17 +1316,17 @@ project ├── key: (1) ├── fd: (1)-->(2-6) └── inner-join (lookup person) - ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null person_id:10!null - ├── key columns: [10] = [1] + ├── columns: person0_.id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null person_id:11!null + ├── key columns: [11] = [1] ├── lookup columns are key - ├── key: (10) - ├── fd: (1)-->(2-6), (1)==(10), (10)==(1) + ├── key: (11) + ├── fd: (1)-->(2-6), (1)==(11), (11)==(1) ├── distinct-on - │ ├── columns: person_id:10 - │ ├── grouping columns: person_id:10 - │ ├── key: (10) + │ ├── columns: person_id:11 + │ ├── grouping columns: person_id:11 + │ ├── key: (11) │ └── scan phones1_ - │ └── columns: person_id:10 + │ └── columns: person_id:11 └── filters (true) opt @@ -1354,9 +1354,9 @@ anti-join (hash) │ ├── key: (1) │ └── fd: (1)-->(2-4) ├── scan calls1_ - │ └── columns: phone_id:9 + │ └── columns: phone_id:10 └── filters - └── phone0_.id:1 = phone_id:9 [outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ]), fd=(1)==(9), (9)==(1)] + └── phone0_.id:1 = phone_id:10 [outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] opt select @@ -1383,25 +1383,25 @@ project ├── key: (1) ├── fd: (1)-->(2-6) └── inner-join (lookup person) - ├── columns: id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null person_id:7!null - ├── key columns: [7] = [1] + ├── columns: id:1!null address:2 createdon:3 name:4 nickname:5 version:6!null person_id:8!null + ├── key columns: [8] = [1] ├── lookup columns are key - ├── key: (7) - ├── fd: (1)-->(2-6), (1)==(7), (7)==(1) + ├── key: (8) + ├── fd: (1)-->(2-6), (1)==(8), (8)==(1) ├── distinct-on - │ ├── columns: person_id:7!null - │ ├── grouping columns: person_id:7!null - │ ├── internal-ordering: +7 opt(8) - │ ├── key: (7) + │ ├── columns: person_id:8!null + │ ├── grouping columns: person_id:8!null + │ ├── internal-ordering: +8 opt(9) + │ ├── key: (8) │ └── select - │ ├── columns: person_id:7!null addresses:8!null - │ ├── fd: ()-->(8) - │ ├── ordering: +7 opt(8) [actual: +7] + │ ├── columns: person_id:8!null addresses:9!null + │ ├── fd: ()-->(9) + │ ├── ordering: +8 opt(9) [actual: +8] │ ├── scan addresses1_ - │ │ ├── columns: person_id:7!null addresses:8 - │ │ └── ordering: +7 opt(8) [actual: +7] + │ │ ├── columns: person_id:8!null addresses:9 + │ │ └── ordering: +8 opt(9) [actual: +8] │ └── filters - │ └── addresses:8 = 'Home address' [outer=(8), constraints=(/8: [/'Home address' - /'Home address']; tight), fd=()-->(8)] + │ └── addresses:9 = 'Home address' [outer=(9), constraints=(/9: [/'Home address' - /'Home address']; tight), fd=()-->(9)] └── filters (true) opt @@ -1427,7 +1427,7 @@ where anti-join (merge) ├── columns: id1_2_:1!null address2_2_:2 createdo3_2_:3 name4_2_:4 nickname5_2_:5 version6_2_:6!null ├── left ordering: +1 - ├── right ordering: +7 + ├── right ordering: +8 ├── key: (1) ├── fd: (1)-->(2-6) ├── scan person0_ @@ -1436,13 +1436,13 @@ anti-join (merge) │ ├── fd: (1)-->(2-6) │ └── ordering: +1 ├── select - │ ├── columns: person_id:7!null addresses:8 - │ ├── ordering: +7 + │ ├── columns: person_id:8!null addresses:9 + │ ├── ordering: +8 │ ├── scan addresses1_ - │ │ ├── columns: person_id:7!null addresses:8 - │ │ └── ordering: +7 + │ │ ├── columns: person_id:8!null addresses:9 + │ │ └── ordering: +8 │ └── filters - │ └── (addresses:8 = 'Home address') IS NOT false [outer=(8)] + │ └── (addresses:9 = 'Home address') IS NOT false [outer=(9)] └── filters (true) exec-ddl @@ -1490,16 +1490,16 @@ project ├── key: (1) ├── fd: (1)-->(2,3) └── select - ├── columns: id:1!null email:2 currentproject_id:3 count:7!null + ├── columns: id:1!null email:2 currentproject_id:3 count:9!null ├── key: (1) - ├── fd: ()-->(7), (1)-->(2,3) + ├── fd: ()-->(9), (1)-->(2,3) ├── group-by - │ ├── columns: id:1!null email:2 currentproject_id:3 count:7!null + │ ├── columns: id:1!null email:2 currentproject_id:3 count:9!null │ ├── grouping columns: id:1!null │ ├── key: (1) - │ ├── fd: (1)-->(2,3,7) + │ ├── fd: (1)-->(2,3,9) │ ├── left-join (hash) - │ │ ├── columns: id:1!null email:2 currentproject_id:3 employee_id:4 + │ │ ├── columns: id:1!null email:2 currentproject_id:3 employee_id:5 │ │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) │ │ ├── fd: (1)-->(2,3) │ │ ├── scan componenti0_ @@ -1507,18 +1507,18 @@ project │ │ │ ├── key: (1) │ │ │ └── fd: (1)-->(2,3) │ │ ├── scan phones1_ - │ │ │ └── columns: employee_id:4!null + │ │ │ └── columns: employee_id:5!null │ │ └── filters - │ │ └── id:1 = employee_id:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] + │ │ └── id:1 = employee_id:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] │ └── aggregations - │ ├── count [as=count:7, outer=(4)] - │ │ └── employee_id:4 + │ ├── count [as=count:9, outer=(5)] + │ │ └── employee_id:5 │ ├── const-agg [as=email:2, outer=(2)] │ │ └── email:2 │ └── const-agg [as=currentproject_id:3, outer=(3)] │ └── currentproject_id:3 └── filters - └── count:7 = 1 [outer=(7), constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] + └── count:9 = 1 [outer=(9), constraints=(/9: [/1 - /1]; tight), fd=()-->(9)] exec-ddl drop table EMPLOYEE, Employee_phones; @@ -1599,13 +1599,13 @@ where and employees1_.employees_id=employee2_.id)) ---- right-join (hash) - ├── columns: id1_0_0_:1!null id1_8_1_:3 location2_0_0_:2 address2_8_1_:4 zip3_8_1_:5 + ├── columns: id1_0_0_:1!null id1_8_1_:4 location2_0_0_:2 address2_8_1_:5 zip3_8_1_:6 ├── key: (1) - ├── fd: (1)-->(2-5), (3)-->(4,5) + ├── fd: (1)-->(2,4-6), (4)-->(5,6) ├── scan location3_ - │ ├── columns: location3_.id:3!null address:4 zip:5!null - │ ├── key: (3) - │ └── fd: (3)-->(4,5) + │ ├── columns: location3_.id:4!null address:5 zip:6!null + │ ├── key: (4) + │ └── fd: (4)-->(5,6) ├── anti-join (hash) │ ├── columns: company0_.id:1!null location_id:2 │ ├── key: (1) @@ -1615,27 +1615,27 @@ right-join (hash) │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ ├── inner-join (hash) - │ │ ├── columns: company_id:6!null employees_id:7!null id:12!null - │ │ ├── fd: (7)==(12), (12)==(7) + │ │ ├── columns: company_id:8!null employees_id:9!null id:17!null + │ │ ├── fd: (9)==(17), (17)==(9) │ │ ├── union-all - │ │ │ ├── columns: id:12!null - │ │ │ ├── left columns: employee.id:8 - │ │ │ ├── right columns: manager.id:10 + │ │ │ ├── columns: id:17!null + │ │ │ ├── left columns: employee.id:11 + │ │ │ ├── right columns: manager.id:14 │ │ │ ├── scan employee - │ │ │ │ ├── columns: employee.id:8!null - │ │ │ │ └── key: (8) + │ │ │ │ ├── columns: employee.id:11!null + │ │ │ │ └── key: (11) │ │ │ └── scan manager - │ │ │ ├── columns: manager.id:10!null - │ │ │ └── key: (10) + │ │ │ ├── columns: manager.id:14!null + │ │ │ └── key: (14) │ │ ├── scan employees1_ - │ │ │ ├── columns: company_id:6!null employees_id:7!null - │ │ │ └── key: (6,7) + │ │ │ ├── columns: company_id:8!null employees_id:9!null + │ │ │ └── key: (8,9) │ │ └── filters - │ │ └── employees_id:7 = id:12 [outer=(7,12), constraints=(/7: (/NULL - ]; /12: (/NULL - ]), fd=(7)==(12), (12)==(7)] + │ │ └── employees_id:9 = id:17 [outer=(9,17), constraints=(/9: (/NULL - ]; /17: (/NULL - ]), fd=(9)==(17), (17)==(9)] │ └── filters - │ └── company0_.id:1 = company_id:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ └── company0_.id:1 = company_id:8 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] └── filters - └── location_id:2 = location3_.id:3 [outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ]), fd=(2)==(3), (3)==(2)] + └── location_id:2 = location3_.id:4 [outer=(2,4), constraints=(/2: (/NULL - ]; /4: (/NULL - ]), fd=(2)==(4), (4)==(2)] exec-ddl drop table Company, Company_Employee, Employee, Manager, Location; @@ -1693,21 +1693,21 @@ where news0_.Newspaper_id=1 ---- project - ├── columns: newspape1_23_0_:1!null news_new2_23_0_:2!null formula140_0_:9 news_id1_21_1_:3!null detail2_21_1_:4 title3_21_1_:5 - ├── key: (3) - ├── fd: ()-->(1), (3)-->(4,5,9), (2)==(3), (3)==(2) + ├── columns: newspape1_23_0_:1!null news_new2_23_0_:2!null formula140_0_:12 news_id1_21_1_:4!null detail2_21_1_:5 title3_21_1_:6 + ├── key: (4) + ├── fd: ()-->(1), (4)-->(5,6,12), (2)==(4), (4)==(2) ├── left-join (lookup news) - │ ├── columns: newspaper_id:1!null news_news_id:2!null news1_.news_id:3!null news1_.detail:4 news1_.title:5 a0.news_id:6 a0.title:8 - │ ├── key columns: [2] = [6] + │ ├── columns: newspaper_id:1!null news_news_id:2!null news1_.news_id:4!null news1_.detail:5 news1_.title:6 a0.news_id:8 a0.title:10 + │ ├── key columns: [2] = [8] │ ├── lookup columns are key - │ ├── key: (3) - │ ├── fd: ()-->(1), (3)-->(4-6,8), (2)==(3), (3)==(2), (6)-->(8) + │ ├── key: (4) + │ ├── fd: ()-->(1), (4)-->(5,6,8,10), (2)==(4), (4)==(2), (8)-->(10) │ ├── inner-join (lookup news) - │ │ ├── columns: newspaper_id:1!null news_news_id:2!null news1_.news_id:3!null news1_.detail:4 news1_.title:5 - │ │ ├── key columns: [2] = [3] + │ │ ├── columns: newspaper_id:1!null news_news_id:2!null news1_.news_id:4!null news1_.detail:5 news1_.title:6 + │ │ ├── key columns: [2] = [4] │ │ ├── lookup columns are key - │ │ ├── key: (3) - │ │ ├── fd: ()-->(1), (3)-->(4,5), (2)==(3), (3)==(2) + │ │ ├── key: (4) + │ │ ├── fd: ()-->(1), (4)-->(5,6), (2)==(4), (4)==(2) │ │ ├── scan news0_ │ │ │ ├── columns: newspaper_id:1!null news_news_id:2!null │ │ │ ├── constraint: /1/2: [/1 - /1] @@ -1716,7 +1716,7 @@ project │ │ └── filters (true) │ └── filters (true) └── projections - └── a0.title:8 [as=formula140_0_:9, outer=(8)] + └── a0.title:10 [as=formula140_0_:12, outer=(10)] exec-ddl drop table News, Newspaper, Newspaper_News; @@ -1773,33 +1773,33 @@ INNER JOIN generationgroup AS generation1_ WHERE ref0_.generationuser_id = 1; ---- project - ├── columns: generati1_2_0_:1!null ref_id2_2_0_:2!null formula131_0_:19 formula132_0_:20 formula133_0_:21 id1_0_1_:3!null age2_0_1_:4 culture3_0_1_:5 descript4_0_1_:6 - ├── key: (3) - ├── fd: ()-->(1), (3)-->(4-6,19-21), (2)==(3), (3)==(2) + ├── columns: generati1_2_0_:1!null ref_id2_2_0_:2!null formula131_0_:24 formula132_0_:25 formula133_0_:26 id1_0_1_:4!null age2_0_1_:5 culture3_0_1_:6 descript4_0_1_:7 + ├── key: (4) + ├── fd: ()-->(1), (4)-->(5-7,24-26), (2)==(4), (4)==(2) ├── left-join (lookup generationgroup) - │ ├── columns: generationuser_id:1!null ref_id:2!null generation1_.id:3!null generation1_.age:4 generation1_.culture:5 generation1_.description:6 a13.id:7 a13.age:8 a15.id:11 a15.culture:13 a13.id:15 a13.description:18 - │ ├── key columns: [2] = [15] + │ ├── columns: generationuser_id:1!null ref_id:2!null generation1_.id:4!null generation1_.age:5 generation1_.culture:6 generation1_.description:7 a13.id:9 a13.age:10 a15.id:14 a15.culture:16 a13.id:19 a13.description:22 + │ ├── key columns: [2] = [19] │ ├── lookup columns are key - │ ├── key: (3) - │ ├── fd: ()-->(1), (3)-->(4-8,11,13,15,18), (2)==(3), (3)==(2), (7)-->(8), (11)-->(13), (15)-->(18) + │ ├── key: (4) + │ ├── fd: ()-->(1), (4)-->(5-7,9,10,14,16,19,22), (2)==(4), (4)==(2), (9)-->(10), (14)-->(16), (19)-->(22) │ ├── left-join (lookup generationgroup) - │ │ ├── columns: generationuser_id:1!null ref_id:2!null generation1_.id:3!null generation1_.age:4 generation1_.culture:5 generation1_.description:6 a13.id:7 a13.age:8 a15.id:11 a15.culture:13 - │ │ ├── key columns: [2] = [11] + │ │ ├── columns: generationuser_id:1!null ref_id:2!null generation1_.id:4!null generation1_.age:5 generation1_.culture:6 generation1_.description:7 a13.id:9 a13.age:10 a15.id:14 a15.culture:16 + │ │ ├── key columns: [2] = [14] │ │ ├── lookup columns are key - │ │ ├── key: (3) - │ │ ├── fd: ()-->(1), (3)-->(4-8,11,13), (2)==(3), (3)==(2), (7)-->(8), (11)-->(13) + │ │ ├── key: (4) + │ │ ├── fd: ()-->(1), (4)-->(5-7,9,10,14,16), (2)==(4), (4)==(2), (9)-->(10), (14)-->(16) │ │ ├── left-join (lookup generationgroup) - │ │ │ ├── columns: generationuser_id:1!null ref_id:2!null generation1_.id:3!null generation1_.age:4 generation1_.culture:5 generation1_.description:6 a13.id:7 a13.age:8 - │ │ │ ├── key columns: [2] = [7] + │ │ │ ├── columns: generationuser_id:1!null ref_id:2!null generation1_.id:4!null generation1_.age:5 generation1_.culture:6 generation1_.description:7 a13.id:9 a13.age:10 + │ │ │ ├── key columns: [2] = [9] │ │ │ ├── lookup columns are key - │ │ │ ├── key: (3) - │ │ │ ├── fd: ()-->(1), (3)-->(4-8), (2)==(3), (3)==(2), (7)-->(8) + │ │ │ ├── key: (4) + │ │ │ ├── fd: ()-->(1), (4)-->(5-7,9,10), (2)==(4), (4)==(2), (9)-->(10) │ │ │ ├── inner-join (lookup generationgroup) - │ │ │ │ ├── columns: generationuser_id:1!null ref_id:2!null generation1_.id:3!null generation1_.age:4 generation1_.culture:5 generation1_.description:6 - │ │ │ │ ├── key columns: [2] = [3] + │ │ │ │ ├── columns: generationuser_id:1!null ref_id:2!null generation1_.id:4!null generation1_.age:5 generation1_.culture:6 generation1_.description:7 + │ │ │ │ ├── key columns: [2] = [4] │ │ │ │ ├── lookup columns are key - │ │ │ │ ├── key: (3) - │ │ │ │ ├── fd: ()-->(1), (3)-->(4-6), (2)==(3), (3)==(2) + │ │ │ │ ├── key: (4) + │ │ │ │ ├── fd: ()-->(1), (4)-->(5-7), (2)==(4), (4)==(2) │ │ │ │ ├── scan ref0_ │ │ │ │ │ ├── columns: generationuser_id:1!null ref_id:2!null │ │ │ │ │ ├── constraint: /1/2: [/1 - /1] @@ -1810,9 +1810,9 @@ project │ │ └── filters (true) │ └── filters (true) └── projections - ├── a13.age:8 [as=formula131_0_:19, outer=(8)] - ├── a15.culture:13 [as=formula132_0_:20, outer=(13)] - └── a13.description:18 [as=formula133_0_:21, outer=(18)] + ├── a13.age:10 [as=formula131_0_:24, outer=(10)] + ├── a15.culture:16 [as=formula132_0_:25, outer=(16)] + └── a13.description:22 [as=formula133_0_:26, outer=(22)] exec-ddl drop table GenerationGroup, GenerationUser, GenerationUser_GenerationGroup; @@ -1862,27 +1862,27 @@ where bids0_.auctionId=$1 ---- project - ├── columns: auctioni4_1_0_:4!null id1_1_0_:1!null id1_1_1_:1!null amount2_1_1_:2 createdd3_1_1_:3 auctioni4_1_1_:4!null formula41_1_:9!null + ├── columns: auctioni4_1_0_:4!null id1_1_0_:1!null id1_1_1_:1!null amount2_1_1_:2 createdd3_1_1_:3 auctioni4_1_1_:4!null formula41_1_:11!null ├── has-placeholder ├── key: (1) - ├── fd: (1)-->(2-4,9) + ├── fd: (1)-->(2-4,11) ├── group-by - │ ├── columns: bids0_.id:1!null amount:2 createddatetime:3 auctionid:4!null true_agg:11 + │ ├── columns: bids0_.id:1!null amount:2 createddatetime:3 auctionid:4!null true_agg:13 │ ├── grouping columns: bids0_.id:1!null │ ├── has-placeholder │ ├── key: (1) - │ ├── fd: (1)-->(2-4,11) + │ ├── fd: (1)-->(2-4,13) │ ├── right-join (hash) - │ │ ├── columns: bids0_.id:1!null amount:2 createddatetime:3 auctionid:4!null successfulbid:8 true:10 + │ │ ├── columns: bids0_.id:1!null amount:2 createddatetime:3 auctionid:4!null successfulbid:9 true:12 │ │ ├── has-placeholder │ │ ├── fd: (1)-->(2-4) │ │ ├── project - │ │ │ ├── columns: true:10!null successfulbid:8 - │ │ │ ├── fd: ()-->(10) + │ │ │ ├── columns: true:12!null successfulbid:9 + │ │ │ ├── fd: ()-->(12) │ │ │ ├── scan a - │ │ │ │ └── columns: successfulbid:8 + │ │ │ │ └── columns: successfulbid:9 │ │ │ └── projections - │ │ │ └── true [as=true:10] + │ │ │ └── true [as=true:12] │ │ ├── select │ │ │ ├── columns: bids0_.id:1!null amount:2 createddatetime:3 auctionid:4!null │ │ │ ├── has-placeholder @@ -1895,10 +1895,10 @@ project │ │ │ └── filters │ │ │ └── auctionid:4 = $1 [outer=(4), constraints=(/4: (/NULL - ])] │ │ └── filters - │ │ └── successfulbid:8 = bids0_.id:1 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] + │ │ └── successfulbid:9 = bids0_.id:1 [outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ]), fd=(1)==(9), (9)==(1)] │ └── aggregations - │ ├── const-not-null-agg [as=true_agg:11, outer=(10)] - │ │ └── true:10 + │ ├── const-not-null-agg [as=true_agg:13, outer=(12)] + │ │ └── true:12 │ ├── const-agg [as=amount:2, outer=(2)] │ │ └── amount:2 │ ├── const-agg [as=createddatetime:3, outer=(3)] @@ -1906,7 +1906,7 @@ project │ └── const-agg [as=auctionid:4, outer=(4)] │ └── auctionid:4 └── projections - └── true_agg:11 IS NOT NULL [as=formula41_1_:9, outer=(11)] + └── true_agg:13 IS NOT NULL [as=formula41_1_:11, outer=(13)] exec-ddl drop table TAuction2, TBid2; @@ -1987,46 +1987,46 @@ WHERE order0_.customerid = 'c111' AND order0_.ordernumber = 0; ---- project - ├── columns: customer1_1_0_:1!null ordernum2_1_0_:2!null orderdat3_1_0_:3!null formula101_0_:18 customer1_2_1_:4 ordernum2_2_1_:5 producti3_2_1_:6 customer1_2_2_:4 ordernum2_2_2_:5 producti3_2_2_:6 quantity4_2_2_:7 + ├── columns: customer1_1_0_:1!null ordernum2_1_0_:2!null orderdat3_1_0_:3!null formula101_0_:22 customer1_2_1_:5 ordernum2_2_1_:6 producti3_2_1_:7 customer1_2_2_:5 ordernum2_2_2_:6 producti3_2_2_:7 quantity4_2_2_:8 ├── immutable - ├── key: (6) - ├── fd: ()-->(1-3), (6)-->(4,5,7,18) + ├── key: (7) + ├── fd: ()-->(1-3), (7)-->(5,6,8,22) ├── group-by - │ ├── columns: order0_.customerid:1!null order0_.ordernumber:2!null orderdate:3!null lineitems1_.customerid:4 lineitems1_.ordernumber:5 lineitems1_.productid:6 lineitems1_.quantity:7 sum:17 - │ ├── grouping columns: lineitems1_.productid:6 + │ ├── columns: order0_.customerid:1!null order0_.ordernumber:2!null orderdate:3!null lineitems1_.customerid:5 lineitems1_.ordernumber:6 lineitems1_.productid:7 lineitems1_.quantity:8 sum:21 + │ ├── grouping columns: lineitems1_.productid:7 │ ├── immutable - │ ├── key: (6) - │ ├── fd: ()-->(1-3), (6)-->(1-5,7,17) + │ ├── key: (7) + │ ├── fd: ()-->(1-3), (7)-->(1-3,5,6,8,21) │ ├── right-join (hash) - │ │ ├── columns: order0_.customerid:1!null order0_.ordernumber:2!null orderdate:3!null lineitems1_.customerid:4 lineitems1_.ordernumber:5 lineitems1_.productid:6 lineitems1_.quantity:7 li.customerid:8 li.ordernumber:9 column16:16 + │ │ ├── columns: order0_.customerid:1!null order0_.ordernumber:2!null orderdate:3!null lineitems1_.customerid:5 lineitems1_.ordernumber:6 lineitems1_.productid:7 lineitems1_.quantity:8 li.customerid:10 li.ordernumber:11 column20:20 │ │ ├── immutable - │ │ ├── fd: ()-->(1-3), (6)-->(4,5,7) + │ │ ├── fd: ()-->(1-3), (7)-->(5,6,8) │ │ ├── project - │ │ │ ├── columns: column16:16 li.customerid:8!null li.ordernumber:9!null + │ │ │ ├── columns: column20:20 li.customerid:10!null li.ordernumber:11!null │ │ │ ├── immutable │ │ │ ├── inner-join (hash) - │ │ │ │ ├── columns: li.customerid:8!null li.ordernumber:9!null li.productid:10!null li.quantity:11 p.productid:12!null cost:14 + │ │ │ │ ├── columns: li.customerid:10!null li.ordernumber:11!null li.productid:12!null li.quantity:13 p.productid:15!null cost:17 │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ │ ├── key: (8,9,12) - │ │ │ │ ├── fd: (8-10)-->(11), (12)-->(14), (10)==(12), (12)==(10) + │ │ │ │ ├── key: (10,11,15) + │ │ │ │ ├── fd: (10-12)-->(13), (15)-->(17), (12)==(15), (15)==(12) │ │ │ │ ├── scan li - │ │ │ │ │ ├── columns: li.customerid:8!null li.ordernumber:9!null li.productid:10!null li.quantity:11 - │ │ │ │ │ ├── key: (8-10) - │ │ │ │ │ └── fd: (8-10)-->(11) + │ │ │ │ │ ├── columns: li.customerid:10!null li.ordernumber:11!null li.productid:12!null li.quantity:13 + │ │ │ │ │ ├── key: (10-12) + │ │ │ │ │ └── fd: (10-12)-->(13) │ │ │ │ ├── scan p - │ │ │ │ │ ├── columns: p.productid:12!null cost:14 - │ │ │ │ │ ├── key: (12) - │ │ │ │ │ └── fd: (12)-->(14) + │ │ │ │ │ ├── columns: p.productid:15!null cost:17 + │ │ │ │ │ ├── key: (15) + │ │ │ │ │ └── fd: (15)-->(17) │ │ │ │ └── filters - │ │ │ │ └── li.productid:10 = p.productid:12 [outer=(10,12), constraints=(/10: (/NULL - ]; /12: (/NULL - ]), fd=(10)==(12), (12)==(10)] + │ │ │ │ └── li.productid:12 = p.productid:15 [outer=(12,15), constraints=(/12: (/NULL - ]; /15: (/NULL - ]), fd=(12)==(15), (15)==(12)] │ │ │ └── projections - │ │ │ └── li.quantity:11::INT8 * cost:14::DECIMAL [as=column16:16, outer=(11,14), immutable] + │ │ │ └── li.quantity:13::INT8 * cost:17::DECIMAL [as=column20:20, outer=(13,17), immutable] │ │ ├── left-join (merge) - │ │ │ ├── columns: order0_.customerid:1!null order0_.ordernumber:2!null orderdate:3!null lineitems1_.customerid:4 lineitems1_.ordernumber:5 lineitems1_.productid:6 lineitems1_.quantity:7 + │ │ │ ├── columns: order0_.customerid:1!null order0_.ordernumber:2!null orderdate:3!null lineitems1_.customerid:5 lineitems1_.ordernumber:6 lineitems1_.productid:7 lineitems1_.quantity:8 │ │ │ ├── left ordering: +1,+2 - │ │ │ ├── right ordering: +4,+5 - │ │ │ ├── key: (6) - │ │ │ ├── fd: ()-->(1-3), (6)-->(4,5,7) + │ │ │ ├── right ordering: +5,+6 + │ │ │ ├── key: (7) + │ │ │ ├── fd: ()-->(1-3), (7)-->(5,6,8) │ │ │ ├── scan order0_ │ │ │ │ ├── columns: order0_.customerid:1!null order0_.ordernumber:2!null orderdate:3!null │ │ │ │ ├── constraint: /1/2: [/'c111'/0 - /'c111'/0] @@ -2034,31 +2034,31 @@ project │ │ │ │ ├── key: () │ │ │ │ └── fd: ()-->(1-3) │ │ │ ├── scan lineitems1_ - │ │ │ │ ├── columns: lineitems1_.customerid:4!null lineitems1_.ordernumber:5!null lineitems1_.productid:6!null lineitems1_.quantity:7 - │ │ │ │ ├── constraint: /4/5/6: [/'c111'/0 - /'c111'/0] - │ │ │ │ ├── key: (6) - │ │ │ │ └── fd: ()-->(4,5), (6)-->(7) + │ │ │ │ ├── columns: lineitems1_.customerid:5!null lineitems1_.ordernumber:6!null lineitems1_.productid:7!null lineitems1_.quantity:8 + │ │ │ │ ├── constraint: /5/6/7: [/'c111'/0 - /'c111'/0] + │ │ │ │ ├── key: (7) + │ │ │ │ └── fd: ()-->(5,6), (7)-->(8) │ │ │ └── filters (true) │ │ └── filters - │ │ ├── li.customerid:8 = order0_.customerid:1 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] - │ │ └── li.ordernumber:9 = order0_.ordernumber:2 [outer=(2,9), constraints=(/2: (/NULL - ]; /9: (/NULL - ]), fd=(2)==(9), (9)==(2)] + │ │ ├── li.customerid:10 = order0_.customerid:1 [outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] + │ │ └── li.ordernumber:11 = order0_.ordernumber:2 [outer=(2,11), constraints=(/2: (/NULL - ]; /11: (/NULL - ]), fd=(2)==(11), (11)==(2)] │ └── aggregations - │ ├── sum [as=sum:17, outer=(16)] - │ │ └── column16:16 + │ ├── sum [as=sum:21, outer=(20)] + │ │ └── column20:20 │ ├── const-agg [as=order0_.customerid:1, outer=(1)] │ │ └── order0_.customerid:1 │ ├── const-agg [as=order0_.ordernumber:2, outer=(2)] │ │ └── order0_.ordernumber:2 │ ├── const-agg [as=orderdate:3, outer=(3)] │ │ └── orderdate:3 - │ ├── const-agg [as=lineitems1_.customerid:4, outer=(4)] - │ │ └── lineitems1_.customerid:4 - │ ├── const-agg [as=lineitems1_.ordernumber:5, outer=(5)] - │ │ └── lineitems1_.ordernumber:5 - │ └── const-agg [as=lineitems1_.quantity:7, outer=(7)] - │ └── lineitems1_.quantity:7 + │ ├── const-agg [as=lineitems1_.customerid:5, outer=(5)] + │ │ └── lineitems1_.customerid:5 + │ ├── const-agg [as=lineitems1_.ordernumber:6, outer=(6)] + │ │ └── lineitems1_.ordernumber:6 + │ └── const-agg [as=lineitems1_.quantity:8, outer=(8)] + │ └── lineitems1_.quantity:8 └── projections - └── sum:17 [as=formula101_0_:18, outer=(17)] + └── sum:21 [as=formula101_0_:22, outer=(21)] opt SELECT @@ -2113,139 +2113,139 @@ FROM LEFT JOIN product AS product3_ ON lineitems2_.productid = product3_.productid; ---- project - ├── columns: customer1_0_0_:1!null customer1_1_1_:4 ordernum2_1_1_:5 customer1_2_2_:7 ordernum2_2_2_:8 producti3_2_2_:9 producti1_3_3_:11 name2_0_0_:2!null address3_0_0_:3!null orderdat3_1_1_:6 formula103_1_:30 customer1_1_0__:4 ordernum2_1_0__:5 ordernum2_0__:5 quantity4_2_2_:10 customer1_2_1__:7 ordernum2_2_1__:8 producti3_2_1__:9 descript2_3_3_:12 cost3_3_3_:13 numberav4_3_3_:14 formula104_3_:31 + ├── columns: customer1_0_0_:1!null customer1_1_1_:5 ordernum2_1_1_:6 customer1_2_2_:9 ordernum2_2_2_:10 producti3_2_2_:11 producti1_3_3_:14 name2_0_0_:2!null address3_0_0_:3!null orderdat3_1_1_:7 formula103_1_:37 customer1_1_0__:5 ordernum2_1_0__:6 ordernum2_0__:6 quantity4_2_2_:12 customer1_2_1__:9 ordernum2_2_1__:10 producti3_2_1__:11 descript2_3_3_:15 cost3_3_3_:16 numberav4_3_3_:17 formula104_3_:38 ├── immutable - ├── key: (1,4,5,7-9) - ├── fd: (1)-->(2,3), (4,5)-->(6), (7-9)-->(10), (11)-->(12-14), (1,4,5,7-9)-->(2,3,6,10-14,30,31) + ├── key: (1,5,6,9-11) + ├── fd: (1)-->(2,3), (5,6)-->(7), (9-11)-->(12), (14)-->(15-17), (1,5,6,9-11)-->(2,3,7,12,14-17,37,38) ├── group-by - │ ├── columns: customer0_.customerid:1!null name:2!null address:3!null orders1_.customerid:4 orders1_.ordernumber:5 orderdate:6 lineitems2_.customerid:7 lineitems2_.ordernumber:8 lineitems2_.productid:9 lineitems2_.quantity:10 product3_.productid:11 product3_.description:12 product3_.cost:13 product3_.numberavailable:14 sum:24 sum:29 - │ ├── grouping columns: customer0_.customerid:1!null orders1_.customerid:4 orders1_.ordernumber:5 lineitems2_.customerid:7 lineitems2_.ordernumber:8 lineitems2_.productid:9 + │ ├── columns: customer0_.customerid:1!null name:2!null address:3!null orders1_.customerid:5 orders1_.ordernumber:6 orderdate:7 lineitems2_.customerid:9 lineitems2_.ordernumber:10 lineitems2_.productid:11 lineitems2_.quantity:12 product3_.productid:14 product3_.description:15 product3_.cost:16 product3_.numberavailable:17 sum:30 sum:36 + │ ├── grouping columns: customer0_.customerid:1!null orders1_.customerid:5 orders1_.ordernumber:6 lineitems2_.customerid:9 lineitems2_.ordernumber:10 lineitems2_.productid:11 │ ├── immutable - │ ├── key: (1,4,5,7-9) - │ ├── fd: (1)-->(2,3), (4,5)-->(6), (7-9)-->(10), (11)-->(12-14), (1,4,5,7-9)-->(2,3,6,10-14,24,29) + │ ├── key: (1,5,6,9-11) + │ ├── fd: (1)-->(2,3), (5,6)-->(7), (9-11)-->(12), (14)-->(15-17), (1,5,6,9-11)-->(2,3,7,12,14-17,30,36) │ ├── left-join (hash) - │ │ ├── columns: customer0_.customerid:1!null name:2!null address:3!null orders1_.customerid:4 orders1_.ordernumber:5 orderdate:6 lineitems2_.customerid:7 lineitems2_.ordernumber:8 lineitems2_.productid:9 lineitems2_.quantity:10 product3_.productid:11 product3_.description:12 product3_.cost:13 product3_.numberavailable:14 sum:24 li.productid:27 li.quantity:28 + │ │ ├── columns: customer0_.customerid:1!null name:2!null address:3!null orders1_.customerid:5 orders1_.ordernumber:6 orderdate:7 lineitems2_.customerid:9 lineitems2_.ordernumber:10 lineitems2_.productid:11 lineitems2_.quantity:12 product3_.productid:14 product3_.description:15 product3_.cost:16 product3_.numberavailable:17 sum:30 li.productid:33 li.quantity:34 │ │ ├── immutable - │ │ ├── fd: (1)-->(2,3), (4,5)-->(6), (7-9)-->(10), (11)-->(12-14), (1,4,5,7-9)-->(2,3,6,10-14,24) + │ │ ├── fd: (1)-->(2,3), (5,6)-->(7), (9-11)-->(12), (14)-->(15-17), (1,5,6,9-11)-->(2,3,7,12,14-17,30) │ │ ├── group-by - │ │ │ ├── columns: customer0_.customerid:1!null name:2!null address:3!null orders1_.customerid:4 orders1_.ordernumber:5 orderdate:6 lineitems2_.customerid:7 lineitems2_.ordernumber:8 lineitems2_.productid:9 lineitems2_.quantity:10 product3_.productid:11 product3_.description:12 product3_.cost:13 product3_.numberavailable:14 sum:24 - │ │ │ ├── grouping columns: customer0_.customerid:1!null orders1_.customerid:4 orders1_.ordernumber:5 lineitems2_.customerid:7 lineitems2_.ordernumber:8 lineitems2_.productid:9 + │ │ │ ├── columns: customer0_.customerid:1!null name:2!null address:3!null orders1_.customerid:5 orders1_.ordernumber:6 orderdate:7 lineitems2_.customerid:9 lineitems2_.ordernumber:10 lineitems2_.productid:11 lineitems2_.quantity:12 product3_.productid:14 product3_.description:15 product3_.cost:16 product3_.numberavailable:17 sum:30 + │ │ │ ├── grouping columns: customer0_.customerid:1!null orders1_.customerid:5 orders1_.ordernumber:6 lineitems2_.customerid:9 lineitems2_.ordernumber:10 lineitems2_.productid:11 │ │ │ ├── immutable - │ │ │ ├── key: (1,4,5,7-9) - │ │ │ ├── fd: (1)-->(2,3), (4,5)-->(6), (7-9)-->(10), (11)-->(12-14), (1,4,5,7-9)-->(2,3,6,10-14,24) + │ │ │ ├── key: (1,5,6,9-11) + │ │ │ ├── fd: (1)-->(2,3), (5,6)-->(7), (9-11)-->(12), (14)-->(15-17), (1,5,6,9-11)-->(2,3,7,12,14-17,30) │ │ │ ├── left-join (hash) - │ │ │ │ ├── columns: customer0_.customerid:1!null name:2!null address:3!null orders1_.customerid:4 orders1_.ordernumber:5 orderdate:6 lineitems2_.customerid:7 lineitems2_.ordernumber:8 lineitems2_.productid:9 lineitems2_.quantity:10 product3_.productid:11 product3_.description:12 product3_.cost:13 product3_.numberavailable:14 li.customerid:15 li.ordernumber:16 column23:23 + │ │ │ │ ├── columns: customer0_.customerid:1!null name:2!null address:3!null orders1_.customerid:5 orders1_.ordernumber:6 orderdate:7 lineitems2_.customerid:9 lineitems2_.ordernumber:10 lineitems2_.productid:11 lineitems2_.quantity:12 product3_.productid:14 product3_.description:15 product3_.cost:16 product3_.numberavailable:17 li.customerid:19 li.ordernumber:20 column29:29 │ │ │ │ ├── immutable - │ │ │ │ ├── fd: (1)-->(2,3), (4,5)-->(6), (7-9)-->(10), (11)-->(12-14), (1,4,5,7-9)-->(11-14) + │ │ │ │ ├── fd: (1)-->(2,3), (5,6)-->(7), (9-11)-->(12), (14)-->(15-17), (1,5,6,9-11)-->(14-17) │ │ │ │ ├── left-join (hash) - │ │ │ │ │ ├── columns: customer0_.customerid:1!null name:2!null address:3!null orders1_.customerid:4 orders1_.ordernumber:5 orderdate:6 lineitems2_.customerid:7 lineitems2_.ordernumber:8 lineitems2_.productid:9 lineitems2_.quantity:10 product3_.productid:11 product3_.description:12 product3_.cost:13 product3_.numberavailable:14 + │ │ │ │ │ ├── columns: customer0_.customerid:1!null name:2!null address:3!null orders1_.customerid:5 orders1_.ordernumber:6 orderdate:7 lineitems2_.customerid:9 lineitems2_.ordernumber:10 lineitems2_.productid:11 lineitems2_.quantity:12 product3_.productid:14 product3_.description:15 product3_.cost:16 product3_.numberavailable:17 │ │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ │ │ │ │ ├── key: (1,4,5,7-9) - │ │ │ │ │ ├── fd: (1)-->(2,3), (4,5)-->(6), (7-9)-->(10), (11)-->(12-14), (1,4,5,7-9)-->(11-14) + │ │ │ │ │ ├── key: (1,5,6,9-11) + │ │ │ │ │ ├── fd: (1)-->(2,3), (5,6)-->(7), (9-11)-->(12), (14)-->(15-17), (1,5,6,9-11)-->(14-17) │ │ │ │ │ ├── left-join (hash) - │ │ │ │ │ │ ├── columns: customer0_.customerid:1!null name:2!null address:3!null orders1_.customerid:4 orders1_.ordernumber:5 orderdate:6 lineitems2_.customerid:7 lineitems2_.ordernumber:8 lineitems2_.productid:9 lineitems2_.quantity:10 - │ │ │ │ │ │ ├── key: (1,4,5,7-9) - │ │ │ │ │ │ ├── fd: (1)-->(2,3), (4,5)-->(6), (7-9)-->(10) + │ │ │ │ │ │ ├── columns: customer0_.customerid:1!null name:2!null address:3!null orders1_.customerid:5 orders1_.ordernumber:6 orderdate:7 lineitems2_.customerid:9 lineitems2_.ordernumber:10 lineitems2_.productid:11 lineitems2_.quantity:12 + │ │ │ │ │ │ ├── key: (1,5,6,9-11) + │ │ │ │ │ │ ├── fd: (1)-->(2,3), (5,6)-->(7), (9-11)-->(12) │ │ │ │ │ │ ├── left-join (merge) - │ │ │ │ │ │ │ ├── columns: customer0_.customerid:1!null name:2!null address:3!null orders1_.customerid:4 orders1_.ordernumber:5 orderdate:6 + │ │ │ │ │ │ │ ├── columns: customer0_.customerid:1!null name:2!null address:3!null orders1_.customerid:5 orders1_.ordernumber:6 orderdate:7 │ │ │ │ │ │ │ ├── left ordering: +1 - │ │ │ │ │ │ │ ├── right ordering: +4 - │ │ │ │ │ │ │ ├── key: (1,4,5) - │ │ │ │ │ │ │ ├── fd: (1)-->(2,3), (4,5)-->(6) + │ │ │ │ │ │ │ ├── right ordering: +5 + │ │ │ │ │ │ │ ├── key: (1,5,6) + │ │ │ │ │ │ │ ├── fd: (1)-->(2,3), (5,6)-->(7) │ │ │ │ │ │ │ ├── scan customer0_ │ │ │ │ │ │ │ │ ├── columns: customer0_.customerid:1!null name:2!null address:3!null │ │ │ │ │ │ │ │ ├── key: (1) │ │ │ │ │ │ │ │ ├── fd: (1)-->(2,3) │ │ │ │ │ │ │ │ └── ordering: +1 │ │ │ │ │ │ │ ├── scan orders1_ - │ │ │ │ │ │ │ │ ├── columns: orders1_.customerid:4!null orders1_.ordernumber:5!null orderdate:6!null - │ │ │ │ │ │ │ │ ├── key: (4,5) - │ │ │ │ │ │ │ │ ├── fd: (4,5)-->(6) - │ │ │ │ │ │ │ │ └── ordering: +4 + │ │ │ │ │ │ │ │ ├── columns: orders1_.customerid:5!null orders1_.ordernumber:6!null orderdate:7!null + │ │ │ │ │ │ │ │ ├── key: (5,6) + │ │ │ │ │ │ │ │ ├── fd: (5,6)-->(7) + │ │ │ │ │ │ │ │ └── ordering: +5 │ │ │ │ │ │ │ └── filters (true) │ │ │ │ │ │ ├── scan lineitems2_ - │ │ │ │ │ │ │ ├── columns: lineitems2_.customerid:7!null lineitems2_.ordernumber:8!null lineitems2_.productid:9!null lineitems2_.quantity:10 - │ │ │ │ │ │ │ ├── key: (7-9) - │ │ │ │ │ │ │ └── fd: (7-9)-->(10) + │ │ │ │ │ │ │ ├── columns: lineitems2_.customerid:9!null lineitems2_.ordernumber:10!null lineitems2_.productid:11!null lineitems2_.quantity:12 + │ │ │ │ │ │ │ ├── key: (9-11) + │ │ │ │ │ │ │ └── fd: (9-11)-->(12) │ │ │ │ │ │ └── filters - │ │ │ │ │ │ ├── orders1_.customerid:4 = lineitems2_.customerid:7 [outer=(4,7), constraints=(/4: (/NULL - ]; /7: (/NULL - ]), fd=(4)==(7), (7)==(4)] - │ │ │ │ │ │ └── orders1_.ordernumber:5 = lineitems2_.ordernumber:8 [outer=(5,8), constraints=(/5: (/NULL - ]; /8: (/NULL - ]), fd=(5)==(8), (8)==(5)] + │ │ │ │ │ │ ├── orders1_.customerid:5 = lineitems2_.customerid:9 [outer=(5,9), constraints=(/5: (/NULL - ]; /9: (/NULL - ]), fd=(5)==(9), (9)==(5)] + │ │ │ │ │ │ └── orders1_.ordernumber:6 = lineitems2_.ordernumber:10 [outer=(6,10), constraints=(/6: (/NULL - ]; /10: (/NULL - ]), fd=(6)==(10), (10)==(6)] │ │ │ │ │ ├── scan product3_ - │ │ │ │ │ │ ├── columns: product3_.productid:11!null product3_.description:12!null product3_.cost:13 product3_.numberavailable:14 - │ │ │ │ │ │ ├── key: (11) - │ │ │ │ │ │ └── fd: (11)-->(12-14) + │ │ │ │ │ │ ├── columns: product3_.productid:14!null product3_.description:15!null product3_.cost:16 product3_.numberavailable:17 + │ │ │ │ │ │ ├── key: (14) + │ │ │ │ │ │ └── fd: (14)-->(15-17) │ │ │ │ │ └── filters - │ │ │ │ │ └── lineitems2_.productid:9 = product3_.productid:11 [outer=(9,11), constraints=(/9: (/NULL - ]; /11: (/NULL - ]), fd=(9)==(11), (11)==(9)] + │ │ │ │ │ └── lineitems2_.productid:11 = product3_.productid:14 [outer=(11,14), constraints=(/11: (/NULL - ]; /14: (/NULL - ]), fd=(11)==(14), (14)==(11)] │ │ │ │ ├── project - │ │ │ │ │ ├── columns: column23:23 li.customerid:15!null li.ordernumber:16!null + │ │ │ │ │ ├── columns: column29:29 li.customerid:19!null li.ordernumber:20!null │ │ │ │ │ ├── immutable │ │ │ │ │ ├── inner-join (hash) - │ │ │ │ │ │ ├── columns: li.customerid:15!null li.ordernumber:16!null li.productid:17!null li.quantity:18 p.productid:19!null p.cost:21 + │ │ │ │ │ │ ├── columns: li.customerid:19!null li.ordernumber:20!null li.productid:21!null li.quantity:22 p.productid:24!null p.cost:26 │ │ │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ │ │ │ ├── key: (15,16,19) - │ │ │ │ │ │ ├── fd: (15-17)-->(18), (19)-->(21), (17)==(19), (19)==(17) + │ │ │ │ │ │ ├── key: (19,20,24) + │ │ │ │ │ │ ├── fd: (19-21)-->(22), (24)-->(26), (21)==(24), (24)==(21) │ │ │ │ │ │ ├── scan li - │ │ │ │ │ │ │ ├── columns: li.customerid:15!null li.ordernumber:16!null li.productid:17!null li.quantity:18 - │ │ │ │ │ │ │ ├── key: (15-17) - │ │ │ │ │ │ │ └── fd: (15-17)-->(18) + │ │ │ │ │ │ │ ├── columns: li.customerid:19!null li.ordernumber:20!null li.productid:21!null li.quantity:22 + │ │ │ │ │ │ │ ├── key: (19-21) + │ │ │ │ │ │ │ └── fd: (19-21)-->(22) │ │ │ │ │ │ ├── scan p - │ │ │ │ │ │ │ ├── columns: p.productid:19!null p.cost:21 - │ │ │ │ │ │ │ ├── key: (19) - │ │ │ │ │ │ │ └── fd: (19)-->(21) + │ │ │ │ │ │ │ ├── columns: p.productid:24!null p.cost:26 + │ │ │ │ │ │ │ ├── key: (24) + │ │ │ │ │ │ │ └── fd: (24)-->(26) │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── li.productid:17 = p.productid:19 [outer=(17,19), constraints=(/17: (/NULL - ]; /19: (/NULL - ]), fd=(17)==(19), (19)==(17)] + │ │ │ │ │ │ └── li.productid:21 = p.productid:24 [outer=(21,24), constraints=(/21: (/NULL - ]; /24: (/NULL - ]), fd=(21)==(24), (24)==(21)] │ │ │ │ │ └── projections - │ │ │ │ │ └── li.quantity:18::INT8 * p.cost:21::DECIMAL [as=column23:23, outer=(18,21), immutable] + │ │ │ │ │ └── li.quantity:22::INT8 * p.cost:26::DECIMAL [as=column29:29, outer=(22,26), immutable] │ │ │ │ └── filters - │ │ │ │ ├── li.customerid:15 = orders1_.customerid:4 [outer=(4,15), constraints=(/4: (/NULL - ]; /15: (/NULL - ]), fd=(4)==(15), (15)==(4)] - │ │ │ │ └── li.ordernumber:16 = orders1_.ordernumber:5 [outer=(5,16), constraints=(/5: (/NULL - ]; /16: (/NULL - ]), fd=(5)==(16), (16)==(5)] + │ │ │ │ ├── li.customerid:19 = orders1_.customerid:5 [outer=(5,19), constraints=(/5: (/NULL - ]; /19: (/NULL - ]), fd=(5)==(19), (19)==(5)] + │ │ │ │ └── li.ordernumber:20 = orders1_.ordernumber:6 [outer=(6,20), constraints=(/6: (/NULL - ]; /20: (/NULL - ]), fd=(6)==(20), (20)==(6)] │ │ │ └── aggregations - │ │ │ ├── sum [as=sum:24, outer=(23)] - │ │ │ │ └── column23:23 + │ │ │ ├── sum [as=sum:30, outer=(29)] + │ │ │ │ └── column29:29 │ │ │ ├── const-agg [as=name:2, outer=(2)] │ │ │ │ └── name:2 │ │ │ ├── const-agg [as=address:3, outer=(3)] │ │ │ │ └── address:3 - │ │ │ ├── const-agg [as=orderdate:6, outer=(6)] - │ │ │ │ └── orderdate:6 - │ │ │ ├── const-agg [as=lineitems2_.quantity:10, outer=(10)] - │ │ │ │ └── lineitems2_.quantity:10 - │ │ │ ├── const-agg [as=product3_.productid:11, outer=(11)] - │ │ │ │ └── product3_.productid:11 - │ │ │ ├── const-agg [as=product3_.description:12, outer=(12)] - │ │ │ │ └── product3_.description:12 - │ │ │ ├── const-agg [as=product3_.cost:13, outer=(13)] - │ │ │ │ └── product3_.cost:13 - │ │ │ └── const-agg [as=product3_.numberavailable:14, outer=(14)] - │ │ │ └── product3_.numberavailable:14 + │ │ │ ├── const-agg [as=orderdate:7, outer=(7)] + │ │ │ │ └── orderdate:7 + │ │ │ ├── const-agg [as=lineitems2_.quantity:12, outer=(12)] + │ │ │ │ └── lineitems2_.quantity:12 + │ │ │ ├── const-agg [as=product3_.productid:14, outer=(14)] + │ │ │ │ └── product3_.productid:14 + │ │ │ ├── const-agg [as=product3_.description:15, outer=(15)] + │ │ │ │ └── product3_.description:15 + │ │ │ ├── const-agg [as=product3_.cost:16, outer=(16)] + │ │ │ │ └── product3_.cost:16 + │ │ │ └── const-agg [as=product3_.numberavailable:17, outer=(17)] + │ │ │ └── product3_.numberavailable:17 │ │ ├── scan li - │ │ │ └── columns: li.productid:27!null li.quantity:28 + │ │ │ └── columns: li.productid:33!null li.quantity:34 │ │ └── filters - │ │ └── li.productid:27 = product3_.productid:11 [outer=(11,27), constraints=(/11: (/NULL - ]; /27: (/NULL - ]), fd=(11)==(27), (27)==(11)] + │ │ └── li.productid:33 = product3_.productid:14 [outer=(14,33), constraints=(/14: (/NULL - ]; /33: (/NULL - ]), fd=(14)==(33), (33)==(14)] │ └── aggregations - │ ├── sum [as=sum:29, outer=(28)] - │ │ └── li.quantity:28 + │ ├── sum [as=sum:36, outer=(34)] + │ │ └── li.quantity:34 │ ├── const-agg [as=name:2, outer=(2)] │ │ └── name:2 │ ├── const-agg [as=address:3, outer=(3)] │ │ └── address:3 - │ ├── const-agg [as=orderdate:6, outer=(6)] - │ │ └── orderdate:6 - │ ├── const-agg [as=lineitems2_.quantity:10, outer=(10)] - │ │ └── lineitems2_.quantity:10 - │ ├── const-agg [as=product3_.productid:11, outer=(11)] - │ │ └── product3_.productid:11 - │ ├── const-agg [as=product3_.description:12, outer=(12)] - │ │ └── product3_.description:12 - │ ├── const-agg [as=product3_.cost:13, outer=(13)] - │ │ └── product3_.cost:13 - │ ├── const-agg [as=product3_.numberavailable:14, outer=(14)] - │ │ └── product3_.numberavailable:14 - │ └── const-agg [as=sum:24, outer=(24)] - │ └── sum:24 + │ ├── const-agg [as=orderdate:7, outer=(7)] + │ │ └── orderdate:7 + │ ├── const-agg [as=lineitems2_.quantity:12, outer=(12)] + │ │ └── lineitems2_.quantity:12 + │ ├── const-agg [as=product3_.productid:14, outer=(14)] + │ │ └── product3_.productid:14 + │ ├── const-agg [as=product3_.description:15, outer=(15)] + │ │ └── product3_.description:15 + │ ├── const-agg [as=product3_.cost:16, outer=(16)] + │ │ └── product3_.cost:16 + │ ├── const-agg [as=product3_.numberavailable:17, outer=(17)] + │ │ └── product3_.numberavailable:17 + │ └── const-agg [as=sum:30, outer=(30)] + │ └── sum:30 └── projections - ├── sum:24 [as=formula103_1_:30, outer=(24)] - └── sum:29 [as=formula104_3_:31, outer=(29)] + ├── sum:30 [as=formula103_1_:37, outer=(30)] + └── sum:36 [as=formula104_3_:38, outer=(36)] opt SELECT @@ -2280,46 +2280,46 @@ WHERE order0_.customerid = 'c111' AND order0_.ordernumber = 0; ---- project - ├── columns: customer1_1_0_:1!null ordernum2_1_0_:2!null orderdat3_1_0_:3!null formula105_0_:18 customer1_2_1_:4 ordernum2_2_1_:5 producti3_2_1_:6 customer1_2_2_:4 ordernum2_2_2_:5 producti3_2_2_:6 quantity4_2_2_:7 + ├── columns: customer1_1_0_:1!null ordernum2_1_0_:2!null orderdat3_1_0_:3!null formula105_0_:22 customer1_2_1_:5 ordernum2_2_1_:6 producti3_2_1_:7 customer1_2_2_:5 ordernum2_2_2_:6 producti3_2_2_:7 quantity4_2_2_:8 ├── immutable - ├── key: (6) - ├── fd: ()-->(1-3), (6)-->(4,5,7,18) + ├── key: (7) + ├── fd: ()-->(1-3), (7)-->(5,6,8,22) ├── group-by - │ ├── columns: order0_.customerid:1!null order0_.ordernumber:2!null orderdate:3!null lineitems1_.customerid:4 lineitems1_.ordernumber:5 lineitems1_.productid:6 lineitems1_.quantity:7 sum:17 - │ ├── grouping columns: lineitems1_.productid:6 + │ ├── columns: order0_.customerid:1!null order0_.ordernumber:2!null orderdate:3!null lineitems1_.customerid:5 lineitems1_.ordernumber:6 lineitems1_.productid:7 lineitems1_.quantity:8 sum:21 + │ ├── grouping columns: lineitems1_.productid:7 │ ├── immutable - │ ├── key: (6) - │ ├── fd: ()-->(1-3), (6)-->(1-5,7,17) + │ ├── key: (7) + │ ├── fd: ()-->(1-3), (7)-->(1-3,5,6,8,21) │ ├── right-join (hash) - │ │ ├── columns: order0_.customerid:1!null order0_.ordernumber:2!null orderdate:3!null lineitems1_.customerid:4 lineitems1_.ordernumber:5 lineitems1_.productid:6 lineitems1_.quantity:7 li.customerid:8 li.ordernumber:9 column16:16 + │ │ ├── columns: order0_.customerid:1!null order0_.ordernumber:2!null orderdate:3!null lineitems1_.customerid:5 lineitems1_.ordernumber:6 lineitems1_.productid:7 lineitems1_.quantity:8 li.customerid:10 li.ordernumber:11 column20:20 │ │ ├── immutable - │ │ ├── fd: ()-->(1-3), (6)-->(4,5,7) + │ │ ├── fd: ()-->(1-3), (7)-->(5,6,8) │ │ ├── project - │ │ │ ├── columns: column16:16 li.customerid:8!null li.ordernumber:9!null + │ │ │ ├── columns: column20:20 li.customerid:10!null li.ordernumber:11!null │ │ │ ├── immutable │ │ │ ├── inner-join (hash) - │ │ │ │ ├── columns: li.customerid:8!null li.ordernumber:9!null li.productid:10!null li.quantity:11 p.productid:12!null cost:14 + │ │ │ │ ├── columns: li.customerid:10!null li.ordernumber:11!null li.productid:12!null li.quantity:13 p.productid:15!null cost:17 │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ │ ├── key: (8,9,12) - │ │ │ │ ├── fd: (8-10)-->(11), (12)-->(14), (10)==(12), (12)==(10) + │ │ │ │ ├── key: (10,11,15) + │ │ │ │ ├── fd: (10-12)-->(13), (15)-->(17), (12)==(15), (15)==(12) │ │ │ │ ├── scan li - │ │ │ │ │ ├── columns: li.customerid:8!null li.ordernumber:9!null li.productid:10!null li.quantity:11 - │ │ │ │ │ ├── key: (8-10) - │ │ │ │ │ └── fd: (8-10)-->(11) + │ │ │ │ │ ├── columns: li.customerid:10!null li.ordernumber:11!null li.productid:12!null li.quantity:13 + │ │ │ │ │ ├── key: (10-12) + │ │ │ │ │ └── fd: (10-12)-->(13) │ │ │ │ ├── scan p - │ │ │ │ │ ├── columns: p.productid:12!null cost:14 - │ │ │ │ │ ├── key: (12) - │ │ │ │ │ └── fd: (12)-->(14) + │ │ │ │ │ ├── columns: p.productid:15!null cost:17 + │ │ │ │ │ ├── key: (15) + │ │ │ │ │ └── fd: (15)-->(17) │ │ │ │ └── filters - │ │ │ │ └── li.productid:10 = p.productid:12 [outer=(10,12), constraints=(/10: (/NULL - ]; /12: (/NULL - ]), fd=(10)==(12), (12)==(10)] + │ │ │ │ └── li.productid:12 = p.productid:15 [outer=(12,15), constraints=(/12: (/NULL - ]; /15: (/NULL - ]), fd=(12)==(15), (15)==(12)] │ │ │ └── projections - │ │ │ └── li.quantity:11::INT8 * cost:14::DECIMAL [as=column16:16, outer=(11,14), immutable] + │ │ │ └── li.quantity:13::INT8 * cost:17::DECIMAL [as=column20:20, outer=(13,17), immutable] │ │ ├── left-join (merge) - │ │ │ ├── columns: order0_.customerid:1!null order0_.ordernumber:2!null orderdate:3!null lineitems1_.customerid:4 lineitems1_.ordernumber:5 lineitems1_.productid:6 lineitems1_.quantity:7 + │ │ │ ├── columns: order0_.customerid:1!null order0_.ordernumber:2!null orderdate:3!null lineitems1_.customerid:5 lineitems1_.ordernumber:6 lineitems1_.productid:7 lineitems1_.quantity:8 │ │ │ ├── left ordering: +1,+2 - │ │ │ ├── right ordering: +4,+5 - │ │ │ ├── key: (6) - │ │ │ ├── fd: ()-->(1-3), (6)-->(4,5,7) + │ │ │ ├── right ordering: +5,+6 + │ │ │ ├── key: (7) + │ │ │ ├── fd: ()-->(1-3), (7)-->(5,6,8) │ │ │ ├── scan order0_ │ │ │ │ ├── columns: order0_.customerid:1!null order0_.ordernumber:2!null orderdate:3!null │ │ │ │ ├── constraint: /1/2: [/'c111'/0 - /'c111'/0] @@ -2327,31 +2327,31 @@ project │ │ │ │ ├── key: () │ │ │ │ └── fd: ()-->(1-3) │ │ │ ├── scan lineitems1_ - │ │ │ │ ├── columns: lineitems1_.customerid:4!null lineitems1_.ordernumber:5!null lineitems1_.productid:6!null lineitems1_.quantity:7 - │ │ │ │ ├── constraint: /4/5/6: [/'c111'/0 - /'c111'/0] - │ │ │ │ ├── key: (6) - │ │ │ │ └── fd: ()-->(4,5), (6)-->(7) + │ │ │ │ ├── columns: lineitems1_.customerid:5!null lineitems1_.ordernumber:6!null lineitems1_.productid:7!null lineitems1_.quantity:8 + │ │ │ │ ├── constraint: /5/6/7: [/'c111'/0 - /'c111'/0] + │ │ │ │ ├── key: (7) + │ │ │ │ └── fd: ()-->(5,6), (7)-->(8) │ │ │ └── filters (true) │ │ └── filters - │ │ ├── li.customerid:8 = order0_.customerid:1 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] - │ │ └── li.ordernumber:9 = order0_.ordernumber:2 [outer=(2,9), constraints=(/2: (/NULL - ]; /9: (/NULL - ]), fd=(2)==(9), (9)==(2)] + │ │ ├── li.customerid:10 = order0_.customerid:1 [outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] + │ │ └── li.ordernumber:11 = order0_.ordernumber:2 [outer=(2,11), constraints=(/2: (/NULL - ]; /11: (/NULL - ]), fd=(2)==(11), (11)==(2)] │ └── aggregations - │ ├── sum [as=sum:17, outer=(16)] - │ │ └── column16:16 + │ ├── sum [as=sum:21, outer=(20)] + │ │ └── column20:20 │ ├── const-agg [as=order0_.customerid:1, outer=(1)] │ │ └── order0_.customerid:1 │ ├── const-agg [as=order0_.ordernumber:2, outer=(2)] │ │ └── order0_.ordernumber:2 │ ├── const-agg [as=orderdate:3, outer=(3)] │ │ └── orderdate:3 - │ ├── const-agg [as=lineitems1_.customerid:4, outer=(4)] - │ │ └── lineitems1_.customerid:4 - │ ├── const-agg [as=lineitems1_.ordernumber:5, outer=(5)] - │ │ └── lineitems1_.ordernumber:5 - │ └── const-agg [as=lineitems1_.quantity:7, outer=(7)] - │ └── lineitems1_.quantity:7 + │ ├── const-agg [as=lineitems1_.customerid:5, outer=(5)] + │ │ └── lineitems1_.customerid:5 + │ ├── const-agg [as=lineitems1_.ordernumber:6, outer=(6)] + │ │ └── lineitems1_.ordernumber:6 + │ └── const-agg [as=lineitems1_.quantity:8, outer=(8)] + │ └── lineitems1_.quantity:8 └── projections - └── sum:17 [as=formula105_0_:18, outer=(17)] + └── sum:21 [as=formula105_0_:22, outer=(21)] opt SELECT @@ -2373,18 +2373,18 @@ FROM customerorder AS order0_; ---- project - ├── columns: customer1_10_:1!null ordernum2_10_:2!null orderdat3_10_:3!null formula273_:14 + ├── columns: customer1_10_:1!null ordernum2_10_:2!null orderdat3_10_:3!null formula273_:17 ├── immutable ├── key: (1,2) - ├── fd: (1,2)-->(3,14) + ├── fd: (1,2)-->(3,17) ├── group-by - │ ├── columns: order0_.customerid:1!null order0_.ordernumber:2!null orderdate:3!null sum:13 + │ ├── columns: order0_.customerid:1!null order0_.ordernumber:2!null orderdate:3!null sum:16 │ ├── grouping columns: order0_.customerid:1!null order0_.ordernumber:2!null │ ├── immutable │ ├── key: (1,2) - │ ├── fd: (1,2)-->(3,13) + │ ├── fd: (1,2)-->(3,16) │ ├── left-join (hash) - │ │ ├── columns: order0_.customerid:1!null order0_.ordernumber:2!null orderdate:3!null li.customerid:4 li.ordernumber:5 column12:12 + │ │ ├── columns: order0_.customerid:1!null order0_.ordernumber:2!null orderdate:3!null li.customerid:5 li.ordernumber:6 column15:15 │ │ ├── multiplicity: left-rows(one-or-more), right-rows(zero-or-one) │ │ ├── immutable │ │ ├── fd: (1,2)-->(3) @@ -2393,35 +2393,35 @@ project │ │ │ ├── key: (1,2) │ │ │ └── fd: (1,2)-->(3) │ │ ├── project - │ │ │ ├── columns: column12:12 li.customerid:4!null li.ordernumber:5!null + │ │ │ ├── columns: column15:15 li.customerid:5!null li.ordernumber:6!null │ │ │ ├── immutable │ │ │ ├── inner-join (hash) - │ │ │ │ ├── columns: li.customerid:4!null li.ordernumber:5!null li.productid:6!null quantity:7 p.productid:8!null cost:10 + │ │ │ │ ├── columns: li.customerid:5!null li.ordernumber:6!null li.productid:7!null quantity:8 p.productid:10!null cost:12 │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ │ ├── key: (4,5,8) - │ │ │ │ ├── fd: (4-6)-->(7), (8)-->(10), (6)==(8), (8)==(6) + │ │ │ │ ├── key: (5,6,10) + │ │ │ │ ├── fd: (5-7)-->(8), (10)-->(12), (7)==(10), (10)==(7) │ │ │ │ ├── scan li - │ │ │ │ │ ├── columns: li.customerid:4!null li.ordernumber:5!null li.productid:6!null quantity:7 - │ │ │ │ │ ├── key: (4-6) - │ │ │ │ │ └── fd: (4-6)-->(7) + │ │ │ │ │ ├── columns: li.customerid:5!null li.ordernumber:6!null li.productid:7!null quantity:8 + │ │ │ │ │ ├── key: (5-7) + │ │ │ │ │ └── fd: (5-7)-->(8) │ │ │ │ ├── scan p - │ │ │ │ │ ├── columns: p.productid:8!null cost:10 - │ │ │ │ │ ├── key: (8) - │ │ │ │ │ └── fd: (8)-->(10) + │ │ │ │ │ ├── columns: p.productid:10!null cost:12 + │ │ │ │ │ ├── key: (10) + │ │ │ │ │ └── fd: (10)-->(12) │ │ │ │ └── filters - │ │ │ │ └── li.productid:6 = p.productid:8 [outer=(6,8), constraints=(/6: (/NULL - ]; /8: (/NULL - ]), fd=(6)==(8), (8)==(6)] + │ │ │ │ └── li.productid:7 = p.productid:10 [outer=(7,10), constraints=(/7: (/NULL - ]; /10: (/NULL - ]), fd=(7)==(10), (10)==(7)] │ │ │ └── projections - │ │ │ └── quantity:7::INT8 * cost:10::DECIMAL [as=column12:12, outer=(7,10), immutable] + │ │ │ └── quantity:8::INT8 * cost:12::DECIMAL [as=column15:15, outer=(8,12), immutable] │ │ └── filters - │ │ ├── li.customerid:4 = order0_.customerid:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] - │ │ └── li.ordernumber:5 = order0_.ordernumber:2 [outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ]), fd=(2)==(5), (5)==(2)] + │ │ ├── li.customerid:5 = order0_.customerid:1 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + │ │ └── li.ordernumber:6 = order0_.ordernumber:2 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] │ └── aggregations - │ ├── sum [as=sum:13, outer=(12)] - │ │ └── column12:12 + │ ├── sum [as=sum:16, outer=(15)] + │ │ └── column15:15 │ └── const-agg [as=orderdate:3, outer=(3)] │ └── orderdate:3 └── projections - └── sum:13 [as=formula273_:14, outer=(13)] + └── sum:16 [as=formula273_:17, outer=(16)] exec-ddl drop table customer, customerorder, lineitem, product @@ -2485,38 +2485,38 @@ group-by ├── key: (1) ├── fd: (1)-->(2-5) ├── select - │ ├── columns: this_.studentid:1!null name:2!null address_city:3 address_state:4 preferredcoursecode:5!null enrolment_.studentid:6!null enrolment_.coursecode:7!null enrolment_.year:9!null max:14!null - │ ├── key: (1,6,7) - │ ├── fd: (1)-->(2-5), (6,7)-->(9), (1,6,7)-->(2-5,9,14), (9)==(14), (14)==(9) + │ ├── columns: this_.studentid:1!null name:2!null address_city:3 address_state:4 preferredcoursecode:5!null enrolment_.studentid:7!null enrolment_.coursecode:8!null enrolment_.year:10!null max:17!null + │ ├── key: (1,7,8) + │ ├── fd: (1)-->(2-5), (7,8)-->(10), (1,7,8)-->(2-5,10,17), (10)==(17), (17)==(10) │ ├── group-by - │ │ ├── columns: this_.studentid:1!null name:2!null address_city:3 address_state:4 preferredcoursecode:5!null enrolment_.studentid:6!null enrolment_.coursecode:7!null enrolment_.year:9!null max:14!null - │ │ ├── grouping columns: this_.studentid:1!null enrolment_.studentid:6!null enrolment_.coursecode:7!null - │ │ ├── key: (1,6,7) - │ │ ├── fd: (1)-->(2-5), (6,7)-->(9), (1,6,7)-->(2-5,9,14) + │ │ ├── columns: this_.studentid:1!null name:2!null address_city:3 address_state:4 preferredcoursecode:5!null enrolment_.studentid:7!null enrolment_.coursecode:8!null enrolment_.year:10!null max:17!null + │ │ ├── grouping columns: this_.studentid:1!null enrolment_.studentid:7!null enrolment_.coursecode:8!null + │ │ ├── key: (1,7,8) + │ │ ├── fd: (1)-->(2-5), (7,8)-->(10), (1,7,8)-->(2-5,10,17) │ │ ├── inner-join (cross) - │ │ │ ├── columns: this_.studentid:1!null name:2!null address_city:3 address_state:4 preferredcoursecode:5!null enrolment_.studentid:6!null enrolment_.coursecode:7!null enrolment_.year:9!null maxstudentenrolment_.coursecode:11!null maxstudentenrolment_.year:13!null - │ │ │ ├── fd: (1)-->(2-5), (5)==(11), (11)==(5), (6,7)-->(9) + │ │ │ ├── columns: this_.studentid:1!null name:2!null address_city:3 address_state:4 preferredcoursecode:5!null enrolment_.studentid:7!null enrolment_.coursecode:8!null enrolment_.year:10!null maxstudentenrolment_.coursecode:13!null maxstudentenrolment_.year:15!null + │ │ │ ├── fd: (1)-->(2-5), (5)==(13), (13)==(5), (7,8)-->(10) │ │ │ ├── inner-join (hash) - │ │ │ │ ├── columns: this_.studentid:1!null name:2!null address_city:3 address_state:4 preferredcoursecode:5!null maxstudentenrolment_.coursecode:11!null maxstudentenrolment_.year:13!null - │ │ │ │ ├── fd: (1)-->(2-5), (5)==(11), (11)==(5) + │ │ │ │ ├── columns: this_.studentid:1!null name:2!null address_city:3 address_state:4 preferredcoursecode:5!null maxstudentenrolment_.coursecode:13!null maxstudentenrolment_.year:15!null + │ │ │ │ ├── fd: (1)-->(2-5), (5)==(13), (13)==(5) │ │ │ │ ├── scan this_ │ │ │ │ │ ├── columns: this_.studentid:1!null name:2!null address_city:3 address_state:4 preferredcoursecode:5 │ │ │ │ │ ├── key: (1) │ │ │ │ │ └── fd: (1)-->(2-5) │ │ │ │ ├── scan maxstudentenrolment_ - │ │ │ │ │ └── columns: maxstudentenrolment_.coursecode:11!null maxstudentenrolment_.year:13!null + │ │ │ │ │ └── columns: maxstudentenrolment_.coursecode:13!null maxstudentenrolment_.year:15!null │ │ │ │ └── filters - │ │ │ │ └── preferredcoursecode:5 = maxstudentenrolment_.coursecode:11 [outer=(5,11), constraints=(/5: (/NULL - ]; /11: (/NULL - ]), fd=(5)==(11), (11)==(5)] + │ │ │ │ └── preferredcoursecode:5 = maxstudentenrolment_.coursecode:13 [outer=(5,13), constraints=(/5: (/NULL - ]; /13: (/NULL - ]), fd=(5)==(13), (13)==(5)] │ │ │ ├── scan enrolment_ - │ │ │ │ ├── columns: enrolment_.studentid:6!null enrolment_.coursecode:7!null enrolment_.year:9!null - │ │ │ │ ├── key: (6,7) - │ │ │ │ └── fd: (6,7)-->(9) + │ │ │ │ ├── columns: enrolment_.studentid:7!null enrolment_.coursecode:8!null enrolment_.year:10!null + │ │ │ │ ├── key: (7,8) + │ │ │ │ └── fd: (7,8)-->(10) │ │ │ └── filters (true) │ │ └── aggregations - │ │ ├── max [as=max:14, outer=(13)] - │ │ │ └── maxstudentenrolment_.year:13 - │ │ ├── const-agg [as=enrolment_.year:9, outer=(9)] - │ │ │ └── enrolment_.year:9 + │ │ ├── max [as=max:17, outer=(15)] + │ │ │ └── maxstudentenrolment_.year:15 + │ │ ├── const-agg [as=enrolment_.year:10, outer=(10)] + │ │ │ └── enrolment_.year:10 │ │ ├── const-agg [as=name:2, outer=(2)] │ │ │ └── name:2 │ │ ├── const-agg [as=address_city:3, outer=(3)] @@ -2526,7 +2526,7 @@ group-by │ │ └── const-agg [as=preferredcoursecode:5, outer=(5)] │ │ └── preferredcoursecode:5 │ └── filters - │ └── enrolment_.year:9 = max:14 [outer=(9,14), constraints=(/9: (/NULL - ]; /14: (/NULL - ]), fd=(9)==(14), (14)==(9)] + │ └── enrolment_.year:10 = max:17 [outer=(10,17), constraints=(/10: (/NULL - ]; /17: (/NULL - ]), fd=(10)==(17), (17)==(10)] └── aggregations ├── const-agg [as=name:2, outer=(2)] │ └── name:2 @@ -2560,16 +2560,16 @@ FROM t_name AS this_; ---- project - ├── columns: id1_0_0_:1!null c_name2_0_0_:2 formula0_0_:6 + ├── columns: id1_0_0_:1!null c_name2_0_0_:2 formula0_0_:8 ├── immutable ├── key: (1) - ├── fd: (1)-->(2), (2)-->(6) + ├── fd: (1)-->(2), (2)-->(8) ├── scan this_ │ ├── columns: this_.id:1!null this_.c_name:2 │ ├── key: (1) │ └── fd: (1)-->(2) └── projections - └── length(this_.c_name:2) [as=formula0_0_:6, outer=(2), immutable] + └── length(this_.c_name:2) [as=formula0_0_:8, outer=(2), immutable] exec-ddl drop table t_name diff --git a/pkg/sql/opt/xform/testdata/external/liquibase b/pkg/sql/opt/xform/testdata/external/liquibase index 5a545945d4d8..a2a9bba34481 100644 --- a/pkg/sql/opt/xform/testdata/external/liquibase +++ b/pkg/sql/opt/xform/testdata/external/liquibase @@ -161,87 +161,87 @@ ON ft.ftserver = fs.oid WHERE ((c.relkind = 'r'::CHAR) OR (c.relkind = 'f'::CHAR)) AND (n.nspname = 'public') ---- project - ├── columns: oid:1!null schemaname:29!null tablename:2!null relacl:26 tableowner:133 description:134 relkind:17!null cluster:92 hasoids:20!null hasindexes:13!null hasrules:22!null tablespace:33 param:27 hastriggers:23!null unlogged:15!null ftoptions:120 srvname:122 reltuples:10!null inhtable:135!null inhschemaname:69 inhtablename:42 + ├── columns: oid:1!null schemaname:30!null tablename:2!null relacl:26 tableowner:144 description:145 relkind:17!null cluster:99 hasoids:20!null hasindexes:13!null hasrules:22!null tablespace:35 param:27 hastriggers:23!null unlogged:15!null ftoptions:128 srvname:131 reltuples:10!null inhtable:146!null inhschemaname:74 inhtablename:46 ├── stable - ├── fd: ()-->(29), (1)-->(2,10,13,15,17,20,22,23,26,27,33,133,134), (2)-->(1,10,13,15,17,20,22,23,26,27,33,133,134) + ├── fd: ()-->(30), (1)-->(2,10,13,15,17,20,22,23,26,27,35,144,145), (2)-->(1,10,13,15,17,20,22,23,26,27,35,144,145) ├── group-by - │ ├── columns: c.oid:1!null c.relname:2!null c.relowner:5!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.nspname:29!null spcname:33 c2.relname:42 n2.nspname:69 ci.relname:92 ftoptions:120 srvname:122 count_rows:132!null rownum:136!null - │ ├── grouping columns: rownum:136!null - │ ├── key: (136) - │ ├── fd: ()-->(29), (1)-->(2,5,10,13,15,17,20,22,23,26,27,33), (2)-->(1,5,10,13,15,17,20,22,23,26,27,33), (136)-->(1,2,5,10,13,15,17,20,22,23,26,27,29,33,42,69,92,120,122,132) + │ ├── columns: c.oid:1!null c.relname:2!null c.relowner:5!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.nspname:30!null spcname:35 c2.relname:46 n2.nspname:74 ci.relname:99 ftoptions:128 srvname:131 count_rows:143!null rownum:147!null + │ ├── grouping columns: rownum:147!null + │ ├── key: (147) + │ ├── fd: ()-->(30), (1)-->(2,5,10,13,15,17,20,22,23,26,27,35), (2)-->(1,5,10,13,15,17,20,22,23,26,27,35), (147)-->(1,2,5,10,13,15,17,20,22,23,26,27,30,35,46,74,99,128,131,143) │ ├── right-join (hash) - │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:28!null n.nspname:29!null t.oid:32 spcname:33 i.inhrelid:38 i.inhparent:39 c2.oid:41 c2.relname:42 c2.relnamespace:43 n2.oid:68 n2.nspname:69 indexrelid:72 indrelid:73 indisclustered:79 ci.oid:91 ci.relname:92 ftrelid:118 ftserver:119 ftoptions:120 fs.oid:121 srvname:122 pg_inherits.inhparent:130 rownum:136!null - │ │ ├── fd: ()-->(3,28,29), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,32,33), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(28), (28)==(3), (32)-->(33), (33)-->(32), (41)-->(42,43), (42,43)-->(41), (39)==(41), (41)==(39), (68)~~>(69), (69)~~>(68), (72)-->(73), (91)-->(92), (118)-->(119,120), (121)-->(122), (122)-->(121), (136)-->(1,2,5,8,10,13,15,17,20,22,23,26,27,32,33,38,39,41-43,68,69,72,73,79,91,92,118-122) + │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:29!null n.nspname:30!null t.oid:34 spcname:35 i.inhrelid:41 i.inhparent:42 c2.oid:45 c2.relname:46 c2.relnamespace:47 n2.oid:73 n2.nspname:74 indexrelid:78 indrelid:79 indisclustered:85 ci.oid:98 ci.relname:99 ftrelid:126 ftserver:127 ftoptions:128 fs.oid:130 srvname:131 pg_inherits.inhparent:140 rownum:147!null + │ │ ├── fd: ()-->(3,29,30), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,34,35), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(29), (29)==(3), (34)-->(35), (35)-->(34), (45)-->(46,47), (46,47)-->(45), (42)==(45), (45)==(42), (73)~~>(74), (74)~~>(73), (78)-->(79), (98)-->(99), (126)-->(127,128), (130)-->(131), (131)-->(130), (147)-->(1,2,5,8,10,13,15,17,20,22,23,26,27,34,35,41,42,45-47,73,74,78,79,85,98,99,126-128,130,131) │ │ ├── scan pg_inherits - │ │ │ └── columns: pg_inherits.inhparent:130!null + │ │ │ └── columns: pg_inherits.inhparent:140!null │ │ ├── ordinality - │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:28!null n.nspname:29!null t.oid:32 spcname:33 i.inhrelid:38 i.inhparent:39 c2.oid:41 c2.relname:42 c2.relnamespace:43 n2.oid:68 n2.nspname:69 indexrelid:72 indrelid:73 indisclustered:79 ci.oid:91 ci.relname:92 ftrelid:118 ftserver:119 ftoptions:120 fs.oid:121 srvname:122 rownum:136!null - │ │ │ ├── key: (136) - │ │ │ ├── fd: ()-->(3,28,29), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,32,33), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(28), (28)==(3), (32)-->(33), (33)-->(32), (41)-->(42,43), (42,43)-->(41), (39)==(41), (41)==(39), (68)~~>(69), (69)~~>(68), (72)-->(73), (91)-->(92), (118)-->(119,120), (121)-->(122), (122)-->(121), (136)-->(1-3,5,8,10,13,15,17,20,22,23,26-29,32,33,38,39,41-43,68,69,72,73,79,91,92,118-122) + │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:29!null n.nspname:30!null t.oid:34 spcname:35 i.inhrelid:41 i.inhparent:42 c2.oid:45 c2.relname:46 c2.relnamespace:47 n2.oid:73 n2.nspname:74 indexrelid:78 indrelid:79 indisclustered:85 ci.oid:98 ci.relname:99 ftrelid:126 ftserver:127 ftoptions:128 fs.oid:130 srvname:131 rownum:147!null + │ │ │ ├── key: (147) + │ │ │ ├── fd: ()-->(3,29,30), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,34,35), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(29), (29)==(3), (34)-->(35), (35)-->(34), (45)-->(46,47), (46,47)-->(45), (42)==(45), (45)==(42), (73)~~>(74), (74)~~>(73), (78)-->(79), (98)-->(99), (126)-->(127,128), (130)-->(131), (131)-->(130), (147)-->(1-3,5,8,10,13,15,17,20,22,23,26,27,29,30,34,35,41,42,45-47,73,74,78,79,85,98,99,126-128,130,131) │ │ │ └── left-join (lookup pg_foreign_server) - │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:28!null n.nspname:29!null t.oid:32 spcname:33 i.inhrelid:38 i.inhparent:39 c2.oid:41 c2.relname:42 c2.relnamespace:43 n2.oid:68 n2.nspname:69 indexrelid:72 indrelid:73 indisclustered:79 ci.oid:91 ci.relname:92 ftrelid:118 ftserver:119 ftoptions:120 fs.oid:121 srvname:122 - │ │ │ ├── key columns: [119] = [121] + │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:29!null n.nspname:30!null t.oid:34 spcname:35 i.inhrelid:41 i.inhparent:42 c2.oid:45 c2.relname:46 c2.relnamespace:47 n2.oid:73 n2.nspname:74 indexrelid:78 indrelid:79 indisclustered:85 ci.oid:98 ci.relname:99 ftrelid:126 ftserver:127 ftoptions:128 fs.oid:130 srvname:131 + │ │ │ ├── key columns: [127] = [130] │ │ │ ├── lookup columns are key - │ │ │ ├── fd: ()-->(3,28,29), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,32,33), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(28), (28)==(3), (32)-->(33), (33)-->(32), (41)-->(42,43), (42,43)-->(41), (39)==(41), (41)==(39), (68)~~>(69), (69)~~>(68), (72)-->(73), (91)-->(92), (118)-->(119,120), (121)-->(122), (122)-->(121) + │ │ │ ├── fd: ()-->(3,29,30), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,34,35), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(29), (29)==(3), (34)-->(35), (35)-->(34), (45)-->(46,47), (46,47)-->(45), (42)==(45), (45)==(42), (73)~~>(74), (74)~~>(73), (78)-->(79), (98)-->(99), (126)-->(127,128), (130)-->(131), (131)-->(130) │ │ │ ├── left-join (lookup pg_foreign_table) - │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:28!null n.nspname:29!null t.oid:32 spcname:33 i.inhrelid:38 i.inhparent:39 c2.oid:41 c2.relname:42 c2.relnamespace:43 n2.oid:68 n2.nspname:69 indexrelid:72 indrelid:73 indisclustered:79 ci.oid:91 ci.relname:92 ftrelid:118 ftserver:119 ftoptions:120 - │ │ │ │ ├── key columns: [1] = [118] + │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:29!null n.nspname:30!null t.oid:34 spcname:35 i.inhrelid:41 i.inhparent:42 c2.oid:45 c2.relname:46 c2.relnamespace:47 n2.oid:73 n2.nspname:74 indexrelid:78 indrelid:79 indisclustered:85 ci.oid:98 ci.relname:99 ftrelid:126 ftserver:127 ftoptions:128 + │ │ │ │ ├── key columns: [1] = [126] │ │ │ │ ├── lookup columns are key - │ │ │ │ ├── fd: ()-->(3,28,29), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,32,33), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(28), (28)==(3), (32)-->(33), (33)-->(32), (41)-->(42,43), (42,43)-->(41), (39)==(41), (41)==(39), (68)~~>(69), (69)~~>(68), (72)-->(73), (91)-->(92), (118)-->(119,120) + │ │ │ │ ├── fd: ()-->(3,29,30), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,34,35), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(29), (29)==(3), (34)-->(35), (35)-->(34), (45)-->(46,47), (46,47)-->(45), (42)==(45), (45)==(42), (73)~~>(74), (74)~~>(73), (78)-->(79), (98)-->(99), (126)-->(127,128) │ │ │ │ ├── left-join (lookup pg_class) - │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:28!null n.nspname:29!null t.oid:32 spcname:33 i.inhrelid:38 i.inhparent:39 c2.oid:41 c2.relname:42 c2.relnamespace:43 n2.oid:68 n2.nspname:69 indexrelid:72 indrelid:73 indisclustered:79 ci.oid:91 ci.relname:92 - │ │ │ │ │ ├── key columns: [72] = [91] + │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:29!null n.nspname:30!null t.oid:34 spcname:35 i.inhrelid:41 i.inhparent:42 c2.oid:45 c2.relname:46 c2.relnamespace:47 n2.oid:73 n2.nspname:74 indexrelid:78 indrelid:79 indisclustered:85 ci.oid:98 ci.relname:99 + │ │ │ │ │ ├── key columns: [78] = [98] │ │ │ │ │ ├── lookup columns are key - │ │ │ │ │ ├── fd: ()-->(3,28,29), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,32,33), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(28), (28)==(3), (32)-->(33), (33)-->(32), (41)-->(42,43), (42,43)-->(41), (39)==(41), (41)==(39), (68)~~>(69), (69)~~>(68), (72)-->(73), (91)-->(92) + │ │ │ │ │ ├── fd: ()-->(3,29,30), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,34,35), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(29), (29)==(3), (34)-->(35), (35)-->(34), (45)-->(46,47), (46,47)-->(45), (42)==(45), (45)==(42), (73)~~>(74), (74)~~>(73), (78)-->(79), (98)-->(99) │ │ │ │ │ ├── right-join (hash) - │ │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:28!null n.nspname:29!null t.oid:32 spcname:33 i.inhrelid:38 i.inhparent:39 c2.oid:41 c2.relname:42 c2.relnamespace:43 n2.oid:68 n2.nspname:69 indexrelid:72 indrelid:73 indisclustered:79 - │ │ │ │ │ │ ├── fd: ()-->(3,28,29), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,32,33), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(28), (28)==(3), (32)-->(33), (33)-->(32), (41)-->(42,43), (42,43)-->(41), (39)==(41), (41)==(39), (68)~~>(69), (69)~~>(68), (72)-->(73) + │ │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:29!null n.nspname:30!null t.oid:34 spcname:35 i.inhrelid:41 i.inhparent:42 c2.oid:45 c2.relname:46 c2.relnamespace:47 n2.oid:73 n2.nspname:74 indexrelid:78 indrelid:79 indisclustered:85 + │ │ │ │ │ │ ├── fd: ()-->(3,29,30), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,34,35), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(29), (29)==(3), (34)-->(35), (35)-->(34), (45)-->(46,47), (46,47)-->(45), (42)==(45), (45)==(42), (73)~~>(74), (74)~~>(73), (78)-->(79) │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: indexrelid:72!null indrelid:73!null indisclustered:79!null - │ │ │ │ │ │ │ ├── key: (72) - │ │ │ │ │ │ │ ├── fd: ()-->(79), (72)-->(73) + │ │ │ │ │ │ │ ├── columns: indexrelid:78!null indrelid:79!null indisclustered:85!null + │ │ │ │ │ │ │ ├── key: (78) + │ │ │ │ │ │ │ ├── fd: ()-->(85), (78)-->(79) │ │ │ │ │ │ │ ├── scan ind - │ │ │ │ │ │ │ │ ├── columns: indexrelid:72!null indrelid:73!null indisclustered:79!null - │ │ │ │ │ │ │ │ ├── key: (72) - │ │ │ │ │ │ │ │ └── fd: (72)-->(73,79) + │ │ │ │ │ │ │ │ ├── columns: indexrelid:78!null indrelid:79!null indisclustered:85!null + │ │ │ │ │ │ │ │ ├── key: (78) + │ │ │ │ │ │ │ │ └── fd: (78)-->(79,85) │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── indisclustered:79 = true [outer=(79), constraints=(/79: [/true - /true]; tight), fd=()-->(79)] + │ │ │ │ │ │ │ └── indisclustered:85 = true [outer=(85), constraints=(/85: [/true - /true]; tight), fd=()-->(85)] │ │ │ │ │ │ ├── right-join (hash) - │ │ │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:28!null n.nspname:29!null t.oid:32 spcname:33 i.inhrelid:38 i.inhparent:39 c2.oid:41 c2.relname:42 c2.relnamespace:43 n2.oid:68 n2.nspname:69 - │ │ │ │ │ │ │ ├── fd: ()-->(3,28,29), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,32,33), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(28), (28)==(3), (32)-->(33), (33)-->(32), (41)-->(42,43), (42,43)-->(41), (39)==(41), (41)==(39), (68)~~>(69), (69)~~>(68) + │ │ │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:29!null n.nspname:30!null t.oid:34 spcname:35 i.inhrelid:41 i.inhparent:42 c2.oid:45 c2.relname:46 c2.relnamespace:47 n2.oid:73 n2.nspname:74 + │ │ │ │ │ │ │ ├── fd: ()-->(3,29,30), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,34,35), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(29), (29)==(3), (34)-->(35), (35)-->(34), (45)-->(46,47), (46,47)-->(45), (42)==(45), (45)==(42), (73)~~>(74), (74)~~>(73) │ │ │ │ │ │ │ ├── left-join (hash) - │ │ │ │ │ │ │ │ ├── columns: i.inhrelid:38!null i.inhparent:39!null c2.oid:41!null c2.relname:42!null c2.relnamespace:43!null n2.oid:68 n2.nspname:69 + │ │ │ │ │ │ │ │ ├── columns: i.inhrelid:41!null i.inhparent:42!null c2.oid:45!null c2.relname:46!null c2.relnamespace:47!null n2.oid:73 n2.nspname:74 │ │ │ │ │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ │ │ │ │ │ │ │ ├── fd: (41)-->(42,43), (42,43)-->(41), (39)==(41), (41)==(39), (68)-->(69), (69)-->(68) + │ │ │ │ │ │ │ │ ├── fd: (45)-->(46,47), (46,47)-->(45), (42)==(45), (45)==(42), (73)-->(74), (74)-->(73) │ │ │ │ │ │ │ │ ├── inner-join (hash) - │ │ │ │ │ │ │ │ │ ├── columns: i.inhrelid:38!null i.inhparent:39!null c2.oid:41!null c2.relname:42!null c2.relnamespace:43!null + │ │ │ │ │ │ │ │ │ ├── columns: i.inhrelid:41!null i.inhparent:42!null c2.oid:45!null c2.relname:46!null c2.relnamespace:47!null │ │ │ │ │ │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ │ │ │ │ │ │ ├── fd: (41)-->(42,43), (42,43)-->(41), (39)==(41), (41)==(39) + │ │ │ │ │ │ │ │ │ ├── fd: (45)-->(46,47), (46,47)-->(45), (42)==(45), (45)==(42) │ │ │ │ │ │ │ │ │ ├── scan i - │ │ │ │ │ │ │ │ │ │ └── columns: i.inhrelid:38!null i.inhparent:39!null + │ │ │ │ │ │ │ │ │ │ └── columns: i.inhrelid:41!null i.inhparent:42!null │ │ │ │ │ │ │ │ │ ├── scan c2@pg_class_relname_nsp_index - │ │ │ │ │ │ │ │ │ │ ├── columns: c2.oid:41!null c2.relname:42!null c2.relnamespace:43!null - │ │ │ │ │ │ │ │ │ │ ├── key: (41) - │ │ │ │ │ │ │ │ │ │ └── fd: (41)-->(42,43), (42,43)-->(41) + │ │ │ │ │ │ │ │ │ │ ├── columns: c2.oid:45!null c2.relname:46!null c2.relnamespace:47!null + │ │ │ │ │ │ │ │ │ │ ├── key: (45) + │ │ │ │ │ │ │ │ │ │ └── fd: (45)-->(46,47), (46,47)-->(45) │ │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ │ └── i.inhparent:39 = c2.oid:41 [outer=(39,41), constraints=(/39: (/NULL - ]; /41: (/NULL - ]), fd=(39)==(41), (41)==(39)] + │ │ │ │ │ │ │ │ │ └── i.inhparent:42 = c2.oid:45 [outer=(42,45), constraints=(/42: (/NULL - ]; /45: (/NULL - ]), fd=(42)==(45), (45)==(42)] │ │ │ │ │ │ │ │ ├── scan n2@pg_namespace_nspname_index - │ │ │ │ │ │ │ │ │ ├── columns: n2.oid:68!null n2.nspname:69!null - │ │ │ │ │ │ │ │ │ ├── key: (68) - │ │ │ │ │ │ │ │ │ └── fd: (68)-->(69), (69)-->(68) + │ │ │ │ │ │ │ │ │ ├── columns: n2.oid:73!null n2.nspname:74!null + │ │ │ │ │ │ │ │ │ ├── key: (73) + │ │ │ │ │ │ │ │ │ └── fd: (73)-->(74), (74)-->(73) │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ └── n2.oid:68 = c2.relnamespace:43 [outer=(43,68), constraints=(/43: (/NULL - ]; /68: (/NULL - ]), fd=(43)==(68), (68)==(43)] + │ │ │ │ │ │ │ │ └── n2.oid:73 = c2.relnamespace:47 [outer=(47,73), constraints=(/47: (/NULL - ]; /73: (/NULL - ]), fd=(47)==(73), (73)==(47)] │ │ │ │ │ │ │ ├── left-join (lookup pg_tablespace) - │ │ │ │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:28!null n.nspname:29!null t.oid:32 spcname:33 - │ │ │ │ │ │ │ │ ├── key columns: [8] = [32] + │ │ │ │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:29!null n.nspname:30!null t.oid:34 spcname:35 + │ │ │ │ │ │ │ │ ├── key columns: [8] = [34] │ │ │ │ │ │ │ │ ├── lookup columns are key │ │ │ │ │ │ │ │ ├── key: (1) - │ │ │ │ │ │ │ │ ├── fd: ()-->(3,28,29), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,32,33), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(28), (28)==(3), (32)-->(33), (33)-->(32) + │ │ │ │ │ │ │ │ ├── fd: ()-->(3,29,30), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,34,35), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(29), (29)==(3), (34)-->(35), (35)-->(34) │ │ │ │ │ │ │ │ ├── inner-join (hash) - │ │ │ │ │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:28!null n.nspname:29!null + │ │ │ │ │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:29!null n.nspname:30!null │ │ │ │ │ │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) │ │ │ │ │ │ │ │ │ ├── key: (1) - │ │ │ │ │ │ │ │ │ ├── fd: ()-->(3,28,29), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(28), (28)==(3) + │ │ │ │ │ │ │ │ │ ├── fd: ()-->(3,29,30), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(29), (29)==(3) │ │ │ │ │ │ │ │ │ ├── select │ │ │ │ │ │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 │ │ │ │ │ │ │ │ │ │ ├── key: (1) @@ -253,26 +253,26 @@ project │ │ │ │ │ │ │ │ │ │ └── filters │ │ │ │ │ │ │ │ │ │ └── (c.relkind:17 = 'r') OR (c.relkind:17 = 'f') [outer=(17), constraints=(/17: [/'f' - /'f'] [/'r' - /'r']; tight)] │ │ │ │ │ │ │ │ │ ├── scan n@pg_namespace_nspname_index - │ │ │ │ │ │ │ │ │ │ ├── columns: n.oid:28!null n.nspname:29!null - │ │ │ │ │ │ │ │ │ │ ├── constraint: /29: [/'public' - /'public'] + │ │ │ │ │ │ │ │ │ │ ├── columns: n.oid:29!null n.nspname:30!null + │ │ │ │ │ │ │ │ │ │ ├── constraint: /30: [/'public' - /'public'] │ │ │ │ │ │ │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ │ │ │ │ │ │ ├── key: () - │ │ │ │ │ │ │ │ │ │ └── fd: ()-->(28,29) + │ │ │ │ │ │ │ │ │ │ └── fd: ()-->(29,30) │ │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ │ └── n.oid:28 = c.relnamespace:3 [outer=(3,28), constraints=(/3: (/NULL - ]; /28: (/NULL - ]), fd=(3)==(28), (28)==(3)] + │ │ │ │ │ │ │ │ │ └── n.oid:29 = c.relnamespace:3 [outer=(3,29), constraints=(/3: (/NULL - ]; /29: (/NULL - ]), fd=(3)==(29), (29)==(3)] │ │ │ │ │ │ │ │ └── filters (true) │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── i.inhrelid:38 = c.oid:1 [outer=(1,38), constraints=(/1: (/NULL - ]; /38: (/NULL - ]), fd=(1)==(38), (38)==(1)] + │ │ │ │ │ │ │ └── i.inhrelid:41 = c.oid:1 [outer=(1,41), constraints=(/1: (/NULL - ]; /41: (/NULL - ]), fd=(1)==(41), (41)==(1)] │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── indrelid:73 = c.oid:1 [outer=(1,73), constraints=(/1: (/NULL - ]; /73: (/NULL - ]), fd=(1)==(73), (73)==(1)] + │ │ │ │ │ │ └── indrelid:79 = c.oid:1 [outer=(1,79), constraints=(/1: (/NULL - ]; /79: (/NULL - ]), fd=(1)==(79), (79)==(1)] │ │ │ │ │ └── filters (true) │ │ │ │ └── filters (true) │ │ │ └── filters (true) │ │ └── filters - │ │ └── pg_inherits.inhparent:130 = c.oid:1 [outer=(1,130), constraints=(/1: (/NULL - ]; /130: (/NULL - ]), fd=(1)==(130), (130)==(1)] + │ │ └── pg_inherits.inhparent:140 = c.oid:1 [outer=(1,140), constraints=(/1: (/NULL - ]; /140: (/NULL - ]), fd=(1)==(140), (140)==(1)] │ └── aggregations - │ ├── count [as=count_rows:132, outer=(130)] - │ │ └── pg_inherits.inhparent:130 + │ ├── count [as=count_rows:143, outer=(140)] + │ │ └── pg_inherits.inhparent:140 │ ├── const-agg [as=c.oid:1, outer=(1)] │ │ └── c.oid:1 │ ├── const-agg [as=c.relname:2, outer=(2)] @@ -297,21 +297,21 @@ project │ │ └── c.relacl:26 │ ├── const-agg [as=c.reloptions:27, outer=(27)] │ │ └── c.reloptions:27 - │ ├── const-agg [as=n.nspname:29, outer=(29)] - │ │ └── n.nspname:29 - │ ├── const-agg [as=spcname:33, outer=(33)] - │ │ └── spcname:33 - │ ├── const-agg [as=c2.relname:42, outer=(42)] - │ │ └── c2.relname:42 - │ ├── const-agg [as=n2.nspname:69, outer=(69)] - │ │ └── n2.nspname:69 - │ ├── const-agg [as=ci.relname:92, outer=(92)] - │ │ └── ci.relname:92 - │ ├── const-agg [as=ftoptions:120, outer=(120)] - │ │ └── ftoptions:120 - │ └── const-agg [as=srvname:122, outer=(122)] - │ └── srvname:122 + │ ├── const-agg [as=n.nspname:30, outer=(30)] + │ │ └── n.nspname:30 + │ ├── const-agg [as=spcname:35, outer=(35)] + │ │ └── spcname:35 + │ ├── const-agg [as=c2.relname:46, outer=(46)] + │ │ └── c2.relname:46 + │ ├── const-agg [as=n2.nspname:74, outer=(74)] + │ │ └── n2.nspname:74 + │ ├── const-agg [as=ci.relname:99, outer=(99)] + │ │ └── ci.relname:99 + │ ├── const-agg [as=ftoptions:128, outer=(128)] + │ │ └── ftoptions:128 + │ └── const-agg [as=srvname:131, outer=(131)] + │ └── srvname:131 └── projections - ├── pg_get_userbyid(c.relowner:5) [as=tableowner:133, outer=(5), stable] - ├── obj_description(c.oid:1) [as=description:134, outer=(1), stable] - └── count_rows:132 > 0 [as=inhtable:135, outer=(132)] + ├── pg_get_userbyid(c.relowner:5) [as=tableowner:144, outer=(5), stable] + ├── obj_description(c.oid:1) [as=description:145, outer=(1), stable] + └── count_rows:143 > 0 [as=inhtable:146, outer=(143)] diff --git a/pkg/sql/opt/xform/testdata/external/navicat b/pkg/sql/opt/xform/testdata/external/navicat index cfc659e2bdcb..58fb47fe56a8 100644 --- a/pkg/sql/opt/xform/testdata/external/navicat +++ b/pkg/sql/opt/xform/testdata/external/navicat @@ -160,92 +160,92 @@ WHERE ((c.relkind = 'r'::CHAR) OR (c.relkind = 'f'::CHAR)) AND (n.nspname = 'pub ORDER BY schemaname, tablename ---- sort - ├── columns: oid:1!null schemaname:29!null tablename:2!null relacl:26 tableowner:133 description:134 relkind:17!null cluster:92 hasoids:20!null hasindexes:13!null hasrules:22!null tablespace:33 param:27 hastriggers:23!null unlogged:15!null ftoptions:120 srvname:122 reltuples:10!null inhtable:135!null inhschemaname:69 inhtablename:42 + ├── columns: oid:1!null schemaname:30!null tablename:2!null relacl:26 tableowner:144 description:145 relkind:17!null cluster:99 hasoids:20!null hasindexes:13!null hasrules:22!null tablespace:35 param:27 hastriggers:23!null unlogged:15!null ftoptions:128 srvname:131 reltuples:10!null inhtable:146!null inhschemaname:74 inhtablename:46 ├── stable - ├── fd: ()-->(29), (1)-->(2,10,13,15,17,20,22,23,26,27,33,133,134), (2)-->(1,10,13,15,17,20,22,23,26,27,33,133,134) - ├── ordering: +2 opt(29) [actual: +2] + ├── fd: ()-->(30), (1)-->(2,10,13,15,17,20,22,23,26,27,35,144,145), (2)-->(1,10,13,15,17,20,22,23,26,27,35,144,145) + ├── ordering: +2 opt(30) [actual: +2] └── project - ├── columns: tableowner:133 description:134 inhtable:135!null c.oid:1!null c.relname:2!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.nspname:29!null spcname:33 c2.relname:42 n2.nspname:69 ci.relname:92 ftoptions:120 srvname:122 + ├── columns: tableowner:144 description:145 inhtable:146!null c.oid:1!null c.relname:2!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.nspname:30!null spcname:35 c2.relname:46 n2.nspname:74 ci.relname:99 ftoptions:128 srvname:131 ├── stable - ├── fd: ()-->(29), (1)-->(2,10,13,15,17,20,22,23,26,27,33,133,134), (2)-->(1,10,13,15,17,20,22,23,26,27,33,133,134) + ├── fd: ()-->(30), (1)-->(2,10,13,15,17,20,22,23,26,27,35,144,145), (2)-->(1,10,13,15,17,20,22,23,26,27,35,144,145) ├── group-by - │ ├── columns: c.oid:1!null c.relname:2!null c.relowner:5!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.nspname:29!null spcname:33 c2.relname:42 n2.nspname:69 ci.relname:92 ftoptions:120 srvname:122 count_rows:132!null rownum:136!null - │ ├── grouping columns: rownum:136!null - │ ├── key: (136) - │ ├── fd: ()-->(29), (1)-->(2,5,10,13,15,17,20,22,23,26,27,33), (2)-->(1,5,10,13,15,17,20,22,23,26,27,33), (136)-->(1,2,5,10,13,15,17,20,22,23,26,27,29,33,42,69,92,120,122,132) + │ ├── columns: c.oid:1!null c.relname:2!null c.relowner:5!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.nspname:30!null spcname:35 c2.relname:46 n2.nspname:74 ci.relname:99 ftoptions:128 srvname:131 count_rows:143!null rownum:147!null + │ ├── grouping columns: rownum:147!null + │ ├── key: (147) + │ ├── fd: ()-->(30), (1)-->(2,5,10,13,15,17,20,22,23,26,27,35), (2)-->(1,5,10,13,15,17,20,22,23,26,27,35), (147)-->(1,2,5,10,13,15,17,20,22,23,26,27,30,35,46,74,99,128,131,143) │ ├── right-join (hash) - │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:28!null n.nspname:29!null t.oid:32 spcname:33 i.inhrelid:38 i.inhparent:39 c2.oid:41 c2.relname:42 c2.relnamespace:43 n2.oid:68 n2.nspname:69 indexrelid:72 indrelid:73 indisclustered:79 ci.oid:91 ci.relname:92 ftrelid:118 ftserver:119 ftoptions:120 fs.oid:121 srvname:122 pg_inherits.inhparent:130 rownum:136!null - │ │ ├── fd: ()-->(3,28,29), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,32,33), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(28), (28)==(3), (32)-->(33), (33)-->(32), (41)-->(42,43), (42,43)-->(41), (39)==(41), (41)==(39), (68)~~>(69), (69)~~>(68), (72)-->(73), (91)-->(92), (118)-->(119,120), (121)-->(122), (122)-->(121), (136)-->(1,2,5,8,10,13,15,17,20,22,23,26,27,32,33,38,39,41-43,68,69,72,73,79,91,92,118-122) + │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:29!null n.nspname:30!null t.oid:34 spcname:35 i.inhrelid:41 i.inhparent:42 c2.oid:45 c2.relname:46 c2.relnamespace:47 n2.oid:73 n2.nspname:74 indexrelid:78 indrelid:79 indisclustered:85 ci.oid:98 ci.relname:99 ftrelid:126 ftserver:127 ftoptions:128 fs.oid:130 srvname:131 pg_inherits.inhparent:140 rownum:147!null + │ │ ├── fd: ()-->(3,29,30), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,34,35), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(29), (29)==(3), (34)-->(35), (35)-->(34), (45)-->(46,47), (46,47)-->(45), (42)==(45), (45)==(42), (73)~~>(74), (74)~~>(73), (78)-->(79), (98)-->(99), (126)-->(127,128), (130)-->(131), (131)-->(130), (147)-->(1,2,5,8,10,13,15,17,20,22,23,26,27,34,35,41,42,45-47,73,74,78,79,85,98,99,126-128,130,131) │ │ ├── scan pg_inherits - │ │ │ └── columns: pg_inherits.inhparent:130!null + │ │ │ └── columns: pg_inherits.inhparent:140!null │ │ ├── ordinality - │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:28!null n.nspname:29!null t.oid:32 spcname:33 i.inhrelid:38 i.inhparent:39 c2.oid:41 c2.relname:42 c2.relnamespace:43 n2.oid:68 n2.nspname:69 indexrelid:72 indrelid:73 indisclustered:79 ci.oid:91 ci.relname:92 ftrelid:118 ftserver:119 ftoptions:120 fs.oid:121 srvname:122 rownum:136!null - │ │ │ ├── key: (136) - │ │ │ ├── fd: ()-->(3,28,29), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,32,33), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(28), (28)==(3), (32)-->(33), (33)-->(32), (41)-->(42,43), (42,43)-->(41), (39)==(41), (41)==(39), (68)~~>(69), (69)~~>(68), (72)-->(73), (91)-->(92), (118)-->(119,120), (121)-->(122), (122)-->(121), (136)-->(1-3,5,8,10,13,15,17,20,22,23,26-29,32,33,38,39,41-43,68,69,72,73,79,91,92,118-122) + │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:29!null n.nspname:30!null t.oid:34 spcname:35 i.inhrelid:41 i.inhparent:42 c2.oid:45 c2.relname:46 c2.relnamespace:47 n2.oid:73 n2.nspname:74 indexrelid:78 indrelid:79 indisclustered:85 ci.oid:98 ci.relname:99 ftrelid:126 ftserver:127 ftoptions:128 fs.oid:130 srvname:131 rownum:147!null + │ │ │ ├── key: (147) + │ │ │ ├── fd: ()-->(3,29,30), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,34,35), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(29), (29)==(3), (34)-->(35), (35)-->(34), (45)-->(46,47), (46,47)-->(45), (42)==(45), (45)==(42), (73)~~>(74), (74)~~>(73), (78)-->(79), (98)-->(99), (126)-->(127,128), (130)-->(131), (131)-->(130), (147)-->(1-3,5,8,10,13,15,17,20,22,23,26,27,29,30,34,35,41,42,45-47,73,74,78,79,85,98,99,126-128,130,131) │ │ │ └── left-join (lookup pg_foreign_server) - │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:28!null n.nspname:29!null t.oid:32 spcname:33 i.inhrelid:38 i.inhparent:39 c2.oid:41 c2.relname:42 c2.relnamespace:43 n2.oid:68 n2.nspname:69 indexrelid:72 indrelid:73 indisclustered:79 ci.oid:91 ci.relname:92 ftrelid:118 ftserver:119 ftoptions:120 fs.oid:121 srvname:122 - │ │ │ ├── key columns: [119] = [121] + │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:29!null n.nspname:30!null t.oid:34 spcname:35 i.inhrelid:41 i.inhparent:42 c2.oid:45 c2.relname:46 c2.relnamespace:47 n2.oid:73 n2.nspname:74 indexrelid:78 indrelid:79 indisclustered:85 ci.oid:98 ci.relname:99 ftrelid:126 ftserver:127 ftoptions:128 fs.oid:130 srvname:131 + │ │ │ ├── key columns: [127] = [130] │ │ │ ├── lookup columns are key - │ │ │ ├── fd: ()-->(3,28,29), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,32,33), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(28), (28)==(3), (32)-->(33), (33)-->(32), (41)-->(42,43), (42,43)-->(41), (39)==(41), (41)==(39), (68)~~>(69), (69)~~>(68), (72)-->(73), (91)-->(92), (118)-->(119,120), (121)-->(122), (122)-->(121) + │ │ │ ├── fd: ()-->(3,29,30), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,34,35), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(29), (29)==(3), (34)-->(35), (35)-->(34), (45)-->(46,47), (46,47)-->(45), (42)==(45), (45)==(42), (73)~~>(74), (74)~~>(73), (78)-->(79), (98)-->(99), (126)-->(127,128), (130)-->(131), (131)-->(130) │ │ │ ├── left-join (lookup pg_foreign_table) - │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:28!null n.nspname:29!null t.oid:32 spcname:33 i.inhrelid:38 i.inhparent:39 c2.oid:41 c2.relname:42 c2.relnamespace:43 n2.oid:68 n2.nspname:69 indexrelid:72 indrelid:73 indisclustered:79 ci.oid:91 ci.relname:92 ftrelid:118 ftserver:119 ftoptions:120 - │ │ │ │ ├── key columns: [1] = [118] + │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:29!null n.nspname:30!null t.oid:34 spcname:35 i.inhrelid:41 i.inhparent:42 c2.oid:45 c2.relname:46 c2.relnamespace:47 n2.oid:73 n2.nspname:74 indexrelid:78 indrelid:79 indisclustered:85 ci.oid:98 ci.relname:99 ftrelid:126 ftserver:127 ftoptions:128 + │ │ │ │ ├── key columns: [1] = [126] │ │ │ │ ├── lookup columns are key - │ │ │ │ ├── fd: ()-->(3,28,29), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,32,33), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(28), (28)==(3), (32)-->(33), (33)-->(32), (41)-->(42,43), (42,43)-->(41), (39)==(41), (41)==(39), (68)~~>(69), (69)~~>(68), (72)-->(73), (91)-->(92), (118)-->(119,120) + │ │ │ │ ├── fd: ()-->(3,29,30), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,34,35), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(29), (29)==(3), (34)-->(35), (35)-->(34), (45)-->(46,47), (46,47)-->(45), (42)==(45), (45)==(42), (73)~~>(74), (74)~~>(73), (78)-->(79), (98)-->(99), (126)-->(127,128) │ │ │ │ ├── left-join (lookup pg_class) - │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:28!null n.nspname:29!null t.oid:32 spcname:33 i.inhrelid:38 i.inhparent:39 c2.oid:41 c2.relname:42 c2.relnamespace:43 n2.oid:68 n2.nspname:69 indexrelid:72 indrelid:73 indisclustered:79 ci.oid:91 ci.relname:92 - │ │ │ │ │ ├── key columns: [72] = [91] + │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:29!null n.nspname:30!null t.oid:34 spcname:35 i.inhrelid:41 i.inhparent:42 c2.oid:45 c2.relname:46 c2.relnamespace:47 n2.oid:73 n2.nspname:74 indexrelid:78 indrelid:79 indisclustered:85 ci.oid:98 ci.relname:99 + │ │ │ │ │ ├── key columns: [78] = [98] │ │ │ │ │ ├── lookup columns are key - │ │ │ │ │ ├── fd: ()-->(3,28,29), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,32,33), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(28), (28)==(3), (32)-->(33), (33)-->(32), (41)-->(42,43), (42,43)-->(41), (39)==(41), (41)==(39), (68)~~>(69), (69)~~>(68), (72)-->(73), (91)-->(92) + │ │ │ │ │ ├── fd: ()-->(3,29,30), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,34,35), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(29), (29)==(3), (34)-->(35), (35)-->(34), (45)-->(46,47), (46,47)-->(45), (42)==(45), (45)==(42), (73)~~>(74), (74)~~>(73), (78)-->(79), (98)-->(99) │ │ │ │ │ ├── right-join (hash) - │ │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:28!null n.nspname:29!null t.oid:32 spcname:33 i.inhrelid:38 i.inhparent:39 c2.oid:41 c2.relname:42 c2.relnamespace:43 n2.oid:68 n2.nspname:69 indexrelid:72 indrelid:73 indisclustered:79 - │ │ │ │ │ │ ├── fd: ()-->(3,28,29), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,32,33), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(28), (28)==(3), (32)-->(33), (33)-->(32), (41)-->(42,43), (42,43)-->(41), (39)==(41), (41)==(39), (68)~~>(69), (69)~~>(68), (72)-->(73) + │ │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:29!null n.nspname:30!null t.oid:34 spcname:35 i.inhrelid:41 i.inhparent:42 c2.oid:45 c2.relname:46 c2.relnamespace:47 n2.oid:73 n2.nspname:74 indexrelid:78 indrelid:79 indisclustered:85 + │ │ │ │ │ │ ├── fd: ()-->(3,29,30), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,34,35), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(29), (29)==(3), (34)-->(35), (35)-->(34), (45)-->(46,47), (46,47)-->(45), (42)==(45), (45)==(42), (73)~~>(74), (74)~~>(73), (78)-->(79) │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: indexrelid:72!null indrelid:73!null indisclustered:79!null - │ │ │ │ │ │ │ ├── key: (72) - │ │ │ │ │ │ │ ├── fd: ()-->(79), (72)-->(73) + │ │ │ │ │ │ │ ├── columns: indexrelid:78!null indrelid:79!null indisclustered:85!null + │ │ │ │ │ │ │ ├── key: (78) + │ │ │ │ │ │ │ ├── fd: ()-->(85), (78)-->(79) │ │ │ │ │ │ │ ├── scan ind - │ │ │ │ │ │ │ │ ├── columns: indexrelid:72!null indrelid:73!null indisclustered:79!null - │ │ │ │ │ │ │ │ ├── key: (72) - │ │ │ │ │ │ │ │ └── fd: (72)-->(73,79) + │ │ │ │ │ │ │ │ ├── columns: indexrelid:78!null indrelid:79!null indisclustered:85!null + │ │ │ │ │ │ │ │ ├── key: (78) + │ │ │ │ │ │ │ │ └── fd: (78)-->(79,85) │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── indisclustered:79 = true [outer=(79), constraints=(/79: [/true - /true]; tight), fd=()-->(79)] + │ │ │ │ │ │ │ └── indisclustered:85 = true [outer=(85), constraints=(/85: [/true - /true]; tight), fd=()-->(85)] │ │ │ │ │ │ ├── right-join (hash) - │ │ │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:28!null n.nspname:29!null t.oid:32 spcname:33 i.inhrelid:38 i.inhparent:39 c2.oid:41 c2.relname:42 c2.relnamespace:43 n2.oid:68 n2.nspname:69 - │ │ │ │ │ │ │ ├── fd: ()-->(3,28,29), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,32,33), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(28), (28)==(3), (32)-->(33), (33)-->(32), (41)-->(42,43), (42,43)-->(41), (39)==(41), (41)==(39), (68)~~>(69), (69)~~>(68) + │ │ │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:29!null n.nspname:30!null t.oid:34 spcname:35 i.inhrelid:41 i.inhparent:42 c2.oid:45 c2.relname:46 c2.relnamespace:47 n2.oid:73 n2.nspname:74 + │ │ │ │ │ │ │ ├── fd: ()-->(3,29,30), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,34,35), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(29), (29)==(3), (34)-->(35), (35)-->(34), (45)-->(46,47), (46,47)-->(45), (42)==(45), (45)==(42), (73)~~>(74), (74)~~>(73) │ │ │ │ │ │ │ ├── left-join (hash) - │ │ │ │ │ │ │ │ ├── columns: i.inhrelid:38!null i.inhparent:39!null c2.oid:41!null c2.relname:42!null c2.relnamespace:43!null n2.oid:68 n2.nspname:69 + │ │ │ │ │ │ │ │ ├── columns: i.inhrelid:41!null i.inhparent:42!null c2.oid:45!null c2.relname:46!null c2.relnamespace:47!null n2.oid:73 n2.nspname:74 │ │ │ │ │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ │ │ │ │ │ │ │ ├── fd: (41)-->(42,43), (42,43)-->(41), (39)==(41), (41)==(39), (68)-->(69), (69)-->(68) + │ │ │ │ │ │ │ │ ├── fd: (45)-->(46,47), (46,47)-->(45), (42)==(45), (45)==(42), (73)-->(74), (74)-->(73) │ │ │ │ │ │ │ │ ├── inner-join (hash) - │ │ │ │ │ │ │ │ │ ├── columns: i.inhrelid:38!null i.inhparent:39!null c2.oid:41!null c2.relname:42!null c2.relnamespace:43!null + │ │ │ │ │ │ │ │ │ ├── columns: i.inhrelid:41!null i.inhparent:42!null c2.oid:45!null c2.relname:46!null c2.relnamespace:47!null │ │ │ │ │ │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ │ │ │ │ │ │ ├── fd: (41)-->(42,43), (42,43)-->(41), (39)==(41), (41)==(39) + │ │ │ │ │ │ │ │ │ ├── fd: (45)-->(46,47), (46,47)-->(45), (42)==(45), (45)==(42) │ │ │ │ │ │ │ │ │ ├── scan i - │ │ │ │ │ │ │ │ │ │ └── columns: i.inhrelid:38!null i.inhparent:39!null + │ │ │ │ │ │ │ │ │ │ └── columns: i.inhrelid:41!null i.inhparent:42!null │ │ │ │ │ │ │ │ │ ├── scan c2@pg_class_relname_nsp_index - │ │ │ │ │ │ │ │ │ │ ├── columns: c2.oid:41!null c2.relname:42!null c2.relnamespace:43!null - │ │ │ │ │ │ │ │ │ │ ├── key: (41) - │ │ │ │ │ │ │ │ │ │ └── fd: (41)-->(42,43), (42,43)-->(41) + │ │ │ │ │ │ │ │ │ │ ├── columns: c2.oid:45!null c2.relname:46!null c2.relnamespace:47!null + │ │ │ │ │ │ │ │ │ │ ├── key: (45) + │ │ │ │ │ │ │ │ │ │ └── fd: (45)-->(46,47), (46,47)-->(45) │ │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ │ └── i.inhparent:39 = c2.oid:41 [outer=(39,41), constraints=(/39: (/NULL - ]; /41: (/NULL - ]), fd=(39)==(41), (41)==(39)] + │ │ │ │ │ │ │ │ │ └── i.inhparent:42 = c2.oid:45 [outer=(42,45), constraints=(/42: (/NULL - ]; /45: (/NULL - ]), fd=(42)==(45), (45)==(42)] │ │ │ │ │ │ │ │ ├── scan n2@pg_namespace_nspname_index - │ │ │ │ │ │ │ │ │ ├── columns: n2.oid:68!null n2.nspname:69!null - │ │ │ │ │ │ │ │ │ ├── key: (68) - │ │ │ │ │ │ │ │ │ └── fd: (68)-->(69), (69)-->(68) + │ │ │ │ │ │ │ │ │ ├── columns: n2.oid:73!null n2.nspname:74!null + │ │ │ │ │ │ │ │ │ ├── key: (73) + │ │ │ │ │ │ │ │ │ └── fd: (73)-->(74), (74)-->(73) │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ └── n2.oid:68 = c2.relnamespace:43 [outer=(43,68), constraints=(/43: (/NULL - ]; /68: (/NULL - ]), fd=(43)==(68), (68)==(43)] + │ │ │ │ │ │ │ │ └── n2.oid:73 = c2.relnamespace:47 [outer=(47,73), constraints=(/47: (/NULL - ]; /73: (/NULL - ]), fd=(47)==(73), (73)==(47)] │ │ │ │ │ │ │ ├── left-join (lookup pg_tablespace) - │ │ │ │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:28!null n.nspname:29!null t.oid:32 spcname:33 - │ │ │ │ │ │ │ │ ├── key columns: [8] = [32] + │ │ │ │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:29!null n.nspname:30!null t.oid:34 spcname:35 + │ │ │ │ │ │ │ │ ├── key columns: [8] = [34] │ │ │ │ │ │ │ │ ├── lookup columns are key │ │ │ │ │ │ │ │ ├── key: (1) - │ │ │ │ │ │ │ │ ├── fd: ()-->(3,28,29), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,32,33), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(28), (28)==(3), (32)-->(33), (33)-->(32) + │ │ │ │ │ │ │ │ ├── fd: ()-->(3,29,30), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27,34,35), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(29), (29)==(3), (34)-->(35), (35)-->(34) │ │ │ │ │ │ │ │ ├── inner-join (hash) - │ │ │ │ │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:28!null n.nspname:29!null + │ │ │ │ │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 n.oid:29!null n.nspname:30!null │ │ │ │ │ │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) │ │ │ │ │ │ │ │ │ ├── key: (1) - │ │ │ │ │ │ │ │ │ ├── fd: ()-->(3,28,29), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(28), (28)==(3) + │ │ │ │ │ │ │ │ │ ├── fd: ()-->(3,29,30), (1)-->(2,5,8,10,13,15,17,20,22,23,26,27), (2)-->(1,5,8,10,13,15,17,20,22,23,26,27), (3)==(29), (29)==(3) │ │ │ │ │ │ │ │ │ ├── select │ │ │ │ │ │ │ │ │ │ ├── columns: c.oid:1!null c.relname:2!null c.relnamespace:3!null c.relowner:5!null c.reltablespace:8!null c.reltuples:10!null c.relhasindex:13!null c.relpersistence:15!null c.relkind:17!null c.relhasoids:20!null c.relhasrules:22!null c.relhastriggers:23!null c.relacl:26 c.reloptions:27 │ │ │ │ │ │ │ │ │ │ ├── key: (1) @@ -257,26 +257,26 @@ sort │ │ │ │ │ │ │ │ │ │ └── filters │ │ │ │ │ │ │ │ │ │ └── (c.relkind:17 = 'r') OR (c.relkind:17 = 'f') [outer=(17), constraints=(/17: [/'f' - /'f'] [/'r' - /'r']; tight)] │ │ │ │ │ │ │ │ │ ├── scan n@pg_namespace_nspname_index - │ │ │ │ │ │ │ │ │ │ ├── columns: n.oid:28!null n.nspname:29!null - │ │ │ │ │ │ │ │ │ │ ├── constraint: /29: [/'public' - /'public'] + │ │ │ │ │ │ │ │ │ │ ├── columns: n.oid:29!null n.nspname:30!null + │ │ │ │ │ │ │ │ │ │ ├── constraint: /30: [/'public' - /'public'] │ │ │ │ │ │ │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ │ │ │ │ │ │ ├── key: () - │ │ │ │ │ │ │ │ │ │ └── fd: ()-->(28,29) + │ │ │ │ │ │ │ │ │ │ └── fd: ()-->(29,30) │ │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ │ └── n.oid:28 = c.relnamespace:3 [outer=(3,28), constraints=(/3: (/NULL - ]; /28: (/NULL - ]), fd=(3)==(28), (28)==(3)] + │ │ │ │ │ │ │ │ │ └── n.oid:29 = c.relnamespace:3 [outer=(3,29), constraints=(/3: (/NULL - ]; /29: (/NULL - ]), fd=(3)==(29), (29)==(3)] │ │ │ │ │ │ │ │ └── filters (true) │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── i.inhrelid:38 = c.oid:1 [outer=(1,38), constraints=(/1: (/NULL - ]; /38: (/NULL - ]), fd=(1)==(38), (38)==(1)] + │ │ │ │ │ │ │ └── i.inhrelid:41 = c.oid:1 [outer=(1,41), constraints=(/1: (/NULL - ]; /41: (/NULL - ]), fd=(1)==(41), (41)==(1)] │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── indrelid:73 = c.oid:1 [outer=(1,73), constraints=(/1: (/NULL - ]; /73: (/NULL - ]), fd=(1)==(73), (73)==(1)] + │ │ │ │ │ │ └── indrelid:79 = c.oid:1 [outer=(1,79), constraints=(/1: (/NULL - ]; /79: (/NULL - ]), fd=(1)==(79), (79)==(1)] │ │ │ │ │ └── filters (true) │ │ │ │ └── filters (true) │ │ │ └── filters (true) │ │ └── filters - │ │ └── pg_inherits.inhparent:130 = c.oid:1 [outer=(1,130), constraints=(/1: (/NULL - ]; /130: (/NULL - ]), fd=(1)==(130), (130)==(1)] + │ │ └── pg_inherits.inhparent:140 = c.oid:1 [outer=(1,140), constraints=(/1: (/NULL - ]; /140: (/NULL - ]), fd=(1)==(140), (140)==(1)] │ └── aggregations - │ ├── count [as=count_rows:132, outer=(130)] - │ │ └── pg_inherits.inhparent:130 + │ ├── count [as=count_rows:143, outer=(140)] + │ │ └── pg_inherits.inhparent:140 │ ├── const-agg [as=c.oid:1, outer=(1)] │ │ └── c.oid:1 │ ├── const-agg [as=c.relname:2, outer=(2)] @@ -301,21 +301,21 @@ sort │ │ └── c.relacl:26 │ ├── const-agg [as=c.reloptions:27, outer=(27)] │ │ └── c.reloptions:27 - │ ├── const-agg [as=n.nspname:29, outer=(29)] - │ │ └── n.nspname:29 - │ ├── const-agg [as=spcname:33, outer=(33)] - │ │ └── spcname:33 - │ ├── const-agg [as=c2.relname:42, outer=(42)] - │ │ └── c2.relname:42 - │ ├── const-agg [as=n2.nspname:69, outer=(69)] - │ │ └── n2.nspname:69 - │ ├── const-agg [as=ci.relname:92, outer=(92)] - │ │ └── ci.relname:92 - │ ├── const-agg [as=ftoptions:120, outer=(120)] - │ │ └── ftoptions:120 - │ └── const-agg [as=srvname:122, outer=(122)] - │ └── srvname:122 + │ ├── const-agg [as=n.nspname:30, outer=(30)] + │ │ └── n.nspname:30 + │ ├── const-agg [as=spcname:35, outer=(35)] + │ │ └── spcname:35 + │ ├── const-agg [as=c2.relname:46, outer=(46)] + │ │ └── c2.relname:46 + │ ├── const-agg [as=n2.nspname:74, outer=(74)] + │ │ └── n2.nspname:74 + │ ├── const-agg [as=ci.relname:99, outer=(99)] + │ │ └── ci.relname:99 + │ ├── const-agg [as=ftoptions:128, outer=(128)] + │ │ └── ftoptions:128 + │ └── const-agg [as=srvname:131, outer=(131)] + │ └── srvname:131 └── projections - ├── pg_get_userbyid(c.relowner:5) [as=tableowner:133, outer=(5), stable] - ├── obj_description(c.oid:1) [as=description:134, outer=(1), stable] - └── count_rows:132 > 0 [as=inhtable:135, outer=(132)] + ├── pg_get_userbyid(c.relowner:5) [as=tableowner:144, outer=(5), stable] + ├── obj_description(c.oid:1) [as=description:145, outer=(1), stable] + └── count_rows:143 > 0 [as=inhtable:146, outer=(143)] diff --git a/pkg/sql/opt/xform/testdata/external/nova b/pkg/sql/opt/xform/testdata/external/nova index f1e1365923fc..395a738133fe 100644 --- a/pkg/sql/opt/xform/testdata/external/nova +++ b/pkg/sql/opt/xform/testdata/external/nova @@ -158,50 +158,50 @@ from (select flavors.created_at as flavors_created_at, order by anon_1.flavors_id asc ---- project - ├── columns: anon_1_flavors_created_at:14 anon_1_flavors_updated_at:15 anon_1_flavors_id:1!null anon_1_flavors_name:2!null anon_1_flavors_memory_mb:3!null anon_1_flavors_vcpus:4!null anon_1_flavors_root_gb:5 anon_1_flavors_ephemeral_gb:6 anon_1_flavors_flavorid:7!null anon_1_flavors_swap:8!null anon_1_flavors_rxtx_factor:9 anon_1_flavors_vcpu_weight:10 anon_1_flavors_disabled:11 anon_1_flavors_is_public:12 flavor_extra_specs_1_created_at:29 flavor_extra_specs_1_updated_at:30 flavor_extra_specs_1_id:25 flavor_extra_specs_1_key:26 flavor_extra_specs_1_value:27 flavor_extra_specs_1_flavor_id:28 + ├── columns: anon_1_flavors_created_at:14 anon_1_flavors_updated_at:15 anon_1_flavors_id:1!null anon_1_flavors_name:2!null anon_1_flavors_memory_mb:3!null anon_1_flavors_vcpus:4!null anon_1_flavors_root_gb:5 anon_1_flavors_ephemeral_gb:6 anon_1_flavors_flavorid:7!null anon_1_flavors_swap:8!null anon_1_flavors_rxtx_factor:9 anon_1_flavors_vcpu_weight:10 anon_1_flavors_disabled:11 anon_1_flavors_is_public:12 flavor_extra_specs_1_created_at:31 flavor_extra_specs_1_updated_at:32 flavor_extra_specs_1_id:27 flavor_extra_specs_1_key:28 flavor_extra_specs_1_value:29 flavor_extra_specs_1_flavor_id:30 ├── immutable, has-placeholder - ├── key: (1,25) - ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (25)-->(26-30), (26,28)-->(25,27,29,30) + ├── key: (1,27) + ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (27)-->(28-32), (28,30)-->(27,29,31,32) ├── ordering: +1 └── left-join (merge) - ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 flavor_extra_specs_1.id:25 key:26 value:27 flavor_extra_specs_1.flavor_id:28 flavor_extra_specs_1.created_at:29 flavor_extra_specs_1.updated_at:30 + ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 flavor_extra_specs_1.id:27 key:28 value:29 flavor_extra_specs_1.flavor_id:30 flavor_extra_specs_1.created_at:31 flavor_extra_specs_1.updated_at:32 ├── left ordering: +1 - ├── right ordering: +28 + ├── right ordering: +30 ├── immutable, has-placeholder - ├── key: (1,25) - ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (25)-->(26-30), (26,28)-->(25,27,29,30) + ├── key: (1,27) + ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (27)-->(28-32), (28,30)-->(27,29,31,32) ├── ordering: +1 ├── limit - │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 + │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 │ ├── internal-ordering: +1 │ ├── immutable, has-placeholder │ ├── key: (1) - │ ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) + │ ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ ├── ordering: +1 │ ├── offset - │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 + │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 │ │ ├── internal-ordering: +1 │ │ ├── has-placeholder │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) + │ │ ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ ├── ordering: +1 │ │ ├── select - │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 + │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 │ │ │ ├── has-placeholder │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) + │ │ │ ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ │ ├── ordering: +1 │ │ │ ├── group-by - │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 + │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 │ │ │ │ ├── grouping columns: flavors.id:1!null │ │ │ │ ├── has-placeholder │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) + │ │ │ │ ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ │ │ ├── ordering: +1 │ │ │ │ ├── left-join (merge) - │ │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 flavor_projects.flavor_id:17 true:22 + │ │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 flavor_projects.flavor_id:18 true:24 │ │ │ │ │ ├── left ordering: +1 - │ │ │ │ │ ├── right ordering: +17 + │ │ │ │ │ ├── right ordering: +18 │ │ │ │ │ ├── has-placeholder │ │ │ │ │ ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ │ │ │ ├── ordering: +1 @@ -219,27 +219,27 @@ project │ │ │ │ │ │ └── filters │ │ │ │ │ │ └── flavorid:7 = $2 [outer=(7), constraints=(/7: (/NULL - ])] │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: true:22!null flavor_projects.flavor_id:17!null + │ │ │ │ │ │ ├── columns: true:24!null flavor_projects.flavor_id:18!null │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ ├── fd: ()-->(22) - │ │ │ │ │ │ ├── ordering: +17 opt(22) [actual: +17] + │ │ │ │ │ │ ├── fd: ()-->(24) + │ │ │ │ │ │ ├── ordering: +18 opt(24) [actual: +18] │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:17!null project_id:18!null + │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:18!null project_id:19!null │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ ├── key: (17,18) - │ │ │ │ │ │ │ ├── ordering: +17 + │ │ │ │ │ │ │ ├── key: (18,19) + │ │ │ │ │ │ │ ├── ordering: +18 │ │ │ │ │ │ │ ├── scan flavor_projects@secondary - │ │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:17!null project_id:18!null - │ │ │ │ │ │ │ │ ├── key: (17,18) - │ │ │ │ │ │ │ │ └── ordering: +17 + │ │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:18!null project_id:19!null + │ │ │ │ │ │ │ │ ├── key: (18,19) + │ │ │ │ │ │ │ │ └── ordering: +18 │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── project_id:18 = $1 [outer=(18), constraints=(/18: (/NULL - ])] + │ │ │ │ │ │ │ └── project_id:19 = $1 [outer=(19), constraints=(/19: (/NULL - ])] │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── true [as=true:22] + │ │ │ │ │ │ └── true [as=true:24] │ │ │ │ │ └── filters (true) │ │ │ │ └── aggregations - │ │ │ │ ├── const-not-null-agg [as=true_agg:23, outer=(22)] - │ │ │ │ │ └── true:22 + │ │ │ │ ├── const-not-null-agg [as=true_agg:25, outer=(24)] + │ │ │ │ │ └── true:24 │ │ │ │ ├── const-agg [as=name:2, outer=(2)] │ │ │ │ │ └── name:2 │ │ │ │ ├── const-agg [as=memory_mb:3, outer=(3)] @@ -267,18 +267,18 @@ project │ │ │ │ └── const-agg [as=flavors.updated_at:15, outer=(15)] │ │ │ │ └── flavors.updated_at:15 │ │ │ └── filters - │ │ │ └── (is_public:12 = true) OR (true_agg:23 IS NOT NULL) [outer=(12,23)] + │ │ │ └── (is_public:12 = true) OR (true_agg:25 IS NOT NULL) [outer=(12,25)] │ │ └── $3 │ └── $4 ├── sort - │ ├── columns: flavor_extra_specs_1.id:25!null key:26!null value:27 flavor_extra_specs_1.flavor_id:28!null flavor_extra_specs_1.created_at:29 flavor_extra_specs_1.updated_at:30 - │ ├── key: (25) - │ ├── fd: (25)-->(26-30), (26,28)-->(25,27,29,30) - │ ├── ordering: +28 + │ ├── columns: flavor_extra_specs_1.id:27!null key:28!null value:29 flavor_extra_specs_1.flavor_id:30!null flavor_extra_specs_1.created_at:31 flavor_extra_specs_1.updated_at:32 + │ ├── key: (27) + │ ├── fd: (27)-->(28-32), (28,30)-->(27,29,31,32) + │ ├── ordering: +30 │ └── scan flavor_extra_specs_1 - │ ├── columns: flavor_extra_specs_1.id:25!null key:26!null value:27 flavor_extra_specs_1.flavor_id:28!null flavor_extra_specs_1.created_at:29 flavor_extra_specs_1.updated_at:30 - │ ├── key: (25) - │ └── fd: (25)-->(26-30), (26,28)-->(25,27,29,30) + │ ├── columns: flavor_extra_specs_1.id:27!null key:28!null value:29 flavor_extra_specs_1.flavor_id:30!null flavor_extra_specs_1.created_at:31 flavor_extra_specs_1.updated_at:32 + │ ├── key: (27) + │ └── fd: (27)-->(28-32), (28,30)-->(27,29,31,32) └── filters (true) opt @@ -337,80 +337,80 @@ from (select flavors.created_at as flavors_created_at, order by anon_1.flavors_flavorid asc, anon_1.flavors_id asc ---- sort - ├── columns: anon_1_flavors_created_at:14 anon_1_flavors_updated_at:15 anon_1_flavors_id:1!null anon_1_flavors_name:2!null anon_1_flavors_memory_mb:3!null anon_1_flavors_vcpus:4!null anon_1_flavors_root_gb:5 anon_1_flavors_ephemeral_gb:6 anon_1_flavors_flavorid:7!null anon_1_flavors_swap:8!null anon_1_flavors_rxtx_factor:9 anon_1_flavors_vcpu_weight:10 anon_1_flavors_disabled:11!null anon_1_flavors_is_public:12 flavor_extra_specs_1_created_at:38 flavor_extra_specs_1_updated_at:39 flavor_extra_specs_1_id:34 flavor_extra_specs_1_key:35 flavor_extra_specs_1_value:36 flavor_extra_specs_1_flavor_id:37 + ├── columns: anon_1_flavors_created_at:14 anon_1_flavors_updated_at:15 anon_1_flavors_id:1!null anon_1_flavors_name:2!null anon_1_flavors_memory_mb:3!null anon_1_flavors_vcpus:4!null anon_1_flavors_root_gb:5 anon_1_flavors_ephemeral_gb:6 anon_1_flavors_flavorid:7!null anon_1_flavors_swap:8!null anon_1_flavors_rxtx_factor:9 anon_1_flavors_vcpu_weight:10 anon_1_flavors_disabled:11!null anon_1_flavors_is_public:12 flavor_extra_specs_1_created_at:41 flavor_extra_specs_1_updated_at:42 flavor_extra_specs_1_id:37 flavor_extra_specs_1_key:38 flavor_extra_specs_1_value:39 flavor_extra_specs_1_flavor_id:40 ├── immutable, has-placeholder - ├── key: (1,34) - ├── fd: ()-->(11), (1)-->(2-10,12,14,15), (7)-->(1-6,8-10,12,14,15), (2)-->(1,3-10,12,14,15), (34)-->(35-39), (35,37)-->(34,36,38,39) + ├── key: (1,37) + ├── fd: ()-->(11), (1)-->(2-10,12,14,15), (7)-->(1-6,8-10,12,14,15), (2)-->(1,3-10,12,14,15), (37)-->(38-42), (38,40)-->(37,39,41,42) ├── ordering: +7 opt(11) [actual: +7] └── project - ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 flavors.created_at:14 flavors.updated_at:15 flavor_extra_specs_1.id:34 key:35 value:36 flavor_extra_specs_1.flavor_id:37 flavor_extra_specs_1.created_at:38 flavor_extra_specs_1.updated_at:39 + ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 flavors.created_at:14 flavors.updated_at:15 flavor_extra_specs_1.id:37 key:38 value:39 flavor_extra_specs_1.flavor_id:40 flavor_extra_specs_1.created_at:41 flavor_extra_specs_1.updated_at:42 ├── immutable, has-placeholder - ├── key: (1,34) - ├── fd: ()-->(11), (1)-->(2-10,12,14,15), (7)-->(1-6,8-10,12,14,15), (2)-->(1,3-10,12,14,15), (34)-->(35-39), (35,37)-->(34,36,38,39) + ├── key: (1,37) + ├── fd: ()-->(11), (1)-->(2-10,12,14,15), (7)-->(1-6,8-10,12,14,15), (2)-->(1,3-10,12,14,15), (37)-->(38-42), (38,40)-->(37,39,41,42) └── right-join (hash) - ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:32 flavor_extra_specs_1.id:34 key:35 value:36 flavor_extra_specs_1.flavor_id:37 flavor_extra_specs_1.created_at:38 flavor_extra_specs_1.updated_at:39 + ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:35 flavor_extra_specs_1.id:37 key:38 value:39 flavor_extra_specs_1.flavor_id:40 flavor_extra_specs_1.created_at:41 flavor_extra_specs_1.updated_at:42 ├── immutable, has-placeholder - ├── key: (1,34) - ├── fd: ()-->(11), (1)-->(2-10,12,14,15,32), (7)-->(1-6,8-10,12,14,15), (2)-->(1,3-10,12,14,15), (34)-->(35-39), (35,37)-->(34,36,38,39) + ├── key: (1,37) + ├── fd: ()-->(11), (1)-->(2-10,12,14,15,35), (7)-->(1-6,8-10,12,14,15), (2)-->(1,3-10,12,14,15), (37)-->(38-42), (38,40)-->(37,39,41,42) ├── scan flavor_extra_specs_1 - │ ├── columns: flavor_extra_specs_1.id:34!null key:35!null value:36 flavor_extra_specs_1.flavor_id:37!null flavor_extra_specs_1.created_at:38 flavor_extra_specs_1.updated_at:39 - │ ├── key: (34) - │ └── fd: (34)-->(35-39), (35,37)-->(34,36,38,39) + │ ├── columns: flavor_extra_specs_1.id:37!null key:38!null value:39 flavor_extra_specs_1.flavor_id:40!null flavor_extra_specs_1.created_at:41 flavor_extra_specs_1.updated_at:42 + │ ├── key: (37) + │ └── fd: (37)-->(38-42), (38,40)-->(37,39,41,42) ├── limit - │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:32 + │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:35 │ ├── internal-ordering: +7 opt(11) │ ├── immutable, has-placeholder │ ├── key: (1) - │ ├── fd: ()-->(11), (1)-->(2-10,12,14,15,32), (7)-->(1-6,8-10,12,14,15), (2)-->(1,3-10,12,14,15) + │ ├── fd: ()-->(11), (1)-->(2-10,12,14,15,35), (7)-->(1-6,8-10,12,14,15), (2)-->(1,3-10,12,14,15) │ ├── offset - │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:32 + │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:35 │ │ ├── internal-ordering: +7 opt(11) │ │ ├── has-placeholder │ │ ├── key: (1) - │ │ ├── fd: ()-->(11), (1)-->(2-10,12,14,15,32), (7)-->(1-6,8-10,12,14,15), (2)-->(1,3-10,12,14,15) + │ │ ├── fd: ()-->(11), (1)-->(2-10,12,14,15,35), (7)-->(1-6,8-10,12,14,15), (2)-->(1,3-10,12,14,15) │ │ ├── ordering: +7 opt(11) [actual: +7] │ │ ├── sort - │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:32 + │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:35 │ │ │ ├── has-placeholder │ │ │ ├── key: (1) - │ │ │ ├── fd: ()-->(11), (1)-->(2-10,12,14,15,32), (7)-->(1-6,8-10,12,14,15), (2)-->(1,3-10,12,14,15) + │ │ │ ├── fd: ()-->(11), (1)-->(2-10,12,14,15,35), (7)-->(1-6,8-10,12,14,15), (2)-->(1,3-10,12,14,15) │ │ │ ├── ordering: +7 opt(11) [actual: +7] │ │ │ └── select - │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:32 + │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:35 │ │ │ ├── has-placeholder │ │ │ ├── key: (1) - │ │ │ ├── fd: ()-->(11), (1)-->(2-10,12,14,15,32), (7)-->(1-6,8-10,12,14,15), (2)-->(1,3-10,12,14,15) + │ │ │ ├── fd: ()-->(11), (1)-->(2-10,12,14,15,35), (7)-->(1-6,8-10,12,14,15), (2)-->(1,3-10,12,14,15) │ │ │ ├── group-by - │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:32 + │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:35 │ │ │ │ ├── grouping columns: flavors.id:1!null │ │ │ │ ├── internal-ordering: +1 opt(11) │ │ │ │ ├── has-placeholder │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: ()-->(11), (1)-->(2-12,14,15,32), (7)-->(1-6,8-10,12,14,15), (2)-->(1,3-10,12,14,15) + │ │ │ │ ├── fd: ()-->(11), (1)-->(2-12,14,15,35), (7)-->(1-6,8-10,12,14,15), (2)-->(1,3-10,12,14,15) │ │ │ │ ├── left-join (merge) - │ │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 flavors.created_at:14 flavors.updated_at:15 flavor_projects.flavor_id:23 true_agg:29 true:31 + │ │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 flavors.created_at:14 flavors.updated_at:15 flavor_projects.flavor_id:25 true_agg:32 true:34 │ │ │ │ │ ├── left ordering: +1 - │ │ │ │ │ ├── right ordering: +23 + │ │ │ │ │ ├── right ordering: +25 │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ ├── fd: ()-->(11), (1)-->(2-10,12,14,15,29), (7)-->(1-6,8-10,12,14,15), (2)-->(1,3-10,12,14,15) + │ │ │ │ │ ├── fd: ()-->(11), (1)-->(2-10,12,14,15,32), (7)-->(1-6,8-10,12,14,15), (2)-->(1,3-10,12,14,15) │ │ │ │ │ ├── ordering: +1 opt(11) [actual: +1] │ │ │ │ │ ├── select - │ │ │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:29 + │ │ │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:32 │ │ │ │ │ │ ├── has-placeholder │ │ │ │ │ │ ├── key: (1) - │ │ │ │ │ │ ├── fd: ()-->(11), (1)-->(2-10,12,14,15,29), (7)-->(1-6,8-10,12,14,15), (2)-->(1,3-10,12,14,15) + │ │ │ │ │ │ ├── fd: ()-->(11), (1)-->(2-10,12,14,15,32), (7)-->(1-6,8-10,12,14,15), (2)-->(1,3-10,12,14,15) │ │ │ │ │ │ ├── ordering: +1 opt(11) [actual: +1] │ │ │ │ │ │ ├── group-by - │ │ │ │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:29 + │ │ │ │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:32 │ │ │ │ │ │ │ ├── grouping columns: flavors.id:1!null │ │ │ │ │ │ │ ├── has-placeholder │ │ │ │ │ │ │ ├── key: (1) - │ │ │ │ │ │ │ ├── fd: ()-->(11), (1)-->(2-12,14,15,29), (7)-->(1-6,8-10,12,14,15), (2)-->(1,3-10,12,14,15) + │ │ │ │ │ │ │ ├── fd: ()-->(11), (1)-->(2-12,14,15,32), (7)-->(1-6,8-10,12,14,15), (2)-->(1,3-10,12,14,15) │ │ │ │ │ │ │ ├── ordering: +1 opt(11) [actual: +1] │ │ │ │ │ │ │ ├── left-join (merge) - │ │ │ │ │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 flavors.created_at:14 flavors.updated_at:15 flavor_projects.flavor_id:17 true:28 + │ │ │ │ │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 flavors.created_at:14 flavors.updated_at:15 flavor_projects.flavor_id:18 true:31 │ │ │ │ │ │ │ │ ├── left ordering: +1 - │ │ │ │ │ │ │ │ ├── right ordering: +17 + │ │ │ │ │ │ │ │ ├── right ordering: +18 │ │ │ │ │ │ │ │ ├── has-placeholder │ │ │ │ │ │ │ │ ├── fd: ()-->(11), (1)-->(2-10,12,14,15), (7)-->(1-6,8-10,12,14,15), (2)-->(1,3-10,12,14,15) │ │ │ │ │ │ │ │ ├── ordering: +1 opt(11) [actual: +1] @@ -427,27 +427,27 @@ sort │ │ │ │ │ │ │ │ │ └── filters │ │ │ │ │ │ │ │ │ └── disabled:11 = false [outer=(11), constraints=(/11: [/false - /false]; tight), fd=()-->(11)] │ │ │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ │ │ ├── columns: true:28!null flavor_projects.flavor_id:17!null + │ │ │ │ │ │ │ │ │ ├── columns: true:31!null flavor_projects.flavor_id:18!null │ │ │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ │ │ ├── fd: ()-->(28) - │ │ │ │ │ │ │ │ │ ├── ordering: +17 opt(28) [actual: +17] + │ │ │ │ │ │ │ │ │ ├── fd: ()-->(31) + │ │ │ │ │ │ │ │ │ ├── ordering: +18 opt(31) [actual: +18] │ │ │ │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:17!null project_id:18!null + │ │ │ │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:18!null project_id:19!null │ │ │ │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ │ │ │ ├── key: (17,18) - │ │ │ │ │ │ │ │ │ │ ├── ordering: +17 + │ │ │ │ │ │ │ │ │ │ ├── key: (18,19) + │ │ │ │ │ │ │ │ │ │ ├── ordering: +18 │ │ │ │ │ │ │ │ │ │ ├── scan flavor_projects@secondary - │ │ │ │ │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:17!null project_id:18!null - │ │ │ │ │ │ │ │ │ │ │ ├── key: (17,18) - │ │ │ │ │ │ │ │ │ │ │ └── ordering: +17 + │ │ │ │ │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:18!null project_id:19!null + │ │ │ │ │ │ │ │ │ │ │ ├── key: (18,19) + │ │ │ │ │ │ │ │ │ │ │ └── ordering: +18 │ │ │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ │ │ └── project_id:18 = $1 [outer=(18), constraints=(/18: (/NULL - ])] + │ │ │ │ │ │ │ │ │ │ └── project_id:19 = $1 [outer=(19), constraints=(/19: (/NULL - ])] │ │ │ │ │ │ │ │ │ └── projections - │ │ │ │ │ │ │ │ │ └── true [as=true:28] + │ │ │ │ │ │ │ │ │ └── true [as=true:31] │ │ │ │ │ │ │ │ └── filters (true) │ │ │ │ │ │ │ └── aggregations - │ │ │ │ │ │ │ ├── const-not-null-agg [as=true_agg:29, outer=(28)] - │ │ │ │ │ │ │ │ └── true:28 + │ │ │ │ │ │ │ ├── const-not-null-agg [as=true_agg:32, outer=(31)] + │ │ │ │ │ │ │ │ └── true:31 │ │ │ │ │ │ │ ├── const-agg [as=name:2, outer=(2)] │ │ │ │ │ │ │ │ └── name:2 │ │ │ │ │ │ │ ├── const-agg [as=memory_mb:3, outer=(3)] @@ -475,29 +475,29 @@ sort │ │ │ │ │ │ │ └── const-agg [as=flavors.updated_at:15, outer=(15)] │ │ │ │ │ │ │ └── flavors.updated_at:15 │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── (is_public:12 = true) OR (true_agg:29 IS NOT NULL) [outer=(12,29)] + │ │ │ │ │ │ └── (is_public:12 = true) OR (true_agg:32 IS NOT NULL) [outer=(12,32)] │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: true:31!null flavor_projects.flavor_id:23!null + │ │ │ │ │ │ ├── columns: true:34!null flavor_projects.flavor_id:25!null │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ ├── fd: ()-->(31) - │ │ │ │ │ │ ├── ordering: +23 opt(31) [actual: +23] + │ │ │ │ │ │ ├── fd: ()-->(34) + │ │ │ │ │ │ ├── ordering: +25 opt(34) [actual: +25] │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:23!null project_id:24!null + │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:25!null project_id:26!null │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ ├── key: (23,24) - │ │ │ │ │ │ │ ├── ordering: +23 + │ │ │ │ │ │ │ ├── key: (25,26) + │ │ │ │ │ │ │ ├── ordering: +25 │ │ │ │ │ │ │ ├── scan flavor_projects@secondary - │ │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:23!null project_id:24!null - │ │ │ │ │ │ │ │ ├── key: (23,24) - │ │ │ │ │ │ │ │ └── ordering: +23 + │ │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:25!null project_id:26!null + │ │ │ │ │ │ │ │ ├── key: (25,26) + │ │ │ │ │ │ │ │ └── ordering: +25 │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── project_id:24 = $2 [outer=(24), constraints=(/24: (/NULL - ])] + │ │ │ │ │ │ │ └── project_id:26 = $2 [outer=(26), constraints=(/26: (/NULL - ])] │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── true [as=true:31] + │ │ │ │ │ │ └── true [as=true:34] │ │ │ │ │ └── filters (true) │ │ │ │ └── aggregations - │ │ │ │ ├── const-not-null-agg [as=true_agg:32, outer=(31)] - │ │ │ │ │ └── true:31 + │ │ │ │ ├── const-not-null-agg [as=true_agg:35, outer=(34)] + │ │ │ │ │ └── true:34 │ │ │ │ ├── const-agg [as=name:2, outer=(2)] │ │ │ │ │ └── name:2 │ │ │ │ ├── const-agg [as=memory_mb:3, outer=(3)] @@ -525,11 +525,11 @@ sort │ │ │ │ └── const-agg [as=flavors.updated_at:15, outer=(15)] │ │ │ │ └── flavors.updated_at:15 │ │ │ └── filters - │ │ │ └── (is_public:12 = true) OR (true_agg:32 IS NOT NULL) [outer=(12,32)] + │ │ │ └── (is_public:12 = true) OR (true_agg:35 IS NOT NULL) [outer=(12,35)] │ │ └── $3 │ └── $4 └── filters - └── flavor_extra_specs_1.flavor_id:37 = flavors.id:1 [outer=(1,37), constraints=(/1: (/NULL - ]; /37: (/NULL - ]), fd=(1)==(37), (37)==(1)] + └── flavor_extra_specs_1.flavor_id:40 = flavors.id:1 [outer=(1,40), constraints=(/1: (/NULL - ]; /40: (/NULL - ]), fd=(1)==(40), (40)==(1)] opt select anon_1.instance_types_created_at as anon_1_instance_types_created_at, @@ -592,67 +592,67 @@ order by anon_1.instance_types_flavorid asc, anon_1.instance_types_id asc ---- sort - ├── columns: anon_1_instance_types_created_at:15 anon_1_instance_types_updated_at:16 anon_1_instance_types_deleted_at:14 anon_1_instance_types_deleted:13!null anon_1_instance_types_id:1!null anon_1_instance_types_name:2 anon_1_instance_types_memory_mb:3!null anon_1_instance_types_vcpus:4!null anon_1_instance_types_root_gb:5 anon_1_instance_types_ephemeral_gb:6 anon_1_instance_types_flavorid:7 anon_1_instance_types_swap:8!null anon_1_instance_types_rxtx_factor:9 anon_1_instance_types_vcpu_weight:10 anon_1_instance_types_disabled:11 anon_1_instance_types_is_public:12 instance_type_extra_specs_1_created_at:34 instance_type_extra_specs_1_updated_at:35 instance_type_extra_specs_1_deleted_at:33 instance_type_extra_specs_1_deleted:32 instance_type_extra_specs_1_id:28 instance_type_extra_specs_1_key:29 instance_type_extra_specs_1_value:30 instance_type_extra_specs_1_instance_type_id:31 + ├── columns: anon_1_instance_types_created_at:15 anon_1_instance_types_updated_at:16 anon_1_instance_types_deleted_at:14 anon_1_instance_types_deleted:13!null anon_1_instance_types_id:1!null anon_1_instance_types_name:2 anon_1_instance_types_memory_mb:3!null anon_1_instance_types_vcpus:4!null anon_1_instance_types_root_gb:5 anon_1_instance_types_ephemeral_gb:6 anon_1_instance_types_flavorid:7 anon_1_instance_types_swap:8!null anon_1_instance_types_rxtx_factor:9 anon_1_instance_types_vcpu_weight:10 anon_1_instance_types_disabled:11 anon_1_instance_types_is_public:12 instance_type_extra_specs_1_created_at:36 instance_type_extra_specs_1_updated_at:37 instance_type_extra_specs_1_deleted_at:35 instance_type_extra_specs_1_deleted:34 instance_type_extra_specs_1_id:30 instance_type_extra_specs_1_key:31 instance_type_extra_specs_1_value:32 instance_type_extra_specs_1_instance_type_id:33 ├── immutable, has-placeholder - ├── key: (1,28) - ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + ├── key: (1,30) + ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (30)-->(31-37), (31,33,34)~~>(30,32,35-37) ├── ordering: +7,+1 └── project - ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_extra_specs_1.id:28 key:29 value:30 instance_type_extra_specs_1.instance_type_id:31 instance_type_extra_specs_1.deleted:32 instance_type_extra_specs_1.deleted_at:33 instance_type_extra_specs_1.created_at:34 instance_type_extra_specs_1.updated_at:35 + ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_extra_specs_1.id:30 key:31 value:32 instance_type_extra_specs_1.instance_type_id:33 instance_type_extra_specs_1.deleted:34 instance_type_extra_specs_1.deleted_at:35 instance_type_extra_specs_1.created_at:36 instance_type_extra_specs_1.updated_at:37 ├── immutable, has-placeholder - ├── key: (1,28) - ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + ├── key: (1,30) + ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (30)-->(31-37), (31,33,34)~~>(30,32,35-37) └── right-join (hash) - ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 instance_type_extra_specs_1.id:28 key:29 value:30 instance_type_extra_specs_1.instance_type_id:31 instance_type_extra_specs_1.deleted:32 instance_type_extra_specs_1.deleted_at:33 instance_type_extra_specs_1.created_at:34 instance_type_extra_specs_1.updated_at:35 + ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 instance_type_extra_specs_1.id:30 key:31 value:32 instance_type_extra_specs_1.instance_type_id:33 instance_type_extra_specs_1.deleted:34 instance_type_extra_specs_1.deleted_at:35 instance_type_extra_specs_1.created_at:36 instance_type_extra_specs_1.updated_at:37 ├── immutable, has-placeholder - ├── key: (1,28) - ├── fd: (1)-->(2-16,26), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + ├── key: (1,30) + ├── fd: (1)-->(2-16,28), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (30)-->(31-37), (31,33,34)~~>(30,32,35-37) ├── select - │ ├── columns: instance_type_extra_specs_1.id:28!null key:29 value:30 instance_type_extra_specs_1.instance_type_id:31!null instance_type_extra_specs_1.deleted:32!null instance_type_extra_specs_1.deleted_at:33 instance_type_extra_specs_1.created_at:34 instance_type_extra_specs_1.updated_at:35 + │ ├── columns: instance_type_extra_specs_1.id:30!null key:31 value:32 instance_type_extra_specs_1.instance_type_id:33!null instance_type_extra_specs_1.deleted:34!null instance_type_extra_specs_1.deleted_at:35 instance_type_extra_specs_1.created_at:36 instance_type_extra_specs_1.updated_at:37 │ ├── has-placeholder - │ ├── key: (28) - │ ├── fd: (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + │ ├── key: (30) + │ ├── fd: (30)-->(31-37), (31,33,34)~~>(30,32,35-37) │ ├── scan instance_type_extra_specs_1 - │ │ ├── columns: instance_type_extra_specs_1.id:28!null key:29 value:30 instance_type_extra_specs_1.instance_type_id:31!null instance_type_extra_specs_1.deleted:32 instance_type_extra_specs_1.deleted_at:33 instance_type_extra_specs_1.created_at:34 instance_type_extra_specs_1.updated_at:35 - │ │ ├── key: (28) - │ │ └── fd: (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + │ │ ├── columns: instance_type_extra_specs_1.id:30!null key:31 value:32 instance_type_extra_specs_1.instance_type_id:33!null instance_type_extra_specs_1.deleted:34 instance_type_extra_specs_1.deleted_at:35 instance_type_extra_specs_1.created_at:36 instance_type_extra_specs_1.updated_at:37 + │ │ ├── key: (30) + │ │ └── fd: (30)-->(31-37), (31,33,34)~~>(30,32,35-37) │ └── filters - │ └── instance_type_extra_specs_1.deleted:32 = $7 [outer=(32), constraints=(/32: (/NULL - ])] + │ └── instance_type_extra_specs_1.deleted:34 = $7 [outer=(34), constraints=(/34: (/NULL - ])] ├── limit - │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ ├── internal-ordering: +7,+1 │ ├── immutable, has-placeholder │ ├── key: (1) - │ ├── fd: (1)-->(2-16,26), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ ├── fd: (1)-->(2-16,28), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ ├── offset - │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ │ ├── internal-ordering: +7,+1 │ │ ├── has-placeholder │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2-16,26), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ ├── fd: (1)-->(2-16,28), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ ├── ordering: +7,+1 │ │ ├── sort - │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ │ │ ├── has-placeholder │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(2-16,26), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ │ ├── fd: (1)-->(2-16,28), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ │ ├── ordering: +7,+1 │ │ │ └── select - │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ │ │ ├── has-placeholder │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(2-16,26), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ │ ├── fd: (1)-->(2-16,28), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ │ ├── group-by - │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ │ │ │ ├── grouping columns: instance_types.id:1!null │ │ │ │ ├── internal-ordering: +1 │ │ │ │ ├── has-placeholder │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(2-16,26), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ │ │ ├── fd: (1)-->(2-16,28), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ │ │ ├── left-join (merge) - │ │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_projects.instance_type_id:18 true:25 + │ │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_projects.instance_type_id:19 true:27 │ │ │ │ │ ├── left ordering: +1 - │ │ │ │ │ ├── right ordering: +18 + │ │ │ │ │ ├── right ordering: +19 │ │ │ │ │ ├── has-placeholder │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ │ │ │ ├── ordering: +1 @@ -670,29 +670,29 @@ sort │ │ │ │ │ │ └── filters │ │ │ │ │ │ └── instance_types.deleted:13 = $1 [outer=(13), constraints=(/13: (/NULL - ])] │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: true:25!null instance_type_projects.instance_type_id:18!null + │ │ │ │ │ │ ├── columns: true:27!null instance_type_projects.instance_type_id:19!null │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ ├── fd: ()-->(25) - │ │ │ │ │ │ ├── ordering: +18 opt(25) [actual: +18] + │ │ │ │ │ │ ├── fd: ()-->(27) + │ │ │ │ │ │ ├── ordering: +19 opt(27) [actual: +19] │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18!null project_id:19!null instance_type_projects.deleted:20!null + │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:19!null project_id:20!null instance_type_projects.deleted:21!null │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ ├── key: (18-20) - │ │ │ │ │ │ │ ├── ordering: +18 + │ │ │ │ │ │ │ ├── key: (19-21) + │ │ │ │ │ │ │ ├── ordering: +19 │ │ │ │ │ │ │ ├── scan instance_type_projects@secondary - │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18!null project_id:19 instance_type_projects.deleted:20 - │ │ │ │ │ │ │ │ ├── lax-key: (18-20) - │ │ │ │ │ │ │ │ └── ordering: +18 + │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:19!null project_id:20 instance_type_projects.deleted:21 + │ │ │ │ │ │ │ │ ├── lax-key: (19-21) + │ │ │ │ │ │ │ │ └── ordering: +19 │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ ├── instance_type_projects.deleted:20 = $2 [outer=(20), constraints=(/20: (/NULL - ])] - │ │ │ │ │ │ │ ├── instance_type_projects.deleted:20 = $3 [outer=(20), constraints=(/20: (/NULL - ])] - │ │ │ │ │ │ │ └── project_id:19 = $4 [outer=(19), constraints=(/19: (/NULL - ])] + │ │ │ │ │ │ │ ├── instance_type_projects.deleted:21 = $2 [outer=(21), constraints=(/21: (/NULL - ])] + │ │ │ │ │ │ │ ├── instance_type_projects.deleted:21 = $3 [outer=(21), constraints=(/21: (/NULL - ])] + │ │ │ │ │ │ │ └── project_id:20 = $4 [outer=(20), constraints=(/20: (/NULL - ])] │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── true [as=true:25] + │ │ │ │ │ │ └── true [as=true:27] │ │ │ │ │ └── filters (true) │ │ │ │ └── aggregations - │ │ │ │ ├── const-not-null-agg [as=true_agg:26, outer=(25)] - │ │ │ │ │ └── true:25 + │ │ │ │ ├── const-not-null-agg [as=true_agg:28, outer=(27)] + │ │ │ │ │ └── true:27 │ │ │ │ ├── const-agg [as=name:2, outer=(2)] │ │ │ │ │ └── name:2 │ │ │ │ ├── const-agg [as=memory_mb:3, outer=(3)] @@ -724,11 +724,11 @@ sort │ │ │ │ └── const-agg [as=instance_types.updated_at:16, outer=(16)] │ │ │ │ └── instance_types.updated_at:16 │ │ │ └── filters - │ │ │ └── (is_public:12 = true) OR (true_agg:26 IS NOT NULL) [outer=(12,26)] + │ │ │ └── (is_public:12 = true) OR (true_agg:28 IS NOT NULL) [outer=(12,28)] │ │ └── $5 │ └── $6 └── filters - └── instance_type_extra_specs_1.instance_type_id:31 = instance_types.id:1 [outer=(1,31), constraints=(/1: (/NULL - ]; /31: (/NULL - ]), fd=(1)==(31), (31)==(1)] + └── instance_type_extra_specs_1.instance_type_id:33 = instance_types.id:1 [outer=(1,33), constraints=(/1: (/NULL - ]; /33: (/NULL - ]), fd=(1)==(33), (33)==(1)] opt select instance_types.created_at as instance_types_created_at, @@ -769,48 +769,48 @@ where instance_types.deleted = $2 order by instance_types.flavorid asc, instance_types.id asc ---- sort - ├── columns: instance_types_created_at:15 instance_types_updated_at:16 instance_types_deleted_at:14 instance_types_deleted:13!null instance_types_id:1!null instance_types_name:2 instance_types_memory_mb:3!null instance_types_vcpus:4!null instance_types_root_gb:5 instance_types_ephemeral_gb:6 instance_types_flavorid:7 instance_types_swap:8!null instance_types_rxtx_factor:9 instance_types_vcpu_weight:10 instance_types_disabled:11 instance_types_is_public:12 instance_type_extra_specs_1_created_at:23 instance_type_extra_specs_1_updated_at:24 instance_type_extra_specs_1_deleted_at:22 instance_type_extra_specs_1_deleted:21 instance_type_extra_specs_1_id:17 instance_type_extra_specs_1_key:18 instance_type_extra_specs_1_value:19 instance_type_extra_specs_1_instance_type_id:20 + ├── columns: instance_types_created_at:15 instance_types_updated_at:16 instance_types_deleted_at:14 instance_types_deleted:13!null instance_types_id:1!null instance_types_name:2 instance_types_memory_mb:3!null instance_types_vcpus:4!null instance_types_root_gb:5 instance_types_ephemeral_gb:6 instance_types_flavorid:7 instance_types_swap:8!null instance_types_rxtx_factor:9 instance_types_vcpu_weight:10 instance_types_disabled:11 instance_types_is_public:12 instance_type_extra_specs_1_created_at:24 instance_type_extra_specs_1_updated_at:25 instance_type_extra_specs_1_deleted_at:23 instance_type_extra_specs_1_deleted:22 instance_type_extra_specs_1_id:18 instance_type_extra_specs_1_key:19 instance_type_extra_specs_1_value:20 instance_type_extra_specs_1_instance_type_id:21 ├── has-placeholder - ├── key: (1,17) - ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (17)-->(18-24), (18,20,21)~~>(17,19,22-24) + ├── key: (1,18) + ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (18)-->(19-25), (19,21,22)~~>(18,20,23-25) ├── ordering: +7,+1 └── project - ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_extra_specs_1.id:17 key:18 value:19 instance_type_extra_specs_1.instance_type_id:20 instance_type_extra_specs_1.deleted:21 instance_type_extra_specs_1.deleted_at:22 instance_type_extra_specs_1.created_at:23 instance_type_extra_specs_1.updated_at:24 + ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_extra_specs_1.id:18 key:19 value:20 instance_type_extra_specs_1.instance_type_id:21 instance_type_extra_specs_1.deleted:22 instance_type_extra_specs_1.deleted_at:23 instance_type_extra_specs_1.created_at:24 instance_type_extra_specs_1.updated_at:25 ├── has-placeholder - ├── key: (1,17) - ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (17)-->(18-24), (18,20,21)~~>(17,19,22-24) + ├── key: (1,18) + ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (18)-->(19-25), (19,21,22)~~>(18,20,23-25) └── right-join (hash) - ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_extra_specs_1.id:17 key:18 value:19 instance_type_extra_specs_1.instance_type_id:20 instance_type_extra_specs_1.deleted:21 instance_type_extra_specs_1.deleted_at:22 instance_type_extra_specs_1.created_at:23 instance_type_extra_specs_1.updated_at:24 true_agg:34 + ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_extra_specs_1.id:18 key:19 value:20 instance_type_extra_specs_1.instance_type_id:21 instance_type_extra_specs_1.deleted:22 instance_type_extra_specs_1.deleted_at:23 instance_type_extra_specs_1.created_at:24 instance_type_extra_specs_1.updated_at:25 true_agg:37 ├── has-placeholder - ├── key: (1,17) - ├── fd: (1)-->(2-16,34), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (17)-->(18-24), (18,20,21)~~>(17,19,22-24) + ├── key: (1,18) + ├── fd: (1)-->(2-16,37), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (18)-->(19-25), (19,21,22)~~>(18,20,23-25) ├── select - │ ├── columns: instance_type_extra_specs_1.id:17!null key:18 value:19 instance_type_extra_specs_1.instance_type_id:20!null instance_type_extra_specs_1.deleted:21!null instance_type_extra_specs_1.deleted_at:22 instance_type_extra_specs_1.created_at:23 instance_type_extra_specs_1.updated_at:24 + │ ├── columns: instance_type_extra_specs_1.id:18!null key:19 value:20 instance_type_extra_specs_1.instance_type_id:21!null instance_type_extra_specs_1.deleted:22!null instance_type_extra_specs_1.deleted_at:23 instance_type_extra_specs_1.created_at:24 instance_type_extra_specs_1.updated_at:25 │ ├── has-placeholder - │ ├── key: (17) - │ ├── fd: (17)-->(18-24), (18,20,21)~~>(17,19,22-24) + │ ├── key: (18) + │ ├── fd: (18)-->(19-25), (19,21,22)~~>(18,20,23-25) │ ├── scan instance_type_extra_specs_1 - │ │ ├── columns: instance_type_extra_specs_1.id:17!null key:18 value:19 instance_type_extra_specs_1.instance_type_id:20!null instance_type_extra_specs_1.deleted:21 instance_type_extra_specs_1.deleted_at:22 instance_type_extra_specs_1.created_at:23 instance_type_extra_specs_1.updated_at:24 - │ │ ├── key: (17) - │ │ └── fd: (17)-->(18-24), (18,20,21)~~>(17,19,22-24) + │ │ ├── columns: instance_type_extra_specs_1.id:18!null key:19 value:20 instance_type_extra_specs_1.instance_type_id:21!null instance_type_extra_specs_1.deleted:22 instance_type_extra_specs_1.deleted_at:23 instance_type_extra_specs_1.created_at:24 instance_type_extra_specs_1.updated_at:25 + │ │ ├── key: (18) + │ │ └── fd: (18)-->(19-25), (19,21,22)~~>(18,20,23-25) │ └── filters - │ └── instance_type_extra_specs_1.deleted:21 = $1 [outer=(21), constraints=(/21: (/NULL - ])] + │ └── instance_type_extra_specs_1.deleted:22 = $1 [outer=(22), constraints=(/22: (/NULL - ])] ├── select - │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:34 + │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:37 │ ├── has-placeholder │ ├── key: (1) - │ ├── fd: (1)-->(2-16,34), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ ├── fd: (1)-->(2-16,37), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ ├── group-by - │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:34 + │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:37 │ │ ├── grouping columns: instance_types.id:1!null │ │ ├── internal-ordering: +1 │ │ ├── has-placeholder │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2-16,34), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ ├── fd: (1)-->(2-16,37), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ ├── left-join (merge) - │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_projects.instance_type_id:26 true:33 + │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_projects.instance_type_id:28 true:36 │ │ │ ├── left ordering: +1 - │ │ │ ├── right ordering: +26 + │ │ │ ├── right ordering: +28 │ │ │ ├── has-placeholder │ │ │ ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ │ ├── ordering: +1 @@ -828,28 +828,28 @@ sort │ │ │ │ └── filters │ │ │ │ └── instance_types.deleted:13 = $2 [outer=(13), constraints=(/13: (/NULL - ])] │ │ │ ├── project - │ │ │ │ ├── columns: true:33!null instance_type_projects.instance_type_id:26!null + │ │ │ │ ├── columns: true:36!null instance_type_projects.instance_type_id:28!null │ │ │ │ ├── has-placeholder - │ │ │ │ ├── fd: ()-->(33) - │ │ │ │ ├── ordering: +26 opt(33) [actual: +26] + │ │ │ │ ├── fd: ()-->(36) + │ │ │ │ ├── ordering: +28 opt(36) [actual: +28] │ │ │ │ ├── select - │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:26!null project_id:27!null instance_type_projects.deleted:28!null + │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:28!null project_id:29!null instance_type_projects.deleted:30!null │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ ├── key: (26-28) - │ │ │ │ │ ├── ordering: +26 + │ │ │ │ │ ├── key: (28-30) + │ │ │ │ │ ├── ordering: +28 │ │ │ │ │ ├── scan instance_type_projects@secondary - │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:26!null project_id:27 instance_type_projects.deleted:28 - │ │ │ │ │ │ ├── lax-key: (26-28) - │ │ │ │ │ │ └── ordering: +26 + │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:28!null project_id:29 instance_type_projects.deleted:30 + │ │ │ │ │ │ ├── lax-key: (28-30) + │ │ │ │ │ │ └── ordering: +28 │ │ │ │ │ └── filters - │ │ │ │ │ ├── instance_type_projects.deleted:28 = $3 [outer=(28), constraints=(/28: (/NULL - ])] - │ │ │ │ │ └── project_id:27 = $4 [outer=(27), constraints=(/27: (/NULL - ])] + │ │ │ │ │ ├── instance_type_projects.deleted:30 = $3 [outer=(30), constraints=(/30: (/NULL - ])] + │ │ │ │ │ └── project_id:29 = $4 [outer=(29), constraints=(/29: (/NULL - ])] │ │ │ │ └── projections - │ │ │ │ └── true [as=true:33] + │ │ │ │ └── true [as=true:36] │ │ │ └── filters (true) │ │ └── aggregations - │ │ ├── const-not-null-agg [as=true_agg:34, outer=(33)] - │ │ │ └── true:33 + │ │ ├── const-not-null-agg [as=true_agg:37, outer=(36)] + │ │ │ └── true:36 │ │ ├── const-agg [as=name:2, outer=(2)] │ │ │ └── name:2 │ │ ├── const-agg [as=memory_mb:3, outer=(3)] @@ -881,9 +881,9 @@ sort │ │ └── const-agg [as=instance_types.updated_at:16, outer=(16)] │ │ └── instance_types.updated_at:16 │ └── filters - │ └── (is_public:12 = true) OR (true_agg:34 IS NOT NULL) [outer=(12,34)] + │ └── (is_public:12 = true) OR (true_agg:37 IS NOT NULL) [outer=(12,37)] └── filters - └── instance_type_extra_specs_1.instance_type_id:20 = instance_types.id:1 [outer=(1,20), constraints=(/1: (/NULL - ]; /20: (/NULL - ]), fd=(1)==(20), (20)==(1)] + └── instance_type_extra_specs_1.instance_type_id:21 = instance_types.id:1 [outer=(1,21), constraints=(/1: (/NULL - ]; /21: (/NULL - ]), fd=(1)==(21), (21)==(1)] opt select anon_1.instance_types_created_at as anon_1_instance_types_created_at, @@ -943,52 +943,52 @@ from (select instance_types.created_at as instance_types_created_at, and instance_type_extra_specs_1.deleted = $7 ---- project - ├── columns: anon_1_instance_types_created_at:15 anon_1_instance_types_updated_at:16 anon_1_instance_types_deleted_at:14 anon_1_instance_types_deleted:13!null anon_1_instance_types_id:1!null anon_1_instance_types_name:2!null anon_1_instance_types_memory_mb:3!null anon_1_instance_types_vcpus:4!null anon_1_instance_types_root_gb:5 anon_1_instance_types_ephemeral_gb:6 anon_1_instance_types_flavorid:7 anon_1_instance_types_swap:8!null anon_1_instance_types_rxtx_factor:9 anon_1_instance_types_vcpu_weight:10 anon_1_instance_types_disabled:11 anon_1_instance_types_is_public:12 instance_type_extra_specs_1_created_at:34 instance_type_extra_specs_1_updated_at:35 instance_type_extra_specs_1_deleted_at:33 instance_type_extra_specs_1_deleted:32 instance_type_extra_specs_1_id:28 instance_type_extra_specs_1_key:29 instance_type_extra_specs_1_value:30 instance_type_extra_specs_1_instance_type_id:31 + ├── columns: anon_1_instance_types_created_at:15 anon_1_instance_types_updated_at:16 anon_1_instance_types_deleted_at:14 anon_1_instance_types_deleted:13!null anon_1_instance_types_id:1!null anon_1_instance_types_name:2!null anon_1_instance_types_memory_mb:3!null anon_1_instance_types_vcpus:4!null anon_1_instance_types_root_gb:5 anon_1_instance_types_ephemeral_gb:6 anon_1_instance_types_flavorid:7 anon_1_instance_types_swap:8!null anon_1_instance_types_rxtx_factor:9 anon_1_instance_types_vcpu_weight:10 anon_1_instance_types_disabled:11 anon_1_instance_types_is_public:12 instance_type_extra_specs_1_created_at:36 instance_type_extra_specs_1_updated_at:37 instance_type_extra_specs_1_deleted_at:35 instance_type_extra_specs_1_deleted:34 instance_type_extra_specs_1_id:30 instance_type_extra_specs_1_key:31 instance_type_extra_specs_1_value:32 instance_type_extra_specs_1_instance_type_id:33 ├── immutable, has-placeholder - ├── key: (1,28) - ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)-->(1,3-12,14-16), (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + ├── key: (1,30) + ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)-->(1,3-12,14-16), (30)-->(31-37), (31,33,34)~~>(30,32,35-37) └── right-join (hash) - ├── columns: instance_types.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 instance_type_extra_specs_1.id:28 key:29 value:30 instance_type_extra_specs_1.instance_type_id:31 instance_type_extra_specs_1.deleted:32 instance_type_extra_specs_1.deleted_at:33 instance_type_extra_specs_1.created_at:34 instance_type_extra_specs_1.updated_at:35 + ├── columns: instance_types.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 instance_type_extra_specs_1.id:30 key:31 value:32 instance_type_extra_specs_1.instance_type_id:33 instance_type_extra_specs_1.deleted:34 instance_type_extra_specs_1.deleted_at:35 instance_type_extra_specs_1.created_at:36 instance_type_extra_specs_1.updated_at:37 ├── immutable, has-placeholder - ├── key: (1,28) - ├── fd: (1)-->(2-16,26), (7,13)~~>(1-6,8-12,14-16), (2,13)-->(1,3-12,14-16), (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + ├── key: (1,30) + ├── fd: (1)-->(2-16,28), (7,13)~~>(1-6,8-12,14-16), (2,13)-->(1,3-12,14-16), (30)-->(31-37), (31,33,34)~~>(30,32,35-37) ├── select - │ ├── columns: instance_type_extra_specs_1.id:28!null key:29 value:30 instance_type_extra_specs_1.instance_type_id:31!null instance_type_extra_specs_1.deleted:32!null instance_type_extra_specs_1.deleted_at:33 instance_type_extra_specs_1.created_at:34 instance_type_extra_specs_1.updated_at:35 + │ ├── columns: instance_type_extra_specs_1.id:30!null key:31 value:32 instance_type_extra_specs_1.instance_type_id:33!null instance_type_extra_specs_1.deleted:34!null instance_type_extra_specs_1.deleted_at:35 instance_type_extra_specs_1.created_at:36 instance_type_extra_specs_1.updated_at:37 │ ├── has-placeholder - │ ├── key: (28) - │ ├── fd: (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + │ ├── key: (30) + │ ├── fd: (30)-->(31-37), (31,33,34)~~>(30,32,35-37) │ ├── scan instance_type_extra_specs_1 - │ │ ├── columns: instance_type_extra_specs_1.id:28!null key:29 value:30 instance_type_extra_specs_1.instance_type_id:31!null instance_type_extra_specs_1.deleted:32 instance_type_extra_specs_1.deleted_at:33 instance_type_extra_specs_1.created_at:34 instance_type_extra_specs_1.updated_at:35 - │ │ ├── key: (28) - │ │ └── fd: (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + │ │ ├── columns: instance_type_extra_specs_1.id:30!null key:31 value:32 instance_type_extra_specs_1.instance_type_id:33!null instance_type_extra_specs_1.deleted:34 instance_type_extra_specs_1.deleted_at:35 instance_type_extra_specs_1.created_at:36 instance_type_extra_specs_1.updated_at:37 + │ │ ├── key: (30) + │ │ └── fd: (30)-->(31-37), (31,33,34)~~>(30,32,35-37) │ └── filters - │ └── instance_type_extra_specs_1.deleted:32 = $7 [outer=(32), constraints=(/32: (/NULL - ])] + │ └── instance_type_extra_specs_1.deleted:34 = $7 [outer=(34), constraints=(/34: (/NULL - ])] ├── limit - │ ├── columns: instance_types.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ ├── columns: instance_types.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ ├── immutable, has-placeholder │ ├── key: (1) - │ ├── fd: (1)-->(2-16,26), (7,13)~~>(1-6,8-12,14-16), (2,13)-->(1,3-12,14-16) + │ ├── fd: (1)-->(2-16,28), (7,13)~~>(1-6,8-12,14-16), (2,13)-->(1,3-12,14-16) │ ├── offset - │ │ ├── columns: instance_types.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ │ ├── columns: instance_types.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ │ ├── has-placeholder │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2-16,26), (7,13)~~>(1-6,8-12,14-16), (2,13)-->(1,3-12,14-16) + │ │ ├── fd: (1)-->(2-16,28), (7,13)~~>(1-6,8-12,14-16), (2,13)-->(1,3-12,14-16) │ │ ├── select - │ │ │ ├── columns: instance_types.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ │ │ ├── columns: instance_types.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ │ │ ├── has-placeholder │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(2-16,26), (7,13)~~>(1-6,8-12,14-16), (2,13)-->(1,3-12,14-16) + │ │ │ ├── fd: (1)-->(2-16,28), (7,13)~~>(1-6,8-12,14-16), (2,13)-->(1,3-12,14-16) │ │ │ ├── group-by - │ │ │ │ ├── columns: instance_types.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ │ │ │ ├── columns: instance_types.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ │ │ │ ├── grouping columns: instance_types.id:1!null │ │ │ │ ├── internal-ordering: +1 │ │ │ │ ├── has-placeholder │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(2-16,26), (7,13)~~>(1-6,8-12,14-16), (2,13)-->(1,3-12,14-16) + │ │ │ │ ├── fd: (1)-->(2-16,28), (7,13)~~>(1-6,8-12,14-16), (2,13)-->(1,3-12,14-16) │ │ │ │ ├── left-join (merge) - │ │ │ │ │ ├── columns: instance_types.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_projects.instance_type_id:18 true:25 + │ │ │ │ │ ├── columns: instance_types.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_projects.instance_type_id:19 true:27 │ │ │ │ │ ├── left ordering: +1 - │ │ │ │ │ ├── right ordering: +18 + │ │ │ │ │ ├── right ordering: +19 │ │ │ │ │ ├── has-placeholder │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)-->(1,3-12,14-16) │ │ │ │ │ ├── ordering: +1 @@ -1007,28 +1007,28 @@ project │ │ │ │ │ │ ├── instance_types.deleted:13 = $1 [outer=(13), constraints=(/13: (/NULL - ])] │ │ │ │ │ │ └── name:2 = $4 [outer=(2), constraints=(/2: (/NULL - ])] │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: true:25!null instance_type_projects.instance_type_id:18!null + │ │ │ │ │ │ ├── columns: true:27!null instance_type_projects.instance_type_id:19!null │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ ├── fd: ()-->(25) - │ │ │ │ │ │ ├── ordering: +18 opt(25) [actual: +18] + │ │ │ │ │ │ ├── fd: ()-->(27) + │ │ │ │ │ │ ├── ordering: +19 opt(27) [actual: +19] │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18!null project_id:19!null instance_type_projects.deleted:20!null + │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:19!null project_id:20!null instance_type_projects.deleted:21!null │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ ├── key: (18-20) - │ │ │ │ │ │ │ ├── ordering: +18 + │ │ │ │ │ │ │ ├── key: (19-21) + │ │ │ │ │ │ │ ├── ordering: +19 │ │ │ │ │ │ │ ├── scan instance_type_projects@secondary - │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18!null project_id:19 instance_type_projects.deleted:20 - │ │ │ │ │ │ │ │ ├── lax-key: (18-20) - │ │ │ │ │ │ │ │ └── ordering: +18 + │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:19!null project_id:20 instance_type_projects.deleted:21 + │ │ │ │ │ │ │ │ ├── lax-key: (19-21) + │ │ │ │ │ │ │ │ └── ordering: +19 │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ ├── instance_type_projects.deleted:20 = $2 [outer=(20), constraints=(/20: (/NULL - ])] - │ │ │ │ │ │ │ └── project_id:19 = $3 [outer=(19), constraints=(/19: (/NULL - ])] + │ │ │ │ │ │ │ ├── instance_type_projects.deleted:21 = $2 [outer=(21), constraints=(/21: (/NULL - ])] + │ │ │ │ │ │ │ └── project_id:20 = $3 [outer=(20), constraints=(/20: (/NULL - ])] │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── true [as=true:25] + │ │ │ │ │ │ └── true [as=true:27] │ │ │ │ │ └── filters (true) │ │ │ │ └── aggregations - │ │ │ │ ├── const-not-null-agg [as=true_agg:26, outer=(25)] - │ │ │ │ │ └── true:25 + │ │ │ │ ├── const-not-null-agg [as=true_agg:28, outer=(27)] + │ │ │ │ │ └── true:27 │ │ │ │ ├── const-agg [as=name:2, outer=(2)] │ │ │ │ │ └── name:2 │ │ │ │ ├── const-agg [as=memory_mb:3, outer=(3)] @@ -1060,11 +1060,11 @@ project │ │ │ │ └── const-agg [as=instance_types.updated_at:16, outer=(16)] │ │ │ │ └── instance_types.updated_at:16 │ │ │ └── filters - │ │ │ └── (is_public:12 = true) OR (true_agg:26 IS NOT NULL) [outer=(12,26)] + │ │ │ └── (is_public:12 = true) OR (true_agg:28 IS NOT NULL) [outer=(12,28)] │ │ └── $5 │ └── $6 └── filters - └── instance_type_extra_specs_1.instance_type_id:31 = instance_types.id:1 [outer=(1,31), constraints=(/1: (/NULL - ]; /31: (/NULL - ]), fd=(1)==(31), (31)==(1)] + └── instance_type_extra_specs_1.instance_type_id:33 = instance_types.id:1 [outer=(1,33), constraints=(/1: (/NULL - ]; /33: (/NULL - ]), fd=(1)==(33), (33)==(1)] opt select anon_1.instance_types_created_at as anon_1_instance_types_created_at, @@ -1124,52 +1124,52 @@ from (select instance_types.created_at as instance_types_created_at, and instance_type_extra_specs_1.deleted = $7 ---- project - ├── columns: anon_1_instance_types_created_at:15 anon_1_instance_types_updated_at:16 anon_1_instance_types_deleted_at:14 anon_1_instance_types_deleted:13!null anon_1_instance_types_id:1!null anon_1_instance_types_name:2 anon_1_instance_types_memory_mb:3!null anon_1_instance_types_vcpus:4!null anon_1_instance_types_root_gb:5 anon_1_instance_types_ephemeral_gb:6 anon_1_instance_types_flavorid:7 anon_1_instance_types_swap:8!null anon_1_instance_types_rxtx_factor:9 anon_1_instance_types_vcpu_weight:10 anon_1_instance_types_disabled:11 anon_1_instance_types_is_public:12 instance_type_extra_specs_1_created_at:34 instance_type_extra_specs_1_updated_at:35 instance_type_extra_specs_1_deleted_at:33 instance_type_extra_specs_1_deleted:32 instance_type_extra_specs_1_id:28 instance_type_extra_specs_1_key:29 instance_type_extra_specs_1_value:30 instance_type_extra_specs_1_instance_type_id:31 + ├── columns: anon_1_instance_types_created_at:15 anon_1_instance_types_updated_at:16 anon_1_instance_types_deleted_at:14 anon_1_instance_types_deleted:13!null anon_1_instance_types_id:1!null anon_1_instance_types_name:2 anon_1_instance_types_memory_mb:3!null anon_1_instance_types_vcpus:4!null anon_1_instance_types_root_gb:5 anon_1_instance_types_ephemeral_gb:6 anon_1_instance_types_flavorid:7 anon_1_instance_types_swap:8!null anon_1_instance_types_rxtx_factor:9 anon_1_instance_types_vcpu_weight:10 anon_1_instance_types_disabled:11 anon_1_instance_types_is_public:12 instance_type_extra_specs_1_created_at:36 instance_type_extra_specs_1_updated_at:37 instance_type_extra_specs_1_deleted_at:35 instance_type_extra_specs_1_deleted:34 instance_type_extra_specs_1_id:30 instance_type_extra_specs_1_key:31 instance_type_extra_specs_1_value:32 instance_type_extra_specs_1_instance_type_id:33 ├── immutable, has-placeholder - ├── key: (1,28) - ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + ├── key: (1,30) + ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (30)-->(31-37), (31,33,34)~~>(30,32,35-37) └── right-join (hash) - ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 instance_type_extra_specs_1.id:28 key:29 value:30 instance_type_extra_specs_1.instance_type_id:31 instance_type_extra_specs_1.deleted:32 instance_type_extra_specs_1.deleted_at:33 instance_type_extra_specs_1.created_at:34 instance_type_extra_specs_1.updated_at:35 + ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 instance_type_extra_specs_1.id:30 key:31 value:32 instance_type_extra_specs_1.instance_type_id:33 instance_type_extra_specs_1.deleted:34 instance_type_extra_specs_1.deleted_at:35 instance_type_extra_specs_1.created_at:36 instance_type_extra_specs_1.updated_at:37 ├── immutable, has-placeholder - ├── key: (1,28) - ├── fd: (1)-->(2-16,26), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + ├── key: (1,30) + ├── fd: (1)-->(2-16,28), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (30)-->(31-37), (31,33,34)~~>(30,32,35-37) ├── select - │ ├── columns: instance_type_extra_specs_1.id:28!null key:29 value:30 instance_type_extra_specs_1.instance_type_id:31!null instance_type_extra_specs_1.deleted:32!null instance_type_extra_specs_1.deleted_at:33 instance_type_extra_specs_1.created_at:34 instance_type_extra_specs_1.updated_at:35 + │ ├── columns: instance_type_extra_specs_1.id:30!null key:31 value:32 instance_type_extra_specs_1.instance_type_id:33!null instance_type_extra_specs_1.deleted:34!null instance_type_extra_specs_1.deleted_at:35 instance_type_extra_specs_1.created_at:36 instance_type_extra_specs_1.updated_at:37 │ ├── has-placeholder - │ ├── key: (28) - │ ├── fd: (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + │ ├── key: (30) + │ ├── fd: (30)-->(31-37), (31,33,34)~~>(30,32,35-37) │ ├── scan instance_type_extra_specs_1 - │ │ ├── columns: instance_type_extra_specs_1.id:28!null key:29 value:30 instance_type_extra_specs_1.instance_type_id:31!null instance_type_extra_specs_1.deleted:32 instance_type_extra_specs_1.deleted_at:33 instance_type_extra_specs_1.created_at:34 instance_type_extra_specs_1.updated_at:35 - │ │ ├── key: (28) - │ │ └── fd: (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + │ │ ├── columns: instance_type_extra_specs_1.id:30!null key:31 value:32 instance_type_extra_specs_1.instance_type_id:33!null instance_type_extra_specs_1.deleted:34 instance_type_extra_specs_1.deleted_at:35 instance_type_extra_specs_1.created_at:36 instance_type_extra_specs_1.updated_at:37 + │ │ ├── key: (30) + │ │ └── fd: (30)-->(31-37), (31,33,34)~~>(30,32,35-37) │ └── filters - │ └── instance_type_extra_specs_1.deleted:32 = $7 [outer=(32), constraints=(/32: (/NULL - ])] + │ └── instance_type_extra_specs_1.deleted:34 = $7 [outer=(34), constraints=(/34: (/NULL - ])] ├── limit - │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ ├── immutable, has-placeholder │ ├── key: (1) - │ ├── fd: (1)-->(2-16,26), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ ├── fd: (1)-->(2-16,28), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ ├── offset - │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ │ ├── has-placeholder │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2-16,26), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ ├── fd: (1)-->(2-16,28), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ ├── select - │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ │ │ ├── has-placeholder │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(2-16,26), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ │ ├── fd: (1)-->(2-16,28), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ │ ├── group-by - │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ │ │ │ ├── grouping columns: instance_types.id:1!null │ │ │ │ ├── internal-ordering: +1 │ │ │ │ ├── has-placeholder │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(2-16,26), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ │ │ ├── fd: (1)-->(2-16,28), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ │ │ ├── left-join (merge) - │ │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_projects.instance_type_id:18 true:25 + │ │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_projects.instance_type_id:19 true:27 │ │ │ │ │ ├── left ordering: +1 - │ │ │ │ │ ├── right ordering: +18 + │ │ │ │ │ ├── right ordering: +19 │ │ │ │ │ ├── has-placeholder │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ │ │ │ ├── ordering: +1 @@ -1188,29 +1188,29 @@ project │ │ │ │ │ │ ├── instance_types.id:1 = $4 [outer=(1), constraints=(/1: (/NULL - ])] │ │ │ │ │ │ └── instance_types.deleted:13 = $1 [outer=(13), constraints=(/13: (/NULL - ])] │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: true:25!null instance_type_projects.instance_type_id:18!null + │ │ │ │ │ │ ├── columns: true:27!null instance_type_projects.instance_type_id:19!null │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ ├── fd: ()-->(25) - │ │ │ │ │ │ ├── ordering: +18 opt(25) [actual: +18] + │ │ │ │ │ │ ├── fd: ()-->(27) + │ │ │ │ │ │ ├── ordering: +19 opt(27) [actual: +19] │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18!null project_id:19!null instance_type_projects.deleted:20!null + │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:19!null project_id:20!null instance_type_projects.deleted:21!null │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ ├── key: (18-20) - │ │ │ │ │ │ │ ├── ordering: +18 + │ │ │ │ │ │ │ ├── key: (19-21) + │ │ │ │ │ │ │ ├── ordering: +19 │ │ │ │ │ │ │ ├── scan instance_type_projects@secondary - │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18!null project_id:19 instance_type_projects.deleted:20 - │ │ │ │ │ │ │ │ ├── lax-key: (18-20) - │ │ │ │ │ │ │ │ └── ordering: +18 + │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:19!null project_id:20 instance_type_projects.deleted:21 + │ │ │ │ │ │ │ │ ├── lax-key: (19-21) + │ │ │ │ │ │ │ │ └── ordering: +19 │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ ├── instance_type_projects.deleted:20 = $2 [outer=(20), constraints=(/20: (/NULL - ])] - │ │ │ │ │ │ │ ├── project_id:19 = $3 [outer=(19), constraints=(/19: (/NULL - ])] - │ │ │ │ │ │ │ └── instance_type_projects.instance_type_id:18 = $4 [outer=(18), constraints=(/18: (/NULL - ])] + │ │ │ │ │ │ │ ├── instance_type_projects.deleted:21 = $2 [outer=(21), constraints=(/21: (/NULL - ])] + │ │ │ │ │ │ │ ├── project_id:20 = $3 [outer=(20), constraints=(/20: (/NULL - ])] + │ │ │ │ │ │ │ └── instance_type_projects.instance_type_id:19 = $4 [outer=(19), constraints=(/19: (/NULL - ])] │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── true [as=true:25] + │ │ │ │ │ │ └── true [as=true:27] │ │ │ │ │ └── filters (true) │ │ │ │ └── aggregations - │ │ │ │ ├── const-not-null-agg [as=true_agg:26, outer=(25)] - │ │ │ │ │ └── true:25 + │ │ │ │ ├── const-not-null-agg [as=true_agg:28, outer=(27)] + │ │ │ │ │ └── true:27 │ │ │ │ ├── const-agg [as=name:2, outer=(2)] │ │ │ │ │ └── name:2 │ │ │ │ ├── const-agg [as=memory_mb:3, outer=(3)] @@ -1242,11 +1242,11 @@ project │ │ │ │ └── const-agg [as=instance_types.updated_at:16, outer=(16)] │ │ │ │ └── instance_types.updated_at:16 │ │ │ └── filters - │ │ │ └── (is_public:12 = true) OR (true_agg:26 IS NOT NULL) [outer=(12,26)] + │ │ │ └── (is_public:12 = true) OR (true_agg:28 IS NOT NULL) [outer=(12,28)] │ │ └── $5 │ └── $6 └── filters - └── instance_type_extra_specs_1.instance_type_id:31 = instance_types.id:1 [outer=(1,31), constraints=(/1: (/NULL - ]; /31: (/NULL - ]), fd=(1)==(31), (31)==(1)] + └── instance_type_extra_specs_1.instance_type_id:33 = instance_types.id:1 [outer=(1,33), constraints=(/1: (/NULL - ]; /33: (/NULL - ]), fd=(1)==(33), (33)==(1)] opt select anon_1.flavors_created_at as anon_1_flavors_created_at, @@ -1297,45 +1297,45 @@ from (select flavors.created_at as flavors_created_at, on flavor_extra_specs_1.flavor_id = anon_1.flavors_id ---- project - ├── columns: anon_1_flavors_created_at:14 anon_1_flavors_updated_at:15 anon_1_flavors_id:1!null anon_1_flavors_name:2!null anon_1_flavors_memory_mb:3!null anon_1_flavors_vcpus:4!null anon_1_flavors_root_gb:5 anon_1_flavors_ephemeral_gb:6 anon_1_flavors_flavorid:7!null anon_1_flavors_swap:8!null anon_1_flavors_rxtx_factor:9 anon_1_flavors_vcpu_weight:10 anon_1_flavors_disabled:11 anon_1_flavors_is_public:12 flavor_extra_specs_1_created_at:29 flavor_extra_specs_1_updated_at:30 flavor_extra_specs_1_id:25 flavor_extra_specs_1_key:26 flavor_extra_specs_1_value:27 flavor_extra_specs_1_flavor_id:28 + ├── columns: anon_1_flavors_created_at:14 anon_1_flavors_updated_at:15 anon_1_flavors_id:1!null anon_1_flavors_name:2!null anon_1_flavors_memory_mb:3!null anon_1_flavors_vcpus:4!null anon_1_flavors_root_gb:5 anon_1_flavors_ephemeral_gb:6 anon_1_flavors_flavorid:7!null anon_1_flavors_swap:8!null anon_1_flavors_rxtx_factor:9 anon_1_flavors_vcpu_weight:10 anon_1_flavors_disabled:11 anon_1_flavors_is_public:12 flavor_extra_specs_1_created_at:31 flavor_extra_specs_1_updated_at:32 flavor_extra_specs_1_id:27 flavor_extra_specs_1_key:28 flavor_extra_specs_1_value:29 flavor_extra_specs_1_flavor_id:30 ├── immutable, has-placeholder - ├── key: (1,25) - ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (25)-->(26-30), (26,28)-->(25,27,29,30) + ├── key: (1,27) + ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (27)-->(28-32), (28,30)-->(27,29,31,32) └── right-join (hash) - ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 flavor_extra_specs_1.id:25 key:26 value:27 flavor_extra_specs_1.flavor_id:28 flavor_extra_specs_1.created_at:29 flavor_extra_specs_1.updated_at:30 + ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 flavor_extra_specs_1.id:27 key:28 value:29 flavor_extra_specs_1.flavor_id:30 flavor_extra_specs_1.created_at:31 flavor_extra_specs_1.updated_at:32 ├── immutable, has-placeholder - ├── key: (1,25) - ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (25)-->(26-30), (26,28)-->(25,27,29,30) + ├── key: (1,27) + ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (27)-->(28-32), (28,30)-->(27,29,31,32) ├── scan flavor_extra_specs_1 - │ ├── columns: flavor_extra_specs_1.id:25!null key:26!null value:27 flavor_extra_specs_1.flavor_id:28!null flavor_extra_specs_1.created_at:29 flavor_extra_specs_1.updated_at:30 - │ ├── key: (25) - │ └── fd: (25)-->(26-30), (26,28)-->(25,27,29,30) + │ ├── columns: flavor_extra_specs_1.id:27!null key:28!null value:29 flavor_extra_specs_1.flavor_id:30!null flavor_extra_specs_1.created_at:31 flavor_extra_specs_1.updated_at:32 + │ ├── key: (27) + │ └── fd: (27)-->(28-32), (28,30)-->(27,29,31,32) ├── limit - │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 + │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 │ ├── immutable, has-placeholder │ ├── key: (1) - │ ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) + │ ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ ├── offset - │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 + │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 │ │ ├── has-placeholder │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) + │ │ ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ ├── select - │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 + │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 │ │ │ ├── has-placeholder │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) + │ │ │ ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ │ ├── group-by - │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 + │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 │ │ │ │ ├── grouping columns: flavors.id:1!null │ │ │ │ ├── internal-ordering: +1 │ │ │ │ ├── has-placeholder │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) + │ │ │ │ ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ │ │ ├── left-join (merge) - │ │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 flavor_projects.flavor_id:17 true:22 + │ │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 flavor_projects.flavor_id:18 true:24 │ │ │ │ │ ├── left ordering: +1 - │ │ │ │ │ ├── right ordering: +17 + │ │ │ │ │ ├── right ordering: +18 │ │ │ │ │ ├── has-placeholder │ │ │ │ │ ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ │ │ │ ├── ordering: +1 @@ -1353,27 +1353,27 @@ project │ │ │ │ │ │ └── filters │ │ │ │ │ │ └── name:2 = $2 [outer=(2), constraints=(/2: (/NULL - ])] │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: true:22!null flavor_projects.flavor_id:17!null + │ │ │ │ │ │ ├── columns: true:24!null flavor_projects.flavor_id:18!null │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ ├── fd: ()-->(22) - │ │ │ │ │ │ ├── ordering: +17 opt(22) [actual: +17] + │ │ │ │ │ │ ├── fd: ()-->(24) + │ │ │ │ │ │ ├── ordering: +18 opt(24) [actual: +18] │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:17!null project_id:18!null + │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:18!null project_id:19!null │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ ├── key: (17,18) - │ │ │ │ │ │ │ ├── ordering: +17 + │ │ │ │ │ │ │ ├── key: (18,19) + │ │ │ │ │ │ │ ├── ordering: +18 │ │ │ │ │ │ │ ├── scan flavor_projects@secondary - │ │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:17!null project_id:18!null - │ │ │ │ │ │ │ │ ├── key: (17,18) - │ │ │ │ │ │ │ │ └── ordering: +17 + │ │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:18!null project_id:19!null + │ │ │ │ │ │ │ │ ├── key: (18,19) + │ │ │ │ │ │ │ │ └── ordering: +18 │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── project_id:18 = $1 [outer=(18), constraints=(/18: (/NULL - ])] + │ │ │ │ │ │ │ └── project_id:19 = $1 [outer=(19), constraints=(/19: (/NULL - ])] │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── true [as=true:22] + │ │ │ │ │ │ └── true [as=true:24] │ │ │ │ │ └── filters (true) │ │ │ │ └── aggregations - │ │ │ │ ├── const-not-null-agg [as=true_agg:23, outer=(22)] - │ │ │ │ │ └── true:22 + │ │ │ │ ├── const-not-null-agg [as=true_agg:25, outer=(24)] + │ │ │ │ │ └── true:24 │ │ │ │ ├── const-agg [as=name:2, outer=(2)] │ │ │ │ │ └── name:2 │ │ │ │ ├── const-agg [as=memory_mb:3, outer=(3)] @@ -1401,11 +1401,11 @@ project │ │ │ │ └── const-agg [as=flavors.updated_at:15, outer=(15)] │ │ │ │ └── flavors.updated_at:15 │ │ │ └── filters - │ │ │ └── (is_public:12 = true) OR (true_agg:23 IS NOT NULL) [outer=(12,23)] + │ │ │ └── (is_public:12 = true) OR (true_agg:25 IS NOT NULL) [outer=(12,25)] │ │ └── $3 │ └── $4 └── filters - └── flavor_extra_specs_1.flavor_id:28 = flavors.id:1 [outer=(1,28), constraints=(/1: (/NULL - ]; /28: (/NULL - ]), fd=(1)==(28), (28)==(1)] + └── flavor_extra_specs_1.flavor_id:30 = flavors.id:1 [outer=(1,30), constraints=(/1: (/NULL - ]; /30: (/NULL - ]), fd=(1)==(30), (30)==(1)] opt select anon_1.flavors_created_at as anon_1_flavors_created_at, @@ -1456,45 +1456,45 @@ from (select flavors.created_at as flavors_created_at, on flavor_extra_specs_1.flavor_id = anon_1.flavors_id ---- project - ├── columns: anon_1_flavors_created_at:14 anon_1_flavors_updated_at:15 anon_1_flavors_id:1!null anon_1_flavors_name:2!null anon_1_flavors_memory_mb:3!null anon_1_flavors_vcpus:4!null anon_1_flavors_root_gb:5 anon_1_flavors_ephemeral_gb:6 anon_1_flavors_flavorid:7!null anon_1_flavors_swap:8!null anon_1_flavors_rxtx_factor:9 anon_1_flavors_vcpu_weight:10 anon_1_flavors_disabled:11 anon_1_flavors_is_public:12 flavor_extra_specs_1_created_at:29 flavor_extra_specs_1_updated_at:30 flavor_extra_specs_1_id:25 flavor_extra_specs_1_key:26 flavor_extra_specs_1_value:27 flavor_extra_specs_1_flavor_id:28 + ├── columns: anon_1_flavors_created_at:14 anon_1_flavors_updated_at:15 anon_1_flavors_id:1!null anon_1_flavors_name:2!null anon_1_flavors_memory_mb:3!null anon_1_flavors_vcpus:4!null anon_1_flavors_root_gb:5 anon_1_flavors_ephemeral_gb:6 anon_1_flavors_flavorid:7!null anon_1_flavors_swap:8!null anon_1_flavors_rxtx_factor:9 anon_1_flavors_vcpu_weight:10 anon_1_flavors_disabled:11 anon_1_flavors_is_public:12 flavor_extra_specs_1_created_at:31 flavor_extra_specs_1_updated_at:32 flavor_extra_specs_1_id:27 flavor_extra_specs_1_key:28 flavor_extra_specs_1_value:29 flavor_extra_specs_1_flavor_id:30 ├── immutable, has-placeholder - ├── key: (1,25) - ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (25)-->(26-30), (26,28)-->(25,27,29,30) + ├── key: (1,27) + ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (27)-->(28-32), (28,30)-->(27,29,31,32) └── right-join (hash) - ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 flavor_extra_specs_1.id:25 key:26 value:27 flavor_extra_specs_1.flavor_id:28 flavor_extra_specs_1.created_at:29 flavor_extra_specs_1.updated_at:30 + ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 flavor_extra_specs_1.id:27 key:28 value:29 flavor_extra_specs_1.flavor_id:30 flavor_extra_specs_1.created_at:31 flavor_extra_specs_1.updated_at:32 ├── immutable, has-placeholder - ├── key: (1,25) - ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (25)-->(26-30), (26,28)-->(25,27,29,30) + ├── key: (1,27) + ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (27)-->(28-32), (28,30)-->(27,29,31,32) ├── scan flavor_extra_specs_1 - │ ├── columns: flavor_extra_specs_1.id:25!null key:26!null value:27 flavor_extra_specs_1.flavor_id:28!null flavor_extra_specs_1.created_at:29 flavor_extra_specs_1.updated_at:30 - │ ├── key: (25) - │ └── fd: (25)-->(26-30), (26,28)-->(25,27,29,30) + │ ├── columns: flavor_extra_specs_1.id:27!null key:28!null value:29 flavor_extra_specs_1.flavor_id:30!null flavor_extra_specs_1.created_at:31 flavor_extra_specs_1.updated_at:32 + │ ├── key: (27) + │ └── fd: (27)-->(28-32), (28,30)-->(27,29,31,32) ├── limit - │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 + │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 │ ├── immutable, has-placeholder │ ├── key: (1) - │ ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) + │ ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ ├── offset - │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 + │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 │ │ ├── has-placeholder │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) + │ │ ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ ├── select - │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 + │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 │ │ │ ├── has-placeholder │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) + │ │ │ ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ │ ├── group-by - │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 + │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 │ │ │ │ ├── grouping columns: flavors.id:1!null │ │ │ │ ├── internal-ordering: +1 │ │ │ │ ├── has-placeholder │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) + │ │ │ │ ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ │ │ ├── left-join (merge) - │ │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 flavor_projects.flavor_id:17 true:22 + │ │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 flavor_projects.flavor_id:18 true:24 │ │ │ │ │ ├── left ordering: +1 - │ │ │ │ │ ├── right ordering: +17 + │ │ │ │ │ ├── right ordering: +18 │ │ │ │ │ ├── has-placeholder │ │ │ │ │ ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ │ │ │ ├── ordering: +1 @@ -1512,27 +1512,27 @@ project │ │ │ │ │ │ └── filters │ │ │ │ │ │ └── flavorid:7 = $2 [outer=(7), constraints=(/7: (/NULL - ])] │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: true:22!null flavor_projects.flavor_id:17!null + │ │ │ │ │ │ ├── columns: true:24!null flavor_projects.flavor_id:18!null │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ ├── fd: ()-->(22) - │ │ │ │ │ │ ├── ordering: +17 opt(22) [actual: +17] + │ │ │ │ │ │ ├── fd: ()-->(24) + │ │ │ │ │ │ ├── ordering: +18 opt(24) [actual: +18] │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:17!null project_id:18!null + │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:18!null project_id:19!null │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ ├── key: (17,18) - │ │ │ │ │ │ │ ├── ordering: +17 + │ │ │ │ │ │ │ ├── key: (18,19) + │ │ │ │ │ │ │ ├── ordering: +18 │ │ │ │ │ │ │ ├── scan flavor_projects@secondary - │ │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:17!null project_id:18!null - │ │ │ │ │ │ │ │ ├── key: (17,18) - │ │ │ │ │ │ │ │ └── ordering: +17 + │ │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:18!null project_id:19!null + │ │ │ │ │ │ │ │ ├── key: (18,19) + │ │ │ │ │ │ │ │ └── ordering: +18 │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── project_id:18 = $1 [outer=(18), constraints=(/18: (/NULL - ])] + │ │ │ │ │ │ │ └── project_id:19 = $1 [outer=(19), constraints=(/19: (/NULL - ])] │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── true [as=true:22] + │ │ │ │ │ │ └── true [as=true:24] │ │ │ │ │ └── filters (true) │ │ │ │ └── aggregations - │ │ │ │ ├── const-not-null-agg [as=true_agg:23, outer=(22)] - │ │ │ │ │ └── true:22 + │ │ │ │ ├── const-not-null-agg [as=true_agg:25, outer=(24)] + │ │ │ │ │ └── true:24 │ │ │ │ ├── const-agg [as=name:2, outer=(2)] │ │ │ │ │ └── name:2 │ │ │ │ ├── const-agg [as=memory_mb:3, outer=(3)] @@ -1560,11 +1560,11 @@ project │ │ │ │ └── const-agg [as=flavors.updated_at:15, outer=(15)] │ │ │ │ └── flavors.updated_at:15 │ │ │ └── filters - │ │ │ └── (is_public:12 = true) OR (true_agg:23 IS NOT NULL) [outer=(12,23)] + │ │ │ └── (is_public:12 = true) OR (true_agg:25 IS NOT NULL) [outer=(12,25)] │ │ └── $3 │ └── $4 └── filters - └── flavor_extra_specs_1.flavor_id:28 = flavors.id:1 [outer=(1,28), constraints=(/1: (/NULL - ]; /28: (/NULL - ]), fd=(1)==(28), (28)==(1)] + └── flavor_extra_specs_1.flavor_id:30 = flavors.id:1 [outer=(1,30), constraints=(/1: (/NULL - ]; /30: (/NULL - ]), fd=(1)==(30), (30)==(1)] opt select anon_1.flavors_created_at as anon_1_flavors_created_at, @@ -1619,60 +1619,60 @@ from (select flavors.created_at as flavors_created_at, order by anon_1.flavors_flavorid asc, anon_1.flavors_id asc ---- sort - ├── columns: anon_1_flavors_created_at:14 anon_1_flavors_updated_at:15 anon_1_flavors_id:1!null anon_1_flavors_name:2!null anon_1_flavors_memory_mb:3!null anon_1_flavors_vcpus:4!null anon_1_flavors_root_gb:5 anon_1_flavors_ephemeral_gb:6 anon_1_flavors_flavorid:7!null anon_1_flavors_swap:8!null anon_1_flavors_rxtx_factor:9 anon_1_flavors_vcpu_weight:10 anon_1_flavors_disabled:11 anon_1_flavors_is_public:12 flavor_extra_specs_1_created_at:29 flavor_extra_specs_1_updated_at:30 flavor_extra_specs_1_id:25 flavor_extra_specs_1_key:26 flavor_extra_specs_1_value:27 flavor_extra_specs_1_flavor_id:28 + ├── columns: anon_1_flavors_created_at:14 anon_1_flavors_updated_at:15 anon_1_flavors_id:1!null anon_1_flavors_name:2!null anon_1_flavors_memory_mb:3!null anon_1_flavors_vcpus:4!null anon_1_flavors_root_gb:5 anon_1_flavors_ephemeral_gb:6 anon_1_flavors_flavorid:7!null anon_1_flavors_swap:8!null anon_1_flavors_rxtx_factor:9 anon_1_flavors_vcpu_weight:10 anon_1_flavors_disabled:11 anon_1_flavors_is_public:12 flavor_extra_specs_1_created_at:31 flavor_extra_specs_1_updated_at:32 flavor_extra_specs_1_id:27 flavor_extra_specs_1_key:28 flavor_extra_specs_1_value:29 flavor_extra_specs_1_flavor_id:30 ├── immutable, has-placeholder - ├── key: (1,25) - ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (25)-->(26-30), (26,28)-->(25,27,29,30) + ├── key: (1,27) + ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (27)-->(28-32), (28,30)-->(27,29,31,32) ├── ordering: +7 └── project - ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 flavor_extra_specs_1.id:25 key:26 value:27 flavor_extra_specs_1.flavor_id:28 flavor_extra_specs_1.created_at:29 flavor_extra_specs_1.updated_at:30 + ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 flavor_extra_specs_1.id:27 key:28 value:29 flavor_extra_specs_1.flavor_id:30 flavor_extra_specs_1.created_at:31 flavor_extra_specs_1.updated_at:32 ├── immutable, has-placeholder - ├── key: (1,25) - ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (25)-->(26-30), (26,28)-->(25,27,29,30) + ├── key: (1,27) + ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (27)-->(28-32), (28,30)-->(27,29,31,32) └── right-join (hash) - ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 flavor_extra_specs_1.id:25 key:26 value:27 flavor_extra_specs_1.flavor_id:28 flavor_extra_specs_1.created_at:29 flavor_extra_specs_1.updated_at:30 + ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 flavor_extra_specs_1.id:27 key:28 value:29 flavor_extra_specs_1.flavor_id:30 flavor_extra_specs_1.created_at:31 flavor_extra_specs_1.updated_at:32 ├── immutable, has-placeholder - ├── key: (1,25) - ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (25)-->(26-30), (26,28)-->(25,27,29,30) + ├── key: (1,27) + ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (27)-->(28-32), (28,30)-->(27,29,31,32) ├── scan flavor_extra_specs_1 - │ ├── columns: flavor_extra_specs_1.id:25!null key:26!null value:27 flavor_extra_specs_1.flavor_id:28!null flavor_extra_specs_1.created_at:29 flavor_extra_specs_1.updated_at:30 - │ ├── key: (25) - │ └── fd: (25)-->(26-30), (26,28)-->(25,27,29,30) + │ ├── columns: flavor_extra_specs_1.id:27!null key:28!null value:29 flavor_extra_specs_1.flavor_id:30!null flavor_extra_specs_1.created_at:31 flavor_extra_specs_1.updated_at:32 + │ ├── key: (27) + │ └── fd: (27)-->(28-32), (28,30)-->(27,29,31,32) ├── limit - │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 + │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 │ ├── internal-ordering: +7 │ ├── immutable, has-placeholder │ ├── key: (1) - │ ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) + │ ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ ├── offset - │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 + │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 │ │ ├── internal-ordering: +7 │ │ ├── has-placeholder │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) + │ │ ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ ├── ordering: +7 │ │ ├── sort - │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 + │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 │ │ │ ├── has-placeholder │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) + │ │ │ ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ │ ├── ordering: +7 │ │ │ └── select - │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 + │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 │ │ │ ├── has-placeholder │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) + │ │ │ ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ │ ├── group-by - │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 + │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 │ │ │ │ ├── grouping columns: flavors.id:1!null │ │ │ │ ├── internal-ordering: +1 │ │ │ │ ├── has-placeholder │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) + │ │ │ │ ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ │ │ ├── left-join (merge) - │ │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 flavor_projects.flavor_id:17 true:22 + │ │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 flavor_projects.flavor_id:18 true:24 │ │ │ │ │ ├── left ordering: +1 - │ │ │ │ │ ├── right ordering: +17 + │ │ │ │ │ ├── right ordering: +18 │ │ │ │ │ ├── has-placeholder │ │ │ │ │ ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ │ │ │ ├── ordering: +1 @@ -1690,27 +1690,27 @@ sort │ │ │ │ │ │ └── filters │ │ │ │ │ │ └── (flavorid:7 > $2) OR ((flavorid:7 = $3) AND (flavors.id:1 > $4)) [outer=(1,7), constraints=(/7: (/NULL - ])] │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: true:22!null flavor_projects.flavor_id:17!null + │ │ │ │ │ │ ├── columns: true:24!null flavor_projects.flavor_id:18!null │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ ├── fd: ()-->(22) - │ │ │ │ │ │ ├── ordering: +17 opt(22) [actual: +17] + │ │ │ │ │ │ ├── fd: ()-->(24) + │ │ │ │ │ │ ├── ordering: +18 opt(24) [actual: +18] │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:17!null project_id:18!null + │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:18!null project_id:19!null │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ ├── key: (17,18) - │ │ │ │ │ │ │ ├── ordering: +17 + │ │ │ │ │ │ │ ├── key: (18,19) + │ │ │ │ │ │ │ ├── ordering: +18 │ │ │ │ │ │ │ ├── scan flavor_projects@secondary - │ │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:17!null project_id:18!null - │ │ │ │ │ │ │ │ ├── key: (17,18) - │ │ │ │ │ │ │ │ └── ordering: +17 + │ │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:18!null project_id:19!null + │ │ │ │ │ │ │ │ ├── key: (18,19) + │ │ │ │ │ │ │ │ └── ordering: +18 │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── project_id:18 = $1 [outer=(18), constraints=(/18: (/NULL - ])] + │ │ │ │ │ │ │ └── project_id:19 = $1 [outer=(19), constraints=(/19: (/NULL - ])] │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── true [as=true:22] + │ │ │ │ │ │ └── true [as=true:24] │ │ │ │ │ └── filters (true) │ │ │ │ └── aggregations - │ │ │ │ ├── const-not-null-agg [as=true_agg:23, outer=(22)] - │ │ │ │ │ └── true:22 + │ │ │ │ ├── const-not-null-agg [as=true_agg:25, outer=(24)] + │ │ │ │ │ └── true:24 │ │ │ │ ├── const-agg [as=name:2, outer=(2)] │ │ │ │ │ └── name:2 │ │ │ │ ├── const-agg [as=memory_mb:3, outer=(3)] @@ -1738,11 +1738,11 @@ sort │ │ │ │ └── const-agg [as=flavors.updated_at:15, outer=(15)] │ │ │ │ └── flavors.updated_at:15 │ │ │ └── filters - │ │ │ └── (is_public:12 = true) OR (true_agg:23 IS NOT NULL) [outer=(12,23)] + │ │ │ └── (is_public:12 = true) OR (true_agg:25 IS NOT NULL) [outer=(12,25)] │ │ └── $5 │ └── $6 └── filters - └── flavor_extra_specs_1.flavor_id:28 = flavors.id:1 [outer=(1,28), constraints=(/1: (/NULL - ]; /28: (/NULL - ]), fd=(1)==(28), (28)==(1)] + └── flavor_extra_specs_1.flavor_id:30 = flavors.id:1 [outer=(1,30), constraints=(/1: (/NULL - ]; /30: (/NULL - ]), fd=(1)==(30), (30)==(1)] opt select anon_1.instance_types_created_at as anon_1_instance_types_created_at, @@ -1804,67 +1804,67 @@ order by anon_1.instance_types_flavorid asc, anon_1.instance_types_id asc ---- sort - ├── columns: anon_1_instance_types_created_at:15 anon_1_instance_types_updated_at:16 anon_1_instance_types_deleted_at:14 anon_1_instance_types_deleted:13!null anon_1_instance_types_id:1!null anon_1_instance_types_name:2 anon_1_instance_types_memory_mb:3!null anon_1_instance_types_vcpus:4!null anon_1_instance_types_root_gb:5 anon_1_instance_types_ephemeral_gb:6 anon_1_instance_types_flavorid:7 anon_1_instance_types_swap:8!null anon_1_instance_types_rxtx_factor:9 anon_1_instance_types_vcpu_weight:10 anon_1_instance_types_disabled:11 anon_1_instance_types_is_public:12 instance_type_extra_specs_1_created_at:34 instance_type_extra_specs_1_updated_at:35 instance_type_extra_specs_1_deleted_at:33 instance_type_extra_specs_1_deleted:32 instance_type_extra_specs_1_id:28 instance_type_extra_specs_1_key:29 instance_type_extra_specs_1_value:30 instance_type_extra_specs_1_instance_type_id:31 + ├── columns: anon_1_instance_types_created_at:15 anon_1_instance_types_updated_at:16 anon_1_instance_types_deleted_at:14 anon_1_instance_types_deleted:13!null anon_1_instance_types_id:1!null anon_1_instance_types_name:2 anon_1_instance_types_memory_mb:3!null anon_1_instance_types_vcpus:4!null anon_1_instance_types_root_gb:5 anon_1_instance_types_ephemeral_gb:6 anon_1_instance_types_flavorid:7 anon_1_instance_types_swap:8!null anon_1_instance_types_rxtx_factor:9 anon_1_instance_types_vcpu_weight:10 anon_1_instance_types_disabled:11 anon_1_instance_types_is_public:12 instance_type_extra_specs_1_created_at:36 instance_type_extra_specs_1_updated_at:37 instance_type_extra_specs_1_deleted_at:35 instance_type_extra_specs_1_deleted:34 instance_type_extra_specs_1_id:30 instance_type_extra_specs_1_key:31 instance_type_extra_specs_1_value:32 instance_type_extra_specs_1_instance_type_id:33 ├── immutable, has-placeholder - ├── key: (1,28) - ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + ├── key: (1,30) + ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (30)-->(31-37), (31,33,34)~~>(30,32,35-37) ├── ordering: +7,+1 └── project - ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_extra_specs_1.id:28 key:29 value:30 instance_type_extra_specs_1.instance_type_id:31 instance_type_extra_specs_1.deleted:32 instance_type_extra_specs_1.deleted_at:33 instance_type_extra_specs_1.created_at:34 instance_type_extra_specs_1.updated_at:35 + ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_extra_specs_1.id:30 key:31 value:32 instance_type_extra_specs_1.instance_type_id:33 instance_type_extra_specs_1.deleted:34 instance_type_extra_specs_1.deleted_at:35 instance_type_extra_specs_1.created_at:36 instance_type_extra_specs_1.updated_at:37 ├── immutable, has-placeholder - ├── key: (1,28) - ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + ├── key: (1,30) + ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (30)-->(31-37), (31,33,34)~~>(30,32,35-37) └── right-join (hash) - ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 instance_type_extra_specs_1.id:28 key:29 value:30 instance_type_extra_specs_1.instance_type_id:31 instance_type_extra_specs_1.deleted:32 instance_type_extra_specs_1.deleted_at:33 instance_type_extra_specs_1.created_at:34 instance_type_extra_specs_1.updated_at:35 + ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 instance_type_extra_specs_1.id:30 key:31 value:32 instance_type_extra_specs_1.instance_type_id:33 instance_type_extra_specs_1.deleted:34 instance_type_extra_specs_1.deleted_at:35 instance_type_extra_specs_1.created_at:36 instance_type_extra_specs_1.updated_at:37 ├── immutable, has-placeholder - ├── key: (1,28) - ├── fd: (1)-->(2-16,26), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + ├── key: (1,30) + ├── fd: (1)-->(2-16,28), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (30)-->(31-37), (31,33,34)~~>(30,32,35-37) ├── select - │ ├── columns: instance_type_extra_specs_1.id:28!null key:29 value:30 instance_type_extra_specs_1.instance_type_id:31!null instance_type_extra_specs_1.deleted:32!null instance_type_extra_specs_1.deleted_at:33 instance_type_extra_specs_1.created_at:34 instance_type_extra_specs_1.updated_at:35 + │ ├── columns: instance_type_extra_specs_1.id:30!null key:31 value:32 instance_type_extra_specs_1.instance_type_id:33!null instance_type_extra_specs_1.deleted:34!null instance_type_extra_specs_1.deleted_at:35 instance_type_extra_specs_1.created_at:36 instance_type_extra_specs_1.updated_at:37 │ ├── has-placeholder - │ ├── key: (28) - │ ├── fd: (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + │ ├── key: (30) + │ ├── fd: (30)-->(31-37), (31,33,34)~~>(30,32,35-37) │ ├── scan instance_type_extra_specs_1 - │ │ ├── columns: instance_type_extra_specs_1.id:28!null key:29 value:30 instance_type_extra_specs_1.instance_type_id:31!null instance_type_extra_specs_1.deleted:32 instance_type_extra_specs_1.deleted_at:33 instance_type_extra_specs_1.created_at:34 instance_type_extra_specs_1.updated_at:35 - │ │ ├── key: (28) - │ │ └── fd: (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + │ │ ├── columns: instance_type_extra_specs_1.id:30!null key:31 value:32 instance_type_extra_specs_1.instance_type_id:33!null instance_type_extra_specs_1.deleted:34 instance_type_extra_specs_1.deleted_at:35 instance_type_extra_specs_1.created_at:36 instance_type_extra_specs_1.updated_at:37 + │ │ ├── key: (30) + │ │ └── fd: (30)-->(31-37), (31,33,34)~~>(30,32,35-37) │ └── filters - │ └── instance_type_extra_specs_1.deleted:32 = $6 [outer=(32), constraints=(/32: (/NULL - ])] + │ └── instance_type_extra_specs_1.deleted:34 = $6 [outer=(34), constraints=(/34: (/NULL - ])] ├── limit - │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ ├── internal-ordering: +7,+1 │ ├── immutable, has-placeholder │ ├── key: (1) - │ ├── fd: (1)-->(2-16,26), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ ├── fd: (1)-->(2-16,28), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ ├── offset - │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ │ ├── internal-ordering: +7,+1 │ │ ├── has-placeholder │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2-16,26), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ ├── fd: (1)-->(2-16,28), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ ├── ordering: +7,+1 │ │ ├── sort - │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ │ │ ├── has-placeholder │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(2-16,26), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ │ ├── fd: (1)-->(2-16,28), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ │ ├── ordering: +7,+1 │ │ │ └── select - │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ │ │ ├── has-placeholder │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(2-16,26), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ │ ├── fd: (1)-->(2-16,28), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ │ ├── group-by - │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ │ │ │ ├── grouping columns: instance_types.id:1!null │ │ │ │ ├── internal-ordering: +1 │ │ │ │ ├── has-placeholder │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(2-16,26), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ │ │ ├── fd: (1)-->(2-16,28), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ │ │ ├── left-join (merge) - │ │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_projects.instance_type_id:18 true:25 + │ │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_projects.instance_type_id:19 true:27 │ │ │ │ │ ├── left ordering: +1 - │ │ │ │ │ ├── right ordering: +18 + │ │ │ │ │ ├── right ordering: +19 │ │ │ │ │ ├── has-placeholder │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ │ │ │ ├── ordering: +1 @@ -1882,28 +1882,28 @@ sort │ │ │ │ │ │ └── filters │ │ │ │ │ │ └── instance_types.deleted:13 = $1 [outer=(13), constraints=(/13: (/NULL - ])] │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: true:25!null instance_type_projects.instance_type_id:18!null + │ │ │ │ │ │ ├── columns: true:27!null instance_type_projects.instance_type_id:19!null │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ ├── fd: ()-->(25) - │ │ │ │ │ │ ├── ordering: +18 opt(25) [actual: +18] + │ │ │ │ │ │ ├── fd: ()-->(27) + │ │ │ │ │ │ ├── ordering: +19 opt(27) [actual: +19] │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18!null project_id:19!null instance_type_projects.deleted:20!null + │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:19!null project_id:20!null instance_type_projects.deleted:21!null │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ ├── key: (18-20) - │ │ │ │ │ │ │ ├── ordering: +18 + │ │ │ │ │ │ │ ├── key: (19-21) + │ │ │ │ │ │ │ ├── ordering: +19 │ │ │ │ │ │ │ ├── scan instance_type_projects@secondary - │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18!null project_id:19 instance_type_projects.deleted:20 - │ │ │ │ │ │ │ │ ├── lax-key: (18-20) - │ │ │ │ │ │ │ │ └── ordering: +18 + │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:19!null project_id:20 instance_type_projects.deleted:21 + │ │ │ │ │ │ │ │ ├── lax-key: (19-21) + │ │ │ │ │ │ │ │ └── ordering: +19 │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ ├── instance_type_projects.deleted:20 = $2 [outer=(20), constraints=(/20: (/NULL - ])] - │ │ │ │ │ │ │ └── project_id:19 = $3 [outer=(19), constraints=(/19: (/NULL - ])] + │ │ │ │ │ │ │ ├── instance_type_projects.deleted:21 = $2 [outer=(21), constraints=(/21: (/NULL - ])] + │ │ │ │ │ │ │ └── project_id:20 = $3 [outer=(20), constraints=(/20: (/NULL - ])] │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── true [as=true:25] + │ │ │ │ │ │ └── true [as=true:27] │ │ │ │ │ └── filters (true) │ │ │ │ └── aggregations - │ │ │ │ ├── const-not-null-agg [as=true_agg:26, outer=(25)] - │ │ │ │ │ └── true:25 + │ │ │ │ ├── const-not-null-agg [as=true_agg:28, outer=(27)] + │ │ │ │ │ └── true:27 │ │ │ │ ├── const-agg [as=name:2, outer=(2)] │ │ │ │ │ └── name:2 │ │ │ │ ├── const-agg [as=memory_mb:3, outer=(3)] @@ -1935,11 +1935,11 @@ sort │ │ │ │ └── const-agg [as=instance_types.updated_at:16, outer=(16)] │ │ │ │ └── instance_types.updated_at:16 │ │ │ └── filters - │ │ │ └── (is_public:12 = true) OR (true_agg:26 IS NOT NULL) [outer=(12,26)] + │ │ │ └── (is_public:12 = true) OR (true_agg:28 IS NOT NULL) [outer=(12,28)] │ │ └── $4 │ └── $5 └── filters - └── instance_type_extra_specs_1.instance_type_id:31 = instance_types.id:1 [outer=(1,31), constraints=(/1: (/NULL - ]; /31: (/NULL - ]), fd=(1)==(31), (31)==(1)] + └── instance_type_extra_specs_1.instance_type_id:33 = instance_types.id:1 [outer=(1,33), constraints=(/1: (/NULL - ]; /33: (/NULL - ]), fd=(1)==(33), (33)==(1)] opt select anon_1.instance_types_created_at as anon_1_instance_types_created_at, @@ -1999,52 +1999,52 @@ from (select instance_types.created_at as instance_types_created_at, and instance_type_extra_specs_1.deleted = $7 ---- project - ├── columns: anon_1_instance_types_created_at:15 anon_1_instance_types_updated_at:16 anon_1_instance_types_deleted_at:14 anon_1_instance_types_deleted:13!null anon_1_instance_types_id:1!null anon_1_instance_types_name:2 anon_1_instance_types_memory_mb:3!null anon_1_instance_types_vcpus:4!null anon_1_instance_types_root_gb:5 anon_1_instance_types_ephemeral_gb:6 anon_1_instance_types_flavorid:7!null anon_1_instance_types_swap:8!null anon_1_instance_types_rxtx_factor:9 anon_1_instance_types_vcpu_weight:10 anon_1_instance_types_disabled:11 anon_1_instance_types_is_public:12 instance_type_extra_specs_1_created_at:34 instance_type_extra_specs_1_updated_at:35 instance_type_extra_specs_1_deleted_at:33 instance_type_extra_specs_1_deleted:32 instance_type_extra_specs_1_id:28 instance_type_extra_specs_1_key:29 instance_type_extra_specs_1_value:30 instance_type_extra_specs_1_instance_type_id:31 + ├── columns: anon_1_instance_types_created_at:15 anon_1_instance_types_updated_at:16 anon_1_instance_types_deleted_at:14 anon_1_instance_types_deleted:13!null anon_1_instance_types_id:1!null anon_1_instance_types_name:2 anon_1_instance_types_memory_mb:3!null anon_1_instance_types_vcpus:4!null anon_1_instance_types_root_gb:5 anon_1_instance_types_ephemeral_gb:6 anon_1_instance_types_flavorid:7!null anon_1_instance_types_swap:8!null anon_1_instance_types_rxtx_factor:9 anon_1_instance_types_vcpu_weight:10 anon_1_instance_types_disabled:11 anon_1_instance_types_is_public:12 instance_type_extra_specs_1_created_at:36 instance_type_extra_specs_1_updated_at:37 instance_type_extra_specs_1_deleted_at:35 instance_type_extra_specs_1_deleted:34 instance_type_extra_specs_1_id:30 instance_type_extra_specs_1_key:31 instance_type_extra_specs_1_value:32 instance_type_extra_specs_1_instance_type_id:33 ├── immutable, has-placeholder - ├── key: (1,28) - ├── fd: (1)-->(2-16), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + ├── key: (1,30) + ├── fd: (1)-->(2-16), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (30)-->(31-37), (31,33,34)~~>(30,32,35-37) └── right-join (hash) - ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 instance_type_extra_specs_1.id:28 key:29 value:30 instance_type_extra_specs_1.instance_type_id:31 instance_type_extra_specs_1.deleted:32 instance_type_extra_specs_1.deleted_at:33 instance_type_extra_specs_1.created_at:34 instance_type_extra_specs_1.updated_at:35 + ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 instance_type_extra_specs_1.id:30 key:31 value:32 instance_type_extra_specs_1.instance_type_id:33 instance_type_extra_specs_1.deleted:34 instance_type_extra_specs_1.deleted_at:35 instance_type_extra_specs_1.created_at:36 instance_type_extra_specs_1.updated_at:37 ├── immutable, has-placeholder - ├── key: (1,28) - ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + ├── key: (1,30) + ├── fd: (1)-->(2-16,28), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (30)-->(31-37), (31,33,34)~~>(30,32,35-37) ├── select - │ ├── columns: instance_type_extra_specs_1.id:28!null key:29 value:30 instance_type_extra_specs_1.instance_type_id:31!null instance_type_extra_specs_1.deleted:32!null instance_type_extra_specs_1.deleted_at:33 instance_type_extra_specs_1.created_at:34 instance_type_extra_specs_1.updated_at:35 + │ ├── columns: instance_type_extra_specs_1.id:30!null key:31 value:32 instance_type_extra_specs_1.instance_type_id:33!null instance_type_extra_specs_1.deleted:34!null instance_type_extra_specs_1.deleted_at:35 instance_type_extra_specs_1.created_at:36 instance_type_extra_specs_1.updated_at:37 │ ├── has-placeholder - │ ├── key: (28) - │ ├── fd: (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + │ ├── key: (30) + │ ├── fd: (30)-->(31-37), (31,33,34)~~>(30,32,35-37) │ ├── scan instance_type_extra_specs_1 - │ │ ├── columns: instance_type_extra_specs_1.id:28!null key:29 value:30 instance_type_extra_specs_1.instance_type_id:31!null instance_type_extra_specs_1.deleted:32 instance_type_extra_specs_1.deleted_at:33 instance_type_extra_specs_1.created_at:34 instance_type_extra_specs_1.updated_at:35 - │ │ ├── key: (28) - │ │ └── fd: (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + │ │ ├── columns: instance_type_extra_specs_1.id:30!null key:31 value:32 instance_type_extra_specs_1.instance_type_id:33!null instance_type_extra_specs_1.deleted:34 instance_type_extra_specs_1.deleted_at:35 instance_type_extra_specs_1.created_at:36 instance_type_extra_specs_1.updated_at:37 + │ │ ├── key: (30) + │ │ └── fd: (30)-->(31-37), (31,33,34)~~>(30,32,35-37) │ └── filters - │ └── instance_type_extra_specs_1.deleted:32 = $7 [outer=(32), constraints=(/32: (/NULL - ])] + │ └── instance_type_extra_specs_1.deleted:34 = $7 [outer=(34), constraints=(/34: (/NULL - ])] ├── limit - │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ ├── immutable, has-placeholder │ ├── key: (1) - │ ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ ├── fd: (1)-->(2-16,28), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ ├── offset - │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ │ ├── has-placeholder │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ ├── fd: (1)-->(2-16,28), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ ├── select - │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ │ │ ├── has-placeholder │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ │ ├── fd: (1)-->(2-16,28), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ │ ├── group-by - │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ │ │ │ ├── grouping columns: instance_types.id:1!null │ │ │ │ ├── internal-ordering: +1 │ │ │ │ ├── has-placeholder │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ │ │ ├── fd: (1)-->(2-16,28), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ │ │ ├── left-join (merge) - │ │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_projects.instance_type_id:18 true:25 + │ │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_projects.instance_type_id:19 true:27 │ │ │ │ │ ├── left ordering: +1 - │ │ │ │ │ ├── right ordering: +18 + │ │ │ │ │ ├── right ordering: +19 │ │ │ │ │ ├── has-placeholder │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ │ │ │ ├── ordering: +1 @@ -2063,28 +2063,28 @@ project │ │ │ │ │ │ ├── instance_types.deleted:13 = $1 [outer=(13), constraints=(/13: (/NULL - ])] │ │ │ │ │ │ └── flavorid:7 = $4 [outer=(7), constraints=(/7: (/NULL - ])] │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: true:25!null instance_type_projects.instance_type_id:18!null + │ │ │ │ │ │ ├── columns: true:27!null instance_type_projects.instance_type_id:19!null │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ ├── fd: ()-->(25) - │ │ │ │ │ │ ├── ordering: +18 opt(25) [actual: +18] + │ │ │ │ │ │ ├── fd: ()-->(27) + │ │ │ │ │ │ ├── ordering: +19 opt(27) [actual: +19] │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18!null project_id:19!null instance_type_projects.deleted:20!null + │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:19!null project_id:20!null instance_type_projects.deleted:21!null │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ ├── key: (18-20) - │ │ │ │ │ │ │ ├── ordering: +18 + │ │ │ │ │ │ │ ├── key: (19-21) + │ │ │ │ │ │ │ ├── ordering: +19 │ │ │ │ │ │ │ ├── scan instance_type_projects@secondary - │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18!null project_id:19 instance_type_projects.deleted:20 - │ │ │ │ │ │ │ │ ├── lax-key: (18-20) - │ │ │ │ │ │ │ │ └── ordering: +18 + │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:19!null project_id:20 instance_type_projects.deleted:21 + │ │ │ │ │ │ │ │ ├── lax-key: (19-21) + │ │ │ │ │ │ │ │ └── ordering: +19 │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ ├── instance_type_projects.deleted:20 = $2 [outer=(20), constraints=(/20: (/NULL - ])] - │ │ │ │ │ │ │ └── project_id:19 = $3 [outer=(19), constraints=(/19: (/NULL - ])] + │ │ │ │ │ │ │ ├── instance_type_projects.deleted:21 = $2 [outer=(21), constraints=(/21: (/NULL - ])] + │ │ │ │ │ │ │ └── project_id:20 = $3 [outer=(20), constraints=(/20: (/NULL - ])] │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── true [as=true:25] + │ │ │ │ │ │ └── true [as=true:27] │ │ │ │ │ └── filters (true) │ │ │ │ └── aggregations - │ │ │ │ ├── const-not-null-agg [as=true_agg:26, outer=(25)] - │ │ │ │ │ └── true:25 + │ │ │ │ ├── const-not-null-agg [as=true_agg:28, outer=(27)] + │ │ │ │ │ └── true:27 │ │ │ │ ├── const-agg [as=name:2, outer=(2)] │ │ │ │ │ └── name:2 │ │ │ │ ├── const-agg [as=memory_mb:3, outer=(3)] @@ -2116,11 +2116,11 @@ project │ │ │ │ └── const-agg [as=instance_types.updated_at:16, outer=(16)] │ │ │ │ └── instance_types.updated_at:16 │ │ │ └── filters - │ │ │ └── (is_public:12 = true) OR (true_agg:26 IS NOT NULL) [outer=(12,26)] + │ │ │ └── (is_public:12 = true) OR (true_agg:28 IS NOT NULL) [outer=(12,28)] │ │ └── $5 │ └── $6 └── filters - └── instance_type_extra_specs_1.instance_type_id:31 = instance_types.id:1 [outer=(1,31), constraints=(/1: (/NULL - ]; /31: (/NULL - ]), fd=(1)==(31), (31)==(1)] + └── instance_type_extra_specs_1.instance_type_id:33 = instance_types.id:1 [outer=(1,33), constraints=(/1: (/NULL - ]; /33: (/NULL - ]), fd=(1)==(33), (33)==(1)] opt select flavors.created_at as flavors_created_at, @@ -2154,41 +2154,41 @@ where flavors.is_public = true order by flavors.flavorid asc, flavors.id asc ---- sort - ├── columns: flavors_created_at:14 flavors_updated_at:15 flavors_id:1!null flavors_name:2!null flavors_memory_mb:3!null flavors_vcpus:4!null flavors_root_gb:5 flavors_ephemeral_gb:6 flavors_flavorid:7!null flavors_swap:8!null flavors_rxtx_factor:9 flavors_vcpu_weight:10 flavors_disabled:11 flavors_is_public:12 flavor_extra_specs_1_created_at:20 flavor_extra_specs_1_updated_at:21 flavor_extra_specs_1_id:16 flavor_extra_specs_1_key:17 flavor_extra_specs_1_value:18 flavor_extra_specs_1_flavor_id:19 + ├── columns: flavors_created_at:14 flavors_updated_at:15 flavors_id:1!null flavors_name:2!null flavors_memory_mb:3!null flavors_vcpus:4!null flavors_root_gb:5 flavors_ephemeral_gb:6 flavors_flavorid:7!null flavors_swap:8!null flavors_rxtx_factor:9 flavors_vcpu_weight:10 flavors_disabled:11 flavors_is_public:12 flavor_extra_specs_1_created_at:21 flavor_extra_specs_1_updated_at:22 flavor_extra_specs_1_id:17 flavor_extra_specs_1_key:18 flavor_extra_specs_1_value:19 flavor_extra_specs_1_flavor_id:20 ├── has-placeholder - ├── key: (1,16) - ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (16)-->(17-21), (17,19)-->(16,18,20,21) + ├── key: (1,17) + ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (17)-->(18-22), (18,20)-->(17,19,21,22) ├── ordering: +7 └── project - ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 flavor_extra_specs_1.id:16 key:17 value:18 flavor_extra_specs_1.flavor_id:19 flavor_extra_specs_1.created_at:20 flavor_extra_specs_1.updated_at:21 + ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 flavor_extra_specs_1.id:17 key:18 value:19 flavor_extra_specs_1.flavor_id:20 flavor_extra_specs_1.created_at:21 flavor_extra_specs_1.updated_at:22 ├── has-placeholder - ├── key: (1,16) - ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (16)-->(17-21), (17,19)-->(16,18,20,21) + ├── key: (1,17) + ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (17)-->(18-22), (18,20)-->(17,19,21,22) └── right-join (hash) - ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 flavor_extra_specs_1.id:16 key:17 value:18 flavor_extra_specs_1.flavor_id:19 flavor_extra_specs_1.created_at:20 flavor_extra_specs_1.updated_at:21 true_agg:29 + ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 flavor_extra_specs_1.id:17 key:18 value:19 flavor_extra_specs_1.flavor_id:20 flavor_extra_specs_1.created_at:21 flavor_extra_specs_1.updated_at:22 true_agg:32 ├── has-placeholder - ├── key: (1,16) - ├── fd: (1)-->(2-12,14,15,29), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (16)-->(17-21), (17,19)-->(16,18,20,21) + ├── key: (1,17) + ├── fd: (1)-->(2-12,14,15,32), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (17)-->(18-22), (18,20)-->(17,19,21,22) ├── scan flavor_extra_specs_1 - │ ├── columns: flavor_extra_specs_1.id:16!null key:17!null value:18 flavor_extra_specs_1.flavor_id:19!null flavor_extra_specs_1.created_at:20 flavor_extra_specs_1.updated_at:21 - │ ├── key: (16) - │ └── fd: (16)-->(17-21), (17,19)-->(16,18,20,21) + │ ├── columns: flavor_extra_specs_1.id:17!null key:18!null value:19 flavor_extra_specs_1.flavor_id:20!null flavor_extra_specs_1.created_at:21 flavor_extra_specs_1.updated_at:22 + │ ├── key: (17) + │ └── fd: (17)-->(18-22), (18,20)-->(17,19,21,22) ├── select - │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:29 + │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:32 │ ├── has-placeholder │ ├── key: (1) - │ ├── fd: (1)-->(2-12,14,15,29), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) + │ ├── fd: (1)-->(2-12,14,15,32), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ ├── group-by - │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:29 + │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:32 │ │ ├── grouping columns: flavors.id:1!null │ │ ├── internal-ordering: +1 │ │ ├── has-placeholder │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2-12,14,15,29), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) + │ │ ├── fd: (1)-->(2-12,14,15,32), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ ├── left-join (merge) - │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 flavor_projects.flavor_id:23 true:28 + │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 flavor_projects.flavor_id:25 true:31 │ │ │ ├── left ordering: +1 - │ │ │ ├── right ordering: +23 + │ │ │ ├── right ordering: +25 │ │ │ ├── has-placeholder │ │ │ ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ │ ├── ordering: +1 @@ -2198,27 +2198,27 @@ sort │ │ │ │ ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ │ │ └── ordering: +1 │ │ │ ├── project - │ │ │ │ ├── columns: true:28!null flavor_projects.flavor_id:23!null + │ │ │ │ ├── columns: true:31!null flavor_projects.flavor_id:25!null │ │ │ │ ├── has-placeholder - │ │ │ │ ├── fd: ()-->(28) - │ │ │ │ ├── ordering: +23 opt(28) [actual: +23] + │ │ │ │ ├── fd: ()-->(31) + │ │ │ │ ├── ordering: +25 opt(31) [actual: +25] │ │ │ │ ├── select - │ │ │ │ │ ├── columns: flavor_projects.flavor_id:23!null project_id:24!null + │ │ │ │ │ ├── columns: flavor_projects.flavor_id:25!null project_id:26!null │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ ├── key: (23,24) - │ │ │ │ │ ├── ordering: +23 + │ │ │ │ │ ├── key: (25,26) + │ │ │ │ │ ├── ordering: +25 │ │ │ │ │ ├── scan flavor_projects@secondary - │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:23!null project_id:24!null - │ │ │ │ │ │ ├── key: (23,24) - │ │ │ │ │ │ └── ordering: +23 + │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:25!null project_id:26!null + │ │ │ │ │ │ ├── key: (25,26) + │ │ │ │ │ │ └── ordering: +25 │ │ │ │ │ └── filters - │ │ │ │ │ └── project_id:24 = $1 [outer=(24), constraints=(/24: (/NULL - ])] + │ │ │ │ │ └── project_id:26 = $1 [outer=(26), constraints=(/26: (/NULL - ])] │ │ │ │ └── projections - │ │ │ │ └── true [as=true:28] + │ │ │ │ └── true [as=true:31] │ │ │ └── filters (true) │ │ └── aggregations - │ │ ├── const-not-null-agg [as=true_agg:29, outer=(28)] - │ │ │ └── true:28 + │ │ ├── const-not-null-agg [as=true_agg:32, outer=(31)] + │ │ │ └── true:31 │ │ ├── const-agg [as=name:2, outer=(2)] │ │ │ └── name:2 │ │ ├── const-agg [as=memory_mb:3, outer=(3)] @@ -2246,9 +2246,9 @@ sort │ │ └── const-agg [as=flavors.updated_at:15, outer=(15)] │ │ └── flavors.updated_at:15 │ └── filters - │ └── (is_public:12 = true) OR (true_agg:29 IS NOT NULL) [outer=(12,29)] + │ └── (is_public:12 = true) OR (true_agg:32 IS NOT NULL) [outer=(12,32)] └── filters - └── flavor_extra_specs_1.flavor_id:19 = flavors.id:1 [outer=(1,19), constraints=(/1: (/NULL - ]; /19: (/NULL - ]), fd=(1)==(19), (19)==(1)] + └── flavor_extra_specs_1.flavor_id:20 = flavors.id:1 [outer=(1,20), constraints=(/1: (/NULL - ]; /20: (/NULL - ]), fd=(1)==(20), (20)==(1)] opt select anon_1.instance_types_created_at as anon_1_instance_types_created_at, @@ -2313,67 +2313,67 @@ order by anon_1.instance_types_flavorid asc, anon_1.instance_types_id asc ---- sort - ├── columns: anon_1_instance_types_created_at:15 anon_1_instance_types_updated_at:16 anon_1_instance_types_deleted_at:14 anon_1_instance_types_deleted:13!null anon_1_instance_types_id:1!null anon_1_instance_types_name:2 anon_1_instance_types_memory_mb:3!null anon_1_instance_types_vcpus:4!null anon_1_instance_types_root_gb:5 anon_1_instance_types_ephemeral_gb:6 anon_1_instance_types_flavorid:7!null anon_1_instance_types_swap:8!null anon_1_instance_types_rxtx_factor:9 anon_1_instance_types_vcpu_weight:10 anon_1_instance_types_disabled:11 anon_1_instance_types_is_public:12 instance_type_extra_specs_1_created_at:34 instance_type_extra_specs_1_updated_at:35 instance_type_extra_specs_1_deleted_at:33 instance_type_extra_specs_1_deleted:32 instance_type_extra_specs_1_id:28 instance_type_extra_specs_1_key:29 instance_type_extra_specs_1_value:30 instance_type_extra_specs_1_instance_type_id:31 + ├── columns: anon_1_instance_types_created_at:15 anon_1_instance_types_updated_at:16 anon_1_instance_types_deleted_at:14 anon_1_instance_types_deleted:13!null anon_1_instance_types_id:1!null anon_1_instance_types_name:2 anon_1_instance_types_memory_mb:3!null anon_1_instance_types_vcpus:4!null anon_1_instance_types_root_gb:5 anon_1_instance_types_ephemeral_gb:6 anon_1_instance_types_flavorid:7!null anon_1_instance_types_swap:8!null anon_1_instance_types_rxtx_factor:9 anon_1_instance_types_vcpu_weight:10 anon_1_instance_types_disabled:11 anon_1_instance_types_is_public:12 instance_type_extra_specs_1_created_at:36 instance_type_extra_specs_1_updated_at:37 instance_type_extra_specs_1_deleted_at:35 instance_type_extra_specs_1_deleted:34 instance_type_extra_specs_1_id:30 instance_type_extra_specs_1_key:31 instance_type_extra_specs_1_value:32 instance_type_extra_specs_1_instance_type_id:33 ├── immutable, has-placeholder - ├── key: (1,28) - ├── fd: (1)-->(2-16), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + ├── key: (1,30) + ├── fd: (1)-->(2-16), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (30)-->(31-37), (31,33,34)~~>(30,32,35-37) ├── ordering: +7,+1 └── project - ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_extra_specs_1.id:28 key:29 value:30 instance_type_extra_specs_1.instance_type_id:31 instance_type_extra_specs_1.deleted:32 instance_type_extra_specs_1.deleted_at:33 instance_type_extra_specs_1.created_at:34 instance_type_extra_specs_1.updated_at:35 + ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_extra_specs_1.id:30 key:31 value:32 instance_type_extra_specs_1.instance_type_id:33 instance_type_extra_specs_1.deleted:34 instance_type_extra_specs_1.deleted_at:35 instance_type_extra_specs_1.created_at:36 instance_type_extra_specs_1.updated_at:37 ├── immutable, has-placeholder - ├── key: (1,28) - ├── fd: (1)-->(2-16), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + ├── key: (1,30) + ├── fd: (1)-->(2-16), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (30)-->(31-37), (31,33,34)~~>(30,32,35-37) └── right-join (hash) - ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 instance_type_extra_specs_1.id:28 key:29 value:30 instance_type_extra_specs_1.instance_type_id:31 instance_type_extra_specs_1.deleted:32 instance_type_extra_specs_1.deleted_at:33 instance_type_extra_specs_1.created_at:34 instance_type_extra_specs_1.updated_at:35 + ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 instance_type_extra_specs_1.id:30 key:31 value:32 instance_type_extra_specs_1.instance_type_id:33 instance_type_extra_specs_1.deleted:34 instance_type_extra_specs_1.deleted_at:35 instance_type_extra_specs_1.created_at:36 instance_type_extra_specs_1.updated_at:37 ├── immutable, has-placeholder - ├── key: (1,28) - ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + ├── key: (1,30) + ├── fd: (1)-->(2-16,28), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (30)-->(31-37), (31,33,34)~~>(30,32,35-37) ├── select - │ ├── columns: instance_type_extra_specs_1.id:28!null key:29 value:30 instance_type_extra_specs_1.instance_type_id:31!null instance_type_extra_specs_1.deleted:32!null instance_type_extra_specs_1.deleted_at:33 instance_type_extra_specs_1.created_at:34 instance_type_extra_specs_1.updated_at:35 + │ ├── columns: instance_type_extra_specs_1.id:30!null key:31 value:32 instance_type_extra_specs_1.instance_type_id:33!null instance_type_extra_specs_1.deleted:34!null instance_type_extra_specs_1.deleted_at:35 instance_type_extra_specs_1.created_at:36 instance_type_extra_specs_1.updated_at:37 │ ├── has-placeholder - │ ├── key: (28) - │ ├── fd: (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + │ ├── key: (30) + │ ├── fd: (30)-->(31-37), (31,33,34)~~>(30,32,35-37) │ ├── scan instance_type_extra_specs_1 - │ │ ├── columns: instance_type_extra_specs_1.id:28!null key:29 value:30 instance_type_extra_specs_1.instance_type_id:31!null instance_type_extra_specs_1.deleted:32 instance_type_extra_specs_1.deleted_at:33 instance_type_extra_specs_1.created_at:34 instance_type_extra_specs_1.updated_at:35 - │ │ ├── key: (28) - │ │ └── fd: (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + │ │ ├── columns: instance_type_extra_specs_1.id:30!null key:31 value:32 instance_type_extra_specs_1.instance_type_id:33!null instance_type_extra_specs_1.deleted:34 instance_type_extra_specs_1.deleted_at:35 instance_type_extra_specs_1.created_at:36 instance_type_extra_specs_1.updated_at:37 + │ │ ├── key: (30) + │ │ └── fd: (30)-->(31-37), (31,33,34)~~>(30,32,35-37) │ └── filters - │ └── instance_type_extra_specs_1.deleted:32 = $9 [outer=(32), constraints=(/32: (/NULL - ])] + │ └── instance_type_extra_specs_1.deleted:34 = $9 [outer=(34), constraints=(/34: (/NULL - ])] ├── limit - │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ ├── internal-ordering: +7,+1 │ ├── immutable, has-placeholder │ ├── key: (1) - │ ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ ├── fd: (1)-->(2-16,28), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ ├── offset - │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ │ ├── internal-ordering: +7,+1 │ │ ├── has-placeholder │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ ├── fd: (1)-->(2-16,28), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ ├── ordering: +7,+1 │ │ ├── sort - │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ │ │ ├── has-placeholder │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ │ ├── fd: (1)-->(2-16,28), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ │ ├── ordering: +7,+1 │ │ │ └── select - │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ │ │ ├── has-placeholder │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ │ ├── fd: (1)-->(2-16,28), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ │ ├── group-by - │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ │ │ │ ├── grouping columns: instance_types.id:1!null │ │ │ │ ├── internal-ordering: +1 │ │ │ │ ├── has-placeholder │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ │ │ ├── fd: (1)-->(2-16,28), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ │ │ ├── left-join (merge) - │ │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_projects.instance_type_id:18 true:25 + │ │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_projects.instance_type_id:19 true:27 │ │ │ │ │ ├── left ordering: +1 - │ │ │ │ │ ├── right ordering: +18 + │ │ │ │ │ ├── right ordering: +19 │ │ │ │ │ ├── has-placeholder │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ │ │ │ ├── ordering: +1 @@ -2392,28 +2392,28 @@ sort │ │ │ │ │ │ ├── instance_types.deleted:13 = $1 [outer=(13), constraints=(/13: (/NULL - ])] │ │ │ │ │ │ └── (flavorid:7 > $4) OR ((flavorid:7 = $5) AND (instance_types.id:1 > $6)) [outer=(1,7), constraints=(/7: (/NULL - ])] │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: true:25!null instance_type_projects.instance_type_id:18!null + │ │ │ │ │ │ ├── columns: true:27!null instance_type_projects.instance_type_id:19!null │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ ├── fd: ()-->(25) - │ │ │ │ │ │ ├── ordering: +18 opt(25) [actual: +18] + │ │ │ │ │ │ ├── fd: ()-->(27) + │ │ │ │ │ │ ├── ordering: +19 opt(27) [actual: +19] │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18!null project_id:19!null instance_type_projects.deleted:20!null + │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:19!null project_id:20!null instance_type_projects.deleted:21!null │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ ├── key: (18-20) - │ │ │ │ │ │ │ ├── ordering: +18 + │ │ │ │ │ │ │ ├── key: (19-21) + │ │ │ │ │ │ │ ├── ordering: +19 │ │ │ │ │ │ │ ├── scan instance_type_projects@secondary - │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18!null project_id:19 instance_type_projects.deleted:20 - │ │ │ │ │ │ │ │ ├── lax-key: (18-20) - │ │ │ │ │ │ │ │ └── ordering: +18 + │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:19!null project_id:20 instance_type_projects.deleted:21 + │ │ │ │ │ │ │ │ ├── lax-key: (19-21) + │ │ │ │ │ │ │ │ └── ordering: +19 │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ ├── instance_type_projects.deleted:20 = $2 [outer=(20), constraints=(/20: (/NULL - ])] - │ │ │ │ │ │ │ └── project_id:19 = $3 [outer=(19), constraints=(/19: (/NULL - ])] + │ │ │ │ │ │ │ ├── instance_type_projects.deleted:21 = $2 [outer=(21), constraints=(/21: (/NULL - ])] + │ │ │ │ │ │ │ └── project_id:20 = $3 [outer=(20), constraints=(/20: (/NULL - ])] │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── true [as=true:25] + │ │ │ │ │ │ └── true [as=true:27] │ │ │ │ │ └── filters (true) │ │ │ │ └── aggregations - │ │ │ │ ├── const-not-null-agg [as=true_agg:26, outer=(25)] - │ │ │ │ │ └── true:25 + │ │ │ │ ├── const-not-null-agg [as=true_agg:28, outer=(27)] + │ │ │ │ │ └── true:27 │ │ │ │ ├── const-agg [as=name:2, outer=(2)] │ │ │ │ │ └── name:2 │ │ │ │ ├── const-agg [as=memory_mb:3, outer=(3)] @@ -2445,11 +2445,11 @@ sort │ │ │ │ └── const-agg [as=instance_types.updated_at:16, outer=(16)] │ │ │ │ └── instance_types.updated_at:16 │ │ │ └── filters - │ │ │ └── (is_public:12 = true) OR (true_agg:26 IS NOT NULL) [outer=(12,26)] + │ │ │ └── (is_public:12 = true) OR (true_agg:28 IS NOT NULL) [outer=(12,28)] │ │ └── $7 │ └── $8 └── filters - └── instance_type_extra_specs_1.instance_type_id:31 = instance_types.id:1 [outer=(1,31), constraints=(/1: (/NULL - ]; /31: (/NULL - ]), fd=(1)==(31), (31)==(1)] + └── instance_type_extra_specs_1.instance_type_id:33 = instance_types.id:1 [outer=(1,33), constraints=(/1: (/NULL - ]; /33: (/NULL - ]), fd=(1)==(33), (33)==(1)] opt select anon_1.flavors_created_at as anon_1_flavors_created_at, @@ -2501,60 +2501,60 @@ from (select flavors.created_at as flavors_created_at, order by anon_1.flavors_flavorid asc, anon_1.flavors_id asc ---- sort - ├── columns: anon_1_flavors_created_at:14 anon_1_flavors_updated_at:15 anon_1_flavors_id:1!null anon_1_flavors_name:2!null anon_1_flavors_memory_mb:3!null anon_1_flavors_vcpus:4!null anon_1_flavors_root_gb:5 anon_1_flavors_ephemeral_gb:6 anon_1_flavors_flavorid:7!null anon_1_flavors_swap:8!null anon_1_flavors_rxtx_factor:9 anon_1_flavors_vcpu_weight:10 anon_1_flavors_disabled:11 anon_1_flavors_is_public:12 flavor_extra_specs_1_created_at:29 flavor_extra_specs_1_updated_at:30 flavor_extra_specs_1_id:25 flavor_extra_specs_1_key:26 flavor_extra_specs_1_value:27 flavor_extra_specs_1_flavor_id:28 + ├── columns: anon_1_flavors_created_at:14 anon_1_flavors_updated_at:15 anon_1_flavors_id:1!null anon_1_flavors_name:2!null anon_1_flavors_memory_mb:3!null anon_1_flavors_vcpus:4!null anon_1_flavors_root_gb:5 anon_1_flavors_ephemeral_gb:6 anon_1_flavors_flavorid:7!null anon_1_flavors_swap:8!null anon_1_flavors_rxtx_factor:9 anon_1_flavors_vcpu_weight:10 anon_1_flavors_disabled:11 anon_1_flavors_is_public:12 flavor_extra_specs_1_created_at:31 flavor_extra_specs_1_updated_at:32 flavor_extra_specs_1_id:27 flavor_extra_specs_1_key:28 flavor_extra_specs_1_value:29 flavor_extra_specs_1_flavor_id:30 ├── immutable, has-placeholder - ├── key: (1,25) - ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (25)-->(26-30), (26,28)-->(25,27,29,30) + ├── key: (1,27) + ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (27)-->(28-32), (28,30)-->(27,29,31,32) ├── ordering: +7 └── project - ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 flavor_extra_specs_1.id:25 key:26 value:27 flavor_extra_specs_1.flavor_id:28 flavor_extra_specs_1.created_at:29 flavor_extra_specs_1.updated_at:30 + ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 flavor_extra_specs_1.id:27 key:28 value:29 flavor_extra_specs_1.flavor_id:30 flavor_extra_specs_1.created_at:31 flavor_extra_specs_1.updated_at:32 ├── immutable, has-placeholder - ├── key: (1,25) - ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (25)-->(26-30), (26,28)-->(25,27,29,30) + ├── key: (1,27) + ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (27)-->(28-32), (28,30)-->(27,29,31,32) └── right-join (hash) - ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 flavor_extra_specs_1.id:25 key:26 value:27 flavor_extra_specs_1.flavor_id:28 flavor_extra_specs_1.created_at:29 flavor_extra_specs_1.updated_at:30 + ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 flavor_extra_specs_1.id:27 key:28 value:29 flavor_extra_specs_1.flavor_id:30 flavor_extra_specs_1.created_at:31 flavor_extra_specs_1.updated_at:32 ├── immutable, has-placeholder - ├── key: (1,25) - ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (25)-->(26-30), (26,28)-->(25,27,29,30) + ├── key: (1,27) + ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (27)-->(28-32), (28,30)-->(27,29,31,32) ├── scan flavor_extra_specs_1 - │ ├── columns: flavor_extra_specs_1.id:25!null key:26!null value:27 flavor_extra_specs_1.flavor_id:28!null flavor_extra_specs_1.created_at:29 flavor_extra_specs_1.updated_at:30 - │ ├── key: (25) - │ └── fd: (25)-->(26-30), (26,28)-->(25,27,29,30) + │ ├── columns: flavor_extra_specs_1.id:27!null key:28!null value:29 flavor_extra_specs_1.flavor_id:30!null flavor_extra_specs_1.created_at:31 flavor_extra_specs_1.updated_at:32 + │ ├── key: (27) + │ └── fd: (27)-->(28-32), (28,30)-->(27,29,31,32) ├── limit - │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 + │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 │ ├── internal-ordering: +7 │ ├── immutable, has-placeholder │ ├── key: (1) - │ ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) + │ ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ ├── offset - │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 + │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 │ │ ├── internal-ordering: +7 │ │ ├── has-placeholder │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) + │ │ ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ ├── ordering: +7 │ │ ├── sort - │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 + │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 │ │ │ ├── has-placeholder │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) + │ │ │ ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ │ ├── ordering: +7 │ │ │ └── select - │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 + │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 │ │ │ ├── has-placeholder │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) + │ │ │ ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ │ ├── group-by - │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 + │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 │ │ │ │ ├── grouping columns: flavors.id:1!null │ │ │ │ ├── internal-ordering: +1 │ │ │ │ ├── has-placeholder │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) + │ │ │ │ ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ │ │ ├── left-join (merge) - │ │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 flavor_projects.flavor_id:17 true:22 + │ │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 flavor_projects.flavor_id:18 true:24 │ │ │ │ │ ├── left ordering: +1 - │ │ │ │ │ ├── right ordering: +17 + │ │ │ │ │ ├── right ordering: +18 │ │ │ │ │ ├── has-placeholder │ │ │ │ │ ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ │ │ │ ├── ordering: +1 @@ -2564,27 +2564,27 @@ sort │ │ │ │ │ │ ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ │ │ │ │ └── ordering: +1 │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: true:22!null flavor_projects.flavor_id:17!null + │ │ │ │ │ │ ├── columns: true:24!null flavor_projects.flavor_id:18!null │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ ├── fd: ()-->(22) - │ │ │ │ │ │ ├── ordering: +17 opt(22) [actual: +17] + │ │ │ │ │ │ ├── fd: ()-->(24) + │ │ │ │ │ │ ├── ordering: +18 opt(24) [actual: +18] │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:17!null project_id:18!null + │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:18!null project_id:19!null │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ ├── key: (17,18) - │ │ │ │ │ │ │ ├── ordering: +17 + │ │ │ │ │ │ │ ├── key: (18,19) + │ │ │ │ │ │ │ ├── ordering: +18 │ │ │ │ │ │ │ ├── scan flavor_projects@secondary - │ │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:17!null project_id:18!null - │ │ │ │ │ │ │ │ ├── key: (17,18) - │ │ │ │ │ │ │ │ └── ordering: +17 + │ │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:18!null project_id:19!null + │ │ │ │ │ │ │ │ ├── key: (18,19) + │ │ │ │ │ │ │ │ └── ordering: +18 │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── project_id:18 = $1 [outer=(18), constraints=(/18: (/NULL - ])] + │ │ │ │ │ │ │ └── project_id:19 = $1 [outer=(19), constraints=(/19: (/NULL - ])] │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── true [as=true:22] + │ │ │ │ │ │ └── true [as=true:24] │ │ │ │ │ └── filters (true) │ │ │ │ └── aggregations - │ │ │ │ ├── const-not-null-agg [as=true_agg:23, outer=(22)] - │ │ │ │ │ └── true:22 + │ │ │ │ ├── const-not-null-agg [as=true_agg:25, outer=(24)] + │ │ │ │ │ └── true:24 │ │ │ │ ├── const-agg [as=name:2, outer=(2)] │ │ │ │ │ └── name:2 │ │ │ │ ├── const-agg [as=memory_mb:3, outer=(3)] @@ -2612,11 +2612,11 @@ sort │ │ │ │ └── const-agg [as=flavors.updated_at:15, outer=(15)] │ │ │ │ └── flavors.updated_at:15 │ │ │ └── filters - │ │ │ └── (is_public:12 = true) OR (true_agg:23 IS NOT NULL) [outer=(12,23)] + │ │ │ └── (is_public:12 = true) OR (true_agg:25 IS NOT NULL) [outer=(12,25)] │ │ └── $2 │ └── $3 └── filters - └── flavor_extra_specs_1.flavor_id:28 = flavors.id:1 [outer=(1,28), constraints=(/1: (/NULL - ]; /28: (/NULL - ]), fd=(1)==(28), (28)==(1)] + └── flavor_extra_specs_1.flavor_id:30 = flavors.id:1 [outer=(1,30), constraints=(/1: (/NULL - ]; /30: (/NULL - ]), fd=(1)==(30), (30)==(1)] opt select anon_1.instance_types_created_at as anon_1_instance_types_created_at, @@ -2686,82 +2686,82 @@ order by anon_1.instance_types_flavorid asc, anon_1.instance_types_id asc ---- project - ├── columns: anon_1_instance_types_created_at:15 anon_1_instance_types_updated_at:16 anon_1_instance_types_deleted_at:14 anon_1_instance_types_deleted:13!null anon_1_instance_types_id:1!null anon_1_instance_types_name:2 anon_1_instance_types_memory_mb:3!null anon_1_instance_types_vcpus:4!null anon_1_instance_types_root_gb:5 anon_1_instance_types_ephemeral_gb:6 anon_1_instance_types_flavorid:7 anon_1_instance_types_swap:8!null anon_1_instance_types_rxtx_factor:9 anon_1_instance_types_vcpu_weight:10 anon_1_instance_types_disabled:11!null anon_1_instance_types_is_public:12 instance_type_extra_specs_1_created_at:45 instance_type_extra_specs_1_updated_at:46 instance_type_extra_specs_1_deleted_at:44 instance_type_extra_specs_1_deleted:43 instance_type_extra_specs_1_id:39 instance_type_extra_specs_1_key:40 instance_type_extra_specs_1_value:41 instance_type_extra_specs_1_instance_type_id:42 + ├── columns: anon_1_instance_types_created_at:15 anon_1_instance_types_updated_at:16 anon_1_instance_types_deleted_at:14 anon_1_instance_types_deleted:13!null anon_1_instance_types_id:1!null anon_1_instance_types_name:2 anon_1_instance_types_memory_mb:3!null anon_1_instance_types_vcpus:4!null anon_1_instance_types_root_gb:5 anon_1_instance_types_ephemeral_gb:6 anon_1_instance_types_flavorid:7 anon_1_instance_types_swap:8!null anon_1_instance_types_rxtx_factor:9 anon_1_instance_types_vcpu_weight:10 anon_1_instance_types_disabled:11!null anon_1_instance_types_is_public:12 instance_type_extra_specs_1_created_at:48 instance_type_extra_specs_1_updated_at:49 instance_type_extra_specs_1_deleted_at:47 instance_type_extra_specs_1_deleted:46 instance_type_extra_specs_1_id:42 instance_type_extra_specs_1_key:43 instance_type_extra_specs_1_value:44 instance_type_extra_specs_1_instance_type_id:45 ├── immutable, has-placeholder - ├── key: (1,39) - ├── fd: ()-->(11), (1)-->(2-10,12-16), (7,13)~~>(1-6,8-10,12,14-16), (2,13)~~>(1,3-10,12,14-16), (39)-->(40-46), (40,42,43)~~>(39,41,44-46) + ├── key: (1,42) + ├── fd: ()-->(11), (1)-->(2-10,12-16), (7,13)~~>(1-6,8-10,12,14-16), (2,13)~~>(1,3-10,12,14-16), (42)-->(43-49), (43,45,46)~~>(42,44,47-49) ├── ordering: +7,+1 opt(11) [actual: +7,+1] └── left-join (lookup instance_type_extra_specs) - ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:37 instance_type_extra_specs_1.id:39 key:40 value:41 instance_type_extra_specs_1.instance_type_id:42 instance_type_extra_specs_1.deleted:43 instance_type_extra_specs_1.deleted_at:44 instance_type_extra_specs_1.created_at:45 instance_type_extra_specs_1.updated_at:46 - ├── key columns: [39] = [39] + ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:40 instance_type_extra_specs_1.id:42 key:43 value:44 instance_type_extra_specs_1.instance_type_id:45 instance_type_extra_specs_1.deleted:46 instance_type_extra_specs_1.deleted_at:47 instance_type_extra_specs_1.created_at:48 instance_type_extra_specs_1.updated_at:49 + ├── key columns: [42] = [42] ├── lookup columns are key ├── immutable, has-placeholder - ├── key: (1,39) - ├── fd: ()-->(11), (1)-->(2-10,12-16,37), (7,13)~~>(1-6,8-10,12,14-16), (2,13)~~>(1,3-10,12,14-16), (39)-->(40-46), (40,42,43)~~>(39,41,44-46) + ├── key: (1,42) + ├── fd: ()-->(11), (1)-->(2-10,12-16,40), (7,13)~~>(1-6,8-10,12,14-16), (2,13)~~>(1,3-10,12,14-16), (42)-->(43-49), (43,45,46)~~>(42,44,47-49) ├── ordering: +7,+1 opt(11) [actual: +7,+1] ├── left-join (lookup instance_type_extra_specs@secondary) - │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:37 instance_type_extra_specs_1.id:39 key:40 instance_type_extra_specs_1.instance_type_id:42 instance_type_extra_specs_1.deleted:43 - │ ├── key columns: [1] = [42] + │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:40 instance_type_extra_specs_1.id:42 key:43 instance_type_extra_specs_1.instance_type_id:45 instance_type_extra_specs_1.deleted:46 + │ ├── key columns: [1] = [45] │ ├── immutable, has-placeholder - │ ├── key: (1,39) - │ ├── fd: ()-->(11), (1)-->(2-10,12-16,37), (7,13)~~>(1-6,8-10,12,14-16), (2,13)~~>(1,3-10,12,14-16), (39)-->(40,42,43), (40,42,43)~~>(39) + │ ├── key: (1,42) + │ ├── fd: ()-->(11), (1)-->(2-10,12-16,40), (7,13)~~>(1-6,8-10,12,14-16), (2,13)~~>(1,3-10,12,14-16), (42)-->(43,45,46), (43,45,46)~~>(42) │ ├── ordering: +7,+1 opt(11) [actual: +7,+1] │ ├── limit - │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:37 + │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:40 │ │ ├── internal-ordering: +7,+1 opt(11) │ │ ├── immutable, has-placeholder │ │ ├── key: (1) - │ │ ├── fd: ()-->(11), (1)-->(2-10,12-16,37), (7,13)~~>(1-6,8-10,12,14-16), (2,13)~~>(1,3-10,12,14-16) + │ │ ├── fd: ()-->(11), (1)-->(2-10,12-16,40), (7,13)~~>(1-6,8-10,12,14-16), (2,13)~~>(1,3-10,12,14-16) │ │ ├── ordering: +7,+1 opt(11) [actual: +7,+1] │ │ ├── offset - │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:37 + │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:40 │ │ │ ├── internal-ordering: +7,+1 opt(11) │ │ │ ├── has-placeholder │ │ │ ├── key: (1) - │ │ │ ├── fd: ()-->(11), (1)-->(2-10,12-16,37), (7,13)~~>(1-6,8-10,12,14-16), (2,13)~~>(1,3-10,12,14-16) + │ │ │ ├── fd: ()-->(11), (1)-->(2-10,12-16,40), (7,13)~~>(1-6,8-10,12,14-16), (2,13)~~>(1,3-10,12,14-16) │ │ │ ├── ordering: +7,+1 opt(11) [actual: +7,+1] │ │ │ ├── sort - │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:37 + │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:40 │ │ │ │ ├── has-placeholder │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: ()-->(11), (1)-->(2-10,12-16,37), (7,13)~~>(1-6,8-10,12,14-16), (2,13)~~>(1,3-10,12,14-16) + │ │ │ │ ├── fd: ()-->(11), (1)-->(2-10,12-16,40), (7,13)~~>(1-6,8-10,12,14-16), (2,13)~~>(1,3-10,12,14-16) │ │ │ │ ├── ordering: +7,+1 opt(11) [actual: +7,+1] │ │ │ │ └── select - │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:37 + │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:40 │ │ │ │ ├── has-placeholder │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: ()-->(11), (1)-->(2-10,12-16,37), (7,13)~~>(1-6,8-10,12,14-16), (2,13)~~>(1,3-10,12,14-16) + │ │ │ │ ├── fd: ()-->(11), (1)-->(2-10,12-16,40), (7,13)~~>(1-6,8-10,12,14-16), (2,13)~~>(1,3-10,12,14-16) │ │ │ │ ├── group-by - │ │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:37 + │ │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:40 │ │ │ │ │ ├── grouping columns: instance_types.id:1!null │ │ │ │ │ ├── internal-ordering: +1 opt(11) │ │ │ │ │ ├── has-placeholder │ │ │ │ │ ├── key: (1) - │ │ │ │ │ ├── fd: ()-->(11), (1)-->(2-16,37), (7,13)~~>(1-6,8-10,12,14-16), (2,13)~~>(1,3-10,12,14-16) + │ │ │ │ │ ├── fd: ()-->(11), (1)-->(2-16,40), (7,13)~~>(1-6,8-10,12,14-16), (2,13)~~>(1,3-10,12,14-16) │ │ │ │ │ ├── left-join (merge) - │ │ │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_projects.instance_type_id:26 true_agg:34 true:36 + │ │ │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_projects.instance_type_id:28 true_agg:37 true:39 │ │ │ │ │ │ ├── left ordering: +1 - │ │ │ │ │ │ ├── right ordering: +26 + │ │ │ │ │ │ ├── right ordering: +28 │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ ├── fd: ()-->(11), (1)-->(2-10,12-16,34), (7,13)~~>(1-6,8-10,12,14-16), (2,13)~~>(1,3-10,12,14-16) + │ │ │ │ │ │ ├── fd: ()-->(11), (1)-->(2-10,12-16,37), (7,13)~~>(1-6,8-10,12,14-16), (2,13)~~>(1,3-10,12,14-16) │ │ │ │ │ │ ├── ordering: +1 opt(11) [actual: +1] │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:34 + │ │ │ │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:37 │ │ │ │ │ │ │ ├── has-placeholder │ │ │ │ │ │ │ ├── key: (1) - │ │ │ │ │ │ │ ├── fd: ()-->(11), (1)-->(2-10,12-16,34), (7,13)~~>(1-6,8-10,12,14-16), (2,13)~~>(1,3-10,12,14-16) + │ │ │ │ │ │ │ ├── fd: ()-->(11), (1)-->(2-10,12-16,37), (7,13)~~>(1-6,8-10,12,14-16), (2,13)~~>(1,3-10,12,14-16) │ │ │ │ │ │ │ ├── ordering: +1 opt(11) [actual: +1] │ │ │ │ │ │ │ ├── group-by - │ │ │ │ │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:34 + │ │ │ │ │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:37 │ │ │ │ │ │ │ │ ├── grouping columns: instance_types.id:1!null │ │ │ │ │ │ │ │ ├── has-placeholder │ │ │ │ │ │ │ │ ├── key: (1) - │ │ │ │ │ │ │ │ ├── fd: ()-->(11), (1)-->(2-16,34), (7,13)~~>(1-6,8-10,12,14-16), (2,13)~~>(1,3-10,12,14-16) + │ │ │ │ │ │ │ │ ├── fd: ()-->(11), (1)-->(2-16,37), (7,13)~~>(1-6,8-10,12,14-16), (2,13)~~>(1,3-10,12,14-16) │ │ │ │ │ │ │ │ ├── ordering: +1 opt(11) [actual: +1] │ │ │ │ │ │ │ │ ├── left-join (merge) - │ │ │ │ │ │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_projects.instance_type_id:18 true:33 + │ │ │ │ │ │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7 swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11!null is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_projects.instance_type_id:19 true:36 │ │ │ │ │ │ │ │ │ ├── left ordering: +1 - │ │ │ │ │ │ │ │ │ ├── right ordering: +18 + │ │ │ │ │ │ │ │ │ ├── right ordering: +19 │ │ │ │ │ │ │ │ │ ├── has-placeholder │ │ │ │ │ │ │ │ │ ├── fd: ()-->(11), (1)-->(2-10,12-16), (7,13)~~>(1-6,8-10,12,14-16), (2,13)~~>(1,3-10,12,14-16) │ │ │ │ │ │ │ │ │ ├── ordering: +1 opt(11) [actual: +1] @@ -2780,28 +2780,28 @@ project │ │ │ │ │ │ │ │ │ │ ├── instance_types.deleted:13 = $1 [outer=(13), constraints=(/13: (/NULL - ])] │ │ │ │ │ │ │ │ │ │ └── disabled:11 = false [outer=(11), constraints=(/11: [/false - /false]; tight), fd=()-->(11)] │ │ │ │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ │ │ │ ├── columns: true:33!null instance_type_projects.instance_type_id:18!null + │ │ │ │ │ │ │ │ │ │ ├── columns: true:36!null instance_type_projects.instance_type_id:19!null │ │ │ │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ │ │ │ ├── fd: ()-->(33) - │ │ │ │ │ │ │ │ │ │ ├── ordering: +18 opt(33) [actual: +18] + │ │ │ │ │ │ │ │ │ │ ├── fd: ()-->(36) + │ │ │ │ │ │ │ │ │ │ ├── ordering: +19 opt(36) [actual: +19] │ │ │ │ │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18!null project_id:19!null instance_type_projects.deleted:20!null + │ │ │ │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:19!null project_id:20!null instance_type_projects.deleted:21!null │ │ │ │ │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ │ │ │ │ ├── key: (18-20) - │ │ │ │ │ │ │ │ │ │ │ ├── ordering: +18 + │ │ │ │ │ │ │ │ │ │ │ ├── key: (19-21) + │ │ │ │ │ │ │ │ │ │ │ ├── ordering: +19 │ │ │ │ │ │ │ │ │ │ │ ├── scan instance_type_projects@secondary - │ │ │ │ │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18!null project_id:19 instance_type_projects.deleted:20 - │ │ │ │ │ │ │ │ │ │ │ │ ├── lax-key: (18-20) - │ │ │ │ │ │ │ │ │ │ │ │ └── ordering: +18 + │ │ │ │ │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:19!null project_id:20 instance_type_projects.deleted:21 + │ │ │ │ │ │ │ │ │ │ │ │ ├── lax-key: (19-21) + │ │ │ │ │ │ │ │ │ │ │ │ └── ordering: +19 │ │ │ │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ │ │ │ ├── instance_type_projects.deleted:20 = $2 [outer=(20), constraints=(/20: (/NULL - ])] - │ │ │ │ │ │ │ │ │ │ │ └── project_id:19 = $3 [outer=(19), constraints=(/19: (/NULL - ])] + │ │ │ │ │ │ │ │ │ │ │ ├── instance_type_projects.deleted:21 = $2 [outer=(21), constraints=(/21: (/NULL - ])] + │ │ │ │ │ │ │ │ │ │ │ └── project_id:20 = $3 [outer=(20), constraints=(/20: (/NULL - ])] │ │ │ │ │ │ │ │ │ │ └── projections - │ │ │ │ │ │ │ │ │ │ └── true [as=true:33] + │ │ │ │ │ │ │ │ │ │ └── true [as=true:36] │ │ │ │ │ │ │ │ │ └── filters (true) │ │ │ │ │ │ │ │ └── aggregations - │ │ │ │ │ │ │ │ ├── const-not-null-agg [as=true_agg:34, outer=(33)] - │ │ │ │ │ │ │ │ │ └── true:33 + │ │ │ │ │ │ │ │ ├── const-not-null-agg [as=true_agg:37, outer=(36)] + │ │ │ │ │ │ │ │ │ └── true:36 │ │ │ │ │ │ │ │ ├── const-agg [as=name:2, outer=(2)] │ │ │ │ │ │ │ │ │ └── name:2 │ │ │ │ │ │ │ │ ├── const-agg [as=memory_mb:3, outer=(3)] @@ -2833,31 +2833,31 @@ project │ │ │ │ │ │ │ │ └── const-agg [as=instance_types.updated_at:16, outer=(16)] │ │ │ │ │ │ │ │ └── instance_types.updated_at:16 │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── (is_public:12 = true) OR (true_agg:34 IS NOT NULL) [outer=(12,34)] + │ │ │ │ │ │ │ └── (is_public:12 = true) OR (true_agg:37 IS NOT NULL) [outer=(12,37)] │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ ├── columns: true:36!null instance_type_projects.instance_type_id:26!null + │ │ │ │ │ │ │ ├── columns: true:39!null instance_type_projects.instance_type_id:28!null │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ ├── fd: ()-->(36) - │ │ │ │ │ │ │ ├── ordering: +26 opt(36) [actual: +26] + │ │ │ │ │ │ │ ├── fd: ()-->(39) + │ │ │ │ │ │ │ ├── ordering: +28 opt(39) [actual: +28] │ │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:26!null project_id:27!null instance_type_projects.deleted:28!null + │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:28!null project_id:29!null instance_type_projects.deleted:30!null │ │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ │ ├── key: (26-28) - │ │ │ │ │ │ │ │ ├── ordering: +26 + │ │ │ │ │ │ │ │ ├── key: (28-30) + │ │ │ │ │ │ │ │ ├── ordering: +28 │ │ │ │ │ │ │ │ ├── scan instance_type_projects@secondary - │ │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:26!null project_id:27 instance_type_projects.deleted:28 - │ │ │ │ │ │ │ │ │ ├── lax-key: (26-28) - │ │ │ │ │ │ │ │ │ └── ordering: +26 + │ │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:28!null project_id:29 instance_type_projects.deleted:30 + │ │ │ │ │ │ │ │ │ ├── lax-key: (28-30) + │ │ │ │ │ │ │ │ │ └── ordering: +28 │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ ├── instance_type_projects.deleted:28 = $4 [outer=(28), constraints=(/28: (/NULL - ])] - │ │ │ │ │ │ │ │ ├── instance_type_projects.deleted:28 = $5 [outer=(28), constraints=(/28: (/NULL - ])] - │ │ │ │ │ │ │ │ └── project_id:27 = $6 [outer=(27), constraints=(/27: (/NULL - ])] + │ │ │ │ │ │ │ │ ├── instance_type_projects.deleted:30 = $4 [outer=(30), constraints=(/30: (/NULL - ])] + │ │ │ │ │ │ │ │ ├── instance_type_projects.deleted:30 = $5 [outer=(30), constraints=(/30: (/NULL - ])] + │ │ │ │ │ │ │ │ └── project_id:29 = $6 [outer=(29), constraints=(/29: (/NULL - ])] │ │ │ │ │ │ │ └── projections - │ │ │ │ │ │ │ └── true [as=true:36] + │ │ │ │ │ │ │ └── true [as=true:39] │ │ │ │ │ │ └── filters (true) │ │ │ │ │ └── aggregations - │ │ │ │ │ ├── const-not-null-agg [as=true_agg:37, outer=(36)] - │ │ │ │ │ │ └── true:36 + │ │ │ │ │ ├── const-not-null-agg [as=true_agg:40, outer=(39)] + │ │ │ │ │ │ └── true:39 │ │ │ │ │ ├── const-agg [as=name:2, outer=(2)] │ │ │ │ │ │ └── name:2 │ │ │ │ │ ├── const-agg [as=memory_mb:3, outer=(3)] @@ -2889,11 +2889,11 @@ project │ │ │ │ │ └── const-agg [as=instance_types.updated_at:16, outer=(16)] │ │ │ │ │ └── instance_types.updated_at:16 │ │ │ │ └── filters - │ │ │ │ └── (is_public:12 = true) OR (true_agg:37 IS NOT NULL) [outer=(12,37)] + │ │ │ │ └── (is_public:12 = true) OR (true_agg:40 IS NOT NULL) [outer=(12,40)] │ │ │ └── $7 │ │ └── $8 │ └── filters - │ └── instance_type_extra_specs_1.deleted:43 = $9 [outer=(43), constraints=(/43: (/NULL - ])] + │ └── instance_type_extra_specs_1.deleted:46 = $9 [outer=(46), constraints=(/46: (/NULL - ])] └── filters (true) opt @@ -2957,67 +2957,67 @@ order by anon_1.instance_types_deleted asc, anon_1.instance_types_id asc ---- sort - ├── columns: anon_1_instance_types_created_at:15 anon_1_instance_types_updated_at:16 anon_1_instance_types_deleted_at:14 anon_1_instance_types_deleted:13!null anon_1_instance_types_id:1!null anon_1_instance_types_name:2 anon_1_instance_types_memory_mb:3!null anon_1_instance_types_vcpus:4!null anon_1_instance_types_root_gb:5 anon_1_instance_types_ephemeral_gb:6 anon_1_instance_types_flavorid:7!null anon_1_instance_types_swap:8!null anon_1_instance_types_rxtx_factor:9 anon_1_instance_types_vcpu_weight:10 anon_1_instance_types_disabled:11 anon_1_instance_types_is_public:12 instance_type_extra_specs_1_created_at:34 instance_type_extra_specs_1_updated_at:35 instance_type_extra_specs_1_deleted_at:33 instance_type_extra_specs_1_deleted:32 instance_type_extra_specs_1_id:28 instance_type_extra_specs_1_key:29 instance_type_extra_specs_1_value:30 instance_type_extra_specs_1_instance_type_id:31 + ├── columns: anon_1_instance_types_created_at:15 anon_1_instance_types_updated_at:16 anon_1_instance_types_deleted_at:14 anon_1_instance_types_deleted:13!null anon_1_instance_types_id:1!null anon_1_instance_types_name:2 anon_1_instance_types_memory_mb:3!null anon_1_instance_types_vcpus:4!null anon_1_instance_types_root_gb:5 anon_1_instance_types_ephemeral_gb:6 anon_1_instance_types_flavorid:7!null anon_1_instance_types_swap:8!null anon_1_instance_types_rxtx_factor:9 anon_1_instance_types_vcpu_weight:10 anon_1_instance_types_disabled:11 anon_1_instance_types_is_public:12 instance_type_extra_specs_1_created_at:36 instance_type_extra_specs_1_updated_at:37 instance_type_extra_specs_1_deleted_at:35 instance_type_extra_specs_1_deleted:34 instance_type_extra_specs_1_id:30 instance_type_extra_specs_1_key:31 instance_type_extra_specs_1_value:32 instance_type_extra_specs_1_instance_type_id:33 ├── immutable, has-placeholder - ├── key: (1,28) - ├── fd: (1)-->(2-16), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + ├── key: (1,30) + ├── fd: (1)-->(2-16), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (30)-->(31-37), (31,33,34)~~>(30,32,35-37) ├── ordering: +13,+1 └── project - ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_extra_specs_1.id:28 key:29 value:30 instance_type_extra_specs_1.instance_type_id:31 instance_type_extra_specs_1.deleted:32 instance_type_extra_specs_1.deleted_at:33 instance_type_extra_specs_1.created_at:34 instance_type_extra_specs_1.updated_at:35 + ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_extra_specs_1.id:30 key:31 value:32 instance_type_extra_specs_1.instance_type_id:33 instance_type_extra_specs_1.deleted:34 instance_type_extra_specs_1.deleted_at:35 instance_type_extra_specs_1.created_at:36 instance_type_extra_specs_1.updated_at:37 ├── immutable, has-placeholder - ├── key: (1,28) - ├── fd: (1)-->(2-16), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + ├── key: (1,30) + ├── fd: (1)-->(2-16), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (30)-->(31-37), (31,33,34)~~>(30,32,35-37) └── right-join (hash) - ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 instance_type_extra_specs_1.id:28 key:29 value:30 instance_type_extra_specs_1.instance_type_id:31 instance_type_extra_specs_1.deleted:32 instance_type_extra_specs_1.deleted_at:33 instance_type_extra_specs_1.created_at:34 instance_type_extra_specs_1.updated_at:35 + ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 instance_type_extra_specs_1.id:30 key:31 value:32 instance_type_extra_specs_1.instance_type_id:33 instance_type_extra_specs_1.deleted:34 instance_type_extra_specs_1.deleted_at:35 instance_type_extra_specs_1.created_at:36 instance_type_extra_specs_1.updated_at:37 ├── immutable, has-placeholder - ├── key: (1,28) - ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + ├── key: (1,30) + ├── fd: (1)-->(2-16,28), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), (30)-->(31-37), (31,33,34)~~>(30,32,35-37) ├── select - │ ├── columns: instance_type_extra_specs_1.id:28!null key:29 value:30 instance_type_extra_specs_1.instance_type_id:31!null instance_type_extra_specs_1.deleted:32!null instance_type_extra_specs_1.deleted_at:33 instance_type_extra_specs_1.created_at:34 instance_type_extra_specs_1.updated_at:35 + │ ├── columns: instance_type_extra_specs_1.id:30!null key:31 value:32 instance_type_extra_specs_1.instance_type_id:33!null instance_type_extra_specs_1.deleted:34!null instance_type_extra_specs_1.deleted_at:35 instance_type_extra_specs_1.created_at:36 instance_type_extra_specs_1.updated_at:37 │ ├── has-placeholder - │ ├── key: (28) - │ ├── fd: (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + │ ├── key: (30) + │ ├── fd: (30)-->(31-37), (31,33,34)~~>(30,32,35-37) │ ├── scan instance_type_extra_specs_1 - │ │ ├── columns: instance_type_extra_specs_1.id:28!null key:29 value:30 instance_type_extra_specs_1.instance_type_id:31!null instance_type_extra_specs_1.deleted:32 instance_type_extra_specs_1.deleted_at:33 instance_type_extra_specs_1.created_at:34 instance_type_extra_specs_1.updated_at:35 - │ │ ├── key: (28) - │ │ └── fd: (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + │ │ ├── columns: instance_type_extra_specs_1.id:30!null key:31 value:32 instance_type_extra_specs_1.instance_type_id:33!null instance_type_extra_specs_1.deleted:34 instance_type_extra_specs_1.deleted_at:35 instance_type_extra_specs_1.created_at:36 instance_type_extra_specs_1.updated_at:37 + │ │ ├── key: (30) + │ │ └── fd: (30)-->(31-37), (31,33,34)~~>(30,32,35-37) │ └── filters - │ └── instance_type_extra_specs_1.deleted:32 = $7 [outer=(32), constraints=(/32: (/NULL - ])] + │ └── instance_type_extra_specs_1.deleted:34 = $7 [outer=(34), constraints=(/34: (/NULL - ])] ├── limit - │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ ├── internal-ordering: +13,+1 │ ├── immutable, has-placeholder │ ├── key: (1) - │ ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ ├── fd: (1)-->(2-16,28), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ ├── offset - │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ │ ├── internal-ordering: +13,+1 │ │ ├── has-placeholder │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ ├── fd: (1)-->(2-16,28), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ ├── ordering: +13,+1 │ │ ├── sort - │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ │ │ ├── has-placeholder │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ │ ├── fd: (1)-->(2-16,28), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ │ ├── ordering: +13,+1 │ │ │ └── select - │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ │ │ ├── has-placeholder │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ │ ├── fd: (1)-->(2-16,28), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ │ ├── group-by - │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:26 + │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 true_agg:28 │ │ │ │ ├── grouping columns: instance_types.id:1!null │ │ │ │ ├── internal-ordering: +1 │ │ │ │ ├── has-placeholder │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ │ │ ├── fd: (1)-->(2-16,28), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ │ │ ├── left-join (merge) - │ │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_projects.instance_type_id:18 true:25 + │ │ │ │ │ ├── columns: instance_types.id:1!null name:2 memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 instance_types.deleted:13!null instance_types.deleted_at:14 instance_types.created_at:15 instance_types.updated_at:16 instance_type_projects.instance_type_id:19 true:27 │ │ │ │ │ ├── left ordering: +1 - │ │ │ │ │ ├── right ordering: +18 + │ │ │ │ │ ├── right ordering: +19 │ │ │ │ │ ├── has-placeholder │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ │ │ │ ├── ordering: +1 @@ -3036,28 +3036,28 @@ sort │ │ │ │ │ │ ├── instance_types.deleted:13 = $1 [outer=(13), constraints=(/13: (/NULL - ])] │ │ │ │ │ │ └── flavorid:7 = $4 [outer=(7), constraints=(/7: (/NULL - ])] │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: true:25!null instance_type_projects.instance_type_id:18!null + │ │ │ │ │ │ ├── columns: true:27!null instance_type_projects.instance_type_id:19!null │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ ├── fd: ()-->(25) - │ │ │ │ │ │ ├── ordering: +18 opt(25) [actual: +18] + │ │ │ │ │ │ ├── fd: ()-->(27) + │ │ │ │ │ │ ├── ordering: +19 opt(27) [actual: +19] │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18!null project_id:19!null instance_type_projects.deleted:20!null + │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:19!null project_id:20!null instance_type_projects.deleted:21!null │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ ├── key: (18-20) - │ │ │ │ │ │ │ ├── ordering: +18 + │ │ │ │ │ │ │ ├── key: (19-21) + │ │ │ │ │ │ │ ├── ordering: +19 │ │ │ │ │ │ │ ├── scan instance_type_projects@secondary - │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18!null project_id:19 instance_type_projects.deleted:20 - │ │ │ │ │ │ │ │ ├── lax-key: (18-20) - │ │ │ │ │ │ │ │ └── ordering: +18 + │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:19!null project_id:20 instance_type_projects.deleted:21 + │ │ │ │ │ │ │ │ ├── lax-key: (19-21) + │ │ │ │ │ │ │ │ └── ordering: +19 │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ ├── instance_type_projects.deleted:20 = $2 [outer=(20), constraints=(/20: (/NULL - ])] - │ │ │ │ │ │ │ └── project_id:19 = $3 [outer=(19), constraints=(/19: (/NULL - ])] + │ │ │ │ │ │ │ ├── instance_type_projects.deleted:21 = $2 [outer=(21), constraints=(/21: (/NULL - ])] + │ │ │ │ │ │ │ └── project_id:20 = $3 [outer=(20), constraints=(/20: (/NULL - ])] │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── true [as=true:25] + │ │ │ │ │ │ └── true [as=true:27] │ │ │ │ │ └── filters (true) │ │ │ │ └── aggregations - │ │ │ │ ├── const-not-null-agg [as=true_agg:26, outer=(25)] - │ │ │ │ │ └── true:25 + │ │ │ │ ├── const-not-null-agg [as=true_agg:28, outer=(27)] + │ │ │ │ │ └── true:27 │ │ │ │ ├── const-agg [as=name:2, outer=(2)] │ │ │ │ │ └── name:2 │ │ │ │ ├── const-agg [as=memory_mb:3, outer=(3)] @@ -3089,11 +3089,11 @@ sort │ │ │ │ └── const-agg [as=instance_types.updated_at:16, outer=(16)] │ │ │ │ └── instance_types.updated_at:16 │ │ │ └── filters - │ │ │ └── (is_public:12 = true) OR (true_agg:26 IS NOT NULL) [outer=(12,26)] + │ │ │ └── (is_public:12 = true) OR (true_agg:28 IS NOT NULL) [outer=(12,28)] │ │ └── $5 │ └── $6 └── filters - └── instance_type_extra_specs_1.instance_type_id:31 = instance_types.id:1 [outer=(1,31), constraints=(/1: (/NULL - ]; /31: (/NULL - ]), fd=(1)==(31), (31)==(1)] + └── instance_type_extra_specs_1.instance_type_id:33 = instance_types.id:1 [outer=(1,33), constraints=(/1: (/NULL - ]; /33: (/NULL - ]), fd=(1)==(33), (33)==(1)] opt select anon_1.flavors_created_at as anon_1_flavors_created_at, @@ -3144,45 +3144,45 @@ from (select flavors.created_at as flavors_created_at, on flavor_extra_specs_1.flavor_id = anon_1.flavors_id ---- project - ├── columns: anon_1_flavors_created_at:14 anon_1_flavors_updated_at:15 anon_1_flavors_id:1!null anon_1_flavors_name:2!null anon_1_flavors_memory_mb:3!null anon_1_flavors_vcpus:4!null anon_1_flavors_root_gb:5 anon_1_flavors_ephemeral_gb:6 anon_1_flavors_flavorid:7!null anon_1_flavors_swap:8!null anon_1_flavors_rxtx_factor:9 anon_1_flavors_vcpu_weight:10 anon_1_flavors_disabled:11 anon_1_flavors_is_public:12 flavor_extra_specs_1_created_at:29 flavor_extra_specs_1_updated_at:30 flavor_extra_specs_1_id:25 flavor_extra_specs_1_key:26 flavor_extra_specs_1_value:27 flavor_extra_specs_1_flavor_id:28 + ├── columns: anon_1_flavors_created_at:14 anon_1_flavors_updated_at:15 anon_1_flavors_id:1!null anon_1_flavors_name:2!null anon_1_flavors_memory_mb:3!null anon_1_flavors_vcpus:4!null anon_1_flavors_root_gb:5 anon_1_flavors_ephemeral_gb:6 anon_1_flavors_flavorid:7!null anon_1_flavors_swap:8!null anon_1_flavors_rxtx_factor:9 anon_1_flavors_vcpu_weight:10 anon_1_flavors_disabled:11 anon_1_flavors_is_public:12 flavor_extra_specs_1_created_at:31 flavor_extra_specs_1_updated_at:32 flavor_extra_specs_1_id:27 flavor_extra_specs_1_key:28 flavor_extra_specs_1_value:29 flavor_extra_specs_1_flavor_id:30 ├── immutable, has-placeholder - ├── key: (1,25) - ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (25)-->(26-30), (26,28)-->(25,27,29,30) + ├── key: (1,27) + ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (27)-->(28-32), (28,30)-->(27,29,31,32) └── right-join (hash) - ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 flavor_extra_specs_1.id:25 key:26 value:27 flavor_extra_specs_1.flavor_id:28 flavor_extra_specs_1.created_at:29 flavor_extra_specs_1.updated_at:30 + ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 flavor_extra_specs_1.id:27 key:28 value:29 flavor_extra_specs_1.flavor_id:30 flavor_extra_specs_1.created_at:31 flavor_extra_specs_1.updated_at:32 ├── immutable, has-placeholder - ├── key: (1,25) - ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (25)-->(26-30), (26,28)-->(25,27,29,30) + ├── key: (1,27) + ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15), (27)-->(28-32), (28,30)-->(27,29,31,32) ├── scan flavor_extra_specs_1 - │ ├── columns: flavor_extra_specs_1.id:25!null key:26!null value:27 flavor_extra_specs_1.flavor_id:28!null flavor_extra_specs_1.created_at:29 flavor_extra_specs_1.updated_at:30 - │ ├── key: (25) - │ └── fd: (25)-->(26-30), (26,28)-->(25,27,29,30) + │ ├── columns: flavor_extra_specs_1.id:27!null key:28!null value:29 flavor_extra_specs_1.flavor_id:30!null flavor_extra_specs_1.created_at:31 flavor_extra_specs_1.updated_at:32 + │ ├── key: (27) + │ └── fd: (27)-->(28-32), (28,30)-->(27,29,31,32) ├── limit - │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 + │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 │ ├── immutable, has-placeholder │ ├── key: (1) - │ ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) + │ ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ ├── offset - │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 + │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 │ │ ├── has-placeholder │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) + │ │ ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ ├── select - │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 + │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 │ │ │ ├── has-placeholder │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) + │ │ │ ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ │ ├── group-by - │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:23 + │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 true_agg:25 │ │ │ │ ├── grouping columns: flavors.id:1!null │ │ │ │ ├── internal-ordering: +1 │ │ │ │ ├── has-placeholder │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(2-12,14,15,23), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) + │ │ │ │ ├── fd: (1)-->(2-12,14,15,25), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ │ │ ├── left-join (merge) - │ │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 flavor_projects.flavor_id:17 true:22 + │ │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 flavors.created_at:14 flavors.updated_at:15 flavor_projects.flavor_id:18 true:24 │ │ │ │ │ ├── left ordering: +1 - │ │ │ │ │ ├── right ordering: +17 + │ │ │ │ │ ├── right ordering: +18 │ │ │ │ │ ├── has-placeholder │ │ │ │ │ ├── fd: (1)-->(2-12,14,15), (7)-->(1-6,8-12,14,15), (2)-->(1,3-12,14,15) │ │ │ │ │ ├── ordering: +1 @@ -3200,28 +3200,28 @@ project │ │ │ │ │ │ └── filters │ │ │ │ │ │ └── flavors.id:1 = $2 [outer=(1), constraints=(/1: (/NULL - ])] │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: true:22!null flavor_projects.flavor_id:17!null + │ │ │ │ │ │ ├── columns: true:24!null flavor_projects.flavor_id:18!null │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ ├── fd: ()-->(22) - │ │ │ │ │ │ ├── ordering: +17 opt(22) [actual: +17] + │ │ │ │ │ │ ├── fd: ()-->(24) + │ │ │ │ │ │ ├── ordering: +18 opt(24) [actual: +18] │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:17!null project_id:18!null + │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:18!null project_id:19!null │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ ├── key: (17,18) - │ │ │ │ │ │ │ ├── ordering: +17 + │ │ │ │ │ │ │ ├── key: (18,19) + │ │ │ │ │ │ │ ├── ordering: +18 │ │ │ │ │ │ │ ├── scan flavor_projects@secondary - │ │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:17!null project_id:18!null - │ │ │ │ │ │ │ │ ├── key: (17,18) - │ │ │ │ │ │ │ │ └── ordering: +17 + │ │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:18!null project_id:19!null + │ │ │ │ │ │ │ │ ├── key: (18,19) + │ │ │ │ │ │ │ │ └── ordering: +18 │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ ├── project_id:18 = $1 [outer=(18), constraints=(/18: (/NULL - ])] - │ │ │ │ │ │ │ └── flavor_projects.flavor_id:17 = $2 [outer=(17), constraints=(/17: (/NULL - ])] + │ │ │ │ │ │ │ ├── project_id:19 = $1 [outer=(19), constraints=(/19: (/NULL - ])] + │ │ │ │ │ │ │ └── flavor_projects.flavor_id:18 = $2 [outer=(18), constraints=(/18: (/NULL - ])] │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── true [as=true:22] + │ │ │ │ │ │ └── true [as=true:24] │ │ │ │ │ └── filters (true) │ │ │ │ └── aggregations - │ │ │ │ ├── const-not-null-agg [as=true_agg:23, outer=(22)] - │ │ │ │ │ └── true:22 + │ │ │ │ ├── const-not-null-agg [as=true_agg:25, outer=(24)] + │ │ │ │ │ └── true:24 │ │ │ │ ├── const-agg [as=name:2, outer=(2)] │ │ │ │ │ └── name:2 │ │ │ │ ├── const-agg [as=memory_mb:3, outer=(3)] @@ -3249,11 +3249,11 @@ project │ │ │ │ └── const-agg [as=flavors.updated_at:15, outer=(15)] │ │ │ │ └── flavors.updated_at:15 │ │ │ └── filters - │ │ │ └── (is_public:12 = true) OR (true_agg:23 IS NOT NULL) [outer=(12,23)] + │ │ │ └── (is_public:12 = true) OR (true_agg:25 IS NOT NULL) [outer=(12,25)] │ │ └── $3 │ └── $4 └── filters - └── flavor_extra_specs_1.flavor_id:28 = flavors.id:1 [outer=(1,28), constraints=(/1: (/NULL - ]; /28: (/NULL - ]), fd=(1)==(28), (28)==(1)] + └── flavor_extra_specs_1.flavor_id:30 = flavors.id:1 [outer=(1,30), constraints=(/1: (/NULL - ]; /30: (/NULL - ]), fd=(1)==(30), (30)==(1)] opt select anon_1.flavors_created_at as anon_1_flavors_created_at, @@ -3306,53 +3306,53 @@ from (select flavors.created_at as flavors_created_at, order by anon_1.flavors_flavorid asc, anon_1.flavors_id asc ---- sort - ├── columns: anon_1_flavors_created_at:14 anon_1_flavors_updated_at:15 anon_1_flavors_id:1!null anon_1_flavors_name:2!null anon_1_flavors_memory_mb:3!null anon_1_flavors_vcpus:4!null anon_1_flavors_root_gb:5 anon_1_flavors_ephemeral_gb:6 anon_1_flavors_flavorid:7!null anon_1_flavors_swap:8!null anon_1_flavors_rxtx_factor:9 anon_1_flavors_vcpu_weight:10 anon_1_flavors_disabled:11 anon_1_flavors_is_public:12 anon_1_flavors_description:13 flavor_extra_specs_1_created_at:29 flavor_extra_specs_1_updated_at:30 flavor_extra_specs_1_id:25 flavor_extra_specs_1_key:26 flavor_extra_specs_1_value:27 flavor_extra_specs_1_flavor_id:28 + ├── columns: anon_1_flavors_created_at:14 anon_1_flavors_updated_at:15 anon_1_flavors_id:1!null anon_1_flavors_name:2!null anon_1_flavors_memory_mb:3!null anon_1_flavors_vcpus:4!null anon_1_flavors_root_gb:5 anon_1_flavors_ephemeral_gb:6 anon_1_flavors_flavorid:7!null anon_1_flavors_swap:8!null anon_1_flavors_rxtx_factor:9 anon_1_flavors_vcpu_weight:10 anon_1_flavors_disabled:11 anon_1_flavors_is_public:12 anon_1_flavors_description:13 flavor_extra_specs_1_created_at:31 flavor_extra_specs_1_updated_at:32 flavor_extra_specs_1_id:27 flavor_extra_specs_1_key:28 flavor_extra_specs_1_value:29 flavor_extra_specs_1_flavor_id:30 ├── immutable, has-placeholder - ├── key: (1,25) - ├── fd: (1)-->(2-15), (7)-->(1-6,8-15), (2)-->(1,3-15), (25)-->(26-30), (26,28)-->(25,27,29,30) + ├── key: (1,27) + ├── fd: (1)-->(2-15), (7)-->(1-6,8-15), (2)-->(1,3-15), (27)-->(28-32), (28,30)-->(27,29,31,32) ├── ordering: +7 └── project - ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 description:13 flavors.created_at:14 flavors.updated_at:15 flavor_extra_specs_1.id:25 key:26 value:27 flavor_extra_specs_1.flavor_id:28 flavor_extra_specs_1.created_at:29 flavor_extra_specs_1.updated_at:30 + ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 description:13 flavors.created_at:14 flavors.updated_at:15 flavor_extra_specs_1.id:27 key:28 value:29 flavor_extra_specs_1.flavor_id:30 flavor_extra_specs_1.created_at:31 flavor_extra_specs_1.updated_at:32 ├── immutable, has-placeholder - ├── key: (1,25) - ├── fd: (1)-->(2-15), (7)-->(1-6,8-15), (2)-->(1,3-15), (25)-->(26-30), (26,28)-->(25,27,29,30) + ├── key: (1,27) + ├── fd: (1)-->(2-15), (7)-->(1-6,8-15), (2)-->(1,3-15), (27)-->(28-32), (28,30)-->(27,29,31,32) └── right-join (hash) - ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 description:13 flavors.created_at:14 flavors.updated_at:15 true_agg:23 flavor_extra_specs_1.id:25 key:26 value:27 flavor_extra_specs_1.flavor_id:28 flavor_extra_specs_1.created_at:29 flavor_extra_specs_1.updated_at:30 + ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 description:13 flavors.created_at:14 flavors.updated_at:15 true_agg:25 flavor_extra_specs_1.id:27 key:28 value:29 flavor_extra_specs_1.flavor_id:30 flavor_extra_specs_1.created_at:31 flavor_extra_specs_1.updated_at:32 ├── immutable, has-placeholder - ├── key: (1,25) - ├── fd: (1)-->(2-15,23), (7)-->(1-6,8-15), (2)-->(1,3-15), (25)-->(26-30), (26,28)-->(25,27,29,30) + ├── key: (1,27) + ├── fd: (1)-->(2-15,25), (7)-->(1-6,8-15), (2)-->(1,3-15), (27)-->(28-32), (28,30)-->(27,29,31,32) ├── scan flavor_extra_specs_1 - │ ├── columns: flavor_extra_specs_1.id:25!null key:26!null value:27 flavor_extra_specs_1.flavor_id:28!null flavor_extra_specs_1.created_at:29 flavor_extra_specs_1.updated_at:30 - │ ├── key: (25) - │ └── fd: (25)-->(26-30), (26,28)-->(25,27,29,30) + │ ├── columns: flavor_extra_specs_1.id:27!null key:28!null value:29 flavor_extra_specs_1.flavor_id:30!null flavor_extra_specs_1.created_at:31 flavor_extra_specs_1.updated_at:32 + │ ├── key: (27) + │ └── fd: (27)-->(28-32), (28,30)-->(27,29,31,32) ├── limit - │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 description:13 flavors.created_at:14 flavors.updated_at:15 true_agg:23 + │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 description:13 flavors.created_at:14 flavors.updated_at:15 true_agg:25 │ ├── internal-ordering: +7 │ ├── immutable, has-placeholder │ ├── key: (1) - │ ├── fd: (1)-->(2-15,23), (7)-->(1-6,8-15), (2)-->(1,3-15) + │ ├── fd: (1)-->(2-15,25), (7)-->(1-6,8-15), (2)-->(1,3-15) │ ├── sort - │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 description:13 flavors.created_at:14 flavors.updated_at:15 true_agg:23 + │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 description:13 flavors.created_at:14 flavors.updated_at:15 true_agg:25 │ │ ├── has-placeholder │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2-15,23), (7)-->(1-6,8-15), (2)-->(1,3-15) + │ │ ├── fd: (1)-->(2-15,25), (7)-->(1-6,8-15), (2)-->(1,3-15) │ │ ├── ordering: +7 │ │ └── select - │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 description:13 flavors.created_at:14 flavors.updated_at:15 true_agg:23 + │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 description:13 flavors.created_at:14 flavors.updated_at:15 true_agg:25 │ │ ├── has-placeholder │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2-15,23), (7)-->(1-6,8-15), (2)-->(1,3-15) + │ │ ├── fd: (1)-->(2-15,25), (7)-->(1-6,8-15), (2)-->(1,3-15) │ │ ├── group-by - │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 description:13 flavors.created_at:14 flavors.updated_at:15 true_agg:23 + │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 description:13 flavors.created_at:14 flavors.updated_at:15 true_agg:25 │ │ │ ├── grouping columns: flavors.id:1!null │ │ │ ├── internal-ordering: +1 │ │ │ ├── has-placeholder │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(2-15,23), (7)-->(1-6,8-15), (2)-->(1,3-15) + │ │ │ ├── fd: (1)-->(2-15,25), (7)-->(1-6,8-15), (2)-->(1,3-15) │ │ │ ├── left-join (merge) - │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 description:13 flavors.created_at:14 flavors.updated_at:15 flavor_projects.flavor_id:17 true:22 + │ │ │ │ ├── columns: flavors.id:1!null name:2!null memory_mb:3!null vcpus:4!null root_gb:5 ephemeral_gb:6 flavorid:7!null swap:8!null rxtx_factor:9 vcpu_weight:10 disabled:11 is_public:12 description:13 flavors.created_at:14 flavors.updated_at:15 flavor_projects.flavor_id:18 true:24 │ │ │ │ ├── left ordering: +1 - │ │ │ │ ├── right ordering: +17 + │ │ │ │ ├── right ordering: +18 │ │ │ │ ├── has-placeholder │ │ │ │ ├── fd: (1)-->(2-15), (7)-->(1-6,8-15), (2)-->(1,3-15) │ │ │ │ ├── ordering: +1 @@ -3362,27 +3362,27 @@ sort │ │ │ │ │ ├── fd: (1)-->(2-15), (7)-->(1-6,8-15), (2)-->(1,3-15) │ │ │ │ │ └── ordering: +1 │ │ │ │ ├── project - │ │ │ │ │ ├── columns: true:22!null flavor_projects.flavor_id:17!null + │ │ │ │ │ ├── columns: true:24!null flavor_projects.flavor_id:18!null │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ ├── fd: ()-->(22) - │ │ │ │ │ ├── ordering: +17 opt(22) [actual: +17] + │ │ │ │ │ ├── fd: ()-->(24) + │ │ │ │ │ ├── ordering: +18 opt(24) [actual: +18] │ │ │ │ │ ├── select - │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:17!null project_id:18!null + │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:18!null project_id:19!null │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ ├── key: (17,18) - │ │ │ │ │ │ ├── ordering: +17 + │ │ │ │ │ │ ├── key: (18,19) + │ │ │ │ │ │ ├── ordering: +18 │ │ │ │ │ │ ├── scan flavor_projects@secondary - │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:17!null project_id:18!null - │ │ │ │ │ │ │ ├── key: (17,18) - │ │ │ │ │ │ │ └── ordering: +17 + │ │ │ │ │ │ │ ├── columns: flavor_projects.flavor_id:18!null project_id:19!null + │ │ │ │ │ │ │ ├── key: (18,19) + │ │ │ │ │ │ │ └── ordering: +18 │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── project_id:18 = $1 [outer=(18), constraints=(/18: (/NULL - ])] + │ │ │ │ │ │ └── project_id:19 = $1 [outer=(19), constraints=(/19: (/NULL - ])] │ │ │ │ │ └── projections - │ │ │ │ │ └── true [as=true:22] + │ │ │ │ │ └── true [as=true:24] │ │ │ │ └── filters (true) │ │ │ └── aggregations - │ │ │ ├── const-not-null-agg [as=true_agg:23, outer=(22)] - │ │ │ │ └── true:22 + │ │ │ ├── const-not-null-agg [as=true_agg:25, outer=(24)] + │ │ │ │ └── true:24 │ │ │ ├── const-agg [as=name:2, outer=(2)] │ │ │ │ └── name:2 │ │ │ ├── const-agg [as=memory_mb:3, outer=(3)] @@ -3412,7 +3412,7 @@ sort │ │ │ └── const-agg [as=flavors.updated_at:15, outer=(15)] │ │ │ └── flavors.updated_at:15 │ │ └── filters - │ │ └── (is_public:12 = true) OR (true_agg:23 IS NOT NULL) [outer=(12,23)] + │ │ └── (is_public:12 = true) OR (true_agg:25 IS NOT NULL) [outer=(12,25)] │ └── $2 └── filters - └── flavor_extra_specs_1.flavor_id:28 = flavors.id:1 [outer=(1,28), constraints=(/1: (/NULL - ]; /28: (/NULL - ]), fd=(1)==(28), (28)==(1)] + └── flavor_extra_specs_1.flavor_id:30 = flavors.id:1 [outer=(1,30), constraints=(/1: (/NULL - ]; /30: (/NULL - ]), fd=(1)==(30), (30)==(1)] diff --git a/pkg/sql/opt/xform/testdata/external/pgadmin b/pkg/sql/opt/xform/testdata/external/pgadmin index 91e171bfcbb4..d56a317adaec 100644 --- a/pkg/sql/opt/xform/testdata/external/pgadmin +++ b/pkg/sql/opt/xform/testdata/external/pgadmin @@ -80,37 +80,37 @@ WHERE ) ---- project - ├── columns: PID:3 User:5 Database:2 "Backend start":10 Client:37 Application:6 Query:19 "Query start":12 "Xact start":11 + ├── columns: PID:3 User:5 Database:2 "Backend start":10 Client:39 Application:6 Query:19 "Query start":12 "Xact start":11 ├── immutable ├── select - │ ├── columns: datname:2 pid:3 username:5 application_name:6 client_addr:7 client_hostname:8 client_port:9 backend_start:10 xact_start:11 query_start:12 query:19 sa.rowid:20!null "?column?":36!null + │ ├── columns: datname:2 pid:3 username:5 application_name:6 client_addr:7 client_hostname:8 client_port:9 backend_start:10 xact_start:11 query_start:12 query:19 sa.rowid:20!null "?column?":38!null │ ├── key: (20) - │ ├── fd: ()-->(36), (20)-->(2,3,5-12,19) + │ ├── fd: ()-->(38), (20)-->(2,3,5-12,19) │ ├── ensure-distinct-on - │ │ ├── columns: datname:2 pid:3 username:5 application_name:6 client_addr:7 client_hostname:8 client_port:9 backend_start:10 xact_start:11 query_start:12 query:19 sa.rowid:20!null "?column?":36 + │ │ ├── columns: datname:2 pid:3 username:5 application_name:6 client_addr:7 client_hostname:8 client_port:9 backend_start:10 xact_start:11 query_start:12 query:19 sa.rowid:20!null "?column?":38 │ │ ├── grouping columns: sa.rowid:20!null │ │ ├── error: "more than one row returned by a subquery used as an expression" │ │ ├── key: (20) - │ │ ├── fd: (20)-->(2,3,5-12,19,36) + │ │ ├── fd: (20)-->(2,3,5-12,19,38) │ │ ├── left-join-apply - │ │ │ ├── columns: datname:2 pid:3 usesysid:4 username:5 application_name:6 client_addr:7 client_hostname:8 client_port:9 backend_start:10 xact_start:11 query_start:12 query:19 sa.rowid:20!null "?column?":36 + │ │ │ ├── columns: datname:2 pid:3 usesysid:4 username:5 application_name:6 client_addr:7 client_hostname:8 client_port:9 backend_start:10 xact_start:11 query_start:12 query:19 sa.rowid:20!null "?column?":38 │ │ │ ├── fd: (20)-->(2-12,19) │ │ │ ├── scan sa │ │ │ │ ├── columns: datname:2 pid:3 usesysid:4 username:5 application_name:6 client_addr:7 client_hostname:8 client_port:9 backend_start:10 xact_start:11 query_start:12 query:19 sa.rowid:20!null │ │ │ │ ├── key: (20) │ │ │ │ └── fd: (20)-->(2-12,19) │ │ │ ├── project - │ │ │ │ ├── columns: "?column?":36 + │ │ │ │ ├── columns: "?column?":38 │ │ │ │ ├── outer: (4) │ │ │ │ ├── select - │ │ │ │ │ ├── columns: oid:21 rolname:22!null rolsuper:23 - │ │ │ │ │ ├── fd: ()-->(22) + │ │ │ │ │ ├── columns: oid:22 rolname:23!null rolsuper:24 + │ │ │ │ │ ├── fd: ()-->(23) │ │ │ │ │ ├── scan r - │ │ │ │ │ │ └── columns: oid:21 rolname:22 rolsuper:23 + │ │ │ │ │ │ └── columns: oid:22 rolname:23 rolsuper:24 │ │ │ │ │ └── filters - │ │ │ │ │ └── rolname:22 = 'opttester' [outer=(22), constraints=(/22: [/'opttester' - /'opttester']; tight), fd=()-->(22)] + │ │ │ │ │ └── rolname:23 = 'opttester' [outer=(23), constraints=(/23: [/'opttester' - /'opttester']; tight), fd=()-->(23)] │ │ │ │ └── projections - │ │ │ │ └── rolsuper:23 OR (oid:21 = usesysid:4) [as="?column?":36, outer=(4,21,23)] + │ │ │ │ └── rolsuper:24 OR (oid:22 = usesysid:4) [as="?column?":38, outer=(4,22,24)] │ │ │ └── filters (true) │ │ └── aggregations │ │ ├── const-agg [as=datname:2, outer=(2)] @@ -135,9 +135,9 @@ project │ │ │ └── query_start:12 │ │ ├── const-agg [as=query:19, outer=(19)] │ │ │ └── query:19 - │ │ └── const-agg [as="?column?":36, outer=(36)] - │ │ └── "?column?":36 + │ │ └── const-agg [as="?column?":38, outer=(38)] + │ │ └── "?column?":38 │ └── filters - │ └── "?column?":36 [outer=(36), constraints=(/36: [/true - /true]; tight), fd=()-->(36)] + │ └── "?column?":38 [outer=(38), constraints=(/38: [/true - /true]; tight), fd=()-->(38)] └── projections - └── CASE WHEN (client_hostname:8 IS NOT NULL) AND (client_hostname:8 != '') THEN (client_hostname:8 || ':') || client_port:9::STRING WHEN (client_addr:7 IS NOT NULL) AND (client_addr:7::STRING != '') THEN (client_addr:7::STRING || ':') || client_port:9::STRING WHEN client_port:9 = -1 THEN 'local pipe' ELSE 'localhost:' || client_port:9::STRING END [as=Client:37, outer=(7-9), immutable] + └── CASE WHEN (client_hostname:8 IS NOT NULL) AND (client_hostname:8 != '') THEN (client_hostname:8 || ':') || client_port:9::STRING WHEN (client_addr:7 IS NOT NULL) AND (client_addr:7::STRING != '') THEN (client_addr:7::STRING || ':') || client_port:9::STRING WHEN client_port:9 = -1 THEN 'local pipe' ELSE 'localhost:' || client_port:9::STRING END [as=Client:39, outer=(7-9), immutable] diff --git a/pkg/sql/opt/xform/testdata/external/pgjdbc b/pkg/sql/opt/xform/testdata/external/pgjdbc index 7d4effcd80f4..17dbbeec6d43 100644 --- a/pkg/sql/opt/xform/testdata/external/pgjdbc +++ b/pkg/sql/opt/xform/testdata/external/pgjdbc @@ -81,43 +81,43 @@ WHERE t.typnamespace = n.oid AND n.nspname != 'pg_catalog'; ---- project - ├── columns: type_cat:71 type_schem:34!null type_name:2!null class_name:71 data_type:72 remarks:73 base_type:74 + ├── columns: type_cat:74 type_schem:35!null type_name:2!null class_name:74 data_type:75 remarks:76 base_type:77 ├── stable - ├── fd: ()-->(71) + ├── fd: ()-->(74) ├── ensure-distinct-on - │ ├── columns: t.oid:1 t.typname:2!null t.typtype:7 nspname:34!null case:70 rownum:75!null - │ ├── grouping columns: rownum:75!null + │ ├── columns: t.oid:1 t.typname:2!null t.typtype:7 nspname:35!null case:73 rownum:78!null + │ ├── grouping columns: rownum:78!null │ ├── error: "more than one row returned by a subquery used as an expression" - │ ├── key: (75) - │ ├── fd: (75)-->(1,2,7,34,70) + │ ├── key: (78) + │ ├── fd: (78)-->(1,2,7,35,73) │ ├── left-join (hash) - │ │ ├── columns: t.oid:1 t.typname:2!null t.typnamespace:3!null t.typtype:7 t.typbasetype:25 n.oid:33!null nspname:34!null pg_type.oid:38 case:70 rownum:75!null - │ │ ├── fd: (3)==(33), (33)==(3), (75)-->(1-3,7,25,33,34) + │ │ ├── columns: t.oid:1 t.typname:2!null t.typnamespace:3!null t.typtype:7 t.typbasetype:25 n.oid:34!null nspname:35!null pg_type.oid:40 case:73 rownum:78!null + │ │ ├── fd: (3)==(34), (34)==(3), (78)-->(1-3,7,25,34,35) │ │ ├── ordinality - │ │ │ ├── columns: t.oid:1 t.typname:2!null t.typnamespace:3!null t.typtype:7 t.typbasetype:25 n.oid:33!null nspname:34!null rownum:75!null - │ │ │ ├── key: (75) - │ │ │ ├── fd: (3)==(33), (33)==(3), (75)-->(1-3,7,25,33,34) + │ │ │ ├── columns: t.oid:1 t.typname:2!null t.typnamespace:3!null t.typtype:7 t.typbasetype:25 n.oid:34!null nspname:35!null rownum:78!null + │ │ │ ├── key: (78) + │ │ │ ├── fd: (3)==(34), (34)==(3), (78)-->(1-3,7,25,34,35) │ │ │ └── inner-join (hash) - │ │ │ ├── columns: t.oid:1 t.typname:2!null t.typnamespace:3!null t.typtype:7 t.typbasetype:25 n.oid:33!null nspname:34!null - │ │ │ ├── fd: (3)==(33), (33)==(3) + │ │ │ ├── columns: t.oid:1 t.typname:2!null t.typnamespace:3!null t.typtype:7 t.typbasetype:25 n.oid:34!null nspname:35!null + │ │ │ ├── fd: (3)==(34), (34)==(3) │ │ │ ├── scan t │ │ │ │ └── columns: t.oid:1 t.typname:2!null t.typnamespace:3 t.typtype:7 t.typbasetype:25 │ │ │ ├── select - │ │ │ │ ├── columns: n.oid:33 nspname:34!null + │ │ │ │ ├── columns: n.oid:34 nspname:35!null │ │ │ │ ├── scan n - │ │ │ │ │ └── columns: n.oid:33 nspname:34!null + │ │ │ │ │ └── columns: n.oid:34 nspname:35!null │ │ │ │ └── filters - │ │ │ │ └── nspname:34 != 'pg_catalog' [outer=(34), constraints=(/34: (/NULL - /'pg_catalog') [/e'pg_catalog\x00' - ]; tight)] + │ │ │ │ └── nspname:35 != 'pg_catalog' [outer=(35), constraints=(/35: (/NULL - /'pg_catalog') [/e'pg_catalog\x00' - ]; tight)] │ │ │ └── filters - │ │ │ └── t.typnamespace:3 = n.oid:33 [outer=(3,33), constraints=(/3: (/NULL - ]; /33: (/NULL - ]), fd=(3)==(33), (33)==(3)] + │ │ │ └── t.typnamespace:3 = n.oid:34 [outer=(3,34), constraints=(/3: (/NULL - ]; /34: (/NULL - ]), fd=(3)==(34), (34)==(3)] │ │ ├── project - │ │ │ ├── columns: case:70!null pg_type.oid:38 + │ │ │ ├── columns: case:73!null pg_type.oid:40 │ │ │ ├── scan pg_type - │ │ │ │ └── columns: pg_type.oid:38 pg_type.typname:39!null + │ │ │ │ └── columns: pg_type.oid:40 pg_type.typname:41!null │ │ │ └── projections - │ │ │ └── CASE WHEN pg_type.typname:39 = 'pgType' THEN 'sqlType' ELSE 'OTHER' END [as=case:70, outer=(39)] + │ │ │ └── CASE WHEN pg_type.typname:41 = 'pgType' THEN 'sqlType' ELSE 'OTHER' END [as=case:73, outer=(41)] │ │ └── filters - │ │ └── pg_type.oid:38 = t.typbasetype:25 [outer=(25,38), constraints=(/25: (/NULL - ]; /38: (/NULL - ]), fd=(25)==(38), (38)==(25)] + │ │ └── pg_type.oid:40 = t.typbasetype:25 [outer=(25,40), constraints=(/25: (/NULL - ]; /40: (/NULL - ]), fd=(25)==(40), (40)==(25)] │ └── aggregations │ ├── const-agg [as=t.oid:1, outer=(1)] │ │ └── t.oid:1 @@ -125,12 +125,12 @@ project │ │ └── t.typname:2 │ ├── const-agg [as=t.typtype:7, outer=(7)] │ │ └── t.typtype:7 - │ ├── const-agg [as=nspname:34, outer=(34)] - │ │ └── nspname:34 - │ └── const-agg [as=case:70, outer=(70)] - │ └── case:70 + │ ├── const-agg [as=nspname:35, outer=(35)] + │ │ └── nspname:35 + │ └── const-agg [as=case:73, outer=(73)] + │ └── case:73 └── projections - ├── NULL [as=type_cat:71] - ├── CASE WHEN t.typtype:7 = 'c' THEN 'STRUCT' ELSE 'DISTINCT' END [as=data_type:72, outer=(7)] - ├── obj_description(t.oid:1, 'pg_type') [as=remarks:73, outer=(1), stable] - └── CASE WHEN t.typtype:7 = 'd' THEN case:70 ELSE CAST(NULL AS STRING) END [as=base_type:74, outer=(7,70)] + ├── NULL [as=type_cat:74] + ├── CASE WHEN t.typtype:7 = 'c' THEN 'STRUCT' ELSE 'DISTINCT' END [as=data_type:75, outer=(7)] + ├── obj_description(t.oid:1, 'pg_type') [as=remarks:76, outer=(1), stable] + └── CASE WHEN t.typtype:7 = 'd' THEN case:73 ELSE CAST(NULL AS STRING) END [as=base_type:77, outer=(7,73)] diff --git a/pkg/sql/opt/xform/testdata/external/postgis-tutorial b/pkg/sql/opt/xform/testdata/external/postgis-tutorial index 6d1d92bbff96..a0ad05d68ead 100644 --- a/pkg/sql/opt/xform/testdata/external/postgis-tutorial +++ b/pkg/sql/opt/xform/testdata/external/postgis-tutorial @@ -6295,17 +6295,17 @@ GROUP BY ORDER BY black_pct DESC ---- -memo (optimized, ~29KB, required=[presentation: route:27,white_pct:32,black_pct:33,popn_total:30] [ordering: -33]) +memo (optimized, ~29KB, required=[presentation: route:29,white_pct:35,black_pct:36,popn_total:33] [ordering: -36]) ├── G1: (project G2 G3 route sum) - │ ├── [presentation: route:27,white_pct:32,black_pct:33,popn_total:30] [ordering: -33] + │ ├── [presentation: route:29,white_pct:35,black_pct:36,popn_total:33] [ordering: -36] │ │ ├── best: (sort G1) │ │ └── cost: 23087936.79 │ └── [] │ ├── best: (project G2 G3 route sum) │ └── cost: 23087934.38 - ├── G2: (group-by G4 G5 cols=(27)) + ├── G2: (group-by G4 G5 cols=(29)) │ └── [] - │ ├── best: (group-by G4 G5 cols=(27)) + │ ├── best: (group-by G4 G5 cols=(29)) │ └── cost: 23087933.71 ├── G3: (projections G6 G7) ├── G4: (inner-join G8 G9 G10) (inner-join G11 G12 G13) (inner-join G9 G8 G10) (inner-join G12 G11 G13) @@ -6319,9 +6319,9 @@ memo (optimized, ~29KB, required=[presentation: route:27,white_pct:32,black_pct: │ └── [] │ ├── best: (inner-join G11 G20 G13) │ └── cost: 19283630.66 - ├── G9: (scan lines,cols=(27)) + ├── G9: (scan lines,cols=(29)) │ └── [] - │ ├── best: (scan lines,cols=(27)) + │ ├── best: (scan lines,cols=(29)) │ └── cost: 22.68 ├── G10: (filters G21) ├── G11: (scan census,cols=(3-5,10)) @@ -6339,9 +6339,9 @@ memo (optimized, ~29KB, required=[presentation: route:27,white_pct:32,black_pct: ├── G17: (mult G26 G27) ├── G18: (variable sum) ├── G19: (mult G28 G27) - ├── G20: (scan subways,cols=(21,26)) + ├── G20: (scan subways,cols=(22,27)) │ └── [] - │ ├── best: (scan subways,cols=(21,26)) + │ ├── best: (scan subways,cols=(22,27)) │ └── cost: 579.40 ├── G21: (gt G29 G30) ├── G22: (function G31 st_dwithin) @@ -6377,45 +6377,45 @@ ORDER BY black_pct DESC ---- sort - ├── columns: route:27 white_pct:32 black_pct:33 popn_total:30 + ├── columns: route:29 white_pct:35 black_pct:36 popn_total:33 ├── immutable - ├── key: (27) - ├── fd: (27)-->(30,32,33) - ├── ordering: -33 + ├── key: (29) + ├── fd: (29)-->(33,35,36) + ├── ordering: -36 └── project - ├── columns: white_pct:32 black_pct:33 route:27 sum:30 + ├── columns: white_pct:35 black_pct:36 route:29 sum:33 ├── immutable - ├── key: (27) - ├── fd: (27)-->(30,32,33) + ├── key: (29) + ├── fd: (29)-->(33,35,36) ├── group-by - │ ├── columns: route:27 sum:29 sum:30 sum:31 - │ ├── grouping columns: route:27 + │ ├── columns: route:29 sum:32 sum:33 sum:34 + │ ├── grouping columns: route:29 │ ├── immutable - │ ├── key: (27) - │ ├── fd: (27)-->(29-31) + │ ├── key: (29) + │ ├── fd: (29)-->(32-34) │ ├── inner-join (cross) - │ │ ├── columns: popn_total:3 popn_white:4 popn_black:5 census.geom:10 routes:21 subways.geom:26 route:27 + │ │ ├── columns: popn_total:3 popn_white:4 popn_black:5 census.geom:10 routes:22 subways.geom:27 route:29 │ │ ├── immutable │ │ ├── inner-join (cross) - │ │ │ ├── columns: popn_total:3 popn_white:4 popn_black:5 census.geom:10 routes:21 subways.geom:26 + │ │ │ ├── columns: popn_total:3 popn_white:4 popn_black:5 census.geom:10 routes:22 subways.geom:27 │ │ │ ├── immutable │ │ │ ├── scan census │ │ │ │ └── columns: popn_total:3 popn_white:4 popn_black:5 census.geom:10 │ │ │ ├── scan subways - │ │ │ │ └── columns: routes:21 subways.geom:26 + │ │ │ │ └── columns: routes:22 subways.geom:27 │ │ │ └── filters - │ │ │ └── st_dwithin(census.geom:10, subways.geom:26, 200.0) [outer=(10,26), immutable] + │ │ │ └── st_dwithin(census.geom:10, subways.geom:27, 200.0) [outer=(10,27), immutable] │ │ ├── scan lines - │ │ │ └── columns: route:27 + │ │ │ └── columns: route:29 │ │ └── filters - │ │ └── strpos(routes:21, route:27) > 0 [outer=(21,27), immutable] + │ │ └── strpos(routes:22, route:29) > 0 [outer=(22,29), immutable] │ └── aggregations - │ ├── sum [as=sum:29, outer=(4)] + │ ├── sum [as=sum:32, outer=(4)] │ │ └── popn_white:4 - │ ├── sum [as=sum:30, outer=(3)] + │ ├── sum [as=sum:33, outer=(3)] │ │ └── popn_total:3 - │ └── sum [as=sum:31, outer=(5)] + │ └── sum [as=sum:34, outer=(5)] │ └── popn_black:5 └── projections - ├── (sum:29 * 100.0) / sum:30 [as=white_pct:32, outer=(29,30), immutable] - └── (sum:31 * 100.0) / sum:30 [as=black_pct:33, outer=(30,31), immutable] + ├── (sum:32 * 100.0) / sum:33 [as=white_pct:35, outer=(32,33), immutable] + └── (sum:34 * 100.0) / sum:33 [as=black_pct:36, outer=(33,34), immutable] diff --git a/pkg/sql/opt/xform/testdata/external/tpcc b/pkg/sql/opt/xform/testdata/external/tpcc index e21b23fc643d..b741ae0685a9 100644 --- a/pkg/sql/opt/xform/testdata/external/tpcc +++ b/pkg/sql/opt/xform/testdata/external/tpcc @@ -29,27 +29,27 @@ project ├── fd: ()-->(9,11) └── update district ├── columns: d_id:1!null d_w_id:2!null d_tax:9 d_next_o_id:11 - ├── fetch columns: d_id:12 d_w_id:13 d_name:14 d_street_1:15 d_street_2:16 d_city:17 d_state:18 d_zip:19 d_tax:20 d_ytd:21 d_next_o_id:22 + ├── fetch columns: d_id:13 d_w_id:14 d_name:15 d_street_1:16 d_street_2:17 d_city:18 d_state:19 d_zip:20 d_tax:21 d_ytd:22 d_next_o_id:23 ├── update-mapping: - │ └── d_next_o_id_new:23 => d_next_o_id:11 + │ └── d_next_o_id_new:25 => d_next_o_id:11 ├── cardinality: [0 - 1] ├── volatile, mutations ├── key: () ├── fd: ()-->(1,2,9,11) └── project - ├── columns: d_next_o_id_new:23 d_id:12!null d_w_id:13!null d_name:14 d_street_1:15 d_street_2:16 d_city:17 d_state:18 d_zip:19 d_tax:20 d_ytd:21 d_next_o_id:22 + ├── columns: d_next_o_id_new:25 d_id:13!null d_w_id:14!null d_name:15 d_street_1:16 d_street_2:17 d_city:18 d_state:19 d_zip:20 d_tax:21 d_ytd:22 d_next_o_id:23 ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(12-23) + ├── fd: ()-->(13-23,25) ├── scan district - │ ├── columns: d_id:12!null d_w_id:13!null d_name:14 d_street_1:15 d_street_2:16 d_city:17 d_state:18 d_zip:19 d_tax:20 d_ytd:21 d_next_o_id:22 - │ ├── constraint: /13/12: [/10/5 - /10/5] + │ ├── columns: d_id:13!null d_w_id:14!null d_name:15 d_street_1:16 d_street_2:17 d_city:18 d_state:19 d_zip:20 d_tax:21 d_ytd:22 d_next_o_id:23 + │ ├── constraint: /14/13: [/10/5 - /10/5] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(12-22) + │ └── fd: ()-->(13-23) └── projections - └── d_next_o_id:22 + 1 [as=d_next_o_id_new:23, outer=(22), immutable] + └── d_next_o_id:23 + 1 [as=d_next_o_id_new:25, outer=(23), immutable] opt format=hide-qual SELECT w_tax FROM warehouse WHERE w_id = 10 @@ -141,41 +141,41 @@ VALUES (100, 5, 10, 50, '2019-08-26 16:50:41', 10, 1) insert "order" ├── columns: ├── insert-mapping: - │ ├── column1:9 => o_id:1 - │ ├── column2:10 => o_d_id:2 - │ ├── column3:11 => o_w_id:3 - │ ├── column4:12 => o_c_id:4 - │ ├── column5:13 => o_entry_d:5 - │ ├── column16:16 => o_carrier_id:6 - │ ├── column6:14 => o_ol_cnt:7 - │ └── column7:15 => o_all_local:8 + │ ├── column1:10 => o_id:1 + │ ├── column2:11 => o_d_id:2 + │ ├── column3:12 => o_w_id:3 + │ ├── column4:13 => o_c_id:4 + │ ├── column5:14 => o_entry_d:5 + │ ├── column17:17 => o_carrier_id:6 + │ ├── column6:15 => o_ol_cnt:7 + │ └── column7:16 => o_all_local:8 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── values - │ ├── columns: column1:9!null column2:10!null column3:11!null column4:12!null column5:13!null column6:14!null column7:15!null column16:16 + │ ├── columns: column1:10!null column2:11!null column3:12!null column4:13!null column5:14!null column6:15!null column7:16!null column17:17 │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(9-16) + │ ├── fd: ()-->(10-17) │ └── (100, 5, 10, 50, '2019-08-26 16:50:41+00:00', 10, 1, NULL) └── f-k-checks └── f-k-checks-item: order(o_w_id,o_d_id,o_c_id) -> customer(c_w_id,c_d_id,c_id) └── anti-join (lookup customer) - ├── columns: column3:17!null column2:18!null column4:19!null - ├── key columns: [17 18 19] = [22 21 20] + ├── columns: column3:18!null column2:19!null column4:20!null + ├── key columns: [18 19 20] = [23 22 21] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(17-19) + ├── fd: ()-->(18-20) ├── with-scan &1 - │ ├── columns: column3:17!null column2:18!null column4:19!null + │ ├── columns: column3:18!null column2:19!null column4:20!null │ ├── mapping: - │ │ ├── column3:11 => column3:17 - │ │ ├── column2:10 => column2:18 - │ │ └── column4:12 => column4:19 + │ │ ├── column3:12 => column3:18 + │ │ ├── column2:11 => column2:19 + │ │ └── column4:13 => column4:20 │ ├── cardinality: [1 - 1] │ ├── key: () - │ └── fd: ()-->(17-19) + │ └── fd: ()-->(18-20) └── filters (true) opt format=hide-qual @@ -184,36 +184,36 @@ INSERT INTO new_order (no_o_id, no_d_id, no_w_id) VALUES (2000, 100, 10) insert new_order ├── columns: ├── insert-mapping: - │ ├── column1:4 => no_o_id:1 - │ ├── column2:5 => no_d_id:2 - │ └── column3:6 => no_w_id:3 + │ ├── column1:5 => no_o_id:1 + │ ├── column2:6 => no_d_id:2 + │ └── column3:7 => no_w_id:3 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── values - │ ├── columns: column1:4!null column2:5!null column3:6!null + │ ├── columns: column1:5!null column2:6!null column3:7!null │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(4-6) + │ ├── fd: ()-->(5-7) │ └── (2000, 100, 10) └── f-k-checks └── f-k-checks-item: new_order(no_w_id,no_d_id,no_o_id) -> order(o_w_id,o_d_id,o_id) └── anti-join (lookup order) - ├── columns: column3:7!null column2:8!null column1:9!null - ├── key columns: [7 8 9] = [12 11 10] + ├── columns: column3:8!null column2:9!null column1:10!null + ├── key columns: [8 9 10] = [13 12 11] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(7-9) + ├── fd: ()-->(8-10) ├── with-scan &1 - │ ├── columns: column3:7!null column2:8!null column1:9!null + │ ├── columns: column3:8!null column2:9!null column1:10!null │ ├── mapping: - │ │ ├── column3:6 => column3:7 - │ │ ├── column2:5 => column2:8 - │ │ └── column1:4 => column1:9 + │ │ ├── column3:7 => column3:8 + │ │ ├── column2:6 => column2:9 + │ │ └── column1:5 => column1:10 │ ├── cardinality: [1 - 1] │ ├── key: () - │ └── fd: ()-->(7-9) + │ └── fd: ()-->(8-10) └── filters (true) opt format=hide-qual @@ -305,23 +305,23 @@ WHERE ---- update stock ├── columns: - ├── fetch columns: s_i_id:18 s_w_id:19 s_quantity:20 s_dist_01:21 s_dist_02:22 s_dist_03:23 s_dist_04:24 s_dist_05:25 s_dist_06:26 s_dist_07:27 s_dist_08:28 s_dist_09:29 s_dist_10:30 s_ytd:31 s_order_cnt:32 s_remote_cnt:33 s_data:34 + ├── fetch columns: s_i_id:19 s_w_id:20 s_quantity:21 s_dist_01:22 s_dist_02:23 s_dist_03:24 s_dist_04:25 s_dist_05:26 s_dist_06:27 s_dist_07:28 s_dist_08:29 s_dist_09:30 s_dist_10:31 s_ytd:32 s_order_cnt:33 s_remote_cnt:34 s_data:35 ├── update-mapping: - │ ├── s_quantity_new:35 => s_quantity:3 - │ ├── s_ytd_new:36 => s_ytd:14 - │ ├── s_order_cnt_new:37 => s_order_cnt:15 - │ └── s_remote_cnt_new:38 => s_remote_cnt:16 + │ ├── s_quantity_new:37 => s_quantity:3 + │ ├── s_ytd_new:38 => s_ytd:14 + │ ├── s_order_cnt_new:39 => s_order_cnt:15 + │ └── s_remote_cnt_new:40 => s_remote_cnt:16 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: s_quantity_new:35 s_ytd_new:36 s_order_cnt_new:37 s_remote_cnt_new:38 s_i_id:18!null s_w_id:19!null s_quantity:20 s_dist_01:21 s_dist_02:22 s_dist_03:23 s_dist_04:24 s_dist_05:25 s_dist_06:26 s_dist_07:27 s_dist_08:28 s_dist_09:29 s_dist_10:30 s_ytd:31 s_order_cnt:32 s_remote_cnt:33 s_data:34 + ├── columns: s_quantity_new:37 s_ytd_new:38 s_order_cnt_new:39 s_remote_cnt_new:40 s_i_id:19!null s_w_id:20!null s_quantity:21 s_dist_01:22 s_dist_02:23 s_dist_03:24 s_dist_04:25 s_dist_05:26 s_dist_06:27 s_dist_07:28 s_dist_08:29 s_dist_09:30 s_dist_10:31 s_ytd:32 s_order_cnt:33 s_remote_cnt:34 s_data:35 ├── cardinality: [0 - 13] ├── volatile - ├── key: (18) - ├── fd: ()-->(19), (18)-->(20-38) + ├── key: (19) + ├── fd: ()-->(20), (19)-->(21-35,37-40) ├── scan stock - │ ├── columns: s_i_id:18!null s_w_id:19!null s_quantity:20 s_dist_01:21 s_dist_02:22 s_dist_03:23 s_dist_04:24 s_dist_05:25 s_dist_06:26 s_dist_07:27 s_dist_08:28 s_dist_09:29 s_dist_10:30 s_ytd:31 s_order_cnt:32 s_remote_cnt:33 s_data:34 - │ ├── constraint: /19/18 + │ ├── columns: s_i_id:19!null s_w_id:20!null s_quantity:21 s_dist_01:22 s_dist_02:23 s_dist_03:24 s_dist_04:25 s_dist_05:26 s_dist_06:27 s_dist_07:28 s_dist_08:29 s_dist_09:30 s_dist_10:31 s_ytd:32 s_order_cnt:33 s_remote_cnt:34 s_data:35 + │ ├── constraint: /20/19 │ │ ├── [/0/6823 - /0/6823] │ │ ├── [/0/7853 - /0/7853] │ │ ├── [/0/8497 - /0/8497] @@ -336,13 +336,13 @@ update stock │ │ ├── [/0/89641 - /0/89641] │ │ └── [/0/89905 - /0/89905] │ ├── cardinality: [0 - 13] - │ ├── key: (18) - │ └── fd: ()-->(19), (18)-->(20-34) + │ ├── key: (19) + │ └── fd: ()-->(20), (19)-->(21-35) └── projections - ├── CASE (s_i_id:18, s_w_id:19) WHEN (6823, 0) THEN 26 WHEN (7853, 0) THEN 10 WHEN (8497, 0) THEN 62 WHEN (10904, 0) THEN 54 WHEN (16152, 0) THEN 80 WHEN (41382, 0) THEN 18 WHEN (55952, 0) THEN 56 WHEN (64817, 0) THEN 26 WHEN (66335, 0) THEN 30 WHEN (76567, 0) THEN 71 WHEN (81680, 0) THEN 51 WHEN (89641, 0) THEN 51 WHEN (89905, 0) THEN 77 ELSE crdb_internal.force_error('', 'unknown case') END [as=s_quantity_new:35, outer=(18,19), volatile] - ├── CASE (s_i_id:18, s_w_id:19) WHEN (6823, 0) THEN 6 WHEN (7853, 0) THEN 9 WHEN (8497, 0) THEN 13 WHEN (10904, 0) THEN 1 WHEN (16152, 0) THEN 2 WHEN (41382, 0) THEN 3 WHEN (55952, 0) THEN 10 WHEN (64817, 0) THEN 31 WHEN (66335, 0) THEN 9 WHEN (76567, 0) THEN 7 WHEN (81680, 0) THEN 4 WHEN (89641, 0) THEN 13 WHEN (89905, 0) THEN 20 ELSE CAST(NULL AS INT8) END [as=s_ytd_new:36, outer=(18,19)] - ├── CASE (s_i_id:18, s_w_id:19) WHEN (6823, 0) THEN 1 WHEN (7853, 0) THEN 1 WHEN (8497, 0) THEN 2 WHEN (10904, 0) THEN 1 WHEN (16152, 0) THEN 1 WHEN (41382, 0) THEN 1 WHEN (55952, 0) THEN 1 WHEN (64817, 0) THEN 4 WHEN (66335, 0) THEN 2 WHEN (76567, 0) THEN 1 WHEN (81680, 0) THEN 1 WHEN (89641, 0) THEN 2 WHEN (89905, 0) THEN 4 ELSE CAST(NULL AS INT8) END [as=s_order_cnt_new:37, outer=(18,19)] - └── CASE (s_i_id:18, s_w_id:19) WHEN (6823, 0) THEN 0 WHEN (7853, 0) THEN 0 WHEN (8497, 0) THEN 0 WHEN (10904, 0) THEN 0 WHEN (16152, 0) THEN 0 WHEN (41382, 0) THEN 0 WHEN (55952, 0) THEN 0 WHEN (64817, 0) THEN 0 WHEN (66335, 0) THEN 0 WHEN (76567, 0) THEN 0 WHEN (81680, 0) THEN 0 WHEN (89641, 0) THEN 0 WHEN (89905, 0) THEN 0 ELSE CAST(NULL AS INT8) END [as=s_remote_cnt_new:38, outer=(18,19)] + ├── CASE (s_i_id:19, s_w_id:20) WHEN (6823, 0) THEN 26 WHEN (7853, 0) THEN 10 WHEN (8497, 0) THEN 62 WHEN (10904, 0) THEN 54 WHEN (16152, 0) THEN 80 WHEN (41382, 0) THEN 18 WHEN (55952, 0) THEN 56 WHEN (64817, 0) THEN 26 WHEN (66335, 0) THEN 30 WHEN (76567, 0) THEN 71 WHEN (81680, 0) THEN 51 WHEN (89641, 0) THEN 51 WHEN (89905, 0) THEN 77 ELSE crdb_internal.force_error('', 'unknown case') END [as=s_quantity_new:37, outer=(19,20), volatile] + ├── CASE (s_i_id:19, s_w_id:20) WHEN (6823, 0) THEN 6 WHEN (7853, 0) THEN 9 WHEN (8497, 0) THEN 13 WHEN (10904, 0) THEN 1 WHEN (16152, 0) THEN 2 WHEN (41382, 0) THEN 3 WHEN (55952, 0) THEN 10 WHEN (64817, 0) THEN 31 WHEN (66335, 0) THEN 9 WHEN (76567, 0) THEN 7 WHEN (81680, 0) THEN 4 WHEN (89641, 0) THEN 13 WHEN (89905, 0) THEN 20 ELSE CAST(NULL AS INT8) END [as=s_ytd_new:38, outer=(19,20)] + ├── CASE (s_i_id:19, s_w_id:20) WHEN (6823, 0) THEN 1 WHEN (7853, 0) THEN 1 WHEN (8497, 0) THEN 2 WHEN (10904, 0) THEN 1 WHEN (16152, 0) THEN 1 WHEN (41382, 0) THEN 1 WHEN (55952, 0) THEN 1 WHEN (64817, 0) THEN 4 WHEN (66335, 0) THEN 2 WHEN (76567, 0) THEN 1 WHEN (81680, 0) THEN 1 WHEN (89641, 0) THEN 2 WHEN (89905, 0) THEN 4 ELSE CAST(NULL AS INT8) END [as=s_order_cnt_new:39, outer=(19,20)] + └── CASE (s_i_id:19, s_w_id:20) WHEN (6823, 0) THEN 0 WHEN (7853, 0) THEN 0 WHEN (8497, 0) THEN 0 WHEN (10904, 0) THEN 0 WHEN (16152, 0) THEN 0 WHEN (41382, 0) THEN 0 WHEN (55952, 0) THEN 0 WHEN (64817, 0) THEN 0 WHEN (66335, 0) THEN 0 WHEN (76567, 0) THEN 0 WHEN (81680, 0) THEN 0 WHEN (89641, 0) THEN 0 WHEN (89905, 0) THEN 0 ELSE CAST(NULL AS INT8) END [as=s_remote_cnt_new:40, outer=(19,20)] opt format=hide-qual INSERT INTO order_line @@ -358,26 +358,26 @@ VALUES insert order_line ├── columns: ├── insert-mapping: - │ ├── column1:11 => ol_o_id:1 - │ ├── column2:12 => ol_d_id:2 - │ ├── column3:13 => ol_w_id:3 - │ ├── column4:14 => ol_number:4 - │ ├── column5:15 => ol_i_id:5 - │ ├── column6:16 => ol_supply_w_id:6 - │ ├── column20:20 => ol_delivery_d:7 - │ ├── column7:17 => ol_quantity:8 - │ ├── ol_amount:21 => order_line.ol_amount:9 - │ └── column9:19 => ol_dist_info:10 + │ ├── column1:12 => ol_o_id:1 + │ ├── column2:13 => ol_d_id:2 + │ ├── column3:14 => ol_w_id:3 + │ ├── column4:15 => ol_number:4 + │ ├── column5:16 => ol_i_id:5 + │ ├── column6:17 => ol_supply_w_id:6 + │ ├── column21:21 => ol_delivery_d:7 + │ ├── column7:18 => ol_quantity:8 + │ ├── ol_amount:22 => order_line.ol_amount:9 + │ └── column9:20 => ol_dist_info:10 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── project - │ ├── columns: ol_amount:21 column20:20 column1:11!null column2:12!null column3:13!null column4:14!null column5:15!null column6:16!null column7:17!null column9:19!null + │ ├── columns: ol_amount:22 column21:21 column1:12!null column2:13!null column3:14!null column4:15!null column5:16!null column6:17!null column7:18!null column9:20!null │ ├── cardinality: [6 - 6] │ ├── immutable - │ ├── fd: ()-->(20) + │ ├── fd: ()-->(21) │ ├── values - │ │ ├── columns: column1:11!null column2:12!null column3:13!null column4:14!null column5:15!null column6:16!null column7:17!null column8:18!null column9:19!null + │ │ ├── columns: column1:12!null column2:13!null column3:14!null column4:15!null column5:16!null column6:17!null column7:18!null column8:19!null column9:20!null │ │ ├── cardinality: [6 - 6] │ │ ├── (3045, 2, 10, 3, 648, 0, 9, 394.470000, 'YhgLRrwsmd68P2bElAgrnp8u') │ │ ├── (3045, 2, 10, 5, 25393, 0, 10, 830.600000, 'dLXe0YhgLRrwsmd68P2bElAg') @@ -386,34 +386,34 @@ insert order_line │ │ ├── (3045, 2, 10, 4, 56624, 0, 6, 273.360000, 'RsaCXoEzmssaF9m9cdLXe0Yh') │ │ └── (3045, 2, 10, 6, 92966, 0, 4, 366.760000, 'saCXoEzmssaF9m9cdLXe0Yhg') │ └── projections - │ ├── crdb_internal.round_decimal_values(column8:18, 2) [as=ol_amount:21, outer=(18), immutable] - │ └── CAST(NULL AS TIMESTAMP) [as=column20:20] + │ ├── crdb_internal.round_decimal_values(column8:19, 2) [as=ol_amount:22, outer=(19), immutable] + │ └── CAST(NULL AS TIMESTAMP) [as=column21:21] └── f-k-checks ├── f-k-checks-item: order_line(ol_w_id,ol_d_id,ol_o_id) -> order(o_w_id,o_d_id,o_id) │ └── anti-join (lookup order) - │ ├── columns: column3:22!null column2:23!null column1:24!null - │ ├── key columns: [22 23 24] = [27 26 25] + │ ├── columns: column3:23!null column2:24!null column1:25!null + │ ├── key columns: [23 24 25] = [28 27 26] │ ├── lookup columns are key │ ├── cardinality: [0 - 6] │ ├── with-scan &1 - │ │ ├── columns: column3:22!null column2:23!null column1:24!null + │ │ ├── columns: column3:23!null column2:24!null column1:25!null │ │ ├── mapping: - │ │ │ ├── column3:13 => column3:22 - │ │ │ ├── column2:12 => column2:23 - │ │ │ └── column1:11 => column1:24 + │ │ │ ├── column3:14 => column3:23 + │ │ │ ├── column2:13 => column2:24 + │ │ │ └── column1:12 => column1:25 │ │ └── cardinality: [6 - 6] │ └── filters (true) └── f-k-checks-item: order_line(ol_supply_w_id,ol_i_id) -> stock(s_w_id,s_i_id) └── anti-join (lookup stock@stock_item_fk_idx) - ├── columns: column6:33!null column5:34!null - ├── key columns: [34 33] = [35 36] + ├── columns: column6:35!null column5:36!null + ├── key columns: [36 35] = [37 38] ├── lookup columns are key ├── cardinality: [0 - 6] ├── with-scan &1 - │ ├── columns: column6:33!null column5:34!null + │ ├── columns: column6:35!null column5:36!null │ ├── mapping: - │ │ ├── column6:16 => column6:33 - │ │ └── column5:15 => column5:34 + │ │ ├── column6:17 => column6:35 + │ │ └── column5:16 => column5:36 │ └── cardinality: [6 - 6] └── filters (true) @@ -438,27 +438,27 @@ project ├── fd: ()-->(2-7) └── update warehouse ├── columns: w_id:1!null w_name:2 w_street_1:3 w_street_2:4 w_city:5 w_state:6 w_zip:7 - ├── fetch columns: w_id:10 w_name:11 w_street_1:12 w_street_2:13 w_city:14 w_state:15 w_zip:16 w_tax:17 warehouse.w_ytd:18 + ├── fetch columns: w_id:11 w_name:12 w_street_1:13 w_street_2:14 w_city:15 w_state:16 w_zip:17 w_tax:18 warehouse.w_ytd:19 ├── update-mapping: - │ └── w_ytd:20 => warehouse.w_ytd:9 + │ └── w_ytd:22 => warehouse.w_ytd:9 ├── cardinality: [0 - 1] ├── volatile, mutations ├── key: () ├── fd: ()-->(1-7) └── project - ├── columns: w_ytd:20 w_id:10!null w_name:11 w_street_1:12 w_street_2:13 w_city:14 w_state:15 w_zip:16 w_tax:17 warehouse.w_ytd:18 + ├── columns: w_ytd:22 w_id:11!null w_name:12 w_street_1:13 w_street_2:14 w_city:15 w_state:16 w_zip:17 w_tax:18 warehouse.w_ytd:19 ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(10-18,20) + ├── fd: ()-->(11-19,22) ├── scan warehouse - │ ├── columns: w_id:10!null w_name:11 w_street_1:12 w_street_2:13 w_city:14 w_state:15 w_zip:16 w_tax:17 warehouse.w_ytd:18 - │ ├── constraint: /10: [/10 - /10] + │ ├── columns: w_id:11!null w_name:12 w_street_1:13 w_street_2:14 w_city:15 w_state:16 w_zip:17 w_tax:18 warehouse.w_ytd:19 + │ ├── constraint: /11: [/10 - /10] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(10-18) + │ └── fd: ()-->(11-19) └── projections - └── crdb_internal.round_decimal_values(warehouse.w_ytd:18::DECIMAL + 3860.61, 2) [as=w_ytd:20, outer=(18), immutable] + └── crdb_internal.round_decimal_values(warehouse.w_ytd:19::DECIMAL + 3860.61, 2) [as=w_ytd:22, outer=(19), immutable] opt format=hide-qual UPDATE district SET d_ytd = d_ytd + 3860.61 WHERE (d_w_id = 10) AND (d_id = 5) @@ -472,27 +472,27 @@ project ├── fd: ()-->(3-8) └── update district ├── columns: d_id:1!null d_w_id:2!null d_name:3 d_street_1:4 d_street_2:5 d_city:6 d_state:7 d_zip:8 - ├── fetch columns: d_id:12 d_w_id:13 d_name:14 d_street_1:15 d_street_2:16 d_city:17 d_state:18 d_zip:19 d_tax:20 district.d_ytd:21 d_next_o_id:22 + ├── fetch columns: d_id:13 d_w_id:14 d_name:15 d_street_1:16 d_street_2:17 d_city:18 d_state:19 d_zip:20 d_tax:21 district.d_ytd:22 d_next_o_id:23 ├── update-mapping: - │ └── d_ytd:24 => district.d_ytd:10 + │ └── d_ytd:26 => district.d_ytd:10 ├── cardinality: [0 - 1] ├── volatile, mutations ├── key: () ├── fd: ()-->(1-8) └── project - ├── columns: d_ytd:24 d_id:12!null d_w_id:13!null d_name:14 d_street_1:15 d_street_2:16 d_city:17 d_state:18 d_zip:19 d_tax:20 district.d_ytd:21 d_next_o_id:22 + ├── columns: d_ytd:26 d_id:13!null d_w_id:14!null d_name:15 d_street_1:16 d_street_2:17 d_city:18 d_state:19 d_zip:20 d_tax:21 district.d_ytd:22 d_next_o_id:23 ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(12-22,24) + ├── fd: ()-->(13-23,26) ├── scan district - │ ├── columns: d_id:12!null d_w_id:13!null d_name:14 d_street_1:15 d_street_2:16 d_city:17 d_state:18 d_zip:19 d_tax:20 district.d_ytd:21 d_next_o_id:22 - │ ├── constraint: /13/12: [/10/5 - /10/5] + │ ├── columns: d_id:13!null d_w_id:14!null d_name:15 d_street_1:16 d_street_2:17 d_city:18 d_state:19 d_zip:20 d_tax:21 district.d_ytd:22 d_next_o_id:23 + │ ├── constraint: /14/13: [/10/5 - /10/5] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(12-22) + │ └── fd: ()-->(13-23) └── projections - └── crdb_internal.round_decimal_values(district.d_ytd:21::DECIMAL + 3860.61, 2) [as=d_ytd:24, outer=(21), immutable] + └── crdb_internal.round_decimal_values(district.d_ytd:22::DECIMAL + 3860.61, 2) [as=d_ytd:26, outer=(22), immutable] opt format=hide-qual SELECT c_id @@ -554,42 +554,42 @@ RETURNING CASE c_credit WHEN 'BC' THEN "left"(c_data, 200) ELSE '' END ---- project - ├── columns: c_first:4 c_middle:5 c_last:6 c_street_1:7 c_street_2:8 c_city:9 c_state:10 c_zip:11 c_phone:12 c_since:13 c_credit:14 c_credit_lim:15 c_discount:16 c_balance:17 case:49 + ├── columns: c_first:4 c_middle:5 c_last:6 c_street_1:7 c_street_2:8 c_city:9 c_state:10 c_zip:11 c_phone:12 c_since:13 c_credit:14 c_credit_lim:15 c_discount:16 c_balance:17 case:51 ├── cardinality: [0 - 1] ├── volatile, mutations ├── key: () - ├── fd: ()-->(4-17,49) + ├── fd: ()-->(4-17,51) ├── update customer │ ├── columns: c_id:1!null c_d_id:2!null c_w_id:3!null c_first:4 c_middle:5 c_last:6 c_street_1:7 c_street_2:8 c_city:9 c_state:10 c_zip:11 c_phone:12 c_since:13 c_credit:14 c_credit_lim:15 c_discount:16 customer.c_balance:17 c_data:21 - │ ├── fetch columns: c_id:22 c_d_id:23 c_w_id:24 c_first:25 c_middle:26 c_last:27 c_street_1:28 c_street_2:29 c_city:30 c_state:31 c_zip:32 c_phone:33 c_since:34 c_credit:35 c_credit_lim:36 c_discount:37 customer.c_balance:38 customer.c_ytd_payment:39 c_payment_cnt:40 c_delivery_cnt:41 c_data:42 + │ ├── fetch columns: c_id:23 c_d_id:24 c_w_id:25 c_first:26 c_middle:27 c_last:28 c_street_1:29 c_street_2:30 c_city:31 c_state:32 c_zip:33 c_phone:34 c_since:35 c_credit:36 c_credit_lim:37 c_discount:38 customer.c_balance:39 customer.c_ytd_payment:40 c_payment_cnt:41 c_delivery_cnt:42 c_data:43 │ ├── update-mapping: - │ │ ├── c_balance:47 => customer.c_balance:17 - │ │ ├── c_ytd_payment:48 => customer.c_ytd_payment:18 - │ │ ├── c_payment_cnt_new:45 => c_payment_cnt:19 - │ │ └── c_data_new:46 => c_data:21 + │ │ ├── c_balance:49 => customer.c_balance:17 + │ │ ├── c_ytd_payment:50 => customer.c_ytd_payment:18 + │ │ ├── c_payment_cnt_new:47 => c_payment_cnt:19 + │ │ └── c_data_new:48 => c_data:21 │ ├── cardinality: [0 - 1] │ ├── volatile, mutations │ ├── key: () │ ├── fd: ()-->(1-17,21) │ └── project - │ ├── columns: c_balance:47 c_ytd_payment:48 c_payment_cnt_new:45 c_data_new:46 c_id:22!null c_d_id:23!null c_w_id:24!null c_first:25 c_middle:26 c_last:27 c_street_1:28 c_street_2:29 c_city:30 c_state:31 c_zip:32 c_phone:33 c_since:34 c_credit:35 c_credit_lim:36 c_discount:37 customer.c_balance:38 customer.c_ytd_payment:39 c_payment_cnt:40 c_delivery_cnt:41 c_data:42 + │ ├── columns: c_balance:49 c_ytd_payment:50 c_payment_cnt_new:47 c_data_new:48 c_id:23!null c_d_id:24!null c_w_id:25!null c_first:26 c_middle:27 c_last:28 c_street_1:29 c_street_2:30 c_city:31 c_state:32 c_zip:33 c_phone:34 c_since:35 c_credit:36 c_credit_lim:37 c_discount:38 customer.c_balance:39 customer.c_ytd_payment:40 c_payment_cnt:41 c_delivery_cnt:42 c_data:43 │ ├── cardinality: [0 - 1] │ ├── immutable │ ├── key: () - │ ├── fd: ()-->(22-42,45-48) + │ ├── fd: ()-->(23-43,47-50) │ ├── scan customer - │ │ ├── columns: c_id:22!null c_d_id:23!null c_w_id:24!null c_first:25 c_middle:26 c_last:27 c_street_1:28 c_street_2:29 c_city:30 c_state:31 c_zip:32 c_phone:33 c_since:34 c_credit:35 c_credit_lim:36 c_discount:37 customer.c_balance:38 customer.c_ytd_payment:39 c_payment_cnt:40 c_delivery_cnt:41 c_data:42 - │ │ ├── constraint: /24/23/22: [/10/5/1343 - /10/5/1343] + │ │ ├── columns: c_id:23!null c_d_id:24!null c_w_id:25!null c_first:26 c_middle:27 c_last:28 c_street_1:29 c_street_2:30 c_city:31 c_state:32 c_zip:33 c_phone:34 c_since:35 c_credit:36 c_credit_lim:37 c_discount:38 customer.c_balance:39 customer.c_ytd_payment:40 c_payment_cnt:41 c_delivery_cnt:42 c_data:43 + │ │ ├── constraint: /25/24/23: [/10/5/1343 - /10/5/1343] │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(22-42) + │ │ └── fd: ()-->(23-43) │ └── projections - │ ├── crdb_internal.round_decimal_values(customer.c_balance:38::DECIMAL - 3860.61, 2) [as=c_balance:47, outer=(38), immutable] - │ ├── crdb_internal.round_decimal_values(customer.c_ytd_payment:39::DECIMAL + 3860.61, 2) [as=c_ytd_payment:48, outer=(39), immutable] - │ ├── c_payment_cnt:40 + 1 [as=c_payment_cnt_new:45, outer=(40), immutable] - │ └── CASE c_credit:35 WHEN 'BC' THEN left((((((c_id:22::STRING || c_d_id:23::STRING) || c_w_id:24::STRING) || '5') || '10') || '3860.61') || c_data:42::STRING, 500) ELSE c_data:42::STRING END [as=c_data_new:46, outer=(22-24,35,42), immutable] + │ ├── crdb_internal.round_decimal_values(customer.c_balance:39::DECIMAL - 3860.61, 2) [as=c_balance:49, outer=(39), immutable] + │ ├── crdb_internal.round_decimal_values(customer.c_ytd_payment:40::DECIMAL + 3860.61, 2) [as=c_ytd_payment:50, outer=(40), immutable] + │ ├── c_payment_cnt:41 + 1 [as=c_payment_cnt_new:47, outer=(41), immutable] + │ └── CASE c_credit:36 WHEN 'BC' THEN left((((((c_id:23::STRING || c_d_id:24::STRING) || c_w_id:25::STRING) || '5') || '10') || '3860.61') || c_data:43::STRING, 500) ELSE c_data:43::STRING END [as=c_data_new:48, outer=(23-25,36,43), immutable] └── projections - └── CASE c_credit:14 WHEN 'BC' THEN left(c_data:21, 200) ELSE '' END [as=case:49, outer=(14,21), immutable] + └── CASE c_credit:14 WHEN 'BC' THEN left(c_data:21, 200) ELSE '' END [as=case:51, outer=(14,21), immutable] opt format=hide-qual INSERT INTO history @@ -600,60 +600,60 @@ VALUES insert history ├── columns: ├── insert-mapping: - │ ├── column18:18 => rowid:1 - │ ├── column1:10 => h_c_id:2 - │ ├── column2:11 => h_c_d_id:3 - │ ├── column3:12 => h_c_w_id:4 - │ ├── column4:13 => h_d_id:5 - │ ├── column5:14 => h_w_id:6 - │ ├── column7:16 => h_date:7 - │ ├── h_amount:19 => history.h_amount:8 - │ └── column8:17 => h_data:9 + │ ├── column19:19 => rowid:1 + │ ├── column1:11 => h_c_id:2 + │ ├── column2:12 => h_c_d_id:3 + │ ├── column3:13 => h_c_w_id:4 + │ ├── column4:14 => h_d_id:5 + │ ├── column5:15 => h_w_id:6 + │ ├── column7:17 => h_date:7 + │ ├── h_amount:20 => history.h_amount:8 + │ └── column8:18 => h_data:9 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── values - │ ├── columns: column1:10!null column2:11!null column3:12!null column4:13!null column5:14!null column7:16!null column8:17!null column18:18 h_amount:19!null + │ ├── columns: column1:11!null column2:12!null column3:13!null column4:14!null column5:15!null column7:17!null column8:18!null column19:19 h_amount:20!null │ ├── cardinality: [1 - 1] │ ├── volatile │ ├── key: () - │ ├── fd: ()-->(10-14,16-19) + │ ├── fd: ()-->(11-15,17-20) │ └── (1343, 5, 10, 5, 10, '2019-08-26 16:50:41+00:00', '8 Kdcgphy3', gen_random_uuid(), 3860.61) └── f-k-checks ├── f-k-checks-item: history(h_c_w_id,h_c_d_id,h_c_id) -> customer(c_w_id,c_d_id,c_id) │ └── anti-join (lookup customer) - │ ├── columns: column3:20!null column2:21!null column1:22!null - │ ├── key columns: [20 21 22] = [25 24 23] + │ ├── columns: column3:21!null column2:22!null column1:23!null + │ ├── key columns: [21 22 23] = [26 25 24] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(20-22) + │ ├── fd: ()-->(21-23) │ ├── with-scan &1 - │ │ ├── columns: column3:20!null column2:21!null column1:22!null + │ │ ├── columns: column3:21!null column2:22!null column1:23!null │ │ ├── mapping: - │ │ │ ├── column3:12 => column3:20 - │ │ │ ├── column2:11 => column2:21 - │ │ │ └── column1:10 => column1:22 + │ │ │ ├── column3:13 => column3:21 + │ │ │ ├── column2:12 => column2:22 + │ │ │ └── column1:11 => column1:23 │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(20-22) + │ │ └── fd: ()-->(21-23) │ └── filters (true) └── f-k-checks-item: history(h_w_id,h_d_id) -> district(d_w_id,d_id) └── anti-join (lookup district) - ├── columns: column5:44!null column4:45!null - ├── key columns: [44 45] = [47 46] + ├── columns: column5:46!null column4:47!null + ├── key columns: [46 47] = [49 48] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(44,45) + ├── fd: ()-->(46,47) ├── with-scan &1 - │ ├── columns: column5:44!null column4:45!null + │ ├── columns: column5:46!null column4:47!null │ ├── mapping: - │ │ ├── column5:14 => column5:44 - │ │ └── column4:13 => column4:45 + │ │ ├── column5:15 => column5:46 + │ │ └── column4:14 => column4:47 │ ├── cardinality: [1 - 1] │ ├── key: () - │ └── fd: ()-->(44,45) + │ └── fd: ()-->(46,47) └── filters (true) # -------------------------------------------------- @@ -777,16 +777,16 @@ FROM order_line WHERE ol_w_id = 10 AND ol_d_id = 100 AND ol_o_id = 1000 ---- scalar-group-by - ├── columns: sum:11 + ├── columns: sum:12 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(11) + ├── fd: ()-->(12) ├── scan order_line │ ├── columns: ol_o_id:1!null ol_d_id:2!null ol_w_id:3!null ol_amount:9 │ ├── constraint: /3/2/-1/4: [/10/100/1000 - /10/100/1000] │ └── fd: ()-->(1-3) └── aggregations - └── sum [as=sum:11, outer=(9)] + └── sum [as=sum:12, outer=(9)] └── ol_amount:9 opt format=hide-qual @@ -816,21 +816,21 @@ project ├── fd: (2)-->(4) └── update "order" ├── columns: o_id:1!null o_d_id:2!null o_w_id:3!null o_c_id:4 - ├── fetch columns: o_id:9 o_d_id:10 o_w_id:11 o_c_id:12 o_entry_d:13 o_carrier_id:14 o_ol_cnt:15 o_all_local:16 + ├── fetch columns: o_id:10 o_d_id:11 o_w_id:12 o_c_id:13 o_entry_d:14 o_carrier_id:15 o_ol_cnt:16 o_all_local:17 ├── update-mapping: - │ └── o_carrier_id_new:17 => o_carrier_id:6 + │ └── o_carrier_id_new:19 => o_carrier_id:6 ├── cardinality: [0 - 10] ├── volatile, mutations ├── key: (2) ├── fd: ()-->(1,3), (2)-->(4) └── project - ├── columns: o_carrier_id_new:17!null o_id:9!null o_d_id:10!null o_w_id:11!null o_c_id:12 o_entry_d:13 o_carrier_id:14 o_ol_cnt:15 o_all_local:16 + ├── columns: o_carrier_id_new:19!null o_id:10!null o_d_id:11!null o_w_id:12!null o_c_id:13 o_entry_d:14 o_carrier_id:15 o_ol_cnt:16 o_all_local:17 ├── cardinality: [0 - 10] - ├── key: (10) - ├── fd: ()-->(9,11,17), (10)-->(12-16) + ├── key: (11) + ├── fd: ()-->(10,12,19), (11)-->(13-17) ├── scan "order" - │ ├── columns: o_id:9!null o_d_id:10!null o_w_id:11!null o_c_id:12 o_entry_d:13 o_carrier_id:14 o_ol_cnt:15 o_all_local:16 - │ ├── constraint: /11/10/-9 + │ ├── columns: o_id:10!null o_d_id:11!null o_w_id:12!null o_c_id:13 o_entry_d:14 o_carrier_id:15 o_ol_cnt:16 o_all_local:17 + │ ├── constraint: /12/11/-10 │ │ ├── [/10/1/2167 - /10/1/2167] │ │ ├── [/10/2/2167 - /10/2/2167] │ │ ├── [/10/3/2167 - /10/3/2167] @@ -842,10 +842,10 @@ project │ │ ├── [/10/9/2167 - /10/9/2167] │ │ └── [/10/10/2167 - /10/10/2167] │ ├── cardinality: [0 - 10] - │ ├── key: (10) - │ └── fd: ()-->(9,11), (10)-->(12-16) + │ ├── key: (11) + │ └── fd: ()-->(10,12), (11)-->(13-17) └── projections - └── 10 [as=o_carrier_id_new:17] + └── 10 [as=o_carrier_id_new:19] opt format=hide-qual UPDATE customer @@ -877,21 +877,21 @@ WHERE c_w_id = 10 AND (c_d_id, c_id) IN ( ---- update customer ├── columns: - ├── fetch columns: c_id:22 c_d_id:23 c_w_id:24 c_first:25 c_middle:26 c_last:27 c_street_1:28 c_street_2:29 c_city:30 c_state:31 c_zip:32 c_phone:33 c_since:34 c_credit:35 c_credit_lim:36 c_discount:37 customer.c_balance:38 c_ytd_payment:39 c_payment_cnt:40 c_delivery_cnt:41 c_data:42 + ├── fetch columns: c_id:23 c_d_id:24 c_w_id:25 c_first:26 c_middle:27 c_last:28 c_street_1:29 c_street_2:30 c_city:31 c_state:32 c_zip:33 c_phone:34 c_since:35 c_credit:36 c_credit_lim:37 c_discount:38 customer.c_balance:39 c_ytd_payment:40 c_payment_cnt:41 c_delivery_cnt:42 c_data:43 ├── update-mapping: - │ ├── c_balance:45 => customer.c_balance:17 - │ └── c_delivery_cnt_new:43 => c_delivery_cnt:20 + │ ├── c_balance:47 => customer.c_balance:17 + │ └── c_delivery_cnt_new:45 => c_delivery_cnt:20 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: c_balance:45 c_delivery_cnt_new:43 c_id:22!null c_d_id:23!null c_w_id:24!null c_first:25 c_middle:26 c_last:27 c_street_1:28 c_street_2:29 c_city:30 c_state:31 c_zip:32 c_phone:33 c_since:34 c_credit:35 c_credit_lim:36 c_discount:37 customer.c_balance:38 c_ytd_payment:39 c_payment_cnt:40 c_delivery_cnt:41 c_data:42 + ├── columns: c_balance:47 c_delivery_cnt_new:45 c_id:23!null c_d_id:24!null c_w_id:25!null c_first:26 c_middle:27 c_last:28 c_street_1:29 c_street_2:30 c_city:31 c_state:32 c_zip:33 c_phone:34 c_since:35 c_credit:36 c_credit_lim:37 c_discount:38 customer.c_balance:39 c_ytd_payment:40 c_payment_cnt:41 c_delivery_cnt:42 c_data:43 ├── cardinality: [0 - 10] ├── immutable - ├── key: (22,23) - ├── fd: ()-->(24), (22,23)-->(25-42,45), (41)-->(43) + ├── key: (23,24) + ├── fd: ()-->(25), (23,24)-->(26-43,47), (42)-->(45) ├── scan customer - │ ├── columns: c_id:22!null c_d_id:23!null c_w_id:24!null c_first:25 c_middle:26 c_last:27 c_street_1:28 c_street_2:29 c_city:30 c_state:31 c_zip:32 c_phone:33 c_since:34 c_credit:35 c_credit_lim:36 c_discount:37 customer.c_balance:38 c_ytd_payment:39 c_payment_cnt:40 c_delivery_cnt:41 c_data:42 - │ ├── constraint: /24/23/22 + │ ├── columns: c_id:23!null c_d_id:24!null c_w_id:25!null c_first:26 c_middle:27 c_last:28 c_street_1:29 c_street_2:30 c_city:31 c_state:32 c_zip:33 c_phone:34 c_since:35 c_credit:36 c_credit_lim:37 c_discount:38 customer.c_balance:39 c_ytd_payment:40 c_payment_cnt:41 c_delivery_cnt:42 c_data:43 + │ ├── constraint: /25/24/23 │ │ ├── [/10/1/1405 - /10/1/1405] │ │ ├── [/10/2/137 - /10/2/137] │ │ ├── [/10/3/309 - /10/3/309] @@ -903,11 +903,11 @@ update customer │ │ ├── [/10/9/1412 - /10/9/1412] │ │ └── [/10/10/417 - /10/10/417] │ ├── cardinality: [0 - 10] - │ ├── key: (22,23) - │ └── fd: ()-->(24), (22,23)-->(25-42) + │ ├── key: (23,24) + │ └── fd: ()-->(25), (23,24)-->(26-43) └── projections - ├── crdb_internal.round_decimal_values(customer.c_balance:38::DECIMAL + CASE c_d_id:23 WHEN 6 THEN 57214.780000 WHEN 8 THEN 67755.430000 WHEN 1 THEN 51177.840000 WHEN 2 THEN 73840.700000 WHEN 4 THEN 45906.990000 WHEN 9 THEN 32523.760000 WHEN 10 THEN 20240.200000 WHEN 3 THEN 75299.790000 WHEN 5 THEN 56543.340000 WHEN 7 THEN 67157.940000 ELSE CAST(NULL AS DECIMAL) END, 2) [as=c_balance:45, outer=(23,38), immutable] - └── c_delivery_cnt:41 + 1 [as=c_delivery_cnt_new:43, outer=(41), immutable] + ├── crdb_internal.round_decimal_values(customer.c_balance:39::DECIMAL + CASE c_d_id:24 WHEN 6 THEN 57214.780000 WHEN 8 THEN 67755.430000 WHEN 1 THEN 51177.840000 WHEN 2 THEN 73840.700000 WHEN 4 THEN 45906.990000 WHEN 9 THEN 32523.760000 WHEN 10 THEN 20240.200000 WHEN 3 THEN 75299.790000 WHEN 5 THEN 56543.340000 WHEN 7 THEN 67157.940000 ELSE CAST(NULL AS DECIMAL) END, 2) [as=c_balance:47, outer=(24,39), immutable] + └── c_delivery_cnt:42 + 1 [as=c_delivery_cnt_new:45, outer=(42), immutable] opt format=hide-qual DELETE FROM new_order @@ -926,12 +926,12 @@ WHERE no_w_id = 10 AND (no_d_id, no_o_id) IN ( ---- delete new_order ├── columns: - ├── fetch columns: no_o_id:4 no_d_id:5 no_w_id:6 + ├── fetch columns: no_o_id:5 no_d_id:6 no_w_id:7 ├── cardinality: [0 - 0] ├── volatile, mutations └── scan new_order - ├── columns: no_o_id:4!null no_d_id:5!null no_w_id:6!null - ├── constraint: /6/5/4 + ├── columns: no_o_id:5!null no_d_id:6!null no_w_id:7!null + ├── constraint: /7/6/5 │ ├── [/10/1/2167 - /10/1/2167] │ ├── [/10/2/2167 - /10/2/2167] │ ├── [/10/3/2167 - /10/3/2167] @@ -943,8 +943,8 @@ delete new_order │ ├── [/10/9/2167 - /10/9/2167] │ └── [/10/10/2167 - /10/10/2167] ├── cardinality: [0 - 10] - ├── key: (5) - └── fd: ()-->(4,6) + ├── key: (6) + └── fd: ()-->(5,7) opt format=hide-qual @@ -965,18 +965,18 @@ WHERE ol_w_id = 10 AND (ol_d_id, ol_o_id) IN ( ---- update order_line ├── columns: - ├── fetch columns: ol_o_id:11 ol_d_id:12 ol_w_id:13 ol_number:14 ol_i_id:15 ol_supply_w_id:16 ol_delivery_d:17 ol_quantity:18 ol_amount:19 ol_dist_info:20 + ├── fetch columns: ol_o_id:12 ol_d_id:13 ol_w_id:14 ol_number:15 ol_i_id:16 ol_supply_w_id:17 ol_delivery_d:18 ol_quantity:19 ol_amount:20 ol_dist_info:21 ├── update-mapping: - │ └── ol_delivery_d_new:21 => ol_delivery_d:7 + │ └── ol_delivery_d_new:23 => ol_delivery_d:7 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: ol_delivery_d_new:21!null ol_o_id:11!null ol_d_id:12!null ol_w_id:13!null ol_number:14!null ol_i_id:15!null ol_supply_w_id:16 ol_delivery_d:17 ol_quantity:18 ol_amount:19 ol_dist_info:20 - ├── key: (12,14) - ├── fd: ()-->(11,13,21), (12,14)-->(15-20) + ├── columns: ol_delivery_d_new:23!null ol_o_id:12!null ol_d_id:13!null ol_w_id:14!null ol_number:15!null ol_i_id:16!null ol_supply_w_id:17 ol_delivery_d:18 ol_quantity:19 ol_amount:20 ol_dist_info:21 + ├── key: (13,15) + ├── fd: ()-->(12,14,23), (13,15)-->(16-21) ├── scan order_line - │ ├── columns: ol_o_id:11!null ol_d_id:12!null ol_w_id:13!null ol_number:14!null ol_i_id:15!null ol_supply_w_id:16 ol_delivery_d:17 ol_quantity:18 ol_amount:19 ol_dist_info:20 - │ ├── constraint: /13/12/-11/14 + │ ├── columns: ol_o_id:12!null ol_d_id:13!null ol_w_id:14!null ol_number:15!null ol_i_id:16!null ol_supply_w_id:17 ol_delivery_d:18 ol_quantity:19 ol_amount:20 ol_dist_info:21 + │ ├── constraint: /14/13/-12/15 │ │ ├── [/10/1/2167 - /10/1/2167] │ │ ├── [/10/2/2167 - /10/2/2167] │ │ ├── [/10/3/2167 - /10/3/2167] @@ -987,10 +987,10 @@ update order_line │ │ ├── [/10/8/2167 - /10/8/2167] │ │ ├── [/10/9/2167 - /10/9/2167] │ │ └── [/10/10/2167 - /10/10/2167] - │ ├── key: (12,14) - │ └── fd: ()-->(11,13), (12,14)-->(15-20) + │ ├── key: (13,15) + │ └── fd: ()-->(12,14), (13,15)-->(16-21) └── projections - └── '2019-08-26 16:50:41+00:00' [as=ol_delivery_d_new:21] + └── '2019-08-26 16:50:41+00:00' [as=ol_delivery_d_new:23] # -------------------------------------------------- # 2.8 The Stock-Level Transaction @@ -1028,28 +1028,28 @@ WHERE ol_w_id = 10 AND s_quantity < 15 ---- scalar-group-by - ├── columns: count:28!null + ├── columns: count:30!null ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(28) + ├── fd: ()-->(30) ├── distinct-on - │ ├── columns: s_i_id:11!null - │ ├── grouping columns: s_i_id:11!null - │ ├── key: (11) + │ ├── columns: s_i_id:12!null + │ ├── grouping columns: s_i_id:12!null + │ ├── key: (12) │ └── inner-join (lookup stock) - │ ├── columns: ol_o_id:1!null ol_d_id:2!null ol_w_id:3!null ol_i_id:5!null s_i_id:11!null s_w_id:12!null s_quantity:13!null - │ ├── key columns: [3 5] = [12 11] + │ ├── columns: ol_o_id:1!null ol_d_id:2!null ol_w_id:3!null ol_i_id:5!null s_i_id:12!null s_w_id:13!null s_quantity:14!null + │ ├── key columns: [3 5] = [13 12] │ ├── lookup columns are key - │ ├── fd: ()-->(2,3,12), (11)-->(13), (5)==(11), (11)==(5), (3)==(12), (12)==(3) + │ ├── fd: ()-->(2,3,13), (12)-->(14), (5)==(12), (12)==(5), (3)==(13), (13)==(3) │ ├── scan order_line │ │ ├── columns: ol_o_id:1!null ol_d_id:2!null ol_w_id:3!null ol_i_id:5!null │ │ ├── constraint: /3/2/-1/4: [/10/100/999 - /10/100/980] │ │ └── fd: ()-->(2,3) │ └── filters - │ ├── s_w_id:12 = 10 [outer=(12), constraints=(/12: [/10 - /10]; tight), fd=()-->(12)] - │ └── s_quantity:13 < 15 [outer=(13), constraints=(/13: (/NULL - /14]; tight)] + │ ├── s_w_id:13 = 10 [outer=(13), constraints=(/13: [/10 - /10]; tight), fd=()-->(13)] + │ └── s_quantity:14 < 15 [outer=(14), constraints=(/14: (/NULL - /14]; tight)] └── aggregations - └── count-rows [as=count:28] + └── count-rows [as=count:30] # -------------------------------------------------- # Consistency Queries @@ -1070,39 +1070,39 @@ ON (w_id = d_w_id) WHERE w_ytd != sum_d_ytd ---- scalar-group-by - ├── columns: count:22!null + ├── columns: count:24!null ├── cardinality: [1 - 1] ├── immutable ├── key: () - ├── fd: ()-->(22) + ├── fd: ()-->(24) ├── inner-join (merge) - │ ├── columns: w_id:1!null w_ytd:9!null d_w_id:11!null sum:21!null + │ ├── columns: w_id:1!null w_ytd:9!null d_w_id:12!null sum:23!null │ ├── left ordering: +1 - │ ├── right ordering: +11 + │ ├── right ordering: +12 │ ├── immutable - │ ├── key: (11) - │ ├── fd: (1)-->(9), (11)-->(21), (1)==(11), (11)==(1) + │ ├── key: (12) + │ ├── fd: (1)-->(9), (12)-->(23), (1)==(12), (12)==(1) │ ├── scan warehouse │ │ ├── columns: w_id:1!null w_ytd:9 │ │ ├── key: (1) │ │ ├── fd: (1)-->(9) │ │ └── ordering: +1 │ ├── group-by - │ │ ├── columns: d_w_id:11!null sum:21 - │ │ ├── grouping columns: d_w_id:11!null - │ │ ├── key: (11) - │ │ ├── fd: (11)-->(21) - │ │ ├── ordering: +11 + │ │ ├── columns: d_w_id:12!null sum:23 + │ │ ├── grouping columns: d_w_id:12!null + │ │ ├── key: (12) + │ │ ├── fd: (12)-->(23) + │ │ ├── ordering: +12 │ │ ├── scan district - │ │ │ ├── columns: d_w_id:11!null d_ytd:19 - │ │ │ └── ordering: +11 + │ │ │ ├── columns: d_w_id:12!null d_ytd:20 + │ │ │ └── ordering: +12 │ │ └── aggregations - │ │ └── sum [as=sum:21, outer=(19)] - │ │ └── d_ytd:19 + │ │ └── sum [as=sum:23, outer=(20)] + │ │ └── d_ytd:20 │ └── filters - │ └── w_ytd:9 != sum:21 [outer=(9,21), immutable, constraints=(/9: (/NULL - ]; /21: (/NULL - ])] + │ └── w_ytd:9 != sum:23 [outer=(9,23), immutable, constraints=(/9: (/NULL - ]; /23: (/NULL - ])] └── aggregations - └── count-rows [as=count_rows:22] + └── count-rows [as=count_rows:24] opt format=hide-qual SELECT d_next_o_id @@ -1122,17 +1122,17 @@ GROUP BY no_d_id, no_w_id ORDER BY no_w_id, no_d_id ---- group-by - ├── columns: max:4!null [hidden: no_d_id:2!null no_w_id:3!null] + ├── columns: max:5!null [hidden: no_d_id:2!null no_w_id:3!null] ├── grouping columns: no_d_id:2!null no_w_id:3!null ├── key: (2,3) - ├── fd: (2,3)-->(4) + ├── fd: (2,3)-->(5) ├── ordering: +3,+2 ├── scan new_order │ ├── columns: no_o_id:1!null no_d_id:2!null no_w_id:3!null │ ├── key: (1-3) │ └── ordering: +3,+2 └── aggregations - └── max [as=max:4, outer=(1)] + └── max [as=max:5, outer=(1)] └── no_o_id:1 opt format=hide-qual @@ -1142,17 +1142,17 @@ GROUP BY o_d_id, o_w_id ORDER BY o_w_id, o_d_id ---- group-by - ├── columns: max:9!null [hidden: o_d_id:2!null o_w_id:3!null] + ├── columns: max:10!null [hidden: o_d_id:2!null o_w_id:3!null] ├── grouping columns: o_d_id:2!null o_w_id:3!null ├── key: (2,3) - ├── fd: (2,3)-->(9) + ├── fd: (2,3)-->(10) ├── ordering: +3,+2 ├── scan "order"@order_idx │ ├── columns: o_id:1!null o_d_id:2!null o_w_id:3!null │ ├── key: (1-3) │ └── ordering: +3,+2 └── aggregations - └── max [as=max:9, outer=(1)] + └── max [as=max:10, outer=(1)] └── o_id:1 opt format=hide-qual @@ -1166,36 +1166,36 @@ FROM WHERE nod != -1 ---- scalar-group-by - ├── columns: count:8!null + ├── columns: count:9!null ├── cardinality: [1 - 1] ├── immutable ├── key: () - ├── fd: ()-->(8) + ├── fd: ()-->(9) ├── select - │ ├── columns: no_d_id:2!null no_w_id:3!null max:4!null min:5!null count_rows:6!null + │ ├── columns: no_d_id:2!null no_w_id:3!null max:5!null min:6!null count_rows:7!null │ ├── immutable │ ├── key: (2,3) - │ ├── fd: (2,3)-->(4-6) + │ ├── fd: (2,3)-->(5-7) │ ├── group-by - │ │ ├── columns: no_d_id:2!null no_w_id:3!null max:4!null min:5!null count_rows:6!null + │ │ ├── columns: no_d_id:2!null no_w_id:3!null max:5!null min:6!null count_rows:7!null │ │ ├── grouping columns: no_d_id:2!null no_w_id:3!null │ │ ├── internal-ordering: +3,+2 │ │ ├── key: (2,3) - │ │ ├── fd: (2,3)-->(4-6) + │ │ ├── fd: (2,3)-->(5-7) │ │ ├── scan new_order │ │ │ ├── columns: no_o_id:1!null no_d_id:2!null no_w_id:3!null │ │ │ ├── key: (1-3) │ │ │ └── ordering: +3,+2 │ │ └── aggregations - │ │ ├── max [as=max:4, outer=(1)] + │ │ ├── max [as=max:5, outer=(1)] │ │ │ └── no_o_id:1 - │ │ ├── min [as=min:5, outer=(1)] + │ │ ├── min [as=min:6, outer=(1)] │ │ │ └── no_o_id:1 - │ │ └── count-rows [as=count_rows:6] + │ │ └── count-rows [as=count_rows:7] │ └── filters - │ └── ((max:4 - min:5) - count_rows:6) != -1 [outer=(4-6), immutable] + │ └── ((max:5 - min:6) - count_rows:7) != -1 [outer=(5-7), immutable] └── aggregations - └── count-rows [as=count_rows:8] + └── count-rows [as=count_rows:9] opt format=hide-qual SELECT sum(o_ol_cnt) @@ -1204,16 +1204,16 @@ GROUP BY o_w_id, o_d_id ORDER BY o_w_id, o_d_id ---- group-by - ├── columns: sum:9 [hidden: o_d_id:2!null o_w_id:3!null] + ├── columns: sum:10 [hidden: o_d_id:2!null o_w_id:3!null] ├── grouping columns: o_d_id:2!null o_w_id:3!null ├── key: (2,3) - ├── fd: (2,3)-->(9) + ├── fd: (2,3)-->(10) ├── ordering: +3,+2 ├── scan "order" │ ├── columns: o_d_id:2!null o_w_id:3!null o_ol_cnt:7 │ └── ordering: +3,+2 └── aggregations - └── sum [as=sum:9, outer=(7)] + └── sum [as=sum:10, outer=(7)] └── o_ol_cnt:7 opt format=hide-qual @@ -1223,19 +1223,19 @@ GROUP BY ol_w_id, ol_d_id ORDER BY ol_w_id, ol_d_id ---- sort - ├── columns: count:11!null [hidden: ol_d_id:2!null ol_w_id:3!null] + ├── columns: count:12!null [hidden: ol_d_id:2!null ol_w_id:3!null] ├── key: (2,3) - ├── fd: (2,3)-->(11) + ├── fd: (2,3)-->(12) ├── ordering: +3,+2 └── group-by - ├── columns: ol_d_id:2!null ol_w_id:3!null count_rows:11!null + ├── columns: ol_d_id:2!null ol_w_id:3!null count_rows:12!null ├── grouping columns: ol_d_id:2!null ol_w_id:3!null ├── key: (2,3) - ├── fd: (2,3)-->(11) + ├── fd: (2,3)-->(12) ├── scan order_line@order_line_stock_fk_idx │ └── columns: ol_d_id:2!null ol_w_id:3!null └── aggregations - └── count-rows [as=count_rows:11] + └── count-rows [as=count_rows:12] opt format=hide-qual (SELECT no_w_id, no_d_id, no_o_id FROM new_order) @@ -1245,23 +1245,23 @@ EXCEPT ALL except-all ├── columns: no_w_id:3!null no_d_id:2!null no_o_id:1!null ├── left columns: no_w_id:3!null no_d_id:2!null no_o_id:1!null - ├── right columns: o_w_id:6 o_d_id:5 o_id:4 + ├── right columns: o_w_id:7 o_d_id:6 o_id:5 ├── scan new_order │ ├── columns: no_o_id:1!null no_d_id:2!null no_w_id:3!null │ └── key: (1-3) └── project - ├── columns: o_id:4!null o_d_id:5!null o_w_id:6!null - ├── key: (4-6) + ├── columns: o_id:5!null o_d_id:6!null o_w_id:7!null + ├── key: (5-7) └── select - ├── columns: o_id:4!null o_d_id:5!null o_w_id:6!null o_carrier_id:9 - ├── key: (4-6) - ├── fd: ()-->(9) + ├── columns: o_id:5!null o_d_id:6!null o_w_id:7!null o_carrier_id:10 + ├── key: (5-7) + ├── fd: ()-->(10) ├── scan "order"@order_idx - │ ├── columns: o_id:4!null o_d_id:5!null o_w_id:6!null o_carrier_id:9 - │ ├── key: (4-6) - │ └── fd: (4-6)-->(9) + │ ├── columns: o_id:5!null o_d_id:6!null o_w_id:7!null o_carrier_id:10 + │ ├── key: (5-7) + │ └── fd: (5-7)-->(10) └── filters - └── o_carrier_id:9 IS NULL [outer=(9), constraints=(/9: [/NULL - /NULL]; tight), fd=()-->(9)] + └── o_carrier_id:10 IS NULL [outer=(10), constraints=(/10: [/NULL - /NULL]; tight), fd=()-->(10)] opt format=hide-qual (SELECT o_w_id, o_d_id, o_id FROM "order" WHERE o_carrier_id IS NULL) @@ -1271,7 +1271,7 @@ EXCEPT ALL except-all ├── columns: o_w_id:3!null o_d_id:2!null o_id:1!null ├── left columns: o_w_id:3!null o_d_id:2!null o_id:1!null - ├── right columns: no_w_id:11 no_d_id:10 no_o_id:9 + ├── right columns: no_w_id:12 no_d_id:11 no_o_id:10 ├── project │ ├── columns: o_id:1!null o_d_id:2!null o_w_id:3!null │ ├── key: (1-3) @@ -1286,8 +1286,8 @@ except-all │ └── filters │ └── o_carrier_id:6 IS NULL [outer=(6), constraints=(/6: [/NULL - /NULL]; tight), fd=()-->(6)] └── scan new_order - ├── columns: no_o_id:9!null no_d_id:10!null no_w_id:11!null - └── key: (9-11) + ├── columns: no_o_id:10!null no_d_id:11!null no_w_id:12!null + └── key: (10-12) opt format=hide-qual ( @@ -1306,20 +1306,20 @@ EXCEPT ALL except-all ├── columns: o_w_id:3!null o_d_id:2!null o_id:1!null o_ol_cnt:7 ├── left columns: o_w_id:3!null o_d_id:2!null o_id:1!null o_ol_cnt:7 - ├── right columns: ol_w_id:11 ol_d_id:10 ol_o_id:9 count_rows:19 + ├── right columns: ol_w_id:12 ol_d_id:11 ol_o_id:10 count_rows:21 ├── scan "order" │ ├── columns: o_id:1!null o_d_id:2!null o_w_id:3!null o_ol_cnt:7 │ ├── key: (1-3) │ └── fd: (1-3)-->(7) └── group-by - ├── columns: ol_o_id:9!null ol_d_id:10!null ol_w_id:11!null count_rows:19!null - ├── grouping columns: ol_o_id:9!null ol_d_id:10!null ol_w_id:11!null - ├── key: (9-11) - ├── fd: (9-11)-->(19) + ├── columns: ol_o_id:10!null ol_d_id:11!null ol_w_id:12!null count_rows:21!null + ├── grouping columns: ol_o_id:10!null ol_d_id:11!null ol_w_id:12!null + ├── key: (10-12) + ├── fd: (10-12)-->(21) ├── scan order_line@order_line_stock_fk_idx - │ └── columns: ol_o_id:9!null ol_d_id:10!null ol_w_id:11!null + │ └── columns: ol_o_id:10!null ol_d_id:11!null ol_w_id:12!null └── aggregations - └── count-rows [as=count_rows:19] + └── count-rows [as=count_rows:21] opt format=hide-qual ( @@ -1336,22 +1336,22 @@ EXCEPT ALL ) ---- except-all - ├── columns: ol_w_id:3!null ol_d_id:2!null ol_o_id:1!null count:11 - ├── left columns: ol_w_id:3!null ol_d_id:2!null ol_o_id:1!null count_rows:11 - ├── right columns: o_w_id:14 o_d_id:13 o_id:12 o_ol_cnt:18 + ├── columns: ol_w_id:3!null ol_d_id:2!null ol_o_id:1!null count:12 + ├── left columns: ol_w_id:3!null ol_d_id:2!null ol_o_id:1!null count_rows:12 + ├── right columns: o_w_id:15 o_d_id:14 o_id:13 o_ol_cnt:19 ├── group-by - │ ├── columns: ol_o_id:1!null ol_d_id:2!null ol_w_id:3!null count_rows:11!null + │ ├── columns: ol_o_id:1!null ol_d_id:2!null ol_w_id:3!null count_rows:12!null │ ├── grouping columns: ol_o_id:1!null ol_d_id:2!null ol_w_id:3!null │ ├── key: (1-3) - │ ├── fd: (1-3)-->(11) + │ ├── fd: (1-3)-->(12) │ ├── scan order_line@order_line_stock_fk_idx │ │ └── columns: ol_o_id:1!null ol_d_id:2!null ol_w_id:3!null │ └── aggregations - │ └── count-rows [as=count_rows:11] + │ └── count-rows [as=count_rows:12] └── scan "order" - ├── columns: o_id:12!null o_d_id:13!null o_w_id:14!null o_ol_cnt:18 - ├── key: (12-14) - └── fd: (12-14)-->(18) + ├── columns: o_id:13!null o_d_id:14!null o_w_id:15!null o_ol_cnt:19 + ├── key: (13-15) + └── fd: (13-15)-->(19) opt format=hide-qual SELECT count(*) @@ -1371,16 +1371,16 @@ ON (ol_w_id = o_w_id AND ol_d_id = o_d_id AND ol_o_id = o_id) WHERE ol_o_id IS NULL OR o_id IS NULL ---- scalar-group-by - ├── columns: count:19!null + ├── columns: count:21!null ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(19) + ├── fd: ()-->(21) ├── select - │ ├── columns: o_id:1 o_d_id:2 o_w_id:3 ol_o_id:9 ol_d_id:10 ol_w_id:11 + │ ├── columns: o_id:1 o_d_id:2 o_w_id:3 ol_o_id:10 ol_d_id:11 ol_w_id:12 │ ├── full-join (merge) - │ │ ├── columns: o_id:1 o_d_id:2 o_w_id:3 ol_o_id:9 ol_d_id:10 ol_w_id:11 + │ │ ├── columns: o_id:1 o_d_id:2 o_w_id:3 ol_o_id:10 ol_d_id:11 ol_w_id:12 │ │ ├── left ordering: +3,+2,-1 - │ │ ├── right ordering: +11,+10,-9 + │ │ ├── right ordering: +12,+11,-10 │ │ ├── project │ │ │ ├── columns: o_id:1!null o_d_id:2!null o_w_id:3!null │ │ │ ├── key: (1-3) @@ -1398,19 +1398,19 @@ scalar-group-by │ │ │ └── filters │ │ │ └── o_carrier_id:6 IS NULL [outer=(6), constraints=(/6: [/NULL - /NULL]; tight), fd=()-->(6)] │ │ ├── project - │ │ │ ├── columns: ol_o_id:9!null ol_d_id:10!null ol_w_id:11!null - │ │ │ ├── ordering: +11,+10,-9 + │ │ │ ├── columns: ol_o_id:10!null ol_d_id:11!null ol_w_id:12!null + │ │ │ ├── ordering: +12,+11,-10 │ │ │ └── select - │ │ │ ├── columns: ol_o_id:9!null ol_d_id:10!null ol_w_id:11!null ol_delivery_d:15 - │ │ │ ├── fd: ()-->(15) - │ │ │ ├── ordering: +11,+10,-9 opt(15) [actual: +11,+10,-9] + │ │ │ ├── columns: ol_o_id:10!null ol_d_id:11!null ol_w_id:12!null ol_delivery_d:16 + │ │ │ ├── fd: ()-->(16) + │ │ │ ├── ordering: +12,+11,-10 opt(16) [actual: +12,+11,-10] │ │ │ ├── scan order_line - │ │ │ │ ├── columns: ol_o_id:9!null ol_d_id:10!null ol_w_id:11!null ol_delivery_d:15 - │ │ │ │ └── ordering: +11,+10,-9 opt(15) [actual: +11,+10,-9] + │ │ │ │ ├── columns: ol_o_id:10!null ol_d_id:11!null ol_w_id:12!null ol_delivery_d:16 + │ │ │ │ └── ordering: +12,+11,-10 opt(16) [actual: +12,+11,-10] │ │ │ └── filters - │ │ │ └── ol_delivery_d:15 IS NULL [outer=(15), constraints=(/15: [/NULL - /NULL]; tight), fd=()-->(15)] + │ │ │ └── ol_delivery_d:16 IS NULL [outer=(16), constraints=(/16: [/NULL - /NULL]; tight), fd=()-->(16)] │ │ └── filters (true) │ └── filters - │ └── (ol_o_id:9 IS NULL) OR (o_id:1 IS NULL) [outer=(1,9)] + │ └── (ol_o_id:10 IS NULL) OR (o_id:1 IS NULL) [outer=(1,10)] └── aggregations - └── count-rows [as=count_rows:19] + └── count-rows [as=count_rows:21] diff --git a/pkg/sql/opt/xform/testdata/external/tpcc-later-stats b/pkg/sql/opt/xform/testdata/external/tpcc-later-stats index 15216389d167..62abfaf14dbd 100644 --- a/pkg/sql/opt/xform/testdata/external/tpcc-later-stats +++ b/pkg/sql/opt/xform/testdata/external/tpcc-later-stats @@ -32,27 +32,27 @@ project ├── fd: ()-->(9,11) └── update district ├── columns: d_id:1!null d_w_id:2!null d_tax:9 d_next_o_id:11 - ├── fetch columns: d_id:12 d_w_id:13 d_name:14 d_street_1:15 d_street_2:16 d_city:17 d_state:18 d_zip:19 d_tax:20 d_ytd:21 d_next_o_id:22 + ├── fetch columns: d_id:13 d_w_id:14 d_name:15 d_street_1:16 d_street_2:17 d_city:18 d_state:19 d_zip:20 d_tax:21 d_ytd:22 d_next_o_id:23 ├── update-mapping: - │ └── d_next_o_id_new:23 => d_next_o_id:11 + │ └── d_next_o_id_new:25 => d_next_o_id:11 ├── cardinality: [0 - 1] ├── volatile, mutations ├── key: () ├── fd: ()-->(1,2,9,11) └── project - ├── columns: d_next_o_id_new:23 d_id:12!null d_w_id:13!null d_name:14 d_street_1:15 d_street_2:16 d_city:17 d_state:18 d_zip:19 d_tax:20 d_ytd:21 d_next_o_id:22 + ├── columns: d_next_o_id_new:25 d_id:13!null d_w_id:14!null d_name:15 d_street_1:16 d_street_2:17 d_city:18 d_state:19 d_zip:20 d_tax:21 d_ytd:22 d_next_o_id:23 ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(12-23) + ├── fd: ()-->(13-23,25) ├── scan district - │ ├── columns: d_id:12!null d_w_id:13!null d_name:14 d_street_1:15 d_street_2:16 d_city:17 d_state:18 d_zip:19 d_tax:20 d_ytd:21 d_next_o_id:22 - │ ├── constraint: /13/12: [/10/5 - /10/5] + │ ├── columns: d_id:13!null d_w_id:14!null d_name:15 d_street_1:16 d_street_2:17 d_city:18 d_state:19 d_zip:20 d_tax:21 d_ytd:22 d_next_o_id:23 + │ ├── constraint: /14/13: [/10/5 - /10/5] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(12-22) + │ └── fd: ()-->(13-23) └── projections - └── d_next_o_id:22 + 1 [as=d_next_o_id_new:23, outer=(22), immutable] + └── d_next_o_id:23 + 1 [as=d_next_o_id_new:25, outer=(23), immutable] opt format=hide-qual SELECT w_tax FROM warehouse WHERE w_id = 10 @@ -144,41 +144,41 @@ VALUES (100, 5, 10, 50, '2019-08-26 16:50:41', 10, 1) insert "order" ├── columns: ├── insert-mapping: - │ ├── column1:9 => o_id:1 - │ ├── column2:10 => o_d_id:2 - │ ├── column3:11 => o_w_id:3 - │ ├── column4:12 => o_c_id:4 - │ ├── column5:13 => o_entry_d:5 - │ ├── column16:16 => o_carrier_id:6 - │ ├── column6:14 => o_ol_cnt:7 - │ └── column7:15 => o_all_local:8 + │ ├── column1:10 => o_id:1 + │ ├── column2:11 => o_d_id:2 + │ ├── column3:12 => o_w_id:3 + │ ├── column4:13 => o_c_id:4 + │ ├── column5:14 => o_entry_d:5 + │ ├── column17:17 => o_carrier_id:6 + │ ├── column6:15 => o_ol_cnt:7 + │ └── column7:16 => o_all_local:8 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── values - │ ├── columns: column1:9!null column2:10!null column3:11!null column4:12!null column5:13!null column6:14!null column7:15!null column16:16 + │ ├── columns: column1:10!null column2:11!null column3:12!null column4:13!null column5:14!null column6:15!null column7:16!null column17:17 │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(9-16) + │ ├── fd: ()-->(10-17) │ └── (100, 5, 10, 50, '2019-08-26 16:50:41+00:00', 10, 1, NULL) └── f-k-checks └── f-k-checks-item: order(o_w_id,o_d_id,o_c_id) -> customer(c_w_id,c_d_id,c_id) └── anti-join (lookup customer) - ├── columns: column3:17!null column2:18!null column4:19!null - ├── key columns: [17 18 19] = [22 21 20] + ├── columns: column3:18!null column2:19!null column4:20!null + ├── key columns: [18 19 20] = [23 22 21] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(17-19) + ├── fd: ()-->(18-20) ├── with-scan &1 - │ ├── columns: column3:17!null column2:18!null column4:19!null + │ ├── columns: column3:18!null column2:19!null column4:20!null │ ├── mapping: - │ │ ├── column3:11 => column3:17 - │ │ ├── column2:10 => column2:18 - │ │ └── column4:12 => column4:19 + │ │ ├── column3:12 => column3:18 + │ │ ├── column2:11 => column2:19 + │ │ └── column4:13 => column4:20 │ ├── cardinality: [1 - 1] │ ├── key: () - │ └── fd: ()-->(17-19) + │ └── fd: ()-->(18-20) └── filters (true) opt format=hide-qual @@ -187,36 +187,36 @@ INSERT INTO new_order (no_o_id, no_d_id, no_w_id) VALUES (2000, 100, 10) insert new_order ├── columns: ├── insert-mapping: - │ ├── column1:4 => no_o_id:1 - │ ├── column2:5 => no_d_id:2 - │ └── column3:6 => no_w_id:3 + │ ├── column1:5 => no_o_id:1 + │ ├── column2:6 => no_d_id:2 + │ └── column3:7 => no_w_id:3 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── values - │ ├── columns: column1:4!null column2:5!null column3:6!null + │ ├── columns: column1:5!null column2:6!null column3:7!null │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(4-6) + │ ├── fd: ()-->(5-7) │ └── (2000, 100, 10) └── f-k-checks └── f-k-checks-item: new_order(no_w_id,no_d_id,no_o_id) -> order(o_w_id,o_d_id,o_id) └── anti-join (lookup order) - ├── columns: column3:7!null column2:8!null column1:9!null - ├── key columns: [7 8 9] = [12 11 10] + ├── columns: column3:8!null column2:9!null column1:10!null + ├── key columns: [8 9 10] = [13 12 11] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(7-9) + ├── fd: ()-->(8-10) ├── with-scan &1 - │ ├── columns: column3:7!null column2:8!null column1:9!null + │ ├── columns: column3:8!null column2:9!null column1:10!null │ ├── mapping: - │ │ ├── column3:6 => column3:7 - │ │ ├── column2:5 => column2:8 - │ │ └── column1:4 => column1:9 + │ │ ├── column3:7 => column3:8 + │ │ ├── column2:6 => column2:9 + │ │ └── column1:5 => column1:10 │ ├── cardinality: [1 - 1] │ ├── key: () - │ └── fd: ()-->(7-9) + │ └── fd: ()-->(8-10) └── filters (true) opt format=hide-qual @@ -308,23 +308,23 @@ WHERE ---- update stock ├── columns: - ├── fetch columns: s_i_id:18 s_w_id:19 s_quantity:20 s_dist_01:21 s_dist_02:22 s_dist_03:23 s_dist_04:24 s_dist_05:25 s_dist_06:26 s_dist_07:27 s_dist_08:28 s_dist_09:29 s_dist_10:30 s_ytd:31 s_order_cnt:32 s_remote_cnt:33 s_data:34 + ├── fetch columns: s_i_id:19 s_w_id:20 s_quantity:21 s_dist_01:22 s_dist_02:23 s_dist_03:24 s_dist_04:25 s_dist_05:26 s_dist_06:27 s_dist_07:28 s_dist_08:29 s_dist_09:30 s_dist_10:31 s_ytd:32 s_order_cnt:33 s_remote_cnt:34 s_data:35 ├── update-mapping: - │ ├── s_quantity_new:35 => s_quantity:3 - │ ├── s_ytd_new:36 => s_ytd:14 - │ ├── s_order_cnt_new:37 => s_order_cnt:15 - │ └── s_remote_cnt_new:38 => s_remote_cnt:16 + │ ├── s_quantity_new:37 => s_quantity:3 + │ ├── s_ytd_new:38 => s_ytd:14 + │ ├── s_order_cnt_new:39 => s_order_cnt:15 + │ └── s_remote_cnt_new:40 => s_remote_cnt:16 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: s_quantity_new:35 s_ytd_new:36 s_order_cnt_new:37 s_remote_cnt_new:38 s_i_id:18!null s_w_id:19!null s_quantity:20 s_dist_01:21 s_dist_02:22 s_dist_03:23 s_dist_04:24 s_dist_05:25 s_dist_06:26 s_dist_07:27 s_dist_08:28 s_dist_09:29 s_dist_10:30 s_ytd:31 s_order_cnt:32 s_remote_cnt:33 s_data:34 + ├── columns: s_quantity_new:37 s_ytd_new:38 s_order_cnt_new:39 s_remote_cnt_new:40 s_i_id:19!null s_w_id:20!null s_quantity:21 s_dist_01:22 s_dist_02:23 s_dist_03:24 s_dist_04:25 s_dist_05:26 s_dist_06:27 s_dist_07:28 s_dist_08:29 s_dist_09:30 s_dist_10:31 s_ytd:32 s_order_cnt:33 s_remote_cnt:34 s_data:35 ├── cardinality: [0 - 13] ├── volatile - ├── key: (18) - ├── fd: ()-->(19), (18)-->(20-38) + ├── key: (19) + ├── fd: ()-->(20), (19)-->(21-35,37-40) ├── scan stock - │ ├── columns: s_i_id:18!null s_w_id:19!null s_quantity:20 s_dist_01:21 s_dist_02:22 s_dist_03:23 s_dist_04:24 s_dist_05:25 s_dist_06:26 s_dist_07:27 s_dist_08:28 s_dist_09:29 s_dist_10:30 s_ytd:31 s_order_cnt:32 s_remote_cnt:33 s_data:34 - │ ├── constraint: /19/18 + │ ├── columns: s_i_id:19!null s_w_id:20!null s_quantity:21 s_dist_01:22 s_dist_02:23 s_dist_03:24 s_dist_04:25 s_dist_05:26 s_dist_06:27 s_dist_07:28 s_dist_08:29 s_dist_09:30 s_dist_10:31 s_ytd:32 s_order_cnt:33 s_remote_cnt:34 s_data:35 + │ ├── constraint: /20/19 │ │ ├── [/0/6823 - /0/6823] │ │ ├── [/0/7853 - /0/7853] │ │ ├── [/0/8497 - /0/8497] @@ -339,13 +339,13 @@ update stock │ │ ├── [/0/89641 - /0/89641] │ │ └── [/0/89905 - /0/89905] │ ├── cardinality: [0 - 13] - │ ├── key: (18) - │ └── fd: ()-->(19), (18)-->(20-34) + │ ├── key: (19) + │ └── fd: ()-->(20), (19)-->(21-35) └── projections - ├── CASE (s_i_id:18, s_w_id:19) WHEN (6823, 0) THEN 26 WHEN (7853, 0) THEN 10 WHEN (8497, 0) THEN 62 WHEN (10904, 0) THEN 54 WHEN (16152, 0) THEN 80 WHEN (41382, 0) THEN 18 WHEN (55952, 0) THEN 56 WHEN (64817, 0) THEN 26 WHEN (66335, 0) THEN 30 WHEN (76567, 0) THEN 71 WHEN (81680, 0) THEN 51 WHEN (89641, 0) THEN 51 WHEN (89905, 0) THEN 77 ELSE crdb_internal.force_error('', 'unknown case') END [as=s_quantity_new:35, outer=(18,19), volatile] - ├── CASE (s_i_id:18, s_w_id:19) WHEN (6823, 0) THEN 6 WHEN (7853, 0) THEN 9 WHEN (8497, 0) THEN 13 WHEN (10904, 0) THEN 1 WHEN (16152, 0) THEN 2 WHEN (41382, 0) THEN 3 WHEN (55952, 0) THEN 10 WHEN (64817, 0) THEN 31 WHEN (66335, 0) THEN 9 WHEN (76567, 0) THEN 7 WHEN (81680, 0) THEN 4 WHEN (89641, 0) THEN 13 WHEN (89905, 0) THEN 20 ELSE CAST(NULL AS INT8) END [as=s_ytd_new:36, outer=(18,19)] - ├── CASE (s_i_id:18, s_w_id:19) WHEN (6823, 0) THEN 1 WHEN (7853, 0) THEN 1 WHEN (8497, 0) THEN 2 WHEN (10904, 0) THEN 1 WHEN (16152, 0) THEN 1 WHEN (41382, 0) THEN 1 WHEN (55952, 0) THEN 1 WHEN (64817, 0) THEN 4 WHEN (66335, 0) THEN 2 WHEN (76567, 0) THEN 1 WHEN (81680, 0) THEN 1 WHEN (89641, 0) THEN 2 WHEN (89905, 0) THEN 4 ELSE CAST(NULL AS INT8) END [as=s_order_cnt_new:37, outer=(18,19)] - └── CASE (s_i_id:18, s_w_id:19) WHEN (6823, 0) THEN 0 WHEN (7853, 0) THEN 0 WHEN (8497, 0) THEN 0 WHEN (10904, 0) THEN 0 WHEN (16152, 0) THEN 0 WHEN (41382, 0) THEN 0 WHEN (55952, 0) THEN 0 WHEN (64817, 0) THEN 0 WHEN (66335, 0) THEN 0 WHEN (76567, 0) THEN 0 WHEN (81680, 0) THEN 0 WHEN (89641, 0) THEN 0 WHEN (89905, 0) THEN 0 ELSE CAST(NULL AS INT8) END [as=s_remote_cnt_new:38, outer=(18,19)] + ├── CASE (s_i_id:19, s_w_id:20) WHEN (6823, 0) THEN 26 WHEN (7853, 0) THEN 10 WHEN (8497, 0) THEN 62 WHEN (10904, 0) THEN 54 WHEN (16152, 0) THEN 80 WHEN (41382, 0) THEN 18 WHEN (55952, 0) THEN 56 WHEN (64817, 0) THEN 26 WHEN (66335, 0) THEN 30 WHEN (76567, 0) THEN 71 WHEN (81680, 0) THEN 51 WHEN (89641, 0) THEN 51 WHEN (89905, 0) THEN 77 ELSE crdb_internal.force_error('', 'unknown case') END [as=s_quantity_new:37, outer=(19,20), volatile] + ├── CASE (s_i_id:19, s_w_id:20) WHEN (6823, 0) THEN 6 WHEN (7853, 0) THEN 9 WHEN (8497, 0) THEN 13 WHEN (10904, 0) THEN 1 WHEN (16152, 0) THEN 2 WHEN (41382, 0) THEN 3 WHEN (55952, 0) THEN 10 WHEN (64817, 0) THEN 31 WHEN (66335, 0) THEN 9 WHEN (76567, 0) THEN 7 WHEN (81680, 0) THEN 4 WHEN (89641, 0) THEN 13 WHEN (89905, 0) THEN 20 ELSE CAST(NULL AS INT8) END [as=s_ytd_new:38, outer=(19,20)] + ├── CASE (s_i_id:19, s_w_id:20) WHEN (6823, 0) THEN 1 WHEN (7853, 0) THEN 1 WHEN (8497, 0) THEN 2 WHEN (10904, 0) THEN 1 WHEN (16152, 0) THEN 1 WHEN (41382, 0) THEN 1 WHEN (55952, 0) THEN 1 WHEN (64817, 0) THEN 4 WHEN (66335, 0) THEN 2 WHEN (76567, 0) THEN 1 WHEN (81680, 0) THEN 1 WHEN (89641, 0) THEN 2 WHEN (89905, 0) THEN 4 ELSE CAST(NULL AS INT8) END [as=s_order_cnt_new:39, outer=(19,20)] + └── CASE (s_i_id:19, s_w_id:20) WHEN (6823, 0) THEN 0 WHEN (7853, 0) THEN 0 WHEN (8497, 0) THEN 0 WHEN (10904, 0) THEN 0 WHEN (16152, 0) THEN 0 WHEN (41382, 0) THEN 0 WHEN (55952, 0) THEN 0 WHEN (64817, 0) THEN 0 WHEN (66335, 0) THEN 0 WHEN (76567, 0) THEN 0 WHEN (81680, 0) THEN 0 WHEN (89641, 0) THEN 0 WHEN (89905, 0) THEN 0 ELSE CAST(NULL AS INT8) END [as=s_remote_cnt_new:40, outer=(19,20)] opt format=hide-qual INSERT INTO order_line @@ -361,26 +361,26 @@ VALUES insert order_line ├── columns: ├── insert-mapping: - │ ├── column1:11 => ol_o_id:1 - │ ├── column2:12 => ol_d_id:2 - │ ├── column3:13 => ol_w_id:3 - │ ├── column4:14 => ol_number:4 - │ ├── column5:15 => ol_i_id:5 - │ ├── column6:16 => ol_supply_w_id:6 - │ ├── column20:20 => ol_delivery_d:7 - │ ├── column7:17 => ol_quantity:8 - │ ├── ol_amount:21 => order_line.ol_amount:9 - │ └── column9:19 => ol_dist_info:10 + │ ├── column1:12 => ol_o_id:1 + │ ├── column2:13 => ol_d_id:2 + │ ├── column3:14 => ol_w_id:3 + │ ├── column4:15 => ol_number:4 + │ ├── column5:16 => ol_i_id:5 + │ ├── column6:17 => ol_supply_w_id:6 + │ ├── column21:21 => ol_delivery_d:7 + │ ├── column7:18 => ol_quantity:8 + │ ├── ol_amount:22 => order_line.ol_amount:9 + │ └── column9:20 => ol_dist_info:10 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── project - │ ├── columns: ol_amount:21 column20:20 column1:11!null column2:12!null column3:13!null column4:14!null column5:15!null column6:16!null column7:17!null column9:19!null + │ ├── columns: ol_amount:22 column21:21 column1:12!null column2:13!null column3:14!null column4:15!null column5:16!null column6:17!null column7:18!null column9:20!null │ ├── cardinality: [6 - 6] │ ├── immutable - │ ├── fd: ()-->(20) + │ ├── fd: ()-->(21) │ ├── values - │ │ ├── columns: column1:11!null column2:12!null column3:13!null column4:14!null column5:15!null column6:16!null column7:17!null column8:18!null column9:19!null + │ │ ├── columns: column1:12!null column2:13!null column3:14!null column4:15!null column5:16!null column6:17!null column7:18!null column8:19!null column9:20!null │ │ ├── cardinality: [6 - 6] │ │ ├── (3045, 2, 10, 3, 648, 0, 9, 394.470000, 'YhgLRrwsmd68P2bElAgrnp8u') │ │ ├── (3045, 2, 10, 5, 25393, 0, 10, 830.600000, 'dLXe0YhgLRrwsmd68P2bElAg') @@ -389,34 +389,34 @@ insert order_line │ │ ├── (3045, 2, 10, 4, 56624, 0, 6, 273.360000, 'RsaCXoEzmssaF9m9cdLXe0Yh') │ │ └── (3045, 2, 10, 6, 92966, 0, 4, 366.760000, 'saCXoEzmssaF9m9cdLXe0Yhg') │ └── projections - │ ├── crdb_internal.round_decimal_values(column8:18, 2) [as=ol_amount:21, outer=(18), immutable] - │ └── CAST(NULL AS TIMESTAMP) [as=column20:20] + │ ├── crdb_internal.round_decimal_values(column8:19, 2) [as=ol_amount:22, outer=(19), immutable] + │ └── CAST(NULL AS TIMESTAMP) [as=column21:21] └── f-k-checks ├── f-k-checks-item: order_line(ol_w_id,ol_d_id,ol_o_id) -> order(o_w_id,o_d_id,o_id) │ └── anti-join (lookup order) - │ ├── columns: column3:22!null column2:23!null column1:24!null - │ ├── key columns: [22 23 24] = [27 26 25] + │ ├── columns: column3:23!null column2:24!null column1:25!null + │ ├── key columns: [23 24 25] = [28 27 26] │ ├── lookup columns are key │ ├── cardinality: [0 - 6] │ ├── with-scan &1 - │ │ ├── columns: column3:22!null column2:23!null column1:24!null + │ │ ├── columns: column3:23!null column2:24!null column1:25!null │ │ ├── mapping: - │ │ │ ├── column3:13 => column3:22 - │ │ │ ├── column2:12 => column2:23 - │ │ │ └── column1:11 => column1:24 + │ │ │ ├── column3:14 => column3:23 + │ │ │ ├── column2:13 => column2:24 + │ │ │ └── column1:12 => column1:25 │ │ └── cardinality: [6 - 6] │ └── filters (true) └── f-k-checks-item: order_line(ol_supply_w_id,ol_i_id) -> stock(s_w_id,s_i_id) └── anti-join (lookup stock@stock_item_fk_idx) - ├── columns: column6:33!null column5:34!null - ├── key columns: [34 33] = [35 36] + ├── columns: column6:35!null column5:36!null + ├── key columns: [36 35] = [37 38] ├── lookup columns are key ├── cardinality: [0 - 6] ├── with-scan &1 - │ ├── columns: column6:33!null column5:34!null + │ ├── columns: column6:35!null column5:36!null │ ├── mapping: - │ │ ├── column6:16 => column6:33 - │ │ └── column5:15 => column5:34 + │ │ ├── column6:17 => column6:35 + │ │ └── column5:16 => column5:36 │ └── cardinality: [6 - 6] └── filters (true) @@ -441,27 +441,27 @@ project ├── fd: ()-->(2-7) └── update warehouse ├── columns: w_id:1!null w_name:2 w_street_1:3 w_street_2:4 w_city:5 w_state:6 w_zip:7 - ├── fetch columns: w_id:10 w_name:11 w_street_1:12 w_street_2:13 w_city:14 w_state:15 w_zip:16 w_tax:17 warehouse.w_ytd:18 + ├── fetch columns: w_id:11 w_name:12 w_street_1:13 w_street_2:14 w_city:15 w_state:16 w_zip:17 w_tax:18 warehouse.w_ytd:19 ├── update-mapping: - │ └── w_ytd:20 => warehouse.w_ytd:9 + │ └── w_ytd:22 => warehouse.w_ytd:9 ├── cardinality: [0 - 1] ├── volatile, mutations ├── key: () ├── fd: ()-->(1-7) └── project - ├── columns: w_ytd:20 w_id:10!null w_name:11 w_street_1:12 w_street_2:13 w_city:14 w_state:15 w_zip:16 w_tax:17 warehouse.w_ytd:18 + ├── columns: w_ytd:22 w_id:11!null w_name:12 w_street_1:13 w_street_2:14 w_city:15 w_state:16 w_zip:17 w_tax:18 warehouse.w_ytd:19 ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(10-18,20) + ├── fd: ()-->(11-19,22) ├── scan warehouse - │ ├── columns: w_id:10!null w_name:11 w_street_1:12 w_street_2:13 w_city:14 w_state:15 w_zip:16 w_tax:17 warehouse.w_ytd:18 - │ ├── constraint: /10: [/10 - /10] + │ ├── columns: w_id:11!null w_name:12 w_street_1:13 w_street_2:14 w_city:15 w_state:16 w_zip:17 w_tax:18 warehouse.w_ytd:19 + │ ├── constraint: /11: [/10 - /10] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(10-18) + │ └── fd: ()-->(11-19) └── projections - └── crdb_internal.round_decimal_values(warehouse.w_ytd:18::DECIMAL + 3860.61, 2) [as=w_ytd:20, outer=(18), immutable] + └── crdb_internal.round_decimal_values(warehouse.w_ytd:19::DECIMAL + 3860.61, 2) [as=w_ytd:22, outer=(19), immutable] opt format=hide-qual UPDATE district SET d_ytd = d_ytd + 3860.61 WHERE (d_w_id = 10) AND (d_id = 5) @@ -475,27 +475,27 @@ project ├── fd: ()-->(3-8) └── update district ├── columns: d_id:1!null d_w_id:2!null d_name:3 d_street_1:4 d_street_2:5 d_city:6 d_state:7 d_zip:8 - ├── fetch columns: d_id:12 d_w_id:13 d_name:14 d_street_1:15 d_street_2:16 d_city:17 d_state:18 d_zip:19 d_tax:20 district.d_ytd:21 d_next_o_id:22 + ├── fetch columns: d_id:13 d_w_id:14 d_name:15 d_street_1:16 d_street_2:17 d_city:18 d_state:19 d_zip:20 d_tax:21 district.d_ytd:22 d_next_o_id:23 ├── update-mapping: - │ └── d_ytd:24 => district.d_ytd:10 + │ └── d_ytd:26 => district.d_ytd:10 ├── cardinality: [0 - 1] ├── volatile, mutations ├── key: () ├── fd: ()-->(1-8) └── project - ├── columns: d_ytd:24 d_id:12!null d_w_id:13!null d_name:14 d_street_1:15 d_street_2:16 d_city:17 d_state:18 d_zip:19 d_tax:20 district.d_ytd:21 d_next_o_id:22 + ├── columns: d_ytd:26 d_id:13!null d_w_id:14!null d_name:15 d_street_1:16 d_street_2:17 d_city:18 d_state:19 d_zip:20 d_tax:21 district.d_ytd:22 d_next_o_id:23 ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(12-22,24) + ├── fd: ()-->(13-23,26) ├── scan district - │ ├── columns: d_id:12!null d_w_id:13!null d_name:14 d_street_1:15 d_street_2:16 d_city:17 d_state:18 d_zip:19 d_tax:20 district.d_ytd:21 d_next_o_id:22 - │ ├── constraint: /13/12: [/10/5 - /10/5] + │ ├── columns: d_id:13!null d_w_id:14!null d_name:15 d_street_1:16 d_street_2:17 d_city:18 d_state:19 d_zip:20 d_tax:21 district.d_ytd:22 d_next_o_id:23 + │ ├── constraint: /14/13: [/10/5 - /10/5] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(12-22) + │ └── fd: ()-->(13-23) └── projections - └── crdb_internal.round_decimal_values(district.d_ytd:21::DECIMAL + 3860.61, 2) [as=d_ytd:24, outer=(21), immutable] + └── crdb_internal.round_decimal_values(district.d_ytd:22::DECIMAL + 3860.61, 2) [as=d_ytd:26, outer=(22), immutable] opt format=hide-qual SELECT c_id @@ -557,42 +557,42 @@ RETURNING CASE c_credit WHEN 'BC' THEN "left"(c_data, 200) ELSE '' END ---- project - ├── columns: c_first:4 c_middle:5 c_last:6 c_street_1:7 c_street_2:8 c_city:9 c_state:10 c_zip:11 c_phone:12 c_since:13 c_credit:14 c_credit_lim:15 c_discount:16 c_balance:17 case:49 + ├── columns: c_first:4 c_middle:5 c_last:6 c_street_1:7 c_street_2:8 c_city:9 c_state:10 c_zip:11 c_phone:12 c_since:13 c_credit:14 c_credit_lim:15 c_discount:16 c_balance:17 case:51 ├── cardinality: [0 - 1] ├── volatile, mutations ├── key: () - ├── fd: ()-->(4-17,49) + ├── fd: ()-->(4-17,51) ├── update customer │ ├── columns: c_id:1!null c_d_id:2!null c_w_id:3!null c_first:4 c_middle:5 c_last:6 c_street_1:7 c_street_2:8 c_city:9 c_state:10 c_zip:11 c_phone:12 c_since:13 c_credit:14 c_credit_lim:15 c_discount:16 customer.c_balance:17 c_data:21 - │ ├── fetch columns: c_id:22 c_d_id:23 c_w_id:24 c_first:25 c_middle:26 c_last:27 c_street_1:28 c_street_2:29 c_city:30 c_state:31 c_zip:32 c_phone:33 c_since:34 c_credit:35 c_credit_lim:36 c_discount:37 customer.c_balance:38 customer.c_ytd_payment:39 c_payment_cnt:40 c_delivery_cnt:41 c_data:42 + │ ├── fetch columns: c_id:23 c_d_id:24 c_w_id:25 c_first:26 c_middle:27 c_last:28 c_street_1:29 c_street_2:30 c_city:31 c_state:32 c_zip:33 c_phone:34 c_since:35 c_credit:36 c_credit_lim:37 c_discount:38 customer.c_balance:39 customer.c_ytd_payment:40 c_payment_cnt:41 c_delivery_cnt:42 c_data:43 │ ├── update-mapping: - │ │ ├── c_balance:47 => customer.c_balance:17 - │ │ ├── c_ytd_payment:48 => customer.c_ytd_payment:18 - │ │ ├── c_payment_cnt_new:45 => c_payment_cnt:19 - │ │ └── c_data_new:46 => c_data:21 + │ │ ├── c_balance:49 => customer.c_balance:17 + │ │ ├── c_ytd_payment:50 => customer.c_ytd_payment:18 + │ │ ├── c_payment_cnt_new:47 => c_payment_cnt:19 + │ │ └── c_data_new:48 => c_data:21 │ ├── cardinality: [0 - 1] │ ├── volatile, mutations │ ├── key: () │ ├── fd: ()-->(1-17,21) │ └── project - │ ├── columns: c_balance:47 c_ytd_payment:48 c_payment_cnt_new:45 c_data_new:46 c_id:22!null c_d_id:23!null c_w_id:24!null c_first:25 c_middle:26 c_last:27 c_street_1:28 c_street_2:29 c_city:30 c_state:31 c_zip:32 c_phone:33 c_since:34 c_credit:35 c_credit_lim:36 c_discount:37 customer.c_balance:38 customer.c_ytd_payment:39 c_payment_cnt:40 c_delivery_cnt:41 c_data:42 + │ ├── columns: c_balance:49 c_ytd_payment:50 c_payment_cnt_new:47 c_data_new:48 c_id:23!null c_d_id:24!null c_w_id:25!null c_first:26 c_middle:27 c_last:28 c_street_1:29 c_street_2:30 c_city:31 c_state:32 c_zip:33 c_phone:34 c_since:35 c_credit:36 c_credit_lim:37 c_discount:38 customer.c_balance:39 customer.c_ytd_payment:40 c_payment_cnt:41 c_delivery_cnt:42 c_data:43 │ ├── cardinality: [0 - 1] │ ├── immutable │ ├── key: () - │ ├── fd: ()-->(22-42,45-48) + │ ├── fd: ()-->(23-43,47-50) │ ├── scan customer - │ │ ├── columns: c_id:22!null c_d_id:23!null c_w_id:24!null c_first:25 c_middle:26 c_last:27 c_street_1:28 c_street_2:29 c_city:30 c_state:31 c_zip:32 c_phone:33 c_since:34 c_credit:35 c_credit_lim:36 c_discount:37 customer.c_balance:38 customer.c_ytd_payment:39 c_payment_cnt:40 c_delivery_cnt:41 c_data:42 - │ │ ├── constraint: /24/23/22: [/10/5/1343 - /10/5/1343] + │ │ ├── columns: c_id:23!null c_d_id:24!null c_w_id:25!null c_first:26 c_middle:27 c_last:28 c_street_1:29 c_street_2:30 c_city:31 c_state:32 c_zip:33 c_phone:34 c_since:35 c_credit:36 c_credit_lim:37 c_discount:38 customer.c_balance:39 customer.c_ytd_payment:40 c_payment_cnt:41 c_delivery_cnt:42 c_data:43 + │ │ ├── constraint: /25/24/23: [/10/5/1343 - /10/5/1343] │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(22-42) + │ │ └── fd: ()-->(23-43) │ └── projections - │ ├── crdb_internal.round_decimal_values(customer.c_balance:38::DECIMAL - 3860.61, 2) [as=c_balance:47, outer=(38), immutable] - │ ├── crdb_internal.round_decimal_values(customer.c_ytd_payment:39::DECIMAL + 3860.61, 2) [as=c_ytd_payment:48, outer=(39), immutable] - │ ├── c_payment_cnt:40 + 1 [as=c_payment_cnt_new:45, outer=(40), immutable] - │ └── CASE c_credit:35 WHEN 'BC' THEN left((((((c_id:22::STRING || c_d_id:23::STRING) || c_w_id:24::STRING) || '5') || '10') || '3860.61') || c_data:42::STRING, 500) ELSE c_data:42::STRING END [as=c_data_new:46, outer=(22-24,35,42), immutable] + │ ├── crdb_internal.round_decimal_values(customer.c_balance:39::DECIMAL - 3860.61, 2) [as=c_balance:49, outer=(39), immutable] + │ ├── crdb_internal.round_decimal_values(customer.c_ytd_payment:40::DECIMAL + 3860.61, 2) [as=c_ytd_payment:50, outer=(40), immutable] + │ ├── c_payment_cnt:41 + 1 [as=c_payment_cnt_new:47, outer=(41), immutable] + │ └── CASE c_credit:36 WHEN 'BC' THEN left((((((c_id:23::STRING || c_d_id:24::STRING) || c_w_id:25::STRING) || '5') || '10') || '3860.61') || c_data:43::STRING, 500) ELSE c_data:43::STRING END [as=c_data_new:48, outer=(23-25,36,43), immutable] └── projections - └── CASE c_credit:14 WHEN 'BC' THEN left(c_data:21, 200) ELSE '' END [as=case:49, outer=(14,21), immutable] + └── CASE c_credit:14 WHEN 'BC' THEN left(c_data:21, 200) ELSE '' END [as=case:51, outer=(14,21), immutable] opt format=hide-qual INSERT INTO history @@ -603,60 +603,60 @@ VALUES insert history ├── columns: ├── insert-mapping: - │ ├── column18:18 => rowid:1 - │ ├── column1:10 => h_c_id:2 - │ ├── column2:11 => h_c_d_id:3 - │ ├── column3:12 => h_c_w_id:4 - │ ├── column4:13 => h_d_id:5 - │ ├── column5:14 => h_w_id:6 - │ ├── column7:16 => h_date:7 - │ ├── h_amount:19 => history.h_amount:8 - │ └── column8:17 => h_data:9 + │ ├── column19:19 => rowid:1 + │ ├── column1:11 => h_c_id:2 + │ ├── column2:12 => h_c_d_id:3 + │ ├── column3:13 => h_c_w_id:4 + │ ├── column4:14 => h_d_id:5 + │ ├── column5:15 => h_w_id:6 + │ ├── column7:17 => h_date:7 + │ ├── h_amount:20 => history.h_amount:8 + │ └── column8:18 => h_data:9 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── values - │ ├── columns: column1:10!null column2:11!null column3:12!null column4:13!null column5:14!null column7:16!null column8:17!null column18:18 h_amount:19!null + │ ├── columns: column1:11!null column2:12!null column3:13!null column4:14!null column5:15!null column7:17!null column8:18!null column19:19 h_amount:20!null │ ├── cardinality: [1 - 1] │ ├── volatile │ ├── key: () - │ ├── fd: ()-->(10-14,16-19) + │ ├── fd: ()-->(11-15,17-20) │ └── (1343, 5, 10, 5, 10, '2019-08-26 16:50:41+00:00', '8 Kdcgphy3', gen_random_uuid(), 3860.61) └── f-k-checks ├── f-k-checks-item: history(h_c_w_id,h_c_d_id,h_c_id) -> customer(c_w_id,c_d_id,c_id) │ └── anti-join (lookup customer) - │ ├── columns: column3:20!null column2:21!null column1:22!null - │ ├── key columns: [20 21 22] = [25 24 23] + │ ├── columns: column3:21!null column2:22!null column1:23!null + │ ├── key columns: [21 22 23] = [26 25 24] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(20-22) + │ ├── fd: ()-->(21-23) │ ├── with-scan &1 - │ │ ├── columns: column3:20!null column2:21!null column1:22!null + │ │ ├── columns: column3:21!null column2:22!null column1:23!null │ │ ├── mapping: - │ │ │ ├── column3:12 => column3:20 - │ │ │ ├── column2:11 => column2:21 - │ │ │ └── column1:10 => column1:22 + │ │ │ ├── column3:13 => column3:21 + │ │ │ ├── column2:12 => column2:22 + │ │ │ └── column1:11 => column1:23 │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(20-22) + │ │ └── fd: ()-->(21-23) │ └── filters (true) └── f-k-checks-item: history(h_w_id,h_d_id) -> district(d_w_id,d_id) └── anti-join (lookup district) - ├── columns: column5:44!null column4:45!null - ├── key columns: [44 45] = [47 46] + ├── columns: column5:46!null column4:47!null + ├── key columns: [46 47] = [49 48] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(44,45) + ├── fd: ()-->(46,47) ├── with-scan &1 - │ ├── columns: column5:44!null column4:45!null + │ ├── columns: column5:46!null column4:47!null │ ├── mapping: - │ │ ├── column5:14 => column5:44 - │ │ └── column4:13 => column4:45 + │ │ ├── column5:15 => column5:46 + │ │ └── column4:14 => column4:47 │ ├── cardinality: [1 - 1] │ ├── key: () - │ └── fd: ()-->(44,45) + │ └── fd: ()-->(46,47) └── filters (true) # -------------------------------------------------- @@ -780,16 +780,16 @@ FROM order_line WHERE ol_w_id = 10 AND ol_d_id = 100 AND ol_o_id = 1000 ---- scalar-group-by - ├── columns: sum:11 + ├── columns: sum:12 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(11) + ├── fd: ()-->(12) ├── scan order_line │ ├── columns: ol_o_id:1!null ol_d_id:2!null ol_w_id:3!null ol_amount:9 │ ├── constraint: /3/2/-1/4: [/10/100/1000 - /10/100/1000] │ └── fd: ()-->(1-3) └── aggregations - └── sum [as=sum:11, outer=(9)] + └── sum [as=sum:12, outer=(9)] └── ol_amount:9 opt format=hide-qual @@ -819,21 +819,21 @@ project ├── fd: (2)-->(4) └── update "order" ├── columns: o_id:1!null o_d_id:2!null o_w_id:3!null o_c_id:4 - ├── fetch columns: o_id:9 o_d_id:10 o_w_id:11 o_c_id:12 o_entry_d:13 o_carrier_id:14 o_ol_cnt:15 o_all_local:16 + ├── fetch columns: o_id:10 o_d_id:11 o_w_id:12 o_c_id:13 o_entry_d:14 o_carrier_id:15 o_ol_cnt:16 o_all_local:17 ├── update-mapping: - │ └── o_carrier_id_new:17 => o_carrier_id:6 + │ └── o_carrier_id_new:19 => o_carrier_id:6 ├── cardinality: [0 - 10] ├── volatile, mutations ├── key: (2) ├── fd: ()-->(1,3), (2)-->(4) └── project - ├── columns: o_carrier_id_new:17!null o_id:9!null o_d_id:10!null o_w_id:11!null o_c_id:12 o_entry_d:13 o_carrier_id:14 o_ol_cnt:15 o_all_local:16 + ├── columns: o_carrier_id_new:19!null o_id:10!null o_d_id:11!null o_w_id:12!null o_c_id:13 o_entry_d:14 o_carrier_id:15 o_ol_cnt:16 o_all_local:17 ├── cardinality: [0 - 10] - ├── key: (10) - ├── fd: ()-->(9,11,17), (10)-->(12-16) + ├── key: (11) + ├── fd: ()-->(10,12,19), (11)-->(13-17) ├── scan "order" - │ ├── columns: o_id:9!null o_d_id:10!null o_w_id:11!null o_c_id:12 o_entry_d:13 o_carrier_id:14 o_ol_cnt:15 o_all_local:16 - │ ├── constraint: /11/10/-9 + │ ├── columns: o_id:10!null o_d_id:11!null o_w_id:12!null o_c_id:13 o_entry_d:14 o_carrier_id:15 o_ol_cnt:16 o_all_local:17 + │ ├── constraint: /12/11/-10 │ │ ├── [/10/1/2167 - /10/1/2167] │ │ ├── [/10/2/2167 - /10/2/2167] │ │ ├── [/10/3/2167 - /10/3/2167] @@ -845,10 +845,10 @@ project │ │ ├── [/10/9/2167 - /10/9/2167] │ │ └── [/10/10/2167 - /10/10/2167] │ ├── cardinality: [0 - 10] - │ ├── key: (10) - │ └── fd: ()-->(9,11), (10)-->(12-16) + │ ├── key: (11) + │ └── fd: ()-->(10,12), (11)-->(13-17) └── projections - └── 10 [as=o_carrier_id_new:17] + └── 10 [as=o_carrier_id_new:19] opt format=hide-qual UPDATE customer @@ -880,21 +880,21 @@ WHERE c_w_id = 10 AND (c_d_id, c_id) IN ( ---- update customer ├── columns: - ├── fetch columns: c_id:22 c_d_id:23 c_w_id:24 c_first:25 c_middle:26 c_last:27 c_street_1:28 c_street_2:29 c_city:30 c_state:31 c_zip:32 c_phone:33 c_since:34 c_credit:35 c_credit_lim:36 c_discount:37 customer.c_balance:38 c_ytd_payment:39 c_payment_cnt:40 c_delivery_cnt:41 c_data:42 + ├── fetch columns: c_id:23 c_d_id:24 c_w_id:25 c_first:26 c_middle:27 c_last:28 c_street_1:29 c_street_2:30 c_city:31 c_state:32 c_zip:33 c_phone:34 c_since:35 c_credit:36 c_credit_lim:37 c_discount:38 customer.c_balance:39 c_ytd_payment:40 c_payment_cnt:41 c_delivery_cnt:42 c_data:43 ├── update-mapping: - │ ├── c_balance:45 => customer.c_balance:17 - │ └── c_delivery_cnt_new:43 => c_delivery_cnt:20 + │ ├── c_balance:47 => customer.c_balance:17 + │ └── c_delivery_cnt_new:45 => c_delivery_cnt:20 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: c_balance:45 c_delivery_cnt_new:43 c_id:22!null c_d_id:23!null c_w_id:24!null c_first:25 c_middle:26 c_last:27 c_street_1:28 c_street_2:29 c_city:30 c_state:31 c_zip:32 c_phone:33 c_since:34 c_credit:35 c_credit_lim:36 c_discount:37 customer.c_balance:38 c_ytd_payment:39 c_payment_cnt:40 c_delivery_cnt:41 c_data:42 + ├── columns: c_balance:47 c_delivery_cnt_new:45 c_id:23!null c_d_id:24!null c_w_id:25!null c_first:26 c_middle:27 c_last:28 c_street_1:29 c_street_2:30 c_city:31 c_state:32 c_zip:33 c_phone:34 c_since:35 c_credit:36 c_credit_lim:37 c_discount:38 customer.c_balance:39 c_ytd_payment:40 c_payment_cnt:41 c_delivery_cnt:42 c_data:43 ├── cardinality: [0 - 10] ├── immutable - ├── key: (22,23) - ├── fd: ()-->(24), (22,23)-->(25-42,45), (41)-->(43) + ├── key: (23,24) + ├── fd: ()-->(25), (23,24)-->(26-43,47), (42)-->(45) ├── scan customer - │ ├── columns: c_id:22!null c_d_id:23!null c_w_id:24!null c_first:25 c_middle:26 c_last:27 c_street_1:28 c_street_2:29 c_city:30 c_state:31 c_zip:32 c_phone:33 c_since:34 c_credit:35 c_credit_lim:36 c_discount:37 customer.c_balance:38 c_ytd_payment:39 c_payment_cnt:40 c_delivery_cnt:41 c_data:42 - │ ├── constraint: /24/23/22 + │ ├── columns: c_id:23!null c_d_id:24!null c_w_id:25!null c_first:26 c_middle:27 c_last:28 c_street_1:29 c_street_2:30 c_city:31 c_state:32 c_zip:33 c_phone:34 c_since:35 c_credit:36 c_credit_lim:37 c_discount:38 customer.c_balance:39 c_ytd_payment:40 c_payment_cnt:41 c_delivery_cnt:42 c_data:43 + │ ├── constraint: /25/24/23 │ │ ├── [/10/1/1405 - /10/1/1405] │ │ ├── [/10/2/137 - /10/2/137] │ │ ├── [/10/3/309 - /10/3/309] @@ -906,11 +906,11 @@ update customer │ │ ├── [/10/9/1412 - /10/9/1412] │ │ └── [/10/10/417 - /10/10/417] │ ├── cardinality: [0 - 10] - │ ├── key: (22,23) - │ └── fd: ()-->(24), (22,23)-->(25-42) + │ ├── key: (23,24) + │ └── fd: ()-->(25), (23,24)-->(26-43) └── projections - ├── crdb_internal.round_decimal_values(customer.c_balance:38::DECIMAL + CASE c_d_id:23 WHEN 6 THEN 57214.780000 WHEN 8 THEN 67755.430000 WHEN 1 THEN 51177.840000 WHEN 2 THEN 73840.700000 WHEN 4 THEN 45906.990000 WHEN 9 THEN 32523.760000 WHEN 10 THEN 20240.200000 WHEN 3 THEN 75299.790000 WHEN 5 THEN 56543.340000 WHEN 7 THEN 67157.940000 ELSE CAST(NULL AS DECIMAL) END, 2) [as=c_balance:45, outer=(23,38), immutable] - └── c_delivery_cnt:41 + 1 [as=c_delivery_cnt_new:43, outer=(41), immutable] + ├── crdb_internal.round_decimal_values(customer.c_balance:39::DECIMAL + CASE c_d_id:24 WHEN 6 THEN 57214.780000 WHEN 8 THEN 67755.430000 WHEN 1 THEN 51177.840000 WHEN 2 THEN 73840.700000 WHEN 4 THEN 45906.990000 WHEN 9 THEN 32523.760000 WHEN 10 THEN 20240.200000 WHEN 3 THEN 75299.790000 WHEN 5 THEN 56543.340000 WHEN 7 THEN 67157.940000 ELSE CAST(NULL AS DECIMAL) END, 2) [as=c_balance:47, outer=(24,39), immutable] + └── c_delivery_cnt:42 + 1 [as=c_delivery_cnt_new:45, outer=(42), immutable] opt format=hide-qual DELETE FROM new_order @@ -929,12 +929,12 @@ WHERE no_w_id = 10 AND (no_d_id, no_o_id) IN ( ---- delete new_order ├── columns: - ├── fetch columns: no_o_id:4 no_d_id:5 no_w_id:6 + ├── fetch columns: no_o_id:5 no_d_id:6 no_w_id:7 ├── cardinality: [0 - 0] ├── volatile, mutations └── scan new_order - ├── columns: no_o_id:4!null no_d_id:5!null no_w_id:6!null - ├── constraint: /6/5/4 + ├── columns: no_o_id:5!null no_d_id:6!null no_w_id:7!null + ├── constraint: /7/6/5 │ ├── [/10/1/2167 - /10/1/2167] │ ├── [/10/2/2167 - /10/2/2167] │ ├── [/10/3/2167 - /10/3/2167] @@ -946,8 +946,8 @@ delete new_order │ ├── [/10/9/2167 - /10/9/2167] │ └── [/10/10/2167 - /10/10/2167] ├── cardinality: [0 - 10] - ├── key: (5) - └── fd: ()-->(4,6) + ├── key: (6) + └── fd: ()-->(5,7) opt format=hide-qual UPDATE order_line @@ -967,18 +967,18 @@ WHERE ol_w_id = 10 AND (ol_d_id, ol_o_id) IN ( ---- update order_line ├── columns: - ├── fetch columns: ol_o_id:11 ol_d_id:12 ol_w_id:13 ol_number:14 ol_i_id:15 ol_supply_w_id:16 ol_delivery_d:17 ol_quantity:18 ol_amount:19 ol_dist_info:20 + ├── fetch columns: ol_o_id:12 ol_d_id:13 ol_w_id:14 ol_number:15 ol_i_id:16 ol_supply_w_id:17 ol_delivery_d:18 ol_quantity:19 ol_amount:20 ol_dist_info:21 ├── update-mapping: - │ └── ol_delivery_d_new:21 => ol_delivery_d:7 + │ └── ol_delivery_d_new:23 => ol_delivery_d:7 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: ol_delivery_d_new:21!null ol_o_id:11!null ol_d_id:12!null ol_w_id:13!null ol_number:14!null ol_i_id:15!null ol_supply_w_id:16 ol_delivery_d:17 ol_quantity:18 ol_amount:19 ol_dist_info:20 - ├── key: (12,14) - ├── fd: ()-->(11,13,21), (12,14)-->(15-20) + ├── columns: ol_delivery_d_new:23!null ol_o_id:12!null ol_d_id:13!null ol_w_id:14!null ol_number:15!null ol_i_id:16!null ol_supply_w_id:17 ol_delivery_d:18 ol_quantity:19 ol_amount:20 ol_dist_info:21 + ├── key: (13,15) + ├── fd: ()-->(12,14,23), (13,15)-->(16-21) ├── scan order_line - │ ├── columns: ol_o_id:11!null ol_d_id:12!null ol_w_id:13!null ol_number:14!null ol_i_id:15!null ol_supply_w_id:16 ol_delivery_d:17 ol_quantity:18 ol_amount:19 ol_dist_info:20 - │ ├── constraint: /13/12/-11/14 + │ ├── columns: ol_o_id:12!null ol_d_id:13!null ol_w_id:14!null ol_number:15!null ol_i_id:16!null ol_supply_w_id:17 ol_delivery_d:18 ol_quantity:19 ol_amount:20 ol_dist_info:21 + │ ├── constraint: /14/13/-12/15 │ │ ├── [/10/1/2167 - /10/1/2167] │ │ ├── [/10/2/2167 - /10/2/2167] │ │ ├── [/10/3/2167 - /10/3/2167] @@ -989,10 +989,10 @@ update order_line │ │ ├── [/10/8/2167 - /10/8/2167] │ │ ├── [/10/9/2167 - /10/9/2167] │ │ └── [/10/10/2167 - /10/10/2167] - │ ├── key: (12,14) - │ └── fd: ()-->(11,13), (12,14)-->(15-20) + │ ├── key: (13,15) + │ └── fd: ()-->(12,14), (13,15)-->(16-21) └── projections - └── '2019-08-26 16:50:41+00:00' [as=ol_delivery_d_new:21] + └── '2019-08-26 16:50:41+00:00' [as=ol_delivery_d_new:23] # -------------------------------------------------- # 2.8 The Stock-Level Transaction @@ -1030,28 +1030,28 @@ WHERE ol_w_id = 10 AND s_quantity < 15 ---- scalar-group-by - ├── columns: count:28!null + ├── columns: count:30!null ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(28) + ├── fd: ()-->(30) ├── distinct-on - │ ├── columns: s_i_id:11!null - │ ├── grouping columns: s_i_id:11!null - │ ├── key: (11) + │ ├── columns: s_i_id:12!null + │ ├── grouping columns: s_i_id:12!null + │ ├── key: (12) │ └── inner-join (lookup stock) - │ ├── columns: ol_o_id:1!null ol_d_id:2!null ol_w_id:3!null ol_i_id:5!null s_i_id:11!null s_w_id:12!null s_quantity:13!null - │ ├── key columns: [3 5] = [12 11] + │ ├── columns: ol_o_id:1!null ol_d_id:2!null ol_w_id:3!null ol_i_id:5!null s_i_id:12!null s_w_id:13!null s_quantity:14!null + │ ├── key columns: [3 5] = [13 12] │ ├── lookup columns are key - │ ├── fd: ()-->(2,3,12), (11)-->(13), (5)==(11), (11)==(5), (3)==(12), (12)==(3) + │ ├── fd: ()-->(2,3,13), (12)-->(14), (5)==(12), (12)==(5), (3)==(13), (13)==(3) │ ├── scan order_line │ │ ├── columns: ol_o_id:1!null ol_d_id:2!null ol_w_id:3!null ol_i_id:5!null │ │ ├── constraint: /3/2/-1/4: [/10/100/999 - /10/100/980] │ │ └── fd: ()-->(2,3) │ └── filters - │ ├── s_w_id:12 = 10 [outer=(12), constraints=(/12: [/10 - /10]; tight), fd=()-->(12)] - │ └── s_quantity:13 < 15 [outer=(13), constraints=(/13: (/NULL - /14]; tight)] + │ ├── s_w_id:13 = 10 [outer=(13), constraints=(/13: [/10 - /10]; tight), fd=()-->(13)] + │ └── s_quantity:14 < 15 [outer=(14), constraints=(/14: (/NULL - /14]; tight)] └── aggregations - └── count-rows [as=count:28] + └── count-rows [as=count:30] # -------------------------------------------------- # Consistency Queries @@ -1072,39 +1072,39 @@ ON (w_id = d_w_id) WHERE w_ytd != sum_d_ytd ---- scalar-group-by - ├── columns: count:22!null + ├── columns: count:24!null ├── cardinality: [1 - 1] ├── immutable ├── key: () - ├── fd: ()-->(22) + ├── fd: ()-->(24) ├── inner-join (merge) - │ ├── columns: w_id:1!null w_ytd:9!null d_w_id:11!null sum:21!null + │ ├── columns: w_id:1!null w_ytd:9!null d_w_id:12!null sum:23!null │ ├── left ordering: +1 - │ ├── right ordering: +11 + │ ├── right ordering: +12 │ ├── immutable - │ ├── key: (11) - │ ├── fd: (1)-->(9), (11)-->(21), (1)==(11), (11)==(1) + │ ├── key: (12) + │ ├── fd: (1)-->(9), (12)-->(23), (1)==(12), (12)==(1) │ ├── scan warehouse │ │ ├── columns: w_id:1!null w_ytd:9 │ │ ├── key: (1) │ │ ├── fd: (1)-->(9) │ │ └── ordering: +1 │ ├── group-by - │ │ ├── columns: d_w_id:11!null sum:21 - │ │ ├── grouping columns: d_w_id:11!null - │ │ ├── key: (11) - │ │ ├── fd: (11)-->(21) - │ │ ├── ordering: +11 + │ │ ├── columns: d_w_id:12!null sum:23 + │ │ ├── grouping columns: d_w_id:12!null + │ │ ├── key: (12) + │ │ ├── fd: (12)-->(23) + │ │ ├── ordering: +12 │ │ ├── scan district - │ │ │ ├── columns: d_w_id:11!null d_ytd:19 - │ │ │ └── ordering: +11 + │ │ │ ├── columns: d_w_id:12!null d_ytd:20 + │ │ │ └── ordering: +12 │ │ └── aggregations - │ │ └── sum [as=sum:21, outer=(19)] - │ │ └── d_ytd:19 + │ │ └── sum [as=sum:23, outer=(20)] + │ │ └── d_ytd:20 │ └── filters - │ └── w_ytd:9 != sum:21 [outer=(9,21), immutable, constraints=(/9: (/NULL - ]; /21: (/NULL - ])] + │ └── w_ytd:9 != sum:23 [outer=(9,23), immutable, constraints=(/9: (/NULL - ]; /23: (/NULL - ])] └── aggregations - └── count-rows [as=count_rows:22] + └── count-rows [as=count_rows:24] opt format=hide-qual SELECT d_next_o_id @@ -1124,17 +1124,17 @@ GROUP BY no_d_id, no_w_id ORDER BY no_w_id, no_d_id ---- group-by - ├── columns: max:4!null [hidden: no_d_id:2!null no_w_id:3!null] + ├── columns: max:5!null [hidden: no_d_id:2!null no_w_id:3!null] ├── grouping columns: no_d_id:2!null no_w_id:3!null ├── key: (2,3) - ├── fd: (2,3)-->(4) + ├── fd: (2,3)-->(5) ├── ordering: +3,+2 ├── scan new_order │ ├── columns: no_o_id:1!null no_d_id:2!null no_w_id:3!null │ ├── key: (1-3) │ └── ordering: +3,+2 └── aggregations - └── max [as=max:4, outer=(1)] + └── max [as=max:5, outer=(1)] └── no_o_id:1 opt format=hide-qual @@ -1144,17 +1144,17 @@ GROUP BY o_d_id, o_w_id ORDER BY o_w_id, o_d_id ---- group-by - ├── columns: max:9!null [hidden: o_d_id:2!null o_w_id:3!null] + ├── columns: max:10!null [hidden: o_d_id:2!null o_w_id:3!null] ├── grouping columns: o_d_id:2!null o_w_id:3!null ├── key: (2,3) - ├── fd: (2,3)-->(9) + ├── fd: (2,3)-->(10) ├── ordering: +3,+2 ├── scan "order"@order_idx │ ├── columns: o_id:1!null o_d_id:2!null o_w_id:3!null │ ├── key: (1-3) │ └── ordering: +3,+2 └── aggregations - └── max [as=max:9, outer=(1)] + └── max [as=max:10, outer=(1)] └── o_id:1 opt format=hide-qual @@ -1168,36 +1168,36 @@ FROM WHERE nod != -1 ---- scalar-group-by - ├── columns: count:8!null + ├── columns: count:9!null ├── cardinality: [1 - 1] ├── immutable ├── key: () - ├── fd: ()-->(8) + ├── fd: ()-->(9) ├── select - │ ├── columns: no_d_id:2!null no_w_id:3!null max:4!null min:5!null count_rows:6!null + │ ├── columns: no_d_id:2!null no_w_id:3!null max:5!null min:6!null count_rows:7!null │ ├── immutable │ ├── key: (2,3) - │ ├── fd: (2,3)-->(4-6) + │ ├── fd: (2,3)-->(5-7) │ ├── group-by - │ │ ├── columns: no_d_id:2!null no_w_id:3!null max:4!null min:5!null count_rows:6!null + │ │ ├── columns: no_d_id:2!null no_w_id:3!null max:5!null min:6!null count_rows:7!null │ │ ├── grouping columns: no_d_id:2!null no_w_id:3!null │ │ ├── internal-ordering: +3,+2 │ │ ├── key: (2,3) - │ │ ├── fd: (2,3)-->(4-6) + │ │ ├── fd: (2,3)-->(5-7) │ │ ├── scan new_order │ │ │ ├── columns: no_o_id:1!null no_d_id:2!null no_w_id:3!null │ │ │ ├── key: (1-3) │ │ │ └── ordering: +3,+2 │ │ └── aggregations - │ │ ├── max [as=max:4, outer=(1)] + │ │ ├── max [as=max:5, outer=(1)] │ │ │ └── no_o_id:1 - │ │ ├── min [as=min:5, outer=(1)] + │ │ ├── min [as=min:6, outer=(1)] │ │ │ └── no_o_id:1 - │ │ └── count-rows [as=count_rows:6] + │ │ └── count-rows [as=count_rows:7] │ └── filters - │ └── ((max:4 - min:5) - count_rows:6) != -1 [outer=(4-6), immutable] + │ └── ((max:5 - min:6) - count_rows:7) != -1 [outer=(5-7), immutable] └── aggregations - └── count-rows [as=count_rows:8] + └── count-rows [as=count_rows:9] opt format=hide-qual SELECT sum(o_ol_cnt) @@ -1206,16 +1206,16 @@ GROUP BY o_w_id, o_d_id ORDER BY o_w_id, o_d_id ---- group-by - ├── columns: sum:9 [hidden: o_d_id:2!null o_w_id:3!null] + ├── columns: sum:10 [hidden: o_d_id:2!null o_w_id:3!null] ├── grouping columns: o_d_id:2!null o_w_id:3!null ├── key: (2,3) - ├── fd: (2,3)-->(9) + ├── fd: (2,3)-->(10) ├── ordering: +3,+2 ├── scan "order" │ ├── columns: o_d_id:2!null o_w_id:3!null o_ol_cnt:7 │ └── ordering: +3,+2 └── aggregations - └── sum [as=sum:9, outer=(7)] + └── sum [as=sum:10, outer=(7)] └── o_ol_cnt:7 opt format=hide-qual @@ -1225,19 +1225,19 @@ GROUP BY ol_w_id, ol_d_id ORDER BY ol_w_id, ol_d_id ---- sort - ├── columns: count:11!null [hidden: ol_d_id:2!null ol_w_id:3!null] + ├── columns: count:12!null [hidden: ol_d_id:2!null ol_w_id:3!null] ├── key: (2,3) - ├── fd: (2,3)-->(11) + ├── fd: (2,3)-->(12) ├── ordering: +3,+2 └── group-by - ├── columns: ol_d_id:2!null ol_w_id:3!null count_rows:11!null + ├── columns: ol_d_id:2!null ol_w_id:3!null count_rows:12!null ├── grouping columns: ol_d_id:2!null ol_w_id:3!null ├── key: (2,3) - ├── fd: (2,3)-->(11) + ├── fd: (2,3)-->(12) ├── scan order_line@order_line_stock_fk_idx │ └── columns: ol_d_id:2!null ol_w_id:3!null └── aggregations - └── count-rows [as=count_rows:11] + └── count-rows [as=count_rows:12] opt format=hide-qual (SELECT no_w_id, no_d_id, no_o_id FROM new_order) @@ -1247,23 +1247,23 @@ EXCEPT ALL except-all ├── columns: no_w_id:3!null no_d_id:2!null no_o_id:1!null ├── left columns: no_w_id:3!null no_d_id:2!null no_o_id:1!null - ├── right columns: o_w_id:6 o_d_id:5 o_id:4 + ├── right columns: o_w_id:7 o_d_id:6 o_id:5 ├── scan new_order │ ├── columns: no_o_id:1!null no_d_id:2!null no_w_id:3!null │ └── key: (1-3) └── project - ├── columns: o_id:4!null o_d_id:5!null o_w_id:6!null - ├── key: (4-6) + ├── columns: o_id:5!null o_d_id:6!null o_w_id:7!null + ├── key: (5-7) └── select - ├── columns: o_id:4!null o_d_id:5!null o_w_id:6!null o_carrier_id:9 - ├── key: (4-6) - ├── fd: ()-->(9) + ├── columns: o_id:5!null o_d_id:6!null o_w_id:7!null o_carrier_id:10 + ├── key: (5-7) + ├── fd: ()-->(10) ├── scan "order"@order_idx - │ ├── columns: o_id:4!null o_d_id:5!null o_w_id:6!null o_carrier_id:9 - │ ├── key: (4-6) - │ └── fd: (4-6)-->(9) + │ ├── columns: o_id:5!null o_d_id:6!null o_w_id:7!null o_carrier_id:10 + │ ├── key: (5-7) + │ └── fd: (5-7)-->(10) └── filters - └── o_carrier_id:9 IS NULL [outer=(9), constraints=(/9: [/NULL - /NULL]; tight), fd=()-->(9)] + └── o_carrier_id:10 IS NULL [outer=(10), constraints=(/10: [/NULL - /NULL]; tight), fd=()-->(10)] opt format=hide-qual (SELECT o_w_id, o_d_id, o_id FROM "order" WHERE o_carrier_id IS NULL) @@ -1273,7 +1273,7 @@ EXCEPT ALL except-all ├── columns: o_w_id:3!null o_d_id:2!null o_id:1!null ├── left columns: o_w_id:3!null o_d_id:2!null o_id:1!null - ├── right columns: no_w_id:11 no_d_id:10 no_o_id:9 + ├── right columns: no_w_id:12 no_d_id:11 no_o_id:10 ├── project │ ├── columns: o_id:1!null o_d_id:2!null o_w_id:3!null │ ├── key: (1-3) @@ -1288,8 +1288,8 @@ except-all │ └── filters │ └── o_carrier_id:6 IS NULL [outer=(6), constraints=(/6: [/NULL - /NULL]; tight), fd=()-->(6)] └── scan new_order - ├── columns: no_o_id:9!null no_d_id:10!null no_w_id:11!null - └── key: (9-11) + ├── columns: no_o_id:10!null no_d_id:11!null no_w_id:12!null + └── key: (10-12) opt format=hide-qual ( @@ -1308,20 +1308,20 @@ EXCEPT ALL except-all ├── columns: o_w_id:3!null o_d_id:2!null o_id:1!null o_ol_cnt:7 ├── left columns: o_w_id:3!null o_d_id:2!null o_id:1!null o_ol_cnt:7 - ├── right columns: ol_w_id:11 ol_d_id:10 ol_o_id:9 count_rows:19 + ├── right columns: ol_w_id:12 ol_d_id:11 ol_o_id:10 count_rows:21 ├── scan "order" │ ├── columns: o_id:1!null o_d_id:2!null o_w_id:3!null o_ol_cnt:7 │ ├── key: (1-3) │ └── fd: (1-3)-->(7) └── group-by - ├── columns: ol_o_id:9!null ol_d_id:10!null ol_w_id:11!null count_rows:19!null - ├── grouping columns: ol_o_id:9!null ol_d_id:10!null ol_w_id:11!null - ├── key: (9-11) - ├── fd: (9-11)-->(19) + ├── columns: ol_o_id:10!null ol_d_id:11!null ol_w_id:12!null count_rows:21!null + ├── grouping columns: ol_o_id:10!null ol_d_id:11!null ol_w_id:12!null + ├── key: (10-12) + ├── fd: (10-12)-->(21) ├── scan order_line@order_line_stock_fk_idx - │ └── columns: ol_o_id:9!null ol_d_id:10!null ol_w_id:11!null + │ └── columns: ol_o_id:10!null ol_d_id:11!null ol_w_id:12!null └── aggregations - └── count-rows [as=count_rows:19] + └── count-rows [as=count_rows:21] opt format=hide-qual ( @@ -1338,22 +1338,22 @@ EXCEPT ALL ) ---- except-all - ├── columns: ol_w_id:3!null ol_d_id:2!null ol_o_id:1!null count:11 - ├── left columns: ol_w_id:3!null ol_d_id:2!null ol_o_id:1!null count_rows:11 - ├── right columns: o_w_id:14 o_d_id:13 o_id:12 o_ol_cnt:18 + ├── columns: ol_w_id:3!null ol_d_id:2!null ol_o_id:1!null count:12 + ├── left columns: ol_w_id:3!null ol_d_id:2!null ol_o_id:1!null count_rows:12 + ├── right columns: o_w_id:15 o_d_id:14 o_id:13 o_ol_cnt:19 ├── group-by - │ ├── columns: ol_o_id:1!null ol_d_id:2!null ol_w_id:3!null count_rows:11!null + │ ├── columns: ol_o_id:1!null ol_d_id:2!null ol_w_id:3!null count_rows:12!null │ ├── grouping columns: ol_o_id:1!null ol_d_id:2!null ol_w_id:3!null │ ├── key: (1-3) - │ ├── fd: (1-3)-->(11) + │ ├── fd: (1-3)-->(12) │ ├── scan order_line@order_line_stock_fk_idx │ │ └── columns: ol_o_id:1!null ol_d_id:2!null ol_w_id:3!null │ └── aggregations - │ └── count-rows [as=count_rows:11] + │ └── count-rows [as=count_rows:12] └── scan "order" - ├── columns: o_id:12!null o_d_id:13!null o_w_id:14!null o_ol_cnt:18 - ├── key: (12-14) - └── fd: (12-14)-->(18) + ├── columns: o_id:13!null o_d_id:14!null o_w_id:15!null o_ol_cnt:19 + ├── key: (13-15) + └── fd: (13-15)-->(19) opt format=hide-qual SELECT count(*) @@ -1373,24 +1373,24 @@ ON (ol_w_id = o_w_id AND ol_d_id = o_d_id AND ol_o_id = o_id) WHERE ol_o_id IS NULL OR o_id IS NULL ---- scalar-group-by - ├── columns: count:19!null + ├── columns: count:21!null ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(19) + ├── fd: ()-->(21) ├── select - │ ├── columns: o_id:1 o_d_id:2 o_w_id:3 ol_o_id:9 ol_d_id:10 ol_w_id:11 + │ ├── columns: o_id:1 o_d_id:2 o_w_id:3 ol_o_id:10 ol_d_id:11 ol_w_id:12 │ ├── full-join (hash) - │ │ ├── columns: o_id:1 o_d_id:2 o_w_id:3 ol_o_id:9 ol_d_id:10 ol_w_id:11 + │ │ ├── columns: o_id:1 o_d_id:2 o_w_id:3 ol_o_id:10 ol_d_id:11 ol_w_id:12 │ │ ├── multiplicity: left-rows(exactly-one), right-rows(one-or-more) │ │ ├── project - │ │ │ ├── columns: ol_o_id:9!null ol_d_id:10!null ol_w_id:11!null + │ │ │ ├── columns: ol_o_id:10!null ol_d_id:11!null ol_w_id:12!null │ │ │ └── select - │ │ │ ├── columns: ol_o_id:9!null ol_d_id:10!null ol_w_id:11!null ol_delivery_d:15 - │ │ │ ├── fd: ()-->(15) + │ │ │ ├── columns: ol_o_id:10!null ol_d_id:11!null ol_w_id:12!null ol_delivery_d:16 + │ │ │ ├── fd: ()-->(16) │ │ │ ├── scan order_line - │ │ │ │ └── columns: ol_o_id:9!null ol_d_id:10!null ol_w_id:11!null ol_delivery_d:15 + │ │ │ │ └── columns: ol_o_id:10!null ol_d_id:11!null ol_w_id:12!null ol_delivery_d:16 │ │ │ └── filters - │ │ │ └── ol_delivery_d:15 IS NULL [outer=(15), constraints=(/15: [/NULL - /NULL]; tight), fd=()-->(15)] + │ │ │ └── ol_delivery_d:16 IS NULL [outer=(16), constraints=(/16: [/NULL - /NULL]; tight), fd=()-->(16)] │ │ ├── project │ │ │ ├── columns: o_id:1!null o_d_id:2!null o_w_id:3!null │ │ │ ├── key: (1-3) @@ -1405,10 +1405,10 @@ scalar-group-by │ │ │ └── filters │ │ │ └── o_carrier_id:6 IS NULL [outer=(6), constraints=(/6: [/NULL - /NULL]; tight), fd=()-->(6)] │ │ └── filters - │ │ ├── ol_w_id:11 = o_w_id:3 [outer=(3,11), constraints=(/3: (/NULL - ]; /11: (/NULL - ]), fd=(3)==(11), (11)==(3)] - │ │ ├── ol_d_id:10 = o_d_id:2 [outer=(2,10), constraints=(/2: (/NULL - ]; /10: (/NULL - ]), fd=(2)==(10), (10)==(2)] - │ │ └── ol_o_id:9 = o_id:1 [outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ]), fd=(1)==(9), (9)==(1)] + │ │ ├── ol_w_id:12 = o_w_id:3 [outer=(3,12), constraints=(/3: (/NULL - ]; /12: (/NULL - ]), fd=(3)==(12), (12)==(3)] + │ │ ├── ol_d_id:11 = o_d_id:2 [outer=(2,11), constraints=(/2: (/NULL - ]; /11: (/NULL - ]), fd=(2)==(11), (11)==(2)] + │ │ └── ol_o_id:10 = o_id:1 [outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] │ └── filters - │ └── (ol_o_id:9 IS NULL) OR (o_id:1 IS NULL) [outer=(1,9)] + │ └── (ol_o_id:10 IS NULL) OR (o_id:1 IS NULL) [outer=(1,10)] └── aggregations - └── count-rows [as=count_rows:19] + └── count-rows [as=count_rows:21] diff --git a/pkg/sql/opt/xform/testdata/external/tpcc-no-stats b/pkg/sql/opt/xform/testdata/external/tpcc-no-stats index e1de1c8dd57e..8d30a2fc8751 100644 --- a/pkg/sql/opt/xform/testdata/external/tpcc-no-stats +++ b/pkg/sql/opt/xform/testdata/external/tpcc-no-stats @@ -26,27 +26,27 @@ project ├── fd: ()-->(9,11) └── update district ├── columns: d_id:1!null d_w_id:2!null d_tax:9 d_next_o_id:11 - ├── fetch columns: d_id:12 d_w_id:13 d_name:14 d_street_1:15 d_street_2:16 d_city:17 d_state:18 d_zip:19 d_tax:20 d_ytd:21 d_next_o_id:22 + ├── fetch columns: d_id:13 d_w_id:14 d_name:15 d_street_1:16 d_street_2:17 d_city:18 d_state:19 d_zip:20 d_tax:21 d_ytd:22 d_next_o_id:23 ├── update-mapping: - │ └── d_next_o_id_new:23 => d_next_o_id:11 + │ └── d_next_o_id_new:25 => d_next_o_id:11 ├── cardinality: [0 - 1] ├── volatile, mutations ├── key: () ├── fd: ()-->(1,2,9,11) └── project - ├── columns: d_next_o_id_new:23 d_id:12!null d_w_id:13!null d_name:14 d_street_1:15 d_street_2:16 d_city:17 d_state:18 d_zip:19 d_tax:20 d_ytd:21 d_next_o_id:22 + ├── columns: d_next_o_id_new:25 d_id:13!null d_w_id:14!null d_name:15 d_street_1:16 d_street_2:17 d_city:18 d_state:19 d_zip:20 d_tax:21 d_ytd:22 d_next_o_id:23 ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(12-23) + ├── fd: ()-->(13-23,25) ├── scan district - │ ├── columns: d_id:12!null d_w_id:13!null d_name:14 d_street_1:15 d_street_2:16 d_city:17 d_state:18 d_zip:19 d_tax:20 d_ytd:21 d_next_o_id:22 - │ ├── constraint: /13/12: [/10/5 - /10/5] + │ ├── columns: d_id:13!null d_w_id:14!null d_name:15 d_street_1:16 d_street_2:17 d_city:18 d_state:19 d_zip:20 d_tax:21 d_ytd:22 d_next_o_id:23 + │ ├── constraint: /14/13: [/10/5 - /10/5] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(12-22) + │ └── fd: ()-->(13-23) └── projections - └── d_next_o_id:22 + 1 [as=d_next_o_id_new:23, outer=(22), immutable] + └── d_next_o_id:23 + 1 [as=d_next_o_id_new:25, outer=(23), immutable] opt format=hide-qual SELECT w_tax FROM warehouse WHERE w_id = 10 @@ -138,41 +138,41 @@ VALUES (100, 5, 10, 50, '2019-08-26 16:50:41', 10, 1) insert "order" ├── columns: ├── insert-mapping: - │ ├── column1:9 => o_id:1 - │ ├── column2:10 => o_d_id:2 - │ ├── column3:11 => o_w_id:3 - │ ├── column4:12 => o_c_id:4 - │ ├── column5:13 => o_entry_d:5 - │ ├── column16:16 => o_carrier_id:6 - │ ├── column6:14 => o_ol_cnt:7 - │ └── column7:15 => o_all_local:8 + │ ├── column1:10 => o_id:1 + │ ├── column2:11 => o_d_id:2 + │ ├── column3:12 => o_w_id:3 + │ ├── column4:13 => o_c_id:4 + │ ├── column5:14 => o_entry_d:5 + │ ├── column17:17 => o_carrier_id:6 + │ ├── column6:15 => o_ol_cnt:7 + │ └── column7:16 => o_all_local:8 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── values - │ ├── columns: column1:9!null column2:10!null column3:11!null column4:12!null column5:13!null column6:14!null column7:15!null column16:16 + │ ├── columns: column1:10!null column2:11!null column3:12!null column4:13!null column5:14!null column6:15!null column7:16!null column17:17 │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(9-16) + │ ├── fd: ()-->(10-17) │ └── (100, 5, 10, 50, '2019-08-26 16:50:41+00:00', 10, 1, NULL) └── f-k-checks └── f-k-checks-item: order(o_w_id,o_d_id,o_c_id) -> customer(c_w_id,c_d_id,c_id) └── anti-join (lookup customer) - ├── columns: column3:17!null column2:18!null column4:19!null - ├── key columns: [17 18 19] = [22 21 20] + ├── columns: column3:18!null column2:19!null column4:20!null + ├── key columns: [18 19 20] = [23 22 21] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(17-19) + ├── fd: ()-->(18-20) ├── with-scan &1 - │ ├── columns: column3:17!null column2:18!null column4:19!null + │ ├── columns: column3:18!null column2:19!null column4:20!null │ ├── mapping: - │ │ ├── column3:11 => column3:17 - │ │ ├── column2:10 => column2:18 - │ │ └── column4:12 => column4:19 + │ │ ├── column3:12 => column3:18 + │ │ ├── column2:11 => column2:19 + │ │ └── column4:13 => column4:20 │ ├── cardinality: [1 - 1] │ ├── key: () - │ └── fd: ()-->(17-19) + │ └── fd: ()-->(18-20) └── filters (true) opt format=hide-qual @@ -181,36 +181,36 @@ INSERT INTO new_order (no_o_id, no_d_id, no_w_id) VALUES (2000, 100, 10) insert new_order ├── columns: ├── insert-mapping: - │ ├── column1:4 => no_o_id:1 - │ ├── column2:5 => no_d_id:2 - │ └── column3:6 => no_w_id:3 + │ ├── column1:5 => no_o_id:1 + │ ├── column2:6 => no_d_id:2 + │ └── column3:7 => no_w_id:3 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── values - │ ├── columns: column1:4!null column2:5!null column3:6!null + │ ├── columns: column1:5!null column2:6!null column3:7!null │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(4-6) + │ ├── fd: ()-->(5-7) │ └── (2000, 100, 10) └── f-k-checks └── f-k-checks-item: new_order(no_w_id,no_d_id,no_o_id) -> order(o_w_id,o_d_id,o_id) └── anti-join (lookup order) - ├── columns: column3:7!null column2:8!null column1:9!null - ├── key columns: [7 8 9] = [12 11 10] + ├── columns: column3:8!null column2:9!null column1:10!null + ├── key columns: [8 9 10] = [13 12 11] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(7-9) + ├── fd: ()-->(8-10) ├── with-scan &1 - │ ├── columns: column3:7!null column2:8!null column1:9!null + │ ├── columns: column3:8!null column2:9!null column1:10!null │ ├── mapping: - │ │ ├── column3:6 => column3:7 - │ │ ├── column2:5 => column2:8 - │ │ └── column1:4 => column1:9 + │ │ ├── column3:7 => column3:8 + │ │ ├── column2:6 => column2:9 + │ │ └── column1:5 => column1:10 │ ├── cardinality: [1 - 1] │ ├── key: () - │ └── fd: ()-->(7-9) + │ └── fd: ()-->(8-10) └── filters (true) opt format=hide-qual @@ -302,23 +302,23 @@ WHERE ---- update stock ├── columns: - ├── fetch columns: s_i_id:18 s_w_id:19 s_quantity:20 s_dist_01:21 s_dist_02:22 s_dist_03:23 s_dist_04:24 s_dist_05:25 s_dist_06:26 s_dist_07:27 s_dist_08:28 s_dist_09:29 s_dist_10:30 s_ytd:31 s_order_cnt:32 s_remote_cnt:33 s_data:34 + ├── fetch columns: s_i_id:19 s_w_id:20 s_quantity:21 s_dist_01:22 s_dist_02:23 s_dist_03:24 s_dist_04:25 s_dist_05:26 s_dist_06:27 s_dist_07:28 s_dist_08:29 s_dist_09:30 s_dist_10:31 s_ytd:32 s_order_cnt:33 s_remote_cnt:34 s_data:35 ├── update-mapping: - │ ├── s_quantity_new:35 => s_quantity:3 - │ ├── s_ytd_new:36 => s_ytd:14 - │ ├── s_order_cnt_new:37 => s_order_cnt:15 - │ └── s_remote_cnt_new:38 => s_remote_cnt:16 + │ ├── s_quantity_new:37 => s_quantity:3 + │ ├── s_ytd_new:38 => s_ytd:14 + │ ├── s_order_cnt_new:39 => s_order_cnt:15 + │ └── s_remote_cnt_new:40 => s_remote_cnt:16 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: s_quantity_new:35 s_ytd_new:36 s_order_cnt_new:37 s_remote_cnt_new:38 s_i_id:18!null s_w_id:19!null s_quantity:20 s_dist_01:21 s_dist_02:22 s_dist_03:23 s_dist_04:24 s_dist_05:25 s_dist_06:26 s_dist_07:27 s_dist_08:28 s_dist_09:29 s_dist_10:30 s_ytd:31 s_order_cnt:32 s_remote_cnt:33 s_data:34 + ├── columns: s_quantity_new:37 s_ytd_new:38 s_order_cnt_new:39 s_remote_cnt_new:40 s_i_id:19!null s_w_id:20!null s_quantity:21 s_dist_01:22 s_dist_02:23 s_dist_03:24 s_dist_04:25 s_dist_05:26 s_dist_06:27 s_dist_07:28 s_dist_08:29 s_dist_09:30 s_dist_10:31 s_ytd:32 s_order_cnt:33 s_remote_cnt:34 s_data:35 ├── cardinality: [0 - 13] ├── volatile - ├── key: (18) - ├── fd: ()-->(19), (18)-->(20-38) + ├── key: (19) + ├── fd: ()-->(20), (19)-->(21-35,37-40) ├── scan stock - │ ├── columns: s_i_id:18!null s_w_id:19!null s_quantity:20 s_dist_01:21 s_dist_02:22 s_dist_03:23 s_dist_04:24 s_dist_05:25 s_dist_06:26 s_dist_07:27 s_dist_08:28 s_dist_09:29 s_dist_10:30 s_ytd:31 s_order_cnt:32 s_remote_cnt:33 s_data:34 - │ ├── constraint: /19/18 + │ ├── columns: s_i_id:19!null s_w_id:20!null s_quantity:21 s_dist_01:22 s_dist_02:23 s_dist_03:24 s_dist_04:25 s_dist_05:26 s_dist_06:27 s_dist_07:28 s_dist_08:29 s_dist_09:30 s_dist_10:31 s_ytd:32 s_order_cnt:33 s_remote_cnt:34 s_data:35 + │ ├── constraint: /20/19 │ │ ├── [/0/6823 - /0/6823] │ │ ├── [/0/7853 - /0/7853] │ │ ├── [/0/8497 - /0/8497] @@ -333,13 +333,13 @@ update stock │ │ ├── [/0/89641 - /0/89641] │ │ └── [/0/89905 - /0/89905] │ ├── cardinality: [0 - 13] - │ ├── key: (18) - │ └── fd: ()-->(19), (18)-->(20-34) + │ ├── key: (19) + │ └── fd: ()-->(20), (19)-->(21-35) └── projections - ├── CASE (s_i_id:18, s_w_id:19) WHEN (6823, 0) THEN 26 WHEN (7853, 0) THEN 10 WHEN (8497, 0) THEN 62 WHEN (10904, 0) THEN 54 WHEN (16152, 0) THEN 80 WHEN (41382, 0) THEN 18 WHEN (55952, 0) THEN 56 WHEN (64817, 0) THEN 26 WHEN (66335, 0) THEN 30 WHEN (76567, 0) THEN 71 WHEN (81680, 0) THEN 51 WHEN (89641, 0) THEN 51 WHEN (89905, 0) THEN 77 ELSE crdb_internal.force_error('', 'unknown case') END [as=s_quantity_new:35, outer=(18,19), volatile] - ├── CASE (s_i_id:18, s_w_id:19) WHEN (6823, 0) THEN 6 WHEN (7853, 0) THEN 9 WHEN (8497, 0) THEN 13 WHEN (10904, 0) THEN 1 WHEN (16152, 0) THEN 2 WHEN (41382, 0) THEN 3 WHEN (55952, 0) THEN 10 WHEN (64817, 0) THEN 31 WHEN (66335, 0) THEN 9 WHEN (76567, 0) THEN 7 WHEN (81680, 0) THEN 4 WHEN (89641, 0) THEN 13 WHEN (89905, 0) THEN 20 ELSE CAST(NULL AS INT8) END [as=s_ytd_new:36, outer=(18,19)] - ├── CASE (s_i_id:18, s_w_id:19) WHEN (6823, 0) THEN 1 WHEN (7853, 0) THEN 1 WHEN (8497, 0) THEN 2 WHEN (10904, 0) THEN 1 WHEN (16152, 0) THEN 1 WHEN (41382, 0) THEN 1 WHEN (55952, 0) THEN 1 WHEN (64817, 0) THEN 4 WHEN (66335, 0) THEN 2 WHEN (76567, 0) THEN 1 WHEN (81680, 0) THEN 1 WHEN (89641, 0) THEN 2 WHEN (89905, 0) THEN 4 ELSE CAST(NULL AS INT8) END [as=s_order_cnt_new:37, outer=(18,19)] - └── CASE (s_i_id:18, s_w_id:19) WHEN (6823, 0) THEN 0 WHEN (7853, 0) THEN 0 WHEN (8497, 0) THEN 0 WHEN (10904, 0) THEN 0 WHEN (16152, 0) THEN 0 WHEN (41382, 0) THEN 0 WHEN (55952, 0) THEN 0 WHEN (64817, 0) THEN 0 WHEN (66335, 0) THEN 0 WHEN (76567, 0) THEN 0 WHEN (81680, 0) THEN 0 WHEN (89641, 0) THEN 0 WHEN (89905, 0) THEN 0 ELSE CAST(NULL AS INT8) END [as=s_remote_cnt_new:38, outer=(18,19)] + ├── CASE (s_i_id:19, s_w_id:20) WHEN (6823, 0) THEN 26 WHEN (7853, 0) THEN 10 WHEN (8497, 0) THEN 62 WHEN (10904, 0) THEN 54 WHEN (16152, 0) THEN 80 WHEN (41382, 0) THEN 18 WHEN (55952, 0) THEN 56 WHEN (64817, 0) THEN 26 WHEN (66335, 0) THEN 30 WHEN (76567, 0) THEN 71 WHEN (81680, 0) THEN 51 WHEN (89641, 0) THEN 51 WHEN (89905, 0) THEN 77 ELSE crdb_internal.force_error('', 'unknown case') END [as=s_quantity_new:37, outer=(19,20), volatile] + ├── CASE (s_i_id:19, s_w_id:20) WHEN (6823, 0) THEN 6 WHEN (7853, 0) THEN 9 WHEN (8497, 0) THEN 13 WHEN (10904, 0) THEN 1 WHEN (16152, 0) THEN 2 WHEN (41382, 0) THEN 3 WHEN (55952, 0) THEN 10 WHEN (64817, 0) THEN 31 WHEN (66335, 0) THEN 9 WHEN (76567, 0) THEN 7 WHEN (81680, 0) THEN 4 WHEN (89641, 0) THEN 13 WHEN (89905, 0) THEN 20 ELSE CAST(NULL AS INT8) END [as=s_ytd_new:38, outer=(19,20)] + ├── CASE (s_i_id:19, s_w_id:20) WHEN (6823, 0) THEN 1 WHEN (7853, 0) THEN 1 WHEN (8497, 0) THEN 2 WHEN (10904, 0) THEN 1 WHEN (16152, 0) THEN 1 WHEN (41382, 0) THEN 1 WHEN (55952, 0) THEN 1 WHEN (64817, 0) THEN 4 WHEN (66335, 0) THEN 2 WHEN (76567, 0) THEN 1 WHEN (81680, 0) THEN 1 WHEN (89641, 0) THEN 2 WHEN (89905, 0) THEN 4 ELSE CAST(NULL AS INT8) END [as=s_order_cnt_new:39, outer=(19,20)] + └── CASE (s_i_id:19, s_w_id:20) WHEN (6823, 0) THEN 0 WHEN (7853, 0) THEN 0 WHEN (8497, 0) THEN 0 WHEN (10904, 0) THEN 0 WHEN (16152, 0) THEN 0 WHEN (41382, 0) THEN 0 WHEN (55952, 0) THEN 0 WHEN (64817, 0) THEN 0 WHEN (66335, 0) THEN 0 WHEN (76567, 0) THEN 0 WHEN (81680, 0) THEN 0 WHEN (89641, 0) THEN 0 WHEN (89905, 0) THEN 0 ELSE CAST(NULL AS INT8) END [as=s_remote_cnt_new:40, outer=(19,20)] opt format=hide-qual INSERT INTO order_line @@ -355,26 +355,26 @@ VALUES insert order_line ├── columns: ├── insert-mapping: - │ ├── column1:11 => ol_o_id:1 - │ ├── column2:12 => ol_d_id:2 - │ ├── column3:13 => ol_w_id:3 - │ ├── column4:14 => ol_number:4 - │ ├── column5:15 => ol_i_id:5 - │ ├── column6:16 => ol_supply_w_id:6 - │ ├── column20:20 => ol_delivery_d:7 - │ ├── column7:17 => ol_quantity:8 - │ ├── ol_amount:21 => order_line.ol_amount:9 - │ └── column9:19 => ol_dist_info:10 + │ ├── column1:12 => ol_o_id:1 + │ ├── column2:13 => ol_d_id:2 + │ ├── column3:14 => ol_w_id:3 + │ ├── column4:15 => ol_number:4 + │ ├── column5:16 => ol_i_id:5 + │ ├── column6:17 => ol_supply_w_id:6 + │ ├── column21:21 => ol_delivery_d:7 + │ ├── column7:18 => ol_quantity:8 + │ ├── ol_amount:22 => order_line.ol_amount:9 + │ └── column9:20 => ol_dist_info:10 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── project - │ ├── columns: ol_amount:21 column20:20 column1:11!null column2:12!null column3:13!null column4:14!null column5:15!null column6:16!null column7:17!null column9:19!null + │ ├── columns: ol_amount:22 column21:21 column1:12!null column2:13!null column3:14!null column4:15!null column5:16!null column6:17!null column7:18!null column9:20!null │ ├── cardinality: [6 - 6] │ ├── immutable - │ ├── fd: ()-->(20) + │ ├── fd: ()-->(21) │ ├── values - │ │ ├── columns: column1:11!null column2:12!null column3:13!null column4:14!null column5:15!null column6:16!null column7:17!null column8:18!null column9:19!null + │ │ ├── columns: column1:12!null column2:13!null column3:14!null column4:15!null column5:16!null column6:17!null column7:18!null column8:19!null column9:20!null │ │ ├── cardinality: [6 - 6] │ │ ├── (3045, 2, 10, 3, 648, 0, 9, 394.470000, 'YhgLRrwsmd68P2bElAgrnp8u') │ │ ├── (3045, 2, 10, 5, 25393, 0, 10, 830.600000, 'dLXe0YhgLRrwsmd68P2bElAg') @@ -383,34 +383,34 @@ insert order_line │ │ ├── (3045, 2, 10, 4, 56624, 0, 6, 273.360000, 'RsaCXoEzmssaF9m9cdLXe0Yh') │ │ └── (3045, 2, 10, 6, 92966, 0, 4, 366.760000, 'saCXoEzmssaF9m9cdLXe0Yhg') │ └── projections - │ ├── crdb_internal.round_decimal_values(column8:18, 2) [as=ol_amount:21, outer=(18), immutable] - │ └── CAST(NULL AS TIMESTAMP) [as=column20:20] + │ ├── crdb_internal.round_decimal_values(column8:19, 2) [as=ol_amount:22, outer=(19), immutable] + │ └── CAST(NULL AS TIMESTAMP) [as=column21:21] └── f-k-checks ├── f-k-checks-item: order_line(ol_w_id,ol_d_id,ol_o_id) -> order(o_w_id,o_d_id,o_id) │ └── anti-join (lookup order) - │ ├── columns: column3:22!null column2:23!null column1:24!null - │ ├── key columns: [22 23 24] = [27 26 25] + │ ├── columns: column3:23!null column2:24!null column1:25!null + │ ├── key columns: [23 24 25] = [28 27 26] │ ├── lookup columns are key │ ├── cardinality: [0 - 6] │ ├── with-scan &1 - │ │ ├── columns: column3:22!null column2:23!null column1:24!null + │ │ ├── columns: column3:23!null column2:24!null column1:25!null │ │ ├── mapping: - │ │ │ ├── column3:13 => column3:22 - │ │ │ ├── column2:12 => column2:23 - │ │ │ └── column1:11 => column1:24 + │ │ │ ├── column3:14 => column3:23 + │ │ │ ├── column2:13 => column2:24 + │ │ │ └── column1:12 => column1:25 │ │ └── cardinality: [6 - 6] │ └── filters (true) └── f-k-checks-item: order_line(ol_supply_w_id,ol_i_id) -> stock(s_w_id,s_i_id) └── anti-join (lookup stock@stock_item_fk_idx) - ├── columns: column6:33!null column5:34!null - ├── key columns: [34 33] = [35 36] + ├── columns: column6:35!null column5:36!null + ├── key columns: [36 35] = [37 38] ├── lookup columns are key ├── cardinality: [0 - 6] ├── with-scan &1 - │ ├── columns: column6:33!null column5:34!null + │ ├── columns: column6:35!null column5:36!null │ ├── mapping: - │ │ ├── column6:16 => column6:33 - │ │ └── column5:15 => column5:34 + │ │ ├── column6:17 => column6:35 + │ │ └── column5:16 => column5:36 │ └── cardinality: [6 - 6] └── filters (true) @@ -435,27 +435,27 @@ project ├── fd: ()-->(2-7) └── update warehouse ├── columns: w_id:1!null w_name:2 w_street_1:3 w_street_2:4 w_city:5 w_state:6 w_zip:7 - ├── fetch columns: w_id:10 w_name:11 w_street_1:12 w_street_2:13 w_city:14 w_state:15 w_zip:16 w_tax:17 warehouse.w_ytd:18 + ├── fetch columns: w_id:11 w_name:12 w_street_1:13 w_street_2:14 w_city:15 w_state:16 w_zip:17 w_tax:18 warehouse.w_ytd:19 ├── update-mapping: - │ └── w_ytd:20 => warehouse.w_ytd:9 + │ └── w_ytd:22 => warehouse.w_ytd:9 ├── cardinality: [0 - 1] ├── volatile, mutations ├── key: () ├── fd: ()-->(1-7) └── project - ├── columns: w_ytd:20 w_id:10!null w_name:11 w_street_1:12 w_street_2:13 w_city:14 w_state:15 w_zip:16 w_tax:17 warehouse.w_ytd:18 + ├── columns: w_ytd:22 w_id:11!null w_name:12 w_street_1:13 w_street_2:14 w_city:15 w_state:16 w_zip:17 w_tax:18 warehouse.w_ytd:19 ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(10-18,20) + ├── fd: ()-->(11-19,22) ├── scan warehouse - │ ├── columns: w_id:10!null w_name:11 w_street_1:12 w_street_2:13 w_city:14 w_state:15 w_zip:16 w_tax:17 warehouse.w_ytd:18 - │ ├── constraint: /10: [/10 - /10] + │ ├── columns: w_id:11!null w_name:12 w_street_1:13 w_street_2:14 w_city:15 w_state:16 w_zip:17 w_tax:18 warehouse.w_ytd:19 + │ ├── constraint: /11: [/10 - /10] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(10-18) + │ └── fd: ()-->(11-19) └── projections - └── crdb_internal.round_decimal_values(warehouse.w_ytd:18::DECIMAL + 3860.61, 2) [as=w_ytd:20, outer=(18), immutable] + └── crdb_internal.round_decimal_values(warehouse.w_ytd:19::DECIMAL + 3860.61, 2) [as=w_ytd:22, outer=(19), immutable] opt format=hide-qual UPDATE district SET d_ytd = d_ytd + 3860.61 WHERE (d_w_id = 10) AND (d_id = 5) @@ -469,27 +469,27 @@ project ├── fd: ()-->(3-8) └── update district ├── columns: d_id:1!null d_w_id:2!null d_name:3 d_street_1:4 d_street_2:5 d_city:6 d_state:7 d_zip:8 - ├── fetch columns: d_id:12 d_w_id:13 d_name:14 d_street_1:15 d_street_2:16 d_city:17 d_state:18 d_zip:19 d_tax:20 district.d_ytd:21 d_next_o_id:22 + ├── fetch columns: d_id:13 d_w_id:14 d_name:15 d_street_1:16 d_street_2:17 d_city:18 d_state:19 d_zip:20 d_tax:21 district.d_ytd:22 d_next_o_id:23 ├── update-mapping: - │ └── d_ytd:24 => district.d_ytd:10 + │ └── d_ytd:26 => district.d_ytd:10 ├── cardinality: [0 - 1] ├── volatile, mutations ├── key: () ├── fd: ()-->(1-8) └── project - ├── columns: d_ytd:24 d_id:12!null d_w_id:13!null d_name:14 d_street_1:15 d_street_2:16 d_city:17 d_state:18 d_zip:19 d_tax:20 district.d_ytd:21 d_next_o_id:22 + ├── columns: d_ytd:26 d_id:13!null d_w_id:14!null d_name:15 d_street_1:16 d_street_2:17 d_city:18 d_state:19 d_zip:20 d_tax:21 district.d_ytd:22 d_next_o_id:23 ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(12-22,24) + ├── fd: ()-->(13-23,26) ├── scan district - │ ├── columns: d_id:12!null d_w_id:13!null d_name:14 d_street_1:15 d_street_2:16 d_city:17 d_state:18 d_zip:19 d_tax:20 district.d_ytd:21 d_next_o_id:22 - │ ├── constraint: /13/12: [/10/5 - /10/5] + │ ├── columns: d_id:13!null d_w_id:14!null d_name:15 d_street_1:16 d_street_2:17 d_city:18 d_state:19 d_zip:20 d_tax:21 district.d_ytd:22 d_next_o_id:23 + │ ├── constraint: /14/13: [/10/5 - /10/5] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(12-22) + │ └── fd: ()-->(13-23) └── projections - └── crdb_internal.round_decimal_values(district.d_ytd:21::DECIMAL + 3860.61, 2) [as=d_ytd:24, outer=(21), immutable] + └── crdb_internal.round_decimal_values(district.d_ytd:22::DECIMAL + 3860.61, 2) [as=d_ytd:26, outer=(22), immutable] opt format=hide-qual SELECT c_id @@ -551,42 +551,42 @@ RETURNING CASE c_credit WHEN 'BC' THEN "left"(c_data, 200) ELSE '' END ---- project - ├── columns: c_first:4 c_middle:5 c_last:6 c_street_1:7 c_street_2:8 c_city:9 c_state:10 c_zip:11 c_phone:12 c_since:13 c_credit:14 c_credit_lim:15 c_discount:16 c_balance:17 case:49 + ├── columns: c_first:4 c_middle:5 c_last:6 c_street_1:7 c_street_2:8 c_city:9 c_state:10 c_zip:11 c_phone:12 c_since:13 c_credit:14 c_credit_lim:15 c_discount:16 c_balance:17 case:51 ├── cardinality: [0 - 1] ├── volatile, mutations ├── key: () - ├── fd: ()-->(4-17,49) + ├── fd: ()-->(4-17,51) ├── update customer │ ├── columns: c_id:1!null c_d_id:2!null c_w_id:3!null c_first:4 c_middle:5 c_last:6 c_street_1:7 c_street_2:8 c_city:9 c_state:10 c_zip:11 c_phone:12 c_since:13 c_credit:14 c_credit_lim:15 c_discount:16 customer.c_balance:17 c_data:21 - │ ├── fetch columns: c_id:22 c_d_id:23 c_w_id:24 c_first:25 c_middle:26 c_last:27 c_street_1:28 c_street_2:29 c_city:30 c_state:31 c_zip:32 c_phone:33 c_since:34 c_credit:35 c_credit_lim:36 c_discount:37 customer.c_balance:38 customer.c_ytd_payment:39 c_payment_cnt:40 c_delivery_cnt:41 c_data:42 + │ ├── fetch columns: c_id:23 c_d_id:24 c_w_id:25 c_first:26 c_middle:27 c_last:28 c_street_1:29 c_street_2:30 c_city:31 c_state:32 c_zip:33 c_phone:34 c_since:35 c_credit:36 c_credit_lim:37 c_discount:38 customer.c_balance:39 customer.c_ytd_payment:40 c_payment_cnt:41 c_delivery_cnt:42 c_data:43 │ ├── update-mapping: - │ │ ├── c_balance:47 => customer.c_balance:17 - │ │ ├── c_ytd_payment:48 => customer.c_ytd_payment:18 - │ │ ├── c_payment_cnt_new:45 => c_payment_cnt:19 - │ │ └── c_data_new:46 => c_data:21 + │ │ ├── c_balance:49 => customer.c_balance:17 + │ │ ├── c_ytd_payment:50 => customer.c_ytd_payment:18 + │ │ ├── c_payment_cnt_new:47 => c_payment_cnt:19 + │ │ └── c_data_new:48 => c_data:21 │ ├── cardinality: [0 - 1] │ ├── volatile, mutations │ ├── key: () │ ├── fd: ()-->(1-17,21) │ └── project - │ ├── columns: c_balance:47 c_ytd_payment:48 c_payment_cnt_new:45 c_data_new:46 c_id:22!null c_d_id:23!null c_w_id:24!null c_first:25 c_middle:26 c_last:27 c_street_1:28 c_street_2:29 c_city:30 c_state:31 c_zip:32 c_phone:33 c_since:34 c_credit:35 c_credit_lim:36 c_discount:37 customer.c_balance:38 customer.c_ytd_payment:39 c_payment_cnt:40 c_delivery_cnt:41 c_data:42 + │ ├── columns: c_balance:49 c_ytd_payment:50 c_payment_cnt_new:47 c_data_new:48 c_id:23!null c_d_id:24!null c_w_id:25!null c_first:26 c_middle:27 c_last:28 c_street_1:29 c_street_2:30 c_city:31 c_state:32 c_zip:33 c_phone:34 c_since:35 c_credit:36 c_credit_lim:37 c_discount:38 customer.c_balance:39 customer.c_ytd_payment:40 c_payment_cnt:41 c_delivery_cnt:42 c_data:43 │ ├── cardinality: [0 - 1] │ ├── immutable │ ├── key: () - │ ├── fd: ()-->(22-42,45-48) + │ ├── fd: ()-->(23-43,47-50) │ ├── scan customer - │ │ ├── columns: c_id:22!null c_d_id:23!null c_w_id:24!null c_first:25 c_middle:26 c_last:27 c_street_1:28 c_street_2:29 c_city:30 c_state:31 c_zip:32 c_phone:33 c_since:34 c_credit:35 c_credit_lim:36 c_discount:37 customer.c_balance:38 customer.c_ytd_payment:39 c_payment_cnt:40 c_delivery_cnt:41 c_data:42 - │ │ ├── constraint: /24/23/22: [/10/5/1343 - /10/5/1343] + │ │ ├── columns: c_id:23!null c_d_id:24!null c_w_id:25!null c_first:26 c_middle:27 c_last:28 c_street_1:29 c_street_2:30 c_city:31 c_state:32 c_zip:33 c_phone:34 c_since:35 c_credit:36 c_credit_lim:37 c_discount:38 customer.c_balance:39 customer.c_ytd_payment:40 c_payment_cnt:41 c_delivery_cnt:42 c_data:43 + │ │ ├── constraint: /25/24/23: [/10/5/1343 - /10/5/1343] │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(22-42) + │ │ └── fd: ()-->(23-43) │ └── projections - │ ├── crdb_internal.round_decimal_values(customer.c_balance:38::DECIMAL - 3860.61, 2) [as=c_balance:47, outer=(38), immutable] - │ ├── crdb_internal.round_decimal_values(customer.c_ytd_payment:39::DECIMAL + 3860.61, 2) [as=c_ytd_payment:48, outer=(39), immutable] - │ ├── c_payment_cnt:40 + 1 [as=c_payment_cnt_new:45, outer=(40), immutable] - │ └── CASE c_credit:35 WHEN 'BC' THEN left((((((c_id:22::STRING || c_d_id:23::STRING) || c_w_id:24::STRING) || '5') || '10') || '3860.61') || c_data:42::STRING, 500) ELSE c_data:42::STRING END [as=c_data_new:46, outer=(22-24,35,42), immutable] + │ ├── crdb_internal.round_decimal_values(customer.c_balance:39::DECIMAL - 3860.61, 2) [as=c_balance:49, outer=(39), immutable] + │ ├── crdb_internal.round_decimal_values(customer.c_ytd_payment:40::DECIMAL + 3860.61, 2) [as=c_ytd_payment:50, outer=(40), immutable] + │ ├── c_payment_cnt:41 + 1 [as=c_payment_cnt_new:47, outer=(41), immutable] + │ └── CASE c_credit:36 WHEN 'BC' THEN left((((((c_id:23::STRING || c_d_id:24::STRING) || c_w_id:25::STRING) || '5') || '10') || '3860.61') || c_data:43::STRING, 500) ELSE c_data:43::STRING END [as=c_data_new:48, outer=(23-25,36,43), immutable] └── projections - └── CASE c_credit:14 WHEN 'BC' THEN left(c_data:21, 200) ELSE '' END [as=case:49, outer=(14,21), immutable] + └── CASE c_credit:14 WHEN 'BC' THEN left(c_data:21, 200) ELSE '' END [as=case:51, outer=(14,21), immutable] opt format=hide-qual INSERT INTO history @@ -597,60 +597,60 @@ VALUES insert history ├── columns: ├── insert-mapping: - │ ├── column18:18 => rowid:1 - │ ├── column1:10 => h_c_id:2 - │ ├── column2:11 => h_c_d_id:3 - │ ├── column3:12 => h_c_w_id:4 - │ ├── column4:13 => h_d_id:5 - │ ├── column5:14 => h_w_id:6 - │ ├── column7:16 => h_date:7 - │ ├── h_amount:19 => history.h_amount:8 - │ └── column8:17 => h_data:9 + │ ├── column19:19 => rowid:1 + │ ├── column1:11 => h_c_id:2 + │ ├── column2:12 => h_c_d_id:3 + │ ├── column3:13 => h_c_w_id:4 + │ ├── column4:14 => h_d_id:5 + │ ├── column5:15 => h_w_id:6 + │ ├── column7:17 => h_date:7 + │ ├── h_amount:20 => history.h_amount:8 + │ └── column8:18 => h_data:9 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── values - │ ├── columns: column1:10!null column2:11!null column3:12!null column4:13!null column5:14!null column7:16!null column8:17!null column18:18 h_amount:19!null + │ ├── columns: column1:11!null column2:12!null column3:13!null column4:14!null column5:15!null column7:17!null column8:18!null column19:19 h_amount:20!null │ ├── cardinality: [1 - 1] │ ├── volatile │ ├── key: () - │ ├── fd: ()-->(10-14,16-19) + │ ├── fd: ()-->(11-15,17-20) │ └── (1343, 5, 10, 5, 10, '2019-08-26 16:50:41+00:00', '8 Kdcgphy3', gen_random_uuid(), 3860.61) └── f-k-checks ├── f-k-checks-item: history(h_c_w_id,h_c_d_id,h_c_id) -> customer(c_w_id,c_d_id,c_id) │ └── anti-join (lookup customer) - │ ├── columns: column3:20!null column2:21!null column1:22!null - │ ├── key columns: [20 21 22] = [25 24 23] + │ ├── columns: column3:21!null column2:22!null column1:23!null + │ ├── key columns: [21 22 23] = [26 25 24] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(20-22) + │ ├── fd: ()-->(21-23) │ ├── with-scan &1 - │ │ ├── columns: column3:20!null column2:21!null column1:22!null + │ │ ├── columns: column3:21!null column2:22!null column1:23!null │ │ ├── mapping: - │ │ │ ├── column3:12 => column3:20 - │ │ │ ├── column2:11 => column2:21 - │ │ │ └── column1:10 => column1:22 + │ │ │ ├── column3:13 => column3:21 + │ │ │ ├── column2:12 => column2:22 + │ │ │ └── column1:11 => column1:23 │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(20-22) + │ │ └── fd: ()-->(21-23) │ └── filters (true) └── f-k-checks-item: history(h_w_id,h_d_id) -> district(d_w_id,d_id) └── anti-join (lookup district) - ├── columns: column5:44!null column4:45!null - ├── key columns: [44 45] = [47 46] + ├── columns: column5:46!null column4:47!null + ├── key columns: [46 47] = [49 48] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(44,45) + ├── fd: ()-->(46,47) ├── with-scan &1 - │ ├── columns: column5:44!null column4:45!null + │ ├── columns: column5:46!null column4:47!null │ ├── mapping: - │ │ ├── column5:14 => column5:44 - │ │ └── column4:13 => column4:45 + │ │ ├── column5:15 => column5:46 + │ │ └── column4:14 => column4:47 │ ├── cardinality: [1 - 1] │ ├── key: () - │ └── fd: ()-->(44,45) + │ └── fd: ()-->(46,47) └── filters (true) # -------------------------------------------------- @@ -778,16 +778,16 @@ FROM order_line WHERE ol_w_id = 10 AND ol_d_id = 100 AND ol_o_id = 1000 ---- scalar-group-by - ├── columns: sum:11 + ├── columns: sum:12 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(11) + ├── fd: ()-->(12) ├── scan order_line │ ├── columns: ol_o_id:1!null ol_d_id:2!null ol_w_id:3!null ol_amount:9 │ ├── constraint: /3/2/-1/4: [/10/100/1000 - /10/100/1000] │ └── fd: ()-->(1-3) └── aggregations - └── sum [as=sum:11, outer=(9)] + └── sum [as=sum:12, outer=(9)] └── ol_amount:9 opt format=hide-qual @@ -817,21 +817,21 @@ project ├── fd: (2)-->(4) └── update "order" ├── columns: o_id:1!null o_d_id:2!null o_w_id:3!null o_c_id:4 - ├── fetch columns: o_id:9 o_d_id:10 o_w_id:11 o_c_id:12 o_entry_d:13 o_carrier_id:14 o_ol_cnt:15 o_all_local:16 + ├── fetch columns: o_id:10 o_d_id:11 o_w_id:12 o_c_id:13 o_entry_d:14 o_carrier_id:15 o_ol_cnt:16 o_all_local:17 ├── update-mapping: - │ └── o_carrier_id_new:17 => o_carrier_id:6 + │ └── o_carrier_id_new:19 => o_carrier_id:6 ├── cardinality: [0 - 10] ├── volatile, mutations ├── key: (2) ├── fd: ()-->(1,3), (2)-->(4) └── project - ├── columns: o_carrier_id_new:17!null o_id:9!null o_d_id:10!null o_w_id:11!null o_c_id:12 o_entry_d:13 o_carrier_id:14 o_ol_cnt:15 o_all_local:16 + ├── columns: o_carrier_id_new:19!null o_id:10!null o_d_id:11!null o_w_id:12!null o_c_id:13 o_entry_d:14 o_carrier_id:15 o_ol_cnt:16 o_all_local:17 ├── cardinality: [0 - 10] - ├── key: (10) - ├── fd: ()-->(9,11,17), (10)-->(12-16) + ├── key: (11) + ├── fd: ()-->(10,12,19), (11)-->(13-17) ├── scan "order" - │ ├── columns: o_id:9!null o_d_id:10!null o_w_id:11!null o_c_id:12 o_entry_d:13 o_carrier_id:14 o_ol_cnt:15 o_all_local:16 - │ ├── constraint: /11/10/-9 + │ ├── columns: o_id:10!null o_d_id:11!null o_w_id:12!null o_c_id:13 o_entry_d:14 o_carrier_id:15 o_ol_cnt:16 o_all_local:17 + │ ├── constraint: /12/11/-10 │ │ ├── [/10/1/2167 - /10/1/2167] │ │ ├── [/10/2/2167 - /10/2/2167] │ │ ├── [/10/3/2167 - /10/3/2167] @@ -843,10 +843,10 @@ project │ │ ├── [/10/9/2167 - /10/9/2167] │ │ └── [/10/10/2167 - /10/10/2167] │ ├── cardinality: [0 - 10] - │ ├── key: (10) - │ └── fd: ()-->(9,11), (10)-->(12-16) + │ ├── key: (11) + │ └── fd: ()-->(10,12), (11)-->(13-17) └── projections - └── 10 [as=o_carrier_id_new:17] + └── 10 [as=o_carrier_id_new:19] opt format=hide-qual UPDATE customer @@ -878,21 +878,21 @@ WHERE c_w_id = 10 AND (c_d_id, c_id) IN ( ---- update customer ├── columns: - ├── fetch columns: c_id:22 c_d_id:23 c_w_id:24 c_first:25 c_middle:26 c_last:27 c_street_1:28 c_street_2:29 c_city:30 c_state:31 c_zip:32 c_phone:33 c_since:34 c_credit:35 c_credit_lim:36 c_discount:37 customer.c_balance:38 c_ytd_payment:39 c_payment_cnt:40 c_delivery_cnt:41 c_data:42 + ├── fetch columns: c_id:23 c_d_id:24 c_w_id:25 c_first:26 c_middle:27 c_last:28 c_street_1:29 c_street_2:30 c_city:31 c_state:32 c_zip:33 c_phone:34 c_since:35 c_credit:36 c_credit_lim:37 c_discount:38 customer.c_balance:39 c_ytd_payment:40 c_payment_cnt:41 c_delivery_cnt:42 c_data:43 ├── update-mapping: - │ ├── c_balance:45 => customer.c_balance:17 - │ └── c_delivery_cnt_new:43 => c_delivery_cnt:20 + │ ├── c_balance:47 => customer.c_balance:17 + │ └── c_delivery_cnt_new:45 => c_delivery_cnt:20 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: c_balance:45 c_delivery_cnt_new:43 c_id:22!null c_d_id:23!null c_w_id:24!null c_first:25 c_middle:26 c_last:27 c_street_1:28 c_street_2:29 c_city:30 c_state:31 c_zip:32 c_phone:33 c_since:34 c_credit:35 c_credit_lim:36 c_discount:37 customer.c_balance:38 c_ytd_payment:39 c_payment_cnt:40 c_delivery_cnt:41 c_data:42 + ├── columns: c_balance:47 c_delivery_cnt_new:45 c_id:23!null c_d_id:24!null c_w_id:25!null c_first:26 c_middle:27 c_last:28 c_street_1:29 c_street_2:30 c_city:31 c_state:32 c_zip:33 c_phone:34 c_since:35 c_credit:36 c_credit_lim:37 c_discount:38 customer.c_balance:39 c_ytd_payment:40 c_payment_cnt:41 c_delivery_cnt:42 c_data:43 ├── cardinality: [0 - 10] ├── immutable - ├── key: (22,23) - ├── fd: ()-->(24), (22,23)-->(25-42,45), (41)-->(43) + ├── key: (23,24) + ├── fd: ()-->(25), (23,24)-->(26-43,47), (42)-->(45) ├── scan customer - │ ├── columns: c_id:22!null c_d_id:23!null c_w_id:24!null c_first:25 c_middle:26 c_last:27 c_street_1:28 c_street_2:29 c_city:30 c_state:31 c_zip:32 c_phone:33 c_since:34 c_credit:35 c_credit_lim:36 c_discount:37 customer.c_balance:38 c_ytd_payment:39 c_payment_cnt:40 c_delivery_cnt:41 c_data:42 - │ ├── constraint: /24/23/22 + │ ├── columns: c_id:23!null c_d_id:24!null c_w_id:25!null c_first:26 c_middle:27 c_last:28 c_street_1:29 c_street_2:30 c_city:31 c_state:32 c_zip:33 c_phone:34 c_since:35 c_credit:36 c_credit_lim:37 c_discount:38 customer.c_balance:39 c_ytd_payment:40 c_payment_cnt:41 c_delivery_cnt:42 c_data:43 + │ ├── constraint: /25/24/23 │ │ ├── [/10/1/1405 - /10/1/1405] │ │ ├── [/10/2/137 - /10/2/137] │ │ ├── [/10/3/309 - /10/3/309] @@ -904,11 +904,11 @@ update customer │ │ ├── [/10/9/1412 - /10/9/1412] │ │ └── [/10/10/417 - /10/10/417] │ ├── cardinality: [0 - 10] - │ ├── key: (22,23) - │ └── fd: ()-->(24), (22,23)-->(25-42) + │ ├── key: (23,24) + │ └── fd: ()-->(25), (23,24)-->(26-43) └── projections - ├── crdb_internal.round_decimal_values(customer.c_balance:38::DECIMAL + CASE c_d_id:23 WHEN 6 THEN 57214.780000 WHEN 8 THEN 67755.430000 WHEN 1 THEN 51177.840000 WHEN 2 THEN 73840.700000 WHEN 4 THEN 45906.990000 WHEN 9 THEN 32523.760000 WHEN 10 THEN 20240.200000 WHEN 3 THEN 75299.790000 WHEN 5 THEN 56543.340000 WHEN 7 THEN 67157.940000 ELSE CAST(NULL AS DECIMAL) END, 2) [as=c_balance:45, outer=(23,38), immutable] - └── c_delivery_cnt:41 + 1 [as=c_delivery_cnt_new:43, outer=(41), immutable] + ├── crdb_internal.round_decimal_values(customer.c_balance:39::DECIMAL + CASE c_d_id:24 WHEN 6 THEN 57214.780000 WHEN 8 THEN 67755.430000 WHEN 1 THEN 51177.840000 WHEN 2 THEN 73840.700000 WHEN 4 THEN 45906.990000 WHEN 9 THEN 32523.760000 WHEN 10 THEN 20240.200000 WHEN 3 THEN 75299.790000 WHEN 5 THEN 56543.340000 WHEN 7 THEN 67157.940000 ELSE CAST(NULL AS DECIMAL) END, 2) [as=c_balance:47, outer=(24,39), immutable] + └── c_delivery_cnt:42 + 1 [as=c_delivery_cnt_new:45, outer=(42), immutable] opt format=hide-qual DELETE FROM new_order @@ -927,12 +927,12 @@ WHERE no_w_id = 10 AND (no_d_id, no_o_id) IN ( ---- delete new_order ├── columns: - ├── fetch columns: no_o_id:4 no_d_id:5 no_w_id:6 + ├── fetch columns: no_o_id:5 no_d_id:6 no_w_id:7 ├── cardinality: [0 - 0] ├── volatile, mutations └── scan new_order - ├── columns: no_o_id:4!null no_d_id:5!null no_w_id:6!null - ├── constraint: /6/5/4 + ├── columns: no_o_id:5!null no_d_id:6!null no_w_id:7!null + ├── constraint: /7/6/5 │ ├── [/10/1/2167 - /10/1/2167] │ ├── [/10/2/2167 - /10/2/2167] │ ├── [/10/3/2167 - /10/3/2167] @@ -944,8 +944,8 @@ delete new_order │ ├── [/10/9/2167 - /10/9/2167] │ └── [/10/10/2167 - /10/10/2167] ├── cardinality: [0 - 10] - ├── key: (5) - └── fd: ()-->(4,6) + ├── key: (6) + └── fd: ()-->(5,7) opt format=hide-qual UPDATE order_line @@ -965,18 +965,18 @@ WHERE ol_w_id = 10 AND (ol_d_id, ol_o_id) IN ( ---- update order_line ├── columns: - ├── fetch columns: ol_o_id:11 ol_d_id:12 ol_w_id:13 ol_number:14 ol_i_id:15 ol_supply_w_id:16 ol_delivery_d:17 ol_quantity:18 ol_amount:19 ol_dist_info:20 + ├── fetch columns: ol_o_id:12 ol_d_id:13 ol_w_id:14 ol_number:15 ol_i_id:16 ol_supply_w_id:17 ol_delivery_d:18 ol_quantity:19 ol_amount:20 ol_dist_info:21 ├── update-mapping: - │ └── ol_delivery_d_new:21 => ol_delivery_d:7 + │ └── ol_delivery_d_new:23 => ol_delivery_d:7 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: ol_delivery_d_new:21!null ol_o_id:11!null ol_d_id:12!null ol_w_id:13!null ol_number:14!null ol_i_id:15!null ol_supply_w_id:16 ol_delivery_d:17 ol_quantity:18 ol_amount:19 ol_dist_info:20 - ├── key: (12,14) - ├── fd: ()-->(11,13,21), (12,14)-->(15-20) + ├── columns: ol_delivery_d_new:23!null ol_o_id:12!null ol_d_id:13!null ol_w_id:14!null ol_number:15!null ol_i_id:16!null ol_supply_w_id:17 ol_delivery_d:18 ol_quantity:19 ol_amount:20 ol_dist_info:21 + ├── key: (13,15) + ├── fd: ()-->(12,14,23), (13,15)-->(16-21) ├── scan order_line - │ ├── columns: ol_o_id:11!null ol_d_id:12!null ol_w_id:13!null ol_number:14!null ol_i_id:15!null ol_supply_w_id:16 ol_delivery_d:17 ol_quantity:18 ol_amount:19 ol_dist_info:20 - │ ├── constraint: /13/12/-11/14 + │ ├── columns: ol_o_id:12!null ol_d_id:13!null ol_w_id:14!null ol_number:15!null ol_i_id:16!null ol_supply_w_id:17 ol_delivery_d:18 ol_quantity:19 ol_amount:20 ol_dist_info:21 + │ ├── constraint: /14/13/-12/15 │ │ ├── [/10/1/2167 - /10/1/2167] │ │ ├── [/10/2/2167 - /10/2/2167] │ │ ├── [/10/3/2167 - /10/3/2167] @@ -987,10 +987,10 @@ update order_line │ │ ├── [/10/8/2167 - /10/8/2167] │ │ ├── [/10/9/2167 - /10/9/2167] │ │ └── [/10/10/2167 - /10/10/2167] - │ ├── key: (12,14) - │ └── fd: ()-->(11,13), (12,14)-->(15-20) + │ ├── key: (13,15) + │ └── fd: ()-->(12,14), (13,15)-->(16-21) └── projections - └── '2019-08-26 16:50:41+00:00' [as=ol_delivery_d_new:21] + └── '2019-08-26 16:50:41+00:00' [as=ol_delivery_d_new:23] # -------------------------------------------------- # 2.8 The Stock-Level Transaction @@ -1028,28 +1028,28 @@ WHERE ol_w_id = 10 AND s_quantity < 15 ---- scalar-group-by - ├── columns: count:28!null + ├── columns: count:30!null ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(28) + ├── fd: ()-->(30) ├── distinct-on - │ ├── columns: s_i_id:11!null - │ ├── grouping columns: s_i_id:11!null - │ ├── key: (11) + │ ├── columns: s_i_id:12!null + │ ├── grouping columns: s_i_id:12!null + │ ├── key: (12) │ └── inner-join (lookup stock) - │ ├── columns: ol_o_id:1!null ol_d_id:2!null ol_w_id:3!null ol_i_id:5!null s_i_id:11!null s_w_id:12!null s_quantity:13!null - │ ├── key columns: [3 5] = [12 11] + │ ├── columns: ol_o_id:1!null ol_d_id:2!null ol_w_id:3!null ol_i_id:5!null s_i_id:12!null s_w_id:13!null s_quantity:14!null + │ ├── key columns: [3 5] = [13 12] │ ├── lookup columns are key - │ ├── fd: ()-->(2,3,12), (11)-->(13), (5)==(11), (11)==(5), (3)==(12), (12)==(3) + │ ├── fd: ()-->(2,3,13), (12)-->(14), (5)==(12), (12)==(5), (3)==(13), (13)==(3) │ ├── scan order_line │ │ ├── columns: ol_o_id:1!null ol_d_id:2!null ol_w_id:3!null ol_i_id:5!null │ │ ├── constraint: /3/2/-1/4: [/10/100/999 - /10/100/980] │ │ └── fd: ()-->(2,3) │ └── filters - │ ├── s_w_id:12 = 10 [outer=(12), constraints=(/12: [/10 - /10]; tight), fd=()-->(12)] - │ └── s_quantity:13 < 15 [outer=(13), constraints=(/13: (/NULL - /14]; tight)] + │ ├── s_w_id:13 = 10 [outer=(13), constraints=(/13: [/10 - /10]; tight), fd=()-->(13)] + │ └── s_quantity:14 < 15 [outer=(14), constraints=(/14: (/NULL - /14]; tight)] └── aggregations - └── count-rows [as=count:28] + └── count-rows [as=count:30] # -------------------------------------------------- # Consistency Queries @@ -1070,34 +1070,34 @@ ON (w_id = d_w_id) WHERE w_ytd != sum_d_ytd ---- scalar-group-by - ├── columns: count:22!null + ├── columns: count:24!null ├── cardinality: [1 - 1] ├── immutable ├── key: () - ├── fd: ()-->(22) + ├── fd: ()-->(24) ├── inner-join (lookup warehouse) - │ ├── columns: w_id:1!null w_ytd:9!null d_w_id:11!null sum:21!null - │ ├── key columns: [11] = [1] + │ ├── columns: w_id:1!null w_ytd:9!null d_w_id:12!null sum:23!null + │ ├── key columns: [12] = [1] │ ├── lookup columns are key │ ├── immutable - │ ├── key: (11) - │ ├── fd: (1)-->(9), (11)-->(21), (1)==(11), (11)==(1) + │ ├── key: (12) + │ ├── fd: (1)-->(9), (12)-->(23), (1)==(12), (12)==(1) │ ├── group-by - │ │ ├── columns: d_w_id:11!null sum:21 - │ │ ├── grouping columns: d_w_id:11!null - │ │ ├── internal-ordering: +11 - │ │ ├── key: (11) - │ │ ├── fd: (11)-->(21) + │ │ ├── columns: d_w_id:12!null sum:23 + │ │ ├── grouping columns: d_w_id:12!null + │ │ ├── internal-ordering: +12 + │ │ ├── key: (12) + │ │ ├── fd: (12)-->(23) │ │ ├── scan district - │ │ │ ├── columns: d_w_id:11!null d_ytd:19 - │ │ │ └── ordering: +11 + │ │ │ ├── columns: d_w_id:12!null d_ytd:20 + │ │ │ └── ordering: +12 │ │ └── aggregations - │ │ └── sum [as=sum:21, outer=(19)] - │ │ └── d_ytd:19 + │ │ └── sum [as=sum:23, outer=(20)] + │ │ └── d_ytd:20 │ └── filters - │ └── w_ytd:9 != sum:21 [outer=(9,21), immutable, constraints=(/9: (/NULL - ]; /21: (/NULL - ])] + │ └── w_ytd:9 != sum:23 [outer=(9,23), immutable, constraints=(/9: (/NULL - ]; /23: (/NULL - ])] └── aggregations - └── count-rows [as=count_rows:22] + └── count-rows [as=count_rows:24] opt format=hide-qual SELECT d_next_o_id @@ -1117,17 +1117,17 @@ GROUP BY no_d_id, no_w_id ORDER BY no_w_id, no_d_id ---- group-by - ├── columns: max:4!null [hidden: no_d_id:2!null no_w_id:3!null] + ├── columns: max:5!null [hidden: no_d_id:2!null no_w_id:3!null] ├── grouping columns: no_d_id:2!null no_w_id:3!null ├── key: (2,3) - ├── fd: (2,3)-->(4) + ├── fd: (2,3)-->(5) ├── ordering: +3,+2 ├── scan new_order │ ├── columns: no_o_id:1!null no_d_id:2!null no_w_id:3!null │ ├── key: (1-3) │ └── ordering: +3,+2 └── aggregations - └── max [as=max:4, outer=(1)] + └── max [as=max:5, outer=(1)] └── no_o_id:1 opt format=hide-qual @@ -1137,17 +1137,17 @@ GROUP BY o_d_id, o_w_id ORDER BY o_w_id, o_d_id ---- group-by - ├── columns: max:9!null [hidden: o_d_id:2!null o_w_id:3!null] + ├── columns: max:10!null [hidden: o_d_id:2!null o_w_id:3!null] ├── grouping columns: o_d_id:2!null o_w_id:3!null ├── key: (2,3) - ├── fd: (2,3)-->(9) + ├── fd: (2,3)-->(10) ├── ordering: +3,+2 ├── scan "order"@order_idx │ ├── columns: o_id:1!null o_d_id:2!null o_w_id:3!null │ ├── key: (1-3) │ └── ordering: +3,+2 └── aggregations - └── max [as=max:9, outer=(1)] + └── max [as=max:10, outer=(1)] └── o_id:1 opt format=hide-qual @@ -1161,36 +1161,36 @@ FROM WHERE nod != -1 ---- scalar-group-by - ├── columns: count:8!null + ├── columns: count:9!null ├── cardinality: [1 - 1] ├── immutable ├── key: () - ├── fd: ()-->(8) + ├── fd: ()-->(9) ├── select - │ ├── columns: no_d_id:2!null no_w_id:3!null max:4!null min:5!null count_rows:6!null + │ ├── columns: no_d_id:2!null no_w_id:3!null max:5!null min:6!null count_rows:7!null │ ├── immutable │ ├── key: (2,3) - │ ├── fd: (2,3)-->(4-6) + │ ├── fd: (2,3)-->(5-7) │ ├── group-by - │ │ ├── columns: no_d_id:2!null no_w_id:3!null max:4!null min:5!null count_rows:6!null + │ │ ├── columns: no_d_id:2!null no_w_id:3!null max:5!null min:6!null count_rows:7!null │ │ ├── grouping columns: no_d_id:2!null no_w_id:3!null │ │ ├── internal-ordering: +3,+2 │ │ ├── key: (2,3) - │ │ ├── fd: (2,3)-->(4-6) + │ │ ├── fd: (2,3)-->(5-7) │ │ ├── scan new_order │ │ │ ├── columns: no_o_id:1!null no_d_id:2!null no_w_id:3!null │ │ │ ├── key: (1-3) │ │ │ └── ordering: +3,+2 │ │ └── aggregations - │ │ ├── max [as=max:4, outer=(1)] + │ │ ├── max [as=max:5, outer=(1)] │ │ │ └── no_o_id:1 - │ │ ├── min [as=min:5, outer=(1)] + │ │ ├── min [as=min:6, outer=(1)] │ │ │ └── no_o_id:1 - │ │ └── count-rows [as=count_rows:6] + │ │ └── count-rows [as=count_rows:7] │ └── filters - │ └── ((max:4 - min:5) - count_rows:6) != -1 [outer=(4-6), immutable] + │ └── ((max:5 - min:6) - count_rows:7) != -1 [outer=(5-7), immutable] └── aggregations - └── count-rows [as=count_rows:8] + └── count-rows [as=count_rows:9] opt format=hide-qual SELECT sum(o_ol_cnt) @@ -1199,16 +1199,16 @@ GROUP BY o_w_id, o_d_id ORDER BY o_w_id, o_d_id ---- group-by - ├── columns: sum:9 [hidden: o_d_id:2!null o_w_id:3!null] + ├── columns: sum:10 [hidden: o_d_id:2!null o_w_id:3!null] ├── grouping columns: o_d_id:2!null o_w_id:3!null ├── key: (2,3) - ├── fd: (2,3)-->(9) + ├── fd: (2,3)-->(10) ├── ordering: +3,+2 ├── scan "order" │ ├── columns: o_d_id:2!null o_w_id:3!null o_ol_cnt:7 │ └── ordering: +3,+2 └── aggregations - └── sum [as=sum:9, outer=(7)] + └── sum [as=sum:10, outer=(7)] └── o_ol_cnt:7 opt format=hide-qual @@ -1218,16 +1218,16 @@ GROUP BY ol_w_id, ol_d_id ORDER BY ol_w_id, ol_d_id ---- group-by - ├── columns: count:11!null [hidden: ol_d_id:2!null ol_w_id:3!null] + ├── columns: count:12!null [hidden: ol_d_id:2!null ol_w_id:3!null] ├── grouping columns: ol_d_id:2!null ol_w_id:3!null ├── key: (2,3) - ├── fd: (2,3)-->(11) + ├── fd: (2,3)-->(12) ├── ordering: +3,+2 ├── scan order_line │ ├── columns: ol_d_id:2!null ol_w_id:3!null │ └── ordering: +3,+2 └── aggregations - └── count-rows [as=count_rows:11] + └── count-rows [as=count_rows:12] opt format=hide-qual (SELECT no_w_id, no_d_id, no_o_id FROM new_order) @@ -1237,23 +1237,23 @@ EXCEPT ALL except-all ├── columns: no_w_id:3!null no_d_id:2!null no_o_id:1!null ├── left columns: no_w_id:3!null no_d_id:2!null no_o_id:1!null - ├── right columns: o_w_id:6 o_d_id:5 o_id:4 + ├── right columns: o_w_id:7 o_d_id:6 o_id:5 ├── scan new_order │ ├── columns: no_o_id:1!null no_d_id:2!null no_w_id:3!null │ └── key: (1-3) └── project - ├── columns: o_id:4!null o_d_id:5!null o_w_id:6!null - ├── key: (4-6) + ├── columns: o_id:5!null o_d_id:6!null o_w_id:7!null + ├── key: (5-7) └── select - ├── columns: o_id:4!null o_d_id:5!null o_w_id:6!null o_carrier_id:9 - ├── key: (4-6) - ├── fd: ()-->(9) + ├── columns: o_id:5!null o_d_id:6!null o_w_id:7!null o_carrier_id:10 + ├── key: (5-7) + ├── fd: ()-->(10) ├── scan "order"@order_idx - │ ├── columns: o_id:4!null o_d_id:5!null o_w_id:6!null o_carrier_id:9 - │ ├── key: (4-6) - │ └── fd: (4-6)-->(9) + │ ├── columns: o_id:5!null o_d_id:6!null o_w_id:7!null o_carrier_id:10 + │ ├── key: (5-7) + │ └── fd: (5-7)-->(10) └── filters - └── o_carrier_id:9 IS NULL [outer=(9), constraints=(/9: [/NULL - /NULL]; tight), fd=()-->(9)] + └── o_carrier_id:10 IS NULL [outer=(10), constraints=(/10: [/NULL - /NULL]; tight), fd=()-->(10)] opt format=hide-qual (SELECT o_w_id, o_d_id, o_id FROM "order" WHERE o_carrier_id IS NULL) @@ -1263,7 +1263,7 @@ EXCEPT ALL except-all ├── columns: o_w_id:3!null o_d_id:2!null o_id:1!null ├── left columns: o_w_id:3!null o_d_id:2!null o_id:1!null - ├── right columns: no_w_id:11 no_d_id:10 no_o_id:9 + ├── right columns: no_w_id:12 no_d_id:11 no_o_id:10 ├── project │ ├── columns: o_id:1!null o_d_id:2!null o_w_id:3!null │ ├── key: (1-3) @@ -1278,8 +1278,8 @@ except-all │ └── filters │ └── o_carrier_id:6 IS NULL [outer=(6), constraints=(/6: [/NULL - /NULL]; tight), fd=()-->(6)] └── scan new_order - ├── columns: no_o_id:9!null no_d_id:10!null no_w_id:11!null - └── key: (9-11) + ├── columns: no_o_id:10!null no_d_id:11!null no_w_id:12!null + └── key: (10-12) opt format=hide-qual ( @@ -1298,20 +1298,20 @@ EXCEPT ALL except-all ├── columns: o_w_id:3!null o_d_id:2!null o_id:1!null o_ol_cnt:7 ├── left columns: o_w_id:3!null o_d_id:2!null o_id:1!null o_ol_cnt:7 - ├── right columns: ol_w_id:11 ol_d_id:10 ol_o_id:9 count_rows:19 + ├── right columns: ol_w_id:12 ol_d_id:11 ol_o_id:10 count_rows:21 ├── scan "order" │ ├── columns: o_id:1!null o_d_id:2!null o_w_id:3!null o_ol_cnt:7 │ ├── key: (1-3) │ └── fd: (1-3)-->(7) └── group-by - ├── columns: ol_o_id:9!null ol_d_id:10!null ol_w_id:11!null count_rows:19!null - ├── grouping columns: ol_o_id:9!null ol_d_id:10!null ol_w_id:11!null - ├── key: (9-11) - ├── fd: (9-11)-->(19) + ├── columns: ol_o_id:10!null ol_d_id:11!null ol_w_id:12!null count_rows:21!null + ├── grouping columns: ol_o_id:10!null ol_d_id:11!null ol_w_id:12!null + ├── key: (10-12) + ├── fd: (10-12)-->(21) ├── scan order_line@order_line_stock_fk_idx - │ └── columns: ol_o_id:9!null ol_d_id:10!null ol_w_id:11!null + │ └── columns: ol_o_id:10!null ol_d_id:11!null ol_w_id:12!null └── aggregations - └── count-rows [as=count_rows:19] + └── count-rows [as=count_rows:21] opt format=hide-qual ( @@ -1328,22 +1328,22 @@ EXCEPT ALL ) ---- except-all - ├── columns: ol_w_id:3!null ol_d_id:2!null ol_o_id:1!null count:11 - ├── left columns: ol_w_id:3!null ol_d_id:2!null ol_o_id:1!null count_rows:11 - ├── right columns: o_w_id:14 o_d_id:13 o_id:12 o_ol_cnt:18 + ├── columns: ol_w_id:3!null ol_d_id:2!null ol_o_id:1!null count:12 + ├── left columns: ol_w_id:3!null ol_d_id:2!null ol_o_id:1!null count_rows:12 + ├── right columns: o_w_id:15 o_d_id:14 o_id:13 o_ol_cnt:19 ├── group-by - │ ├── columns: ol_o_id:1!null ol_d_id:2!null ol_w_id:3!null count_rows:11!null + │ ├── columns: ol_o_id:1!null ol_d_id:2!null ol_w_id:3!null count_rows:12!null │ ├── grouping columns: ol_o_id:1!null ol_d_id:2!null ol_w_id:3!null │ ├── key: (1-3) - │ ├── fd: (1-3)-->(11) + │ ├── fd: (1-3)-->(12) │ ├── scan order_line@order_line_stock_fk_idx │ │ └── columns: ol_o_id:1!null ol_d_id:2!null ol_w_id:3!null │ └── aggregations - │ └── count-rows [as=count_rows:11] + │ └── count-rows [as=count_rows:12] └── scan "order" - ├── columns: o_id:12!null o_d_id:13!null o_w_id:14!null o_ol_cnt:18 - ├── key: (12-14) - └── fd: (12-14)-->(18) + ├── columns: o_id:13!null o_d_id:14!null o_w_id:15!null o_ol_cnt:19 + ├── key: (13-15) + └── fd: (13-15)-->(19) opt format=hide-qual SELECT count(*) @@ -1363,14 +1363,14 @@ ON (ol_w_id = o_w_id AND ol_d_id = o_d_id AND ol_o_id = o_id) WHERE ol_o_id IS NULL OR o_id IS NULL ---- scalar-group-by - ├── columns: count:19!null + ├── columns: count:21!null ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(19) + ├── fd: ()-->(21) ├── select - │ ├── columns: o_id:1 o_d_id:2 o_w_id:3 ol_o_id:9 ol_d_id:10 ol_w_id:11 + │ ├── columns: o_id:1 o_d_id:2 o_w_id:3 ol_o_id:10 ol_d_id:11 ol_w_id:12 │ ├── full-join (hash) - │ │ ├── columns: o_id:1 o_d_id:2 o_w_id:3 ol_o_id:9 ol_d_id:10 ol_w_id:11 + │ │ ├── columns: o_id:1 o_d_id:2 o_w_id:3 ol_o_id:10 ol_d_id:11 ol_w_id:12 │ │ ├── multiplicity: left-rows(one-or-more), right-rows(exactly-one) │ │ ├── project │ │ │ ├── columns: o_id:1!null o_d_id:2!null o_w_id:3!null @@ -1386,19 +1386,19 @@ scalar-group-by │ │ │ └── filters │ │ │ └── o_carrier_id:6 IS NULL [outer=(6), constraints=(/6: [/NULL - /NULL]; tight), fd=()-->(6)] │ │ ├── project - │ │ │ ├── columns: ol_o_id:9!null ol_d_id:10!null ol_w_id:11!null + │ │ │ ├── columns: ol_o_id:10!null ol_d_id:11!null ol_w_id:12!null │ │ │ └── select - │ │ │ ├── columns: ol_o_id:9!null ol_d_id:10!null ol_w_id:11!null ol_delivery_d:15 - │ │ │ ├── fd: ()-->(15) + │ │ │ ├── columns: ol_o_id:10!null ol_d_id:11!null ol_w_id:12!null ol_delivery_d:16 + │ │ │ ├── fd: ()-->(16) │ │ │ ├── scan order_line - │ │ │ │ └── columns: ol_o_id:9!null ol_d_id:10!null ol_w_id:11!null ol_delivery_d:15 + │ │ │ │ └── columns: ol_o_id:10!null ol_d_id:11!null ol_w_id:12!null ol_delivery_d:16 │ │ │ └── filters - │ │ │ └── ol_delivery_d:15 IS NULL [outer=(15), constraints=(/15: [/NULL - /NULL]; tight), fd=()-->(15)] + │ │ │ └── ol_delivery_d:16 IS NULL [outer=(16), constraints=(/16: [/NULL - /NULL]; tight), fd=()-->(16)] │ │ └── filters - │ │ ├── ol_w_id:11 = o_w_id:3 [outer=(3,11), constraints=(/3: (/NULL - ]; /11: (/NULL - ]), fd=(3)==(11), (11)==(3)] - │ │ ├── ol_d_id:10 = o_d_id:2 [outer=(2,10), constraints=(/2: (/NULL - ]; /10: (/NULL - ]), fd=(2)==(10), (10)==(2)] - │ │ └── ol_o_id:9 = o_id:1 [outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ]), fd=(1)==(9), (9)==(1)] + │ │ ├── ol_w_id:12 = o_w_id:3 [outer=(3,12), constraints=(/3: (/NULL - ]; /12: (/NULL - ]), fd=(3)==(12), (12)==(3)] + │ │ ├── ol_d_id:11 = o_d_id:2 [outer=(2,11), constraints=(/2: (/NULL - ]; /11: (/NULL - ]), fd=(2)==(11), (11)==(2)] + │ │ └── ol_o_id:10 = o_id:1 [outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] │ └── filters - │ └── (ol_o_id:9 IS NULL) OR (o_id:1 IS NULL) [outer=(1,9)] + │ └── (ol_o_id:10 IS NULL) OR (o_id:1 IS NULL) [outer=(1,10)] └── aggregations - └── count-rows [as=count_rows:19] + └── count-rows [as=count_rows:21] diff --git a/pkg/sql/opt/xform/testdata/external/tpce-no-stats b/pkg/sql/opt/xform/testdata/external/tpce-no-stats index 385c3c5af13d..b18c82794ef1 100644 --- a/pkg/sql/opt/xform/testdata/external/tpce-no-stats +++ b/pkg/sql/opt/xform/testdata/external/tpce-no-stats @@ -39,32 +39,32 @@ GROUP BY b_name ORDER BY 2 DESC ---- sort - ├── columns: b_name:39!null sum:44!null + ├── columns: b_name:44!null sum:50!null ├── immutable - ├── key: (39) - ├── fd: (39)-->(44) - ├── ordering: -44 + ├── key: (44) + ├── fd: (44)-->(50) + ├── ordering: -50 └── project - ├── columns: sum:44!null b_name:39!null + ├── columns: sum:50!null b_name:44!null ├── immutable - ├── key: (39) - ├── fd: (39)-->(44) + ├── key: (44) + ├── fd: (44)-->(50) ├── group-by - │ ├── columns: b_name:39!null sum:43!null - │ ├── grouping columns: b_name:39!null + │ ├── columns: b_name:44!null sum:49!null + │ ├── grouping columns: b_name:44!null │ ├── immutable - │ ├── key: (39) - │ ├── fd: (39)-->(43) + │ ├── key: (44) + │ ├── fd: (44)-->(49) │ ├── project - │ │ ├── columns: column42:42!null b_name:39!null + │ │ ├── columns: column48:48!null b_name:44!null │ │ ├── immutable │ │ ├── inner-join (hash) - │ │ │ ├── columns: sc_id:1!null sc_name:2!null in_id:3!null in_sc_id:5!null co_id:6!null co_in_id:9!null s_symb:15!null s_co_id:20!null tr_s_symb:33!null tr_qty:34!null tr_bid_price:35!null tr_b_id:36!null b_id:37!null b_name:39!null + │ │ │ ├── columns: sc_id:1!null sc_name:2!null in_id:4!null in_sc_id:6!null co_id:8!null co_in_id:11!null s_symb:18!null s_co_id:23!null tr_s_symb:37!null tr_qty:38!null tr_bid_price:39!null tr_b_id:40!null b_id:42!null b_name:44!null │ │ │ ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) - │ │ │ ├── fd: ()-->(1,2,5), (6)-->(9), (15)-->(20), (37)-->(39), (39)-->(37), (36)==(37), (37)==(36), (15)==(33), (33)==(15), (6)==(20), (20)==(6), (3)==(9), (9)==(3), (1)==(5), (5)==(1) + │ │ │ ├── fd: ()-->(1,2,6), (8)-->(11), (18)-->(23), (42)-->(44), (44)-->(42), (40)==(42), (42)==(40), (18)==(37), (37)==(18), (8)==(23), (23)==(8), (4)==(11), (11)==(4), (1)==(6), (6)==(1) │ │ │ ├── scan broker@secondary - │ │ │ │ ├── columns: b_id:37!null b_name:39!null - │ │ │ │ ├── constraint: /39 + │ │ │ │ ├── columns: b_id:42!null b_name:44!null + │ │ │ │ ├── constraint: /44 │ │ │ │ │ ├── [/'Broker1' - /'Broker1'] │ │ │ │ │ ├── [/'Broker10' - /'Broker10'] │ │ │ │ │ ├── [/'Broker11' - /'Broker11'] @@ -96,27 +96,27 @@ sort │ │ │ │ │ ├── [/'Broker8' - /'Broker8'] │ │ │ │ │ └── [/'Broker9' - /'Broker9'] │ │ │ │ ├── cardinality: [0 - 30] - │ │ │ │ ├── key: (37) - │ │ │ │ └── fd: (37)-->(39), (39)-->(37) + │ │ │ │ ├── key: (42) + │ │ │ │ └── fd: (42)-->(44), (44)-->(42) │ │ │ ├── inner-join (lookup trade_request@secondary) - │ │ │ │ ├── columns: sc_id:1!null sc_name:2!null in_id:3!null in_sc_id:5!null co_id:6!null co_in_id:9!null s_symb:15!null s_co_id:20!null tr_s_symb:33!null tr_qty:34!null tr_bid_price:35!null tr_b_id:36!null - │ │ │ │ ├── key columns: [15] = [33] - │ │ │ │ ├── fd: ()-->(1,2,5), (6)-->(9), (15)-->(20), (15)==(33), (33)==(15), (6)==(20), (20)==(6), (3)==(9), (9)==(3), (1)==(5), (5)==(1) + │ │ │ │ ├── columns: sc_id:1!null sc_name:2!null in_id:4!null in_sc_id:6!null co_id:8!null co_in_id:11!null s_symb:18!null s_co_id:23!null tr_s_symb:37!null tr_qty:38!null tr_bid_price:39!null tr_b_id:40!null + │ │ │ │ ├── key columns: [18] = [37] + │ │ │ │ ├── fd: ()-->(1,2,6), (8)-->(11), (18)-->(23), (18)==(37), (37)==(18), (8)==(23), (23)==(8), (4)==(11), (11)==(4), (1)==(6), (6)==(1) │ │ │ │ ├── inner-join (lookup security@secondary) - │ │ │ │ │ ├── columns: sc_id:1!null sc_name:2!null in_id:3!null in_sc_id:5!null co_id:6!null co_in_id:9!null s_symb:15!null s_co_id:20!null - │ │ │ │ │ ├── key columns: [6] = [20] - │ │ │ │ │ ├── key: (15) - │ │ │ │ │ ├── fd: ()-->(1,2,5), (6)-->(9), (15)-->(20), (6)==(20), (20)==(6), (3)==(9), (9)==(3), (1)==(5), (5)==(1) + │ │ │ │ │ ├── columns: sc_id:1!null sc_name:2!null in_id:4!null in_sc_id:6!null co_id:8!null co_in_id:11!null s_symb:18!null s_co_id:23!null + │ │ │ │ │ ├── key columns: [8] = [23] + │ │ │ │ │ ├── key: (18) + │ │ │ │ │ ├── fd: ()-->(1,2,6), (8)-->(11), (18)-->(23), (8)==(23), (23)==(8), (4)==(11), (11)==(4), (1)==(6), (6)==(1) │ │ │ │ │ ├── inner-join (lookup company@company_auto_index_fk_co_in_id_ref_industry) - │ │ │ │ │ │ ├── columns: sc_id:1!null sc_name:2!null in_id:3!null in_sc_id:5!null co_id:6!null co_in_id:9!null - │ │ │ │ │ │ ├── key columns: [3] = [9] - │ │ │ │ │ │ ├── key: (6) - │ │ │ │ │ │ ├── fd: ()-->(1,2,5), (6)-->(9), (3)==(9), (9)==(3), (1)==(5), (5)==(1) + │ │ │ │ │ │ ├── columns: sc_id:1!null sc_name:2!null in_id:4!null in_sc_id:6!null co_id:8!null co_in_id:11!null + │ │ │ │ │ │ ├── key columns: [4] = [11] + │ │ │ │ │ │ ├── key: (8) + │ │ │ │ │ │ ├── fd: ()-->(1,2,6), (8)-->(11), (4)==(11), (11)==(4), (1)==(6), (6)==(1) │ │ │ │ │ │ ├── inner-join (lookup industry@industry_auto_index_fk_in_sc_id_ref_sector) - │ │ │ │ │ │ │ ├── columns: sc_id:1!null sc_name:2!null in_id:3!null in_sc_id:5!null - │ │ │ │ │ │ │ ├── key columns: [1] = [5] - │ │ │ │ │ │ │ ├── key: (3) - │ │ │ │ │ │ │ ├── fd: ()-->(1,2,5), (1)==(5), (5)==(1) + │ │ │ │ │ │ │ ├── columns: sc_id:1!null sc_name:2!null in_id:4!null in_sc_id:6!null + │ │ │ │ │ │ │ ├── key columns: [1] = [6] + │ │ │ │ │ │ │ ├── key: (4) + │ │ │ │ │ │ │ ├── fd: ()-->(1,2,6), (1)==(6), (6)==(1) │ │ │ │ │ │ │ ├── scan sector@secondary │ │ │ │ │ │ │ │ ├── columns: sc_id:1!null sc_name:2!null │ │ │ │ │ │ │ │ ├── constraint: /2: [/'Energy' - /'Energy'] @@ -128,14 +128,14 @@ sort │ │ │ │ │ └── filters (true) │ │ │ │ └── filters (true) │ │ │ └── filters - │ │ │ └── tr_b_id:36 = b_id:37 [outer=(36,37), constraints=(/36: (/NULL - ]; /37: (/NULL - ]), fd=(36)==(37), (37)==(36)] + │ │ │ └── tr_b_id:40 = b_id:42 [outer=(40,42), constraints=(/40: (/NULL - ]; /42: (/NULL - ]), fd=(40)==(42), (42)==(40)] │ │ └── projections - │ │ └── tr_qty:34::INT8 * tr_bid_price:35::DECIMAL [as=column42:42, outer=(34,35), immutable] + │ │ └── tr_qty:38::INT8 * tr_bid_price:39::DECIMAL [as=column48:48, outer=(38,39), immutable] │ └── aggregations - │ └── sum [as=sum:43, outer=(42)] - │ └── column42:42 + │ └── sum [as=sum:49, outer=(48)] + │ └── column48:48 └── projections - └── sum:43::FLOAT8 [as=sum:44, outer=(43), immutable] + └── sum:49::FLOAT8 [as=sum:50, outer=(49), immutable] # -------------------------------------------------- # T2 @@ -213,49 +213,49 @@ LEFT JOIN holding_summary ON hs_ca_id = ca_id LIMIT 10 ---- limit - ├── columns: ca_id:1!null ca_bal:17!null coalesce:18 - ├── internal-ordering: +18 + ├── columns: ca_id:1!null ca_bal:20!null coalesce:21 + ├── internal-ordering: +21 ├── cardinality: [0 - 10] ├── immutable ├── key: (1) - ├── fd: (1)-->(17,18) - ├── ordering: +18 + ├── fd: (1)-->(20,21) + ├── ordering: +21 ├── sort - │ ├── columns: ca_id:1!null ca_bal:17!null coalesce:18 + │ ├── columns: ca_id:1!null ca_bal:20!null coalesce:21 │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(17,18) - │ ├── ordering: +18 + │ ├── fd: (1)-->(20,21) + │ ├── ordering: +21 │ ├── limit hint: 10.00 │ └── project - │ ├── columns: ca_bal:17!null coalesce:18 ca_id:1!null + │ ├── columns: ca_bal:20!null coalesce:21 ca_id:1!null │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(17,18) + │ ├── fd: (1)-->(20,21) │ ├── group-by - │ │ ├── columns: ca_id:1!null customer_account.ca_bal:6!null sum:16 + │ │ ├── columns: ca_id:1!null customer_account.ca_bal:6!null sum:19 │ │ ├── grouping columns: ca_id:1!null │ │ ├── internal-ordering: +1 │ │ ├── immutable │ │ ├── key: (1) - │ │ ├── fd: (1)-->(6,16) + │ │ ├── fd: (1)-->(6,19) │ │ ├── project - │ │ │ ├── columns: column15:15 ca_id:1!null customer_account.ca_bal:6!null + │ │ │ ├── columns: column18:18 ca_id:1!null customer_account.ca_bal:6!null │ │ │ ├── immutable │ │ │ ├── fd: (1)-->(6) │ │ │ ├── ordering: +1 │ │ │ ├── inner-join (lookup last_trade) - │ │ │ │ ├── columns: ca_id:1!null ca_c_id:3!null customer_account.ca_bal:6!null hs_ca_id:7 hs_s_symb:8!null hs_qty:9 lt_s_symb:10!null lt_price:12!null - │ │ │ │ ├── key columns: [8] = [10] + │ │ │ │ ├── columns: ca_id:1!null ca_c_id:3!null customer_account.ca_bal:6!null hs_ca_id:8 hs_s_symb:9!null hs_qty:10 lt_s_symb:12!null lt_price:14!null + │ │ │ │ ├── key columns: [9] = [12] │ │ │ │ ├── lookup columns are key - │ │ │ │ ├── key: (1,7,10) - │ │ │ │ ├── fd: ()-->(3), (1)-->(6), (7,8)-->(9), (10)-->(12), (8)==(10), (10)==(8) + │ │ │ │ ├── key: (1,8,12) + │ │ │ │ ├── fd: ()-->(3), (1)-->(6), (8,9)-->(10), (12)-->(14), (9)==(12), (12)==(9) │ │ │ │ ├── ordering: +1 opt(3) [actual: +1] │ │ │ │ ├── left-join (lookup holding_summary) - │ │ │ │ │ ├── columns: ca_id:1!null ca_c_id:3!null customer_account.ca_bal:6!null hs_ca_id:7 hs_s_symb:8 hs_qty:9 - │ │ │ │ │ ├── key columns: [1] = [7] - │ │ │ │ │ ├── key: (1,7,8) - │ │ │ │ │ ├── fd: ()-->(3), (1)-->(6), (7,8)-->(9) + │ │ │ │ │ ├── columns: ca_id:1!null ca_c_id:3!null customer_account.ca_bal:6!null hs_ca_id:8 hs_s_symb:9 hs_qty:10 + │ │ │ │ │ ├── key columns: [1] = [8] + │ │ │ │ │ ├── key: (1,8,9) + │ │ │ │ │ ├── fd: ()-->(3), (1)-->(6), (8,9)-->(10) │ │ │ │ │ ├── ordering: +1 opt(3) [actual: +1] │ │ │ │ │ ├── index-join customer_account │ │ │ │ │ │ ├── columns: ca_id:1!null ca_c_id:3!null customer_account.ca_bal:6!null @@ -271,15 +271,15 @@ limit │ │ │ │ │ └── filters (true) │ │ │ │ └── filters (true) │ │ │ └── projections - │ │ │ └── hs_qty:9::INT8 * lt_price:12::DECIMAL [as=column15:15, outer=(9,12), immutable] + │ │ │ └── hs_qty:10::INT8 * lt_price:14::DECIMAL [as=column18:18, outer=(10,14), immutable] │ │ └── aggregations - │ │ ├── sum [as=sum:16, outer=(15)] - │ │ │ └── column15:15 + │ │ ├── sum [as=sum:19, outer=(18)] + │ │ │ └── column18:18 │ │ └── const-agg [as=customer_account.ca_bal:6, outer=(6)] │ │ └── customer_account.ca_bal:6 │ └── projections - │ ├── customer_account.ca_bal:6::FLOAT8 [as=ca_bal:17, outer=(6), immutable] - │ └── COALESCE(sum:16, 0)::FLOAT8 [as=coalesce:18, outer=(16), immutable] + │ ├── customer_account.ca_bal:6::FLOAT8 [as=ca_bal:20, outer=(6), immutable] + │ └── COALESCE(sum:19, 0)::FLOAT8 [as=coalesce:21, outer=(19), immutable] └── 10 # Q4 @@ -299,36 +299,36 @@ ORDER BY th_dts DESC LIMIT 30 ---- project - ├── columns: t_id:1!null t_s_symb:6!null t_qty:7!null st_name:20!null th_dts:17!null + ├── columns: t_id:1!null t_s_symb:6!null t_qty:7!null st_name:22!null th_dts:18!null ├── cardinality: [0 - 30] ├── fd: (1)-->(6,7) - ├── ordering: -17 + ├── ordering: -18 └── limit - ├── columns: t_id:1!null t_dts:2!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null th_t_id:16!null th_dts:17!null th_st_id:18!null st_id:19!null st_name:20!null - ├── internal-ordering: -17 opt(9) + ├── columns: t_id:1!null t_dts:2!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null th_t_id:17!null th_dts:18!null th_st_id:19!null st_id:21!null st_name:22!null + ├── internal-ordering: -18 opt(9) ├── cardinality: [0 - 30] - ├── key: (16,19) - ├── fd: ()-->(9), (1)-->(2,6,7), (16,18)-->(17), (19)-->(20), (18)==(19), (19)==(18), (1)==(16), (16)==(1) - ├── ordering: -17 opt(9) [actual: -17] + ├── key: (17,21) + ├── fd: ()-->(9), (1)-->(2,6,7), (17,19)-->(18), (21)-->(22), (19)==(21), (21)==(19), (1)==(17), (17)==(1) + ├── ordering: -18 opt(9) [actual: -18] ├── inner-join (lookup status_type) - │ ├── columns: t_id:1!null t_dts:2!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null th_t_id:16!null th_dts:17!null th_st_id:18!null st_id:19!null st_name:20!null - │ ├── key columns: [18] = [19] + │ ├── columns: t_id:1!null t_dts:2!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null th_t_id:17!null th_dts:18!null th_st_id:19!null st_id:21!null st_name:22!null + │ ├── key columns: [19] = [21] │ ├── lookup columns are key - │ ├── key: (16,19) - │ ├── fd: ()-->(9), (1)-->(2,6,7), (16,18)-->(17), (19)-->(20), (18)==(19), (19)==(18), (1)==(16), (16)==(1) - │ ├── ordering: -17 opt(9) [actual: -17] + │ ├── key: (17,21) + │ ├── fd: ()-->(9), (1)-->(2,6,7), (17,19)-->(18), (21)-->(22), (19)==(21), (21)==(19), (1)==(17), (17)==(1) + │ ├── ordering: -18 opt(9) [actual: -18] │ ├── limit hint: 30.00 │ ├── sort - │ │ ├── columns: t_id:1!null t_dts:2!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null th_t_id:16!null th_dts:17!null th_st_id:18!null - │ │ ├── key: (16,18) - │ │ ├── fd: ()-->(9), (1)-->(2,6,7), (16,18)-->(17), (1)==(16), (16)==(1) - │ │ ├── ordering: -17 opt(9) [actual: -17] + │ │ ├── columns: t_id:1!null t_dts:2!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null th_t_id:17!null th_dts:18!null th_st_id:19!null + │ │ ├── key: (17,19) + │ │ ├── fd: ()-->(9), (1)-->(2,6,7), (17,19)-->(18), (1)==(17), (17)==(1) + │ │ ├── ordering: -18 opt(9) [actual: -18] │ │ ├── limit hint: 100.00 │ │ └── inner-join (lookup trade_history) - │ │ ├── columns: t_id:1!null t_dts:2!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null th_t_id:16!null th_dts:17!null th_st_id:18!null - │ │ ├── key columns: [1] = [16] - │ │ ├── key: (16,18) - │ │ ├── fd: ()-->(9), (1)-->(2,6,7), (16,18)-->(17), (1)==(16), (16)==(1) + │ │ ├── columns: t_id:1!null t_dts:2!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null th_t_id:17!null th_dts:18!null th_st_id:19!null + │ │ ├── key columns: [1] = [17] + │ │ ├── key: (17,19) + │ │ ├── fd: ()-->(9), (1)-->(2,6,7), (17,19)-->(18), (1)==(17), (17)==(1) │ │ ├── scan trade@secondary │ │ │ ├── columns: t_id:1!null t_dts:2!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null │ │ │ ├── constraint: /9/-2/1: [/0 - /0] @@ -357,29 +357,29 @@ UPDATE last_trade ---- update last_trade ├── columns: - ├── fetch columns: lt_s_symb:6 lt_dts:7 last_trade.lt_price:8 lt_open_price:9 lt_vol:10 + ├── fetch columns: lt_s_symb:7 lt_dts:8 last_trade.lt_price:9 lt_open_price:10 lt_vol:11 ├── update-mapping: - │ ├── lt_dts_new:13 => lt_dts:2 - │ ├── lt_price:14 => last_trade.lt_price:3 - │ └── lt_vol_new:12 => lt_vol:5 + │ ├── lt_dts_new:15 => lt_dts:2 + │ ├── lt_price:16 => last_trade.lt_price:3 + │ └── lt_vol_new:14 => lt_vol:5 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: lt_price:14!null lt_vol_new:12!null lt_dts_new:13!null lt_s_symb:6!null lt_dts:7!null last_trade.lt_price:8!null lt_open_price:9!null lt_vol:10!null + ├── columns: lt_price:16!null lt_vol_new:14!null lt_dts_new:15!null lt_s_symb:7!null lt_dts:8!null last_trade.lt_price:9!null lt_open_price:10!null lt_vol:11!null ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(6-10,12-14) + ├── fd: ()-->(7-11,14-16) ├── scan last_trade - │ ├── columns: lt_s_symb:6!null lt_dts:7!null last_trade.lt_price:8!null lt_open_price:9!null lt_vol:10!null - │ ├── constraint: /6: [/'SYMB' - /'SYMB'] + │ ├── columns: lt_s_symb:7!null lt_dts:8!null last_trade.lt_price:9!null lt_open_price:10!null lt_vol:11!null + │ ├── constraint: /7: [/'SYMB' - /'SYMB'] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(6-10) + │ └── fd: ()-->(7-11) └── projections - ├── 1E+2 [as=lt_price:14] - ├── lt_vol:10 + 20 [as=lt_vol_new:12, outer=(10), immutable] - └── '2020-06-17 22:27:42.148484+00:00' [as=lt_dts_new:13] + ├── 1E+2 [as=lt_price:16] + ├── lt_vol:11 + 20 [as=lt_vol_new:14, outer=(11), immutable] + └── '2020-06-17 22:27:42.148484+00:00' [as=lt_dts_new:15] # Q2 opt @@ -393,10 +393,10 @@ SELECT tr_t_id, tr_bid_price::FLOAT8, tr_tt_id, tr_qty ) ---- project - ├── columns: tr_t_id:1!null tr_bid_price:7!null tr_tt_id:2!null tr_qty:4!null + ├── columns: tr_t_id:1!null tr_bid_price:8!null tr_tt_id:2!null tr_qty:4!null ├── immutable ├── key: (1) - ├── fd: (1)-->(2,4,7) + ├── fd: (1)-->(2,4,8) ├── select │ ├── columns: tr_t_id:1!null tr_tt_id:2!null tr_s_symb:3!null tr_qty:4!null trade_request.tr_bid_price:5!null │ ├── immutable @@ -414,7 +414,7 @@ project │ └── filters │ └── (((tr_tt_id:2 = 'Stop-Loss') AND (trade_request.tr_bid_price:5 >= 1E+2)) OR ((tr_tt_id:2 = 'Limit-Sell') AND (trade_request.tr_bid_price:5 <= 1E+2))) OR ((tr_tt_id:2 = 'Limit-Buy') AND (trade_request.tr_bid_price:5 >= 1E+2)) [outer=(2,5), immutable, constraints=(/2: [/'Limit-Buy' - /'Limit-Buy'] [/'Limit-Sell' - /'Limit-Sell'] [/'Stop-Loss' - /'Stop-Loss']; /5: (/NULL - ])] └── projections - └── trade_request.tr_bid_price:5::FLOAT8 [as=tr_bid_price:7, outer=(5), immutable] + └── trade_request.tr_bid_price:5::FLOAT8 [as=tr_bid_price:8, outer=(5), immutable] # Q3 opt @@ -424,50 +424,50 @@ UPDATE trade ---- update trade ├── columns: - ├── fetch columns: t_id:16 t_dts:17 t_st_id:18 t_tt_id:19 t_is_cash:20 t_s_symb:21 t_qty:22 t_bid_price:23 t_ca_id:24 t_exec_name:25 t_trade_price:26 t_chrg:27 t_comm:28 t_tax:29 t_lifo:30 + ├── fetch columns: t_id:17 t_dts:18 t_st_id:19 t_tt_id:20 t_is_cash:21 t_s_symb:22 t_qty:23 t_bid_price:24 t_ca_id:25 t_exec_name:26 t_trade_price:27 t_chrg:28 t_comm:29 t_tax:30 t_lifo:31 ├── update-mapping: - │ ├── t_dts_new:31 => t_dts:2 - │ └── t_st_id_new:32 => t_st_id:3 - ├── check columns: check1:33 check2:34 check3:35 check4:36 check5:37 + │ ├── t_dts_new:33 => t_dts:2 + │ └── t_st_id_new:34 => t_st_id:3 + ├── check columns: check1:35 check2:36 check3:37 check4:38 check5:39 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── project - │ ├── columns: check1:33!null check2:34 check3:35!null check4:36!null check5:37!null t_dts_new:31!null t_st_id_new:32!null t_id:16!null t_dts:17!null t_st_id:18!null t_tt_id:19!null t_is_cash:20!null t_s_symb:21!null t_qty:22!null t_bid_price:23 t_ca_id:24!null t_exec_name:25!null t_trade_price:26 t_chrg:27!null t_comm:28!null t_tax:29!null t_lifo:30!null + │ ├── columns: check1:35!null check2:36 check3:37!null check4:38!null check5:39!null t_dts_new:33!null t_st_id_new:34!null t_id:17!null t_dts:18!null t_st_id:19!null t_tt_id:20!null t_is_cash:21!null t_s_symb:22!null t_qty:23!null t_bid_price:24 t_ca_id:25!null t_exec_name:26!null t_trade_price:27 t_chrg:28!null t_comm:29!null t_tax:30!null t_lifo:31!null │ ├── cardinality: [0 - 1] │ ├── immutable │ ├── key: () - │ ├── fd: ()-->(16-37) + │ ├── fd: ()-->(17-31,33-39) │ ├── scan trade - │ │ ├── columns: t_id:16!null t_dts:17!null t_st_id:18!null t_tt_id:19!null t_is_cash:20!null t_s_symb:21!null t_qty:22!null t_bid_price:23 t_ca_id:24!null t_exec_name:25!null t_trade_price:26 t_chrg:27!null t_comm:28!null t_tax:29!null t_lifo:30!null - │ │ ├── constraint: /16: [/0 - /0] + │ │ ├── columns: t_id:17!null t_dts:18!null t_st_id:19!null t_tt_id:20!null t_is_cash:21!null t_s_symb:22!null t_qty:23!null t_bid_price:24 t_ca_id:25!null t_exec_name:26!null t_trade_price:27 t_chrg:28!null t_comm:29!null t_tax:30!null t_lifo:31!null + │ │ ├── constraint: /17: [/0 - /0] │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(16-30) + │ │ └── fd: ()-->(17-31) │ └── projections - │ ├── t_qty:22 > 0 [as=check1:33, outer=(22)] - │ ├── t_bid_price:23 > 0 [as=check2:34, outer=(23), immutable] - │ ├── t_chrg:27 >= 0 [as=check3:35, outer=(27), immutable] - │ ├── t_comm:28 >= 0 [as=check4:36, outer=(28), immutable] - │ ├── t_tax:29 >= 0 [as=check5:37, outer=(29), immutable] - │ ├── '2020-06-15 22:27:42.148484+00:00' [as=t_dts_new:31] - │ └── 'SBMT' [as=t_st_id_new:32] + │ ├── t_qty:23 > 0 [as=check1:35, outer=(23)] + │ ├── t_bid_price:24 > 0 [as=check2:36, outer=(24), immutable] + │ ├── t_chrg:28 >= 0 [as=check3:37, outer=(28), immutable] + │ ├── t_comm:29 >= 0 [as=check4:38, outer=(29), immutable] + │ ├── t_tax:30 >= 0 [as=check5:39, outer=(30), immutable] + │ ├── '2020-06-15 22:27:42.148484+00:00' [as=t_dts_new:33] + │ └── 'SBMT' [as=t_st_id_new:34] └── f-k-checks └── f-k-checks-item: trade(t_st_id) -> status_type(st_id) └── anti-join (lookup status_type) - ├── columns: t_st_id_new:38!null - ├── key columns: [38] = [39] + ├── columns: t_st_id_new:40!null + ├── key columns: [40] = [41] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(38) + ├── fd: ()-->(40) ├── with-scan &1 - │ ├── columns: t_st_id_new:38!null + │ ├── columns: t_st_id_new:40!null │ ├── mapping: - │ │ └── t_st_id_new:32 => t_st_id_new:38 + │ │ └── t_st_id_new:34 => t_st_id_new:40 │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(38) + │ └── fd: ()-->(40) └── filters (true) # Q4 @@ -476,15 +476,15 @@ DELETE FROM trade_request WHERE tr_t_id = 0 ---- delete trade_request ├── columns: - ├── fetch columns: tr_t_id:7 tr_tt_id:8 tr_s_symb:9 tr_b_id:12 + ├── fetch columns: tr_t_id:8 tr_tt_id:9 tr_s_symb:10 tr_b_id:13 ├── cardinality: [0 - 0] ├── volatile, mutations └── scan trade_request - ├── columns: tr_t_id:7!null tr_tt_id:8!null tr_s_symb:9!null tr_b_id:12!null - ├── constraint: /7: [/0 - /0] + ├── columns: tr_t_id:8!null tr_tt_id:9!null tr_s_symb:10!null tr_b_id:13!null + ├── constraint: /8: [/0 - /0] ├── cardinality: [0 - 1] ├── key: () - └── fd: ()-->(7-9,12) + └── fd: ()-->(8-10,13) # Q5 opt @@ -494,50 +494,50 @@ VALUES (0, '2020-06-17 22:27:42.148484+00:00'::TIMESTAMP, 'SBMT') insert trade_history ├── columns: ├── insert-mapping: - │ ├── column1:4 => th_t_id:1 - │ ├── column2:5 => th_dts:2 - │ └── column3:6 => th_st_id:3 + │ ├── column1:5 => th_t_id:1 + │ ├── column2:6 => th_dts:2 + │ └── column3:7 => th_st_id:3 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── values - │ ├── columns: column1:4!null column2:5!null column3:6!null + │ ├── columns: column1:5!null column2:6!null column3:7!null │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(4-6) + │ ├── fd: ()-->(5-7) │ └── (0, '2020-06-17 22:27:42.148484+00:00', 'SBMT') └── f-k-checks ├── f-k-checks-item: trade_history(th_t_id) -> trade(t_id) │ └── anti-join (lookup trade) - │ ├── columns: column1:7!null - │ ├── key columns: [7] = [8] + │ ├── columns: column1:8!null + │ ├── key columns: [8] = [9] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(7) + │ ├── fd: ()-->(8) │ ├── with-scan &1 - │ │ ├── columns: column1:7!null + │ │ ├── columns: column1:8!null │ │ ├── mapping: - │ │ │ └── column1:4 => column1:7 + │ │ │ └── column1:5 => column1:8 │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(7) + │ │ └── fd: ()-->(8) │ └── filters (true) └── f-k-checks-item: trade_history(th_st_id) -> status_type(st_id) └── anti-join (lookup status_type) - ├── columns: column3:23!null - ├── key columns: [23] = [24] + ├── columns: column3:25!null + ├── key columns: [25] = [26] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(23) + ├── fd: ()-->(25) ├── with-scan &1 - │ ├── columns: column3:23!null + │ ├── columns: column3:25!null │ ├── mapping: - │ │ └── column3:6 => column3:23 + │ │ └── column3:7 => column3:25 │ ├── cardinality: [1 - 1] │ ├── key: () - │ └── fd: ()-->(23) + │ └── fd: ()-->(25) └── filters (true) # -------------------------------------------------- @@ -559,15 +559,15 @@ SELECT wi_s_symb project ├── columns: wi_s_symb:2!null └── inner-join (lookup watch_item) - ├── columns: wi_wl_id:1!null wi_s_symb:2!null wl_id:3!null wl_c_id:4!null - ├── key columns: [3] = [1] - ├── key: (2,3) - ├── fd: ()-->(4), (1)==(3), (3)==(1) + ├── columns: wi_wl_id:1!null wi_s_symb:2!null wl_id:4!null wl_c_id:5!null + ├── key columns: [4] = [1] + ├── key: (2,4) + ├── fd: ()-->(5), (1)==(4), (4)==(1) ├── scan watch_list@watch_list_auto_index_fk_wl_c_id_ref_customer - │ ├── columns: wl_id:3!null wl_c_id:4!null - │ ├── constraint: /4/3: [/0 - /0] - │ ├── key: (3) - │ └── fd: ()-->(4) + │ ├── columns: wl_id:4!null wl_c_id:5!null + │ ├── constraint: /5/4: [/0 - /0] + │ ├── key: (4) + │ └── fd: ()-->(5) └── filters (true) # Q2 @@ -586,24 +586,24 @@ WHERE s_co_id = co_id ---- project - ├── columns: s_symb:13!null - ├── key: (13) + ├── columns: s_symb:15!null + ├── key: (15) └── inner-join (hash) - ├── columns: in_id:1!null in_name:2!null co_id:4!null co_in_id:7!null s_symb:13!null s_co_id:18!null + ├── columns: in_id:1!null in_name:2!null co_id:5!null co_in_id:8!null s_symb:15!null s_co_id:20!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - ├── key: (13) - ├── fd: ()-->(2), (4)-->(7), (13)-->(18), (4)==(18), (18)==(4), (1)==(7), (7)==(1) + ├── key: (15) + ├── fd: ()-->(2), (5)-->(8), (15)-->(20), (5)==(20), (20)==(5), (1)==(8), (8)==(1) ├── scan security@secondary - │ ├── columns: s_symb:13!null s_co_id:18!null - │ ├── constraint: /18/14: [/1 - /5000] - │ ├── key: (13) - │ └── fd: (13)-->(18) + │ ├── columns: s_symb:15!null s_co_id:20!null + │ ├── constraint: /20/16: [/1 - /5000] + │ ├── key: (15) + │ └── fd: (15)-->(20) ├── inner-join (lookup company@company_auto_index_fk_co_in_id_ref_industry) - │ ├── columns: in_id:1!null in_name:2!null co_id:4!null co_in_id:7!null - │ ├── key columns: [1] = [7] + │ ├── columns: in_id:1!null in_name:2!null co_id:5!null co_in_id:8!null + │ ├── key columns: [1] = [8] │ ├── cardinality: [0 - 5000] - │ ├── key: (4) - │ ├── fd: ()-->(2), (4)-->(7), (1)==(7), (7)==(1) + │ ├── key: (5) + │ ├── fd: ()-->(2), (5)-->(8), (1)==(8), (8)==(1) │ ├── select │ │ ├── columns: in_id:1!null in_name:2!null │ │ ├── key: (1) @@ -615,9 +615,9 @@ project │ │ └── filters │ │ └── in_name:2 = 'Software' [outer=(2), constraints=(/2: [/'Software' - /'Software']; tight), fd=()-->(2)] │ └── filters - │ └── (co_id:4 >= 1) AND (co_id:4 <= 5000) [outer=(4), constraints=(/4: [/1 - /5000]; tight)] + │ └── (co_id:5 >= 1) AND (co_id:5 <= 5000) [outer=(5), constraints=(/5: [/1 - /5000]; tight)] └── filters - └── s_co_id:18 = co_id:4 [outer=(4,18), constraints=(/4: (/NULL - ]; /18: (/NULL - ]), fd=(4)==(18), (18)==(4)] + └── s_co_id:20 = co_id:5 [outer=(5,20), constraints=(/5: (/NULL - ]; /20: (/NULL - ]), fd=(5)==(20), (20)==(5)] # Q3 opt @@ -744,53 +744,53 @@ WHERE s_symb = 'SYMB' AND ea.ad_zc_code =zea.zc_code ---- project - ├── columns: s_name:4!null co_id:17!null co_name:19!null co_sp_rate:21!null co_ceo:22!null co_desc:24!null co_open_date:25!null co_st_id:18!null ad_line1:27 ad_line2:28 zc_town:32!null zc_div:33!null ad_zc_code:29!null ad_ctry:30 s_num_out:7!null s_start_date:8!null s_exch_date:9!null s_pe:49!null s_52wk_high:50!null s_52wk_high_date:12!null s_52wk_low:51!null s_52wk_low_date:14!null s_dividend:52!null s_yield:53!null zc_div:48!null ad_ctry:45 ad_line1:42 ad_line2:43 zc_town:47!null ad_zc_code:44!null ex_close:38!null ex_desc:39 ex_name:35!null ex_num_symb:36!null ex_open:37!null + ├── columns: s_name:4!null co_id:18!null co_name:20!null co_sp_rate:22!null co_ceo:23!null co_desc:25!null co_open_date:26!null co_st_id:19!null ad_line1:29 ad_line2:30 zc_town:35!null zc_div:36!null ad_zc_code:31!null ad_ctry:32 s_num_out:7!null s_start_date:8!null s_exch_date:9!null s_pe:56!null s_52wk_high:57!null s_52wk_high_date:12!null s_52wk_low:58!null s_52wk_low_date:14!null s_dividend:59!null s_yield:60!null zc_div:54!null ad_ctry:50 ad_line1:47 ad_line2:48 zc_town:53!null ad_zc_code:49!null ex_close:42!null ex_desc:43 ex_name:39!null ex_num_symb:40!null ex_open:41!null ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(4,7-9,12,14,17-19,21,22,24,25,27-30,32,33,35-39,42-45,47-53) + ├── fd: ()-->(4,7-9,12,14,18-20,22,23,25,26,29-32,35,36,39-43,47-50,53,54,56-60) ├── inner-join (lookup zip_code) - │ ├── columns: s_symb:1!null s_name:4!null s_ex_id:5!null s_co_id:6!null s_num_out:7!null s_start_date:8!null s_exch_date:9!null security.s_pe:10!null security.s_52wk_high:11!null s_52wk_high_date:12!null security.s_52wk_low:13!null s_52wk_low_date:14!null security.s_dividend:15!null security.s_yield:16!null co_id:17!null co_st_id:18!null co_name:19!null co_sp_rate:21!null co_ceo:22!null co_ad_id:23!null co_desc:24!null co_open_date:25!null ca.ad_id:26!null ca.ad_line1:27 ca.ad_line2:28 ca.ad_zc_code:29!null ca.ad_ctry:30 zca.zc_code:31!null zca.zc_town:32!null zca.zc_div:33!null ex_id:34!null ex_name:35!null ex_num_symb:36!null ex_open:37!null ex_close:38!null ex_desc:39 ex_ad_id:40!null ea.ad_id:41!null ea.ad_line1:42 ea.ad_line2:43 ea.ad_zc_code:44!null ea.ad_ctry:45 zea.zc_code:46!null zea.zc_town:47!null zea.zc_div:48!null - │ ├── key columns: [44] = [46] + │ ├── columns: s_symb:1!null s_name:4!null s_ex_id:5!null s_co_id:6!null s_num_out:7!null s_start_date:8!null s_exch_date:9!null security.s_pe:10!null security.s_52wk_high:11!null s_52wk_high_date:12!null security.s_52wk_low:13!null s_52wk_low_date:14!null security.s_dividend:15!null security.s_yield:16!null co_id:18!null co_st_id:19!null co_name:20!null co_sp_rate:22!null co_ceo:23!null co_ad_id:24!null co_desc:25!null co_open_date:26!null ca.ad_id:28!null ca.ad_line1:29 ca.ad_line2:30 ca.ad_zc_code:31!null ca.ad_ctry:32 zca.zc_code:34!null zca.zc_town:35!null zca.zc_div:36!null ex_id:38!null ex_name:39!null ex_num_symb:40!null ex_open:41!null ex_close:42!null ex_desc:43 ex_ad_id:44!null ea.ad_id:46!null ea.ad_line1:47 ea.ad_line2:48 ea.ad_zc_code:49!null ea.ad_ctry:50 zea.zc_code:52!null zea.zc_town:53!null zea.zc_div:54!null + │ ├── key columns: [49] = [52] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(1,4-19,21-48) + │ ├── fd: ()-->(1,4-16,18-20,22-26,28-32,34-36,38-44,46-50,52-54) │ ├── inner-join (lookup address) - │ │ ├── columns: s_symb:1!null s_name:4!null s_ex_id:5!null s_co_id:6!null s_num_out:7!null s_start_date:8!null s_exch_date:9!null security.s_pe:10!null security.s_52wk_high:11!null s_52wk_high_date:12!null security.s_52wk_low:13!null s_52wk_low_date:14!null security.s_dividend:15!null security.s_yield:16!null co_id:17!null co_st_id:18!null co_name:19!null co_sp_rate:21!null co_ceo:22!null co_ad_id:23!null co_desc:24!null co_open_date:25!null ca.ad_id:26!null ca.ad_line1:27 ca.ad_line2:28 ca.ad_zc_code:29!null ca.ad_ctry:30 zca.zc_code:31!null zca.zc_town:32!null zca.zc_div:33!null ex_id:34!null ex_name:35!null ex_num_symb:36!null ex_open:37!null ex_close:38!null ex_desc:39 ex_ad_id:40!null ea.ad_id:41!null ea.ad_line1:42 ea.ad_line2:43 ea.ad_zc_code:44!null ea.ad_ctry:45 - │ │ ├── key columns: [40] = [41] + │ │ ├── columns: s_symb:1!null s_name:4!null s_ex_id:5!null s_co_id:6!null s_num_out:7!null s_start_date:8!null s_exch_date:9!null security.s_pe:10!null security.s_52wk_high:11!null s_52wk_high_date:12!null security.s_52wk_low:13!null s_52wk_low_date:14!null security.s_dividend:15!null security.s_yield:16!null co_id:18!null co_st_id:19!null co_name:20!null co_sp_rate:22!null co_ceo:23!null co_ad_id:24!null co_desc:25!null co_open_date:26!null ca.ad_id:28!null ca.ad_line1:29 ca.ad_line2:30 ca.ad_zc_code:31!null ca.ad_ctry:32 zca.zc_code:34!null zca.zc_town:35!null zca.zc_div:36!null ex_id:38!null ex_name:39!null ex_num_symb:40!null ex_open:41!null ex_close:42!null ex_desc:43 ex_ad_id:44!null ea.ad_id:46!null ea.ad_line1:47 ea.ad_line2:48 ea.ad_zc_code:49!null ea.ad_ctry:50 + │ │ ├── key columns: [44] = [46] │ │ ├── lookup columns are key │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(1,4-19,21-45) + │ │ ├── fd: ()-->(1,4-16,18-20,22-26,28-32,34-36,38-44,46-50) │ │ ├── inner-join (lookup exchange) - │ │ │ ├── columns: s_symb:1!null s_name:4!null s_ex_id:5!null s_co_id:6!null s_num_out:7!null s_start_date:8!null s_exch_date:9!null security.s_pe:10!null security.s_52wk_high:11!null s_52wk_high_date:12!null security.s_52wk_low:13!null s_52wk_low_date:14!null security.s_dividend:15!null security.s_yield:16!null co_id:17!null co_st_id:18!null co_name:19!null co_sp_rate:21!null co_ceo:22!null co_ad_id:23!null co_desc:24!null co_open_date:25!null ca.ad_id:26!null ca.ad_line1:27 ca.ad_line2:28 ca.ad_zc_code:29!null ca.ad_ctry:30 zca.zc_code:31!null zca.zc_town:32!null zca.zc_div:33!null ex_id:34!null ex_name:35!null ex_num_symb:36!null ex_open:37!null ex_close:38!null ex_desc:39 ex_ad_id:40!null - │ │ │ ├── key columns: [5] = [34] + │ │ │ ├── columns: s_symb:1!null s_name:4!null s_ex_id:5!null s_co_id:6!null s_num_out:7!null s_start_date:8!null s_exch_date:9!null security.s_pe:10!null security.s_52wk_high:11!null s_52wk_high_date:12!null security.s_52wk_low:13!null s_52wk_low_date:14!null security.s_dividend:15!null security.s_yield:16!null co_id:18!null co_st_id:19!null co_name:20!null co_sp_rate:22!null co_ceo:23!null co_ad_id:24!null co_desc:25!null co_open_date:26!null ca.ad_id:28!null ca.ad_line1:29 ca.ad_line2:30 ca.ad_zc_code:31!null ca.ad_ctry:32 zca.zc_code:34!null zca.zc_town:35!null zca.zc_div:36!null ex_id:38!null ex_name:39!null ex_num_symb:40!null ex_open:41!null ex_close:42!null ex_desc:43 ex_ad_id:44!null + │ │ │ ├── key columns: [5] = [38] │ │ │ ├── lookup columns are key │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(1,4-19,21-40) + │ │ │ ├── fd: ()-->(1,4-16,18-20,22-26,28-32,34-36,38-44) │ │ │ ├── inner-join (lookup zip_code) - │ │ │ │ ├── columns: s_symb:1!null s_name:4!null s_ex_id:5!null s_co_id:6!null s_num_out:7!null s_start_date:8!null s_exch_date:9!null security.s_pe:10!null security.s_52wk_high:11!null s_52wk_high_date:12!null security.s_52wk_low:13!null s_52wk_low_date:14!null security.s_dividend:15!null security.s_yield:16!null co_id:17!null co_st_id:18!null co_name:19!null co_sp_rate:21!null co_ceo:22!null co_ad_id:23!null co_desc:24!null co_open_date:25!null ca.ad_id:26!null ca.ad_line1:27 ca.ad_line2:28 ca.ad_zc_code:29!null ca.ad_ctry:30 zca.zc_code:31!null zca.zc_town:32!null zca.zc_div:33!null - │ │ │ │ ├── key columns: [29] = [31] + │ │ │ │ ├── columns: s_symb:1!null s_name:4!null s_ex_id:5!null s_co_id:6!null s_num_out:7!null s_start_date:8!null s_exch_date:9!null security.s_pe:10!null security.s_52wk_high:11!null s_52wk_high_date:12!null security.s_52wk_low:13!null s_52wk_low_date:14!null security.s_dividend:15!null security.s_yield:16!null co_id:18!null co_st_id:19!null co_name:20!null co_sp_rate:22!null co_ceo:23!null co_ad_id:24!null co_desc:25!null co_open_date:26!null ca.ad_id:28!null ca.ad_line1:29 ca.ad_line2:30 ca.ad_zc_code:31!null ca.ad_ctry:32 zca.zc_code:34!null zca.zc_town:35!null zca.zc_div:36!null + │ │ │ │ ├── key columns: [31] = [34] │ │ │ │ ├── lookup columns are key │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ ├── key: () - │ │ │ │ ├── fd: ()-->(1,4-19,21-33) + │ │ │ │ ├── fd: ()-->(1,4-16,18-20,22-26,28-32,34-36) │ │ │ │ ├── inner-join (lookup address) - │ │ │ │ │ ├── columns: s_symb:1!null s_name:4!null s_ex_id:5!null s_co_id:6!null s_num_out:7!null s_start_date:8!null s_exch_date:9!null security.s_pe:10!null security.s_52wk_high:11!null s_52wk_high_date:12!null security.s_52wk_low:13!null s_52wk_low_date:14!null security.s_dividend:15!null security.s_yield:16!null co_id:17!null co_st_id:18!null co_name:19!null co_sp_rate:21!null co_ceo:22!null co_ad_id:23!null co_desc:24!null co_open_date:25!null ca.ad_id:26!null ca.ad_line1:27 ca.ad_line2:28 ca.ad_zc_code:29!null ca.ad_ctry:30 - │ │ │ │ │ ├── key columns: [23] = [26] + │ │ │ │ │ ├── columns: s_symb:1!null s_name:4!null s_ex_id:5!null s_co_id:6!null s_num_out:7!null s_start_date:8!null s_exch_date:9!null security.s_pe:10!null security.s_52wk_high:11!null s_52wk_high_date:12!null security.s_52wk_low:13!null s_52wk_low_date:14!null security.s_dividend:15!null security.s_yield:16!null co_id:18!null co_st_id:19!null co_name:20!null co_sp_rate:22!null co_ceo:23!null co_ad_id:24!null co_desc:25!null co_open_date:26!null ca.ad_id:28!null ca.ad_line1:29 ca.ad_line2:30 ca.ad_zc_code:31!null ca.ad_ctry:32 + │ │ │ │ │ ├── key columns: [24] = [28] │ │ │ │ │ ├── lookup columns are key │ │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ │ ├── key: () - │ │ │ │ │ ├── fd: ()-->(1,4-19,21-30) + │ │ │ │ │ ├── fd: ()-->(1,4-16,18-20,22-26,28-32) │ │ │ │ │ ├── inner-join (lookup company) - │ │ │ │ │ │ ├── columns: s_symb:1!null s_name:4!null s_ex_id:5!null s_co_id:6!null s_num_out:7!null s_start_date:8!null s_exch_date:9!null security.s_pe:10!null security.s_52wk_high:11!null s_52wk_high_date:12!null security.s_52wk_low:13!null s_52wk_low_date:14!null security.s_dividend:15!null security.s_yield:16!null co_id:17!null co_st_id:18!null co_name:19!null co_sp_rate:21!null co_ceo:22!null co_ad_id:23!null co_desc:24!null co_open_date:25!null - │ │ │ │ │ │ ├── key columns: [6] = [17] + │ │ │ │ │ │ ├── columns: s_symb:1!null s_name:4!null s_ex_id:5!null s_co_id:6!null s_num_out:7!null s_start_date:8!null s_exch_date:9!null security.s_pe:10!null security.s_52wk_high:11!null s_52wk_high_date:12!null security.s_52wk_low:13!null s_52wk_low_date:14!null security.s_dividend:15!null security.s_yield:16!null co_id:18!null co_st_id:19!null co_name:20!null co_sp_rate:22!null co_ceo:23!null co_ad_id:24!null co_desc:25!null co_open_date:26!null + │ │ │ │ │ │ ├── key columns: [6] = [18] │ │ │ │ │ │ ├── lookup columns are key │ │ │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ │ │ ├── key: () - │ │ │ │ │ │ ├── fd: ()-->(1,4-19,21-25) + │ │ │ │ │ │ ├── fd: ()-->(1,4-16,18-20,22-26) │ │ │ │ │ │ ├── scan security │ │ │ │ │ │ │ ├── columns: s_symb:1!null s_name:4!null s_ex_id:5!null s_co_id:6!null s_num_out:7!null s_start_date:8!null s_exch_date:9!null security.s_pe:10!null security.s_52wk_high:11!null s_52wk_high_date:12!null security.s_52wk_low:13!null s_52wk_low_date:14!null security.s_dividend:15!null security.s_yield:16!null │ │ │ │ │ │ │ ├── constraint: /1: [/'SYMB' - /'SYMB'] @@ -804,11 +804,11 @@ project │ │ └── filters (true) │ └── filters (true) └── projections - ├── security.s_pe:10::FLOAT8 [as=s_pe:49, outer=(10), immutable] - ├── security.s_52wk_high:11::FLOAT8 [as=s_52wk_high:50, outer=(11), immutable] - ├── security.s_52wk_low:13::FLOAT8 [as=s_52wk_low:51, outer=(13), immutable] - ├── security.s_dividend:15::FLOAT8 [as=s_dividend:52, outer=(15), immutable] - └── security.s_yield:16::FLOAT8 [as=s_yield:53, outer=(16), immutable] + ├── security.s_pe:10::FLOAT8 [as=s_pe:56, outer=(10), immutable] + ├── security.s_52wk_high:11::FLOAT8 [as=s_52wk_high:57, outer=(11), immutable] + ├── security.s_52wk_low:13::FLOAT8 [as=s_52wk_low:58, outer=(13), immutable] + ├── security.s_dividend:15::FLOAT8 [as=s_dividend:59, outer=(15), immutable] + └── security.s_yield:16::FLOAT8 [as=s_yield:60, outer=(16), immutable] # Q1 Version 2 @@ -861,59 +861,59 @@ INNER LOOKUP JOIN zip_code AS zea ON ea.ad_zc_code = zea.zc_code WHERE s_symb = 'SYMB' ---- project - ├── columns: s_name:4!null co_id:17!null co_name:19!null co_sp_rate:21!null co_ceo:22!null co_desc:24!null co_open_date:25!null co_st_id:18!null ad_line1:27 ad_line2:28 zc_town:32!null zc_div:33!null ad_zc_code:29!null ad_ctry:30 s_num_out:7!null s_start_date:8!null s_exch_date:9!null s_pe:49!null s_52wk_high:50!null s_52wk_high_date:12!null s_52wk_low:51!null s_52wk_low_date:14!null s_dividend:52!null s_yield:53!null zc_div:48!null ad_ctry:45 ad_line1:42 ad_line2:43 zc_town:47!null ad_zc_code:44!null ex_close:38!null ex_desc:39 ex_name:35!null ex_num_symb:36!null ex_open:37!null + ├── columns: s_name:4!null co_id:18!null co_name:20!null co_sp_rate:22!null co_ceo:23!null co_desc:25!null co_open_date:26!null co_st_id:19!null ad_line1:29 ad_line2:30 zc_town:35!null zc_div:36!null ad_zc_code:31!null ad_ctry:32 s_num_out:7!null s_start_date:8!null s_exch_date:9!null s_pe:56!null s_52wk_high:57!null s_52wk_high_date:12!null s_52wk_low:58!null s_52wk_low_date:14!null s_dividend:59!null s_yield:60!null zc_div:54!null ad_ctry:50 ad_line1:47 ad_line2:48 zc_town:53!null ad_zc_code:49!null ex_close:42!null ex_desc:43 ex_name:39!null ex_num_symb:40!null ex_open:41!null ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(4,7-9,12,14,17-19,21,22,24,25,27-30,32,33,35-39,42-45,47-53) + ├── fd: ()-->(4,7-9,12,14,18-20,22,23,25,26,29-32,35,36,39-43,47-50,53,54,56-60) ├── inner-join (lookup zip_code) - │ ├── columns: s_symb:1!null s_name:4!null s_ex_id:5!null s_co_id:6!null s_num_out:7!null s_start_date:8!null s_exch_date:9!null security.s_pe:10!null security.s_52wk_high:11!null s_52wk_high_date:12!null security.s_52wk_low:13!null s_52wk_low_date:14!null security.s_dividend:15!null security.s_yield:16!null co_id:17!null co_st_id:18!null co_name:19!null co_sp_rate:21!null co_ceo:22!null co_ad_id:23!null co_desc:24!null co_open_date:25!null ca.ad_id:26!null ca.ad_line1:27 ca.ad_line2:28 ca.ad_zc_code:29!null ca.ad_ctry:30 zca.zc_code:31!null zca.zc_town:32!null zca.zc_div:33!null ex_id:34!null ex_name:35!null ex_num_symb:36!null ex_open:37!null ex_close:38!null ex_desc:39 ex_ad_id:40!null ea.ad_id:41!null ea.ad_line1:42 ea.ad_line2:43 ea.ad_zc_code:44!null ea.ad_ctry:45 zea.zc_code:46!null zea.zc_town:47!null zea.zc_div:48!null + │ ├── columns: s_symb:1!null s_name:4!null s_ex_id:5!null s_co_id:6!null s_num_out:7!null s_start_date:8!null s_exch_date:9!null security.s_pe:10!null security.s_52wk_high:11!null s_52wk_high_date:12!null security.s_52wk_low:13!null s_52wk_low_date:14!null security.s_dividend:15!null security.s_yield:16!null co_id:18!null co_st_id:19!null co_name:20!null co_sp_rate:22!null co_ceo:23!null co_ad_id:24!null co_desc:25!null co_open_date:26!null ca.ad_id:28!null ca.ad_line1:29 ca.ad_line2:30 ca.ad_zc_code:31!null ca.ad_ctry:32 zca.zc_code:34!null zca.zc_town:35!null zca.zc_div:36!null ex_id:38!null ex_name:39!null ex_num_symb:40!null ex_open:41!null ex_close:42!null ex_desc:43 ex_ad_id:44!null ea.ad_id:46!null ea.ad_line1:47 ea.ad_line2:48 ea.ad_zc_code:49!null ea.ad_ctry:50 zea.zc_code:52!null zea.zc_town:53!null zea.zc_div:54!null │ ├── flags: force lookup join (into right side) - │ ├── key columns: [44] = [46] + │ ├── key columns: [49] = [52] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(1,4-19,21-48) + │ ├── fd: ()-->(1,4-16,18-20,22-26,28-32,34-36,38-44,46-50,52-54) │ ├── inner-join (lookup address) - │ │ ├── columns: s_symb:1!null s_name:4!null s_ex_id:5!null s_co_id:6!null s_num_out:7!null s_start_date:8!null s_exch_date:9!null security.s_pe:10!null security.s_52wk_high:11!null s_52wk_high_date:12!null security.s_52wk_low:13!null s_52wk_low_date:14!null security.s_dividend:15!null security.s_yield:16!null co_id:17!null co_st_id:18!null co_name:19!null co_sp_rate:21!null co_ceo:22!null co_ad_id:23!null co_desc:24!null co_open_date:25!null ca.ad_id:26!null ca.ad_line1:27 ca.ad_line2:28 ca.ad_zc_code:29!null ca.ad_ctry:30 zca.zc_code:31!null zca.zc_town:32!null zca.zc_div:33!null ex_id:34!null ex_name:35!null ex_num_symb:36!null ex_open:37!null ex_close:38!null ex_desc:39 ex_ad_id:40!null ea.ad_id:41!null ea.ad_line1:42 ea.ad_line2:43 ea.ad_zc_code:44!null ea.ad_ctry:45 + │ │ ├── columns: s_symb:1!null s_name:4!null s_ex_id:5!null s_co_id:6!null s_num_out:7!null s_start_date:8!null s_exch_date:9!null security.s_pe:10!null security.s_52wk_high:11!null s_52wk_high_date:12!null security.s_52wk_low:13!null s_52wk_low_date:14!null security.s_dividend:15!null security.s_yield:16!null co_id:18!null co_st_id:19!null co_name:20!null co_sp_rate:22!null co_ceo:23!null co_ad_id:24!null co_desc:25!null co_open_date:26!null ca.ad_id:28!null ca.ad_line1:29 ca.ad_line2:30 ca.ad_zc_code:31!null ca.ad_ctry:32 zca.zc_code:34!null zca.zc_town:35!null zca.zc_div:36!null ex_id:38!null ex_name:39!null ex_num_symb:40!null ex_open:41!null ex_close:42!null ex_desc:43 ex_ad_id:44!null ea.ad_id:46!null ea.ad_line1:47 ea.ad_line2:48 ea.ad_zc_code:49!null ea.ad_ctry:50 │ │ ├── flags: force lookup join (into right side) - │ │ ├── key columns: [40] = [41] + │ │ ├── key columns: [44] = [46] │ │ ├── lookup columns are key │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(1,4-19,21-45) + │ │ ├── fd: ()-->(1,4-16,18-20,22-26,28-32,34-36,38-44,46-50) │ │ ├── inner-join (lookup exchange) - │ │ │ ├── columns: s_symb:1!null s_name:4!null s_ex_id:5!null s_co_id:6!null s_num_out:7!null s_start_date:8!null s_exch_date:9!null security.s_pe:10!null security.s_52wk_high:11!null s_52wk_high_date:12!null security.s_52wk_low:13!null s_52wk_low_date:14!null security.s_dividend:15!null security.s_yield:16!null co_id:17!null co_st_id:18!null co_name:19!null co_sp_rate:21!null co_ceo:22!null co_ad_id:23!null co_desc:24!null co_open_date:25!null ca.ad_id:26!null ca.ad_line1:27 ca.ad_line2:28 ca.ad_zc_code:29!null ca.ad_ctry:30 zca.zc_code:31!null zca.zc_town:32!null zca.zc_div:33!null ex_id:34!null ex_name:35!null ex_num_symb:36!null ex_open:37!null ex_close:38!null ex_desc:39 ex_ad_id:40!null + │ │ │ ├── columns: s_symb:1!null s_name:4!null s_ex_id:5!null s_co_id:6!null s_num_out:7!null s_start_date:8!null s_exch_date:9!null security.s_pe:10!null security.s_52wk_high:11!null s_52wk_high_date:12!null security.s_52wk_low:13!null s_52wk_low_date:14!null security.s_dividend:15!null security.s_yield:16!null co_id:18!null co_st_id:19!null co_name:20!null co_sp_rate:22!null co_ceo:23!null co_ad_id:24!null co_desc:25!null co_open_date:26!null ca.ad_id:28!null ca.ad_line1:29 ca.ad_line2:30 ca.ad_zc_code:31!null ca.ad_ctry:32 zca.zc_code:34!null zca.zc_town:35!null zca.zc_div:36!null ex_id:38!null ex_name:39!null ex_num_symb:40!null ex_open:41!null ex_close:42!null ex_desc:43 ex_ad_id:44!null │ │ │ ├── flags: force lookup join (into right side) - │ │ │ ├── key columns: [5] = [34] + │ │ │ ├── key columns: [5] = [38] │ │ │ ├── lookup columns are key │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(1,4-19,21-40) + │ │ │ ├── fd: ()-->(1,4-16,18-20,22-26,28-32,34-36,38-44) │ │ │ ├── inner-join (lookup zip_code) - │ │ │ │ ├── columns: s_symb:1!null s_name:4!null s_ex_id:5!null s_co_id:6!null s_num_out:7!null s_start_date:8!null s_exch_date:9!null security.s_pe:10!null security.s_52wk_high:11!null s_52wk_high_date:12!null security.s_52wk_low:13!null s_52wk_low_date:14!null security.s_dividend:15!null security.s_yield:16!null co_id:17!null co_st_id:18!null co_name:19!null co_sp_rate:21!null co_ceo:22!null co_ad_id:23!null co_desc:24!null co_open_date:25!null ca.ad_id:26!null ca.ad_line1:27 ca.ad_line2:28 ca.ad_zc_code:29!null ca.ad_ctry:30 zca.zc_code:31!null zca.zc_town:32!null zca.zc_div:33!null + │ │ │ │ ├── columns: s_symb:1!null s_name:4!null s_ex_id:5!null s_co_id:6!null s_num_out:7!null s_start_date:8!null s_exch_date:9!null security.s_pe:10!null security.s_52wk_high:11!null s_52wk_high_date:12!null security.s_52wk_low:13!null s_52wk_low_date:14!null security.s_dividend:15!null security.s_yield:16!null co_id:18!null co_st_id:19!null co_name:20!null co_sp_rate:22!null co_ceo:23!null co_ad_id:24!null co_desc:25!null co_open_date:26!null ca.ad_id:28!null ca.ad_line1:29 ca.ad_line2:30 ca.ad_zc_code:31!null ca.ad_ctry:32 zca.zc_code:34!null zca.zc_town:35!null zca.zc_div:36!null │ │ │ │ ├── flags: force lookup join (into right side) - │ │ │ │ ├── key columns: [29] = [31] + │ │ │ │ ├── key columns: [31] = [34] │ │ │ │ ├── lookup columns are key │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ ├── key: () - │ │ │ │ ├── fd: ()-->(1,4-19,21-33) + │ │ │ │ ├── fd: ()-->(1,4-16,18-20,22-26,28-32,34-36) │ │ │ │ ├── inner-join (lookup address) - │ │ │ │ │ ├── columns: s_symb:1!null s_name:4!null s_ex_id:5!null s_co_id:6!null s_num_out:7!null s_start_date:8!null s_exch_date:9!null security.s_pe:10!null security.s_52wk_high:11!null s_52wk_high_date:12!null security.s_52wk_low:13!null s_52wk_low_date:14!null security.s_dividend:15!null security.s_yield:16!null co_id:17!null co_st_id:18!null co_name:19!null co_sp_rate:21!null co_ceo:22!null co_ad_id:23!null co_desc:24!null co_open_date:25!null ca.ad_id:26!null ca.ad_line1:27 ca.ad_line2:28 ca.ad_zc_code:29!null ca.ad_ctry:30 + │ │ │ │ │ ├── columns: s_symb:1!null s_name:4!null s_ex_id:5!null s_co_id:6!null s_num_out:7!null s_start_date:8!null s_exch_date:9!null security.s_pe:10!null security.s_52wk_high:11!null s_52wk_high_date:12!null security.s_52wk_low:13!null s_52wk_low_date:14!null security.s_dividend:15!null security.s_yield:16!null co_id:18!null co_st_id:19!null co_name:20!null co_sp_rate:22!null co_ceo:23!null co_ad_id:24!null co_desc:25!null co_open_date:26!null ca.ad_id:28!null ca.ad_line1:29 ca.ad_line2:30 ca.ad_zc_code:31!null ca.ad_ctry:32 │ │ │ │ │ ├── flags: force lookup join (into right side) - │ │ │ │ │ ├── key columns: [23] = [26] + │ │ │ │ │ ├── key columns: [24] = [28] │ │ │ │ │ ├── lookup columns are key │ │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ │ ├── key: () - │ │ │ │ │ ├── fd: ()-->(1,4-19,21-30) + │ │ │ │ │ ├── fd: ()-->(1,4-16,18-20,22-26,28-32) │ │ │ │ │ ├── inner-join (lookup company) - │ │ │ │ │ │ ├── columns: s_symb:1!null s_name:4!null s_ex_id:5!null s_co_id:6!null s_num_out:7!null s_start_date:8!null s_exch_date:9!null security.s_pe:10!null security.s_52wk_high:11!null s_52wk_high_date:12!null security.s_52wk_low:13!null s_52wk_low_date:14!null security.s_dividend:15!null security.s_yield:16!null co_id:17!null co_st_id:18!null co_name:19!null co_sp_rate:21!null co_ceo:22!null co_ad_id:23!null co_desc:24!null co_open_date:25!null + │ │ │ │ │ │ ├── columns: s_symb:1!null s_name:4!null s_ex_id:5!null s_co_id:6!null s_num_out:7!null s_start_date:8!null s_exch_date:9!null security.s_pe:10!null security.s_52wk_high:11!null s_52wk_high_date:12!null security.s_52wk_low:13!null s_52wk_low_date:14!null security.s_dividend:15!null security.s_yield:16!null co_id:18!null co_st_id:19!null co_name:20!null co_sp_rate:22!null co_ceo:23!null co_ad_id:24!null co_desc:25!null co_open_date:26!null │ │ │ │ │ │ ├── flags: force lookup join (into right side) - │ │ │ │ │ │ ├── key columns: [6] = [17] + │ │ │ │ │ │ ├── key columns: [6] = [18] │ │ │ │ │ │ ├── lookup columns are key │ │ │ │ │ │ ├── cardinality: [0 - 1] │ │ │ │ │ │ ├── key: () - │ │ │ │ │ │ ├── fd: ()-->(1,4-19,21-25) + │ │ │ │ │ │ ├── fd: ()-->(1,4-16,18-20,22-26) │ │ │ │ │ │ ├── scan security │ │ │ │ │ │ │ ├── columns: s_symb:1!null s_name:4!null s_ex_id:5!null s_co_id:6!null s_num_out:7!null s_start_date:8!null s_exch_date:9!null security.s_pe:10!null security.s_52wk_high:11!null s_52wk_high_date:12!null security.s_52wk_low:13!null s_52wk_low_date:14!null security.s_dividend:15!null security.s_yield:16!null │ │ │ │ │ │ │ ├── constraint: /1: [/'SYMB' - /'SYMB'] @@ -927,11 +927,11 @@ project │ │ └── filters (true) │ └── filters (true) └── projections - ├── security.s_pe:10::FLOAT8 [as=s_pe:49, outer=(10), immutable] - ├── security.s_52wk_high:11::FLOAT8 [as=s_52wk_high:50, outer=(11), immutable] - ├── security.s_52wk_low:13::FLOAT8 [as=s_52wk_low:51, outer=(13), immutable] - ├── security.s_dividend:15::FLOAT8 [as=s_dividend:52, outer=(15), immutable] - └── security.s_yield:16::FLOAT8 [as=s_yield:53, outer=(16), immutable] + ├── security.s_pe:10::FLOAT8 [as=s_pe:56, outer=(10), immutable] + ├── security.s_52wk_high:11::FLOAT8 [as=s_52wk_high:57, outer=(11), immutable] + ├── security.s_52wk_low:13::FLOAT8 [as=s_52wk_low:58, outer=(13), immutable] + ├── security.s_dividend:15::FLOAT8 [as=s_dividend:59, outer=(15), immutable] + └── security.s_yield:16::FLOAT8 [as=s_yield:60, outer=(16), immutable] # Q2 opt @@ -941,22 +941,22 @@ SELECT co_name, in_name LIMIT 3 ---- project - ├── columns: co_name:6!null in_name:14!null + ├── columns: co_name:7!null in_name:16!null ├── cardinality: [0 - 3] └── inner-join (lookup industry) - ├── columns: cp_co_id:1!null cp_comp_co_id:2!null cp_in_id:3!null co_id:4!null co_name:6!null in_id:13!null in_name:14!null - ├── key columns: [3] = [13] + ├── columns: cp_co_id:1!null cp_comp_co_id:2!null cp_in_id:3!null co_id:5!null co_name:7!null in_id:15!null in_name:16!null + ├── key columns: [3] = [15] ├── lookup columns are key ├── cardinality: [0 - 3] - ├── key: (4,13) - ├── fd: ()-->(1), (4)-->(6), (6)-->(4), (2)==(4), (4)==(2), (13)-->(14), (3)==(13), (13)==(3) + ├── key: (5,15) + ├── fd: ()-->(1), (5)-->(7), (7)-->(5), (2)==(5), (5)==(2), (15)-->(16), (3)==(15), (15)==(3) ├── inner-join (lookup company) - │ ├── columns: cp_co_id:1!null cp_comp_co_id:2!null cp_in_id:3!null co_id:4!null co_name:6!null - │ ├── key columns: [2] = [4] + │ ├── columns: cp_co_id:1!null cp_comp_co_id:2!null cp_in_id:3!null co_id:5!null co_name:7!null + │ ├── key columns: [2] = [5] │ ├── lookup columns are key │ ├── cardinality: [0 - 3] - │ ├── key: (3,4) - │ ├── fd: ()-->(1), (4)-->(6), (6)-->(4), (2)==(4), (4)==(2) + │ ├── key: (3,5) + │ ├── fd: ()-->(1), (5)-->(7), (7)-->(5), (2)==(5), (5)==(2) │ ├── scan company_competitor │ │ ├── columns: cp_co_id:1!null cp_comp_co_id:2!null cp_in_id:3!null │ │ ├── constraint: /1/2/3: [/0 - /0] @@ -987,11 +987,11 @@ ORDER BY fi_year ASC, fi_qtr ASC LIMIT 20 ---- project - ├── columns: fi_year:2!null fi_qtr:3!null fi_qtr_start_date:4!null fi_revenue:15!null fi_net_earn:16!null fi_basic_eps:17!null fi_dilut_eps:18!null fi_margin:19!null fi_inventory:20!null fi_assets:21!null fi_liability:22!null fi_out_basic:13!null fi_out_dilut:14!null + ├── columns: fi_year:2!null fi_qtr:3!null fi_qtr_start_date:4!null fi_revenue:16!null fi_net_earn:17!null fi_basic_eps:18!null fi_dilut_eps:19!null fi_margin:20!null fi_inventory:21!null fi_assets:22!null fi_liability:23!null fi_out_basic:13!null fi_out_dilut:14!null ├── cardinality: [0 - 20] ├── immutable ├── key: (2,3) - ├── fd: (2,3)-->(4,13-22) + ├── fd: (2,3)-->(4,13,14,16-23) ├── ordering: +2,+3 ├── scan financial │ ├── columns: fi_co_id:1!null fi_year:2!null fi_qtr:3!null fi_qtr_start_date:4!null financial.fi_revenue:5!null financial.fi_net_earn:6!null financial.fi_basic_eps:7!null financial.fi_dilut_eps:8!null financial.fi_margin:9!null financial.fi_inventory:10!null financial.fi_assets:11!null financial.fi_liability:12!null fi_out_basic:13!null fi_out_dilut:14!null @@ -1001,14 +1001,14 @@ project │ ├── fd: ()-->(1), (2,3)-->(4-14) │ └── ordering: +2,+3 opt(1) [actual: +2,+3] └── projections - ├── financial.fi_revenue:5::FLOAT8 [as=fi_revenue:15, outer=(5), immutable] - ├── financial.fi_net_earn:6::FLOAT8 [as=fi_net_earn:16, outer=(6), immutable] - ├── financial.fi_basic_eps:7::FLOAT8 [as=fi_basic_eps:17, outer=(7), immutable] - ├── financial.fi_dilut_eps:8::FLOAT8 [as=fi_dilut_eps:18, outer=(8), immutable] - ├── financial.fi_margin:9::FLOAT8 [as=fi_margin:19, outer=(9), immutable] - ├── financial.fi_inventory:10::FLOAT8 [as=fi_inventory:20, outer=(10), immutable] - ├── financial.fi_assets:11::FLOAT8 [as=fi_assets:21, outer=(11), immutable] - └── financial.fi_liability:12::FLOAT8 [as=fi_liability:22, outer=(12), immutable] + ├── financial.fi_revenue:5::FLOAT8 [as=fi_revenue:16, outer=(5), immutable] + ├── financial.fi_net_earn:6::FLOAT8 [as=fi_net_earn:17, outer=(6), immutable] + ├── financial.fi_basic_eps:7::FLOAT8 [as=fi_basic_eps:18, outer=(7), immutable] + ├── financial.fi_dilut_eps:8::FLOAT8 [as=fi_dilut_eps:19, outer=(8), immutable] + ├── financial.fi_margin:9::FLOAT8 [as=fi_margin:20, outer=(9), immutable] + ├── financial.fi_inventory:10::FLOAT8 [as=fi_inventory:21, outer=(10), immutable] + ├── financial.fi_assets:11::FLOAT8 [as=fi_assets:22, outer=(11), immutable] + └── financial.fi_liability:12::FLOAT8 [as=fi_liability:23, outer=(12), immutable] # Q4 opt @@ -1019,11 +1019,11 @@ ORDER BY dm_date ASC LIMIT 10 ---- project - ├── columns: dm_date:1!null dm_close:7!null dm_high:8!null dm_low:9!null dm_vol:6!null + ├── columns: dm_date:1!null dm_close:8!null dm_high:9!null dm_low:10!null dm_vol:6!null ├── cardinality: [0 - 10] ├── immutable ├── key: (1) - ├── fd: (1)-->(6-9) + ├── fd: (1)-->(6,8-10) ├── ordering: +1 ├── scan daily_market@secondary │ ├── columns: dm_date:1!null dm_s_symb:2!null daily_market.dm_close:3!null daily_market.dm_high:4!null daily_market.dm_low:5!null dm_vol:6!null @@ -1033,9 +1033,9 @@ project │ ├── fd: ()-->(2), (1)-->(3-6) │ └── ordering: +1 opt(2) [actual: +1] └── projections - ├── daily_market.dm_close:3::FLOAT8 [as=dm_close:7, outer=(3), immutable] - ├── daily_market.dm_high:4::FLOAT8 [as=dm_high:8, outer=(4), immutable] - └── daily_market.dm_low:5::FLOAT8 [as=dm_low:9, outer=(5), immutable] + ├── daily_market.dm_close:3::FLOAT8 [as=dm_close:8, outer=(3), immutable] + ├── daily_market.dm_high:4::FLOAT8 [as=dm_high:9, outer=(4), immutable] + └── daily_market.dm_low:5::FLOAT8 [as=dm_low:10, outer=(5), immutable] # Q5 opt @@ -1044,11 +1044,11 @@ SELECT lt_price::FLOAT8, lt_open_price::FLOAT8, lt_vol WHERE lt_s_symb = 'SYMB' ---- project - ├── columns: lt_price:6!null lt_open_price:7!null lt_vol:5!null + ├── columns: lt_price:7!null lt_open_price:8!null lt_vol:5!null ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(5-7) + ├── fd: ()-->(5,7,8) ├── scan last_trade │ ├── columns: lt_s_symb:1!null last_trade.lt_price:3!null last_trade.lt_open_price:4!null lt_vol:5!null │ ├── constraint: /1: [/'SYMB' - /'SYMB'] @@ -1056,8 +1056,8 @@ project │ ├── key: () │ └── fd: ()-->(1,3-5) └── projections - ├── last_trade.lt_price:3::FLOAT8 [as=lt_price:6, outer=(3), immutable] - └── last_trade.lt_open_price:4::FLOAT8 [as=lt_open_price:7, outer=(4), immutable] + ├── last_trade.lt_price:3::FLOAT8 [as=lt_price:7, outer=(3), immutable] + └── last_trade.lt_open_price:4::FLOAT8 [as=lt_open_price:8, outer=(4), immutable] # Q6 opt @@ -1067,15 +1067,15 @@ SELECT ni_dts, ni_source, ni_author, ni_item LIMIT 2 ---- project - ├── columns: ni_dts:7!null ni_source:8!null ni_author:9 ni_item:6!null + ├── columns: ni_dts:8!null ni_source:9!null ni_author:10 ni_item:7!null ├── cardinality: [0 - 2] └── inner-join (lookup news_item) - ├── columns: nx_ni_id:1!null nx_co_id:2!null ni_id:3!null ni_item:6!null ni_dts:7!null ni_source:8!null ni_author:9 - ├── key columns: [1] = [3] + ├── columns: nx_ni_id:1!null nx_co_id:2!null ni_id:4!null ni_item:7!null ni_dts:8!null ni_source:9!null ni_author:10 + ├── key columns: [1] = [4] ├── lookup columns are key ├── cardinality: [0 - 2] - ├── key: (3) - ├── fd: ()-->(2), (3)-->(6-9), (1)==(3), (3)==(1) + ├── key: (4) + ├── fd: ()-->(2), (4)-->(7-10), (1)==(4), (4)==(1) ├── scan news_xref@news_xref_auto_index_fk_nx_co_id_ref_company │ ├── columns: nx_ni_id:1!null nx_co_id:2!null │ ├── constraint: /2/1: [/0 - /0] @@ -1092,15 +1092,15 @@ SELECT ni_dts, ni_source, ni_author, ni_headline, ni_summary LIMIT 2 ---- project - ├── columns: ni_dts:7!null ni_source:8!null ni_author:9 ni_headline:4!null ni_summary:5!null + ├── columns: ni_dts:8!null ni_source:9!null ni_author:10 ni_headline:5!null ni_summary:6!null ├── cardinality: [0 - 2] └── inner-join (lookup news_item) - ├── columns: nx_ni_id:1!null nx_co_id:2!null ni_id:3!null ni_headline:4!null ni_summary:5!null ni_dts:7!null ni_source:8!null ni_author:9 - ├── key columns: [1] = [3] + ├── columns: nx_ni_id:1!null nx_co_id:2!null ni_id:4!null ni_headline:5!null ni_summary:6!null ni_dts:8!null ni_source:9!null ni_author:10 + ├── key columns: [1] = [4] ├── lookup columns are key ├── cardinality: [0 - 2] - ├── key: (3) - ├── fd: ()-->(2), (3)-->(4,5,7-9), (1)==(3), (3)==(1) + ├── key: (4) + ├── fd: ()-->(2), (4)-->(5,6,8-10), (1)==(4), (4)==(1) ├── scan news_xref@news_xref_auto_index_fk_nx_co_id_ref_company │ ├── columns: nx_ni_id:1!null nx_co_id:2!null │ ├── constraint: /2/1: [/0 - /0] @@ -1133,18 +1133,18 @@ SELECT t_bid_price::FLOAT8, WHERE t_id = 0 AND t_tt_id = tt_id ---- project - ├── columns: t_bid_price:20 t_exec_name:10!null t_is_cash:5!null tt_is_mrkt:19!null t_trade_price:21 + ├── columns: t_bid_price:22 t_exec_name:10!null t_is_cash:5!null tt_is_mrkt:20!null t_trade_price:23 ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(5,10,19-21) + ├── fd: ()-->(5,10,20,22,23) ├── inner-join (lookup trade_type) - │ ├── columns: t_id:1!null t_tt_id:4!null t_is_cash:5!null trade.t_bid_price:8 t_exec_name:10!null trade.t_trade_price:11 tt_id:16!null tt_is_mrkt:19!null - │ ├── key columns: [4] = [16] + │ ├── columns: t_id:1!null t_tt_id:4!null t_is_cash:5!null trade.t_bid_price:8 t_exec_name:10!null trade.t_trade_price:11 tt_id:17!null tt_is_mrkt:20!null + │ ├── key columns: [4] = [17] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(1,4,5,8,10,11,16,19) + │ ├── fd: ()-->(1,4,5,8,10,11,17,20) │ ├── scan trade │ │ ├── columns: t_id:1!null t_tt_id:4!null t_is_cash:5!null trade.t_bid_price:8 t_exec_name:10!null trade.t_trade_price:11 │ │ ├── constraint: /1: [/0 - /0] @@ -1153,8 +1153,8 @@ project │ │ └── fd: ()-->(1,4,5,8,10,11) │ └── filters (true) └── projections - ├── trade.t_bid_price:8::FLOAT8 [as=t_bid_price:20, outer=(8), immutable] - └── trade.t_trade_price:11::FLOAT8 [as=t_trade_price:21, outer=(11), immutable] + ├── trade.t_bid_price:8::FLOAT8 [as=t_bid_price:22, outer=(8), immutable] + └── trade.t_trade_price:11::FLOAT8 [as=t_trade_price:23, outer=(11), immutable] # Q2 opt @@ -1163,11 +1163,11 @@ SELECT se_amt::FLOAT8, se_cash_due_date, se_cash_type WHERE se_t_id = 0 ---- project - ├── columns: se_amt:5!null se_cash_due_date:3!null se_cash_type:2!null + ├── columns: se_amt:6!null se_cash_due_date:3!null se_cash_type:2!null ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(2,3,5) + ├── fd: ()-->(2,3,6) ├── scan settlement │ ├── columns: se_t_id:1!null se_cash_type:2!null se_cash_due_date:3!null settlement.se_amt:4!null │ ├── constraint: /1: [/0 - /0] @@ -1175,18 +1175,18 @@ project │ ├── key: () │ └── fd: ()-->(1-4) └── projections - └── settlement.se_amt:4::FLOAT8 [as=se_amt:5, outer=(4), immutable] + └── settlement.se_amt:4::FLOAT8 [as=se_amt:6, outer=(4), immutable] # Q3 opt SELECT ct_amt::FLOAT8, ct_dts, ct_name FROM cash_transaction WHERE ct_t_id = 0 ---- project - ├── columns: ct_amt:5!null ct_dts:2!null ct_name:4 + ├── columns: ct_amt:6!null ct_dts:2!null ct_name:4 ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(2,4,5) + ├── fd: ()-->(2,4,6) ├── scan cash_transaction │ ├── columns: ct_t_id:1!null ct_dts:2!null cash_transaction.ct_amt:3!null ct_name:4 │ ├── constraint: /1: [/0 - /0] @@ -1194,7 +1194,7 @@ project │ ├── key: () │ └── fd: ()-->(1-4) └── projections - └── cash_transaction.ct_amt:3::FLOAT8 [as=ct_amt:5, outer=(3), immutable] + └── cash_transaction.ct_amt:3::FLOAT8 [as=ct_amt:6, outer=(3), immutable] # Q4 opt @@ -1245,11 +1245,11 @@ ORDER BY t_dts ASC LIMIT 20 ---- project - ├── columns: t_id:1!null t_bid_price:16 t_exec_name:10!null t_is_cash:5!null t_trade_price:17 [hidden: t_dts:2!null] + ├── columns: t_id:1!null t_bid_price:17 t_exec_name:10!null t_is_cash:5!null t_trade_price:18 [hidden: t_dts:2!null] ├── cardinality: [0 - 20] ├── immutable ├── key: (1) - ├── fd: (1)-->(2,5,10,16,17) + ├── fd: (1)-->(2,5,10,17,18) ├── ordering: +2 ├── scan trade@secondary,rev │ ├── columns: t_id:1!null t_dts:2!null t_is_cash:5!null trade.t_bid_price:8 t_ca_id:9!null t_exec_name:10!null trade.t_trade_price:11 @@ -1259,8 +1259,8 @@ project │ ├── fd: ()-->(9), (1)-->(2,5,8,10,11) │ └── ordering: +2 opt(9) [actual: +2] └── projections - ├── trade.t_bid_price:8::FLOAT8 [as=t_bid_price:16, outer=(8), immutable] - └── trade.t_trade_price:11::FLOAT8 [as=t_trade_price:17, outer=(11), immutable] + ├── trade.t_bid_price:8::FLOAT8 [as=t_bid_price:17, outer=(8), immutable] + └── trade.t_trade_price:11::FLOAT8 [as=t_trade_price:18, outer=(11), immutable] # Q6 opt @@ -1316,11 +1316,11 @@ ORDER BY t_dts ASC LIMIT 20 ---- project - ├── columns: t_id:1!null t_ca_id:9!null t_exec_name:10!null t_is_cash:5!null t_trade_price:16 t_qty:7!null t_dts:2!null t_tt_id:4!null + ├── columns: t_id:1!null t_ca_id:9!null t_exec_name:10!null t_is_cash:5!null t_trade_price:17 t_qty:7!null t_dts:2!null t_tt_id:4!null ├── cardinality: [0 - 20] ├── immutable ├── key: (1) - ├── fd: (1)-->(2,4,5,7,9,10,16) + ├── fd: (1)-->(2,4,5,7,9,10,17) ├── ordering: +2 ├── scan trade@secondary │ ├── columns: t_id:1!null t_dts:2!null t_tt_id:4!null t_is_cash:5!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null trade.t_trade_price:11 @@ -1330,7 +1330,7 @@ project │ ├── fd: ()-->(6), (1)-->(2,4,5,7,9-11) │ └── ordering: +2 opt(6) [actual: +2] └── projections - └── trade.t_trade_price:11::FLOAT8 [as=t_trade_price:16, outer=(11), immutable] + └── trade.t_trade_price:11::FLOAT8 [as=t_trade_price:17, outer=(11), immutable] # Q9 opt @@ -1377,21 +1377,21 @@ limit │ ├── fd: (1,2)-->(3,4) │ ├── limit hint: 20.00 │ └── project - │ ├── columns: hh_h_t_id:1!null hh_t_id:2!null hh_before_qty:3!null hh_after_qty:4!null hh_h_t_id:5!null - │ ├── key: (2,5) - │ ├── fd: (1,2)-->(3,4), (1)==(5), (5)==(1) + │ ├── columns: hh_h_t_id:1!null hh_t_id:2!null hh_before_qty:3!null hh_after_qty:4!null hh_h_t_id:6!null + │ ├── key: (2,6) + │ ├── fd: (1,2)-->(3,4), (1)==(6), (6)==(1) │ ├── limit hint: 20.00 │ └── inner-join (lookup holding_history) - │ ├── columns: hh_h_t_id:1!null hh_t_id:2!null hh_before_qty:3!null hh_after_qty:4!null hh_h_t_id:5!null hh_t_id:6!null - │ ├── key columns: [5] = [1] - │ ├── key: (2,5) - │ ├── fd: ()-->(6), (1,2)-->(3,4), (1)==(5), (5)==(1) + │ ├── columns: hh_h_t_id:1!null hh_t_id:2!null hh_before_qty:3!null hh_after_qty:4!null hh_h_t_id:6!null hh_t_id:7!null + │ ├── key columns: [6] = [1] + │ ├── key: (2,6) + │ ├── fd: ()-->(7), (1,2)-->(3,4), (1)==(6), (6)==(1) │ ├── limit hint: 20.00 │ ├── scan holding_history@holding_history_auto_index_fk_hh_t_id_ref_trade - │ │ ├── columns: hh_h_t_id:5!null hh_t_id:6!null - │ │ ├── constraint: /6/5: [/0 - /0] - │ │ ├── key: (5) - │ │ ├── fd: ()-->(6) + │ │ ├── columns: hh_h_t_id:6!null hh_t_id:7!null + │ │ ├── constraint: /7/6: [/0 - /0] + │ │ ├── key: (6) + │ │ ├── fd: ()-->(7) │ │ └── limit hint: 10.00 │ └── filters (true) └── 20 @@ -1563,11 +1563,11 @@ opt SELECT lt_price::FLOAT8 FROM last_trade WHERE lt_s_symb = 'SYMB' ---- project - ├── columns: lt_price:6!null + ├── columns: lt_price:7!null ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(6) + ├── fd: ()-->(7) ├── scan last_trade │ ├── columns: lt_s_symb:1!null last_trade.lt_price:3!null │ ├── constraint: /1: [/'SYMB' - /'SYMB'] @@ -1575,7 +1575,7 @@ project │ ├── key: () │ └── fd: ()-->(1,3) └── projections - └── last_trade.lt_price:3::FLOAT8 [as=lt_price:6, outer=(3), immutable] + └── last_trade.lt_price:3::FLOAT8 [as=lt_price:7, outer=(3), immutable] # Q10 opt @@ -1621,7 +1621,7 @@ opt ORDER BY h_dts DESC ---- project - ├── columns: h_qty:6!null h_price:7!null [hidden: h_dts:4!null] + ├── columns: h_qty:6!null h_price:8!null [hidden: h_dts:4!null] ├── immutable ├── ordering: -4 ├── scan holding@secondary,rev @@ -1630,7 +1630,7 @@ project │ ├── fd: ()-->(2,3) │ └── ordering: -4 opt(2,3) [actual: -4] └── projections - └── holding.h_price:5::FLOAT8 [as=h_price:7, outer=(5), immutable] + └── holding.h_price:5::FLOAT8 [as=h_price:8, outer=(5), immutable] # Q13 opt @@ -1640,7 +1640,7 @@ opt ORDER BY h_dts ASC ---- project - ├── columns: h_qty:6!null h_price:7!null [hidden: h_dts:4!null] + ├── columns: h_qty:6!null h_price:8!null [hidden: h_dts:4!null] ├── immutable ├── ordering: +4 ├── scan holding@secondary @@ -1649,7 +1649,7 @@ project │ ├── fd: ()-->(2,3) │ └── ordering: +4 opt(2,3) [actual: +4] └── projections - └── holding.h_price:5::FLOAT8 [as=h_price:7, outer=(5), immutable] + └── holding.h_price:5::FLOAT8 [as=h_price:8, outer=(5), immutable] # Q14 opt @@ -1658,41 +1658,41 @@ SELECT sum(tx_rate)::FLOAT8 WHERE tx_id IN (SELECT cx_tx_id FROM customer_taxrate WHERE cx_c_id = 0) ---- project - ├── columns: sum:7 + ├── columns: sum:9 ├── cardinality: [1 - 1] ├── immutable ├── key: () - ├── fd: ()-->(7) + ├── fd: ()-->(9) ├── scalar-group-by - │ ├── columns: sum:6 + │ ├── columns: sum:8 │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(6) + │ ├── fd: ()-->(8) │ ├── project │ │ ├── columns: tx_id:1!null tx_rate:3!null │ │ ├── key: (1) │ │ ├── fd: (1)-->(3) │ │ └── project - │ │ ├── columns: tx_id:1!null tx_rate:3!null cx_tx_id:4!null - │ │ ├── key: (4) - │ │ ├── fd: (1)-->(3), (1)==(4), (4)==(1) + │ │ ├── columns: tx_id:1!null tx_rate:3!null cx_tx_id:5!null + │ │ ├── key: (5) + │ │ ├── fd: (1)-->(3), (1)==(5), (5)==(1) │ │ └── inner-join (lookup taxrate) - │ │ ├── columns: tx_id:1!null tx_rate:3!null cx_tx_id:4!null cx_c_id:5!null - │ │ ├── key columns: [4] = [1] + │ │ ├── columns: tx_id:1!null tx_rate:3!null cx_tx_id:5!null cx_c_id:6!null + │ │ ├── key columns: [5] = [1] │ │ ├── lookup columns are key - │ │ ├── key: (4) - │ │ ├── fd: ()-->(5), (1)-->(3), (1)==(4), (4)==(1) + │ │ ├── key: (5) + │ │ ├── fd: ()-->(6), (1)-->(3), (1)==(5), (5)==(1) │ │ ├── scan customer_taxrate@customer_taxrate_auto_index_fk_cx_c_id_ref_customer - │ │ │ ├── columns: cx_tx_id:4!null cx_c_id:5!null - │ │ │ ├── constraint: /5/4: [/0 - /0] - │ │ │ ├── key: (4) - │ │ │ └── fd: ()-->(5) + │ │ │ ├── columns: cx_tx_id:5!null cx_c_id:6!null + │ │ │ ├── constraint: /6/5: [/0 - /0] + │ │ │ ├── key: (5) + │ │ │ └── fd: ()-->(6) │ │ └── filters (true) │ └── aggregations - │ └── sum [as=sum:6, outer=(3)] + │ └── sum [as=sum:8, outer=(3)] │ └── tx_rate:3 └── projections - └── sum:6::FLOAT8 [as=sum:7, outer=(6), immutable] + └── sum:8::FLOAT8 [as=sum:9, outer=(8), immutable] # Q15 opt @@ -1705,7 +1705,7 @@ SELECT cr_rate::FLOAT8 AND cr_to_qty >= 10 ---- project - ├── columns: cr_rate:7!null + ├── columns: cr_rate:8!null ├── immutable ├── select │ ├── columns: cr_c_tier:1!null cr_tt_id:2!null cr_ex_id:3!null cr_from_qty:4!null cr_to_qty:5!null commission_rate.cr_rate:6!null @@ -1720,7 +1720,7 @@ project │ └── filters │ └── cr_to_qty:5 >= 10 [outer=(5), constraints=(/5: [/10 - ]; tight)] └── projections - └── commission_rate.cr_rate:6::FLOAT8 [as=cr_rate:7, outer=(6), immutable] + └── commission_rate.cr_rate:6::FLOAT8 [as=cr_rate:8, outer=(6), immutable] # Q16 opt @@ -1729,11 +1729,11 @@ SELECT ch_chrg::FLOAT8 WHERE ch_c_tier = 1 AND ch_tt_id = 'TMB' ---- project - ├── columns: ch_chrg:4!null + ├── columns: ch_chrg:5!null ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(4) + ├── fd: ()-->(5) ├── scan charge │ ├── columns: ch_tt_id:1!null ch_c_tier:2!null charge.ch_chrg:3!null │ ├── constraint: /1/2: [/'TMB'/1 - /'TMB'/1] @@ -1741,18 +1741,18 @@ project │ ├── key: () │ └── fd: ()-->(1-3) └── projections - └── charge.ch_chrg:3::FLOAT8 [as=ch_chrg:4, outer=(3), immutable] + └── charge.ch_chrg:3::FLOAT8 [as=ch_chrg:5, outer=(3), immutable] # Q17 opt SELECT ca_bal::FLOAT8 FROM customer_account WHERE ca_id = 0 ---- project - ├── columns: ca_bal:7!null + ├── columns: ca_bal:8!null ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(7) + ├── fd: ()-->(8) ├── scan customer_account │ ├── columns: ca_id:1!null customer_account.ca_bal:6!null │ ├── constraint: /1: [/0 - /0] @@ -1760,7 +1760,7 @@ project │ ├── key: () │ └── fd: ()-->(1,6) └── projections - └── customer_account.ca_bal:6::FLOAT8 [as=ca_bal:7, outer=(6), immutable] + └── customer_account.ca_bal:6::FLOAT8 [as=ca_bal:8, outer=(6), immutable] # Q18 opt @@ -1769,26 +1769,26 @@ SELECT sum(hs_qty * lt_price)::FLOAT8 WHERE hs_ca_id = 0 AND lt_s_symb = hs_s_symb ---- project - ├── columns: sum:11 + ├── columns: sum:13 ├── cardinality: [1 - 1] ├── immutable ├── key: () - ├── fd: ()-->(11) + ├── fd: ()-->(13) ├── scalar-group-by - │ ├── columns: sum:10 + │ ├── columns: sum:12 │ ├── cardinality: [1 - 1] │ ├── immutable │ ├── key: () - │ ├── fd: ()-->(10) + │ ├── fd: ()-->(12) │ ├── project - │ │ ├── columns: column9:9!null + │ │ ├── columns: column11:11!null │ │ ├── immutable │ │ ├── inner-join (lookup last_trade) - │ │ │ ├── columns: hs_ca_id:1!null hs_s_symb:2!null hs_qty:3!null lt_s_symb:4!null lt_price:6!null - │ │ │ ├── key columns: [2] = [4] + │ │ │ ├── columns: hs_ca_id:1!null hs_s_symb:2!null hs_qty:3!null lt_s_symb:5!null lt_price:7!null + │ │ │ ├── key columns: [2] = [5] │ │ │ ├── lookup columns are key - │ │ │ ├── key: (4) - │ │ │ ├── fd: ()-->(1), (2)-->(3), (4)-->(6), (2)==(4), (4)==(2) + │ │ │ ├── key: (5) + │ │ │ ├── fd: ()-->(1), (2)-->(3), (5)-->(7), (2)==(5), (5)==(2) │ │ │ ├── scan holding_summary │ │ │ │ ├── columns: hs_ca_id:1!null hs_s_symb:2!null hs_qty:3!null │ │ │ │ ├── constraint: /1/2: [/0 - /0] @@ -1796,12 +1796,12 @@ project │ │ │ │ └── fd: ()-->(1), (2)-->(3) │ │ │ └── filters (true) │ │ └── projections - │ │ └── hs_qty:3::INT8 * lt_price:6::DECIMAL [as=column9:9, outer=(3,6), immutable] + │ │ └── hs_qty:3::INT8 * lt_price:7::DECIMAL [as=column11:11, outer=(3,7), immutable] │ └── aggregations - │ └── sum [as=sum:10, outer=(9)] - │ └── column9:9 + │ └── sum [as=sum:12, outer=(11)] + │ └── column11:11 └── projections - └── sum:10::FLOAT8 [as=sum:11, outer=(10), immutable] + └── sum:12::FLOAT8 [as=sum:13, outer=(12), immutable] # Q19 opt @@ -1844,95 +1844,95 @@ VALUES ( insert trade ├── columns: ├── insert-mapping: - │ ├── column1:16 => t_id:1 - │ ├── column2:17 => t_dts:2 - │ ├── column3:18 => t_st_id:3 - │ ├── column4:19 => t_tt_id:4 - │ ├── column5:20 => t_is_cash:5 - │ ├── column6:21 => t_s_symb:6 - │ ├── column7:22 => t_qty:7 - │ ├── t_bid_price:31 => trade.t_bid_price:8 - │ ├── column9:24 => t_ca_id:9 - │ ├── column10:25 => t_exec_name:10 - │ ├── t_trade_price:32 => trade.t_trade_price:11 - │ ├── t_chrg:33 => trade.t_chrg:12 - │ ├── t_comm:34 => trade.t_comm:13 - │ ├── t_tax:35 => trade.t_tax:14 - │ └── column15:30 => t_lifo:15 - ├── check columns: check1:36 check2:37 check3:38 check4:39 check5:40 + │ ├── column1:17 => t_id:1 + │ ├── column2:18 => t_dts:2 + │ ├── column3:19 => t_st_id:3 + │ ├── column4:20 => t_tt_id:4 + │ ├── column5:21 => t_is_cash:5 + │ ├── column6:22 => t_s_symb:6 + │ ├── column7:23 => t_qty:7 + │ ├── t_bid_price:32 => trade.t_bid_price:8 + │ ├── column9:25 => t_ca_id:9 + │ ├── column10:26 => t_exec_name:10 + │ ├── t_trade_price:33 => trade.t_trade_price:11 + │ ├── t_chrg:34 => trade.t_chrg:12 + │ ├── t_comm:35 => trade.t_comm:13 + │ ├── t_tax:36 => trade.t_tax:14 + │ └── column15:31 => t_lifo:15 + ├── check columns: check1:37 check2:38 check3:39 check4:40 check5:41 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── values - │ ├── columns: column1:16!null column2:17!null column3:18!null column4:19!null column5:20!null column6:21!null column7:22!null column9:24!null column10:25!null column15:30!null t_bid_price:31!null t_trade_price:32 t_chrg:33!null t_comm:34!null t_tax:35!null check1:36!null check2:37!null check3:38!null check4:39!null check5:40!null + │ ├── columns: column1:17!null column2:18!null column3:19!null column4:20!null column5:21!null column6:22!null column7:23!null column9:25!null column10:26!null column15:31!null t_bid_price:32!null t_trade_price:33 t_chrg:34!null t_comm:35!null t_tax:36!null check1:37!null check2:38!null check3:39!null check4:40!null check5:41!null │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(16-22,24,25,30-40) + │ ├── fd: ()-->(17-23,25,26,31-41) │ └── (0, '2020-06-17 22:27:42.148484+00:00', 'SBMT', 'TMB', true, 'SYMB', 10, 0, 'Name', true, 1E+2, NULL, 1, 0, 0, true, true, true, true, true) └── f-k-checks ├── f-k-checks-item: trade(t_st_id) -> status_type(st_id) │ └── anti-join (lookup status_type) - │ ├── columns: column3:41!null - │ ├── key columns: [41] = [42] + │ ├── columns: column3:42!null + │ ├── key columns: [42] = [43] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(41) + │ ├── fd: ()-->(42) │ ├── with-scan &1 - │ │ ├── columns: column3:41!null + │ │ ├── columns: column3:42!null │ │ ├── mapping: - │ │ │ └── column3:18 => column3:41 + │ │ │ └── column3:19 => column3:42 │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(41) + │ │ └── fd: ()-->(42) │ └── filters (true) ├── f-k-checks-item: trade(t_tt_id) -> trade_type(tt_id) │ └── anti-join (lookup trade_type) - │ ├── columns: column4:44!null - │ ├── key columns: [44] = [45] + │ ├── columns: column4:46!null + │ ├── key columns: [46] = [47] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(44) + │ ├── fd: ()-->(46) │ ├── with-scan &1 - │ │ ├── columns: column4:44!null + │ │ ├── columns: column4:46!null │ │ ├── mapping: - │ │ │ └── column4:19 => column4:44 + │ │ │ └── column4:20 => column4:46 │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(44) + │ │ └── fd: ()-->(46) │ └── filters (true) ├── f-k-checks-item: trade(t_s_symb) -> security(s_symb) │ └── anti-join (lookup security) - │ ├── columns: column6:49!null - │ ├── key columns: [49] = [50] + │ ├── columns: column6:52!null + │ ├── key columns: [52] = [53] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(49) + │ ├── fd: ()-->(52) │ ├── with-scan &1 - │ │ ├── columns: column6:49!null + │ │ ├── columns: column6:52!null │ │ ├── mapping: - │ │ │ └── column6:21 => column6:49 + │ │ │ └── column6:22 => column6:52 │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(49) + │ │ └── fd: ()-->(52) │ └── filters (true) └── f-k-checks-item: trade(t_ca_id) -> customer_account(ca_id) └── anti-join (lookup customer_account) - ├── columns: column9:66!null - ├── key columns: [66] = [67] + ├── columns: column9:70!null + ├── key columns: [70] = [71] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(66) + ├── fd: ()-->(70) ├── with-scan &1 - │ ├── columns: column9:66!null + │ ├── columns: column9:70!null │ ├── mapping: - │ │ └── column9:24 => column9:66 + │ │ └── column9:25 => column9:70 │ ├── cardinality: [1 - 1] │ ├── key: () - │ └── fd: ()-->(66) + │ └── fd: ()-->(70) └── filters (true) # Q20 @@ -1943,86 +1943,86 @@ VALUES (0, 'TMB', 'SYMB', 10, '100.00':::FLOAT8::DECIMAL, 0) insert trade_request ├── columns: ├── insert-mapping: - │ ├── column1:7 => tr_t_id:1 - │ ├── column2:8 => tr_tt_id:2 - │ ├── column3:9 => tr_s_symb:3 - │ ├── column4:10 => tr_qty:4 - │ ├── tr_bid_price:13 => trade_request.tr_bid_price:5 - │ └── column6:12 => tr_b_id:6 - ├── check columns: check1:14 check2:15 + │ ├── column1:8 => tr_t_id:1 + │ ├── column2:9 => tr_tt_id:2 + │ ├── column3:10 => tr_s_symb:3 + │ ├── column4:11 => tr_qty:4 + │ ├── tr_bid_price:14 => trade_request.tr_bid_price:5 + │ └── column6:13 => tr_b_id:6 + ├── check columns: check1:15 check2:16 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── values - │ ├── columns: column1:7!null column2:8!null column3:9!null column4:10!null column6:12!null tr_bid_price:13!null check1:14!null check2:15!null + │ ├── columns: column1:8!null column2:9!null column3:10!null column4:11!null column6:13!null tr_bid_price:14!null check1:15!null check2:16!null │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(7-10,12-15) + │ ├── fd: ()-->(8-11,13-16) │ └── (0, 'TMB', 'SYMB', 10, 0, 1E+2, true, true) └── f-k-checks ├── f-k-checks-item: trade_request(tr_t_id) -> trade(t_id) │ └── anti-join (lookup trade) - │ ├── columns: column1:16!null - │ ├── key columns: [16] = [17] + │ ├── columns: column1:17!null + │ ├── key columns: [17] = [18] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(16) + │ ├── fd: ()-->(17) │ ├── with-scan &1 - │ │ ├── columns: column1:16!null + │ │ ├── columns: column1:17!null │ │ ├── mapping: - │ │ │ └── column1:7 => column1:16 + │ │ │ └── column1:8 => column1:17 │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(16) + │ │ └── fd: ()-->(17) │ └── filters (true) ├── f-k-checks-item: trade_request(tr_tt_id) -> trade_type(tt_id) │ └── anti-join (lookup trade_type) - │ ├── columns: column2:32!null - │ ├── key columns: [32] = [33] + │ ├── columns: column2:34!null + │ ├── key columns: [34] = [35] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(32) + │ ├── fd: ()-->(34) │ ├── with-scan &1 - │ │ ├── columns: column2:32!null + │ │ ├── columns: column2:34!null │ │ ├── mapping: - │ │ │ └── column2:8 => column2:32 + │ │ │ └── column2:9 => column2:34 │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(32) + │ │ └── fd: ()-->(34) │ └── filters (true) ├── f-k-checks-item: trade_request(tr_s_symb) -> security(s_symb) │ └── anti-join (lookup security) - │ ├── columns: column3:37!null - │ ├── key columns: [37] = [38] + │ ├── columns: column3:40!null + │ ├── key columns: [40] = [41] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(37) + │ ├── fd: ()-->(40) │ ├── with-scan &1 - │ │ ├── columns: column3:37!null + │ │ ├── columns: column3:40!null │ │ ├── mapping: - │ │ │ └── column3:9 => column3:37 + │ │ │ └── column3:10 => column3:40 │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(37) + │ │ └── fd: ()-->(40) │ └── filters (true) └── f-k-checks-item: trade_request(tr_b_id) -> broker(b_id) └── anti-join (lookup broker) - ├── columns: column6:54!null - ├── key columns: [54] = [55] + ├── columns: column6:58!null + ├── key columns: [58] = [59] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(54) + ├── fd: ()-->(58) ├── with-scan &1 - │ ├── columns: column6:54!null + │ ├── columns: column6:58!null │ ├── mapping: - │ │ └── column6:12 => column6:54 + │ │ └── column6:13 => column6:58 │ ├── cardinality: [1 - 1] │ ├── key: () - │ └── fd: ()-->(54) + │ └── fd: ()-->(58) └── filters (true) # Q21 @@ -2033,50 +2033,50 @@ VALUES (0, '2020-06-15 22:27:42.148484+00:00'::TIMESTAMP, 'SBMT') insert trade_history ├── columns: ├── insert-mapping: - │ ├── column1:4 => th_t_id:1 - │ ├── column2:5 => th_dts:2 - │ └── column3:6 => th_st_id:3 + │ ├── column1:5 => th_t_id:1 + │ ├── column2:6 => th_dts:2 + │ └── column3:7 => th_st_id:3 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── values - │ ├── columns: column1:4!null column2:5!null column3:6!null + │ ├── columns: column1:5!null column2:6!null column3:7!null │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(4-6) + │ ├── fd: ()-->(5-7) │ └── (0, '2020-06-15 22:27:42.148484+00:00', 'SBMT') └── f-k-checks ├── f-k-checks-item: trade_history(th_t_id) -> trade(t_id) │ └── anti-join (lookup trade) - │ ├── columns: column1:7!null - │ ├── key columns: [7] = [8] + │ ├── columns: column1:8!null + │ ├── key columns: [8] = [9] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(7) + │ ├── fd: ()-->(8) │ ├── with-scan &1 - │ │ ├── columns: column1:7!null + │ │ ├── columns: column1:8!null │ │ ├── mapping: - │ │ │ └── column1:4 => column1:7 + │ │ │ └── column1:5 => column1:8 │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(7) + │ │ └── fd: ()-->(8) │ └── filters (true) └── f-k-checks-item: trade_history(th_st_id) -> status_type(st_id) └── anti-join (lookup status_type) - ├── columns: column3:23!null - ├── key columns: [23] = [24] + ├── columns: column3:25!null + ├── key columns: [25] = [26] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(23) + ├── fd: ()-->(25) ├── with-scan &1 - │ ├── columns: column3:23!null + │ ├── columns: column3:25!null │ ├── mapping: - │ │ └── column3:6 => column3:23 + │ │ └── column3:7 => column3:25 │ ├── cardinality: [1 - 1] │ ├── key: () - │ └── fd: ()-->(23) + │ └── fd: ()-->(25) └── filters (true) # -------------------------------------------------- @@ -2098,11 +2098,11 @@ SELECT t_ca_id, t_tt_id, t_s_symb, t_qty, t_chrg::FLOAT8, t_lifo, t_is_cash WHERE t_id = 0 ---- project - ├── columns: t_ca_id:9!null t_tt_id:4!null t_s_symb:6!null t_qty:7!null t_chrg:16!null t_lifo:15!null t_is_cash:5!null + ├── columns: t_ca_id:9!null t_tt_id:4!null t_s_symb:6!null t_qty:7!null t_chrg:17!null t_lifo:15!null t_is_cash:5!null ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(4-7,9,15,16) + ├── fd: ()-->(4-7,9,15,17) ├── scan trade │ ├── columns: t_id:1!null t_tt_id:4!null t_is_cash:5!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null trade.t_chrg:12!null t_lifo:15!null │ ├── constraint: /1: [/0 - /0] @@ -2110,7 +2110,7 @@ project │ ├── key: () │ └── fd: ()-->(1,4-7,9,12,15) └── projections - └── trade.t_chrg:12::FLOAT8 [as=t_chrg:16, outer=(12), immutable] + └── trade.t_chrg:12::FLOAT8 [as=t_chrg:17, outer=(12), immutable] # Q2 opt @@ -2171,50 +2171,50 @@ INSERT INTO holding_summary (hs_ca_id, hs_s_symb, hs_qty) VALUES (0, 'SYMB', 10) insert holding_summary ├── columns: ├── insert-mapping: - │ ├── column1:4 => hs_ca_id:1 - │ ├── column2:5 => hs_s_symb:2 - │ └── column3:6 => hs_qty:3 + │ ├── column1:5 => hs_ca_id:1 + │ ├── column2:6 => hs_s_symb:2 + │ └── column3:7 => hs_qty:3 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── values - │ ├── columns: column1:4!null column2:5!null column3:6!null + │ ├── columns: column1:5!null column2:6!null column3:7!null │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(4-6) + │ ├── fd: ()-->(5-7) │ └── (0, 'SYMB', 10) └── f-k-checks ├── f-k-checks-item: holding_summary(hs_ca_id) -> customer_account(ca_id) │ └── anti-join (lookup customer_account) - │ ├── columns: column1:7!null - │ ├── key columns: [7] = [8] + │ ├── columns: column1:8!null + │ ├── key columns: [8] = [9] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(7) + │ ├── fd: ()-->(8) │ ├── with-scan &1 - │ │ ├── columns: column1:7!null + │ │ ├── columns: column1:8!null │ │ ├── mapping: - │ │ │ └── column1:4 => column1:7 + │ │ │ └── column1:5 => column1:8 │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(7) + │ │ └── fd: ()-->(8) │ └── filters (true) └── f-k-checks-item: holding_summary(hs_s_symb) -> security(s_symb) └── anti-join (lookup security) - ├── columns: column2:14!null - ├── key columns: [14] = [15] + ├── columns: column2:16!null + ├── key columns: [16] = [17] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(14) + ├── fd: ()-->(16) ├── with-scan &1 - │ ├── columns: column2:14!null + │ ├── columns: column2:16!null │ ├── mapping: - │ │ └── column2:5 => column2:14 + │ │ └── column2:6 => column2:16 │ ├── cardinality: [1 - 1] │ ├── key: () - │ └── fd: ()-->(14) + │ └── fd: ()-->(16) └── filters (true) # Q6 @@ -2225,25 +2225,25 @@ UPDATE holding_summary ---- update holding_summary ├── columns: - ├── fetch columns: hs_ca_id:4 hs_s_symb:5 hs_qty:6 + ├── fetch columns: hs_ca_id:5 hs_s_symb:6 hs_qty:7 ├── update-mapping: - │ └── hs_qty_new:7 => hs_qty:3 + │ └── hs_qty_new:9 => hs_qty:3 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: hs_qty_new:7!null hs_ca_id:4!null hs_s_symb:5!null hs_qty:6!null + ├── columns: hs_qty_new:9!null hs_ca_id:5!null hs_s_symb:6!null hs_qty:7!null ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(4-7) + ├── fd: ()-->(5-7,9) ├── scan holding_summary - │ ├── columns: hs_ca_id:4!null hs_s_symb:5!null hs_qty:6!null - │ ├── constraint: /4/5: [/0/'SYMB' - /0/'SYMB'] + │ ├── columns: hs_ca_id:5!null hs_s_symb:6!null hs_qty:7!null + │ ├── constraint: /5/6: [/0/'SYMB' - /0/'SYMB'] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(4-6) + │ └── fd: ()-->(5-7) └── projections - └── hs_qty:6::INT8 + 10 [as=hs_qty_new:7, outer=(6), immutable] + └── hs_qty:7::INT8 + 10 [as=hs_qty_new:9, outer=(7), immutable] # Q7 opt @@ -2253,10 +2253,10 @@ opt ORDER BY h_dts DESC ---- project - ├── columns: h_t_id:1!null h_qty:6!null h_price:7!null [hidden: h_dts:4!null] + ├── columns: h_t_id:1!null h_qty:6!null h_price:8!null [hidden: h_dts:4!null] ├── immutable ├── key: (1) - ├── fd: (1)-->(4,6,7) + ├── fd: (1)-->(4,6,8) ├── ordering: -4 ├── scan holding@secondary,rev │ ├── columns: h_t_id:1!null h_ca_id:2!null h_s_symb:3!null h_dts:4!null holding.h_price:5!null h_qty:6!null @@ -2265,7 +2265,7 @@ project │ ├── fd: ()-->(2,3), (1)-->(4-6) │ └── ordering: -4 opt(2,3) [actual: -4] └── projections - └── holding.h_price:5::FLOAT8 [as=h_price:7, outer=(5), immutable] + └── holding.h_price:5::FLOAT8 [as=h_price:8, outer=(5), immutable] # Q8 opt @@ -2275,10 +2275,10 @@ opt ORDER BY h_dts ASC ---- project - ├── columns: h_t_id:1!null h_qty:6!null h_price:7!null [hidden: h_dts:4!null] + ├── columns: h_t_id:1!null h_qty:6!null h_price:8!null [hidden: h_dts:4!null] ├── immutable ├── key: (1) - ├── fd: (1)-->(4,6,7) + ├── fd: (1)-->(4,6,8) ├── ordering: +4 ├── scan holding@secondary │ ├── columns: h_t_id:1!null h_ca_id:2!null h_s_symb:3!null h_dts:4!null holding.h_price:5!null h_qty:6!null @@ -2287,7 +2287,7 @@ project │ ├── fd: ()-->(2,3), (1)-->(4-6) │ └── ordering: +4 opt(2,3) [actual: +4] └── projections - └── holding.h_price:5::FLOAT8 [as=h_price:7, outer=(5), immutable] + └── holding.h_price:5::FLOAT8 [as=h_price:8, outer=(5), immutable] # Q9 opt @@ -2297,51 +2297,51 @@ VALUES (0, 0, 0, 10) insert holding_history ├── columns: ├── insert-mapping: - │ ├── column1:5 => hh_h_t_id:1 - │ ├── column2:6 => hh_t_id:2 - │ ├── column3:7 => hh_before_qty:3 - │ └── column4:8 => hh_after_qty:4 + │ ├── column1:6 => hh_h_t_id:1 + │ ├── column2:7 => hh_t_id:2 + │ ├── column3:8 => hh_before_qty:3 + │ └── column4:9 => hh_after_qty:4 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── values - │ ├── columns: column1:5!null column2:6!null column3:7!null column4:8!null + │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(5-8) + │ ├── fd: ()-->(6-9) │ └── (0, 0, 0, 10) └── f-k-checks ├── f-k-checks-item: holding_history(hh_h_t_id) -> trade(t_id) │ └── anti-join (lookup trade) - │ ├── columns: column1:9!null - │ ├── key columns: [9] = [10] + │ ├── columns: column1:10!null + │ ├── key columns: [10] = [11] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(9) + │ ├── fd: ()-->(10) │ ├── with-scan &1 - │ │ ├── columns: column1:9!null + │ │ ├── columns: column1:10!null │ │ ├── mapping: - │ │ │ └── column1:5 => column1:9 + │ │ │ └── column1:6 => column1:10 │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(9) + │ │ └── fd: ()-->(10) │ └── filters (true) └── f-k-checks-item: holding_history(hh_t_id) -> trade(t_id) └── anti-join (lookup trade) - ├── columns: column2:25!null - ├── key columns: [25] = [26] + ├── columns: column2:27!null + ├── key columns: [27] = [28] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(25) + ├── fd: ()-->(27) ├── with-scan &1 - │ ├── columns: column2:25!null + │ ├── columns: column2:27!null │ ├── mapping: - │ │ └── column2:6 => column2:25 + │ │ └── column2:7 => column2:27 │ ├── cardinality: [1 - 1] │ ├── key: () - │ └── fd: ()-->(25) + │ └── fd: ()-->(27) └── filters (true) # Q10 @@ -2350,27 +2350,27 @@ UPDATE holding SET h_qty = h_qty + 10 WHERE h_t_id = 0 ---- update holding ├── columns: - ├── fetch columns: h_t_id:7 h_ca_id:8 h_s_symb:9 h_dts:10 h_price:11 h_qty:12 + ├── fetch columns: h_t_id:8 h_ca_id:9 h_s_symb:10 h_dts:11 h_price:12 h_qty:13 ├── update-mapping: - │ └── h_qty_new:13 => h_qty:6 - ├── check columns: check1:14 + │ └── h_qty_new:15 => h_qty:6 + ├── check columns: check1:16 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: check1:14!null h_qty_new:13!null h_t_id:7!null h_ca_id:8!null h_s_symb:9!null h_dts:10!null h_price:11!null h_qty:12!null + ├── columns: check1:16!null h_qty_new:15!null h_t_id:8!null h_ca_id:9!null h_s_symb:10!null h_dts:11!null h_price:12!null h_qty:13!null ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(7-14) + ├── fd: ()-->(8-13,15,16) ├── scan holding - │ ├── columns: h_t_id:7!null h_ca_id:8!null h_s_symb:9!null h_dts:10!null h_price:11!null h_qty:12!null - │ ├── constraint: /7: [/0 - /0] + │ ├── columns: h_t_id:8!null h_ca_id:9!null h_s_symb:10!null h_dts:11!null h_price:12!null h_qty:13!null + │ ├── constraint: /8: [/0 - /0] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(7-12) + │ └── fd: ()-->(8-13) └── projections - ├── h_price:11 > 0 [as=check1:14, outer=(11), immutable] - └── h_qty:12::INT8 + 10 [as=h_qty_new:13, outer=(12), immutable] + ├── h_price:12 > 0 [as=check1:16, outer=(12), immutable] + └── h_qty:13::INT8 + 10 [as=h_qty_new:15, outer=(13), immutable] # Q11 opt @@ -2378,15 +2378,15 @@ DELETE FROM holding WHERE h_t_id = 0 ---- delete holding ├── columns: - ├── fetch columns: h_t_id:7 h_ca_id:8 h_s_symb:9 h_dts:10 + ├── fetch columns: h_t_id:8 h_ca_id:9 h_s_symb:10 h_dts:11 ├── cardinality: [0 - 0] ├── volatile, mutations └── scan holding - ├── columns: h_t_id:7!null h_ca_id:8!null h_s_symb:9!null h_dts:10!null - ├── constraint: /7: [/0 - /0] + ├── columns: h_t_id:8!null h_ca_id:9!null h_s_symb:10!null h_dts:11!null + ├── constraint: /8: [/0 - /0] ├── cardinality: [0 - 1] ├── key: () - └── fd: ()-->(7-10) + └── fd: ()-->(8-11) # Q12 opt @@ -2402,55 +2402,55 @@ VALUES ( insert holding ├── columns: ├── insert-mapping: - │ ├── column1:7 => h_t_id:1 - │ ├── column2:8 => h_ca_id:2 - │ ├── column3:9 => h_s_symb:3 - │ ├── column4:10 => h_dts:4 - │ ├── h_price:13 => holding.h_price:5 - │ └── column6:12 => h_qty:6 - ├── check columns: check1:14 + │ ├── column1:8 => h_t_id:1 + │ ├── column2:9 => h_ca_id:2 + │ ├── column3:10 => h_s_symb:3 + │ ├── column4:11 => h_dts:4 + │ ├── h_price:14 => holding.h_price:5 + │ └── column6:13 => h_qty:6 + ├── check columns: check1:15 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── values - │ ├── columns: column1:7!null column2:8!null column3:9!null column4:10!null column6:12!null h_price:13!null check1:14!null + │ ├── columns: column1:8!null column2:9!null column3:10!null column4:11!null column6:13!null h_price:14!null check1:15!null │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(7-10,12-14) + │ ├── fd: ()-->(8-11,13-15) │ └── (0, 0, 'SYMB', '2020-06-17 22:27:42.148484+00:00', 10, 1E+2, true) └── f-k-checks ├── f-k-checks-item: holding(h_t_id) -> trade(t_id) │ └── anti-join (lookup trade) - │ ├── columns: column1:15!null - │ ├── key columns: [15] = [16] + │ ├── columns: column1:16!null + │ ├── key columns: [16] = [17] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(15) + │ ├── fd: ()-->(16) │ ├── with-scan &1 - │ │ ├── columns: column1:15!null + │ │ ├── columns: column1:16!null │ │ ├── mapping: - │ │ │ └── column1:7 => column1:15 + │ │ │ └── column1:8 => column1:16 │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(15) + │ │ └── fd: ()-->(16) │ └── filters (true) └── f-k-checks-item: holding(h_ca_id,h_s_symb) -> holding_summary(hs_ca_id,hs_s_symb) └── anti-join (lookup holding_summary) - ├── columns: column2:31!null column3:32!null - ├── key columns: [31 32] = [33 34] + ├── columns: column2:33!null column3:34!null + ├── key columns: [33 34] = [35 36] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(31,32) + ├── fd: ()-->(33,34) ├── with-scan &1 - │ ├── columns: column2:31!null column3:32!null + │ ├── columns: column2:33!null column3:34!null │ ├── mapping: - │ │ ├── column2:8 => column2:31 - │ │ └── column3:9 => column3:32 + │ │ ├── column2:9 => column2:33 + │ │ └── column3:10 => column3:34 │ ├── cardinality: [1 - 1] │ ├── key: () - │ └── fd: ()-->(31,32) + │ └── fd: ()-->(33,34) └── filters (true) # Q13 @@ -2459,32 +2459,32 @@ DELETE FROM holding_summary WHERE hs_ca_id = 0 AND hs_s_symb = 'SYMB' ---- delete holding_summary ├── columns: - ├── fetch columns: holding_summary.hs_ca_id:4 holding_summary.hs_s_symb:5 + ├── fetch columns: holding_summary.hs_ca_id:5 holding_summary.hs_s_symb:6 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── scan holding_summary - │ ├── columns: holding_summary.hs_ca_id:4!null holding_summary.hs_s_symb:5!null - │ ├── constraint: /4/5: [/0/'SYMB' - /0/'SYMB'] + │ ├── columns: holding_summary.hs_ca_id:5!null holding_summary.hs_s_symb:6!null + │ ├── constraint: /5/6: [/0/'SYMB' - /0/'SYMB'] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(4,5) + │ └── fd: ()-->(5,6) └── f-k-checks └── f-k-checks-item: holding(h_ca_id,h_s_symb) -> holding_summary(hs_ca_id,hs_s_symb) └── semi-join (lookup holding@secondary) - ├── columns: hs_ca_id:7!null hs_s_symb:8!null - ├── key columns: [7 8] = [10 11] + ├── columns: hs_ca_id:9!null hs_s_symb:10!null + ├── key columns: [9 10] = [12 13] ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(7,8) + ├── fd: ()-->(9,10) ├── with-scan &1 - │ ├── columns: hs_ca_id:7!null hs_s_symb:8!null + │ ├── columns: hs_ca_id:9!null hs_s_symb:10!null │ ├── mapping: - │ │ ├── holding_summary.hs_ca_id:4 => hs_ca_id:7 - │ │ └── holding_summary.hs_s_symb:5 => hs_s_symb:8 + │ │ ├── holding_summary.hs_ca_id:5 => hs_ca_id:9 + │ │ └── holding_summary.hs_s_symb:6 => hs_s_symb:10 │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(7,8) + │ └── fd: ()-->(9,10) └── filters (true) # Q14 @@ -2495,50 +2495,50 @@ VALUES (0, 'SYMB', 10) insert holding_summary ├── columns: ├── insert-mapping: - │ ├── column1:4 => hs_ca_id:1 - │ ├── column2:5 => hs_s_symb:2 - │ └── column3:6 => hs_qty:3 + │ ├── column1:5 => hs_ca_id:1 + │ ├── column2:6 => hs_s_symb:2 + │ └── column3:7 => hs_qty:3 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── values - │ ├── columns: column1:4!null column2:5!null column3:6!null + │ ├── columns: column1:5!null column2:6!null column3:7!null │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(4-6) + │ ├── fd: ()-->(5-7) │ └── (0, 'SYMB', 10) └── f-k-checks ├── f-k-checks-item: holding_summary(hs_ca_id) -> customer_account(ca_id) │ └── anti-join (lookup customer_account) - │ ├── columns: column1:7!null - │ ├── key columns: [7] = [8] + │ ├── columns: column1:8!null + │ ├── key columns: [8] = [9] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(7) + │ ├── fd: ()-->(8) │ ├── with-scan &1 - │ │ ├── columns: column1:7!null + │ │ ├── columns: column1:8!null │ │ ├── mapping: - │ │ │ └── column1:4 => column1:7 + │ │ │ └── column1:5 => column1:8 │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(7) + │ │ └── fd: ()-->(8) │ └── filters (true) └── f-k-checks-item: holding_summary(hs_s_symb) -> security(s_symb) └── anti-join (lookup security) - ├── columns: column2:14!null - ├── key columns: [14] = [15] + ├── columns: column2:16!null + ├── key columns: [16] = [17] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(14) + ├── fd: ()-->(16) ├── with-scan &1 - │ ├── columns: column2:14!null + │ ├── columns: column2:16!null │ ├── mapping: - │ │ └── column2:5 => column2:14 + │ │ └── column2:6 => column2:16 │ ├── cardinality: [1 - 1] │ ├── key: () - │ └── fd: ()-->(14) + │ └── fd: ()-->(16) └── filters (true) @@ -2550,25 +2550,25 @@ WHERE hs_ca_id = 0 and hs_s_symb = 'SYMB' ---- update holding_summary ├── columns: - ├── fetch columns: hs_ca_id:4 hs_s_symb:5 hs_qty:6 + ├── fetch columns: hs_ca_id:5 hs_s_symb:6 hs_qty:7 ├── update-mapping: - │ └── hs_qty_new:7 => hs_qty:3 + │ └── hs_qty_new:9 => hs_qty:3 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: hs_qty_new:7!null hs_ca_id:4!null hs_s_symb:5!null hs_qty:6!null + ├── columns: hs_qty_new:9!null hs_ca_id:5!null hs_s_symb:6!null hs_qty:7!null ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(4-7) + ├── fd: ()-->(5-7,9) ├── scan holding_summary - │ ├── columns: hs_ca_id:4!null hs_s_symb:5!null hs_qty:6!null - │ ├── constraint: /4/5: [/0/'SYMB' - /0/'SYMB'] + │ ├── columns: hs_ca_id:5!null hs_s_symb:6!null hs_qty:7!null + │ ├── constraint: /5/6: [/0/'SYMB' - /0/'SYMB'] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(4-6) + │ └── fd: ()-->(5-7) └── projections - └── hs_qty:6::INT8 + 10 [as=hs_qty_new:7, outer=(6), immutable] + └── hs_qty:7::INT8 + 10 [as=hs_qty_new:9, outer=(7), immutable] # Q16 opt @@ -2579,51 +2579,51 @@ VALUES (0, 0, 0, 10) insert holding_history ├── columns: ├── insert-mapping: - │ ├── column1:5 => hh_h_t_id:1 - │ ├── column2:6 => hh_t_id:2 - │ ├── column3:7 => hh_before_qty:3 - │ └── column4:8 => hh_after_qty:4 + │ ├── column1:6 => hh_h_t_id:1 + │ ├── column2:7 => hh_t_id:2 + │ ├── column3:8 => hh_before_qty:3 + │ └── column4:9 => hh_after_qty:4 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── values - │ ├── columns: column1:5!null column2:6!null column3:7!null column4:8!null + │ ├── columns: column1:6!null column2:7!null column3:8!null column4:9!null │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(5-8) + │ ├── fd: ()-->(6-9) │ └── (0, 0, 0, 10) └── f-k-checks ├── f-k-checks-item: holding_history(hh_h_t_id) -> trade(t_id) │ └── anti-join (lookup trade) - │ ├── columns: column1:9!null - │ ├── key columns: [9] = [10] + │ ├── columns: column1:10!null + │ ├── key columns: [10] = [11] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(9) + │ ├── fd: ()-->(10) │ ├── with-scan &1 - │ │ ├── columns: column1:9!null + │ │ ├── columns: column1:10!null │ │ ├── mapping: - │ │ │ └── column1:5 => column1:9 + │ │ │ └── column1:6 => column1:10 │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(9) + │ │ └── fd: ()-->(10) │ └── filters (true) └── f-k-checks-item: holding_history(hh_t_id) -> trade(t_id) └── anti-join (lookup trade) - ├── columns: column2:25!null - ├── key columns: [25] = [26] + ├── columns: column2:27!null + ├── key columns: [27] = [28] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(25) + ├── fd: ()-->(27) ├── with-scan &1 - │ ├── columns: column2:25!null + │ ├── columns: column2:27!null │ ├── mapping: - │ │ └── column2:6 => column2:25 + │ │ └── column2:7 => column2:27 │ ├── cardinality: [1 - 1] │ ├── key: () - │ └── fd: ()-->(25) + │ └── fd: ()-->(27) └── filters (true) # Q17 @@ -2637,41 +2637,41 @@ SELECT sum(tx_rate)::FLOAT8 WHERE tx_id IN (SELECT cx_tx_id FROM customer_taxrate WHERE cx_c_id = 0) ---- project - ├── columns: sum:7 + ├── columns: sum:9 ├── cardinality: [1 - 1] ├── immutable ├── key: () - ├── fd: ()-->(7) + ├── fd: ()-->(9) ├── scalar-group-by - │ ├── columns: sum:6 + │ ├── columns: sum:8 │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(6) + │ ├── fd: ()-->(8) │ ├── project │ │ ├── columns: tx_id:1!null tx_rate:3!null │ │ ├── key: (1) │ │ ├── fd: (1)-->(3) │ │ └── project - │ │ ├── columns: tx_id:1!null tx_rate:3!null cx_tx_id:4!null - │ │ ├── key: (4) - │ │ ├── fd: (1)-->(3), (1)==(4), (4)==(1) + │ │ ├── columns: tx_id:1!null tx_rate:3!null cx_tx_id:5!null + │ │ ├── key: (5) + │ │ ├── fd: (1)-->(3), (1)==(5), (5)==(1) │ │ └── inner-join (lookup taxrate) - │ │ ├── columns: tx_id:1!null tx_rate:3!null cx_tx_id:4!null cx_c_id:5!null - │ │ ├── key columns: [4] = [1] + │ │ ├── columns: tx_id:1!null tx_rate:3!null cx_tx_id:5!null cx_c_id:6!null + │ │ ├── key columns: [5] = [1] │ │ ├── lookup columns are key - │ │ ├── key: (4) - │ │ ├── fd: ()-->(5), (1)-->(3), (1)==(4), (4)==(1) + │ │ ├── key: (5) + │ │ ├── fd: ()-->(6), (1)-->(3), (1)==(5), (5)==(1) │ │ ├── scan customer_taxrate@customer_taxrate_auto_index_fk_cx_c_id_ref_customer - │ │ │ ├── columns: cx_tx_id:4!null cx_c_id:5!null - │ │ │ ├── constraint: /5/4: [/0 - /0] - │ │ │ ├── key: (4) - │ │ │ └── fd: ()-->(5) + │ │ │ ├── columns: cx_tx_id:5!null cx_c_id:6!null + │ │ │ ├── constraint: /6/5: [/0 - /0] + │ │ │ ├── key: (5) + │ │ │ └── fd: ()-->(6) │ │ └── filters (true) │ └── aggregations - │ └── sum [as=sum:6, outer=(3)] + │ └── sum [as=sum:8, outer=(3)] │ └── tx_rate:3 └── projections - └── sum:6::FLOAT8 [as=sum:7, outer=(6), immutable] + └── sum:8::FLOAT8 [as=sum:9, outer=(8), immutable] # Q18 opt @@ -2679,31 +2679,31 @@ UPDATE trade SET t_tax = '0.00':::FLOAT8::DECIMAL WHERE t_id = 0 ---- update trade ├── columns: - ├── fetch columns: t_id:16 t_dts:17 t_st_id:18 t_tt_id:19 t_is_cash:20 t_s_symb:21 t_qty:22 t_bid_price:23 t_ca_id:24 t_exec_name:25 t_trade_price:26 t_chrg:27 t_comm:28 trade.t_tax:29 t_lifo:30 + ├── fetch columns: t_id:17 t_dts:18 t_st_id:19 t_tt_id:20 t_is_cash:21 t_s_symb:22 t_qty:23 t_bid_price:24 t_ca_id:25 t_exec_name:26 t_trade_price:27 t_chrg:28 t_comm:29 trade.t_tax:30 t_lifo:31 ├── update-mapping: - │ └── t_tax:32 => trade.t_tax:14 - ├── check columns: check1:33 check2:34 check3:35 check4:36 check5:37 + │ └── t_tax:34 => trade.t_tax:14 + ├── check columns: check1:35 check2:36 check3:37 check4:38 check5:39 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: check1:33!null check2:34 check3:35!null check4:36!null check5:37!null t_tax:32!null t_id:16!null t_dts:17!null t_st_id:18!null t_tt_id:19!null t_is_cash:20!null t_s_symb:21!null t_qty:22!null t_bid_price:23 t_ca_id:24!null t_exec_name:25!null t_trade_price:26 t_chrg:27!null t_comm:28!null trade.t_tax:29!null t_lifo:30!null + ├── columns: check1:35!null check2:36 check3:37!null check4:38!null check5:39!null t_tax:34!null t_id:17!null t_dts:18!null t_st_id:19!null t_tt_id:20!null t_is_cash:21!null t_s_symb:22!null t_qty:23!null t_bid_price:24 t_ca_id:25!null t_exec_name:26!null t_trade_price:27 t_chrg:28!null t_comm:29!null trade.t_tax:30!null t_lifo:31!null ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(16-30,32-37) + ├── fd: ()-->(17-31,34-39) ├── scan trade - │ ├── columns: t_id:16!null t_dts:17!null t_st_id:18!null t_tt_id:19!null t_is_cash:20!null t_s_symb:21!null t_qty:22!null t_bid_price:23 t_ca_id:24!null t_exec_name:25!null t_trade_price:26 t_chrg:27!null t_comm:28!null trade.t_tax:29!null t_lifo:30!null - │ ├── constraint: /16: [/0 - /0] + │ ├── columns: t_id:17!null t_dts:18!null t_st_id:19!null t_tt_id:20!null t_is_cash:21!null t_s_symb:22!null t_qty:23!null t_bid_price:24 t_ca_id:25!null t_exec_name:26!null t_trade_price:27 t_chrg:28!null t_comm:29!null trade.t_tax:30!null t_lifo:31!null + │ ├── constraint: /17: [/0 - /0] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(16-30) + │ └── fd: ()-->(17-31) └── projections - ├── t_qty:22 > 0 [as=check1:33, outer=(22)] - ├── t_bid_price:23 > 0 [as=check2:34, outer=(23), immutable] - ├── t_chrg:27 >= 0 [as=check3:35, outer=(27), immutable] - ├── t_comm:28 >= 0 [as=check4:36, outer=(28), immutable] - ├── true [as=check5:37] - └── 0 [as=t_tax:32] + ├── t_qty:23 > 0 [as=check1:35, outer=(23)] + ├── t_bid_price:24 > 0 [as=check2:36, outer=(24), immutable] + ├── t_chrg:28 >= 0 [as=check3:37, outer=(28), immutable] + ├── t_comm:29 >= 0 [as=check4:38, outer=(29), immutable] + ├── true [as=check5:39] + └── 0 [as=t_tax:34] # Q19 opt @@ -2749,11 +2749,11 @@ SELECT cr_rate::FLOAT8 LIMIT 1 ---- project - ├── columns: cr_rate:7!null + ├── columns: cr_rate:8!null ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(7) + ├── fd: ()-->(8) ├── limit │ ├── columns: cr_c_tier:1!null cr_tt_id:2!null cr_ex_id:3!null cr_from_qty:4!null cr_to_qty:5!null commission_rate.cr_rate:6!null │ ├── cardinality: [0 - 1] @@ -2774,7 +2774,7 @@ project │ │ └── cr_to_qty:5 >= 10 [outer=(5), constraints=(/5: [/10 - ]; tight)] │ └── 1 └── projections - └── commission_rate.cr_rate:6::FLOAT8 [as=cr_rate:7, outer=(6), immutable] + └── commission_rate.cr_rate:6::FLOAT8 [as=cr_rate:8, outer=(6), immutable] # Q22 opt @@ -2787,54 +2787,54 @@ UPDATE trade ---- update trade ├── columns: - ├── fetch columns: t_id:16 t_dts:17 t_st_id:18 t_tt_id:19 t_is_cash:20 t_s_symb:21 t_qty:22 t_bid_price:23 t_ca_id:24 t_exec_name:25 trade.t_trade_price:26 t_chrg:27 trade.t_comm:28 t_tax:29 t_lifo:30 + ├── fetch columns: t_id:17 t_dts:18 t_st_id:19 t_tt_id:20 t_is_cash:21 t_s_symb:22 t_qty:23 t_bid_price:24 t_ca_id:25 t_exec_name:26 trade.t_trade_price:27 t_chrg:28 trade.t_comm:29 t_tax:30 t_lifo:31 ├── update-mapping: - │ ├── t_dts_new:32 => t_dts:2 - │ ├── t_st_id_new:33 => t_st_id:3 - │ ├── t_trade_price:35 => trade.t_trade_price:11 - │ └── t_comm:36 => trade.t_comm:13 - ├── check columns: check1:37 check2:38 check3:39 check4:40 check5:41 + │ ├── t_dts_new:34 => t_dts:2 + │ ├── t_st_id_new:35 => t_st_id:3 + │ ├── t_trade_price:37 => trade.t_trade_price:11 + │ └── t_comm:38 => trade.t_comm:13 + ├── check columns: check1:39 check2:40 check3:41 check4:42 check5:43 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── project - │ ├── columns: check1:37!null check2:38 check3:39!null check4:40!null check5:41!null t_comm:36!null t_trade_price:35!null t_dts_new:32!null t_st_id_new:33!null t_id:16!null t_dts:17!null t_st_id:18!null t_tt_id:19!null t_is_cash:20!null t_s_symb:21!null t_qty:22!null t_bid_price:23 t_ca_id:24!null t_exec_name:25!null trade.t_trade_price:26 t_chrg:27!null trade.t_comm:28!null t_tax:29!null t_lifo:30!null + │ ├── columns: check1:39!null check2:40 check3:41!null check4:42!null check5:43!null t_comm:38!null t_trade_price:37!null t_dts_new:34!null t_st_id_new:35!null t_id:17!null t_dts:18!null t_st_id:19!null t_tt_id:20!null t_is_cash:21!null t_s_symb:22!null t_qty:23!null t_bid_price:24 t_ca_id:25!null t_exec_name:26!null trade.t_trade_price:27 t_chrg:28!null trade.t_comm:29!null t_tax:30!null t_lifo:31!null │ ├── cardinality: [0 - 1] │ ├── immutable │ ├── key: () - │ ├── fd: ()-->(16-30,32,33,35-41) + │ ├── fd: ()-->(17-31,34,35,37-43) │ ├── scan trade - │ │ ├── columns: t_id:16!null t_dts:17!null t_st_id:18!null t_tt_id:19!null t_is_cash:20!null t_s_symb:21!null t_qty:22!null t_bid_price:23 t_ca_id:24!null t_exec_name:25!null trade.t_trade_price:26 t_chrg:27!null trade.t_comm:28!null t_tax:29!null t_lifo:30!null - │ │ ├── constraint: /16: [/0 - /0] + │ │ ├── columns: t_id:17!null t_dts:18!null t_st_id:19!null t_tt_id:20!null t_is_cash:21!null t_s_symb:22!null t_qty:23!null t_bid_price:24 t_ca_id:25!null t_exec_name:26!null trade.t_trade_price:27 t_chrg:28!null trade.t_comm:29!null t_tax:30!null t_lifo:31!null + │ │ ├── constraint: /17: [/0 - /0] │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(16-30) + │ │ └── fd: ()-->(17-31) │ └── projections - │ ├── t_qty:22 > 0 [as=check1:37, outer=(22)] - │ ├── t_bid_price:23 > 0 [as=check2:38, outer=(23), immutable] - │ ├── t_chrg:27 >= 0 [as=check3:39, outer=(27), immutable] - │ ├── true [as=check4:40] - │ ├── t_tax:29 >= 0 [as=check5:41, outer=(29), immutable] - │ ├── 0 [as=t_comm:36] - │ ├── 1E+2 [as=t_trade_price:35] - │ ├── '2020-06-17 22:27:42.148484+00:00' [as=t_dts_new:32] - │ └── 'SBMT' [as=t_st_id_new:33] + │ ├── t_qty:23 > 0 [as=check1:39, outer=(23)] + │ ├── t_bid_price:24 > 0 [as=check2:40, outer=(24), immutable] + │ ├── t_chrg:28 >= 0 [as=check3:41, outer=(28), immutable] + │ ├── true [as=check4:42] + │ ├── t_tax:30 >= 0 [as=check5:43, outer=(30), immutable] + │ ├── 0 [as=t_comm:38] + │ ├── 1E+2 [as=t_trade_price:37] + │ ├── '2020-06-17 22:27:42.148484+00:00' [as=t_dts_new:34] + │ └── 'SBMT' [as=t_st_id_new:35] └── f-k-checks └── f-k-checks-item: trade(t_st_id) -> status_type(st_id) └── anti-join (lookup status_type) - ├── columns: t_st_id_new:42!null - ├── key columns: [42] = [43] + ├── columns: t_st_id_new:44!null + ├── key columns: [44] = [45] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(42) + ├── fd: ()-->(44) ├── with-scan &1 - │ ├── columns: t_st_id_new:42!null + │ ├── columns: t_st_id_new:44!null │ ├── mapping: - │ │ └── t_st_id_new:33 => t_st_id_new:42 + │ │ └── t_st_id_new:35 => t_st_id_new:44 │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(42) + │ └── fd: ()-->(44) └── filters (true) # Q23 @@ -2845,50 +2845,50 @@ VALUES (0, '2020-06-17 22:27:42.148484+00:00'::TIMESTAMP, 'SBMT') insert trade_history ├── columns: ├── insert-mapping: - │ ├── column1:4 => th_t_id:1 - │ ├── column2:5 => th_dts:2 - │ └── column3:6 => th_st_id:3 + │ ├── column1:5 => th_t_id:1 + │ ├── column2:6 => th_dts:2 + │ └── column3:7 => th_st_id:3 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── values - │ ├── columns: column1:4!null column2:5!null column3:6!null + │ ├── columns: column1:5!null column2:6!null column3:7!null │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(4-6) + │ ├── fd: ()-->(5-7) │ └── (0, '2020-06-17 22:27:42.148484+00:00', 'SBMT') └── f-k-checks ├── f-k-checks-item: trade_history(th_t_id) -> trade(t_id) │ └── anti-join (lookup trade) - │ ├── columns: column1:7!null - │ ├── key columns: [7] = [8] + │ ├── columns: column1:8!null + │ ├── key columns: [8] = [9] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(7) + │ ├── fd: ()-->(8) │ ├── with-scan &1 - │ │ ├── columns: column1:7!null + │ │ ├── columns: column1:8!null │ │ ├── mapping: - │ │ │ └── column1:4 => column1:7 + │ │ │ └── column1:5 => column1:8 │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(7) + │ │ └── fd: ()-->(8) │ └── filters (true) └── f-k-checks-item: trade_history(th_st_id) -> status_type(st_id) └── anti-join (lookup status_type) - ├── columns: column3:23!null - ├── key columns: [23] = [24] + ├── columns: column3:25!null + ├── key columns: [25] = [26] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(23) + ├── fd: ()-->(25) ├── with-scan &1 - │ ├── columns: column3:23!null + │ ├── columns: column3:25!null │ ├── mapping: - │ │ └── column3:6 => column3:23 + │ │ └── column3:7 => column3:25 │ ├── cardinality: [1 - 1] │ ├── key: () - │ └── fd: ()-->(23) + │ └── fd: ()-->(25) └── filters (true) # Q24 @@ -2900,27 +2900,27 @@ UPDATE broker ---- update broker ├── columns: - ├── fetch columns: b_id:6 b_st_id:7 b_name:8 b_num_trades:9 broker.b_comm_total:10 + ├── fetch columns: b_id:7 b_st_id:8 b_name:9 b_num_trades:10 broker.b_comm_total:11 ├── update-mapping: - │ ├── b_num_trades_new:12 => b_num_trades:4 - │ └── b_comm_total:13 => broker.b_comm_total:5 + │ ├── b_num_trades_new:14 => b_num_trades:4 + │ └── b_comm_total:15 => broker.b_comm_total:5 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: b_comm_total:13 b_num_trades_new:12!null b_id:6!null b_st_id:7!null b_name:8!null b_num_trades:9!null broker.b_comm_total:10!null + ├── columns: b_comm_total:15 b_num_trades_new:14!null b_id:7!null b_st_id:8!null b_name:9!null b_num_trades:10!null broker.b_comm_total:11!null ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(6-10,12,13) + ├── fd: ()-->(7-11,14,15) ├── scan broker - │ ├── columns: b_id:6!null b_st_id:7!null b_name:8!null b_num_trades:9!null broker.b_comm_total:10!null - │ ├── constraint: /6: [/0 - /0] + │ ├── columns: b_id:7!null b_st_id:8!null b_name:9!null b_num_trades:10!null broker.b_comm_total:11!null + │ ├── constraint: /7: [/0 - /0] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(6-10) + │ └── fd: ()-->(7-11) └── projections - ├── crdb_internal.round_decimal_values(broker.b_comm_total:10::DECIMAL, 2) [as=b_comm_total:13, outer=(10), immutable] - └── b_num_trades:9 + 1 [as=b_num_trades_new:12, outer=(9), immutable] + ├── crdb_internal.round_decimal_values(broker.b_comm_total:11::DECIMAL, 2) [as=b_comm_total:15, outer=(11), immutable] + └── b_num_trades:10 + 1 [as=b_num_trades_new:14, outer=(10), immutable] # Q25 opt @@ -2930,35 +2930,35 @@ VALUES (0, 'Cash Account', '2020-06-19'::DATE, '100.00':::FLOAT8::DECIMAL) insert settlement ├── columns: ├── insert-mapping: - │ ├── column1:5 => se_t_id:1 - │ ├── column2:6 => se_cash_type:2 - │ ├── column3:7 => se_cash_due_date:3 - │ └── se_amt:9 => settlement.se_amt:4 + │ ├── column1:6 => se_t_id:1 + │ ├── column2:7 => se_cash_type:2 + │ ├── column3:8 => se_cash_due_date:3 + │ └── se_amt:10 => settlement.se_amt:4 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── values - │ ├── columns: column1:5!null column2:6!null column3:7!null se_amt:9!null + │ ├── columns: column1:6!null column2:7!null column3:8!null se_amt:10!null │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(5-7,9) + │ ├── fd: ()-->(6-8,10) │ └── (0, 'Cash Account', '2020-06-19', 1E+2) └── f-k-checks └── f-k-checks-item: settlement(se_t_id) -> trade(t_id) └── anti-join (lookup trade) - ├── columns: column1:10!null - ├── key columns: [10] = [11] + ├── columns: column1:11!null + ├── key columns: [11] = [12] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(10) + ├── fd: ()-->(11) ├── with-scan &1 - │ ├── columns: column1:10!null + │ ├── columns: column1:11!null │ ├── mapping: - │ │ └── column1:5 => column1:10 + │ │ └── column1:6 => column1:11 │ ├── cardinality: [1 - 1] │ ├── key: () - │ └── fd: ()-->(10) + │ └── fd: ()-->(11) └── filters (true) # Q26 @@ -2969,38 +2969,38 @@ opt RETURNING ca_bal::FLOAT8 ---- project - ├── columns: ca_bal:16!null + ├── columns: ca_bal:18!null ├── cardinality: [0 - 1] ├── volatile, mutations ├── key: () - ├── fd: ()-->(16) + ├── fd: ()-->(18) ├── update customer_account │ ├── columns: ca_id:1!null customer_account.ca_bal:6!null - │ ├── fetch columns: ca_id:7 ca_b_id:8 ca_c_id:9 ca_name:10 ca_tax_st:11 customer_account.ca_bal:12 + │ ├── fetch columns: ca_id:8 ca_b_id:9 ca_c_id:10 ca_name:11 ca_tax_st:12 customer_account.ca_bal:13 │ ├── update-mapping: - │ │ └── ca_bal:14 => customer_account.ca_bal:6 - │ ├── check columns: check1:15 + │ │ └── ca_bal:16 => customer_account.ca_bal:6 + │ ├── check columns: check1:17 │ ├── cardinality: [0 - 1] │ ├── volatile, mutations │ ├── key: () │ ├── fd: ()-->(1,6) │ └── project - │ ├── columns: check1:15!null ca_bal:14 ca_id:7!null ca_b_id:8!null ca_c_id:9!null ca_name:10 ca_tax_st:11!null customer_account.ca_bal:12!null + │ ├── columns: check1:17!null ca_bal:16 ca_id:8!null ca_b_id:9!null ca_c_id:10!null ca_name:11 ca_tax_st:12!null customer_account.ca_bal:13!null │ ├── cardinality: [0 - 1] │ ├── immutable │ ├── key: () - │ ├── fd: ()-->(7-12,14,15) + │ ├── fd: ()-->(8-13,16,17) │ ├── scan customer_account - │ │ ├── columns: ca_id:7!null ca_b_id:8!null ca_c_id:9!null ca_name:10 ca_tax_st:11!null customer_account.ca_bal:12!null - │ │ ├── constraint: /7: [/0 - /0] + │ │ ├── columns: ca_id:8!null ca_b_id:9!null ca_c_id:10!null ca_name:11 ca_tax_st:12!null customer_account.ca_bal:13!null + │ │ ├── constraint: /8: [/0 - /0] │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(7-12) + │ │ └── fd: ()-->(8-13) │ └── projections - │ ├── ca_tax_st:11 IN (0, 1, 2) [as=check1:15, outer=(11)] - │ └── crdb_internal.round_decimal_values(customer_account.ca_bal:12::DECIMAL + 1E+2, 2) [as=ca_bal:14, outer=(12), immutable] + │ ├── ca_tax_st:12 IN (0, 1, 2) [as=check1:17, outer=(12)] + │ └── crdb_internal.round_decimal_values(customer_account.ca_bal:13::DECIMAL + 1E+2, 2) [as=ca_bal:16, outer=(13), immutable] └── projections - └── customer_account.ca_bal:6::FLOAT8 [as=ca_bal:16, outer=(6), immutable] + └── customer_account.ca_bal:6::FLOAT8 [as=ca_bal:18, outer=(6), immutable] # Q27 opt @@ -3015,35 +3015,35 @@ VALUES ( insert cash_transaction ├── columns: ├── insert-mapping: - │ ├── column1:5 => ct_t_id:1 - │ ├── column2:6 => ct_dts:2 - │ ├── ct_amt:9 => cash_transaction.ct_amt:3 - │ └── column4:8 => ct_name:4 + │ ├── column1:6 => ct_t_id:1 + │ ├── column2:7 => ct_dts:2 + │ ├── ct_amt:10 => cash_transaction.ct_amt:3 + │ └── column4:9 => ct_name:4 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── values - │ ├── columns: column1:5!null column2:6!null column4:8!null ct_amt:9!null + │ ├── columns: column1:6!null column2:7!null column4:9!null ct_amt:10!null │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(5,6,8,9) + │ ├── fd: ()-->(6,7,9,10) │ └── (0, '2020-06-18 22:27:42.148484+00:00', 'Market Buy 10 shares of SYMB', 1E+2) └── f-k-checks └── f-k-checks-item: cash_transaction(ct_t_id) -> trade(t_id) └── anti-join (lookup trade) - ├── columns: column1:10!null - ├── key columns: [10] = [11] + ├── columns: column1:11!null + ├── key columns: [11] = [12] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(10) + ├── fd: ()-->(11) ├── with-scan &1 - │ ├── columns: column1:10!null + │ ├── columns: column1:11!null │ ├── mapping: - │ │ └── column1:5 => column1:10 + │ │ └── column1:6 => column1:11 │ ├── cardinality: [1 - 1] │ ├── key: () - │ └── fd: ()-->(10) + │ └── fd: ()-->(11) └── filters (true) # Q28 @@ -3051,11 +3051,11 @@ opt SELECT ca_bal::FLOAT8 FROM customer_account WHERE ca_id = 0 ---- project - ├── columns: ca_bal:7!null + ├── columns: ca_bal:8!null ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(7) + ├── fd: ()-->(8) ├── scan customer_account │ ├── columns: ca_id:1!null customer_account.ca_bal:6!null │ ├── constraint: /1: [/0 - /0] @@ -3063,7 +3063,7 @@ project │ ├── key: () │ └── fd: ()-->(1,6) └── projections - └── customer_account.ca_bal:6::FLOAT8 [as=ca_bal:7, outer=(6), immutable] + └── customer_account.ca_bal:6::FLOAT8 [as=ca_bal:8, outer=(6), immutable] # -------------------------------------------------- # T9 @@ -3099,43 +3099,43 @@ ORDER BY t_dts DESC LIMIT 50 ---- project - ├── columns: t_id:1!null t_dts:2!null st_name:17!null tt_name:19!null t_s_symb:6!null t_qty:7!null t_exec_name:10!null t_chrg:45!null s_name:25!null ex_name:39!null + ├── columns: t_id:1!null t_dts:2!null st_name:18!null tt_name:21!null t_s_symb:6!null t_qty:7!null t_exec_name:10!null t_chrg:50!null s_name:28!null ex_name:43!null ├── cardinality: [0 - 50] ├── immutable ├── key: (1) - ├── fd: (1)-->(2,6,7,10,17,19,25,39,45), (6)-->(25,39) + ├── fd: (1)-->(2,6,7,10,18,21,28,43,50), (6)-->(28,43) ├── ordering: -2 ├── inner-join (lookup exchange) - │ ├── columns: t_id:1!null t_dts:2!null t_st_id:3!null t_tt_id:4!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null trade.t_chrg:12!null st_id:16!null st_name:17!null tt_id:18!null tt_name:19!null s_symb:22!null s_name:25!null s_ex_id:26!null ex_id:38!null ex_name:39!null - │ ├── key columns: [26] = [38] + │ ├── columns: t_id:1!null t_dts:2!null t_st_id:3!null t_tt_id:4!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null trade.t_chrg:12!null st_id:17!null st_name:18!null tt_id:20!null tt_name:21!null s_symb:25!null s_name:28!null s_ex_id:29!null ex_id:42!null ex_name:43!null + │ ├── key columns: [29] = [42] │ ├── lookup columns are key │ ├── cardinality: [0 - 50] │ ├── key: (1) - │ ├── fd: ()-->(9), (1)-->(2-4,6,7,10,12), (16)-->(17), (3)==(16), (16)==(3), (18)-->(19), (4)==(18), (18)==(4), (22)-->(25,26), (38)-->(39), (26)==(38), (38)==(26), (6)==(22), (22)==(6) + │ ├── fd: ()-->(9), (1)-->(2-4,6,7,10,12), (17)-->(18), (3)==(17), (17)==(3), (20)-->(21), (4)==(20), (20)==(4), (25)-->(28,29), (42)-->(43), (29)==(42), (42)==(29), (6)==(25), (25)==(6) │ ├── ordering: -2 opt(9) [actual: -2] │ ├── inner-join (lookup security) - │ │ ├── columns: t_id:1!null t_dts:2!null t_st_id:3!null t_tt_id:4!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null trade.t_chrg:12!null st_id:16!null st_name:17!null tt_id:18!null tt_name:19!null s_symb:22!null s_name:25!null s_ex_id:26!null - │ │ ├── key columns: [6] = [22] + │ │ ├── columns: t_id:1!null t_dts:2!null t_st_id:3!null t_tt_id:4!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null trade.t_chrg:12!null st_id:17!null st_name:18!null tt_id:20!null tt_name:21!null s_symb:25!null s_name:28!null s_ex_id:29!null + │ │ ├── key columns: [6] = [25] │ │ ├── lookup columns are key │ │ ├── cardinality: [0 - 50] │ │ ├── key: (1) - │ │ ├── fd: ()-->(9), (16)-->(17), (18)-->(19), (1)-->(2-4,6,7,10,12), (22)-->(25,26), (6)==(22), (22)==(6), (4)==(18), (18)==(4), (3)==(16), (16)==(3) + │ │ ├── fd: ()-->(9), (17)-->(18), (20)-->(21), (1)-->(2-4,6,7,10,12), (25)-->(28,29), (6)==(25), (25)==(6), (4)==(20), (20)==(4), (3)==(17), (17)==(3) │ │ ├── ordering: -2 opt(9) [actual: -2] │ │ ├── inner-join (lookup trade_type) - │ │ │ ├── columns: t_id:1!null t_dts:2!null t_st_id:3!null t_tt_id:4!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null trade.t_chrg:12!null st_id:16!null st_name:17!null tt_id:18!null tt_name:19!null - │ │ │ ├── key columns: [4] = [18] + │ │ │ ├── columns: t_id:1!null t_dts:2!null t_st_id:3!null t_tt_id:4!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null trade.t_chrg:12!null st_id:17!null st_name:18!null tt_id:20!null tt_name:21!null + │ │ │ ├── key columns: [4] = [20] │ │ │ ├── lookup columns are key │ │ │ ├── cardinality: [0 - 50] │ │ │ ├── key: (1) - │ │ │ ├── fd: ()-->(9), (1)-->(2-4,6,7,10,12), (16)-->(17), (3)==(16), (16)==(3), (18)-->(19), (4)==(18), (18)==(4) + │ │ │ ├── fd: ()-->(9), (1)-->(2-4,6,7,10,12), (17)-->(18), (3)==(17), (17)==(3), (20)-->(21), (4)==(20), (20)==(4) │ │ │ ├── ordering: -2 opt(9) [actual: -2] │ │ │ ├── inner-join (lookup status_type) - │ │ │ │ ├── columns: t_id:1!null t_dts:2!null t_st_id:3!null t_tt_id:4!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null trade.t_chrg:12!null st_id:16!null st_name:17!null - │ │ │ │ ├── key columns: [3] = [16] + │ │ │ │ ├── columns: t_id:1!null t_dts:2!null t_st_id:3!null t_tt_id:4!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null trade.t_chrg:12!null st_id:17!null st_name:18!null + │ │ │ │ ├── key columns: [3] = [17] │ │ │ │ ├── lookup columns are key │ │ │ │ ├── cardinality: [0 - 50] │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: ()-->(9), (1)-->(2-4,6,7,10,12), (16)-->(17), (3)==(16), (16)==(3) + │ │ │ │ ├── fd: ()-->(9), (1)-->(2-4,6,7,10,12), (17)-->(18), (3)==(17), (17)==(3) │ │ │ │ ├── ordering: -2 opt(9) [actual: -2] │ │ │ │ ├── scan trade@secondary │ │ │ │ │ ├── columns: t_id:1!null t_dts:2!null t_st_id:3!null t_tt_id:4!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null trade.t_chrg:12!null @@ -3149,7 +3149,7 @@ project │ │ └── filters (true) │ └── filters (true) └── projections - └── trade.t_chrg:12::FLOAT8 [as=t_chrg:45, outer=(12), immutable] + └── trade.t_chrg:12::FLOAT8 [as=t_chrg:50, outer=(12), immutable] # Q1 Version 2 # @@ -3187,47 +3187,47 @@ INNER LOOKUP JOIN security ON s_symb = t_s_symb ORDER BY t_dts DESC ---- sort - ├── columns: t_id:1!null t_dts:2!null st_name:40!null tt_name:42!null t_s_symb:6!null t_qty:7!null t_exec_name:10!null t_chrg:45!null s_name:19!null ex_name:33!null + ├── columns: t_id:1!null t_dts:2!null st_name:43!null tt_name:46!null t_s_symb:6!null t_qty:7!null t_exec_name:10!null t_chrg:50!null s_name:20!null ex_name:35!null ├── cardinality: [0 - 50] ├── immutable ├── key: (1) - ├── fd: (1)-->(2,6,7,10,19,33,40,42,45), (6)-->(19,33) + ├── fd: (1)-->(2,6,7,10,20,35,43,46,50), (6)-->(20,35) ├── ordering: -2 └── project - ├── columns: t_chrg:45!null t_id:1!null t_dts:2!null t_s_symb:6!null t_qty:7!null t_exec_name:10!null s_name:19!null ex_name:33!null st_name:40!null tt_name:42!null + ├── columns: t_chrg:50!null t_id:1!null t_dts:2!null t_s_symb:6!null t_qty:7!null t_exec_name:10!null s_name:20!null ex_name:35!null st_name:43!null tt_name:46!null ├── cardinality: [0 - 50] ├── immutable ├── key: (1) - ├── fd: (1)-->(2,6,7,10,19,33,40,42,45), (6)-->(19,33) + ├── fd: (1)-->(2,6,7,10,20,35,43,46,50), (6)-->(20,35) ├── inner-join (hash) - │ ├── columns: t_id:1!null t_dts:2!null t_st_id:3!null t_tt_id:4!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null trade.t_chrg:12!null s_symb:16!null s_name:19!null s_ex_id:20!null ex_id:32!null ex_name:33!null st_id:39!null st_name:40!null tt_id:41!null tt_name:42!null + │ ├── columns: t_id:1!null t_dts:2!null t_st_id:3!null t_tt_id:4!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null trade.t_chrg:12!null s_symb:17!null s_name:20!null s_ex_id:21!null ex_id:34!null ex_name:35!null st_id:42!null st_name:43!null tt_id:45!null tt_name:46!null │ ├── flags: force hash join (store right side) │ ├── cardinality: [0 - 50] │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) │ ├── key: (1) - │ ├── fd: ()-->(9), (1)-->(2-4,6,7,10,12), (16)-->(19,20), (6)==(16), (16)==(6), (32)-->(33), (20)==(32), (32)==(20), (39)-->(40), (3)==(39), (39)==(3), (41)-->(42), (4)==(41), (41)==(4) + │ ├── fd: ()-->(9), (1)-->(2-4,6,7,10,12), (17)-->(20,21), (6)==(17), (17)==(6), (34)-->(35), (21)==(34), (34)==(21), (42)-->(43), (3)==(42), (42)==(3), (45)-->(46), (4)==(45), (45)==(4) │ ├── inner-join (hash) - │ │ ├── columns: t_id:1!null t_dts:2!null t_st_id:3!null t_tt_id:4!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null trade.t_chrg:12!null s_symb:16!null s_name:19!null s_ex_id:20!null ex_id:32!null ex_name:33!null st_id:39!null st_name:40!null + │ │ ├── columns: t_id:1!null t_dts:2!null t_st_id:3!null t_tt_id:4!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null trade.t_chrg:12!null s_symb:17!null s_name:20!null s_ex_id:21!null ex_id:34!null ex_name:35!null st_id:42!null st_name:43!null │ │ ├── flags: force hash join (store right side) │ │ ├── cardinality: [0 - 50] │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) │ │ ├── key: (1) - │ │ ├── fd: ()-->(9), (1)-->(2-4,6,7,10,12), (16)-->(19,20), (6)==(16), (16)==(6), (32)-->(33), (20)==(32), (32)==(20), (39)-->(40), (3)==(39), (39)==(3) + │ │ ├── fd: ()-->(9), (1)-->(2-4,6,7,10,12), (17)-->(20,21), (6)==(17), (17)==(6), (34)-->(35), (21)==(34), (34)==(21), (42)-->(43), (3)==(42), (42)==(3) │ │ ├── inner-join (hash) - │ │ │ ├── columns: t_id:1!null t_dts:2!null t_st_id:3!null t_tt_id:4!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null trade.t_chrg:12!null s_symb:16!null s_name:19!null s_ex_id:20!null ex_id:32!null ex_name:33!null + │ │ │ ├── columns: t_id:1!null t_dts:2!null t_st_id:3!null t_tt_id:4!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null trade.t_chrg:12!null s_symb:17!null s_name:20!null s_ex_id:21!null ex_id:34!null ex_name:35!null │ │ │ ├── flags: force hash join (store right side) │ │ │ ├── cardinality: [0 - 50] │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) │ │ │ ├── key: (1) - │ │ │ ├── fd: ()-->(9), (1)-->(2-4,6,7,10,12), (16)-->(19,20), (6)==(16), (16)==(6), (32)-->(33), (20)==(32), (32)==(20) + │ │ │ ├── fd: ()-->(9), (1)-->(2-4,6,7,10,12), (17)-->(20,21), (6)==(17), (17)==(6), (34)-->(35), (21)==(34), (34)==(21) │ │ │ ├── inner-join (lookup security) - │ │ │ │ ├── columns: t_id:1!null t_dts:2!null t_st_id:3!null t_tt_id:4!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null trade.t_chrg:12!null s_symb:16!null s_name:19!null s_ex_id:20!null + │ │ │ │ ├── columns: t_id:1!null t_dts:2!null t_st_id:3!null t_tt_id:4!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null trade.t_chrg:12!null s_symb:17!null s_name:20!null s_ex_id:21!null │ │ │ │ ├── flags: force lookup join (into right side) - │ │ │ │ ├── key columns: [6] = [16] + │ │ │ │ ├── key columns: [6] = [17] │ │ │ │ ├── lookup columns are key │ │ │ │ ├── cardinality: [0 - 50] │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: ()-->(9), (1)-->(2-4,6,7,10,12), (16)-->(19,20), (6)==(16), (16)==(6) + │ │ │ │ ├── fd: ()-->(9), (1)-->(2-4,6,7,10,12), (17)-->(20,21), (6)==(17), (17)==(6) │ │ │ │ ├── scan trade@secondary │ │ │ │ │ ├── columns: t_id:1!null t_dts:2!null t_st_id:3!null t_tt_id:4!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null trade.t_chrg:12!null │ │ │ │ │ ├── constraint: /9/-2/1: [/0 - /0] @@ -3236,25 +3236,25 @@ sort │ │ │ │ │ └── fd: ()-->(9), (1)-->(2-4,6,7,10,12) │ │ │ │ └── filters (true) │ │ │ ├── scan exchange - │ │ │ │ ├── columns: ex_id:32!null ex_name:33!null - │ │ │ │ ├── key: (32) - │ │ │ │ └── fd: (32)-->(33) + │ │ │ │ ├── columns: ex_id:34!null ex_name:35!null + │ │ │ │ ├── key: (34) + │ │ │ │ └── fd: (34)-->(35) │ │ │ └── filters - │ │ │ └── ex_id:32 = s_ex_id:20 [outer=(20,32), constraints=(/20: (/NULL - ]; /32: (/NULL - ]), fd=(20)==(32), (32)==(20)] + │ │ │ └── ex_id:34 = s_ex_id:21 [outer=(21,34), constraints=(/21: (/NULL - ]; /34: (/NULL - ]), fd=(21)==(34), (34)==(21)] │ │ ├── scan status_type - │ │ │ ├── columns: st_id:39!null st_name:40!null - │ │ │ ├── key: (39) - │ │ │ └── fd: (39)-->(40) + │ │ │ ├── columns: st_id:42!null st_name:43!null + │ │ │ ├── key: (42) + │ │ │ └── fd: (42)-->(43) │ │ └── filters - │ │ └── st_id:39 = t_st_id:3 [outer=(3,39), constraints=(/3: (/NULL - ]; /39: (/NULL - ]), fd=(3)==(39), (39)==(3)] + │ │ └── st_id:42 = t_st_id:3 [outer=(3,42), constraints=(/3: (/NULL - ]; /42: (/NULL - ]), fd=(3)==(42), (42)==(3)] │ ├── scan trade_type - │ │ ├── columns: tt_id:41!null tt_name:42!null - │ │ ├── key: (41) - │ │ └── fd: (41)-->(42) + │ │ ├── columns: tt_id:45!null tt_name:46!null + │ │ ├── key: (45) + │ │ └── fd: (45)-->(46) │ └── filters - │ └── tt_id:41 = t_tt_id:4 [outer=(4,41), constraints=(/4: (/NULL - ]; /41: (/NULL - ]), fd=(4)==(41), (41)==(4)] + │ └── tt_id:45 = t_tt_id:4 [outer=(4,45), constraints=(/4: (/NULL - ]; /45: (/NULL - ]), fd=(4)==(45), (45)==(4)] └── projections - └── trade.t_chrg:12::FLOAT8 [as=t_chrg:45, outer=(12), immutable] + └── trade.t_chrg:12::FLOAT8 [as=t_chrg:50, outer=(12), immutable] # Q2 opt @@ -3263,24 +3263,24 @@ SELECT c_l_name, c_f_name, b_name WHERE ca_id = 0 AND c_id = ca_c_id AND b_id = ca_b_id ---- project - ├── columns: c_l_name:10!null c_f_name:11!null b_name:33!null + ├── columns: c_l_name:11!null c_f_name:12!null b_name:35!null ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(10,11,33) + ├── fd: ()-->(11,12,35) └── inner-join (lookup broker) - ├── columns: ca_id:1!null ca_b_id:2!null ca_c_id:3!null c_id:7!null c_l_name:10!null c_f_name:11!null b_id:31!null b_name:33!null - ├── key columns: [2] = [31] + ├── columns: ca_id:1!null ca_b_id:2!null ca_c_id:3!null c_id:8!null c_l_name:11!null c_f_name:12!null b_id:33!null b_name:35!null + ├── key columns: [2] = [33] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(1-3,7,10,11,31,33) + ├── fd: ()-->(1-3,8,11,12,33,35) ├── inner-join (lookup customer) - │ ├── columns: ca_id:1!null ca_b_id:2!null ca_c_id:3!null c_id:7!null c_l_name:10!null c_f_name:11!null - │ ├── key columns: [3] = [7] + │ ├── columns: ca_id:1!null ca_b_id:2!null ca_c_id:3!null c_id:8!null c_l_name:11!null c_f_name:12!null + │ ├── key columns: [3] = [8] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(1-3,7,10,11) + │ ├── fd: ()-->(1-3,8,11,12) │ ├── scan customer_account │ │ ├── columns: ca_id:1!null ca_b_id:2!null ca_c_id:3!null │ │ ├── constraint: /1: [/0 - /0] @@ -3331,31 +3331,31 @@ UPDATE trade ---- update trade ├── columns: - ├── fetch columns: t_id:16 t_dts:17 t_st_id:18 t_tt_id:19 t_is_cash:20 t_s_symb:21 t_qty:22 t_bid_price:23 t_ca_id:24 t_exec_name:25 t_trade_price:26 t_chrg:27 t_comm:28 t_tax:29 t_lifo:30 + ├── fetch columns: t_id:17 t_dts:18 t_st_id:19 t_tt_id:20 t_is_cash:21 t_s_symb:22 t_qty:23 t_bid_price:24 t_ca_id:25 t_exec_name:26 t_trade_price:27 t_chrg:28 t_comm:29 t_tax:30 t_lifo:31 ├── update-mapping: - │ └── t_exec_name_new:31 => t_exec_name:10 - ├── check columns: check1:32 check2:33 check3:34 check4:35 check5:36 + │ └── t_exec_name_new:33 => t_exec_name:10 + ├── check columns: check1:34 check2:35 check3:36 check4:37 check5:38 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: check1:32!null check2:33 check3:34!null check4:35!null check5:36!null t_exec_name_new:31 t_id:16!null t_dts:17!null t_st_id:18!null t_tt_id:19!null t_is_cash:20!null t_s_symb:21!null t_qty:22!null t_bid_price:23 t_ca_id:24!null t_exec_name:25!null t_trade_price:26 t_chrg:27!null t_comm:28!null t_tax:29!null t_lifo:30!null + ├── columns: check1:34!null check2:35 check3:36!null check4:37!null check5:38!null t_exec_name_new:33 t_id:17!null t_dts:18!null t_st_id:19!null t_tt_id:20!null t_is_cash:21!null t_s_symb:22!null t_qty:23!null t_bid_price:24 t_ca_id:25!null t_exec_name:26!null t_trade_price:27 t_chrg:28!null t_comm:29!null t_tax:30!null t_lifo:31!null ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(16-36) + ├── fd: ()-->(17-31,33-38) ├── scan trade - │ ├── columns: t_id:16!null t_dts:17!null t_st_id:18!null t_tt_id:19!null t_is_cash:20!null t_s_symb:21!null t_qty:22!null t_bid_price:23 t_ca_id:24!null t_exec_name:25!null t_trade_price:26 t_chrg:27!null t_comm:28!null t_tax:29!null t_lifo:30!null - │ ├── constraint: /16: [/0 - /0] + │ ├── columns: t_id:17!null t_dts:18!null t_st_id:19!null t_tt_id:20!null t_is_cash:21!null t_s_symb:22!null t_qty:23!null t_bid_price:24 t_ca_id:25!null t_exec_name:26!null t_trade_price:27 t_chrg:28!null t_comm:29!null t_tax:30!null t_lifo:31!null + │ ├── constraint: /17: [/0 - /0] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(16-30) + │ └── fd: ()-->(17-31) └── projections - ├── t_qty:22 > 0 [as=check1:32, outer=(22)] - ├── t_bid_price:23 > 0 [as=check2:33, outer=(23), immutable] - ├── t_chrg:27 >= 0 [as=check3:34, outer=(27), immutable] - ├── t_comm:28 >= 0 [as=check4:35, outer=(28), immutable] - ├── t_tax:29 >= 0 [as=check5:36, outer=(29), immutable] - └── CASE t_exec_name:25 LIKE '% X %' WHEN true THEN replace(t_exec_name:25, ' X ', ' ') ELSE replace(t_exec_name:25, ' ', ' X ') END [as=t_exec_name_new:31, outer=(25), immutable] + ├── t_qty:23 > 0 [as=check1:34, outer=(23)] + ├── t_bid_price:24 > 0 [as=check2:35, outer=(24), immutable] + ├── t_chrg:28 >= 0 [as=check3:36, outer=(28), immutable] + ├── t_comm:29 >= 0 [as=check4:37, outer=(29), immutable] + ├── t_tax:30 >= 0 [as=check5:38, outer=(30), immutable] + └── CASE t_exec_name:26 LIKE '% X %' WHEN true THEN replace(t_exec_name:26, ' X ', ' ') ELSE replace(t_exec_name:26, ' ', ' X ') END [as=t_exec_name_new:33, outer=(26), immutable] # Q3 opt @@ -3368,18 +3368,18 @@ SELECT t_bid_price::FLOAT8, WHERE t_id = 0 AND t_tt_id = tt_id ---- project - ├── columns: t_bid_price:20 t_exec_name:10!null t_is_cash:5!null tt_is_mrkt:19!null t_trade_price:21 + ├── columns: t_bid_price:22 t_exec_name:10!null t_is_cash:5!null tt_is_mrkt:20!null t_trade_price:23 ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(5,10,19-21) + ├── fd: ()-->(5,10,20,22,23) ├── inner-join (lookup trade_type) - │ ├── columns: t_id:1!null t_tt_id:4!null t_is_cash:5!null trade.t_bid_price:8 t_exec_name:10!null trade.t_trade_price:11 tt_id:16!null tt_is_mrkt:19!null - │ ├── key columns: [4] = [16] + │ ├── columns: t_id:1!null t_tt_id:4!null t_is_cash:5!null trade.t_bid_price:8 t_exec_name:10!null trade.t_trade_price:11 tt_id:17!null tt_is_mrkt:20!null + │ ├── key columns: [4] = [17] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(1,4,5,8,10,11,16,19) + │ ├── fd: ()-->(1,4,5,8,10,11,17,20) │ ├── scan trade │ │ ├── columns: t_id:1!null t_tt_id:4!null t_is_cash:5!null trade.t_bid_price:8 t_exec_name:10!null trade.t_trade_price:11 │ │ ├── constraint: /1: [/0 - /0] @@ -3388,8 +3388,8 @@ project │ │ └── fd: ()-->(1,4,5,8,10,11) │ └── filters (true) └── projections - ├── trade.t_bid_price:8::FLOAT8 [as=t_bid_price:20, outer=(8), immutable] - └── trade.t_trade_price:11::FLOAT8 [as=t_trade_price:21, outer=(11), immutable] + ├── trade.t_bid_price:8::FLOAT8 [as=t_bid_price:22, outer=(8), immutable] + └── trade.t_trade_price:11::FLOAT8 [as=t_trade_price:23, outer=(11), immutable] # Q4 opt @@ -3398,11 +3398,11 @@ SELECT se_amt::FLOAT8, se_cash_due_date, se_cash_type WHERE se_t_id = 0 ---- project - ├── columns: se_amt:5!null se_cash_due_date:3!null se_cash_type:2!null + ├── columns: se_amt:6!null se_cash_due_date:3!null se_cash_type:2!null ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(2,3,5) + ├── fd: ()-->(2,3,6) ├── scan settlement │ ├── columns: se_t_id:1!null se_cash_type:2!null se_cash_due_date:3!null settlement.se_amt:4!null │ ├── constraint: /1: [/0 - /0] @@ -3410,18 +3410,18 @@ project │ ├── key: () │ └── fd: ()-->(1-4) └── projections - └── settlement.se_amt:4::FLOAT8 [as=se_amt:5, outer=(4), immutable] + └── settlement.se_amt:4::FLOAT8 [as=se_amt:6, outer=(4), immutable] # Q5 opt SELECT ct_amt::FLOAT8, ct_dts, ct_name FROM cash_transaction WHERE ct_t_id = 0 ---- project - ├── columns: ct_amt:5!null ct_dts:2!null ct_name:4 + ├── columns: ct_amt:6!null ct_dts:2!null ct_name:4 ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(2,4,5) + ├── fd: ()-->(2,4,6) ├── scan cash_transaction │ ├── columns: ct_t_id:1!null ct_dts:2!null cash_transaction.ct_amt:3!null ct_name:4 │ ├── constraint: /1: [/0 - /0] @@ -3429,7 +3429,7 @@ project │ ├── key: () │ └── fd: ()-->(1-4) └── projections - └── cash_transaction.ct_amt:3::FLOAT8 [as=ct_amt:5, outer=(3), immutable] + └── cash_transaction.ct_amt:3::FLOAT8 [as=ct_amt:6, outer=(3), immutable] # Q6 opt @@ -3480,11 +3480,11 @@ ORDER BY t_dts ASC LIMIT 20 ---- project - ├── columns: t_id:1!null t_bid_price:16 t_exec_name:10!null t_is_cash:5!null t_trade_price:17 [hidden: t_dts:2!null] + ├── columns: t_id:1!null t_bid_price:17 t_exec_name:10!null t_is_cash:5!null t_trade_price:18 [hidden: t_dts:2!null] ├── cardinality: [0 - 20] ├── immutable ├── key: (1) - ├── fd: (1)-->(2,5,10,16,17) + ├── fd: (1)-->(2,5,10,17,18) ├── ordering: +2 ├── scan trade@secondary,rev │ ├── columns: t_id:1!null t_dts:2!null t_is_cash:5!null trade.t_bid_price:8 t_ca_id:9!null t_exec_name:10!null trade.t_trade_price:11 @@ -3494,8 +3494,8 @@ project │ ├── fd: ()-->(9), (1)-->(2,5,8,10,11) │ └── ordering: +2 opt(9) [actual: +2] └── projections - ├── trade.t_bid_price:8::FLOAT8 [as=t_bid_price:16, outer=(8), immutable] - └── trade.t_trade_price:11::FLOAT8 [as=t_trade_price:17, outer=(11), immutable] + ├── trade.t_bid_price:8::FLOAT8 [as=t_bid_price:17, outer=(8), immutable] + └── trade.t_trade_price:11::FLOAT8 [as=t_trade_price:18, outer=(11), immutable] # Q8 opt @@ -3517,24 +3517,24 @@ UPDATE settlement ---- update settlement ├── columns: - ├── fetch columns: se_t_id:5 se_cash_type:6 se_cash_due_date:7 se_amt:8 + ├── fetch columns: se_t_id:6 se_cash_type:7 se_cash_due_date:8 se_amt:9 ├── update-mapping: - │ └── se_cash_type_new:9 => se_cash_type:2 + │ └── se_cash_type_new:11 => se_cash_type:2 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: se_cash_type_new:9!null se_t_id:5!null se_cash_type:6!null se_cash_due_date:7!null se_amt:8!null + ├── columns: se_cash_type_new:11!null se_t_id:6!null se_cash_type:7!null se_cash_due_date:8!null se_amt:9!null ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(5-9) + ├── fd: ()-->(6-9,11) ├── scan settlement - │ ├── columns: se_t_id:5!null se_cash_type:6!null se_cash_due_date:7!null se_amt:8!null - │ ├── constraint: /5: [/0 - /0] + │ ├── columns: se_t_id:6!null se_cash_type:7!null se_cash_due_date:8!null se_amt:9!null + │ ├── constraint: /6: [/0 - /0] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(5-8) + │ └── fd: ()-->(6-9) └── projections - └── CASE se_cash_type:6 = 'Cash Account' WHEN true THEN 'Cash' ELSE 'Cash Account' END [as=se_cash_type_new:9, outer=(6)] + └── CASE se_cash_type:7 = 'Cash Account' WHEN true THEN 'Cash' ELSE 'Cash Account' END [as=se_cash_type_new:11, outer=(7)] # Q9 @@ -3598,36 +3598,36 @@ ORDER BY t_dts ASC LIMIT 20 ---- project - ├── columns: t_id:1!null t_ca_id:9!null t_exec_name:10!null t_is_cash:5!null t_trade_price:36 t_qty:7!null s_name:23!null t_dts:2!null t_tt_id:4!null tt_name:17!null + ├── columns: t_id:1!null t_ca_id:9!null t_exec_name:10!null t_is_cash:5!null t_trade_price:39 t_qty:7!null s_name:25!null t_dts:2!null t_tt_id:4!null tt_name:18!null ├── cardinality: [0 - 20] ├── immutable ├── key: (1) - ├── fd: ()-->(23), (1)-->(2,4,5,7,9,10,17,36), (4)-->(17) - ├── ordering: +2 opt(23) [actual: +2] + ├── fd: ()-->(25), (1)-->(2,4,5,7,9,10,18,39), (4)-->(18) + ├── ordering: +2 opt(25) [actual: +2] ├── limit - │ ├── columns: t_id:1!null t_dts:2!null t_tt_id:4!null t_is_cash:5!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null trade.t_trade_price:11 tt_id:16!null tt_name:17!null s_symb:20!null s_name:23!null - │ ├── internal-ordering: +2 opt(6,20,23) + │ ├── columns: t_id:1!null t_dts:2!null t_tt_id:4!null t_is_cash:5!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null trade.t_trade_price:11 tt_id:17!null tt_name:18!null s_symb:22!null s_name:25!null + │ ├── internal-ordering: +2 opt(6,22,25) │ ├── cardinality: [0 - 20] │ ├── key: (1) - │ ├── fd: ()-->(6,20,23), (1)-->(2,4,5,7,9-11), (16)-->(17), (4)==(16), (16)==(4) - │ ├── ordering: +2 opt(6,20,23) [actual: +2] + │ ├── fd: ()-->(6,22,25), (1)-->(2,4,5,7,9-11), (17)-->(18), (4)==(17), (17)==(4) + │ ├── ordering: +2 opt(6,22,25) [actual: +2] │ ├── sort - │ │ ├── columns: t_id:1!null t_dts:2!null t_tt_id:4!null t_is_cash:5!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null trade.t_trade_price:11 tt_id:16!null tt_name:17!null s_symb:20!null s_name:23!null + │ │ ├── columns: t_id:1!null t_dts:2!null t_tt_id:4!null t_is_cash:5!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null trade.t_trade_price:11 tt_id:17!null tt_name:18!null s_symb:22!null s_name:25!null │ │ ├── key: (1) - │ │ ├── fd: ()-->(6,20,23), (1)-->(2,4,5,7,9-11), (16)-->(17), (4)==(16), (16)==(4) - │ │ ├── ordering: +2 opt(6,20,23) [actual: +2] + │ │ ├── fd: ()-->(6,22,25), (1)-->(2,4,5,7,9-11), (17)-->(18), (4)==(17), (17)==(4) + │ │ ├── ordering: +2 opt(6,22,25) [actual: +2] │ │ ├── limit hint: 20.00 │ │ └── inner-join (cross) - │ │ ├── columns: t_id:1!null t_dts:2!null t_tt_id:4!null t_is_cash:5!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null trade.t_trade_price:11 tt_id:16!null tt_name:17!null s_symb:20!null s_name:23!null + │ │ ├── columns: t_id:1!null t_dts:2!null t_tt_id:4!null t_is_cash:5!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null trade.t_trade_price:11 tt_id:17!null tt_name:18!null s_symb:22!null s_name:25!null │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) │ │ ├── key: (1) - │ │ ├── fd: ()-->(6,20,23), (1)-->(2,4,5,7,9-11), (16)-->(17), (4)==(16), (16)==(4) + │ │ ├── fd: ()-->(6,22,25), (1)-->(2,4,5,7,9-11), (17)-->(18), (4)==(17), (17)==(4) │ │ ├── inner-join (lookup trade_type) - │ │ │ ├── columns: t_id:1!null t_dts:2!null t_tt_id:4!null t_is_cash:5!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null trade.t_trade_price:11 tt_id:16!null tt_name:17!null - │ │ │ ├── key columns: [4] = [16] + │ │ │ ├── columns: t_id:1!null t_dts:2!null t_tt_id:4!null t_is_cash:5!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null trade.t_trade_price:11 tt_id:17!null tt_name:18!null + │ │ │ ├── key columns: [4] = [17] │ │ │ ├── lookup columns are key │ │ │ ├── key: (1) - │ │ │ ├── fd: ()-->(6), (1)-->(2,4,5,7,9-11), (16)-->(17), (4)==(16), (16)==(4) + │ │ │ ├── fd: ()-->(6), (1)-->(2,4,5,7,9-11), (17)-->(18), (4)==(17), (17)==(4) │ │ │ ├── scan trade@secondary │ │ │ │ ├── columns: t_id:1!null t_dts:2!null t_tt_id:4!null t_is_cash:5!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null trade.t_trade_price:11 │ │ │ │ ├── constraint: /6/2/1: [/'SYMB'/'2020-06-15 22:27:42.148484+00:00' - /'SYMB'/'2020-06-17 22:27:42.148484+00:00'] @@ -3635,15 +3635,15 @@ project │ │ │ │ └── fd: ()-->(6), (1)-->(2,4,5,7,9-11) │ │ │ └── filters (true) │ │ ├── scan security - │ │ │ ├── columns: s_symb:20!null s_name:23!null - │ │ │ ├── constraint: /20: [/'SYMB' - /'SYMB'] + │ │ │ ├── columns: s_symb:22!null s_name:25!null + │ │ │ ├── constraint: /22: [/'SYMB' - /'SYMB'] │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ └── fd: ()-->(20,23) + │ │ │ └── fd: ()-->(22,25) │ │ └── filters (true) │ └── 20 └── projections - └── trade.t_trade_price:11::FLOAT8 [as=t_trade_price:36, outer=(11), immutable] + └── trade.t_trade_price:11::FLOAT8 [as=t_trade_price:39, outer=(11), immutable] # Q10 Version 2 # @@ -3682,41 +3682,41 @@ INNER LOOKUP JOIN security ON s_symb = t_s_symb ORDER BY t_dts ASC ---- sort - ├── columns: t_id:1!null t_ca_id:9!null t_exec_name:10!null t_is_cash:5!null t_trade_price:16 t_qty:7!null s_name:20!null t_dts:2!null t_tt_id:4!null tt_name:34!null + ├── columns: t_id:1!null t_ca_id:9!null t_exec_name:10!null t_is_cash:5!null t_trade_price:17 t_qty:7!null s_name:21!null t_dts:2!null t_tt_id:4!null tt_name:36!null ├── cardinality: [0 - 20] ├── immutable ├── key: (1) - ├── fd: ()-->(20), (1)-->(2,4,5,7,9,10,16), (4)-->(34) - ├── ordering: +2 opt(20) [actual: +2] + ├── fd: ()-->(21), (1)-->(2,4,5,7,9,10,17), (4)-->(36) + ├── ordering: +2 opt(21) [actual: +2] └── project - ├── columns: t_id:1!null t_dts:2!null t_tt_id:4!null t_is_cash:5!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null t_trade_price:16 s_name:20!null tt_name:34!null + ├── columns: t_id:1!null t_dts:2!null t_tt_id:4!null t_is_cash:5!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null t_trade_price:17 s_name:21!null tt_name:36!null ├── cardinality: [0 - 20] ├── immutable ├── key: (1) - ├── fd: ()-->(20), (1)-->(2,4,5,7,9,10,16), (4)-->(34) + ├── fd: ()-->(21), (1)-->(2,4,5,7,9,10,17), (4)-->(36) └── inner-join (hash) - ├── columns: t_id:1!null t_dts:2!null t_tt_id:4!null t_is_cash:5!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null t_trade_price:16 s_symb:17!null s_name:20!null tt_id:33!null tt_name:34!null + ├── columns: t_id:1!null t_dts:2!null t_tt_id:4!null t_is_cash:5!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null t_trade_price:17 s_symb:18!null s_name:21!null tt_id:35!null tt_name:36!null ├── flags: force hash join (store right side) ├── cardinality: [0 - 20] ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) ├── immutable ├── key: (1) - ├── fd: ()-->(6,17,20), (1)-->(2,4,5,7,9,10,16), (6)==(17), (17)==(6), (33)-->(34), (4)==(33), (33)==(4) + ├── fd: ()-->(6,18,21), (1)-->(2,4,5,7,9,10,17), (6)==(18), (18)==(6), (35)-->(36), (4)==(35), (35)==(4) ├── inner-join (lookup security) - │ ├── columns: t_id:1!null t_dts:2!null t_tt_id:4!null t_is_cash:5!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null t_trade_price:16 s_symb:17!null s_name:20!null + │ ├── columns: t_id:1!null t_dts:2!null t_tt_id:4!null t_is_cash:5!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null t_trade_price:17 s_symb:18!null s_name:21!null │ ├── flags: force lookup join (into right side) - │ ├── key columns: [6] = [17] + │ ├── key columns: [6] = [18] │ ├── lookup columns are key │ ├── cardinality: [0 - 20] │ ├── immutable │ ├── key: (1) - │ ├── fd: ()-->(6,17,20), (1)-->(2,4,5,7,9,10,16), (6)==(17), (17)==(6) + │ ├── fd: ()-->(6,18,21), (1)-->(2,4,5,7,9,10,17), (6)==(18), (18)==(6) │ ├── project - │ │ ├── columns: t_trade_price:16 t_id:1!null t_dts:2!null t_tt_id:4!null t_is_cash:5!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null + │ │ ├── columns: t_trade_price:17 t_id:1!null t_dts:2!null t_tt_id:4!null t_is_cash:5!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null │ │ ├── cardinality: [0 - 20] │ │ ├── immutable │ │ ├── key: (1) - │ │ ├── fd: ()-->(6), (1)-->(2,4,5,7,9,10,16) + │ │ ├── fd: ()-->(6), (1)-->(2,4,5,7,9,10,17) │ │ ├── scan trade@secondary │ │ │ ├── columns: t_id:1!null t_dts:2!null t_tt_id:4!null t_is_cash:5!null t_s_symb:6!null t_qty:7!null t_ca_id:9!null t_exec_name:10!null trade.t_trade_price:11 │ │ │ ├── constraint: /6/2/1: [/'SYMB'/'2020-06-15 22:27:42.148484+00:00' - /'SYMB'/'2020-06-17 22:27:42.148484+00:00'] @@ -3724,14 +3724,14 @@ sort │ │ │ ├── key: (1) │ │ │ └── fd: ()-->(6), (1)-->(2,4,5,7,9-11) │ │ └── projections - │ │ └── trade.t_trade_price:11::FLOAT8 [as=t_trade_price:16, outer=(11), immutable] + │ │ └── trade.t_trade_price:11::FLOAT8 [as=t_trade_price:17, outer=(11), immutable] │ └── filters (true) ├── scan trade_type - │ ├── columns: tt_id:33!null tt_name:34!null - │ ├── key: (33) - │ └── fd: (33)-->(34) + │ ├── columns: tt_id:35!null tt_name:36!null + │ ├── key: (35) + │ └── fd: (35)-->(36) └── filters - └── tt_id:33 = t_tt_id:4 [outer=(4,33), constraints=(/4: (/NULL - ]; /33: (/NULL - ]), fd=(4)==(33), (33)==(4)] + └── tt_id:35 = t_tt_id:4 [outer=(4,35), constraints=(/4: (/NULL - ]; /35: (/NULL - ]), fd=(4)==(35), (35)==(4)] # Q11 opt @@ -3747,25 +3747,25 @@ UPDATE cash_transaction ---- update cash_transaction ├── columns: - ├── fetch columns: ct_t_id:5 ct_dts:6 ct_amt:7 ct_name:8 + ├── fetch columns: ct_t_id:6 ct_dts:7 ct_amt:8 ct_name:9 ├── update-mapping: - │ └── ct_name_new:9 => ct_name:4 + │ └── ct_name_new:11 => ct_name:4 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: ct_name_new:9 ct_t_id:5!null ct_dts:6!null ct_amt:7!null ct_name:8 + ├── columns: ct_name_new:11 ct_t_id:6!null ct_dts:7!null ct_amt:8!null ct_name:9 ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(5-9) + ├── fd: ()-->(6-9,11) ├── scan cash_transaction - │ ├── columns: ct_t_id:5!null ct_dts:6!null ct_amt:7!null ct_name:8 - │ ├── constraint: /5: [/0 - /0] + │ ├── columns: ct_t_id:6!null ct_dts:7!null ct_amt:8!null ct_name:9 + │ ├── constraint: /6: [/0 - /0] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(5-8) + │ └── fd: ()-->(6-9) └── projections - └── concat_ws(' ', 'TMB', '10', CASE ct_name:8 LIKE '% shares of %' WHEN true THEN 'Shares of' ELSE 'shares of' END, 'SYMB') [as=ct_name_new:9, outer=(8), immutable] + └── concat_ws(' ', 'TMB', '10', CASE ct_name:9 LIKE '% shares of %' WHEN true THEN 'Shares of' ELSE 'shares of' END, 'SYMB') [as=ct_name_new:11, outer=(9), immutable] # -------------------------------------------------- # T11 @@ -3810,16 +3810,16 @@ UPDATE account_permission SET ap_acl = '0011' WHERE ap_ca_id = 0 AND ap_acl = NU ---- update account_permission ├── columns: - ├── fetch columns: ap_ca_id:6 ap_acl:7 ap_tax_id:8 ap_l_name:9 ap_f_name:10 + ├── fetch columns: ap_ca_id:7 ap_acl:8 ap_tax_id:9 ap_l_name:10 ap_f_name:11 ├── update-mapping: - │ └── ap_acl_new:11 => ap_acl:2 + │ └── ap_acl_new:13 => ap_acl:2 ├── cardinality: [0 - 0] ├── volatile, mutations └── values - ├── columns: ap_ca_id:6!null ap_acl:7!null ap_tax_id:8!null ap_l_name:9!null ap_f_name:10!null ap_acl_new:11!null + ├── columns: ap_ca_id:7!null ap_acl:8!null ap_tax_id:9!null ap_l_name:10!null ap_f_name:11!null ap_acl_new:13!null ├── cardinality: [0 - 0] ├── key: () - └── fd: ()-->(6-11) + └── fd: ()-->(7-11,13) # Q3 opt @@ -3833,18 +3833,18 @@ project ├── key: () ├── fd: ()-->(1,3) └── inner-join (lookup address) - ├── columns: ad_id:1!null ad_line2:3 c_id:6!null c_ad_id:15!null - ├── key columns: [15] = [1] + ├── columns: ad_id:1!null ad_line2:3 c_id:7!null c_ad_id:16!null + ├── key columns: [16] = [1] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(1,3,6,15) + ├── fd: ()-->(1,3,7,16) ├── scan customer - │ ├── columns: c_id:6!null c_ad_id:15!null - │ ├── constraint: /6: [/0 - /0] + │ ├── columns: c_id:7!null c_ad_id:16!null + │ ├── constraint: /7: [/0 - /0] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(6,15) + │ └── fd: ()-->(7,16) └── filters (true) # Q4 @@ -3859,18 +3859,18 @@ project ├── key: () ├── fd: ()-->(1,3) └── inner-join (lookup address) - ├── columns: ad_id:1!null ad_line2:3 co_id:6!null co_ad_id:12!null - ├── key columns: [12] = [1] + ├── columns: ad_id:1!null ad_line2:3 co_id:7!null co_ad_id:13!null + ├── key columns: [13] = [1] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(1,3,6,12) + ├── fd: ()-->(1,3,7,13) ├── scan company - │ ├── columns: co_id:6!null co_ad_id:12!null - │ ├── constraint: /6: [/0 - /0] + │ ├── columns: co_id:7!null co_ad_id:13!null + │ ├── constraint: /7: [/0 - /0] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(6,12) + │ └── fd: ()-->(7,13) └── filters (true) # Q5 @@ -3879,24 +3879,24 @@ UPDATE address SET ad_line2 = '' WHERE ad_id = 0 ---- update address ├── columns: - ├── fetch columns: ad_id:6 ad_line1:7 ad_line2:8 ad_zc_code:9 ad_ctry:10 + ├── fetch columns: ad_id:7 ad_line1:8 ad_line2:9 ad_zc_code:10 ad_ctry:11 ├── update-mapping: - │ └── ad_line2_new:11 => ad_line2:3 + │ └── ad_line2_new:13 => ad_line2:3 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: ad_line2_new:11!null ad_id:6!null ad_line1:7 ad_line2:8 ad_zc_code:9!null ad_ctry:10 + ├── columns: ad_line2_new:13!null ad_id:7!null ad_line1:8 ad_line2:9 ad_zc_code:10!null ad_ctry:11 ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(6-11) + ├── fd: ()-->(7-11,13) ├── scan address - │ ├── columns: ad_id:6!null ad_line1:7 ad_line2:8 ad_zc_code:9!null ad_ctry:10 - │ ├── constraint: /6: [/0 - /0] + │ ├── columns: ad_id:7!null ad_line1:8 ad_line2:9 ad_zc_code:10!null ad_ctry:11 + │ ├── constraint: /7: [/0 - /0] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(6-10) + │ └── fd: ()-->(7-11) └── projections - └── '' [as=ad_line2_new:11] + └── '' [as=ad_line2_new:13] # Q6 opt @@ -3920,24 +3920,24 @@ UPDATE company SET co_sp_rate = 'AAA' WHERE co_id = 0 ---- update company ├── columns: - ├── fetch columns: co_id:10 co_st_id:11 co_name:12 co_in_id:13 co_sp_rate:14 co_ceo:15 co_ad_id:16 co_desc:17 co_open_date:18 + ├── fetch columns: co_id:11 co_st_id:12 co_name:13 co_in_id:14 co_sp_rate:15 co_ceo:16 co_ad_id:17 co_desc:18 co_open_date:19 ├── update-mapping: - │ └── co_sp_rate_new:19 => co_sp_rate:5 + │ └── co_sp_rate_new:21 => co_sp_rate:5 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: co_sp_rate_new:19!null co_id:10!null co_st_id:11!null co_name:12!null co_in_id:13!null co_sp_rate:14!null co_ceo:15!null co_ad_id:16!null co_desc:17!null co_open_date:18!null + ├── columns: co_sp_rate_new:21!null co_id:11!null co_st_id:12!null co_name:13!null co_in_id:14!null co_sp_rate:15!null co_ceo:16!null co_ad_id:17!null co_desc:18!null co_open_date:19!null ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(10-19) + ├── fd: ()-->(11-19,21) ├── scan company - │ ├── columns: co_id:10!null co_st_id:11!null co_name:12!null co_in_id:13!null co_sp_rate:14!null co_ceo:15!null co_ad_id:16!null co_desc:17!null co_open_date:18!null - │ ├── constraint: /10: [/0 - /0] + │ ├── columns: co_id:11!null co_st_id:12!null co_name:13!null co_in_id:14!null co_sp_rate:15!null co_ceo:16!null co_ad_id:17!null co_desc:18!null co_open_date:19!null + │ ├── constraint: /11: [/0 - /0] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(10-18) + │ └── fd: ()-->(11-19) └── projections - └── 'AAA' [as=co_sp_rate_new:19] + └── 'AAA' [as=co_sp_rate_new:21] # Q8 opt @@ -3961,26 +3961,26 @@ UPDATE customer SET c_email_2 = '' WHERE c_id = 0 ---- update customer ├── columns: - ├── fetch columns: c_id:25 c_tax_id:26 c_st_id:27 c_l_name:28 c_f_name:29 c_m_name:30 c_gndr:31 c_tier:32 c_dob:33 c_ad_id:34 c_ctry_1:35 c_area_1:36 c_local_1:37 c_ext_1:38 c_ctry_2:39 c_area_2:40 c_local_2:41 c_ext_2:42 c_ctry_3:43 c_area_3:44 c_local_3:45 c_ext_3:46 c_email_1:47 c_email_2:48 + ├── fetch columns: c_id:26 c_tax_id:27 c_st_id:28 c_l_name:29 c_f_name:30 c_m_name:31 c_gndr:32 c_tier:33 c_dob:34 c_ad_id:35 c_ctry_1:36 c_area_1:37 c_local_1:38 c_ext_1:39 c_ctry_2:40 c_area_2:41 c_local_2:42 c_ext_2:43 c_ctry_3:44 c_area_3:45 c_local_3:46 c_ext_3:47 c_email_1:48 c_email_2:49 ├── update-mapping: - │ └── c_email_2_new:49 => c_email_2:24 - ├── check columns: check1:50 + │ └── c_email_2_new:51 => c_email_2:24 + ├── check columns: check1:52 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: check1:50!null c_email_2_new:49!null c_id:25!null c_tax_id:26!null c_st_id:27!null c_l_name:28!null c_f_name:29!null c_m_name:30 c_gndr:31 c_tier:32!null c_dob:33!null c_ad_id:34!null c_ctry_1:35 c_area_1:36 c_local_1:37 c_ext_1:38 c_ctry_2:39 c_area_2:40 c_local_2:41 c_ext_2:42 c_ctry_3:43 c_area_3:44 c_local_3:45 c_ext_3:46 c_email_1:47 c_email_2:48 + ├── columns: check1:52!null c_email_2_new:51!null c_id:26!null c_tax_id:27!null c_st_id:28!null c_l_name:29!null c_f_name:30!null c_m_name:31 c_gndr:32 c_tier:33!null c_dob:34!null c_ad_id:35!null c_ctry_1:36 c_area_1:37 c_local_1:38 c_ext_1:39 c_ctry_2:40 c_area_2:41 c_local_2:42 c_ext_2:43 c_ctry_3:44 c_area_3:45 c_local_3:46 c_ext_3:47 c_email_1:48 c_email_2:49 ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(25-50) + ├── fd: ()-->(26-49,51,52) ├── scan customer - │ ├── columns: c_id:25!null c_tax_id:26!null c_st_id:27!null c_l_name:28!null c_f_name:29!null c_m_name:30 c_gndr:31 c_tier:32!null c_dob:33!null c_ad_id:34!null c_ctry_1:35 c_area_1:36 c_local_1:37 c_ext_1:38 c_ctry_2:39 c_area_2:40 c_local_2:41 c_ext_2:42 c_ctry_3:43 c_area_3:44 c_local_3:45 c_ext_3:46 c_email_1:47 c_email_2:48 - │ ├── constraint: /25: [/0 - /0] + │ ├── columns: c_id:26!null c_tax_id:27!null c_st_id:28!null c_l_name:29!null c_f_name:30!null c_m_name:31 c_gndr:32 c_tier:33!null c_dob:34!null c_ad_id:35!null c_ctry_1:36 c_area_1:37 c_local_1:38 c_ext_1:39 c_ctry_2:40 c_area_2:41 c_local_2:42 c_ext_2:43 c_ctry_3:44 c_area_3:45 c_local_3:46 c_ext_3:47 c_email_1:48 c_email_2:49 + │ ├── constraint: /26: [/0 - /0] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(25-48) + │ └── fd: ()-->(26-49) └── projections - ├── c_tier:32 IN (1, 2, 3) [as=check1:50, outer=(32)] - └── '' [as=c_email_2_new:49] + ├── c_tier:33 IN (1, 2, 3) [as=check1:52, outer=(33)] + └── '' [as=c_email_2_new:51] # Q10 opt @@ -4007,41 +4007,41 @@ UPDATE customer_taxrate ---- update customer_taxrate ├── columns: - ├── fetch columns: cx_tx_id:3 cx_c_id:4 + ├── fetch columns: cx_tx_id:4 cx_c_id:5 ├── update-mapping: - │ └── cx_tx_id_new:5 => cx_tx_id:1 + │ └── cx_tx_id_new:7 => cx_tx_id:1 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── project - │ ├── columns: cx_tx_id_new:5!null cx_tx_id:3!null cx_c_id:4!null + │ ├── columns: cx_tx_id_new:7!null cx_tx_id:4!null cx_c_id:5!null │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(3-5) + │ ├── fd: ()-->(4,5,7) │ ├── scan customer_taxrate - │ │ ├── columns: cx_tx_id:3!null cx_c_id:4!null - │ │ ├── constraint: /3/4: [/'US13'/0 - /'US13'/0] + │ │ ├── columns: cx_tx_id:4!null cx_c_id:5!null + │ │ ├── constraint: /4/5: [/'US13'/0 - /'US13'/0] │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(3,4) + │ │ └── fd: ()-->(4,5) │ └── projections - │ └── 'US12' [as=cx_tx_id_new:5] + │ └── 'US12' [as=cx_tx_id_new:7] └── f-k-checks └── f-k-checks-item: customer_taxrate(cx_tx_id) -> taxrate(tx_id) └── anti-join (lookup taxrate) - ├── columns: cx_tx_id_new:6!null - ├── key columns: [6] = [7] + ├── columns: cx_tx_id_new:8!null + ├── key columns: [8] = [9] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(6) + ├── fd: ()-->(8) ├── with-scan &1 - │ ├── columns: cx_tx_id_new:6!null + │ ├── columns: cx_tx_id_new:8!null │ ├── mapping: - │ │ └── cx_tx_id_new:5 => cx_tx_id_new:6 + │ │ └── cx_tx_id_new:7 => cx_tx_id_new:8 │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(6) + │ └── fd: ()-->(8) └── filters (true) # Q12 @@ -4052,30 +4052,30 @@ UPDATE daily_market ---- update daily_market ├── columns: - ├── fetch columns: dm_date:7 dm_s_symb:8 dm_close:9 dm_high:10 dm_low:11 dm_vol:12 + ├── fetch columns: dm_date:8 dm_s_symb:9 dm_close:10 dm_high:11 dm_low:12 dm_vol:13 ├── update-mapping: - │ └── dm_vol_new:13 => dm_vol:6 + │ └── dm_vol_new:15 => dm_vol:6 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: dm_vol_new:13!null dm_date:7!null dm_s_symb:8!null dm_close:9!null dm_high:10!null dm_low:11!null dm_vol:12!null + ├── columns: dm_vol_new:15!null dm_date:8!null dm_s_symb:9!null dm_close:10!null dm_high:11!null dm_low:12!null dm_vol:13!null ├── immutable - ├── key: (7) - ├── fd: ()-->(8), (7)-->(9-12), (12)-->(13) + ├── key: (8) + ├── fd: ()-->(9), (8)-->(10-13), (13)-->(15) ├── select - │ ├── columns: dm_date:7!null dm_s_symb:8!null dm_close:9!null dm_high:10!null dm_low:11!null dm_vol:12!null + │ ├── columns: dm_date:8!null dm_s_symb:9!null dm_close:10!null dm_high:11!null dm_low:12!null dm_vol:13!null │ ├── immutable - │ ├── key: (7) - │ ├── fd: ()-->(8), (7)-->(9-12) + │ ├── key: (8) + │ ├── fd: ()-->(9), (8)-->(10-13) │ ├── scan daily_market@secondary - │ │ ├── columns: dm_date:7!null dm_s_symb:8!null dm_close:9!null dm_high:10!null dm_low:11!null dm_vol:12!null - │ │ ├── constraint: /8/7: [/'SYMB' - /'SYMB'] - │ │ ├── key: (7) - │ │ └── fd: ()-->(8), (7)-->(9-12) + │ │ ├── columns: dm_date:8!null dm_s_symb:9!null dm_close:10!null dm_high:11!null dm_low:12!null dm_vol:13!null + │ │ ├── constraint: /9/8: [/'SYMB' - /'SYMB'] + │ │ ├── key: (8) + │ │ └── fd: ()-->(9), (8)-->(10-13) │ └── filters - │ └── substring(dm_date:7::CHAR(8), 1, 2) = 'Thursday' [outer=(7), immutable] + │ └── substring(dm_date:8::CHAR(8), 1, 2) = 'Thursday' [outer=(8), immutable] └── projections - └── dm_vol:12 + 1 [as=dm_vol_new:13, outer=(12), immutable] + └── dm_vol:13 + 1 [as=dm_vol_new:15, outer=(13), immutable] # Q13 opt @@ -4105,42 +4105,42 @@ SET ---- update exchange ├── columns: - ├── fetch columns: ex_id:8 ex_name:9 ex_num_symb:10 ex_open:11 ex_close:12 ex_desc:13 ex_ad_id:14 + ├── fetch columns: ex_id:9 ex_name:10 ex_num_symb:11 ex_open:12 ex_close:13 ex_desc:14 ex_ad_id:15 ├── update-mapping: - │ └── ex_desc_new:23 => ex_desc:6 + │ └── ex_desc_new:26 => ex_desc:6 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: ex_desc_new:23 ex_id:8!null ex_name:9!null ex_num_symb:10!null ex_open:11!null ex_close:12!null ex_desc:13 ex_ad_id:14!null + ├── columns: ex_desc_new:26 ex_id:9!null ex_name:10!null ex_num_symb:11!null ex_open:12!null ex_close:13!null ex_desc:14 ex_ad_id:15!null ├── immutable - ├── key: (8) - ├── fd: (8)-->(9-14), (13)-->(23) + ├── key: (9) + ├── fd: (9)-->(10-15), (14)-->(26) ├── scan exchange - │ ├── columns: ex_id:8!null ex_name:9!null ex_num_symb:10!null ex_open:11!null ex_close:12!null ex_desc:13 ex_ad_id:14!null - │ ├── key: (8) - │ └── fd: (8)-->(9-14) + │ ├── columns: ex_id:9!null ex_name:10!null ex_num_symb:11!null ex_open:12!null ex_close:13!null ex_desc:14 ex_ad_id:15!null + │ ├── key: (9) + │ └── fd: (9)-->(10-15) └── projections - └── case [as=ex_desc_new:23, outer=(13), immutable, subquery] + └── case [as=ex_desc_new:26, outer=(14), immutable, subquery] ├── eq │ ├── subquery │ │ └── scalar-group-by - │ │ ├── columns: count_rows:22!null + │ │ ├── columns: count_rows:25!null │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(22) + │ │ ├── fd: ()-->(25) │ │ ├── select - │ │ │ ├── columns: ex_desc:20!null + │ │ │ ├── columns: ex_desc:22!null │ │ │ ├── scan exchange - │ │ │ │ └── columns: ex_desc:20 + │ │ │ │ └── columns: ex_desc:22 │ │ │ └── filters - │ │ │ └── ex_desc:20 LIKE '%LAST UPDATED%' [outer=(20), constraints=(/20: (/NULL - ])] + │ │ │ └── ex_desc:22 LIKE '%LAST UPDATED%' [outer=(22), constraints=(/22: (/NULL - ])] │ │ └── aggregations - │ │ └── count-rows [as=count_rows:22] + │ │ └── count-rows [as=count_rows:25] │ └── 0 ├── when │ ├── true - │ └── concat(ex_desc:13::STRING, ' LAST UPDATED ', '2017-05-10 13:00:00+00:00') - └── concat(substring(ex_desc:13, 1, char_length(ex_desc:13) - 25), '2017-05-10 13:00:00+00:00') + │ └── concat(ex_desc:14::STRING, ' LAST UPDATED ', '2017-05-10 13:00:00+00:00') + └── concat(substring(ex_desc:14, 1, char_length(ex_desc:14) - 25), '2017-05-10 13:00:00+00:00') # Q14 opt @@ -4161,50 +4161,50 @@ WHERE fi_co_id = 0 ---- update financial ├── columns: - ├── fetch columns: fi_co_id:15 fi_year:16 fi_qtr:17 fi_qtr_start_date:18 fi_revenue:19 fi_net_earn:20 fi_basic_eps:21 fi_dilut_eps:22 fi_margin:23 fi_inventory:24 fi_assets:25 fi_liability:26 fi_out_basic:27 fi_out_dilut:28 + ├── fetch columns: fi_co_id:16 fi_year:17 fi_qtr:18 fi_qtr_start_date:19 fi_revenue:20 fi_net_earn:21 fi_basic_eps:22 fi_dilut_eps:23 fi_margin:24 fi_inventory:25 fi_assets:26 fi_liability:27 fi_out_basic:28 fi_out_dilut:29 ├── update-mapping: - │ └── fi_qtr_start_date_new:44 => fi_qtr_start_date:4 - ├── check columns: check1:45 + │ └── fi_qtr_start_date_new:47 => fi_qtr_start_date:4 + ├── check columns: check1:48 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: check1:45!null fi_qtr_start_date_new:44 fi_co_id:15!null fi_year:16!null fi_qtr:17!null fi_qtr_start_date:18!null fi_revenue:19!null fi_net_earn:20!null fi_basic_eps:21!null fi_dilut_eps:22!null fi_margin:23!null fi_inventory:24!null fi_assets:25!null fi_liability:26!null fi_out_basic:27!null fi_out_dilut:28!null + ├── columns: check1:48!null fi_qtr_start_date_new:47 fi_co_id:16!null fi_year:17!null fi_qtr:18!null fi_qtr_start_date:19!null fi_revenue:20!null fi_net_earn:21!null fi_basic_eps:22!null fi_dilut_eps:23!null fi_margin:24!null fi_inventory:25!null fi_assets:26!null fi_liability:27!null fi_out_basic:28!null fi_out_dilut:29!null ├── immutable - ├── key: (16,17) - ├── fd: ()-->(15), (16,17)-->(18-28), (17)-->(45), (18)-->(44) + ├── key: (17,18) + ├── fd: ()-->(16), (17,18)-->(19-29), (18)-->(48), (19)-->(47) ├── scan financial - │ ├── columns: fi_co_id:15!null fi_year:16!null fi_qtr:17!null fi_qtr_start_date:18!null fi_revenue:19!null fi_net_earn:20!null fi_basic_eps:21!null fi_dilut_eps:22!null fi_margin:23!null fi_inventory:24!null fi_assets:25!null fi_liability:26!null fi_out_basic:27!null fi_out_dilut:28!null - │ ├── constraint: /15/16/17: [/0 - /0] - │ ├── key: (16,17) - │ └── fd: ()-->(15), (16,17)-->(18-28) + │ ├── columns: fi_co_id:16!null fi_year:17!null fi_qtr:18!null fi_qtr_start_date:19!null fi_revenue:20!null fi_net_earn:21!null fi_basic_eps:22!null fi_dilut_eps:23!null fi_margin:24!null fi_inventory:25!null fi_assets:26!null fi_liability:27!null fi_out_basic:28!null fi_out_dilut:29!null + │ ├── constraint: /16/17/18: [/0 - /0] + │ ├── key: (17,18) + │ └── fd: ()-->(16), (17,18)-->(19-29) └── projections - ├── fi_qtr:17 IN (1, 2, 3, 4) [as=check1:45, outer=(17)] - └── case [as=fi_qtr_start_date_new:44, outer=(18), immutable, subquery] + ├── fi_qtr:18 IN (1, 2, 3, 4) [as=check1:48, outer=(18)] + └── case [as=fi_qtr_start_date_new:47, outer=(19), immutable, subquery] ├── gt │ ├── subquery │ │ └── scalar-group-by - │ │ ├── columns: count_rows:43!null + │ │ ├── columns: count_rows:46!null │ │ ├── cardinality: [1 - 1] │ │ ├── immutable │ │ ├── key: () - │ │ ├── fd: ()-->(43) + │ │ ├── fd: ()-->(46) │ │ ├── select - │ │ │ ├── columns: fi_co_id:29!null fi_qtr_start_date:32!null + │ │ │ ├── columns: fi_co_id:31!null fi_qtr_start_date:34!null │ │ │ ├── immutable - │ │ │ ├── fd: ()-->(29) + │ │ │ ├── fd: ()-->(31) │ │ │ ├── scan financial - │ │ │ │ ├── columns: fi_co_id:29!null fi_qtr_start_date:32!null - │ │ │ │ ├── constraint: /29/30/31: [/0 - /0] - │ │ │ │ └── fd: ()-->(29) + │ │ │ │ ├── columns: fi_co_id:31!null fi_qtr_start_date:34!null + │ │ │ │ ├── constraint: /31/32/33: [/0 - /0] + │ │ │ │ └── fd: ()-->(31) │ │ │ └── filters - │ │ │ └── substring(fi_qtr_start_date:32::CHAR(8), 7, 2) = '01' [outer=(32), immutable] + │ │ │ └── substring(fi_qtr_start_date:34::CHAR(8), 7, 2) = '01' [outer=(34), immutable] │ │ └── aggregations - │ │ └── count-rows [as=count_rows:43] + │ │ └── count-rows [as=count_rows:46] │ └── 0 ├── when │ ├── true - │ └── fi_qtr_start_date:18 + 1 - └── fi_qtr_start_date:18 - 1 + │ └── fi_qtr_start_date:19 + 1 + └── fi_qtr_start_date:19 - 1 # Q15 opt @@ -4224,44 +4224,44 @@ WHERE ---- update news_item ├── columns: - ├── fetch columns: ni_id:8 ni_headline:9 ni_summary:10 ni_dts:12 ni_source:13 ni_author:14 + ├── fetch columns: ni_id:9 ni_headline:10 ni_summary:11 ni_dts:13 ni_source:14 ni_author:15 ├── update-mapping: - │ └── ni_dts_new:17 => ni_dts:5 + │ └── ni_dts_new:20 => ni_dts:5 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: ni_dts_new:17!null ni_id:8!null ni_headline:9!null ni_summary:10!null ni_dts:12!null ni_source:13!null ni_author:14 + ├── columns: ni_dts_new:20!null ni_id:9!null ni_headline:10!null ni_summary:11!null ni_dts:13!null ni_source:14!null ni_author:15 ├── immutable - ├── key: (8) - ├── fd: (8)-->(9,10,12-14), (12)-->(17) + ├── key: (9) + ├── fd: (9)-->(10,11,13-15), (13)-->(20) ├── select - │ ├── columns: ni_id:8!null ni_headline:9!null ni_summary:10!null ni_dts:12!null ni_source:13!null ni_author:14 - │ ├── key: (8) - │ ├── fd: (8)-->(9,10,12-14) + │ ├── columns: ni_id:9!null ni_headline:10!null ni_summary:11!null ni_dts:13!null ni_source:14!null ni_author:15 + │ ├── key: (9) + │ ├── fd: (9)-->(10,11,13-15) │ ├── scan news_item - │ │ ├── columns: ni_id:8!null ni_headline:9!null ni_summary:10!null ni_dts:12!null ni_source:13!null ni_author:14 - │ │ ├── key: (8) - │ │ └── fd: (8)-->(9,10,12-14) + │ │ ├── columns: ni_id:9!null ni_headline:10!null ni_summary:11!null ni_dts:13!null ni_source:14!null ni_author:15 + │ │ ├── key: (9) + │ │ └── fd: (9)-->(10,11,13-15) │ └── filters - │ └── eq [outer=(8), subquery, constraints=(/8: (/NULL - ])] - │ ├── ni_id:8 + │ └── eq [outer=(9), subquery, constraints=(/9: (/NULL - ])] + │ ├── ni_id:9 │ └── subquery │ └── max1-row - │ ├── columns: nx_ni_id:15!null + │ ├── columns: nx_ni_id:17!null │ ├── error: "more than one row returned by a subquery used as an expression" │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(15) + │ ├── fd: ()-->(17) │ └── project - │ ├── columns: nx_ni_id:15!null - │ ├── key: (15) + │ ├── columns: nx_ni_id:17!null + │ ├── key: (17) │ └── scan news_xref@news_xref_auto_index_fk_nx_co_id_ref_company - │ ├── columns: nx_ni_id:15!null nx_co_id:16!null - │ ├── constraint: /16/15: [/0 - /0] - │ ├── key: (15) - │ └── fd: ()-->(16) + │ ├── columns: nx_ni_id:17!null nx_co_id:18!null + │ ├── constraint: /18/17: [/0 - /0] + │ ├── key: (17) + │ └── fd: ()-->(18) └── projections - └── ni_dts:12 + '1 day' [as=ni_dts_new:17, outer=(12), immutable] + └── ni_dts:13 + '1 day' [as=ni_dts_new:20, outer=(13), immutable] # Q16 opt @@ -4271,25 +4271,25 @@ UPDATE security ---- update security ├── columns: - ├── fetch columns: s_symb:17 s_issue:18 s_st_id:19 s_name:20 s_ex_id:21 s_co_id:22 s_num_out:23 s_start_date:24 s_exch_date:25 s_pe:26 s_52wk_high:27 s_52wk_high_date:28 s_52wk_low:29 s_52wk_low_date:30 s_dividend:31 s_yield:32 + ├── fetch columns: s_symb:18 s_issue:19 s_st_id:20 s_name:21 s_ex_id:22 s_co_id:23 s_num_out:24 s_start_date:25 s_exch_date:26 s_pe:27 s_52wk_high:28 s_52wk_high_date:29 s_52wk_low:30 s_52wk_low_date:31 s_dividend:32 s_yield:33 ├── update-mapping: - │ └── s_exch_date_new:33 => s_exch_date:9 + │ └── s_exch_date_new:35 => s_exch_date:9 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: s_exch_date_new:33!null s_symb:17!null s_issue:18!null s_st_id:19!null s_name:20!null s_ex_id:21!null s_co_id:22!null s_num_out:23!null s_start_date:24!null s_exch_date:25!null s_pe:26!null s_52wk_high:27!null s_52wk_high_date:28!null s_52wk_low:29!null s_52wk_low_date:30!null s_dividend:31!null s_yield:32!null + ├── columns: s_exch_date_new:35!null s_symb:18!null s_issue:19!null s_st_id:20!null s_name:21!null s_ex_id:22!null s_co_id:23!null s_num_out:24!null s_start_date:25!null s_exch_date:26!null s_pe:27!null s_52wk_high:28!null s_52wk_high_date:29!null s_52wk_low:30!null s_52wk_low_date:31!null s_dividend:32!null s_yield:33!null ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(17-33) + ├── fd: ()-->(18-33,35) ├── scan security - │ ├── columns: s_symb:17!null s_issue:18!null s_st_id:19!null s_name:20!null s_ex_id:21!null s_co_id:22!null s_num_out:23!null s_start_date:24!null s_exch_date:25!null s_pe:26!null s_52wk_high:27!null s_52wk_high_date:28!null s_52wk_low:29!null s_52wk_low_date:30!null s_dividend:31!null s_yield:32!null - │ ├── constraint: /17: [/'SYMB' - /'SYMB'] + │ ├── columns: s_symb:18!null s_issue:19!null s_st_id:20!null s_name:21!null s_ex_id:22!null s_co_id:23!null s_num_out:24!null s_start_date:25!null s_exch_date:26!null s_pe:27!null s_52wk_high:28!null s_52wk_high_date:29!null s_52wk_low:30!null s_52wk_low_date:31!null s_dividend:32!null s_yield:33!null + │ ├── constraint: /18: [/'SYMB' - /'SYMB'] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(17-32) + │ └── fd: ()-->(18-33) └── projections - └── s_exch_date:25 + 1 [as=s_exch_date_new:33, outer=(25), immutable] + └── s_exch_date:26 + 1 [as=s_exch_date_new:35, outer=(26), immutable] # Q17 opt @@ -4309,45 +4309,45 @@ WHERE tx_id = 'US12' ---- update taxrate ├── columns: - ├── fetch columns: tx_id:4 tx_name:5 tx_rate:6 + ├── fetch columns: tx_id:5 tx_name:6 tx_rate:7 ├── update-mapping: - │ └── tx_name_new:10 => tx_name:2 - ├── check columns: check1:11 + │ └── tx_name_new:13 => tx_name:2 + ├── check columns: check1:14 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: check1:11!null tx_name_new:10 tx_id:4!null tx_name:5!null tx_rate:6!null + ├── columns: check1:14!null tx_name_new:13 tx_id:5!null tx_name:6!null tx_rate:7!null ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(4-6,10,11) + ├── fd: ()-->(5-7,13,14) ├── scan taxrate - │ ├── columns: tx_id:4!null tx_name:5!null tx_rate:6!null - │ ├── constraint: /4: [/'US12' - /'US12'] + │ ├── columns: tx_id:5!null tx_name:6!null tx_rate:7!null + │ ├── constraint: /5: [/'US12' - /'US12'] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(4-6) + │ └── fd: ()-->(5-7) └── projections - ├── tx_rate:6 >= 0 [as=check1:11, outer=(6), immutable] - └── case [as=tx_name_new:10, outer=(5), immutable, subquery] + ├── tx_rate:7 >= 0 [as=check1:14, outer=(7), immutable] + └── case [as=tx_name_new:13, outer=(6), immutable, subquery] ├── like │ ├── subquery │ │ └── project - │ │ ├── columns: tx_name:8!null + │ │ ├── columns: tx_name:10!null │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(8) + │ │ ├── fd: ()-->(10) │ │ └── scan taxrate - │ │ ├── columns: tx_id:7!null tx_name:8!null - │ │ ├── constraint: /7: [/'US12' - /'US12'] + │ │ ├── columns: tx_id:9!null tx_name:10!null + │ │ ├── constraint: /9: [/'US12' - /'US12'] │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(7,8) + │ │ └── fd: ()-->(9,10) │ └── '% Tax %' ├── when │ ├── true - │ └── replace(tx_name:5, ' TAX ', ' tax ') - └── replace(tx_name:5, ' tax ', ' Tax ') + │ └── replace(tx_name:6, ' TAX ', ' tax ') + └── replace(tx_name:6, ' tax ', ' Tax ') # Q18 opt @@ -4368,44 +4368,44 @@ WHERE rownum = ( project ├── columns: wi_s_symb:2!null └── inner-join (lookup watch_item) - ├── columns: wi_wl_id:1!null wi_s_symb:2!null wl_id:3!null wl_c_id:4!null ordinality:5!null - ├── key columns: [3] = [1] - ├── key: (2,3) - ├── fd: ()-->(4), (3)-->(5), (5)-->(3), (1)==(3), (3)==(1) + ├── columns: wi_wl_id:1!null wi_s_symb:2!null wl_id:4!null wl_c_id:5!null ordinality:7!null + ├── key columns: [4] = [1] + ├── key: (2,4) + ├── fd: ()-->(5), (4)-->(7), (7)-->(4), (1)==(4), (4)==(1) ├── select - │ ├── columns: wl_id:3!null wl_c_id:4!null ordinality:5!null - │ ├── key: (3) - │ ├── fd: ()-->(4), (3)-->(5), (5)-->(3) + │ ├── columns: wl_id:4!null wl_c_id:5!null ordinality:7!null + │ ├── key: (4) + │ ├── fd: ()-->(5), (4)-->(7), (7)-->(4) │ ├── ordinality - │ │ ├── columns: wl_id:3!null wl_c_id:4!null ordinality:5!null - │ │ ├── key: (3) - │ │ ├── fd: (3)-->(4,5), (5)-->(3,4) + │ │ ├── columns: wl_id:4!null wl_c_id:5!null ordinality:7!null + │ │ ├── key: (4) + │ │ ├── fd: (4)-->(5,7), (7)-->(4,5) │ │ └── scan watch_list - │ │ ├── columns: wl_id:3!null wl_c_id:4!null - │ │ ├── key: (3) - │ │ └── fd: (3)-->(4) + │ │ ├── columns: wl_id:4!null wl_c_id:5!null + │ │ ├── key: (4) + │ │ └── fd: (4)-->(5) │ └── filters - │ ├── wl_c_id:4 = 0 [outer=(4), constraints=(/4: [/0 - /0]; tight), fd=()-->(4)] - │ └── eq [outer=(5), subquery, constraints=(/5: (/NULL - ])] - │ ├── ordinality:5 + │ ├── wl_c_id:5 = 0 [outer=(5), constraints=(/5: [/0 - /0]; tight), fd=()-->(5)] + │ └── eq [outer=(7), subquery, constraints=(/7: (/NULL - ])] + │ ├── ordinality:7 │ └── subquery │ └── scalar-group-by - │ ├── columns: count_rows:10!null + │ ├── columns: count_rows:14!null │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(10) + │ ├── fd: ()-->(14) │ ├── inner-join (lookup watch_item) - │ │ ├── columns: wi_wl_id:6!null wl_id:8!null wl_c_id:9!null - │ │ ├── key columns: [8] = [6] - │ │ ├── fd: ()-->(9), (6)==(8), (8)==(6) + │ │ ├── columns: wi_wl_id:8!null wl_id:11!null wl_c_id:12!null + │ │ ├── key columns: [11] = [8] + │ │ ├── fd: ()-->(12), (8)==(11), (11)==(8) │ │ ├── scan watch_list@watch_list_auto_index_fk_wl_c_id_ref_customer - │ │ │ ├── columns: wl_id:8!null wl_c_id:9!null - │ │ │ ├── constraint: /9/8: [/0 - /0] - │ │ │ ├── key: (8) - │ │ │ └── fd: ()-->(9) + │ │ │ ├── columns: wl_id:11!null wl_c_id:12!null + │ │ │ ├── constraint: /12/11: [/0 - /0] + │ │ │ ├── key: (11) + │ │ │ └── fd: ()-->(12) │ │ └── filters (true) │ └── aggregations - │ └── count-rows [as=count_rows:10] + │ └── count-rows [as=count_rows:14] └── filters (true) # Q19 @@ -4431,7 +4431,7 @@ limit ├── anti-join (merge) │ ├── columns: s_symb:1!null │ ├── left ordering: +1 - │ ├── right ordering: +18 + │ ├── right ordering: +19 │ ├── key: (1) │ ├── ordering: +1 │ ├── limit hint: 1.00 @@ -4441,26 +4441,26 @@ limit │ │ ├── key: (1) │ │ └── ordering: +1 │ ├── sort - │ │ ├── columns: wi_wl_id:17!null wi_s_symb:18!null wl_id:19!null wl_c_id:20!null - │ │ ├── key: (18,19) - │ │ ├── fd: ()-->(20), (17)==(19), (19)==(17) - │ │ ├── ordering: +18 opt(20) [actual: +18] + │ │ ├── columns: wi_wl_id:18!null wi_s_symb:19!null wl_id:21!null wl_c_id:22!null + │ │ ├── key: (19,21) + │ │ ├── fd: ()-->(22), (18)==(21), (21)==(18) + │ │ ├── ordering: +19 opt(22) [actual: +19] │ │ └── inner-join (hash) - │ │ ├── columns: wi_wl_id:17!null wi_s_symb:18!null wl_id:19!null wl_c_id:20!null + │ │ ├── columns: wi_wl_id:18!null wi_s_symb:19!null wl_id:21!null wl_c_id:22!null │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ ├── key: (18,19) - │ │ ├── fd: ()-->(20), (17)==(19), (19)==(17) + │ │ ├── key: (19,21) + │ │ ├── fd: ()-->(22), (18)==(21), (21)==(18) │ │ ├── scan watch_item@watch_item_auto_index_fk_wi_s_symb_ref_security - │ │ │ ├── columns: wi_wl_id:17!null wi_s_symb:18!null - │ │ │ ├── constraint: /18/17: [/e'SYMB\x00' - ] - │ │ │ └── key: (17,18) + │ │ │ ├── columns: wi_wl_id:18!null wi_s_symb:19!null + │ │ │ ├── constraint: /19/18: [/e'SYMB\x00' - ] + │ │ │ └── key: (18,19) │ │ ├── scan watch_list@watch_list_auto_index_fk_wl_c_id_ref_customer - │ │ │ ├── columns: wl_id:19!null wl_c_id:20!null - │ │ │ ├── constraint: /20/19: [/0 - /0] - │ │ │ ├── key: (19) - │ │ │ └── fd: ()-->(20) + │ │ │ ├── columns: wl_id:21!null wl_c_id:22!null + │ │ │ ├── constraint: /22/21: [/0 - /0] + │ │ │ ├── key: (21) + │ │ │ └── fd: ()-->(22) │ │ └── filters - │ │ └── wi_wl_id:17 = wl_id:19 [outer=(17,19), constraints=(/17: (/NULL - ]; /19: (/NULL - ]), fd=(17)==(19), (19)==(17)] + │ │ └── wi_wl_id:18 = wl_id:21 [outer=(18,21), constraints=(/18: (/NULL - ]; /21: (/NULL - ]), fd=(18)==(21), (21)==(18)] │ └── filters (true) └── 1 @@ -4473,49 +4473,54 @@ UPDATE watch_item ---- update watch_item ├── columns: - ├── fetch columns: wi_wl_id:3 wi_s_symb:4 + ├── fetch columns: wi_wl_id:4 wi_s_symb:5 ├── update-mapping: - │ └── wi_s_symb_new:7 => wi_s_symb:2 + │ └── wi_s_symb_new:10 => wi_s_symb:2 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── project - │ ├── columns: wi_s_symb_new:7!null wi_wl_id:3!null wi_s_symb:4!null wl_id:5!null wl_c_id:6!null - │ ├── key: (5) - │ ├── fd: ()-->(4,6,7), (3)==(5), (5)==(3) + │ ├── columns: wi_s_symb_new:10!null wi_wl_id:4!null wi_s_symb:5!null wl_id:7!null wl_c_id:8!null watch_list.crdb_internal_mvcc_timestamp:9 + │ ├── key: (7) + │ ├── fd: ()-->(5,8,10), (7)-->(9), (4)==(7), (7)==(4) │ ├── inner-join (merge) - │ │ ├── columns: wi_wl_id:3!null wi_s_symb:4!null wl_id:5!null wl_c_id:6!null - │ │ ├── left ordering: +3 - │ │ ├── right ordering: +5 - │ │ ├── key: (5) - │ │ ├── fd: ()-->(4,6), (3)==(5), (5)==(3) + │ │ ├── columns: wi_wl_id:4!null wi_s_symb:5!null wl_id:7!null wl_c_id:8!null watch_list.crdb_internal_mvcc_timestamp:9 + │ │ ├── left ordering: +4 + │ │ ├── right ordering: +7 + │ │ ├── key: (7) + │ │ ├── fd: ()-->(5,8), (7)-->(9), (4)==(7), (7)==(4) │ │ ├── scan watch_item@watch_item_auto_index_fk_wi_s_symb_ref_security - │ │ │ ├── columns: wi_wl_id:3!null wi_s_symb:4!null - │ │ │ ├── constraint: /4/3: [/'SYMB' - /'SYMB'] - │ │ │ ├── key: (3) - │ │ │ ├── fd: ()-->(4) - │ │ │ └── ordering: +3 opt(4) [actual: +3] - │ │ ├── scan watch_list@watch_list_auto_index_fk_wl_c_id_ref_customer - │ │ │ ├── columns: wl_id:5!null wl_c_id:6!null - │ │ │ ├── constraint: /6/5: [/0 - /0] - │ │ │ ├── key: (5) - │ │ │ ├── fd: ()-->(6) - │ │ │ └── ordering: +5 opt(6) [actual: +5] + │ │ │ ├── columns: wi_wl_id:4!null wi_s_symb:5!null + │ │ │ ├── constraint: /5/4: [/'SYMB' - /'SYMB'] + │ │ │ ├── key: (4) + │ │ │ ├── fd: ()-->(5) + │ │ │ └── ordering: +4 opt(5) [actual: +4] + │ │ ├── index-join watch_list + │ │ │ ├── columns: wl_id:7!null wl_c_id:8!null watch_list.crdb_internal_mvcc_timestamp:9 + │ │ │ ├── key: (7) + │ │ │ ├── fd: ()-->(8), (7)-->(9) + │ │ │ ├── ordering: +7 opt(8) [actual: +7] + │ │ │ └── scan watch_list@watch_list_auto_index_fk_wl_c_id_ref_customer + │ │ │ ├── columns: wl_id:7!null wl_c_id:8!null + │ │ │ ├── constraint: /8/7: [/0 - /0] + │ │ │ ├── key: (7) + │ │ │ ├── fd: ()-->(8) + │ │ │ └── ordering: +7 opt(8) [actual: +7] │ │ └── filters (true) │ └── projections - │ └── 'SYMB' [as=wi_s_symb_new:7] + │ └── 'SYMB' [as=wi_s_symb_new:10] └── f-k-checks └── f-k-checks-item: watch_item(wi_s_symb) -> security(s_symb) └── anti-join (lookup security) - ├── columns: wi_s_symb_new:8!null - ├── key columns: [8] = [9] + ├── columns: wi_s_symb_new:11!null + ├── key columns: [11] = [12] ├── lookup columns are key - ├── fd: ()-->(8) + ├── fd: ()-->(11) ├── with-scan &1 - │ ├── columns: wi_s_symb_new:8!null + │ ├── columns: wi_s_symb_new:11!null │ ├── mapping: - │ │ └── wi_s_symb_new:7 => wi_s_symb_new:8 - │ └── fd: ()-->(8) + │ │ └── wi_s_symb_new:10 => wi_s_symb_new:11 + │ └── fd: ()-->(11) └── filters (true) # -------------------------------------------------- @@ -4544,50 +4549,50 @@ VALUES (0, '2020-06-17 22:27:42.148484+00:00'::TIMESTAMP, 'SBMT') insert trade_history ├── columns: ├── insert-mapping: - │ ├── column1:4 => th_t_id:1 - │ ├── column2:5 => th_dts:2 - │ └── column3:6 => th_st_id:3 + │ ├── column1:5 => th_t_id:1 + │ ├── column2:6 => th_dts:2 + │ └── column3:7 => th_st_id:3 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── values - │ ├── columns: column1:4!null column2:5!null column3:6!null + │ ├── columns: column1:5!null column2:6!null column3:7!null │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(4-6) + │ ├── fd: ()-->(5-7) │ └── (0, '2020-06-17 22:27:42.148484+00:00', 'SBMT') └── f-k-checks ├── f-k-checks-item: trade_history(th_t_id) -> trade(t_id) │ └── anti-join (lookup trade) - │ ├── columns: column1:7!null - │ ├── key columns: [7] = [8] + │ ├── columns: column1:8!null + │ ├── key columns: [8] = [9] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(7) + │ ├── fd: ()-->(8) │ ├── with-scan &1 - │ │ ├── columns: column1:7!null + │ │ ├── columns: column1:8!null │ │ ├── mapping: - │ │ │ └── column1:4 => column1:7 + │ │ │ └── column1:5 => column1:8 │ │ ├── cardinality: [1 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(7) + │ │ └── fd: ()-->(8) │ └── filters (true) └── f-k-checks-item: trade_history(th_st_id) -> status_type(st_id) └── anti-join (lookup status_type) - ├── columns: column3:23!null - ├── key columns: [23] = [24] + ├── columns: column3:25!null + ├── key columns: [25] = [26] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(23) + ├── fd: ()-->(25) ├── with-scan &1 - │ ├── columns: column3:23!null + │ ├── columns: column3:25!null │ ├── mapping: - │ │ └── column3:6 => column3:23 + │ │ └── column3:7 => column3:25 │ ├── cardinality: [1 - 1] │ ├── key: () - │ └── fd: ()-->(23) + │ └── fd: ()-->(25) └── filters (true) # Q3 @@ -4598,50 +4603,50 @@ WHERE t_id = 0 ---- update trade ├── columns: - ├── fetch columns: t_id:16 t_dts:17 t_st_id:18 t_tt_id:19 t_is_cash:20 t_s_symb:21 t_qty:22 t_bid_price:23 t_ca_id:24 t_exec_name:25 t_trade_price:26 t_chrg:27 t_comm:28 t_tax:29 t_lifo:30 + ├── fetch columns: t_id:17 t_dts:18 t_st_id:19 t_tt_id:20 t_is_cash:21 t_s_symb:22 t_qty:23 t_bid_price:24 t_ca_id:25 t_exec_name:26 t_trade_price:27 t_chrg:28 t_comm:29 t_tax:30 t_lifo:31 ├── update-mapping: - │ ├── t_dts_new:32 => t_dts:2 - │ └── t_st_id_new:31 => t_st_id:3 - ├── check columns: check1:33 check2:34 check3:35 check4:36 check5:37 + │ ├── t_dts_new:34 => t_dts:2 + │ └── t_st_id_new:33 => t_st_id:3 + ├── check columns: check1:35 check2:36 check3:37 check4:38 check5:39 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── project - │ ├── columns: check1:33!null check2:34 check3:35!null check4:36!null check5:37!null t_st_id_new:31!null t_dts_new:32!null t_id:16!null t_dts:17!null t_st_id:18!null t_tt_id:19!null t_is_cash:20!null t_s_symb:21!null t_qty:22!null t_bid_price:23 t_ca_id:24!null t_exec_name:25!null t_trade_price:26 t_chrg:27!null t_comm:28!null t_tax:29!null t_lifo:30!null + │ ├── columns: check1:35!null check2:36 check3:37!null check4:38!null check5:39!null t_st_id_new:33!null t_dts_new:34!null t_id:17!null t_dts:18!null t_st_id:19!null t_tt_id:20!null t_is_cash:21!null t_s_symb:22!null t_qty:23!null t_bid_price:24 t_ca_id:25!null t_exec_name:26!null t_trade_price:27 t_chrg:28!null t_comm:29!null t_tax:30!null t_lifo:31!null │ ├── cardinality: [0 - 1] │ ├── immutable │ ├── key: () - │ ├── fd: ()-->(16-37) + │ ├── fd: ()-->(17-31,33-39) │ ├── scan trade - │ │ ├── columns: t_id:16!null t_dts:17!null t_st_id:18!null t_tt_id:19!null t_is_cash:20!null t_s_symb:21!null t_qty:22!null t_bid_price:23 t_ca_id:24!null t_exec_name:25!null t_trade_price:26 t_chrg:27!null t_comm:28!null t_tax:29!null t_lifo:30!null - │ │ ├── constraint: /16: [/0 - /0] + │ │ ├── columns: t_id:17!null t_dts:18!null t_st_id:19!null t_tt_id:20!null t_is_cash:21!null t_s_symb:22!null t_qty:23!null t_bid_price:24 t_ca_id:25!null t_exec_name:26!null t_trade_price:27 t_chrg:28!null t_comm:29!null t_tax:30!null t_lifo:31!null + │ │ ├── constraint: /17: [/0 - /0] │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(16-30) + │ │ └── fd: ()-->(17-31) │ └── projections - │ ├── t_qty:22 > 0 [as=check1:33, outer=(22)] - │ ├── t_bid_price:23 > 0 [as=check2:34, outer=(23), immutable] - │ ├── t_chrg:27 >= 0 [as=check3:35, outer=(27), immutable] - │ ├── t_comm:28 >= 0 [as=check4:36, outer=(28), immutable] - │ ├── t_tax:29 >= 0 [as=check5:37, outer=(29), immutable] - │ ├── 'SBMT' [as=t_st_id_new:31] - │ └── '2020-06-17 22:27:42.148484+00:00' [as=t_dts_new:32] + │ ├── t_qty:23 > 0 [as=check1:35, outer=(23)] + │ ├── t_bid_price:24 > 0 [as=check2:36, outer=(24), immutable] + │ ├── t_chrg:28 >= 0 [as=check3:37, outer=(28), immutable] + │ ├── t_comm:29 >= 0 [as=check4:38, outer=(29), immutable] + │ ├── t_tax:30 >= 0 [as=check5:39, outer=(30), immutable] + │ ├── 'SBMT' [as=t_st_id_new:33] + │ └── '2020-06-17 22:27:42.148484+00:00' [as=t_dts_new:34] └── f-k-checks └── f-k-checks-item: trade(t_st_id) -> status_type(st_id) └── anti-join (lookup status_type) - ├── columns: t_st_id_new:38!null - ├── key columns: [38] = [39] + ├── columns: t_st_id_new:40!null + ├── key columns: [40] = [41] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(38) + ├── fd: ()-->(40) ├── with-scan &1 - │ ├── columns: t_st_id_new:38!null + │ ├── columns: t_st_id_new:40!null │ ├── mapping: - │ │ └── t_st_id_new:31 => t_st_id_new:38 + │ │ └── t_st_id_new:33 => t_st_id_new:40 │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(38) + │ └── fd: ()-->(40) └── filters (true) # Q4 @@ -4650,16 +4655,16 @@ DELETE FROM trade_request WHERE true ---- delete trade_request ├── columns: - ├── fetch columns: tr_t_id:7 tr_tt_id:8 tr_s_symb:9 tr_b_id:12 + ├── fetch columns: tr_t_id:8 tr_tt_id:9 tr_s_symb:10 tr_b_id:13 ├── cardinality: [0 - 0] ├── volatile, mutations └── scan trade_request - ├── columns: tr_t_id:7!null tr_tt_id:8!null tr_s_symb:9!null tr_b_id:12!null + ├── columns: tr_t_id:8!null tr_tt_id:9!null tr_s_symb:10!null tr_b_id:13!null ├── check constraint expressions - │ ├── tr_qty:10 > 0 [outer=(10), constraints=(/10: [/1 - ]; tight)] - │ └── tr_bid_price:11 > 0 [outer=(11), immutable, constraints=(/11: (/0 - ]; tight)] - ├── key: (7) - └── fd: (7)-->(8,9,12) + │ ├── tr_qty:11 > 0 [outer=(11), constraints=(/11: [/1 - ]; tight)] + │ └── tr_bid_price:12 > 0 [outer=(12), immutable, constraints=(/12: (/0 - ]; tight)] + ├── key: (8) + └── fd: (8)-->(9,10,13) # Q5 opt @@ -4682,48 +4687,48 @@ WHERE t_id = 0 ---- update trade ├── columns: - ├── fetch columns: t_id:16 t_dts:17 t_st_id:18 t_tt_id:19 t_is_cash:20 t_s_symb:21 t_qty:22 t_bid_price:23 t_ca_id:24 t_exec_name:25 t_trade_price:26 t_chrg:27 t_comm:28 t_tax:29 t_lifo:30 + ├── fetch columns: t_id:17 t_dts:18 t_st_id:19 t_tt_id:20 t_is_cash:21 t_s_symb:22 t_qty:23 t_bid_price:24 t_ca_id:25 t_exec_name:26 t_trade_price:27 t_chrg:28 t_comm:29 t_tax:30 t_lifo:31 ├── update-mapping: - │ ├── t_dts_new:32 => t_dts:2 - │ └── t_st_id_new:31 => t_st_id:3 - ├── check columns: check1:33 check2:34 check3:35 check4:36 check5:37 + │ ├── t_dts_new:34 => t_dts:2 + │ └── t_st_id_new:33 => t_st_id:3 + ├── check columns: check1:35 check2:36 check3:37 check4:38 check5:39 ├── input binding: &1 ├── cardinality: [0 - 0] ├── volatile, mutations ├── project - │ ├── columns: check1:33!null check2:34 check3:35!null check4:36!null check5:37!null t_st_id_new:31!null t_dts_new:32!null t_id:16!null t_dts:17!null t_st_id:18!null t_tt_id:19!null t_is_cash:20!null t_s_symb:21!null t_qty:22!null t_bid_price:23 t_ca_id:24!null t_exec_name:25!null t_trade_price:26 t_chrg:27!null t_comm:28!null t_tax:29!null t_lifo:30!null + │ ├── columns: check1:35!null check2:36 check3:37!null check4:38!null check5:39!null t_st_id_new:33!null t_dts_new:34!null t_id:17!null t_dts:18!null t_st_id:19!null t_tt_id:20!null t_is_cash:21!null t_s_symb:22!null t_qty:23!null t_bid_price:24 t_ca_id:25!null t_exec_name:26!null t_trade_price:27 t_chrg:28!null t_comm:29!null t_tax:30!null t_lifo:31!null │ ├── cardinality: [0 - 1] │ ├── immutable │ ├── key: () - │ ├── fd: ()-->(16-37) + │ ├── fd: ()-->(17-31,33-39) │ ├── scan trade - │ │ ├── columns: t_id:16!null t_dts:17!null t_st_id:18!null t_tt_id:19!null t_is_cash:20!null t_s_symb:21!null t_qty:22!null t_bid_price:23 t_ca_id:24!null t_exec_name:25!null t_trade_price:26 t_chrg:27!null t_comm:28!null t_tax:29!null t_lifo:30!null - │ │ ├── constraint: /16: [/0 - /0] + │ │ ├── columns: t_id:17!null t_dts:18!null t_st_id:19!null t_tt_id:20!null t_is_cash:21!null t_s_symb:22!null t_qty:23!null t_bid_price:24 t_ca_id:25!null t_exec_name:26!null t_trade_price:27 t_chrg:28!null t_comm:29!null t_tax:30!null t_lifo:31!null + │ │ ├── constraint: /17: [/0 - /0] │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(16-30) + │ │ └── fd: ()-->(17-31) │ └── projections - │ ├── t_qty:22 > 0 [as=check1:33, outer=(22)] - │ ├── t_bid_price:23 > 0 [as=check2:34, outer=(23), immutable] - │ ├── t_chrg:27 >= 0 [as=check3:35, outer=(27), immutable] - │ ├── t_comm:28 >= 0 [as=check4:36, outer=(28), immutable] - │ ├── t_tax:29 >= 0 [as=check5:37, outer=(29), immutable] - │ ├── 'CNCL' [as=t_st_id_new:31] - │ └── '2020-06-17 22:27:42.148484+00:00' [as=t_dts_new:32] + │ ├── t_qty:23 > 0 [as=check1:35, outer=(23)] + │ ├── t_bid_price:24 > 0 [as=check2:36, outer=(24), immutable] + │ ├── t_chrg:28 >= 0 [as=check3:37, outer=(28), immutable] + │ ├── t_comm:29 >= 0 [as=check4:38, outer=(29), immutable] + │ ├── t_tax:30 >= 0 [as=check5:39, outer=(30), immutable] + │ ├── 'CNCL' [as=t_st_id_new:33] + │ └── '2020-06-17 22:27:42.148484+00:00' [as=t_dts_new:34] └── f-k-checks └── f-k-checks-item: trade(t_st_id) -> status_type(st_id) └── anti-join (lookup status_type) - ├── columns: t_st_id_new:38!null - ├── key columns: [38] = [39] + ├── columns: t_st_id_new:40!null + ├── key columns: [40] = [41] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(38) + ├── fd: ()-->(40) ├── with-scan &1 - │ ├── columns: t_st_id_new:38!null + │ ├── columns: t_st_id_new:40!null │ ├── mapping: - │ │ └── t_st_id_new:31 => t_st_id_new:38 + │ │ └── t_st_id_new:33 => t_st_id_new:40 │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(38) + │ └── fd: ()-->(40) └── filters (true) diff --git a/pkg/sql/opt/xform/testdata/external/tpch b/pkg/sql/opt/xform/testdata/external/tpch index e960a6d796b9..4159d78a339a 100644 --- a/pkg/sql/opt/xform/testdata/external/tpch +++ b/pkg/sql/opt/xform/testdata/external/tpch @@ -41,19 +41,19 @@ ORDER BY l_linestatus; ---- sort - ├── columns: l_returnflag:9!null l_linestatus:10!null sum_qty:17!null sum_base_price:18!null sum_disc_price:20!null sum_charge:22!null avg_qty:23!null avg_price:24!null avg_disc:25!null count_order:26!null + ├── columns: l_returnflag:9!null l_linestatus:10!null sum_qty:18!null sum_base_price:19!null sum_disc_price:21!null sum_charge:23!null avg_qty:24!null avg_price:25!null avg_disc:26!null count_order:27!null ├── immutable ├── key: (9,10) - ├── fd: (9,10)-->(17,18,20,22-26) + ├── fd: (9,10)-->(18,19,21,23-27) ├── ordering: +9,+10 └── group-by - ├── columns: l_returnflag:9!null l_linestatus:10!null sum:17!null sum:18!null sum:20!null sum:22!null avg:23!null avg:24!null avg:25!null count_rows:26!null + ├── columns: l_returnflag:9!null l_linestatus:10!null sum:18!null sum:19!null sum:21!null sum:23!null avg:24!null avg:25!null avg:26!null count_rows:27!null ├── grouping columns: l_returnflag:9!null l_linestatus:10!null ├── immutable ├── key: (9,10) - ├── fd: (9,10)-->(17,18,20,22-26) + ├── fd: (9,10)-->(18,19,21,23-27) ├── project - │ ├── columns: column19:19!null column21:21!null l_quantity:5!null l_extendedprice:6!null l_discount:7!null l_returnflag:9!null l_linestatus:10!null + │ ├── columns: column20:20!null column22:22!null l_quantity:5!null l_extendedprice:6!null l_discount:7!null l_returnflag:9!null l_linestatus:10!null │ ├── immutable │ ├── select │ │ ├── columns: l_quantity:5!null l_extendedprice:6!null l_discount:7!null l_tax:8!null l_returnflag:9!null l_linestatus:10!null l_shipdate:11!null @@ -62,24 +62,24 @@ sort │ │ └── filters │ │ └── l_shipdate:11 <= '1998-09-02' [outer=(11), constraints=(/11: (/NULL - /'1998-09-02']; tight)] │ └── projections - │ ├── l_extendedprice:6 * (1.0 - l_discount:7) [as=column19:19, outer=(6,7), immutable] - │ └── (l_extendedprice:6 * (1.0 - l_discount:7)) * (l_tax:8 + 1.0) [as=column21:21, outer=(6-8), immutable] + │ ├── l_extendedprice:6 * (1.0 - l_discount:7) [as=column20:20, outer=(6,7), immutable] + │ └── (l_extendedprice:6 * (1.0 - l_discount:7)) * (l_tax:8 + 1.0) [as=column22:22, outer=(6-8), immutable] └── aggregations - ├── sum [as=sum:17, outer=(5)] + ├── sum [as=sum:18, outer=(5)] │ └── l_quantity:5 - ├── sum [as=sum:18, outer=(6)] + ├── sum [as=sum:19, outer=(6)] │ └── l_extendedprice:6 - ├── sum [as=sum:20, outer=(19)] - │ └── column19:19 - ├── sum [as=sum:22, outer=(21)] - │ └── column21:21 - ├── avg [as=avg:23, outer=(5)] + ├── sum [as=sum:21, outer=(20)] + │ └── column20:20 + ├── sum [as=sum:23, outer=(22)] + │ └── column22:22 + ├── avg [as=avg:24, outer=(5)] │ └── l_quantity:5 - ├── avg [as=avg:24, outer=(6)] + ├── avg [as=avg:25, outer=(6)] │ └── l_extendedprice:6 - ├── avg [as=avg:25, outer=(7)] + ├── avg [as=avg:26, outer=(7)] │ └── l_discount:7 - └── count-rows [as=count_rows:26] + └── count-rows [as=count_rows:27] # -------------------------------------------------- # Q2 @@ -142,52 +142,52 @@ ORDER BY LIMIT 100; ---- project - ├── columns: s_acctbal:15!null s_name:11!null n_name:23!null p_partkey:1!null p_mfgr:3!null s_address:12!null s_phone:14!null s_comment:16!null + ├── columns: s_acctbal:16!null s_name:12!null n_name:26!null p_partkey:1!null p_mfgr:3!null s_address:13!null s_phone:15!null s_comment:17!null ├── cardinality: [0 - 100] ├── fd: (1)-->(3) - ├── ordering: -15,+23,+11,+1 + ├── ordering: -16,+26,+12,+1 └── limit - ├── columns: p_partkey:1!null p_mfgr:3!null s_name:11!null s_address:12!null s_phone:14!null s_acctbal:15!null s_comment:16!null ps_partkey:17!null ps_suppkey:18!null ps_supplycost:20!null n_name:23!null min:48!null - ├── internal-ordering: -15,+23,+11,+(1|17) + ├── columns: p_partkey:1!null p_mfgr:3!null s_name:12!null s_address:13!null s_phone:15!null s_acctbal:16!null s_comment:17!null ps_partkey:19!null ps_suppkey:20!null ps_supplycost:22!null n_name:26!null min:57!null + ├── internal-ordering: -16,+26,+12,+(1|19) ├── cardinality: [0 - 100] - ├── key: (17,18) - ├── fd: (1)-->(3), (17,18)-->(1,3,11,12,14-16,20,23,48), (1)==(17), (17)==(1), (18)-->(11,12,14-16,23), (20)==(48), (48)==(20) - ├── ordering: -15,+23,+11,+(1|17) [actual: -15,+23,+11,+1] + ├── key: (19,20) + ├── fd: (1)-->(3), (19,20)-->(1,3,12,13,15-17,22,26,57), (1)==(19), (19)==(1), (20)-->(12,13,15-17,26), (22)==(57), (57)==(22) + ├── ordering: -16,+26,+12,+(1|19) [actual: -16,+26,+12,+1] ├── sort - │ ├── columns: p_partkey:1!null p_mfgr:3!null s_name:11!null s_address:12!null s_phone:14!null s_acctbal:15!null s_comment:16!null ps_partkey:17!null ps_suppkey:18!null ps_supplycost:20!null n_name:23!null min:48!null - │ ├── key: (17,18) - │ ├── fd: (1)-->(3), (17,18)-->(1,3,11,12,14-16,20,23,48), (1)==(17), (17)==(1), (18)-->(11,12,14-16,23), (20)==(48), (48)==(20) - │ ├── ordering: -15,+23,+11,+(1|17) [actual: -15,+23,+11,+1] + │ ├── columns: p_partkey:1!null p_mfgr:3!null s_name:12!null s_address:13!null s_phone:15!null s_acctbal:16!null s_comment:17!null ps_partkey:19!null ps_suppkey:20!null ps_supplycost:22!null n_name:26!null min:57!null + │ ├── key: (19,20) + │ ├── fd: (1)-->(3), (19,20)-->(1,3,12,13,15-17,22,26,57), (1)==(19), (19)==(1), (20)-->(12,13,15-17,26), (22)==(57), (57)==(22) + │ ├── ordering: -16,+26,+12,+(1|19) [actual: -16,+26,+12,+1] │ ├── limit hint: 100.00 │ └── select - │ ├── columns: p_partkey:1!null p_mfgr:3!null s_name:11!null s_address:12!null s_phone:14!null s_acctbal:15!null s_comment:16!null ps_partkey:17!null ps_suppkey:18!null ps_supplycost:20!null n_name:23!null min:48!null - │ ├── key: (17,18) - │ ├── fd: (1)-->(3), (17,18)-->(1,3,11,12,14-16,20,23,48), (1)==(17), (17)==(1), (18)-->(11,12,14-16,23), (20)==(48), (48)==(20) + │ ├── columns: p_partkey:1!null p_mfgr:3!null s_name:12!null s_address:13!null s_phone:15!null s_acctbal:16!null s_comment:17!null ps_partkey:19!null ps_suppkey:20!null ps_supplycost:22!null n_name:26!null min:57!null + │ ├── key: (19,20) + │ ├── fd: (1)-->(3), (19,20)-->(1,3,12,13,15-17,22,26,57), (1)==(19), (19)==(1), (20)-->(12,13,15-17,26), (22)==(57), (57)==(22) │ ├── group-by - │ │ ├── columns: p_partkey:1!null p_mfgr:3!null s_name:11!null s_address:12!null s_phone:14!null s_acctbal:15!null s_comment:16!null ps_partkey:17!null ps_suppkey:18!null ps_supplycost:20!null n_name:23!null min:48!null - │ │ ├── grouping columns: ps_partkey:17!null ps_suppkey:18!null - │ │ ├── key: (17,18) - │ │ ├── fd: (1)-->(3), (17,18)-->(1,3,11,12,14-16,20,23,48), (1)==(17), (17)==(1), (18)-->(11,12,14-16,23) + │ │ ├── columns: p_partkey:1!null p_mfgr:3!null s_name:12!null s_address:13!null s_phone:15!null s_acctbal:16!null s_comment:17!null ps_partkey:19!null ps_suppkey:20!null ps_supplycost:22!null n_name:26!null min:57!null + │ │ ├── grouping columns: ps_partkey:19!null ps_suppkey:20!null + │ │ ├── key: (19,20) + │ │ ├── fd: (1)-->(3), (19,20)-->(1,3,12,13,15-17,22,26,57), (1)==(19), (19)==(1), (20)-->(12,13,15-17,26) │ │ ├── inner-join (hash) - │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_suppkey:10!null s_name:11!null s_address:12!null s_nationkey:13!null s_phone:14!null s_acctbal:15!null s_comment:16!null ps_partkey:17!null ps_suppkey:18!null ps_supplycost:20!null n_nationkey:22!null n_name:23!null n_regionkey:24!null r_regionkey:26!null r_name:27!null ps_partkey:29!null ps_suppkey:30!null ps_supplycost:32!null s_suppkey:34!null s_nationkey:37!null n_nationkey:41!null n_regionkey:43!null r_regionkey:45!null r_name:46!null + │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_suppkey:11!null s_name:12!null s_address:13!null s_nationkey:14!null s_phone:15!null s_acctbal:16!null s_comment:17!null ps_partkey:19!null ps_suppkey:20!null ps_supplycost:22!null n_nationkey:25!null n_name:26!null n_regionkey:27!null r_regionkey:30!null r_name:31!null ps_partkey:34!null ps_suppkey:35!null ps_supplycost:37!null s_suppkey:40!null s_nationkey:43!null n_nationkey:48!null n_regionkey:50!null r_regionkey:53!null r_name:54!null │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ ├── key: (18,29,34) - │ │ │ ├── fd: ()-->(6,27,46), (1)-->(3,5), (10)-->(11-16), (17,18)-->(20), (10)==(18), (18)==(10), (22)-->(23,24), (24)==(26), (26)==(24), (13)==(22), (22)==(13), (1)==(17,29), (17)==(1,29), (29,30)-->(32), (34)-->(37), (41)-->(43), (43)==(45), (45)==(43), (37)==(41), (41)==(37), (30)==(34), (34)==(30), (29)==(1,17) + │ │ │ ├── key: (20,34,40) + │ │ │ ├── fd: ()-->(6,31,54), (1)-->(3,5), (11)-->(12-17), (19,20)-->(22), (11)==(20), (20)==(11), (25)-->(26,27), (27)==(30), (30)==(27), (14)==(25), (25)==(14), (1)==(19,34), (19)==(1,34), (34,35)-->(37), (40)-->(43), (48)-->(50), (50)==(53), (53)==(50), (43)==(48), (48)==(43), (35)==(40), (40)==(35), (34)==(1,19) │ │ │ ├── inner-join (lookup partsupp) - │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_suppkey:10!null s_name:11!null s_address:12!null s_nationkey:13!null s_phone:14!null s_acctbal:15!null s_comment:16!null ps_partkey:17!null ps_suppkey:18!null ps_supplycost:20!null n_nationkey:22!null n_name:23!null n_regionkey:24!null r_regionkey:26!null r_name:27!null ps_partkey:29!null ps_suppkey:30!null ps_supplycost:32!null - │ │ │ │ ├── key columns: [1] = [29] - │ │ │ │ ├── key: (18,29,30) - │ │ │ │ ├── fd: ()-->(6,27), (1)-->(3,5), (22)-->(23,24), (10)-->(11-16), (17,18)-->(20), (29,30)-->(32), (17)==(1,29), (29)==(1,17), (10)==(18), (18)==(10), (13)==(22), (22)==(13), (24)==(26), (26)==(24), (1)==(17,29) + │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_suppkey:11!null s_name:12!null s_address:13!null s_nationkey:14!null s_phone:15!null s_acctbal:16!null s_comment:17!null ps_partkey:19!null ps_suppkey:20!null ps_supplycost:22!null n_nationkey:25!null n_name:26!null n_regionkey:27!null r_regionkey:30!null r_name:31!null ps_partkey:34!null ps_suppkey:35!null ps_supplycost:37!null + │ │ │ │ ├── key columns: [1] = [34] + │ │ │ │ ├── key: (20,34,35) + │ │ │ │ ├── fd: ()-->(6,31), (1)-->(3,5), (25)-->(26,27), (11)-->(12-17), (19,20)-->(22), (34,35)-->(37), (19)==(1,34), (34)==(1,19), (11)==(20), (20)==(11), (14)==(25), (25)==(14), (27)==(30), (30)==(27), (1)==(19,34) │ │ │ │ ├── inner-join (hash) - │ │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_suppkey:10!null s_name:11!null s_address:12!null s_nationkey:13!null s_phone:14!null s_acctbal:15!null s_comment:16!null ps_partkey:17!null ps_suppkey:18!null ps_supplycost:20!null n_nationkey:22!null n_name:23!null n_regionkey:24!null r_regionkey:26!null r_name:27!null + │ │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_suppkey:11!null s_name:12!null s_address:13!null s_nationkey:14!null s_phone:15!null s_acctbal:16!null s_comment:17!null ps_partkey:19!null ps_suppkey:20!null ps_supplycost:22!null n_nationkey:25!null n_name:26!null n_regionkey:27!null r_regionkey:30!null r_name:31!null │ │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ │ │ ├── key: (17,18) - │ │ │ │ │ ├── fd: ()-->(6,27), (1)-->(3,5), (10)-->(11-16), (17,18)-->(20), (10)==(18), (18)==(10), (22)-->(23,24), (24)==(26), (26)==(24), (13)==(22), (22)==(13), (1)==(17), (17)==(1) + │ │ │ │ │ ├── key: (19,20) + │ │ │ │ │ ├── fd: ()-->(6,31), (1)-->(3,5), (11)-->(12-17), (19,20)-->(22), (11)==(20), (20)==(11), (25)-->(26,27), (27)==(30), (30)==(27), (14)==(25), (25)==(14), (1)==(19), (19)==(1) │ │ │ │ │ ├── inner-join (lookup partsupp) - │ │ │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null ps_partkey:17!null ps_suppkey:18!null ps_supplycost:20!null - │ │ │ │ │ │ ├── key columns: [1] = [17] - │ │ │ │ │ │ ├── key: (17,18) - │ │ │ │ │ │ ├── fd: ()-->(6), (1)-->(3,5), (17,18)-->(20), (1)==(17), (17)==(1) + │ │ │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null ps_partkey:19!null ps_suppkey:20!null ps_supplycost:22!null + │ │ │ │ │ │ ├── key columns: [1] = [19] + │ │ │ │ │ │ ├── key: (19,20) + │ │ │ │ │ │ ├── fd: ()-->(6), (1)-->(3,5), (19,20)-->(22), (1)==(19), (19)==(1) │ │ │ │ │ │ ├── select │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null │ │ │ │ │ │ │ ├── key: (1) @@ -201,95 +201,95 @@ project │ │ │ │ │ │ │ └── p_type:5 LIKE '%BRASS' [outer=(5), constraints=(/5: (/NULL - ])] │ │ │ │ │ │ └── filters (true) │ │ │ │ │ ├── inner-join (hash) - │ │ │ │ │ │ ├── columns: s_suppkey:10!null s_name:11!null s_address:12!null s_nationkey:13!null s_phone:14!null s_acctbal:15!null s_comment:16!null n_nationkey:22!null n_name:23!null n_regionkey:24!null r_regionkey:26!null r_name:27!null + │ │ │ │ │ │ ├── columns: s_suppkey:11!null s_name:12!null s_address:13!null s_nationkey:14!null s_phone:15!null s_acctbal:16!null s_comment:17!null n_nationkey:25!null n_name:26!null n_regionkey:27!null r_regionkey:30!null r_name:31!null │ │ │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ │ │ │ ├── key: (10) - │ │ │ │ │ │ ├── fd: ()-->(27), (10)-->(11-16), (22)-->(23,24), (24)==(26), (26)==(24), (13)==(22), (22)==(13) + │ │ │ │ │ │ ├── key: (11) + │ │ │ │ │ │ ├── fd: ()-->(31), (11)-->(12-17), (25)-->(26,27), (27)==(30), (30)==(27), (14)==(25), (25)==(14) │ │ │ │ │ │ ├── scan supplier - │ │ │ │ │ │ │ ├── columns: s_suppkey:10!null s_name:11!null s_address:12!null s_nationkey:13!null s_phone:14!null s_acctbal:15!null s_comment:16!null - │ │ │ │ │ │ │ ├── key: (10) - │ │ │ │ │ │ │ └── fd: (10)-->(11-16) + │ │ │ │ │ │ │ ├── columns: s_suppkey:11!null s_name:12!null s_address:13!null s_nationkey:14!null s_phone:15!null s_acctbal:16!null s_comment:17!null + │ │ │ │ │ │ │ ├── key: (11) + │ │ │ │ │ │ │ └── fd: (11)-->(12-17) │ │ │ │ │ │ ├── inner-join (hash) - │ │ │ │ │ │ │ ├── columns: n_nationkey:22!null n_name:23!null n_regionkey:24!null r_regionkey:26!null r_name:27!null + │ │ │ │ │ │ │ ├── columns: n_nationkey:25!null n_name:26!null n_regionkey:27!null r_regionkey:30!null r_name:31!null │ │ │ │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ │ │ │ │ ├── key: (22) - │ │ │ │ │ │ │ ├── fd: ()-->(27), (22)-->(23,24), (24)==(26), (26)==(24) + │ │ │ │ │ │ │ ├── key: (25) + │ │ │ │ │ │ │ ├── fd: ()-->(31), (25)-->(26,27), (27)==(30), (30)==(27) │ │ │ │ │ │ │ ├── scan nation - │ │ │ │ │ │ │ │ ├── columns: n_nationkey:22!null n_name:23!null n_regionkey:24!null - │ │ │ │ │ │ │ │ ├── key: (22) - │ │ │ │ │ │ │ │ └── fd: (22)-->(23,24) + │ │ │ │ │ │ │ │ ├── columns: n_nationkey:25!null n_name:26!null n_regionkey:27!null + │ │ │ │ │ │ │ │ ├── key: (25) + │ │ │ │ │ │ │ │ └── fd: (25)-->(26,27) │ │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ │ ├── columns: r_regionkey:26!null r_name:27!null - │ │ │ │ │ │ │ │ ├── key: (26) - │ │ │ │ │ │ │ │ ├── fd: ()-->(27) + │ │ │ │ │ │ │ │ ├── columns: r_regionkey:30!null r_name:31!null + │ │ │ │ │ │ │ │ ├── key: (30) + │ │ │ │ │ │ │ │ ├── fd: ()-->(31) │ │ │ │ │ │ │ │ ├── scan region - │ │ │ │ │ │ │ │ │ ├── columns: r_regionkey:26!null r_name:27!null - │ │ │ │ │ │ │ │ │ ├── key: (26) - │ │ │ │ │ │ │ │ │ └── fd: (26)-->(27) + │ │ │ │ │ │ │ │ │ ├── columns: r_regionkey:30!null r_name:31!null + │ │ │ │ │ │ │ │ │ ├── key: (30) + │ │ │ │ │ │ │ │ │ └── fd: (30)-->(31) │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ └── r_name:27 = 'EUROPE' [outer=(27), constraints=(/27: [/'EUROPE' - /'EUROPE']; tight), fd=()-->(27)] + │ │ │ │ │ │ │ │ └── r_name:31 = 'EUROPE' [outer=(31), constraints=(/31: [/'EUROPE' - /'EUROPE']; tight), fd=()-->(31)] │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── n_regionkey:24 = r_regionkey:26 [outer=(24,26), constraints=(/24: (/NULL - ]; /26: (/NULL - ]), fd=(24)==(26), (26)==(24)] + │ │ │ │ │ │ │ └── n_regionkey:27 = r_regionkey:30 [outer=(27,30), constraints=(/27: (/NULL - ]; /30: (/NULL - ]), fd=(27)==(30), (30)==(27)] │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── s_nationkey:13 = n_nationkey:22 [outer=(13,22), constraints=(/13: (/NULL - ]; /22: (/NULL - ]), fd=(13)==(22), (22)==(13)] + │ │ │ │ │ │ └── s_nationkey:14 = n_nationkey:25 [outer=(14,25), constraints=(/14: (/NULL - ]; /25: (/NULL - ]), fd=(14)==(25), (25)==(14)] │ │ │ │ │ └── filters - │ │ │ │ │ └── s_suppkey:10 = ps_suppkey:18 [outer=(10,18), constraints=(/10: (/NULL - ]; /18: (/NULL - ]), fd=(10)==(18), (18)==(10)] + │ │ │ │ │ └── s_suppkey:11 = ps_suppkey:20 [outer=(11,20), constraints=(/11: (/NULL - ]; /20: (/NULL - ]), fd=(11)==(20), (20)==(11)] │ │ │ │ └── filters (true) │ │ │ ├── inner-join (lookup supplier@s_nk) - │ │ │ │ ├── columns: s_suppkey:34!null s_nationkey:37!null n_nationkey:41!null n_regionkey:43!null r_regionkey:45!null r_name:46!null - │ │ │ │ ├── key columns: [41] = [37] - │ │ │ │ ├── key: (34) - │ │ │ │ ├── fd: ()-->(46), (34)-->(37), (41)-->(43), (43)==(45), (45)==(43), (37)==(41), (41)==(37) + │ │ │ │ ├── columns: s_suppkey:40!null s_nationkey:43!null n_nationkey:48!null n_regionkey:50!null r_regionkey:53!null r_name:54!null + │ │ │ │ ├── key columns: [48] = [43] + │ │ │ │ ├── key: (40) + │ │ │ │ ├── fd: ()-->(54), (40)-->(43), (48)-->(50), (50)==(53), (53)==(50), (43)==(48), (48)==(43) │ │ │ │ ├── inner-join (merge) - │ │ │ │ │ ├── columns: n_nationkey:41!null n_regionkey:43!null r_regionkey:45!null r_name:46!null - │ │ │ │ │ ├── left ordering: +43 - │ │ │ │ │ ├── right ordering: +45 - │ │ │ │ │ ├── key: (41) - │ │ │ │ │ ├── fd: ()-->(46), (41)-->(43), (43)==(45), (45)==(43) + │ │ │ │ │ ├── columns: n_nationkey:48!null n_regionkey:50!null r_regionkey:53!null r_name:54!null + │ │ │ │ │ ├── left ordering: +50 + │ │ │ │ │ ├── right ordering: +53 + │ │ │ │ │ ├── key: (48) + │ │ │ │ │ ├── fd: ()-->(54), (48)-->(50), (50)==(53), (53)==(50) │ │ │ │ │ ├── scan nation@n_rk - │ │ │ │ │ │ ├── columns: n_nationkey:41!null n_regionkey:43!null - │ │ │ │ │ │ ├── key: (41) - │ │ │ │ │ │ ├── fd: (41)-->(43) - │ │ │ │ │ │ └── ordering: +43 + │ │ │ │ │ │ ├── columns: n_nationkey:48!null n_regionkey:50!null + │ │ │ │ │ │ ├── key: (48) + │ │ │ │ │ │ ├── fd: (48)-->(50) + │ │ │ │ │ │ └── ordering: +50 │ │ │ │ │ ├── select - │ │ │ │ │ │ ├── columns: r_regionkey:45!null r_name:46!null - │ │ │ │ │ │ ├── key: (45) - │ │ │ │ │ │ ├── fd: ()-->(46) - │ │ │ │ │ │ ├── ordering: +45 opt(46) [actual: +45] + │ │ │ │ │ │ ├── columns: r_regionkey:53!null r_name:54!null + │ │ │ │ │ │ ├── key: (53) + │ │ │ │ │ │ ├── fd: ()-->(54) + │ │ │ │ │ │ ├── ordering: +53 opt(54) [actual: +53] │ │ │ │ │ │ ├── scan region - │ │ │ │ │ │ │ ├── columns: r_regionkey:45!null r_name:46!null - │ │ │ │ │ │ │ ├── key: (45) - │ │ │ │ │ │ │ ├── fd: (45)-->(46) - │ │ │ │ │ │ │ └── ordering: +45 opt(46) [actual: +45] + │ │ │ │ │ │ │ ├── columns: r_regionkey:53!null r_name:54!null + │ │ │ │ │ │ │ ├── key: (53) + │ │ │ │ │ │ │ ├── fd: (53)-->(54) + │ │ │ │ │ │ │ └── ordering: +53 opt(54) [actual: +53] │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── r_name:46 = 'EUROPE' [outer=(46), constraints=(/46: [/'EUROPE' - /'EUROPE']; tight), fd=()-->(46)] + │ │ │ │ │ │ └── r_name:54 = 'EUROPE' [outer=(54), constraints=(/54: [/'EUROPE' - /'EUROPE']; tight), fd=()-->(54)] │ │ │ │ │ └── filters (true) │ │ │ │ └── filters (true) │ │ │ └── filters - │ │ │ └── s_suppkey:34 = ps_suppkey:30 [outer=(30,34), constraints=(/30: (/NULL - ]; /34: (/NULL - ]), fd=(30)==(34), (34)==(30)] + │ │ │ └── s_suppkey:40 = ps_suppkey:35 [outer=(35,40), constraints=(/35: (/NULL - ]; /40: (/NULL - ]), fd=(35)==(40), (40)==(35)] │ │ └── aggregations - │ │ ├── min [as=min:48, outer=(32)] - │ │ │ └── ps_supplycost:32 - │ │ ├── const-agg [as=s_name:11, outer=(11)] - │ │ │ └── s_name:11 - │ │ ├── const-agg [as=s_address:12, outer=(12)] - │ │ │ └── s_address:12 - │ │ ├── const-agg [as=s_phone:14, outer=(14)] - │ │ │ └── s_phone:14 - │ │ ├── const-agg [as=s_acctbal:15, outer=(15)] - │ │ │ └── s_acctbal:15 - │ │ ├── const-agg [as=s_comment:16, outer=(16)] - │ │ │ └── s_comment:16 - │ │ ├── const-agg [as=ps_supplycost:20, outer=(20)] - │ │ │ └── ps_supplycost:20 - │ │ ├── const-agg [as=n_name:23, outer=(23)] - │ │ │ └── n_name:23 + │ │ ├── min [as=min:57, outer=(37)] + │ │ │ └── ps_supplycost:37 + │ │ ├── const-agg [as=s_name:12, outer=(12)] + │ │ │ └── s_name:12 + │ │ ├── const-agg [as=s_address:13, outer=(13)] + │ │ │ └── s_address:13 + │ │ ├── const-agg [as=s_phone:15, outer=(15)] + │ │ │ └── s_phone:15 + │ │ ├── const-agg [as=s_acctbal:16, outer=(16)] + │ │ │ └── s_acctbal:16 + │ │ ├── const-agg [as=s_comment:17, outer=(17)] + │ │ │ └── s_comment:17 + │ │ ├── const-agg [as=ps_supplycost:22, outer=(22)] + │ │ │ └── ps_supplycost:22 + │ │ ├── const-agg [as=n_name:26, outer=(26)] + │ │ │ └── n_name:26 │ │ ├── const-agg [as=p_mfgr:3, outer=(3)] │ │ │ └── p_mfgr:3 │ │ └── const-agg [as=p_partkey:1, outer=(1)] │ │ └── p_partkey:1 │ └── filters - │ └── ps_supplycost:20 = min:48 [outer=(20,48), constraints=(/20: (/NULL - ]; /48: (/NULL - ]), fd=(20)==(48), (48)==(20)] + │ └── ps_supplycost:22 = min:57 [outer=(22,57), constraints=(/22: (/NULL - ]; /57: (/NULL - ]), fd=(22)==(57), (57)==(22)] └── 100 # -------------------------------------------------- @@ -329,49 +329,49 @@ ORDER BY LIMIT 10; ---- limit - ├── columns: l_orderkey:18!null revenue:35!null o_orderdate:13!null o_shippriority:16!null - ├── internal-ordering: -35,+13 + ├── columns: l_orderkey:20!null revenue:38!null o_orderdate:14!null o_shippriority:17!null + ├── internal-ordering: -38,+14 ├── cardinality: [0 - 10] ├── immutable - ├── key: (18) - ├── fd: (18)-->(13,16,35) - ├── ordering: -35,+13 + ├── key: (20) + ├── fd: (20)-->(14,17,38) + ├── ordering: -38,+14 ├── sort - │ ├── columns: o_orderdate:13!null o_shippriority:16!null l_orderkey:18!null sum:35!null + │ ├── columns: o_orderdate:14!null o_shippriority:17!null l_orderkey:20!null sum:38!null │ ├── immutable - │ ├── key: (18) - │ ├── fd: (18)-->(13,16,35) - │ ├── ordering: -35,+13 + │ ├── key: (20) + │ ├── fd: (20)-->(14,17,38) + │ ├── ordering: -38,+14 │ ├── limit hint: 10.00 │ └── group-by - │ ├── columns: o_orderdate:13!null o_shippriority:16!null l_orderkey:18!null sum:35!null - │ ├── grouping columns: l_orderkey:18!null + │ ├── columns: o_orderdate:14!null o_shippriority:17!null l_orderkey:20!null sum:38!null + │ ├── grouping columns: l_orderkey:20!null │ ├── immutable - │ ├── key: (18) - │ ├── fd: (18)-->(13,16,35) + │ ├── key: (20) + │ ├── fd: (20)-->(14,17,38) │ ├── project - │ │ ├── columns: column34:34!null o_orderdate:13!null o_shippriority:16!null l_orderkey:18!null + │ │ ├── columns: column37:37!null o_orderdate:14!null o_shippriority:17!null l_orderkey:20!null │ │ ├── immutable - │ │ ├── fd: (18)-->(13,16) + │ │ ├── fd: (20)-->(14,17) │ │ ├── inner-join (lookup lineitem) - │ │ │ ├── columns: c_custkey:1!null c_mktsegment:7!null o_orderkey:9!null o_custkey:10!null o_orderdate:13!null o_shippriority:16!null l_orderkey:18!null l_extendedprice:23!null l_discount:24!null l_shipdate:28!null - │ │ │ ├── key columns: [9] = [18] - │ │ │ ├── fd: ()-->(7), (9)-->(10,13,16), (9)==(18), (18)==(9), (1)==(10), (10)==(1) + │ │ │ ├── columns: c_custkey:1!null c_mktsegment:7!null o_orderkey:10!null o_custkey:11!null o_orderdate:14!null o_shippriority:17!null l_orderkey:20!null l_extendedprice:25!null l_discount:26!null l_shipdate:30!null + │ │ │ ├── key columns: [10] = [20] + │ │ │ ├── fd: ()-->(7), (10)-->(11,14,17), (10)==(20), (20)==(10), (1)==(11), (11)==(1) │ │ │ ├── inner-join (hash) - │ │ │ │ ├── columns: c_custkey:1!null c_mktsegment:7!null o_orderkey:9!null o_custkey:10!null o_orderdate:13!null o_shippriority:16!null + │ │ │ │ ├── columns: c_custkey:1!null c_mktsegment:7!null o_orderkey:10!null o_custkey:11!null o_orderdate:14!null o_shippriority:17!null │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ │ ├── key: (9) - │ │ │ │ ├── fd: ()-->(7), (9)-->(10,13,16), (1)==(10), (10)==(1) + │ │ │ │ ├── key: (10) + │ │ │ │ ├── fd: ()-->(7), (10)-->(11,14,17), (1)==(11), (11)==(1) │ │ │ │ ├── select - │ │ │ │ │ ├── columns: o_orderkey:9!null o_custkey:10!null o_orderdate:13!null o_shippriority:16!null - │ │ │ │ │ ├── key: (9) - │ │ │ │ │ ├── fd: (9)-->(10,13,16) + │ │ │ │ │ ├── columns: o_orderkey:10!null o_custkey:11!null o_orderdate:14!null o_shippriority:17!null + │ │ │ │ │ ├── key: (10) + │ │ │ │ │ ├── fd: (10)-->(11,14,17) │ │ │ │ │ ├── scan orders - │ │ │ │ │ │ ├── columns: o_orderkey:9!null o_custkey:10!null o_orderdate:13!null o_shippriority:16!null - │ │ │ │ │ │ ├── key: (9) - │ │ │ │ │ │ └── fd: (9)-->(10,13,16) + │ │ │ │ │ │ ├── columns: o_orderkey:10!null o_custkey:11!null o_orderdate:14!null o_shippriority:17!null + │ │ │ │ │ │ ├── key: (10) + │ │ │ │ │ │ └── fd: (10)-->(11,14,17) │ │ │ │ │ └── filters - │ │ │ │ │ └── o_orderdate:13 < '1995-03-15' [outer=(13), constraints=(/13: (/NULL - /'1995-03-14']; tight)] + │ │ │ │ │ └── o_orderdate:14 < '1995-03-15' [outer=(14), constraints=(/14: (/NULL - /'1995-03-14']; tight)] │ │ │ │ ├── select │ │ │ │ │ ├── columns: c_custkey:1!null c_mktsegment:7!null │ │ │ │ │ ├── key: (1) @@ -383,18 +383,18 @@ limit │ │ │ │ │ └── filters │ │ │ │ │ └── c_mktsegment:7 = 'BUILDING' [outer=(7), constraints=(/7: [/'BUILDING' - /'BUILDING']; tight), fd=()-->(7)] │ │ │ │ └── filters - │ │ │ │ └── c_custkey:1 = o_custkey:10 [outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] + │ │ │ │ └── c_custkey:1 = o_custkey:11 [outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ]), fd=(1)==(11), (11)==(1)] │ │ │ └── filters - │ │ │ └── l_shipdate:28 > '1995-03-15' [outer=(28), constraints=(/28: [/'1995-03-16' - ]; tight)] + │ │ │ └── l_shipdate:30 > '1995-03-15' [outer=(30), constraints=(/30: [/'1995-03-16' - ]; tight)] │ │ └── projections - │ │ └── l_extendedprice:23 * (1.0 - l_discount:24) [as=column34:34, outer=(23,24), immutable] + │ │ └── l_extendedprice:25 * (1.0 - l_discount:26) [as=column37:37, outer=(25,26), immutable] │ └── aggregations - │ ├── sum [as=sum:35, outer=(34)] - │ │ └── column34:34 - │ ├── const-agg [as=o_orderdate:13, outer=(13)] - │ │ └── o_orderdate:13 - │ └── const-agg [as=o_shippriority:16, outer=(16)] - │ └── o_shippriority:16 + │ ├── sum [as=sum:38, outer=(37)] + │ │ └── column37:37 + │ ├── const-agg [as=o_orderdate:14, outer=(14)] + │ │ └── o_orderdate:14 + │ └── const-agg [as=o_shippriority:17, outer=(17)] + │ └── o_shippriority:17 └── 10 # -------------------------------------------------- @@ -432,18 +432,18 @@ ORDER BY o_orderpriority; ---- sort - ├── columns: o_orderpriority:6!null order_count:26!null + ├── columns: o_orderpriority:6!null order_count:28!null ├── key: (6) - ├── fd: (6)-->(26) + ├── fd: (6)-->(28) ├── ordering: +6 └── group-by - ├── columns: o_orderpriority:6!null count_rows:26!null + ├── columns: o_orderpriority:6!null count_rows:28!null ├── grouping columns: o_orderpriority:6!null ├── key: (6) - ├── fd: (6)-->(26) + ├── fd: (6)-->(28) ├── semi-join (lookup lineitem) │ ├── columns: o_orderkey:1!null o_orderdate:5!null o_orderpriority:6!null - │ ├── key columns: [1] = [10] + │ ├── key columns: [1] = [11] │ ├── key: (1) │ ├── fd: (1)-->(5,6) │ ├── index-join orders @@ -456,9 +456,9 @@ sort │ │ ├── key: (1) │ │ └── fd: (1)-->(5) │ └── filters - │ └── l_commitdate:21 < l_receiptdate:22 [outer=(21,22), constraints=(/21: (/NULL - ]; /22: (/NULL - ])] + │ └── l_commitdate:22 < l_receiptdate:23 [outer=(22,23), constraints=(/22: (/NULL - ]; /23: (/NULL - ])] └── aggregations - └── count-rows [as=count_rows:26] + └── count-rows [as=count_rows:28] # -------------------------------------------------- # Q5 @@ -501,84 +501,84 @@ ORDER BY revenue DESC; ---- sort - ├── columns: n_name:42!null revenue:49!null + ├── columns: n_name:46!null revenue:55!null ├── immutable - ├── key: (42) - ├── fd: (42)-->(49) - ├── ordering: -49 + ├── key: (46) + ├── fd: (46)-->(55) + ├── ordering: -55 └── group-by - ├── columns: n_name:42!null sum:49!null - ├── grouping columns: n_name:42!null + ├── columns: n_name:46!null sum:55!null + ├── grouping columns: n_name:46!null ├── immutable - ├── key: (42) - ├── fd: (42)-->(49) + ├── key: (46) + ├── fd: (46)-->(55) ├── project - │ ├── columns: column48:48!null n_name:42!null + │ ├── columns: column54:54!null n_name:46!null │ ├── immutable │ ├── inner-join (hash) - │ │ ├── columns: c_custkey:1!null c_nationkey:4!null o_orderkey:9!null o_custkey:10!null o_orderdate:13!null l_orderkey:18!null l_suppkey:20!null l_extendedprice:23!null l_discount:24!null s_suppkey:34!null s_nationkey:37!null n_nationkey:41!null n_name:42!null n_regionkey:43!null r_regionkey:45!null r_name:46!null + │ │ ├── columns: c_custkey:1!null c_nationkey:4!null o_orderkey:10!null o_custkey:11!null o_orderdate:14!null l_orderkey:20!null l_suppkey:22!null l_extendedprice:25!null l_discount:26!null s_suppkey:37!null s_nationkey:40!null n_nationkey:45!null n_name:46!null n_regionkey:47!null r_regionkey:50!null r_name:51!null │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ ├── fd: ()-->(46), (1)-->(4), (9)-->(10,13), (34)-->(37), (41)-->(42,43), (43)==(45), (45)==(43), (37)==(4,41), (41)==(4,37), (20)==(34), (34)==(20), (9)==(18), (18)==(9), (1)==(10), (10)==(1), (4)==(37,41) + │ │ ├── fd: ()-->(51), (1)-->(4), (10)-->(11,14), (37)-->(40), (45)-->(46,47), (47)==(50), (50)==(47), (40)==(4,45), (45)==(4,40), (22)==(37), (37)==(22), (10)==(20), (20)==(10), (1)==(11), (11)==(1), (4)==(40,45) │ │ ├── inner-join (lookup lineitem) - │ │ │ ├── columns: c_custkey:1!null c_nationkey:4!null o_orderkey:9!null o_custkey:10!null o_orderdate:13!null l_orderkey:18!null l_suppkey:20!null l_extendedprice:23!null l_discount:24!null n_nationkey:41!null n_name:42!null n_regionkey:43!null r_regionkey:45!null r_name:46!null - │ │ │ ├── key columns: [9] = [18] - │ │ │ ├── fd: ()-->(46), (9)-->(10,13), (1)-->(4), (41)-->(42,43), (43)==(45), (45)==(43), (4)==(41), (41)==(4), (1)==(10), (10)==(1), (9)==(18), (18)==(9) + │ │ │ ├── columns: c_custkey:1!null c_nationkey:4!null o_orderkey:10!null o_custkey:11!null o_orderdate:14!null l_orderkey:20!null l_suppkey:22!null l_extendedprice:25!null l_discount:26!null n_nationkey:45!null n_name:46!null n_regionkey:47!null r_regionkey:50!null r_name:51!null + │ │ │ ├── key columns: [10] = [20] + │ │ │ ├── fd: ()-->(51), (10)-->(11,14), (1)-->(4), (45)-->(46,47), (47)==(50), (50)==(47), (4)==(45), (45)==(4), (1)==(11), (11)==(1), (10)==(20), (20)==(10) │ │ │ ├── inner-join (hash) - │ │ │ │ ├── columns: c_custkey:1!null c_nationkey:4!null o_orderkey:9!null o_custkey:10!null o_orderdate:13!null n_nationkey:41!null n_name:42!null n_regionkey:43!null r_regionkey:45!null r_name:46!null + │ │ │ │ ├── columns: c_custkey:1!null c_nationkey:4!null o_orderkey:10!null o_custkey:11!null o_orderdate:14!null n_nationkey:45!null n_name:46!null n_regionkey:47!null r_regionkey:50!null r_name:51!null │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ │ ├── key: (9) - │ │ │ │ ├── fd: ()-->(46), (9)-->(10,13), (1)-->(4), (41)-->(42,43), (43)==(45), (45)==(43), (4)==(41), (41)==(4), (1)==(10), (10)==(1) + │ │ │ │ ├── key: (10) + │ │ │ │ ├── fd: ()-->(51), (10)-->(11,14), (1)-->(4), (45)-->(46,47), (47)==(50), (50)==(47), (4)==(45), (45)==(4), (1)==(11), (11)==(1) │ │ │ │ ├── index-join orders - │ │ │ │ │ ├── columns: o_orderkey:9!null o_custkey:10!null o_orderdate:13!null - │ │ │ │ │ ├── key: (9) - │ │ │ │ │ ├── fd: (9)-->(10,13) + │ │ │ │ │ ├── columns: o_orderkey:10!null o_custkey:11!null o_orderdate:14!null + │ │ │ │ │ ├── key: (10) + │ │ │ │ │ ├── fd: (10)-->(11,14) │ │ │ │ │ └── scan orders@o_od - │ │ │ │ │ ├── columns: o_orderkey:9!null o_orderdate:13!null - │ │ │ │ │ ├── constraint: /13/9: [/'1994-01-01' - /'1994-12-31'] - │ │ │ │ │ ├── key: (9) - │ │ │ │ │ └── fd: (9)-->(13) + │ │ │ │ │ ├── columns: o_orderkey:10!null o_orderdate:14!null + │ │ │ │ │ ├── constraint: /14/10: [/'1994-01-01' - /'1994-12-31'] + │ │ │ │ │ ├── key: (10) + │ │ │ │ │ └── fd: (10)-->(14) │ │ │ │ ├── inner-join (lookup customer@c_nk) - │ │ │ │ │ ├── columns: c_custkey:1!null c_nationkey:4!null n_nationkey:41!null n_name:42!null n_regionkey:43!null r_regionkey:45!null r_name:46!null - │ │ │ │ │ ├── key columns: [41] = [4] + │ │ │ │ │ ├── columns: c_custkey:1!null c_nationkey:4!null n_nationkey:45!null n_name:46!null n_regionkey:47!null r_regionkey:50!null r_name:51!null + │ │ │ │ │ ├── key columns: [45] = [4] │ │ │ │ │ ├── key: (1) - │ │ │ │ │ ├── fd: ()-->(46), (1)-->(4), (41)-->(42,43), (43)==(45), (45)==(43), (4)==(41), (41)==(4) + │ │ │ │ │ ├── fd: ()-->(51), (1)-->(4), (45)-->(46,47), (47)==(50), (50)==(47), (4)==(45), (45)==(4) │ │ │ │ │ ├── inner-join (hash) - │ │ │ │ │ │ ├── columns: n_nationkey:41!null n_name:42!null n_regionkey:43!null r_regionkey:45!null r_name:46!null + │ │ │ │ │ │ ├── columns: n_nationkey:45!null n_name:46!null n_regionkey:47!null r_regionkey:50!null r_name:51!null │ │ │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ │ │ │ ├── key: (41) - │ │ │ │ │ │ ├── fd: ()-->(46), (41)-->(42,43), (43)==(45), (45)==(43) + │ │ │ │ │ │ ├── key: (45) + │ │ │ │ │ │ ├── fd: ()-->(51), (45)-->(46,47), (47)==(50), (50)==(47) │ │ │ │ │ │ ├── scan nation - │ │ │ │ │ │ │ ├── columns: n_nationkey:41!null n_name:42!null n_regionkey:43!null - │ │ │ │ │ │ │ ├── key: (41) - │ │ │ │ │ │ │ └── fd: (41)-->(42,43) - │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: r_regionkey:45!null r_name:46!null + │ │ │ │ │ │ │ ├── columns: n_nationkey:45!null n_name:46!null n_regionkey:47!null │ │ │ │ │ │ │ ├── key: (45) - │ │ │ │ │ │ │ ├── fd: ()-->(46) + │ │ │ │ │ │ │ └── fd: (45)-->(46,47) + │ │ │ │ │ │ ├── select + │ │ │ │ │ │ │ ├── columns: r_regionkey:50!null r_name:51!null + │ │ │ │ │ │ │ ├── key: (50) + │ │ │ │ │ │ │ ├── fd: ()-->(51) │ │ │ │ │ │ │ ├── scan region - │ │ │ │ │ │ │ │ ├── columns: r_regionkey:45!null r_name:46!null - │ │ │ │ │ │ │ │ ├── key: (45) - │ │ │ │ │ │ │ │ └── fd: (45)-->(46) + │ │ │ │ │ │ │ │ ├── columns: r_regionkey:50!null r_name:51!null + │ │ │ │ │ │ │ │ ├── key: (50) + │ │ │ │ │ │ │ │ └── fd: (50)-->(51) │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── r_name:46 = 'ASIA' [outer=(46), constraints=(/46: [/'ASIA' - /'ASIA']; tight), fd=()-->(46)] + │ │ │ │ │ │ │ └── r_name:51 = 'ASIA' [outer=(51), constraints=(/51: [/'ASIA' - /'ASIA']; tight), fd=()-->(51)] │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── n_regionkey:43 = r_regionkey:45 [outer=(43,45), constraints=(/43: (/NULL - ]; /45: (/NULL - ]), fd=(43)==(45), (45)==(43)] + │ │ │ │ │ │ └── n_regionkey:47 = r_regionkey:50 [outer=(47,50), constraints=(/47: (/NULL - ]; /50: (/NULL - ]), fd=(47)==(50), (50)==(47)] │ │ │ │ │ └── filters (true) │ │ │ │ └── filters - │ │ │ │ └── c_custkey:1 = o_custkey:10 [outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] + │ │ │ │ └── c_custkey:1 = o_custkey:11 [outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ]), fd=(1)==(11), (11)==(1)] │ │ │ └── filters (true) │ │ ├── scan supplier@s_nk - │ │ │ ├── columns: s_suppkey:34!null s_nationkey:37!null - │ │ │ ├── key: (34) - │ │ │ └── fd: (34)-->(37) + │ │ │ ├── columns: s_suppkey:37!null s_nationkey:40!null + │ │ │ ├── key: (37) + │ │ │ └── fd: (37)-->(40) │ │ └── filters - │ │ ├── l_suppkey:20 = s_suppkey:34 [outer=(20,34), constraints=(/20: (/NULL - ]; /34: (/NULL - ]), fd=(20)==(34), (34)==(20)] - │ │ └── c_nationkey:4 = s_nationkey:37 [outer=(4,37), constraints=(/4: (/NULL - ]; /37: (/NULL - ]), fd=(4)==(37), (37)==(4)] + │ │ ├── l_suppkey:22 = s_suppkey:37 [outer=(22,37), constraints=(/22: (/NULL - ]; /37: (/NULL - ]), fd=(22)==(37), (37)==(22)] + │ │ └── c_nationkey:4 = s_nationkey:40 [outer=(4,40), constraints=(/4: (/NULL - ]; /40: (/NULL - ]), fd=(4)==(40), (40)==(4)] │ └── projections - │ └── l_extendedprice:23 * (1.0 - l_discount:24) [as=column48:48, outer=(23,24), immutable] + │ └── l_extendedprice:25 * (1.0 - l_discount:26) [as=column54:54, outer=(25,26), immutable] └── aggregations - └── sum [as=sum:49, outer=(48)] - └── column48:48 + └── sum [as=sum:55, outer=(54)] + └── column54:54 # -------------------------------------------------- # Q6 @@ -607,13 +607,13 @@ WHERE AND l_quantity < 24; ---- scalar-group-by - ├── columns: revenue:18 + ├── columns: revenue:19 ├── cardinality: [1 - 1] ├── immutable ├── key: () - ├── fd: ()-->(18) + ├── fd: ()-->(19) ├── project - │ ├── columns: column17:17!null + │ ├── columns: column18:18!null │ ├── immutable │ ├── select │ │ ├── columns: l_quantity:5!null l_extendedprice:6!null l_discount:7!null l_shipdate:11!null @@ -628,10 +628,10 @@ scalar-group-by │ │ ├── (l_discount:7 >= 0.05) AND (l_discount:7 <= 0.07) [outer=(7), constraints=(/7: [/0.05 - /0.07]; tight)] │ │ └── l_quantity:5 < 24.0 [outer=(5), constraints=(/5: (/NULL - /23.999999999999996]; tight)] │ └── projections - │ └── l_extendedprice:6 * l_discount:7 [as=column17:17, outer=(6,7), immutable] + │ └── l_extendedprice:6 * l_discount:7 [as=column18:18, outer=(6,7), immutable] └── aggregations - └── sum [as=sum:18, outer=(17)] - └── column17:17 + └── sum [as=sum:19, outer=(18)] + └── column18:18 # -------------------------------------------------- # Q7 @@ -686,76 +686,76 @@ ORDER BY l_year; ---- sort - ├── columns: supp_nation:42!null cust_nation:46!null l_year:49 revenue:51!null + ├── columns: supp_nation:46!null cust_nation:51!null l_year:55 revenue:57!null ├── immutable - ├── key: (42,46,49) - ├── fd: (42,46,49)-->(51) - ├── ordering: +42,+46,+49 + ├── key: (46,51,55) + ├── fd: (46,51,55)-->(57) + ├── ordering: +46,+51,+55 └── group-by - ├── columns: n1.n_name:42!null n2.n_name:46!null l_year:49 sum:51!null - ├── grouping columns: n1.n_name:42!null n2.n_name:46!null l_year:49 + ├── columns: n1.n_name:46!null n2.n_name:51!null l_year:55 sum:57!null + ├── grouping columns: n1.n_name:46!null n2.n_name:51!null l_year:55 ├── immutable - ├── key: (42,46,49) - ├── fd: (42,46,49)-->(51) + ├── key: (46,51,55) + ├── fd: (46,51,55)-->(57) ├── project - │ ├── columns: l_year:49 volume:50!null n1.n_name:42!null n2.n_name:46!null + │ ├── columns: l_year:55 volume:56!null n1.n_name:46!null n2.n_name:51!null │ ├── immutable │ ├── inner-join (hash) - │ │ ├── columns: s_suppkey:1!null s_nationkey:4!null l_orderkey:8!null l_suppkey:10!null l_extendedprice:13!null l_discount:14!null l_shipdate:18!null o_orderkey:24!null o_custkey:25!null c_custkey:33!null c_nationkey:36!null n1.n_nationkey:41!null n1.n_name:42!null n2.n_nationkey:45!null n2.n_name:46!null + │ │ ├── columns: s_suppkey:1!null s_nationkey:4!null l_orderkey:9!null l_suppkey:11!null l_extendedprice:14!null l_discount:15!null l_shipdate:19!null o_orderkey:26!null o_custkey:27!null c_custkey:36!null c_nationkey:39!null n1.n_nationkey:45!null n1.n_name:46!null n2.n_nationkey:50!null n2.n_name:51!null │ │ ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) - │ │ ├── fd: (1)-->(4), (24)-->(25), (33)-->(36), (41)-->(42), (45)-->(46), (36)==(45), (45)==(36), (25)==(33), (33)==(25), (8)==(24), (24)==(8), (1)==(10), (10)==(1), (4)==(41), (41)==(4) + │ │ ├── fd: (1)-->(4), (26)-->(27), (36)-->(39), (45)-->(46), (50)-->(51), (39)==(50), (50)==(39), (27)==(36), (36)==(27), (9)==(26), (26)==(9), (1)==(11), (11)==(1), (4)==(45), (45)==(4) │ │ ├── scan customer@c_nk - │ │ │ ├── columns: c_custkey:33!null c_nationkey:36!null - │ │ │ ├── key: (33) - │ │ │ └── fd: (33)-->(36) + │ │ │ ├── columns: c_custkey:36!null c_nationkey:39!null + │ │ │ ├── key: (36) + │ │ │ └── fd: (36)-->(39) │ │ ├── inner-join (lookup orders) - │ │ │ ├── columns: s_suppkey:1!null s_nationkey:4!null l_orderkey:8!null l_suppkey:10!null l_extendedprice:13!null l_discount:14!null l_shipdate:18!null o_orderkey:24!null o_custkey:25!null n1.n_nationkey:41!null n1.n_name:42!null n2.n_nationkey:45!null n2.n_name:46!null - │ │ │ ├── key columns: [8] = [24] + │ │ │ ├── columns: s_suppkey:1!null s_nationkey:4!null l_orderkey:9!null l_suppkey:11!null l_extendedprice:14!null l_discount:15!null l_shipdate:19!null o_orderkey:26!null o_custkey:27!null n1.n_nationkey:45!null n1.n_name:46!null n2.n_nationkey:50!null n2.n_name:51!null + │ │ │ ├── key columns: [9] = [26] │ │ │ ├── lookup columns are key - │ │ │ ├── fd: (24)-->(25), (1)-->(4), (41)-->(42), (45)-->(46), (4)==(41), (41)==(4), (1)==(10), (10)==(1), (8)==(24), (24)==(8) + │ │ │ ├── fd: (26)-->(27), (1)-->(4), (45)-->(46), (50)-->(51), (4)==(45), (45)==(4), (1)==(11), (11)==(1), (9)==(26), (26)==(9) │ │ │ ├── inner-join (lookup lineitem) - │ │ │ │ ├── columns: s_suppkey:1!null s_nationkey:4!null l_orderkey:8!null l_suppkey:10!null l_extendedprice:13!null l_discount:14!null l_shipdate:18!null n1.n_nationkey:41!null n1.n_name:42!null n2.n_nationkey:45!null n2.n_name:46!null - │ │ │ │ ├── key columns: [8 11] = [8 11] + │ │ │ │ ├── columns: s_suppkey:1!null s_nationkey:4!null l_orderkey:9!null l_suppkey:11!null l_extendedprice:14!null l_discount:15!null l_shipdate:19!null n1.n_nationkey:45!null n1.n_name:46!null n2.n_nationkey:50!null n2.n_name:51!null + │ │ │ │ ├── key columns: [9 12] = [9 12] │ │ │ │ ├── lookup columns are key - │ │ │ │ ├── fd: (1)-->(4), (41)-->(42), (45)-->(46), (4)==(41), (41)==(4), (1)==(10), (10)==(1) + │ │ │ │ ├── fd: (1)-->(4), (45)-->(46), (50)-->(51), (4)==(45), (45)==(4), (1)==(11), (11)==(1) │ │ │ │ ├── inner-join (lookup lineitem@l_sk) - │ │ │ │ │ ├── columns: s_suppkey:1!null s_nationkey:4!null l_orderkey:8!null l_suppkey:10!null l_linenumber:11!null n1.n_nationkey:41!null n1.n_name:42!null n2.n_nationkey:45!null n2.n_name:46!null - │ │ │ │ │ ├── key columns: [1] = [10] - │ │ │ │ │ ├── key: (8,11,45) - │ │ │ │ │ ├── fd: (1)-->(4), (41)-->(42), (45)-->(46), (4)==(41), (41)==(4), (8,11)-->(10), (1)==(10), (10)==(1) + │ │ │ │ │ ├── columns: s_suppkey:1!null s_nationkey:4!null l_orderkey:9!null l_suppkey:11!null l_linenumber:12!null n1.n_nationkey:45!null n1.n_name:46!null n2.n_nationkey:50!null n2.n_name:51!null + │ │ │ │ │ ├── key columns: [1] = [11] + │ │ │ │ │ ├── key: (9,12,50) + │ │ │ │ │ ├── fd: (1)-->(4), (45)-->(46), (50)-->(51), (4)==(45), (45)==(4), (9,12)-->(11), (1)==(11), (11)==(1) │ │ │ │ │ ├── inner-join (lookup supplier@s_nk) - │ │ │ │ │ │ ├── columns: s_suppkey:1!null s_nationkey:4!null n1.n_nationkey:41!null n1.n_name:42!null n2.n_nationkey:45!null n2.n_name:46!null - │ │ │ │ │ │ ├── key columns: [41] = [4] - │ │ │ │ │ │ ├── key: (1,45) - │ │ │ │ │ │ ├── fd: (1)-->(4), (41)-->(42), (45)-->(46), (4)==(41), (41)==(4) + │ │ │ │ │ │ ├── columns: s_suppkey:1!null s_nationkey:4!null n1.n_nationkey:45!null n1.n_name:46!null n2.n_nationkey:50!null n2.n_name:51!null + │ │ │ │ │ │ ├── key columns: [45] = [4] + │ │ │ │ │ │ ├── key: (1,50) + │ │ │ │ │ │ ├── fd: (1)-->(4), (45)-->(46), (50)-->(51), (4)==(45), (45)==(4) │ │ │ │ │ │ ├── inner-join (cross) - │ │ │ │ │ │ │ ├── columns: n1.n_nationkey:41!null n1.n_name:42!null n2.n_nationkey:45!null n2.n_name:46!null - │ │ │ │ │ │ │ ├── key: (41,45) - │ │ │ │ │ │ │ ├── fd: (41)-->(42), (45)-->(46) + │ │ │ │ │ │ │ ├── columns: n1.n_nationkey:45!null n1.n_name:46!null n2.n_nationkey:50!null n2.n_name:51!null + │ │ │ │ │ │ │ ├── key: (45,50) + │ │ │ │ │ │ │ ├── fd: (45)-->(46), (50)-->(51) │ │ │ │ │ │ │ ├── scan n1 - │ │ │ │ │ │ │ │ ├── columns: n1.n_nationkey:41!null n1.n_name:42!null - │ │ │ │ │ │ │ │ ├── key: (41) - │ │ │ │ │ │ │ │ └── fd: (41)-->(42) - │ │ │ │ │ │ │ ├── scan n2 - │ │ │ │ │ │ │ │ ├── columns: n2.n_nationkey:45!null n2.n_name:46!null + │ │ │ │ │ │ │ │ ├── columns: n1.n_nationkey:45!null n1.n_name:46!null │ │ │ │ │ │ │ │ ├── key: (45) │ │ │ │ │ │ │ │ └── fd: (45)-->(46) + │ │ │ │ │ │ │ ├── scan n2 + │ │ │ │ │ │ │ │ ├── columns: n2.n_nationkey:50!null n2.n_name:51!null + │ │ │ │ │ │ │ │ ├── key: (50) + │ │ │ │ │ │ │ │ └── fd: (50)-->(51) │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── ((n1.n_name:42 = 'FRANCE') AND (n2.n_name:46 = 'GERMANY')) OR ((n1.n_name:42 = 'GERMANY') AND (n2.n_name:46 = 'FRANCE')) [outer=(42,46), constraints=(/42: [/'FRANCE' - /'FRANCE'] [/'GERMANY' - /'GERMANY']; /46: [/'FRANCE' - /'FRANCE'] [/'GERMANY' - /'GERMANY'])] + │ │ │ │ │ │ │ └── ((n1.n_name:46 = 'FRANCE') AND (n2.n_name:51 = 'GERMANY')) OR ((n1.n_name:46 = 'GERMANY') AND (n2.n_name:51 = 'FRANCE')) [outer=(46,51), constraints=(/46: [/'FRANCE' - /'FRANCE'] [/'GERMANY' - /'GERMANY']; /51: [/'FRANCE' - /'FRANCE'] [/'GERMANY' - /'GERMANY'])] │ │ │ │ │ │ └── filters (true) │ │ │ │ │ └── filters (true) │ │ │ │ └── filters - │ │ │ │ └── (l_shipdate:18 >= '1995-01-01') AND (l_shipdate:18 <= '1996-12-31') [outer=(18), constraints=(/18: [/'1995-01-01' - /'1996-12-31']; tight)] + │ │ │ │ └── (l_shipdate:19 >= '1995-01-01') AND (l_shipdate:19 <= '1996-12-31') [outer=(19), constraints=(/19: [/'1995-01-01' - /'1996-12-31']; tight)] │ │ │ └── filters (true) │ │ └── filters - │ │ ├── c_custkey:33 = o_custkey:25 [outer=(25,33), constraints=(/25: (/NULL - ]; /33: (/NULL - ]), fd=(25)==(33), (33)==(25)] - │ │ └── c_nationkey:36 = n2.n_nationkey:45 [outer=(36,45), constraints=(/36: (/NULL - ]; /45: (/NULL - ]), fd=(36)==(45), (45)==(36)] + │ │ ├── c_custkey:36 = o_custkey:27 [outer=(27,36), constraints=(/27: (/NULL - ]; /36: (/NULL - ]), fd=(27)==(36), (36)==(27)] + │ │ └── c_nationkey:39 = n2.n_nationkey:50 [outer=(39,50), constraints=(/39: (/NULL - ]; /50: (/NULL - ]), fd=(39)==(50), (50)==(39)] │ └── projections - │ ├── extract('year', l_shipdate:18) [as=l_year:49, outer=(18), immutable] - │ └── l_extendedprice:13 * (1.0 - l_discount:14) [as=volume:50, outer=(13,14), immutable] + │ ├── extract('year', l_shipdate:19) [as=l_year:55, outer=(19), immutable] + │ └── l_extendedprice:14 * (1.0 - l_discount:15) [as=volume:56, outer=(14,15), immutable] └── aggregations - └── sum [as=sum:51, outer=(50)] - └── volume:50 + └── sum [as=sum:57, outer=(56)] + └── volume:56 # -------------------------------------------------- # Q8 @@ -809,89 +809,89 @@ ORDER BY o_year; ---- sort - ├── columns: o_year:61 mkt_share:66!null + ├── columns: o_year:69 mkt_share:74!null ├── immutable - ├── key: (61) - ├── fd: (61)-->(66) - ├── ordering: +61 + ├── key: (69) + ├── fd: (69)-->(74) + ├── ordering: +69 └── project - ├── columns: mkt_share:66!null o_year:61 + ├── columns: mkt_share:74!null o_year:69 ├── immutable - ├── key: (61) - ├── fd: (61)-->(66) + ├── key: (69) + ├── fd: (69)-->(74) ├── group-by - │ ├── columns: o_year:61 sum:64!null sum:65!null - │ ├── grouping columns: o_year:61 + │ ├── columns: o_year:69 sum:72!null sum:73!null + │ ├── grouping columns: o_year:69 │ ├── immutable - │ ├── key: (61) - │ ├── fd: (61)-->(64,65) + │ ├── key: (69) + │ ├── fd: (69)-->(72,73) │ ├── project - │ │ ├── columns: column63:63!null o_year:61 volume:62!null + │ │ ├── columns: column71:71!null o_year:69 volume:70!null │ │ ├── immutable │ │ ├── project - │ │ │ ├── columns: o_year:61 volume:62!null n2.n_name:55!null + │ │ │ ├── columns: o_year:69 volume:70!null n2.n_name:61!null │ │ │ ├── immutable │ │ │ ├── inner-join (hash) - │ │ │ │ ├── columns: p_partkey:1!null p_type:5!null s_suppkey:10!null s_nationkey:13!null l_orderkey:17!null l_partkey:18!null l_suppkey:19!null l_extendedprice:22!null l_discount:23!null o_orderkey:33!null o_custkey:34!null o_orderdate:37!null c_custkey:42!null c_nationkey:45!null n1.n_nationkey:50!null n1.n_regionkey:52!null n2.n_nationkey:54!null n2.n_name:55!null r_regionkey:58!null r_name:59!null + │ │ │ │ ├── columns: p_partkey:1!null p_type:5!null s_suppkey:11!null s_nationkey:14!null l_orderkey:19!null l_partkey:20!null l_suppkey:21!null l_extendedprice:24!null l_discount:25!null o_orderkey:36!null o_custkey:37!null o_orderdate:40!null c_custkey:46!null c_nationkey:49!null n1.n_nationkey:55!null n1.n_regionkey:57!null n2.n_nationkey:60!null n2.n_name:61!null r_regionkey:65!null r_name:66!null │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ │ │ │ ├── fd: ()-->(5,59), (10)-->(13), (33)-->(34,37), (42)-->(45), (50)-->(52), (52)==(58), (58)==(52), (45)==(50), (50)==(45), (34)==(42), (42)==(34), (17)==(33), (33)==(17), (10)==(19), (19)==(10), (54)-->(55), (13)==(54), (54)==(13), (1)==(18), (18)==(1) + │ │ │ │ ├── fd: ()-->(5,66), (11)-->(14), (36)-->(37,40), (46)-->(49), (55)-->(57), (57)==(65), (65)==(57), (49)==(55), (55)==(49), (37)==(46), (46)==(37), (19)==(36), (36)==(19), (11)==(21), (21)==(11), (60)-->(61), (14)==(60), (60)==(14), (1)==(20), (20)==(1) │ │ │ │ ├── inner-join (hash) - │ │ │ │ │ ├── columns: p_partkey:1!null p_type:5!null s_suppkey:10!null s_nationkey:13!null l_orderkey:17!null l_partkey:18!null l_suppkey:19!null l_extendedprice:22!null l_discount:23!null o_orderkey:33!null o_custkey:34!null o_orderdate:37!null c_custkey:42!null c_nationkey:45!null n1.n_nationkey:50!null n1.n_regionkey:52!null r_regionkey:58!null r_name:59!null + │ │ │ │ │ ├── columns: p_partkey:1!null p_type:5!null s_suppkey:11!null s_nationkey:14!null l_orderkey:19!null l_partkey:20!null l_suppkey:21!null l_extendedprice:24!null l_discount:25!null o_orderkey:36!null o_custkey:37!null o_orderdate:40!null c_custkey:46!null c_nationkey:49!null n1.n_nationkey:55!null n1.n_regionkey:57!null r_regionkey:65!null r_name:66!null │ │ │ │ │ ├── multiplicity: left-rows(zero-or-more), right-rows(exactly-one) - │ │ │ │ │ ├── fd: ()-->(5,59), (10)-->(13), (33)-->(34,37), (42)-->(45), (50)-->(52), (52)==(58), (58)==(52), (45)==(50), (50)==(45), (34)==(42), (42)==(34), (17)==(33), (33)==(17), (10)==(19), (19)==(10), (1)==(18), (18)==(1) + │ │ │ │ │ ├── fd: ()-->(5,66), (11)-->(14), (36)-->(37,40), (46)-->(49), (55)-->(57), (57)==(65), (65)==(57), (49)==(55), (55)==(49), (37)==(46), (46)==(37), (19)==(36), (36)==(19), (11)==(21), (21)==(11), (1)==(20), (20)==(1) │ │ │ │ │ ├── scan supplier@s_nk - │ │ │ │ │ │ ├── columns: s_suppkey:10!null s_nationkey:13!null - │ │ │ │ │ │ ├── key: (10) - │ │ │ │ │ │ └── fd: (10)-->(13) + │ │ │ │ │ │ ├── columns: s_suppkey:11!null s_nationkey:14!null + │ │ │ │ │ │ ├── key: (11) + │ │ │ │ │ │ └── fd: (11)-->(14) │ │ │ │ │ ├── inner-join (hash) - │ │ │ │ │ │ ├── columns: p_partkey:1!null p_type:5!null l_orderkey:17!null l_partkey:18!null l_suppkey:19!null l_extendedprice:22!null l_discount:23!null o_orderkey:33!null o_custkey:34!null o_orderdate:37!null c_custkey:42!null c_nationkey:45!null n1.n_nationkey:50!null n1.n_regionkey:52!null r_regionkey:58!null r_name:59!null + │ │ │ │ │ │ ├── columns: p_partkey:1!null p_type:5!null l_orderkey:19!null l_partkey:20!null l_suppkey:21!null l_extendedprice:24!null l_discount:25!null o_orderkey:36!null o_custkey:37!null o_orderdate:40!null c_custkey:46!null c_nationkey:49!null n1.n_nationkey:55!null n1.n_regionkey:57!null r_regionkey:65!null r_name:66!null │ │ │ │ │ │ ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) - │ │ │ │ │ │ ├── fd: ()-->(5,59), (33)-->(34,37), (42)-->(45), (50)-->(52), (52)==(58), (58)==(52), (45)==(50), (50)==(45), (34)==(42), (42)==(34), (17)==(33), (33)==(17), (1)==(18), (18)==(1) + │ │ │ │ │ │ ├── fd: ()-->(5,66), (36)-->(37,40), (46)-->(49), (55)-->(57), (57)==(65), (65)==(57), (49)==(55), (55)==(49), (37)==(46), (46)==(37), (19)==(36), (36)==(19), (1)==(20), (20)==(1) │ │ │ │ │ │ ├── inner-join (lookup customer@c_nk) - │ │ │ │ │ │ │ ├── columns: c_custkey:42!null c_nationkey:45!null n1.n_nationkey:50!null n1.n_regionkey:52!null r_regionkey:58!null r_name:59!null - │ │ │ │ │ │ │ ├── key columns: [50] = [45] - │ │ │ │ │ │ │ ├── key: (42) - │ │ │ │ │ │ │ ├── fd: ()-->(59), (42)-->(45), (50)-->(52), (52)==(58), (58)==(52), (45)==(50), (50)==(45) + │ │ │ │ │ │ │ ├── columns: c_custkey:46!null c_nationkey:49!null n1.n_nationkey:55!null n1.n_regionkey:57!null r_regionkey:65!null r_name:66!null + │ │ │ │ │ │ │ ├── key columns: [55] = [49] + │ │ │ │ │ │ │ ├── key: (46) + │ │ │ │ │ │ │ ├── fd: ()-->(66), (46)-->(49), (55)-->(57), (57)==(65), (65)==(57), (49)==(55), (55)==(49) │ │ │ │ │ │ │ ├── inner-join (merge) - │ │ │ │ │ │ │ │ ├── columns: n1.n_nationkey:50!null n1.n_regionkey:52!null r_regionkey:58!null r_name:59!null - │ │ │ │ │ │ │ │ ├── left ordering: +52 - │ │ │ │ │ │ │ │ ├── right ordering: +58 - │ │ │ │ │ │ │ │ ├── key: (50) - │ │ │ │ │ │ │ │ ├── fd: ()-->(59), (50)-->(52), (52)==(58), (58)==(52) + │ │ │ │ │ │ │ │ ├── columns: n1.n_nationkey:55!null n1.n_regionkey:57!null r_regionkey:65!null r_name:66!null + │ │ │ │ │ │ │ │ ├── left ordering: +57 + │ │ │ │ │ │ │ │ ├── right ordering: +65 + │ │ │ │ │ │ │ │ ├── key: (55) + │ │ │ │ │ │ │ │ ├── fd: ()-->(66), (55)-->(57), (57)==(65), (65)==(57) │ │ │ │ │ │ │ │ ├── scan n1@n_rk - │ │ │ │ │ │ │ │ │ ├── columns: n1.n_nationkey:50!null n1.n_regionkey:52!null - │ │ │ │ │ │ │ │ │ ├── key: (50) - │ │ │ │ │ │ │ │ │ ├── fd: (50)-->(52) - │ │ │ │ │ │ │ │ │ └── ordering: +52 + │ │ │ │ │ │ │ │ │ ├── columns: n1.n_nationkey:55!null n1.n_regionkey:57!null + │ │ │ │ │ │ │ │ │ ├── key: (55) + │ │ │ │ │ │ │ │ │ ├── fd: (55)-->(57) + │ │ │ │ │ │ │ │ │ └── ordering: +57 │ │ │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ │ │ ├── columns: r_regionkey:58!null r_name:59!null - │ │ │ │ │ │ │ │ │ ├── key: (58) - │ │ │ │ │ │ │ │ │ ├── fd: ()-->(59) - │ │ │ │ │ │ │ │ │ ├── ordering: +58 opt(59) [actual: +58] + │ │ │ │ │ │ │ │ │ ├── columns: r_regionkey:65!null r_name:66!null + │ │ │ │ │ │ │ │ │ ├── key: (65) + │ │ │ │ │ │ │ │ │ ├── fd: ()-->(66) + │ │ │ │ │ │ │ │ │ ├── ordering: +65 opt(66) [actual: +65] │ │ │ │ │ │ │ │ │ ├── scan region - │ │ │ │ │ │ │ │ │ │ ├── columns: r_regionkey:58!null r_name:59!null - │ │ │ │ │ │ │ │ │ │ ├── key: (58) - │ │ │ │ │ │ │ │ │ │ ├── fd: (58)-->(59) - │ │ │ │ │ │ │ │ │ │ └── ordering: +58 opt(59) [actual: +58] + │ │ │ │ │ │ │ │ │ │ ├── columns: r_regionkey:65!null r_name:66!null + │ │ │ │ │ │ │ │ │ │ ├── key: (65) + │ │ │ │ │ │ │ │ │ │ ├── fd: (65)-->(66) + │ │ │ │ │ │ │ │ │ │ └── ordering: +65 opt(66) [actual: +65] │ │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ │ └── r_name:59 = 'AMERICA' [outer=(59), constraints=(/59: [/'AMERICA' - /'AMERICA']; tight), fd=()-->(59)] + │ │ │ │ │ │ │ │ │ └── r_name:66 = 'AMERICA' [outer=(66), constraints=(/66: [/'AMERICA' - /'AMERICA']; tight), fd=()-->(66)] │ │ │ │ │ │ │ │ └── filters (true) │ │ │ │ │ │ │ └── filters (true) │ │ │ │ │ │ ├── inner-join (lookup orders) - │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_type:5!null l_orderkey:17!null l_partkey:18!null l_suppkey:19!null l_extendedprice:22!null l_discount:23!null o_orderkey:33!null o_custkey:34!null o_orderdate:37!null - │ │ │ │ │ │ │ ├── key columns: [17] = [33] + │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_type:5!null l_orderkey:19!null l_partkey:20!null l_suppkey:21!null l_extendedprice:24!null l_discount:25!null o_orderkey:36!null o_custkey:37!null o_orderdate:40!null + │ │ │ │ │ │ │ ├── key columns: [19] = [36] │ │ │ │ │ │ │ ├── lookup columns are key - │ │ │ │ │ │ │ ├── fd: ()-->(5), (33)-->(34,37), (17)==(33), (33)==(17), (1)==(18), (18)==(1) + │ │ │ │ │ │ │ ├── fd: ()-->(5), (36)-->(37,40), (19)==(36), (36)==(19), (1)==(20), (20)==(1) │ │ │ │ │ │ │ ├── inner-join (lookup lineitem) - │ │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_type:5!null l_orderkey:17!null l_partkey:18!null l_suppkey:19!null l_extendedprice:22!null l_discount:23!null - │ │ │ │ │ │ │ │ ├── key columns: [17 20] = [17 20] + │ │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_type:5!null l_orderkey:19!null l_partkey:20!null l_suppkey:21!null l_extendedprice:24!null l_discount:25!null + │ │ │ │ │ │ │ │ ├── key columns: [19 22] = [19 22] │ │ │ │ │ │ │ │ ├── lookup columns are key - │ │ │ │ │ │ │ │ ├── fd: ()-->(5), (1)==(18), (18)==(1) + │ │ │ │ │ │ │ │ ├── fd: ()-->(5), (1)==(20), (20)==(1) │ │ │ │ │ │ │ │ ├── inner-join (lookup lineitem@l_pk) - │ │ │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_type:5!null l_orderkey:17!null l_partkey:18!null l_linenumber:20!null - │ │ │ │ │ │ │ │ │ ├── key columns: [1] = [18] - │ │ │ │ │ │ │ │ │ ├── key: (17,20) - │ │ │ │ │ │ │ │ │ ├── fd: ()-->(5), (17,20)-->(18), (1)==(18), (18)==(1) + │ │ │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_type:5!null l_orderkey:19!null l_partkey:20!null l_linenumber:22!null + │ │ │ │ │ │ │ │ │ ├── key columns: [1] = [20] + │ │ │ │ │ │ │ │ │ ├── key: (19,22) + │ │ │ │ │ │ │ │ │ ├── fd: ()-->(5), (19,22)-->(20), (1)==(20), (20)==(1) │ │ │ │ │ │ │ │ │ ├── select │ │ │ │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_type:5!null │ │ │ │ │ │ │ │ │ │ ├── key: (1) @@ -905,29 +905,29 @@ sort │ │ │ │ │ │ │ │ │ └── filters (true) │ │ │ │ │ │ │ │ └── filters (true) │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── (o_orderdate:37 >= '1995-01-01') AND (o_orderdate:37 <= '1996-12-31') [outer=(37), constraints=(/37: [/'1995-01-01' - /'1996-12-31']; tight)] + │ │ │ │ │ │ │ └── (o_orderdate:40 >= '1995-01-01') AND (o_orderdate:40 <= '1996-12-31') [outer=(40), constraints=(/40: [/'1995-01-01' - /'1996-12-31']; tight)] │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── o_custkey:34 = c_custkey:42 [outer=(34,42), constraints=(/34: (/NULL - ]; /42: (/NULL - ]), fd=(34)==(42), (42)==(34)] + │ │ │ │ │ │ └── o_custkey:37 = c_custkey:46 [outer=(37,46), constraints=(/37: (/NULL - ]; /46: (/NULL - ]), fd=(37)==(46), (46)==(37)] │ │ │ │ │ └── filters - │ │ │ │ │ └── s_suppkey:10 = l_suppkey:19 [outer=(10,19), constraints=(/10: (/NULL - ]; /19: (/NULL - ]), fd=(10)==(19), (19)==(10)] + │ │ │ │ │ └── s_suppkey:11 = l_suppkey:21 [outer=(11,21), constraints=(/11: (/NULL - ]; /21: (/NULL - ]), fd=(11)==(21), (21)==(11)] │ │ │ │ ├── scan n2 - │ │ │ │ │ ├── columns: n2.n_nationkey:54!null n2.n_name:55!null - │ │ │ │ │ ├── key: (54) - │ │ │ │ │ └── fd: (54)-->(55) + │ │ │ │ │ ├── columns: n2.n_nationkey:60!null n2.n_name:61!null + │ │ │ │ │ ├── key: (60) + │ │ │ │ │ └── fd: (60)-->(61) │ │ │ │ └── filters - │ │ │ │ └── s_nationkey:13 = n2.n_nationkey:54 [outer=(13,54), constraints=(/13: (/NULL - ]; /54: (/NULL - ]), fd=(13)==(54), (54)==(13)] + │ │ │ │ └── s_nationkey:14 = n2.n_nationkey:60 [outer=(14,60), constraints=(/14: (/NULL - ]; /60: (/NULL - ]), fd=(14)==(60), (60)==(14)] │ │ │ └── projections - │ │ │ ├── extract('year', o_orderdate:37) [as=o_year:61, outer=(37), immutable] - │ │ │ └── l_extendedprice:22 * (1.0 - l_discount:23) [as=volume:62, outer=(22,23), immutable] + │ │ │ ├── extract('year', o_orderdate:40) [as=o_year:69, outer=(40), immutable] + │ │ │ └── l_extendedprice:24 * (1.0 - l_discount:25) [as=volume:70, outer=(24,25), immutable] │ │ └── projections - │ │ └── CASE WHEN n2.n_name:55 = 'BRAZIL' THEN volume:62 ELSE 0.0 END [as=column63:63, outer=(55,62)] + │ │ └── CASE WHEN n2.n_name:61 = 'BRAZIL' THEN volume:70 ELSE 0.0 END [as=column71:71, outer=(61,70)] │ └── aggregations - │ ├── sum [as=sum:64, outer=(63)] - │ │ └── column63:63 - │ └── sum [as=sum:65, outer=(62)] - │ └── volume:62 + │ ├── sum [as=sum:72, outer=(71)] + │ │ └── column71:71 + │ └── sum [as=sum:73, outer=(70)] + │ └── volume:70 └── projections - └── sum:64 / sum:65 [as=mkt_share:66, outer=(64,65), immutable] + └── sum:72 / sum:73 [as=mkt_share:74, outer=(72,73), immutable] # -------------------------------------------------- # Q9 @@ -979,53 +979,53 @@ ORDER BY o_year DESC; ---- sort - ├── columns: nation:48!null o_year:51 sum_profit:53!null + ├── columns: nation:53!null o_year:57 sum_profit:59!null ├── immutable - ├── key: (48,51) - ├── fd: (48,51)-->(53) - ├── ordering: +48,-51 + ├── key: (53,57) + ├── fd: (53,57)-->(59) + ├── ordering: +53,-57 └── group-by - ├── columns: n_name:48!null o_year:51 sum:53!null - ├── grouping columns: n_name:48!null o_year:51 + ├── columns: n_name:53!null o_year:57 sum:59!null + ├── grouping columns: n_name:53!null o_year:57 ├── immutable - ├── key: (48,51) - ├── fd: (48,51)-->(53) + ├── key: (53,57) + ├── fd: (53,57)-->(59) ├── project - │ ├── columns: o_year:51 amount:52!null n_name:48!null + │ ├── columns: o_year:57 amount:58!null n_name:53!null │ ├── immutable │ ├── inner-join (hash) - │ │ ├── columns: p_partkey:1!null p_name:2!null s_suppkey:10!null s_nationkey:13!null l_orderkey:17!null l_partkey:18!null l_suppkey:19!null l_quantity:21!null l_extendedprice:22!null l_discount:23!null ps_partkey:33!null ps_suppkey:34!null ps_supplycost:36!null o_orderkey:38!null o_orderdate:42!null n_nationkey:47!null n_name:48!null + │ │ ├── columns: p_partkey:1!null p_name:2!null s_suppkey:11!null s_nationkey:14!null l_orderkey:19!null l_partkey:20!null l_suppkey:21!null l_quantity:23!null l_extendedprice:24!null l_discount:25!null ps_partkey:36!null ps_suppkey:37!null ps_supplycost:39!null o_orderkey:42!null o_orderdate:46!null n_nationkey:52!null n_name:53!null │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ │ ├── fd: (1)-->(2), (10)-->(13), (33,34)-->(36), (19)==(10,34), (34)==(10,19), (18)==(1,33), (33)==(1,18), (38)-->(42), (17)==(38), (38)==(17), (10)==(19,34), (47)-->(48), (13)==(47), (47)==(13), (1)==(18,33) + │ │ ├── fd: (1)-->(2), (11)-->(14), (36,37)-->(39), (21)==(11,37), (37)==(11,21), (20)==(1,36), (36)==(1,20), (42)-->(46), (19)==(42), (42)==(19), (11)==(21,37), (52)-->(53), (14)==(52), (52)==(14), (1)==(20,36) │ │ ├── inner-join (hash) - │ │ │ ├── columns: p_partkey:1!null p_name:2!null s_suppkey:10!null s_nationkey:13!null l_orderkey:17!null l_partkey:18!null l_suppkey:19!null l_quantity:21!null l_extendedprice:22!null l_discount:23!null ps_partkey:33!null ps_suppkey:34!null ps_supplycost:36!null o_orderkey:38!null o_orderdate:42!null + │ │ │ ├── columns: p_partkey:1!null p_name:2!null s_suppkey:11!null s_nationkey:14!null l_orderkey:19!null l_partkey:20!null l_suppkey:21!null l_quantity:23!null l_extendedprice:24!null l_discount:25!null ps_partkey:36!null ps_suppkey:37!null ps_supplycost:39!null o_orderkey:42!null o_orderdate:46!null │ │ │ ├── multiplicity: left-rows(zero-or-more), right-rows(exactly-one) - │ │ │ ├── fd: (1)-->(2), (10)-->(13), (33,34)-->(36), (19)==(10,34), (34)==(10,19), (18)==(1,33), (33)==(1,18), (38)-->(42), (17)==(38), (38)==(17), (10)==(19,34), (1)==(18,33) + │ │ │ ├── fd: (1)-->(2), (11)-->(14), (36,37)-->(39), (21)==(11,37), (37)==(11,21), (20)==(1,36), (36)==(1,20), (42)-->(46), (19)==(42), (42)==(19), (11)==(21,37), (1)==(20,36) │ │ │ ├── scan supplier@s_nk - │ │ │ │ ├── columns: s_suppkey:10!null s_nationkey:13!null - │ │ │ │ ├── key: (10) - │ │ │ │ └── fd: (10)-->(13) + │ │ │ │ ├── columns: s_suppkey:11!null s_nationkey:14!null + │ │ │ │ ├── key: (11) + │ │ │ │ └── fd: (11)-->(14) │ │ │ ├── inner-join (lookup orders) - │ │ │ │ ├── columns: p_partkey:1!null p_name:2!null l_orderkey:17!null l_partkey:18!null l_suppkey:19!null l_quantity:21!null l_extendedprice:22!null l_discount:23!null ps_partkey:33!null ps_suppkey:34!null ps_supplycost:36!null o_orderkey:38!null o_orderdate:42!null - │ │ │ │ ├── key columns: [17] = [38] + │ │ │ │ ├── columns: p_partkey:1!null p_name:2!null l_orderkey:19!null l_partkey:20!null l_suppkey:21!null l_quantity:23!null l_extendedprice:24!null l_discount:25!null ps_partkey:36!null ps_suppkey:37!null ps_supplycost:39!null o_orderkey:42!null o_orderdate:46!null + │ │ │ │ ├── key columns: [19] = [42] │ │ │ │ ├── lookup columns are key - │ │ │ │ ├── fd: (1)-->(2), (33,34)-->(36), (19)==(34), (34)==(19), (18)==(1,33), (33)==(1,18), (38)-->(42), (17)==(38), (38)==(17), (1)==(18,33) + │ │ │ │ ├── fd: (1)-->(2), (36,37)-->(39), (21)==(37), (37)==(21), (20)==(1,36), (36)==(1,20), (42)-->(46), (19)==(42), (42)==(19), (1)==(20,36) │ │ │ │ ├── inner-join (lookup lineitem) - │ │ │ │ │ ├── columns: p_partkey:1!null p_name:2!null l_orderkey:17!null l_partkey:18!null l_suppkey:19!null l_quantity:21!null l_extendedprice:22!null l_discount:23!null ps_partkey:33!null ps_suppkey:34!null ps_supplycost:36!null - │ │ │ │ │ ├── key columns: [17 20] = [17 20] + │ │ │ │ │ ├── columns: p_partkey:1!null p_name:2!null l_orderkey:19!null l_partkey:20!null l_suppkey:21!null l_quantity:23!null l_extendedprice:24!null l_discount:25!null ps_partkey:36!null ps_suppkey:37!null ps_supplycost:39!null + │ │ │ │ │ ├── key columns: [19 22] = [19 22] │ │ │ │ │ ├── lookup columns are key - │ │ │ │ │ ├── fd: (1)-->(2), (33,34)-->(36), (19)==(34), (34)==(19), (18)==(1,33), (33)==(1,18), (1)==(18,33) + │ │ │ │ │ ├── fd: (1)-->(2), (36,37)-->(39), (21)==(37), (37)==(21), (20)==(1,36), (36)==(1,20), (1)==(20,36) │ │ │ │ │ ├── inner-join (lookup lineitem@l_pk_sk) - │ │ │ │ │ │ ├── columns: p_partkey:1!null p_name:2!null l_orderkey:17!null l_partkey:18!null l_suppkey:19!null l_linenumber:20!null ps_partkey:33!null ps_suppkey:34!null ps_supplycost:36!null - │ │ │ │ │ │ ├── key columns: [33 34] = [18 19] - │ │ │ │ │ │ ├── key: (17,20) - │ │ │ │ │ │ ├── fd: (1)-->(2), (33,34)-->(36), (1)==(18,33), (33)==(1,18), (17,20)-->(18,19), (18)==(1,33), (19)==(34), (34)==(19) + │ │ │ │ │ │ ├── columns: p_partkey:1!null p_name:2!null l_orderkey:19!null l_partkey:20!null l_suppkey:21!null l_linenumber:22!null ps_partkey:36!null ps_suppkey:37!null ps_supplycost:39!null + │ │ │ │ │ │ ├── key columns: [36 37] = [20 21] + │ │ │ │ │ │ ├── key: (19,22) + │ │ │ │ │ │ ├── fd: (1)-->(2), (36,37)-->(39), (1)==(20,36), (36)==(1,20), (19,22)-->(20,21), (20)==(1,36), (21)==(37), (37)==(21) │ │ │ │ │ │ ├── inner-join (merge) - │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_name:2!null ps_partkey:33!null ps_suppkey:34!null ps_supplycost:36!null + │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_name:2!null ps_partkey:36!null ps_suppkey:37!null ps_supplycost:39!null │ │ │ │ │ │ │ ├── left ordering: +1 - │ │ │ │ │ │ │ ├── right ordering: +33 - │ │ │ │ │ │ │ ├── key: (33,34) - │ │ │ │ │ │ │ ├── fd: (1)-->(2), (33,34)-->(36), (1)==(33), (33)==(1) + │ │ │ │ │ │ │ ├── right ordering: +36 + │ │ │ │ │ │ │ ├── key: (36,37) + │ │ │ │ │ │ │ ├── fd: (1)-->(2), (36,37)-->(39), (1)==(36), (36)==(1) │ │ │ │ │ │ │ ├── select │ │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_name:2!null │ │ │ │ │ │ │ │ ├── key: (1) @@ -1039,28 +1039,28 @@ sort │ │ │ │ │ │ │ │ └── filters │ │ │ │ │ │ │ │ └── p_name:2 LIKE '%green%' [outer=(2), constraints=(/2: (/NULL - ])] │ │ │ │ │ │ │ ├── scan partsupp - │ │ │ │ │ │ │ │ ├── columns: ps_partkey:33!null ps_suppkey:34!null ps_supplycost:36!null - │ │ │ │ │ │ │ │ ├── key: (33,34) - │ │ │ │ │ │ │ │ ├── fd: (33,34)-->(36) - │ │ │ │ │ │ │ │ └── ordering: +33 + │ │ │ │ │ │ │ │ ├── columns: ps_partkey:36!null ps_suppkey:37!null ps_supplycost:39!null + │ │ │ │ │ │ │ │ ├── key: (36,37) + │ │ │ │ │ │ │ │ ├── fd: (36,37)-->(39) + │ │ │ │ │ │ │ │ └── ordering: +36 │ │ │ │ │ │ │ └── filters (true) │ │ │ │ │ │ └── filters (true) │ │ │ │ │ └── filters (true) │ │ │ │ └── filters (true) │ │ │ └── filters - │ │ │ └── s_suppkey:10 = l_suppkey:19 [outer=(10,19), constraints=(/10: (/NULL - ]; /19: (/NULL - ]), fd=(10)==(19), (19)==(10)] + │ │ │ └── s_suppkey:11 = l_suppkey:21 [outer=(11,21), constraints=(/11: (/NULL - ]; /21: (/NULL - ]), fd=(11)==(21), (21)==(11)] │ │ ├── scan nation - │ │ │ ├── columns: n_nationkey:47!null n_name:48!null - │ │ │ ├── key: (47) - │ │ │ └── fd: (47)-->(48) + │ │ │ ├── columns: n_nationkey:52!null n_name:53!null + │ │ │ ├── key: (52) + │ │ │ └── fd: (52)-->(53) │ │ └── filters - │ │ └── s_nationkey:13 = n_nationkey:47 [outer=(13,47), constraints=(/13: (/NULL - ]; /47: (/NULL - ]), fd=(13)==(47), (47)==(13)] + │ │ └── s_nationkey:14 = n_nationkey:52 [outer=(14,52), constraints=(/14: (/NULL - ]; /52: (/NULL - ]), fd=(14)==(52), (52)==(14)] │ └── projections - │ ├── extract('year', o_orderdate:42) [as=o_year:51, outer=(42), immutable] - │ └── (l_extendedprice:22 * (1.0 - l_discount:23)) - (ps_supplycost:36 * l_quantity:21) [as=amount:52, outer=(21-23,36), immutable] + │ ├── extract('year', o_orderdate:46) [as=o_year:57, outer=(46), immutable] + │ └── (l_extendedprice:24 * (1.0 - l_discount:25)) - (ps_supplycost:39 * l_quantity:23) [as=amount:58, outer=(23-25,39), immutable] └── aggregations - └── sum [as=sum:53, outer=(52)] - └── amount:52 + └── sum [as=sum:59, outer=(58)] + └── amount:58 # -------------------------------------------------- # Q10 @@ -1111,70 +1111,70 @@ ORDER BY LIMIT 20; ---- limit - ├── columns: c_custkey:1!null c_name:2!null revenue:39!null c_acctbal:6!null n_name:35!null c_address:3!null c_phone:5!null c_comment:8!null - ├── internal-ordering: -39 + ├── columns: c_custkey:1!null c_name:2!null revenue:43!null c_acctbal:6!null n_name:38!null c_address:3!null c_phone:5!null c_comment:8!null + ├── internal-ordering: -43 ├── cardinality: [0 - 20] ├── immutable ├── key: (1) - ├── fd: (1)-->(2,3,5,6,8,35,39) - ├── ordering: -39 + ├── fd: (1)-->(2,3,5,6,8,38,43) + ├── ordering: -43 ├── sort - │ ├── columns: c_custkey:1!null c_name:2!null c_address:3!null c_phone:5!null c_acctbal:6!null c_comment:8!null n_name:35!null sum:39!null + │ ├── columns: c_custkey:1!null c_name:2!null c_address:3!null c_phone:5!null c_acctbal:6!null c_comment:8!null n_name:38!null sum:43!null │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(2,3,5,6,8,35,39) - │ ├── ordering: -39 + │ ├── fd: (1)-->(2,3,5,6,8,38,43) + │ ├── ordering: -43 │ ├── limit hint: 20.00 │ └── group-by - │ ├── columns: c_custkey:1!null c_name:2!null c_address:3!null c_phone:5!null c_acctbal:6!null c_comment:8!null n_name:35!null sum:39!null + │ ├── columns: c_custkey:1!null c_name:2!null c_address:3!null c_phone:5!null c_acctbal:6!null c_comment:8!null n_name:38!null sum:43!null │ ├── grouping columns: c_custkey:1!null │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(2,3,5,6,8,35,39) + │ ├── fd: (1)-->(2,3,5,6,8,38,43) │ ├── project - │ │ ├── columns: column38:38!null c_custkey:1!null c_name:2!null c_address:3!null c_phone:5!null c_acctbal:6!null c_comment:8!null n_name:35!null + │ │ ├── columns: column42:42!null c_custkey:1!null c_name:2!null c_address:3!null c_phone:5!null c_acctbal:6!null c_comment:8!null n_name:38!null │ │ ├── immutable - │ │ ├── fd: (1)-->(2,3,5,6,8,35) + │ │ ├── fd: (1)-->(2,3,5,6,8,38) │ │ ├── inner-join (hash) - │ │ │ ├── columns: c_custkey:1!null c_name:2!null c_address:3!null c_nationkey:4!null c_phone:5!null c_acctbal:6!null c_comment:8!null o_orderkey:9!null o_custkey:10!null o_orderdate:13!null l_orderkey:18!null l_extendedprice:23!null l_discount:24!null l_returnflag:26!null n_nationkey:34!null n_name:35!null + │ │ │ ├── columns: c_custkey:1!null c_name:2!null c_address:3!null c_nationkey:4!null c_phone:5!null c_acctbal:6!null c_comment:8!null o_orderkey:10!null o_custkey:11!null o_orderdate:14!null l_orderkey:20!null l_extendedprice:25!null l_discount:26!null l_returnflag:28!null n_nationkey:37!null n_name:38!null │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ │ │ ├── fd: ()-->(26), (1)-->(2-6,8), (9)-->(10,13), (9)==(18), (18)==(9), (1)==(10), (10)==(1), (34)-->(35), (4)==(34), (34)==(4) + │ │ │ ├── fd: ()-->(28), (1)-->(2-6,8), (10)-->(11,14), (10)==(20), (20)==(10), (1)==(11), (11)==(1), (37)-->(38), (4)==(37), (37)==(4) │ │ │ ├── inner-join (hash) - │ │ │ │ ├── columns: c_custkey:1!null c_name:2!null c_address:3!null c_nationkey:4!null c_phone:5!null c_acctbal:6!null c_comment:8!null o_orderkey:9!null o_custkey:10!null o_orderdate:13!null l_orderkey:18!null l_extendedprice:23!null l_discount:24!null l_returnflag:26!null + │ │ │ │ ├── columns: c_custkey:1!null c_name:2!null c_address:3!null c_nationkey:4!null c_phone:5!null c_acctbal:6!null c_comment:8!null o_orderkey:10!null o_custkey:11!null o_orderdate:14!null l_orderkey:20!null l_extendedprice:25!null l_discount:26!null l_returnflag:28!null │ │ │ │ ├── multiplicity: left-rows(zero-or-more), right-rows(exactly-one) - │ │ │ │ ├── fd: ()-->(26), (1)-->(2-6,8), (9)-->(10,13), (9)==(18), (18)==(9), (1)==(10), (10)==(1) + │ │ │ │ ├── fd: ()-->(28), (1)-->(2-6,8), (10)-->(11,14), (10)==(20), (20)==(10), (1)==(11), (11)==(1) │ │ │ │ ├── scan customer │ │ │ │ │ ├── columns: c_custkey:1!null c_name:2!null c_address:3!null c_nationkey:4!null c_phone:5!null c_acctbal:6!null c_comment:8!null │ │ │ │ │ ├── key: (1) │ │ │ │ │ └── fd: (1)-->(2-6,8) │ │ │ │ ├── inner-join (lookup lineitem) - │ │ │ │ │ ├── columns: o_orderkey:9!null o_custkey:10!null o_orderdate:13!null l_orderkey:18!null l_extendedprice:23!null l_discount:24!null l_returnflag:26!null - │ │ │ │ │ ├── key columns: [9] = [18] - │ │ │ │ │ ├── fd: ()-->(26), (9)-->(10,13), (9)==(18), (18)==(9) + │ │ │ │ │ ├── columns: o_orderkey:10!null o_custkey:11!null o_orderdate:14!null l_orderkey:20!null l_extendedprice:25!null l_discount:26!null l_returnflag:28!null + │ │ │ │ │ ├── key columns: [10] = [20] + │ │ │ │ │ ├── fd: ()-->(28), (10)-->(11,14), (10)==(20), (20)==(10) │ │ │ │ │ ├── index-join orders - │ │ │ │ │ │ ├── columns: o_orderkey:9!null o_custkey:10!null o_orderdate:13!null - │ │ │ │ │ │ ├── key: (9) - │ │ │ │ │ │ ├── fd: (9)-->(10,13) + │ │ │ │ │ │ ├── columns: o_orderkey:10!null o_custkey:11!null o_orderdate:14!null + │ │ │ │ │ │ ├── key: (10) + │ │ │ │ │ │ ├── fd: (10)-->(11,14) │ │ │ │ │ │ └── scan orders@o_od - │ │ │ │ │ │ ├── columns: o_orderkey:9!null o_orderdate:13!null - │ │ │ │ │ │ ├── constraint: /13/9: [/'1993-10-01' - /'1993-12-31'] - │ │ │ │ │ │ ├── key: (9) - │ │ │ │ │ │ └── fd: (9)-->(13) + │ │ │ │ │ │ ├── columns: o_orderkey:10!null o_orderdate:14!null + │ │ │ │ │ │ ├── constraint: /14/10: [/'1993-10-01' - /'1993-12-31'] + │ │ │ │ │ │ ├── key: (10) + │ │ │ │ │ │ └── fd: (10)-->(14) │ │ │ │ │ └── filters - │ │ │ │ │ └── l_returnflag:26 = 'R' [outer=(26), constraints=(/26: [/'R' - /'R']; tight), fd=()-->(26)] + │ │ │ │ │ └── l_returnflag:28 = 'R' [outer=(28), constraints=(/28: [/'R' - /'R']; tight), fd=()-->(28)] │ │ │ │ └── filters - │ │ │ │ └── c_custkey:1 = o_custkey:10 [outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] + │ │ │ │ └── c_custkey:1 = o_custkey:11 [outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ]), fd=(1)==(11), (11)==(1)] │ │ │ ├── scan nation - │ │ │ │ ├── columns: n_nationkey:34!null n_name:35!null - │ │ │ │ ├── key: (34) - │ │ │ │ └── fd: (34)-->(35) + │ │ │ │ ├── columns: n_nationkey:37!null n_name:38!null + │ │ │ │ ├── key: (37) + │ │ │ │ └── fd: (37)-->(38) │ │ │ └── filters - │ │ │ └── c_nationkey:4 = n_nationkey:34 [outer=(4,34), constraints=(/4: (/NULL - ]; /34: (/NULL - ]), fd=(4)==(34), (34)==(4)] + │ │ │ └── c_nationkey:4 = n_nationkey:37 [outer=(4,37), constraints=(/4: (/NULL - ]; /37: (/NULL - ]), fd=(4)==(37), (37)==(4)] │ │ └── projections - │ │ └── l_extendedprice:23 * (1.0 - l_discount:24) [as=column38:38, outer=(23,24), immutable] + │ │ └── l_extendedprice:25 * (1.0 - l_discount:26) [as=column42:42, outer=(25,26), immutable] │ └── aggregations - │ ├── sum [as=sum:39, outer=(38)] - │ │ └── column38:38 + │ ├── sum [as=sum:43, outer=(42)] + │ │ └── column42:42 │ ├── const-agg [as=c_name:2, outer=(2)] │ │ └── c_name:2 │ ├── const-agg [as=c_address:3, outer=(3)] @@ -1185,8 +1185,8 @@ limit │ │ └── c_acctbal:6 │ ├── const-agg [as=c_comment:8, outer=(8)] │ │ └── c_comment:8 - │ └── const-agg [as=n_name:35, outer=(35)] - │ └── n_name:35 + │ └── const-agg [as=n_name:38, outer=(38)] + │ └── n_name:38 └── 20 # -------------------------------------------------- @@ -1229,113 +1229,113 @@ ORDER BY value DESC; ---- sort - ├── columns: ps_partkey:1!null value:18!null + ├── columns: ps_partkey:1!null value:21!null ├── immutable ├── key: (1) - ├── fd: (1)-->(18) - ├── ordering: -18 + ├── fd: (1)-->(21) + ├── ordering: -21 └── select - ├── columns: ps_partkey:1!null sum:18!null + ├── columns: ps_partkey:1!null sum:21!null ├── immutable ├── key: (1) - ├── fd: (1)-->(18) + ├── fd: (1)-->(21) ├── group-by - │ ├── columns: ps_partkey:1!null sum:18!null + │ ├── columns: ps_partkey:1!null sum:21!null │ ├── grouping columns: ps_partkey:1!null │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(18) + │ ├── fd: (1)-->(21) │ ├── project - │ │ ├── columns: column17:17!null ps_partkey:1!null + │ │ ├── columns: column20:20!null ps_partkey:1!null │ │ ├── immutable │ │ ├── inner-join (lookup partsupp) - │ │ │ ├── columns: ps_partkey:1!null ps_suppkey:2!null ps_availqty:3!null ps_supplycost:4!null s_suppkey:6!null s_nationkey:9!null n_nationkey:13!null n_name:14!null + │ │ │ ├── columns: ps_partkey:1!null ps_suppkey:2!null ps_availqty:3!null ps_supplycost:4!null s_suppkey:7!null s_nationkey:10!null n_nationkey:15!null n_name:16!null │ │ │ ├── key columns: [1 2] = [1 2] │ │ │ ├── lookup columns are key - │ │ │ ├── key: (1,6) - │ │ │ ├── fd: ()-->(14), (1,2)-->(3,4), (6)-->(9), (9)==(13), (13)==(9), (2)==(6), (6)==(2) + │ │ │ ├── key: (1,7) + │ │ │ ├── fd: ()-->(16), (1,2)-->(3,4), (7)-->(10), (10)==(15), (15)==(10), (2)==(7), (7)==(2) │ │ │ ├── inner-join (lookup partsupp@ps_sk) - │ │ │ │ ├── columns: ps_partkey:1!null ps_suppkey:2!null s_suppkey:6!null s_nationkey:9!null n_nationkey:13!null n_name:14!null - │ │ │ │ ├── key columns: [6] = [2] - │ │ │ │ ├── key: (1,6) - │ │ │ │ ├── fd: ()-->(14), (6)-->(9), (9)==(13), (13)==(9), (2)==(6), (6)==(2) + │ │ │ │ ├── columns: ps_partkey:1!null ps_suppkey:2!null s_suppkey:7!null s_nationkey:10!null n_nationkey:15!null n_name:16!null + │ │ │ │ ├── key columns: [7] = [2] + │ │ │ │ ├── key: (1,7) + │ │ │ │ ├── fd: ()-->(16), (7)-->(10), (10)==(15), (15)==(10), (2)==(7), (7)==(2) │ │ │ │ ├── inner-join (lookup supplier@s_nk) - │ │ │ │ │ ├── columns: s_suppkey:6!null s_nationkey:9!null n_nationkey:13!null n_name:14!null - │ │ │ │ │ ├── key columns: [13] = [9] - │ │ │ │ │ ├── key: (6) - │ │ │ │ │ ├── fd: ()-->(14), (6)-->(9), (9)==(13), (13)==(9) + │ │ │ │ │ ├── columns: s_suppkey:7!null s_nationkey:10!null n_nationkey:15!null n_name:16!null + │ │ │ │ │ ├── key columns: [15] = [10] + │ │ │ │ │ ├── key: (7) + │ │ │ │ │ ├── fd: ()-->(16), (7)-->(10), (10)==(15), (15)==(10) │ │ │ │ │ ├── select - │ │ │ │ │ │ ├── columns: n_nationkey:13!null n_name:14!null - │ │ │ │ │ │ ├── key: (13) - │ │ │ │ │ │ ├── fd: ()-->(14) + │ │ │ │ │ │ ├── columns: n_nationkey:15!null n_name:16!null + │ │ │ │ │ │ ├── key: (15) + │ │ │ │ │ │ ├── fd: ()-->(16) │ │ │ │ │ │ ├── scan nation - │ │ │ │ │ │ │ ├── columns: n_nationkey:13!null n_name:14!null - │ │ │ │ │ │ │ ├── key: (13) - │ │ │ │ │ │ │ └── fd: (13)-->(14) + │ │ │ │ │ │ │ ├── columns: n_nationkey:15!null n_name:16!null + │ │ │ │ │ │ │ ├── key: (15) + │ │ │ │ │ │ │ └── fd: (15)-->(16) │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── n_name:14 = 'GERMANY' [outer=(14), constraints=(/14: [/'GERMANY' - /'GERMANY']; tight), fd=()-->(14)] + │ │ │ │ │ │ └── n_name:16 = 'GERMANY' [outer=(16), constraints=(/16: [/'GERMANY' - /'GERMANY']; tight), fd=()-->(16)] │ │ │ │ │ └── filters (true) │ │ │ │ └── filters (true) │ │ │ └── filters (true) │ │ └── projections - │ │ └── ps_supplycost:4 * ps_availqty:3::FLOAT8 [as=column17:17, outer=(3,4), immutable] + │ │ └── ps_supplycost:4 * ps_availqty:3::FLOAT8 [as=column20:20, outer=(3,4), immutable] │ └── aggregations - │ └── sum [as=sum:18, outer=(17)] - │ └── column17:17 + │ └── sum [as=sum:21, outer=(20)] + │ └── column20:20 └── filters - └── gt [outer=(18), immutable, subquery, constraints=(/18: (/NULL - ])] - ├── sum:18 + └── gt [outer=(21), immutable, subquery, constraints=(/21: (/NULL - ])] + ├── sum:21 └── subquery └── project - ├── columns: "?column?":37 + ├── columns: "?column?":43 ├── cardinality: [1 - 1] ├── immutable ├── key: () - ├── fd: ()-->(37) + ├── fd: ()-->(43) ├── scalar-group-by - │ ├── columns: sum:36 + │ ├── columns: sum:42 │ ├── cardinality: [1 - 1] │ ├── immutable │ ├── key: () - │ ├── fd: ()-->(36) + │ ├── fd: ()-->(42) │ ├── project - │ │ ├── columns: column35:35!null + │ │ ├── columns: column41:41!null │ │ ├── immutable │ │ ├── inner-join (lookup partsupp) - │ │ │ ├── columns: ps_suppkey:20!null ps_availqty:21!null ps_supplycost:22!null s_suppkey:24!null s_nationkey:27!null n_nationkey:31!null n_name:32!null - │ │ │ ├── key columns: [19 20] = [19 20] + │ │ │ ├── columns: ps_suppkey:23!null ps_availqty:24!null ps_supplycost:25!null s_suppkey:28!null s_nationkey:31!null n_nationkey:36!null n_name:37!null + │ │ │ ├── key columns: [22 23] = [22 23] │ │ │ ├── lookup columns are key - │ │ │ ├── fd: ()-->(32), (24)-->(27), (27)==(31), (31)==(27), (20)==(24), (24)==(20) + │ │ │ ├── fd: ()-->(37), (28)-->(31), (31)==(36), (36)==(31), (23)==(28), (28)==(23) │ │ │ ├── inner-join (lookup partsupp@ps_sk) - │ │ │ │ ├── columns: ps_partkey:19!null ps_suppkey:20!null s_suppkey:24!null s_nationkey:27!null n_nationkey:31!null n_name:32!null - │ │ │ │ ├── key columns: [24] = [20] - │ │ │ │ ├── key: (19,24) - │ │ │ │ ├── fd: ()-->(32), (24)-->(27), (27)==(31), (31)==(27), (20)==(24), (24)==(20) + │ │ │ │ ├── columns: ps_partkey:22!null ps_suppkey:23!null s_suppkey:28!null s_nationkey:31!null n_nationkey:36!null n_name:37!null + │ │ │ │ ├── key columns: [28] = [23] + │ │ │ │ ├── key: (22,28) + │ │ │ │ ├── fd: ()-->(37), (28)-->(31), (31)==(36), (36)==(31), (23)==(28), (28)==(23) │ │ │ │ ├── inner-join (lookup supplier@s_nk) - │ │ │ │ │ ├── columns: s_suppkey:24!null s_nationkey:27!null n_nationkey:31!null n_name:32!null - │ │ │ │ │ ├── key columns: [31] = [27] - │ │ │ │ │ ├── key: (24) - │ │ │ │ │ ├── fd: ()-->(32), (24)-->(27), (27)==(31), (31)==(27) + │ │ │ │ │ ├── columns: s_suppkey:28!null s_nationkey:31!null n_nationkey:36!null n_name:37!null + │ │ │ │ │ ├── key columns: [36] = [31] + │ │ │ │ │ ├── key: (28) + │ │ │ │ │ ├── fd: ()-->(37), (28)-->(31), (31)==(36), (36)==(31) │ │ │ │ │ ├── select - │ │ │ │ │ │ ├── columns: n_nationkey:31!null n_name:32!null - │ │ │ │ │ │ ├── key: (31) - │ │ │ │ │ │ ├── fd: ()-->(32) + │ │ │ │ │ │ ├── columns: n_nationkey:36!null n_name:37!null + │ │ │ │ │ │ ├── key: (36) + │ │ │ │ │ │ ├── fd: ()-->(37) │ │ │ │ │ │ ├── scan nation - │ │ │ │ │ │ │ ├── columns: n_nationkey:31!null n_name:32!null - │ │ │ │ │ │ │ ├── key: (31) - │ │ │ │ │ │ │ └── fd: (31)-->(32) + │ │ │ │ │ │ │ ├── columns: n_nationkey:36!null n_name:37!null + │ │ │ │ │ │ │ ├── key: (36) + │ │ │ │ │ │ │ └── fd: (36)-->(37) │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── n_name:32 = 'GERMANY' [outer=(32), constraints=(/32: [/'GERMANY' - /'GERMANY']; tight), fd=()-->(32)] + │ │ │ │ │ │ └── n_name:37 = 'GERMANY' [outer=(37), constraints=(/37: [/'GERMANY' - /'GERMANY']; tight), fd=()-->(37)] │ │ │ │ │ └── filters (true) │ │ │ │ └── filters (true) │ │ │ └── filters (true) │ │ └── projections - │ │ └── ps_supplycost:22 * ps_availqty:21::FLOAT8 [as=column35:35, outer=(21,22), immutable] + │ │ └── ps_supplycost:25 * ps_availqty:24::FLOAT8 [as=column41:41, outer=(24,25), immutable] │ └── aggregations - │ └── sum [as=sum:36, outer=(35)] - │ └── column35:35 + │ └── sum [as=sum:42, outer=(41)] + │ └── column41:41 └── projections - └── sum:36 * 0.0001 [as="?column?":37, outer=(36), immutable] + └── sum:42 * 0.0001 [as="?column?":43, outer=(42), immutable] # -------------------------------------------------- # Q12 @@ -1382,44 +1382,44 @@ ORDER BY l_shipmode; ---- sort - ├── columns: l_shipmode:24!null high_line_count:27!null low_line_count:29!null - ├── key: (24) - ├── fd: (24)-->(27,29) - ├── ordering: +24 + ├── columns: l_shipmode:25!null high_line_count:29!null low_line_count:31!null + ├── key: (25) + ├── fd: (25)-->(29,31) + ├── ordering: +25 └── group-by - ├── columns: l_shipmode:24!null sum:27!null sum:29!null - ├── grouping columns: l_shipmode:24!null - ├── key: (24) - ├── fd: (24)-->(27,29) + ├── columns: l_shipmode:25!null sum:29!null sum:31!null + ├── grouping columns: l_shipmode:25!null + ├── key: (25) + ├── fd: (25)-->(29,31) ├── project - │ ├── columns: column26:26!null column28:28!null l_shipmode:24!null + │ ├── columns: column28:28!null column30:30!null l_shipmode:25!null │ ├── inner-join (lookup orders) - │ │ ├── columns: o_orderkey:1!null o_orderpriority:6!null l_orderkey:10!null l_shipdate:20!null l_commitdate:21!null l_receiptdate:22!null l_shipmode:24!null - │ │ ├── key columns: [10] = [1] + │ │ ├── columns: o_orderkey:1!null o_orderpriority:6!null l_orderkey:11!null l_shipdate:21!null l_commitdate:22!null l_receiptdate:23!null l_shipmode:25!null + │ │ ├── key columns: [11] = [1] │ │ ├── lookup columns are key - │ │ ├── fd: (1)-->(6), (1)==(10), (10)==(1) + │ │ ├── fd: (1)-->(6), (1)==(11), (11)==(1) │ │ ├── select - │ │ │ ├── columns: l_orderkey:10!null l_shipdate:20!null l_commitdate:21!null l_receiptdate:22!null l_shipmode:24!null + │ │ │ ├── columns: l_orderkey:11!null l_shipdate:21!null l_commitdate:22!null l_receiptdate:23!null l_shipmode:25!null │ │ │ ├── index-join lineitem - │ │ │ │ ├── columns: l_orderkey:10!null l_shipdate:20!null l_commitdate:21!null l_receiptdate:22!null l_shipmode:24!null + │ │ │ │ ├── columns: l_orderkey:11!null l_shipdate:21!null l_commitdate:22!null l_receiptdate:23!null l_shipmode:25!null │ │ │ │ └── scan lineitem@l_rd - │ │ │ │ ├── columns: l_orderkey:10!null l_linenumber:13!null l_receiptdate:22!null - │ │ │ │ ├── constraint: /22/10/13: [/'1994-01-01' - /'1994-12-31'] - │ │ │ │ ├── key: (10,13) - │ │ │ │ └── fd: (10,13)-->(22) + │ │ │ │ ├── columns: l_orderkey:11!null l_linenumber:14!null l_receiptdate:23!null + │ │ │ │ ├── constraint: /23/11/14: [/'1994-01-01' - /'1994-12-31'] + │ │ │ │ ├── key: (11,14) + │ │ │ │ └── fd: (11,14)-->(23) │ │ │ └── filters - │ │ │ ├── l_shipmode:24 IN ('MAIL', 'SHIP') [outer=(24), constraints=(/24: [/'MAIL' - /'MAIL'] [/'SHIP' - /'SHIP']; tight)] - │ │ │ ├── l_commitdate:21 < l_receiptdate:22 [outer=(21,22), constraints=(/21: (/NULL - ]; /22: (/NULL - ])] - │ │ │ └── l_shipdate:20 < l_commitdate:21 [outer=(20,21), constraints=(/20: (/NULL - ]; /21: (/NULL - ])] + │ │ │ ├── l_shipmode:25 IN ('MAIL', 'SHIP') [outer=(25), constraints=(/25: [/'MAIL' - /'MAIL'] [/'SHIP' - /'SHIP']; tight)] + │ │ │ ├── l_commitdate:22 < l_receiptdate:23 [outer=(22,23), constraints=(/22: (/NULL - ]; /23: (/NULL - ])] + │ │ │ └── l_shipdate:21 < l_commitdate:22 [outer=(21,22), constraints=(/21: (/NULL - ]; /22: (/NULL - ])] │ │ └── filters (true) │ └── projections - │ ├── CASE WHEN (o_orderpriority:6 = '1-URGENT') OR (o_orderpriority:6 = '2-HIGH') THEN 1 ELSE 0 END [as=column26:26, outer=(6)] - │ └── CASE WHEN (o_orderpriority:6 != '1-URGENT') AND (o_orderpriority:6 != '2-HIGH') THEN 1 ELSE 0 END [as=column28:28, outer=(6)] + │ ├── CASE WHEN (o_orderpriority:6 = '1-URGENT') OR (o_orderpriority:6 = '2-HIGH') THEN 1 ELSE 0 END [as=column28:28, outer=(6)] + │ └── CASE WHEN (o_orderpriority:6 != '1-URGENT') AND (o_orderpriority:6 != '2-HIGH') THEN 1 ELSE 0 END [as=column30:30, outer=(6)] └── aggregations - ├── sum [as=sum:27, outer=(26)] - │ └── column26:26 - └── sum [as=sum:29, outer=(28)] - └── column28:28 + ├── sum [as=sum:29, outer=(28)] + │ └── column28:28 + └── sum [as=sum:31, outer=(30)] + └── column30:30 # -------------------------------------------------- # Q13 @@ -1454,44 +1454,44 @@ ORDER BY c_count DESC; ---- sort - ├── columns: c_count:18!null custdist:19!null - ├── key: (18) - ├── fd: (18)-->(19) - ├── ordering: -19,-18 + ├── columns: c_count:20!null custdist:21!null + ├── key: (20) + ├── fd: (20)-->(21) + ├── ordering: -21,-20 └── group-by - ├── columns: count:18!null count_rows:19!null - ├── grouping columns: count:18!null - ├── key: (18) - ├── fd: (18)-->(19) + ├── columns: count:20!null count_rows:21!null + ├── grouping columns: count:20!null + ├── key: (20) + ├── fd: (20)-->(21) ├── group-by - │ ├── columns: c_custkey:1!null count:18!null + │ ├── columns: c_custkey:1!null count:20!null │ ├── grouping columns: c_custkey:1!null │ ├── key: (1) - │ ├── fd: (1)-->(18) + │ ├── fd: (1)-->(20) │ ├── right-join (hash) - │ │ ├── columns: c_custkey:1!null o_orderkey:9 o_custkey:10 o_comment:17 - │ │ ├── key: (1,9) - │ │ ├── fd: (9)-->(10,17) + │ │ ├── columns: c_custkey:1!null o_orderkey:10 o_custkey:11 o_comment:18 + │ │ ├── key: (1,10) + │ │ ├── fd: (10)-->(11,18) │ │ ├── select - │ │ │ ├── columns: o_orderkey:9!null o_custkey:10!null o_comment:17!null - │ │ │ ├── key: (9) - │ │ │ ├── fd: (9)-->(10,17) + │ │ │ ├── columns: o_orderkey:10!null o_custkey:11!null o_comment:18!null + │ │ │ ├── key: (10) + │ │ │ ├── fd: (10)-->(11,18) │ │ │ ├── scan orders - │ │ │ │ ├── columns: o_orderkey:9!null o_custkey:10!null o_comment:17!null - │ │ │ │ ├── key: (9) - │ │ │ │ └── fd: (9)-->(10,17) + │ │ │ │ ├── columns: o_orderkey:10!null o_custkey:11!null o_comment:18!null + │ │ │ │ ├── key: (10) + │ │ │ │ └── fd: (10)-->(11,18) │ │ │ └── filters - │ │ │ └── o_comment:17 NOT LIKE '%special%requests%' [outer=(17), constraints=(/17: (/NULL - ])] + │ │ │ └── o_comment:18 NOT LIKE '%special%requests%' [outer=(18), constraints=(/18: (/NULL - ])] │ │ ├── scan customer@c_nk │ │ │ ├── columns: c_custkey:1!null │ │ │ └── key: (1) │ │ └── filters - │ │ └── c_custkey:1 = o_custkey:10 [outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] + │ │ └── c_custkey:1 = o_custkey:11 [outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ]), fd=(1)==(11), (11)==(1)] │ └── aggregations - │ └── count [as=count:18, outer=(9)] - │ └── o_orderkey:9 + │ └── count [as=count:20, outer=(10)] + │ └── o_orderkey:10 └── aggregations - └── count-rows [as=count_rows:19] + └── count-rows [as=count_rows:21] # -------------------------------------------------- # Q14 @@ -1520,28 +1520,28 @@ WHERE AND l_shipdate < DATE '1995-09-01' + INTERVAL '1' MONTH; ---- project - ├── columns: promo_revenue:30 + ├── columns: promo_revenue:32 ├── cardinality: [1 - 1] ├── immutable ├── key: () - ├── fd: ()-->(30) + ├── fd: ()-->(32) ├── scalar-group-by - │ ├── columns: sum:27 sum:29 + │ ├── columns: sum:29 sum:31 │ ├── cardinality: [1 - 1] │ ├── immutable │ ├── key: () - │ ├── fd: ()-->(27,29) + │ ├── fd: ()-->(29,31) │ ├── project - │ │ ├── columns: column26:26!null column28:28!null + │ │ ├── columns: column28:28!null column30:30!null │ │ ├── immutable │ │ ├── inner-join (hash) - │ │ │ ├── columns: l_partkey:2!null l_extendedprice:6!null l_discount:7!null l_shipdate:11!null p_partkey:17!null p_type:21!null + │ │ │ ├── columns: l_partkey:2!null l_extendedprice:6!null l_discount:7!null l_shipdate:11!null p_partkey:18!null p_type:22!null │ │ │ ├── multiplicity: left-rows(zero-or-more), right-rows(exactly-one) - │ │ │ ├── fd: (17)-->(21), (2)==(17), (17)==(2) + │ │ │ ├── fd: (18)-->(22), (2)==(18), (18)==(2) │ │ │ ├── scan part - │ │ │ │ ├── columns: p_partkey:17!null p_type:21!null - │ │ │ │ ├── key: (17) - │ │ │ │ └── fd: (17)-->(21) + │ │ │ │ ├── columns: p_partkey:18!null p_type:22!null + │ │ │ │ ├── key: (18) + │ │ │ │ └── fd: (18)-->(22) │ │ │ ├── index-join lineitem │ │ │ │ ├── columns: l_partkey:2!null l_extendedprice:6!null l_discount:7!null l_shipdate:11!null │ │ │ │ └── scan lineitem@l_sd @@ -1550,17 +1550,17 @@ project │ │ │ │ ├── key: (1,4) │ │ │ │ └── fd: (1,4)-->(11) │ │ │ └── filters - │ │ │ └── l_partkey:2 = p_partkey:17 [outer=(2,17), constraints=(/2: (/NULL - ]; /17: (/NULL - ]), fd=(2)==(17), (17)==(2)] + │ │ │ └── l_partkey:2 = p_partkey:18 [outer=(2,18), constraints=(/2: (/NULL - ]; /18: (/NULL - ]), fd=(2)==(18), (18)==(2)] │ │ └── projections - │ │ ├── CASE WHEN p_type:21 LIKE 'PROMO%' THEN l_extendedprice:6 * (1.0 - l_discount:7) ELSE 0.0 END [as=column26:26, outer=(6,7,21), immutable] - │ │ └── l_extendedprice:6 * (1.0 - l_discount:7) [as=column28:28, outer=(6,7), immutable] + │ │ ├── CASE WHEN p_type:22 LIKE 'PROMO%' THEN l_extendedprice:6 * (1.0 - l_discount:7) ELSE 0.0 END [as=column28:28, outer=(6,7,22), immutable] + │ │ └── l_extendedprice:6 * (1.0 - l_discount:7) [as=column30:30, outer=(6,7), immutable] │ └── aggregations - │ ├── sum [as=sum:27, outer=(26)] - │ │ └── column26:26 - │ └── sum [as=sum:29, outer=(28)] - │ └── column28:28 + │ ├── sum [as=sum:29, outer=(28)] + │ │ └── column28:28 + │ └── sum [as=sum:31, outer=(30)] + │ └── column30:30 └── projections - └── (sum:27 * 100.0) / sum:29 [as=promo_revenue:30, outer=(27,29), immutable] + └── (sum:29 * 100.0) / sum:31 [as=promo_revenue:32, outer=(29,31), immutable] # -------------------------------------------------- # Q15 @@ -1609,90 +1609,90 @@ ORDER BY s_suppkey; ---- project - ├── columns: s_suppkey:1!null s_name:2!null s_address:3!null s_phone:5!null total_revenue:25!null + ├── columns: s_suppkey:1!null s_name:2!null s_address:3!null s_phone:5!null total_revenue:27!null ├── immutable ├── key: (1) - ├── fd: (1)-->(2,3,5,25) + ├── fd: (1)-->(2,3,5,27) ├── ordering: +1 └── inner-join (merge) - ├── columns: s_suppkey:1!null s_name:2!null s_address:3!null s_phone:5!null l_suppkey:10!null sum:25!null + ├── columns: s_suppkey:1!null s_name:2!null s_address:3!null s_phone:5!null l_suppkey:11!null sum:27!null ├── left ordering: +1 - ├── right ordering: +10 + ├── right ordering: +11 ├── immutable - ├── key: (10) - ├── fd: (1)-->(2,3,5), (10)-->(25), (1)==(10), (10)==(1) - ├── ordering: +(1|10) [actual: +1] + ├── key: (11) + ├── fd: (1)-->(2,3,5), (11)-->(27), (1)==(11), (11)==(1) + ├── ordering: +(1|11) [actual: +1] ├── scan supplier │ ├── columns: s_suppkey:1!null s_name:2!null s_address:3!null s_phone:5!null │ ├── key: (1) │ ├── fd: (1)-->(2,3,5) │ └── ordering: +1 ├── sort - │ ├── columns: l_suppkey:10!null sum:25!null + │ ├── columns: l_suppkey:11!null sum:27!null │ ├── immutable - │ ├── key: (10) - │ ├── fd: (10)-->(25) - │ ├── ordering: +10 + │ ├── key: (11) + │ ├── fd: (11)-->(27) + │ ├── ordering: +11 │ └── select - │ ├── columns: l_suppkey:10!null sum:25!null + │ ├── columns: l_suppkey:11!null sum:27!null │ ├── immutable - │ ├── key: (10) - │ ├── fd: (10)-->(25) + │ ├── key: (11) + │ ├── fd: (11)-->(27) │ ├── group-by - │ │ ├── columns: l_suppkey:10!null sum:25!null - │ │ ├── grouping columns: l_suppkey:10!null + │ │ ├── columns: l_suppkey:11!null sum:27!null + │ │ ├── grouping columns: l_suppkey:11!null │ │ ├── immutable - │ │ ├── key: (10) - │ │ ├── fd: (10)-->(25) + │ │ ├── key: (11) + │ │ ├── fd: (11)-->(27) │ │ ├── project - │ │ │ ├── columns: column24:24!null l_suppkey:10!null + │ │ │ ├── columns: column26:26!null l_suppkey:11!null │ │ │ ├── immutable │ │ │ ├── index-join lineitem - │ │ │ │ ├── columns: l_suppkey:10!null l_extendedprice:13!null l_discount:14!null l_shipdate:18!null + │ │ │ │ ├── columns: l_suppkey:11!null l_extendedprice:14!null l_discount:15!null l_shipdate:19!null │ │ │ │ └── scan lineitem@l_sd - │ │ │ │ ├── columns: l_orderkey:8!null l_linenumber:11!null l_shipdate:18!null - │ │ │ │ ├── constraint: /18/8/11: [/'1996-01-01' - /'1996-03-31'] - │ │ │ │ ├── key: (8,11) - │ │ │ │ └── fd: (8,11)-->(18) + │ │ │ │ ├── columns: l_orderkey:9!null l_linenumber:12!null l_shipdate:19!null + │ │ │ │ ├── constraint: /19/9/12: [/'1996-01-01' - /'1996-03-31'] + │ │ │ │ ├── key: (9,12) + │ │ │ │ └── fd: (9,12)-->(19) │ │ │ └── projections - │ │ │ └── l_extendedprice:13 * (1.0 - l_discount:14) [as=column24:24, outer=(13,14), immutable] + │ │ │ └── l_extendedprice:14 * (1.0 - l_discount:15) [as=column26:26, outer=(14,15), immutable] │ │ └── aggregations - │ │ └── sum [as=sum:25, outer=(24)] - │ │ └── column24:24 + │ │ └── sum [as=sum:27, outer=(26)] + │ │ └── column26:26 │ └── filters - │ └── eq [outer=(25), immutable, subquery, constraints=(/25: (/NULL - ])] - │ ├── sum:25 + │ └── eq [outer=(27), immutable, subquery, constraints=(/27: (/NULL - ])] + │ ├── sum:27 │ └── subquery │ └── scalar-group-by - │ ├── columns: max:44 + │ ├── columns: max:47 │ ├── cardinality: [1 - 1] │ ├── immutable │ ├── key: () - │ ├── fd: ()-->(44) + │ ├── fd: ()-->(47) │ ├── group-by - │ │ ├── columns: l_suppkey:28!null sum:43!null - │ │ ├── grouping columns: l_suppkey:28!null + │ │ ├── columns: l_suppkey:30!null sum:46!null + │ │ ├── grouping columns: l_suppkey:30!null │ │ ├── immutable - │ │ ├── key: (28) - │ │ ├── fd: (28)-->(43) + │ │ ├── key: (30) + │ │ ├── fd: (30)-->(46) │ │ ├── project - │ │ │ ├── columns: column42:42!null l_suppkey:28!null + │ │ │ ├── columns: column45:45!null l_suppkey:30!null │ │ │ ├── immutable │ │ │ ├── index-join lineitem - │ │ │ │ ├── columns: l_suppkey:28!null l_extendedprice:31!null l_discount:32!null l_shipdate:36!null + │ │ │ │ ├── columns: l_suppkey:30!null l_extendedprice:33!null l_discount:34!null l_shipdate:38!null │ │ │ │ └── scan lineitem@l_sd - │ │ │ │ ├── columns: l_orderkey:26!null l_linenumber:29!null l_shipdate:36!null - │ │ │ │ ├── constraint: /36/26/29: [/'1996-01-01' - /'1996-03-31'] - │ │ │ │ ├── key: (26,29) - │ │ │ │ └── fd: (26,29)-->(36) + │ │ │ │ ├── columns: l_orderkey:28!null l_linenumber:31!null l_shipdate:38!null + │ │ │ │ ├── constraint: /38/28/31: [/'1996-01-01' - /'1996-03-31'] + │ │ │ │ ├── key: (28,31) + │ │ │ │ └── fd: (28,31)-->(38) │ │ │ └── projections - │ │ │ └── l_extendedprice:31 * (1.0 - l_discount:32) [as=column42:42, outer=(31,32), immutable] + │ │ │ └── l_extendedprice:33 * (1.0 - l_discount:34) [as=column45:45, outer=(33,34), immutable] │ │ └── aggregations - │ │ └── sum [as=sum:43, outer=(42)] - │ │ └── column42:42 + │ │ └── sum [as=sum:46, outer=(45)] + │ │ └── column45:45 │ └── aggregations - │ └── max [as=max:44, outer=(43)] - │ └── sum:43 + │ └── max [as=max:47, outer=(46)] + │ └── sum:46 └── filters (true) # -------------------------------------------------- @@ -1742,62 +1742,62 @@ ORDER BY p_size; ---- sort - ├── columns: p_brand:9!null p_type:10!null p_size:11!null supplier_cnt:22!null - ├── key: (9-11) - ├── fd: (9-11)-->(22) - ├── ordering: -22,+9,+10,+11 + ├── columns: p_brand:10!null p_type:11!null p_size:12!null supplier_cnt:25!null + ├── key: (10-12) + ├── fd: (10-12)-->(25) + ├── ordering: -25,+10,+11,+12 └── group-by - ├── columns: p_brand:9!null p_type:10!null p_size:11!null count:22!null - ├── grouping columns: p_brand:9!null p_type:10!null p_size:11!null - ├── key: (9-11) - ├── fd: (9-11)-->(22) + ├── columns: p_brand:10!null p_type:11!null p_size:12!null count:25!null + ├── grouping columns: p_brand:10!null p_type:11!null p_size:12!null + ├── key: (10-12) + ├── fd: (10-12)-->(25) ├── distinct-on - │ ├── columns: ps_suppkey:2!null p_brand:9!null p_type:10!null p_size:11!null - │ ├── grouping columns: ps_suppkey:2!null p_brand:9!null p_type:10!null p_size:11!null - │ ├── key: (2,9-11) + │ ├── columns: ps_suppkey:2!null p_brand:10!null p_type:11!null p_size:12!null + │ ├── grouping columns: ps_suppkey:2!null p_brand:10!null p_type:11!null p_size:12!null + │ ├── key: (2,10-12) │ └── inner-join (hash) - │ ├── columns: ps_partkey:1!null ps_suppkey:2!null p_partkey:6!null p_brand:9!null p_type:10!null p_size:11!null + │ ├── columns: ps_partkey:1!null ps_suppkey:2!null p_partkey:7!null p_brand:10!null p_type:11!null p_size:12!null │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ ├── key: (2,6) - │ ├── fd: (6)-->(9-11), (1)==(6), (6)==(1) + │ ├── key: (2,7) + │ ├── fd: (7)-->(10-12), (1)==(7), (7)==(1) │ ├── anti-join (merge) │ │ ├── columns: ps_partkey:1!null ps_suppkey:2!null │ │ ├── left ordering: +2 - │ │ ├── right ordering: +15 + │ │ ├── right ordering: +17 │ │ ├── key: (1,2) │ │ ├── scan partsupp@ps_sk │ │ │ ├── columns: ps_partkey:1!null ps_suppkey:2!null │ │ │ ├── key: (1,2) │ │ │ └── ordering: +2 │ │ ├── select - │ │ │ ├── columns: s_suppkey:15!null s_comment:21!null - │ │ │ ├── key: (15) - │ │ │ ├── fd: (15)-->(21) - │ │ │ ├── ordering: +15 + │ │ │ ├── columns: s_suppkey:17!null s_comment:23!null + │ │ │ ├── key: (17) + │ │ │ ├── fd: (17)-->(23) + │ │ │ ├── ordering: +17 │ │ │ ├── scan supplier - │ │ │ │ ├── columns: s_suppkey:15!null s_comment:21!null - │ │ │ │ ├── key: (15) - │ │ │ │ ├── fd: (15)-->(21) - │ │ │ │ └── ordering: +15 + │ │ │ │ ├── columns: s_suppkey:17!null s_comment:23!null + │ │ │ │ ├── key: (17) + │ │ │ │ ├── fd: (17)-->(23) + │ │ │ │ └── ordering: +17 │ │ │ └── filters - │ │ │ └── s_comment:21 LIKE '%Customer%Complaints%' [outer=(21), constraints=(/21: (/NULL - ])] + │ │ │ └── s_comment:23 LIKE '%Customer%Complaints%' [outer=(23), constraints=(/23: (/NULL - ])] │ │ └── filters (true) │ ├── select - │ │ ├── columns: p_partkey:6!null p_brand:9!null p_type:10!null p_size:11!null - │ │ ├── key: (6) - │ │ ├── fd: (6)-->(9-11) + │ │ ├── columns: p_partkey:7!null p_brand:10!null p_type:11!null p_size:12!null + │ │ ├── key: (7) + │ │ ├── fd: (7)-->(10-12) │ │ ├── scan part - │ │ │ ├── columns: p_partkey:6!null p_brand:9!null p_type:10!null p_size:11!null - │ │ │ ├── key: (6) - │ │ │ └── fd: (6)-->(9-11) + │ │ │ ├── columns: p_partkey:7!null p_brand:10!null p_type:11!null p_size:12!null + │ │ │ ├── key: (7) + │ │ │ └── fd: (7)-->(10-12) │ │ └── filters - │ │ ├── p_brand:9 != 'Brand#45' [outer=(9), constraints=(/9: (/NULL - /'Brand#45') [/e'Brand#45\x00' - ]; tight)] - │ │ ├── p_type:10 NOT LIKE 'MEDIUM POLISHED %' [outer=(10), constraints=(/10: (/NULL - ])] - │ │ └── p_size:11 IN (3, 9, 14, 19, 23, 36, 45, 49) [outer=(11), constraints=(/11: [/3 - /3] [/9 - /9] [/14 - /14] [/19 - /19] [/23 - /23] [/36 - /36] [/45 - /45] [/49 - /49]; tight)] + │ │ ├── p_brand:10 != 'Brand#45' [outer=(10), constraints=(/10: (/NULL - /'Brand#45') [/e'Brand#45\x00' - ]; tight)] + │ │ ├── p_type:11 NOT LIKE 'MEDIUM POLISHED %' [outer=(11), constraints=(/11: (/NULL - ])] + │ │ └── p_size:12 IN (3, 9, 14, 19, 23, 36, 45, 49) [outer=(12), constraints=(/12: [/3 - /3] [/9 - /9] [/14 - /14] [/19 - /19] [/23 - /23] [/36 - /36] [/45 - /45] [/49 - /49]; tight)] │ └── filters - │ └── p_partkey:6 = ps_partkey:1 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ └── p_partkey:7 = ps_partkey:1 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] └── aggregations - └── count-rows [as=count:22] + └── count-rows [as=count:25] # -------------------------------------------------- # Q17 @@ -1832,80 +1832,80 @@ WHERE ); ---- project - ├── columns: avg_yearly:45 + ├── columns: avg_yearly:48 ├── cardinality: [1 - 1] ├── immutable ├── key: () - ├── fd: ()-->(45) + ├── fd: ()-->(48) ├── scalar-group-by - │ ├── columns: sum:44 + │ ├── columns: sum:47 │ ├── cardinality: [1 - 1] │ ├── immutable │ ├── key: () - │ ├── fd: ()-->(44) + │ ├── fd: ()-->(47) │ ├── inner-join (lookup lineitem) - │ │ ├── columns: l_partkey:2!null l_quantity:5!null l_extendedprice:6!null p_partkey:17!null "?column?":43!null + │ │ ├── columns: l_partkey:2!null l_quantity:5!null l_extendedprice:6!null p_partkey:18!null "?column?":46!null │ │ ├── key columns: [1 4] = [1 4] │ │ ├── lookup columns are key │ │ ├── immutable - │ │ ├── fd: (17)-->(43), (2)==(17), (17)==(2) + │ │ ├── fd: (18)-->(46), (2)==(18), (18)==(2) │ │ ├── inner-join (lookup lineitem@l_pk) - │ │ │ ├── columns: l_orderkey:1!null l_partkey:2!null l_linenumber:4!null p_partkey:17!null "?column?":43 - │ │ │ ├── key columns: [17] = [2] + │ │ │ ├── columns: l_orderkey:1!null l_partkey:2!null l_linenumber:4!null p_partkey:18!null "?column?":46 + │ │ │ ├── key columns: [18] = [2] │ │ │ ├── immutable │ │ │ ├── key: (1,4) - │ │ │ ├── fd: (17)-->(43), (1,4)-->(2), (2)==(17), (17)==(2) + │ │ │ ├── fd: (18)-->(46), (1,4)-->(2), (2)==(18), (18)==(2) │ │ │ ├── project - │ │ │ │ ├── columns: "?column?":43 p_partkey:17!null + │ │ │ │ ├── columns: "?column?":46 p_partkey:18!null │ │ │ │ ├── immutable - │ │ │ │ ├── key: (17) - │ │ │ │ ├── fd: (17)-->(43) + │ │ │ │ ├── key: (18) + │ │ │ │ ├── fd: (18)-->(46) │ │ │ │ ├── group-by - │ │ │ │ │ ├── columns: p_partkey:17!null avg:42 - │ │ │ │ │ ├── grouping columns: p_partkey:17!null - │ │ │ │ │ ├── internal-ordering: +17 opt(20,23) - │ │ │ │ │ ├── key: (17) - │ │ │ │ │ ├── fd: (17)-->(42) + │ │ │ │ │ ├── columns: p_partkey:18!null avg:45 + │ │ │ │ │ ├── grouping columns: p_partkey:18!null + │ │ │ │ │ ├── internal-ordering: +18 opt(21,24) + │ │ │ │ │ ├── key: (18) + │ │ │ │ │ ├── fd: (18)-->(45) │ │ │ │ │ ├── left-join (lookup lineitem) - │ │ │ │ │ │ ├── columns: p_partkey:17!null p_brand:20!null p_container:23!null l_partkey:27 l_quantity:30 - │ │ │ │ │ │ ├── key columns: [26 29] = [26 29] + │ │ │ │ │ │ ├── columns: p_partkey:18!null p_brand:21!null p_container:24!null l_partkey:29 l_quantity:32 + │ │ │ │ │ │ ├── key columns: [28 31] = [28 31] │ │ │ │ │ │ ├── lookup columns are key - │ │ │ │ │ │ ├── fd: ()-->(20,23) - │ │ │ │ │ │ ├── ordering: +17 opt(20,23) [actual: +17] + │ │ │ │ │ │ ├── fd: ()-->(21,24) + │ │ │ │ │ │ ├── ordering: +18 opt(21,24) [actual: +18] │ │ │ │ │ │ ├── left-join (lookup lineitem@l_pk) - │ │ │ │ │ │ │ ├── columns: p_partkey:17!null p_brand:20!null p_container:23!null l_orderkey:26 l_partkey:27 l_linenumber:29 - │ │ │ │ │ │ │ ├── key columns: [17] = [27] - │ │ │ │ │ │ │ ├── key: (17,26,29) - │ │ │ │ │ │ │ ├── fd: ()-->(20,23), (26,29)-->(27) - │ │ │ │ │ │ │ ├── ordering: +17 opt(20,23) [actual: +17] + │ │ │ │ │ │ │ ├── columns: p_partkey:18!null p_brand:21!null p_container:24!null l_orderkey:28 l_partkey:29 l_linenumber:31 + │ │ │ │ │ │ │ ├── key columns: [18] = [29] + │ │ │ │ │ │ │ ├── key: (18,28,31) + │ │ │ │ │ │ │ ├── fd: ()-->(21,24), (28,31)-->(29) + │ │ │ │ │ │ │ ├── ordering: +18 opt(21,24) [actual: +18] │ │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ │ ├── columns: p_partkey:17!null p_brand:20!null p_container:23!null - │ │ │ │ │ │ │ │ ├── key: (17) - │ │ │ │ │ │ │ │ ├── fd: ()-->(20,23) - │ │ │ │ │ │ │ │ ├── ordering: +17 opt(20,23) [actual: +17] + │ │ │ │ │ │ │ │ ├── columns: p_partkey:18!null p_brand:21!null p_container:24!null + │ │ │ │ │ │ │ │ ├── key: (18) + │ │ │ │ │ │ │ │ ├── fd: ()-->(21,24) + │ │ │ │ │ │ │ │ ├── ordering: +18 opt(21,24) [actual: +18] │ │ │ │ │ │ │ │ ├── scan part - │ │ │ │ │ │ │ │ │ ├── columns: p_partkey:17!null p_brand:20!null p_container:23!null - │ │ │ │ │ │ │ │ │ ├── key: (17) - │ │ │ │ │ │ │ │ │ ├── fd: (17)-->(20,23) - │ │ │ │ │ │ │ │ │ └── ordering: +17 opt(20,23) [actual: +17] + │ │ │ │ │ │ │ │ │ ├── columns: p_partkey:18!null p_brand:21!null p_container:24!null + │ │ │ │ │ │ │ │ │ ├── key: (18) + │ │ │ │ │ │ │ │ │ ├── fd: (18)-->(21,24) + │ │ │ │ │ │ │ │ │ └── ordering: +18 opt(21,24) [actual: +18] │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ ├── p_brand:20 = 'Brand#23' [outer=(20), constraints=(/20: [/'Brand#23' - /'Brand#23']; tight), fd=()-->(20)] - │ │ │ │ │ │ │ │ └── p_container:23 = 'MED BOX' [outer=(23), constraints=(/23: [/'MED BOX' - /'MED BOX']; tight), fd=()-->(23)] + │ │ │ │ │ │ │ │ ├── p_brand:21 = 'Brand#23' [outer=(21), constraints=(/21: [/'Brand#23' - /'Brand#23']; tight), fd=()-->(21)] + │ │ │ │ │ │ │ │ └── p_container:24 = 'MED BOX' [outer=(24), constraints=(/24: [/'MED BOX' - /'MED BOX']; tight), fd=()-->(24)] │ │ │ │ │ │ │ └── filters (true) │ │ │ │ │ │ └── filters (true) │ │ │ │ │ └── aggregations - │ │ │ │ │ └── avg [as=avg:42, outer=(30)] - │ │ │ │ │ └── l_quantity:30 + │ │ │ │ │ └── avg [as=avg:45, outer=(32)] + │ │ │ │ │ └── l_quantity:32 │ │ │ │ └── projections - │ │ │ │ └── avg:42 * 0.2 [as="?column?":43, outer=(42), immutable] + │ │ │ │ └── avg:45 * 0.2 [as="?column?":46, outer=(45), immutable] │ │ │ └── filters (true) │ │ └── filters - │ │ └── l_quantity:5 < "?column?":43 [outer=(5,43), constraints=(/5: (/NULL - ]; /43: (/NULL - ])] + │ │ └── l_quantity:5 < "?column?":46 [outer=(5,46), constraints=(/5: (/NULL - ]; /46: (/NULL - ])] │ └── aggregations - │ └── sum [as=sum:44, outer=(6)] + │ └── sum [as=sum:47, outer=(6)] │ └── l_extendedprice:6 └── projections - └── sum:44 / 7.0 [as=avg_yearly:45, outer=(44)] + └── sum:47 / 7.0 [as=avg_yearly:48, outer=(47)] # -------------------------------------------------- # Q18 @@ -1954,84 +1954,84 @@ ORDER BY LIMIT 100; ---- limit - ├── columns: c_name:2!null c_custkey:1!null o_orderkey:9!null o_orderdate:13!null o_totalprice:12!null sum:51!null - ├── internal-ordering: -12,+13 + ├── columns: c_name:2!null c_custkey:1!null o_orderkey:10!null o_orderdate:14!null o_totalprice:13!null sum:55!null + ├── internal-ordering: -13,+14 ├── cardinality: [0 - 100] - ├── key: (9) - ├── fd: (1)-->(2), (9)-->(1,2,12,13,51) - ├── ordering: -12,+13 + ├── key: (10) + ├── fd: (1)-->(2), (10)-->(1,2,13,14,55) + ├── ordering: -13,+14 ├── sort - │ ├── columns: c_custkey:1!null c_name:2!null o_orderkey:9!null o_totalprice:12!null o_orderdate:13!null sum:51!null - │ ├── key: (9) - │ ├── fd: (1)-->(2), (9)-->(1,2,12,13,51) - │ ├── ordering: -12,+13 + │ ├── columns: c_custkey:1!null c_name:2!null o_orderkey:10!null o_totalprice:13!null o_orderdate:14!null sum:55!null + │ ├── key: (10) + │ ├── fd: (1)-->(2), (10)-->(1,2,13,14,55) + │ ├── ordering: -13,+14 │ ├── limit hint: 100.00 │ └── group-by - │ ├── columns: c_custkey:1!null c_name:2!null o_orderkey:9!null o_totalprice:12!null o_orderdate:13!null sum:51!null - │ ├── grouping columns: o_orderkey:9!null - │ ├── key: (9) - │ ├── fd: (1)-->(2), (9)-->(1,2,12,13,51) + │ ├── columns: c_custkey:1!null c_name:2!null o_orderkey:10!null o_totalprice:13!null o_orderdate:14!null sum:55!null + │ ├── grouping columns: o_orderkey:10!null + │ ├── key: (10) + │ ├── fd: (1)-->(2), (10)-->(1,2,13,14,55) │ ├── inner-join (hash) - │ │ ├── columns: c_custkey:1!null c_name:2!null o_orderkey:9!null o_custkey:10!null o_totalprice:12!null o_orderdate:13!null l_orderkey:18!null l_quantity:22!null + │ │ ├── columns: c_custkey:1!null c_name:2!null o_orderkey:10!null o_custkey:11!null o_totalprice:13!null o_orderdate:14!null l_orderkey:20!null l_quantity:24!null │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ ├── fd: (1)-->(2), (9)-->(10,12,13), (9)==(18), (18)==(9), (1)==(10), (10)==(1) + │ │ ├── fd: (1)-->(2), (10)-->(11,13,14), (10)==(20), (20)==(10), (1)==(11), (11)==(1) │ │ ├── scan lineitem - │ │ │ └── columns: l_orderkey:18!null l_quantity:22!null + │ │ │ └── columns: l_orderkey:20!null l_quantity:24!null │ │ ├── inner-join (hash) - │ │ │ ├── columns: c_custkey:1!null c_name:2!null o_orderkey:9!null o_custkey:10!null o_totalprice:12!null o_orderdate:13!null + │ │ │ ├── columns: c_custkey:1!null c_name:2!null o_orderkey:10!null o_custkey:11!null o_totalprice:13!null o_orderdate:14!null │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ │ │ ├── key: (9) - │ │ │ ├── fd: (1)-->(2), (9)-->(10,12,13), (1)==(10), (10)==(1) + │ │ │ ├── key: (10) + │ │ │ ├── fd: (1)-->(2), (10)-->(11,13,14), (1)==(11), (11)==(1) │ │ │ ├── semi-join (merge) - │ │ │ │ ├── columns: o_orderkey:9!null o_custkey:10!null o_totalprice:12!null o_orderdate:13!null - │ │ │ │ ├── left ordering: +9 - │ │ │ │ ├── right ordering: +34 - │ │ │ │ ├── key: (9) - │ │ │ │ ├── fd: (9)-->(10,12,13) + │ │ │ │ ├── columns: o_orderkey:10!null o_custkey:11!null o_totalprice:13!null o_orderdate:14!null + │ │ │ │ ├── left ordering: +10 + │ │ │ │ ├── right ordering: +37 + │ │ │ │ ├── key: (10) + │ │ │ │ ├── fd: (10)-->(11,13,14) │ │ │ │ ├── scan orders - │ │ │ │ │ ├── columns: o_orderkey:9!null o_custkey:10!null o_totalprice:12!null o_orderdate:13!null - │ │ │ │ │ ├── key: (9) - │ │ │ │ │ ├── fd: (9)-->(10,12,13) - │ │ │ │ │ └── ordering: +9 + │ │ │ │ │ ├── columns: o_orderkey:10!null o_custkey:11!null o_totalprice:13!null o_orderdate:14!null + │ │ │ │ │ ├── key: (10) + │ │ │ │ │ ├── fd: (10)-->(11,13,14) + │ │ │ │ │ └── ordering: +10 │ │ │ │ ├── select - │ │ │ │ │ ├── columns: l_orderkey:34!null sum:50!null - │ │ │ │ │ ├── key: (34) - │ │ │ │ │ ├── fd: (34)-->(50) - │ │ │ │ │ ├── ordering: +34 + │ │ │ │ │ ├── columns: l_orderkey:37!null sum:54!null + │ │ │ │ │ ├── key: (37) + │ │ │ │ │ ├── fd: (37)-->(54) + │ │ │ │ │ ├── ordering: +37 │ │ │ │ │ ├── group-by - │ │ │ │ │ │ ├── columns: l_orderkey:34!null sum:50!null - │ │ │ │ │ │ ├── grouping columns: l_orderkey:34!null - │ │ │ │ │ │ ├── key: (34) - │ │ │ │ │ │ ├── fd: (34)-->(50) - │ │ │ │ │ │ ├── ordering: +34 + │ │ │ │ │ │ ├── columns: l_orderkey:37!null sum:54!null + │ │ │ │ │ │ ├── grouping columns: l_orderkey:37!null + │ │ │ │ │ │ ├── key: (37) + │ │ │ │ │ │ ├── fd: (37)-->(54) + │ │ │ │ │ │ ├── ordering: +37 │ │ │ │ │ │ ├── scan lineitem - │ │ │ │ │ │ │ ├── columns: l_orderkey:34!null l_quantity:38!null - │ │ │ │ │ │ │ └── ordering: +34 + │ │ │ │ │ │ │ ├── columns: l_orderkey:37!null l_quantity:41!null + │ │ │ │ │ │ │ └── ordering: +37 │ │ │ │ │ │ └── aggregations - │ │ │ │ │ │ └── sum [as=sum:50, outer=(38)] - │ │ │ │ │ │ └── l_quantity:38 + │ │ │ │ │ │ └── sum [as=sum:54, outer=(41)] + │ │ │ │ │ │ └── l_quantity:41 │ │ │ │ │ └── filters - │ │ │ │ │ └── sum:50 > 300.0 [outer=(50), constraints=(/50: [/300.00000000000006 - ]; tight)] + │ │ │ │ │ └── sum:54 > 300.0 [outer=(54), constraints=(/54: [/300.00000000000006 - ]; tight)] │ │ │ │ └── filters (true) │ │ │ ├── scan customer │ │ │ │ ├── columns: c_custkey:1!null c_name:2!null │ │ │ │ ├── key: (1) │ │ │ │ └── fd: (1)-->(2) │ │ │ └── filters - │ │ │ └── c_custkey:1 = o_custkey:10 [outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] + │ │ │ └── c_custkey:1 = o_custkey:11 [outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ]), fd=(1)==(11), (11)==(1)] │ │ └── filters - │ │ └── o_orderkey:9 = l_orderkey:18 [outer=(9,18), constraints=(/9: (/NULL - ]; /18: (/NULL - ]), fd=(9)==(18), (18)==(9)] + │ │ └── o_orderkey:10 = l_orderkey:20 [outer=(10,20), constraints=(/10: (/NULL - ]; /20: (/NULL - ]), fd=(10)==(20), (20)==(10)] │ └── aggregations - │ ├── sum [as=sum:51, outer=(22)] - │ │ └── l_quantity:22 + │ ├── sum [as=sum:55, outer=(24)] + │ │ └── l_quantity:24 │ ├── const-agg [as=c_custkey:1, outer=(1)] │ │ └── c_custkey:1 │ ├── const-agg [as=c_name:2, outer=(2)] │ │ └── c_name:2 - │ ├── const-agg [as=o_totalprice:12, outer=(12)] - │ │ └── o_totalprice:12 - │ └── const-agg [as=o_orderdate:13, outer=(13)] - │ └── o_orderdate:13 + │ ├── const-agg [as=o_totalprice:13, outer=(13)] + │ │ └── o_totalprice:13 + │ └── const-agg [as=o_orderdate:14, outer=(14)] + │ └── o_orderdate:14 └── 100 # -------------------------------------------------- @@ -2084,18 +2084,18 @@ WHERE ); ---- scalar-group-by - ├── columns: revenue:27 + ├── columns: revenue:29 ├── cardinality: [1 - 1] ├── immutable ├── key: () - ├── fd: ()-->(27) + ├── fd: ()-->(29) ├── project - │ ├── columns: column26:26!null + │ ├── columns: column28:28!null │ ├── immutable │ ├── inner-join (hash) - │ │ ├── columns: l_partkey:2!null l_quantity:5!null l_extendedprice:6!null l_discount:7!null l_shipinstruct:14!null l_shipmode:15!null p_partkey:17!null p_brand:20!null p_size:22!null p_container:23!null + │ │ ├── columns: l_partkey:2!null l_quantity:5!null l_extendedprice:6!null l_discount:7!null l_shipinstruct:14!null l_shipmode:15!null p_partkey:18!null p_brand:21!null p_size:23!null p_container:24!null │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ ├── fd: ()-->(14), (17)-->(20,22,23), (2)==(17), (17)==(2) + │ │ ├── fd: ()-->(14), (18)-->(21,23,24), (2)==(18), (18)==(2) │ │ ├── select │ │ │ ├── columns: l_partkey:2!null l_quantity:5!null l_extendedprice:6!null l_discount:7!null l_shipinstruct:14!null l_shipmode:15!null │ │ │ ├── fd: ()-->(14) @@ -2105,23 +2105,23 @@ scalar-group-by │ │ │ ├── l_shipmode:15 IN ('AIR', 'AIR REG') [outer=(15), constraints=(/15: [/'AIR' - /'AIR'] [/'AIR REG' - /'AIR REG']; tight)] │ │ │ └── l_shipinstruct:14 = 'DELIVER IN PERSON' [outer=(14), constraints=(/14: [/'DELIVER IN PERSON' - /'DELIVER IN PERSON']; tight), fd=()-->(14)] │ │ ├── select - │ │ │ ├── columns: p_partkey:17!null p_brand:20!null p_size:22!null p_container:23!null - │ │ │ ├── key: (17) - │ │ │ ├── fd: (17)-->(20,22,23) + │ │ │ ├── columns: p_partkey:18!null p_brand:21!null p_size:23!null p_container:24!null + │ │ │ ├── key: (18) + │ │ │ ├── fd: (18)-->(21,23,24) │ │ │ ├── scan part - │ │ │ │ ├── columns: p_partkey:17!null p_brand:20!null p_size:22!null p_container:23!null - │ │ │ │ ├── key: (17) - │ │ │ │ └── fd: (17)-->(20,22,23) + │ │ │ │ ├── columns: p_partkey:18!null p_brand:21!null p_size:23!null p_container:24!null + │ │ │ │ ├── key: (18) + │ │ │ │ └── fd: (18)-->(21,23,24) │ │ │ └── filters - │ │ │ └── p_size:22 >= 1 [outer=(22), constraints=(/22: [/1 - ]; tight)] + │ │ │ └── p_size:23 >= 1 [outer=(23), constraints=(/23: [/1 - ]; tight)] │ │ └── filters - │ │ ├── p_partkey:17 = l_partkey:2 [outer=(2,17), constraints=(/2: (/NULL - ]; /17: (/NULL - ]), fd=(2)==(17), (17)==(2)] - │ │ └── ((((((p_brand:20 = 'Brand#12') AND (p_container:23 IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG'))) AND (l_quantity:5 >= 1.0)) AND (l_quantity:5 <= 11.0)) AND (p_size:22 <= 5)) OR (((((p_brand:20 = 'Brand#23') AND (p_container:23 IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG'))) AND (l_quantity:5 >= 10.0)) AND (l_quantity:5 <= 20.0)) AND (p_size:22 <= 10))) OR (((((p_brand:20 = 'Brand#34') AND (p_container:23 IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG'))) AND (l_quantity:5 >= 20.0)) AND (l_quantity:5 <= 30.0)) AND (p_size:22 <= 15)) [outer=(5,20,22,23), constraints=(/5: [/1.0 - /30.0]; /20: [/'Brand#12' - /'Brand#12'] [/'Brand#23' - /'Brand#23'] [/'Brand#34' - /'Brand#34']; /22: (/NULL - /15]; /23: [/'LG BOX' - /'LG BOX'] [/'LG CASE' - /'LG CASE'] [/'LG PACK' - /'LG PACK'] [/'LG PKG' - /'LG PKG'] [/'MED BAG' - /'MED BAG'] [/'MED BOX' - /'MED BOX'] [/'MED PACK' - /'MED PACK'] [/'MED PKG' - /'MED PKG'] [/'SM BOX' - /'SM BOX'] [/'SM CASE' - /'SM CASE'] [/'SM PACK' - /'SM PACK'] [/'SM PKG' - /'SM PKG'])] + │ │ ├── p_partkey:18 = l_partkey:2 [outer=(2,18), constraints=(/2: (/NULL - ]; /18: (/NULL - ]), fd=(2)==(18), (18)==(2)] + │ │ └── ((((((p_brand:21 = 'Brand#12') AND (p_container:24 IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG'))) AND (l_quantity:5 >= 1.0)) AND (l_quantity:5 <= 11.0)) AND (p_size:23 <= 5)) OR (((((p_brand:21 = 'Brand#23') AND (p_container:24 IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG'))) AND (l_quantity:5 >= 10.0)) AND (l_quantity:5 <= 20.0)) AND (p_size:23 <= 10))) OR (((((p_brand:21 = 'Brand#34') AND (p_container:24 IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG'))) AND (l_quantity:5 >= 20.0)) AND (l_quantity:5 <= 30.0)) AND (p_size:23 <= 15)) [outer=(5,21,23,24), constraints=(/5: [/1.0 - /30.0]; /21: [/'Brand#12' - /'Brand#12'] [/'Brand#23' - /'Brand#23'] [/'Brand#34' - /'Brand#34']; /23: (/NULL - /15]; /24: [/'LG BOX' - /'LG BOX'] [/'LG CASE' - /'LG CASE'] [/'LG PACK' - /'LG PACK'] [/'LG PKG' - /'LG PKG'] [/'MED BAG' - /'MED BAG'] [/'MED BOX' - /'MED BOX'] [/'MED PACK' - /'MED PACK'] [/'MED PKG' - /'MED PKG'] [/'SM BOX' - /'SM BOX'] [/'SM CASE' - /'SM CASE'] [/'SM PACK' - /'SM PACK'] [/'SM PKG' - /'SM PKG'])] │ └── projections - │ └── l_extendedprice:6 * (1.0 - l_discount:7) [as=column26:26, outer=(6,7), immutable] + │ └── l_extendedprice:6 * (1.0 - l_discount:7) [as=column28:28, outer=(6,7), immutable] └── aggregations - └── sum [as=sum:27, outer=(26)] - └── column26:26 + └── sum [as=sum:29, outer=(28)] + └── column28:28 # -------------------------------------------------- # Q20 @@ -2184,11 +2184,11 @@ sort ├── columns: s_name:2!null s_address:3!null ├── immutable └── inner-join (hash) - ├── columns: s_suppkey:1!null s_name:2!null s_address:3!null s_nationkey:4!null n_nationkey:8!null n_name:9!null + ├── columns: s_suppkey:1!null s_name:2!null s_address:3!null s_nationkey:4!null n_nationkey:9!null n_name:10!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) ├── immutable ├── key: (1) - ├── fd: ()-->(9), (1)-->(2-4), (4)==(8), (8)==(4) + ├── fd: ()-->(10), (1)-->(2-4), (4)==(9), (9)==(4) ├── semi-join (hash) │ ├── columns: s_suppkey:1!null s_name:2!null s_address:3!null s_nationkey:4!null │ ├── immutable @@ -2199,80 +2199,80 @@ sort │ │ ├── key: (1) │ │ └── fd: (1)-->(2-4) │ ├── project - │ │ ├── columns: ps_partkey:12!null ps_suppkey:13!null + │ │ ├── columns: ps_partkey:14!null ps_suppkey:15!null │ │ ├── immutable - │ │ ├── key: (12,13) + │ │ ├── key: (14,15) │ │ └── project - │ │ ├── columns: ps_partkey:12!null ps_suppkey:13!null p_partkey:17!null + │ │ ├── columns: ps_partkey:14!null ps_suppkey:15!null p_partkey:20!null │ │ ├── immutable - │ │ ├── key: (13,17) - │ │ ├── fd: (12)==(17), (17)==(12) + │ │ ├── key: (15,20) + │ │ ├── fd: (14)==(20), (20)==(14) │ │ └── inner-join (hash) - │ │ ├── columns: ps_partkey:12!null ps_suppkey:13!null ps_availqty:14!null p_partkey:17!null p_name:18!null sum:42 + │ │ ├── columns: ps_partkey:14!null ps_suppkey:15!null ps_availqty:16!null p_partkey:20!null p_name:21!null sum:47 │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) │ │ ├── immutable - │ │ ├── key: (13,17) - │ │ ├── fd: (12,13)-->(14,42), (17)-->(18), (12)==(17), (17)==(12) + │ │ ├── key: (15,20) + │ │ ├── fd: (14,15)-->(16,47), (20)-->(21), (14)==(20), (20)==(14) │ │ ├── select - │ │ │ ├── columns: ps_partkey:12!null ps_suppkey:13!null ps_availqty:14!null sum:42 + │ │ │ ├── columns: ps_partkey:14!null ps_suppkey:15!null ps_availqty:16!null sum:47 │ │ │ ├── immutable - │ │ │ ├── key: (12,13) - │ │ │ ├── fd: (12,13)-->(14,42) + │ │ │ ├── key: (14,15) + │ │ │ ├── fd: (14,15)-->(16,47) │ │ │ ├── group-by - │ │ │ │ ├── columns: ps_partkey:12!null ps_suppkey:13!null ps_availqty:14!null sum:42 - │ │ │ │ ├── grouping columns: ps_partkey:12!null ps_suppkey:13!null - │ │ │ │ ├── key: (12,13) - │ │ │ │ ├── fd: (12,13)-->(14,42) + │ │ │ │ ├── columns: ps_partkey:14!null ps_suppkey:15!null ps_availqty:16!null sum:47 + │ │ │ │ ├── grouping columns: ps_partkey:14!null ps_suppkey:15!null + │ │ │ │ ├── key: (14,15) + │ │ │ │ ├── fd: (14,15)-->(16,47) │ │ │ │ ├── right-join (hash) - │ │ │ │ │ ├── columns: ps_partkey:12!null ps_suppkey:13!null ps_availqty:14!null l_partkey:27 l_suppkey:28 l_quantity:30 l_shipdate:36 - │ │ │ │ │ ├── fd: (12,13)-->(14) + │ │ │ │ │ ├── columns: ps_partkey:14!null ps_suppkey:15!null ps_availqty:16!null l_partkey:31 l_suppkey:32 l_quantity:34 l_shipdate:40 + │ │ │ │ │ ├── fd: (14,15)-->(16) │ │ │ │ │ ├── index-join lineitem - │ │ │ │ │ │ ├── columns: l_partkey:27!null l_suppkey:28!null l_quantity:30!null l_shipdate:36!null + │ │ │ │ │ │ ├── columns: l_partkey:31!null l_suppkey:32!null l_quantity:34!null l_shipdate:40!null │ │ │ │ │ │ └── scan lineitem@l_sd - │ │ │ │ │ │ ├── columns: l_orderkey:26!null l_linenumber:29!null l_shipdate:36!null - │ │ │ │ │ │ ├── constraint: /36/26/29: [/'1994-01-01' - /'1994-12-31'] - │ │ │ │ │ │ ├── key: (26,29) - │ │ │ │ │ │ └── fd: (26,29)-->(36) + │ │ │ │ │ │ ├── columns: l_orderkey:30!null l_linenumber:33!null l_shipdate:40!null + │ │ │ │ │ │ ├── constraint: /40/30/33: [/'1994-01-01' - /'1994-12-31'] + │ │ │ │ │ │ ├── key: (30,33) + │ │ │ │ │ │ └── fd: (30,33)-->(40) │ │ │ │ │ ├── scan partsupp - │ │ │ │ │ │ ├── columns: ps_partkey:12!null ps_suppkey:13!null ps_availqty:14!null - │ │ │ │ │ │ ├── key: (12,13) - │ │ │ │ │ │ └── fd: (12,13)-->(14) + │ │ │ │ │ │ ├── columns: ps_partkey:14!null ps_suppkey:15!null ps_availqty:16!null + │ │ │ │ │ │ ├── key: (14,15) + │ │ │ │ │ │ └── fd: (14,15)-->(16) │ │ │ │ │ └── filters - │ │ │ │ │ ├── l_partkey:27 = ps_partkey:12 [outer=(12,27), constraints=(/12: (/NULL - ]; /27: (/NULL - ]), fd=(12)==(27), (27)==(12)] - │ │ │ │ │ └── l_suppkey:28 = ps_suppkey:13 [outer=(13,28), constraints=(/13: (/NULL - ]; /28: (/NULL - ]), fd=(13)==(28), (28)==(13)] + │ │ │ │ │ ├── l_partkey:31 = ps_partkey:14 [outer=(14,31), constraints=(/14: (/NULL - ]; /31: (/NULL - ]), fd=(14)==(31), (31)==(14)] + │ │ │ │ │ └── l_suppkey:32 = ps_suppkey:15 [outer=(15,32), constraints=(/15: (/NULL - ]; /32: (/NULL - ]), fd=(15)==(32), (32)==(15)] │ │ │ │ └── aggregations - │ │ │ │ ├── sum [as=sum:42, outer=(30)] - │ │ │ │ │ └── l_quantity:30 - │ │ │ │ └── const-agg [as=ps_availqty:14, outer=(14)] - │ │ │ │ └── ps_availqty:14 + │ │ │ │ ├── sum [as=sum:47, outer=(34)] + │ │ │ │ │ └── l_quantity:34 + │ │ │ │ └── const-agg [as=ps_availqty:16, outer=(16)] + │ │ │ │ └── ps_availqty:16 │ │ │ └── filters - │ │ │ └── ps_availqty:14 > (sum:42 * 0.5) [outer=(14,42), immutable, constraints=(/14: (/NULL - ])] + │ │ │ └── ps_availqty:16 > (sum:47 * 0.5) [outer=(16,47), immutable, constraints=(/16: (/NULL - ])] │ │ ├── select - │ │ │ ├── columns: p_partkey:17!null p_name:18!null - │ │ │ ├── key: (17) - │ │ │ ├── fd: (17)-->(18) + │ │ │ ├── columns: p_partkey:20!null p_name:21!null + │ │ │ ├── key: (20) + │ │ │ ├── fd: (20)-->(21) │ │ │ ├── scan part - │ │ │ │ ├── columns: p_partkey:17!null p_name:18!null - │ │ │ │ ├── key: (17) - │ │ │ │ └── fd: (17)-->(18) + │ │ │ │ ├── columns: p_partkey:20!null p_name:21!null + │ │ │ │ ├── key: (20) + │ │ │ │ └── fd: (20)-->(21) │ │ │ └── filters - │ │ │ └── p_name:18 LIKE 'forest%' [outer=(18), constraints=(/18: [/'forest' - /'foresu'); tight)] + │ │ │ └── p_name:21 LIKE 'forest%' [outer=(21), constraints=(/21: [/'forest' - /'foresu'); tight)] │ │ └── filters - │ │ └── ps_partkey:12 = p_partkey:17 [outer=(12,17), constraints=(/12: (/NULL - ]; /17: (/NULL - ]), fd=(12)==(17), (17)==(12)] + │ │ └── ps_partkey:14 = p_partkey:20 [outer=(14,20), constraints=(/14: (/NULL - ]; /20: (/NULL - ]), fd=(14)==(20), (20)==(14)] │ └── filters - │ └── s_suppkey:1 = ps_suppkey:13 [outer=(1,13), constraints=(/1: (/NULL - ]; /13: (/NULL - ]), fd=(1)==(13), (13)==(1)] + │ └── s_suppkey:1 = ps_suppkey:15 [outer=(1,15), constraints=(/1: (/NULL - ]; /15: (/NULL - ]), fd=(1)==(15), (15)==(1)] ├── select - │ ├── columns: n_nationkey:8!null n_name:9!null - │ ├── key: (8) - │ ├── fd: ()-->(9) + │ ├── columns: n_nationkey:9!null n_name:10!null + │ ├── key: (9) + │ ├── fd: ()-->(10) │ ├── scan nation - │ │ ├── columns: n_nationkey:8!null n_name:9!null - │ │ ├── key: (8) - │ │ └── fd: (8)-->(9) + │ │ ├── columns: n_nationkey:9!null n_name:10!null + │ │ ├── key: (9) + │ │ └── fd: (9)-->(10) │ └── filters - │ └── n_name:9 = 'CANADA' [outer=(9), constraints=(/9: [/'CANADA' - /'CANADA']; tight), fd=()-->(9)] + │ └── n_name:10 = 'CANADA' [outer=(10), constraints=(/10: [/'CANADA' - /'CANADA']; tight), fd=()-->(10)] └── filters - └── s_nationkey:4 = n_nationkey:8 [outer=(4,8), constraints=(/4: (/NULL - ]; /8: (/NULL - ]), fd=(4)==(8), (8)==(4)] + └── s_nationkey:4 = n_nationkey:9 [outer=(4,9), constraints=(/4: (/NULL - ]; /9: (/NULL - ]), fd=(4)==(9), (9)==(4)] # -------------------------------------------------- # Q21 @@ -2327,90 +2327,90 @@ ORDER BY LIMIT 100; ---- limit - ├── columns: s_name:2!null numwait:69!null - ├── internal-ordering: -69,+2 + ├── columns: s_name:2!null numwait:75!null + ├── internal-ordering: -75,+2 ├── cardinality: [0 - 100] ├── key: (2) - ├── fd: (2)-->(69) - ├── ordering: -69,+2 + ├── fd: (2)-->(75) + ├── ordering: -75,+2 ├── sort - │ ├── columns: s_name:2!null count_rows:69!null + │ ├── columns: s_name:2!null count_rows:75!null │ ├── key: (2) - │ ├── fd: (2)-->(69) - │ ├── ordering: -69,+2 + │ ├── fd: (2)-->(75) + │ ├── ordering: -75,+2 │ ├── limit hint: 100.00 │ └── group-by - │ ├── columns: s_name:2!null count_rows:69!null + │ ├── columns: s_name:2!null count_rows:75!null │ ├── grouping columns: s_name:2!null │ ├── key: (2) - │ ├── fd: (2)-->(69) + │ ├── fd: (2)-->(75) │ ├── inner-join (lookup orders) - │ │ ├── columns: s_suppkey:1!null s_name:2!null s_nationkey:4!null l1.l_orderkey:8!null l1.l_suppkey:10!null l1.l_commitdate:19!null l1.l_receiptdate:20!null o_orderkey:24!null o_orderstatus:26!null n_nationkey:33!null n_name:34!null - │ │ ├── key columns: [8] = [24] + │ │ ├── columns: s_suppkey:1!null s_name:2!null s_nationkey:4!null l1.l_orderkey:9!null l1.l_suppkey:11!null l1.l_commitdate:20!null l1.l_receiptdate:21!null o_orderkey:26!null o_orderstatus:28!null n_nationkey:36!null n_name:37!null + │ │ ├── key columns: [9] = [26] │ │ ├── lookup columns are key - │ │ ├── fd: ()-->(26,34), (1)-->(2,4), (8)==(24), (24)==(8), (1)==(10), (10)==(1), (4)==(33), (33)==(4) + │ │ ├── fd: ()-->(28,37), (1)-->(2,4), (9)==(26), (26)==(9), (1)==(11), (11)==(1), (4)==(36), (36)==(4) │ │ ├── inner-join (hash) - │ │ │ ├── columns: s_suppkey:1!null s_name:2!null s_nationkey:4!null l1.l_orderkey:8!null l1.l_suppkey:10!null l1.l_commitdate:19!null l1.l_receiptdate:20!null n_nationkey:33!null n_name:34!null + │ │ │ ├── columns: s_suppkey:1!null s_name:2!null s_nationkey:4!null l1.l_orderkey:9!null l1.l_suppkey:11!null l1.l_commitdate:20!null l1.l_receiptdate:21!null n_nationkey:36!null n_name:37!null │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ ├── fd: ()-->(34), (1)-->(2,4), (4)==(33), (33)==(4), (1)==(10), (10)==(1) + │ │ │ ├── fd: ()-->(37), (1)-->(2,4), (4)==(36), (36)==(4), (1)==(11), (11)==(1) │ │ │ ├── semi-join (hash) - │ │ │ │ ├── columns: l1.l_orderkey:8!null l1.l_suppkey:10!null l1.l_commitdate:19!null l1.l_receiptdate:20!null + │ │ │ │ ├── columns: l1.l_orderkey:9!null l1.l_suppkey:11!null l1.l_commitdate:20!null l1.l_receiptdate:21!null │ │ │ │ ├── anti-join (merge) - │ │ │ │ │ ├── columns: l1.l_orderkey:8!null l1.l_suppkey:10!null l1.l_commitdate:19!null l1.l_receiptdate:20!null - │ │ │ │ │ ├── left ordering: +8 - │ │ │ │ │ ├── right ordering: +53 + │ │ │ │ │ ├── columns: l1.l_orderkey:9!null l1.l_suppkey:11!null l1.l_commitdate:20!null l1.l_receiptdate:21!null + │ │ │ │ │ ├── left ordering: +9 + │ │ │ │ │ ├── right ordering: +58 │ │ │ │ │ ├── select - │ │ │ │ │ │ ├── columns: l1.l_orderkey:8!null l1.l_suppkey:10!null l1.l_commitdate:19!null l1.l_receiptdate:20!null - │ │ │ │ │ │ ├── ordering: +8 + │ │ │ │ │ │ ├── columns: l1.l_orderkey:9!null l1.l_suppkey:11!null l1.l_commitdate:20!null l1.l_receiptdate:21!null + │ │ │ │ │ │ ├── ordering: +9 │ │ │ │ │ │ ├── scan l1 - │ │ │ │ │ │ │ ├── columns: l1.l_orderkey:8!null l1.l_suppkey:10!null l1.l_commitdate:19!null l1.l_receiptdate:20!null - │ │ │ │ │ │ │ └── ordering: +8 + │ │ │ │ │ │ │ ├── columns: l1.l_orderkey:9!null l1.l_suppkey:11!null l1.l_commitdate:20!null l1.l_receiptdate:21!null + │ │ │ │ │ │ │ └── ordering: +9 │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── l1.l_receiptdate:20 > l1.l_commitdate:19 [outer=(19,20), constraints=(/19: (/NULL - ]; /20: (/NULL - ])] + │ │ │ │ │ │ └── l1.l_receiptdate:21 > l1.l_commitdate:20 [outer=(20,21), constraints=(/20: (/NULL - ]; /21: (/NULL - ])] │ │ │ │ │ ├── select - │ │ │ │ │ │ ├── columns: l3.l_orderkey:53!null l3.l_suppkey:55!null l3.l_commitdate:64!null l3.l_receiptdate:65!null - │ │ │ │ │ │ ├── ordering: +53 + │ │ │ │ │ │ ├── columns: l3.l_orderkey:58!null l3.l_suppkey:60!null l3.l_commitdate:69!null l3.l_receiptdate:70!null + │ │ │ │ │ │ ├── ordering: +58 │ │ │ │ │ │ ├── scan l3 - │ │ │ │ │ │ │ ├── columns: l3.l_orderkey:53!null l3.l_suppkey:55!null l3.l_commitdate:64!null l3.l_receiptdate:65!null - │ │ │ │ │ │ │ └── ordering: +53 + │ │ │ │ │ │ │ ├── columns: l3.l_orderkey:58!null l3.l_suppkey:60!null l3.l_commitdate:69!null l3.l_receiptdate:70!null + │ │ │ │ │ │ │ └── ordering: +58 │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── l3.l_receiptdate:65 > l3.l_commitdate:64 [outer=(64,65), constraints=(/64: (/NULL - ]; /65: (/NULL - ])] + │ │ │ │ │ │ └── l3.l_receiptdate:70 > l3.l_commitdate:69 [outer=(69,70), constraints=(/69: (/NULL - ]; /70: (/NULL - ])] │ │ │ │ │ └── filters - │ │ │ │ │ └── l3.l_suppkey:55 != l1.l_suppkey:10 [outer=(10,55), constraints=(/10: (/NULL - ]; /55: (/NULL - ])] + │ │ │ │ │ └── l3.l_suppkey:60 != l1.l_suppkey:11 [outer=(11,60), constraints=(/11: (/NULL - ]; /60: (/NULL - ])] │ │ │ │ ├── scan l2@l_sk - │ │ │ │ │ └── columns: l2.l_orderkey:37!null l2.l_suppkey:39!null + │ │ │ │ │ └── columns: l2.l_orderkey:41!null l2.l_suppkey:43!null │ │ │ │ └── filters - │ │ │ │ ├── l2.l_orderkey:37 = l1.l_orderkey:8 [outer=(8,37), constraints=(/8: (/NULL - ]; /37: (/NULL - ]), fd=(8)==(37), (37)==(8)] - │ │ │ │ └── l2.l_suppkey:39 != l1.l_suppkey:10 [outer=(10,39), constraints=(/10: (/NULL - ]; /39: (/NULL - ])] + │ │ │ │ ├── l2.l_orderkey:41 = l1.l_orderkey:9 [outer=(9,41), constraints=(/9: (/NULL - ]; /41: (/NULL - ]), fd=(9)==(41), (41)==(9)] + │ │ │ │ └── l2.l_suppkey:43 != l1.l_suppkey:11 [outer=(11,43), constraints=(/11: (/NULL - ]; /43: (/NULL - ])] │ │ │ ├── inner-join (lookup supplier) - │ │ │ │ ├── columns: s_suppkey:1!null s_name:2!null s_nationkey:4!null n_nationkey:33!null n_name:34!null + │ │ │ │ ├── columns: s_suppkey:1!null s_name:2!null s_nationkey:4!null n_nationkey:36!null n_name:37!null │ │ │ │ ├── key columns: [1] = [1] │ │ │ │ ├── lookup columns are key │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: ()-->(34), (1)-->(2,4), (4)==(33), (33)==(4) + │ │ │ │ ├── fd: ()-->(37), (1)-->(2,4), (4)==(36), (36)==(4) │ │ │ │ ├── inner-join (lookup supplier@s_nk) - │ │ │ │ │ ├── columns: s_suppkey:1!null s_nationkey:4!null n_nationkey:33!null n_name:34!null - │ │ │ │ │ ├── key columns: [33] = [4] + │ │ │ │ │ ├── columns: s_suppkey:1!null s_nationkey:4!null n_nationkey:36!null n_name:37!null + │ │ │ │ │ ├── key columns: [36] = [4] │ │ │ │ │ ├── key: (1) - │ │ │ │ │ ├── fd: ()-->(34), (1)-->(4), (4)==(33), (33)==(4) + │ │ │ │ │ ├── fd: ()-->(37), (1)-->(4), (4)==(36), (36)==(4) │ │ │ │ │ ├── select - │ │ │ │ │ │ ├── columns: n_nationkey:33!null n_name:34!null - │ │ │ │ │ │ ├── key: (33) - │ │ │ │ │ │ ├── fd: ()-->(34) + │ │ │ │ │ │ ├── columns: n_nationkey:36!null n_name:37!null + │ │ │ │ │ │ ├── key: (36) + │ │ │ │ │ │ ├── fd: ()-->(37) │ │ │ │ │ │ ├── scan nation - │ │ │ │ │ │ │ ├── columns: n_nationkey:33!null n_name:34!null - │ │ │ │ │ │ │ ├── key: (33) - │ │ │ │ │ │ │ └── fd: (33)-->(34) + │ │ │ │ │ │ │ ├── columns: n_nationkey:36!null n_name:37!null + │ │ │ │ │ │ │ ├── key: (36) + │ │ │ │ │ │ │ └── fd: (36)-->(37) │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── n_name:34 = 'SAUDI ARABIA' [outer=(34), constraints=(/34: [/'SAUDI ARABIA' - /'SAUDI ARABIA']; tight), fd=()-->(34)] + │ │ │ │ │ │ └── n_name:37 = 'SAUDI ARABIA' [outer=(37), constraints=(/37: [/'SAUDI ARABIA' - /'SAUDI ARABIA']; tight), fd=()-->(37)] │ │ │ │ │ └── filters (true) │ │ │ │ └── filters (true) │ │ │ └── filters - │ │ │ └── s_suppkey:1 = l1.l_suppkey:10 [outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] + │ │ │ └── s_suppkey:1 = l1.l_suppkey:11 [outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ]), fd=(1)==(11), (11)==(1)] │ │ └── filters - │ │ └── o_orderstatus:26 = 'F' [outer=(26), constraints=(/26: [/'F' - /'F']; tight), fd=()-->(26)] + │ │ └── o_orderstatus:28 = 'F' [outer=(28), constraints=(/28: [/'F' - /'F']; tight), fd=()-->(28)] │ └── aggregations - │ └── count-rows [as=count_rows:69] + │ └── count-rows [as=count_rows:75] └── 100 # -------------------------------------------------- @@ -2463,23 +2463,23 @@ ORDER BY cntrycode; ---- sort - ├── columns: cntrycode:27 numcust:28!null totacctbal:29!null + ├── columns: cntrycode:30 numcust:31!null totacctbal:32!null ├── immutable - ├── key: (27) - ├── fd: (27)-->(28,29) - ├── ordering: +27 + ├── key: (30) + ├── fd: (30)-->(31,32) + ├── ordering: +30 └── group-by - ├── columns: cntrycode:27 count_rows:28!null sum:29!null - ├── grouping columns: cntrycode:27 + ├── columns: cntrycode:30 count_rows:31!null sum:32!null + ├── grouping columns: cntrycode:30 ├── immutable - ├── key: (27) - ├── fd: (27)-->(28,29) + ├── key: (30) + ├── fd: (30)-->(31,32) ├── project - │ ├── columns: cntrycode:27 c_acctbal:6!null + │ ├── columns: cntrycode:30 c_acctbal:6!null │ ├── immutable │ ├── anti-join (lookup orders@o_ck) │ │ ├── columns: c_custkey:1!null c_phone:5!null c_acctbal:6!null - │ │ ├── key columns: [1] = [19] + │ │ ├── key columns: [1] = [21] │ │ ├── immutable │ │ ├── key: (1) │ │ ├── fd: (1)-->(5,6) @@ -2498,26 +2498,26 @@ sort │ │ │ ├── c_acctbal:6 │ │ │ └── subquery │ │ │ └── scalar-group-by - │ │ │ ├── columns: avg:17 + │ │ │ ├── columns: avg:19 │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── immutable │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(17) + │ │ │ ├── fd: ()-->(19) │ │ │ ├── select - │ │ │ │ ├── columns: c_phone:13!null c_acctbal:14!null + │ │ │ │ ├── columns: c_phone:14!null c_acctbal:15!null │ │ │ │ ├── immutable │ │ │ │ ├── scan customer - │ │ │ │ │ └── columns: c_phone:13!null c_acctbal:14!null + │ │ │ │ │ └── columns: c_phone:14!null c_acctbal:15!null │ │ │ │ └── filters - │ │ │ │ ├── c_acctbal:14 > 0.0 [outer=(14), constraints=(/14: [/5e-324 - ]; tight)] - │ │ │ │ └── substring(c_phone:13, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31') [outer=(13), immutable] + │ │ │ │ ├── c_acctbal:15 > 0.0 [outer=(15), constraints=(/15: [/5e-324 - ]; tight)] + │ │ │ │ └── substring(c_phone:14, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31') [outer=(14), immutable] │ │ │ └── aggregations - │ │ │ └── avg [as=avg:17, outer=(14)] - │ │ │ └── c_acctbal:14 + │ │ │ └── avg [as=avg:19, outer=(15)] + │ │ │ └── c_acctbal:15 │ │ └── filters (true) │ └── projections - │ └── substring(c_phone:5, 1, 2) [as=cntrycode:27, outer=(5), immutable] + │ └── substring(c_phone:5, 1, 2) [as=cntrycode:30, outer=(5), immutable] └── aggregations - ├── count-rows [as=count_rows:28] - └── sum [as=sum:29, outer=(6)] + ├── count-rows [as=count_rows:31] + └── sum [as=sum:32, outer=(6)] └── c_acctbal:6 diff --git a/pkg/sql/opt/xform/testdata/external/tpch-no-stats b/pkg/sql/opt/xform/testdata/external/tpch-no-stats index a15e640f6cba..98d118aad9ad 100644 --- a/pkg/sql/opt/xform/testdata/external/tpch-no-stats +++ b/pkg/sql/opt/xform/testdata/external/tpch-no-stats @@ -38,18 +38,18 @@ ORDER BY l_linestatus; ---- group-by - ├── columns: l_returnflag:9!null l_linestatus:10!null sum_qty:17!null sum_base_price:18!null sum_disc_price:20!null sum_charge:22!null avg_qty:23!null avg_price:24!null avg_disc:25!null count_order:26!null + ├── columns: l_returnflag:9!null l_linestatus:10!null sum_qty:18!null sum_base_price:19!null sum_disc_price:21!null sum_charge:23!null avg_qty:24!null avg_price:25!null avg_disc:26!null count_order:27!null ├── grouping columns: l_returnflag:9!null l_linestatus:10!null ├── immutable ├── key: (9,10) - ├── fd: (9,10)-->(17,18,20,22-26) + ├── fd: (9,10)-->(18,19,21,23-27) ├── ordering: +9,+10 ├── sort - │ ├── columns: l_quantity:5!null l_extendedprice:6!null l_discount:7!null l_returnflag:9!null l_linestatus:10!null column19:19!null column21:21!null + │ ├── columns: l_quantity:5!null l_extendedprice:6!null l_discount:7!null l_returnflag:9!null l_linestatus:10!null column20:20!null column22:22!null │ ├── immutable │ ├── ordering: +9,+10 │ └── project - │ ├── columns: column19:19!null column21:21!null l_quantity:5!null l_extendedprice:6!null l_discount:7!null l_returnflag:9!null l_linestatus:10!null + │ ├── columns: column20:20!null column22:22!null l_quantity:5!null l_extendedprice:6!null l_discount:7!null l_returnflag:9!null l_linestatus:10!null │ ├── immutable │ ├── select │ │ ├── columns: l_quantity:5!null l_extendedprice:6!null l_discount:7!null l_tax:8!null l_returnflag:9!null l_linestatus:10!null l_shipdate:11!null @@ -58,24 +58,24 @@ group-by │ │ └── filters │ │ └── l_shipdate:11 <= '1998-09-02' [outer=(11), constraints=(/11: (/NULL - /'1998-09-02']; tight)] │ └── projections - │ ├── l_extendedprice:6 * (1.0 - l_discount:7) [as=column19:19, outer=(6,7), immutable] - │ └── (l_extendedprice:6 * (1.0 - l_discount:7)) * (l_tax:8 + 1.0) [as=column21:21, outer=(6-8), immutable] + │ ├── l_extendedprice:6 * (1.0 - l_discount:7) [as=column20:20, outer=(6,7), immutable] + │ └── (l_extendedprice:6 * (1.0 - l_discount:7)) * (l_tax:8 + 1.0) [as=column22:22, outer=(6-8), immutable] └── aggregations - ├── sum [as=sum:17, outer=(5)] + ├── sum [as=sum:18, outer=(5)] │ └── l_quantity:5 - ├── sum [as=sum:18, outer=(6)] + ├── sum [as=sum:19, outer=(6)] │ └── l_extendedprice:6 - ├── sum [as=sum:20, outer=(19)] - │ └── column19:19 - ├── sum [as=sum:22, outer=(21)] - │ └── column21:21 - ├── avg [as=avg:23, outer=(5)] + ├── sum [as=sum:21, outer=(20)] + │ └── column20:20 + ├── sum [as=sum:23, outer=(22)] + │ └── column22:22 + ├── avg [as=avg:24, outer=(5)] │ └── l_quantity:5 - ├── avg [as=avg:24, outer=(6)] + ├── avg [as=avg:25, outer=(6)] │ └── l_extendedprice:6 - ├── avg [as=avg:25, outer=(7)] + ├── avg [as=avg:26, outer=(7)] │ └── l_discount:7 - └── count-rows [as=count_rows:26] + └── count-rows [as=count_rows:27] # -------------------------------------------------- # Q2 @@ -142,91 +142,91 @@ ORDER BY LIMIT 100; ---- project - ├── columns: s_acctbal:15!null s_name:11!null n_name:23!null p_partkey:1!null p_mfgr:3!null s_address:12!null s_phone:14!null s_comment:16!null + ├── columns: s_acctbal:16!null s_name:12!null n_name:26!null p_partkey:1!null p_mfgr:3!null s_address:13!null s_phone:15!null s_comment:17!null ├── cardinality: [0 - 100] ├── fd: (1)-->(3) - ├── ordering: -15,+23,+11,+1 + ├── ordering: -16,+26,+12,+1 └── limit - ├── columns: p_partkey:1!null p_mfgr:3!null s_name:11!null s_address:12!null s_phone:14!null s_acctbal:15!null s_comment:16!null ps_partkey:17!null ps_suppkey:18!null ps_supplycost:20!null n_name:23!null min:48!null - ├── internal-ordering: -15,+23,+11,+(1|17) + ├── columns: p_partkey:1!null p_mfgr:3!null s_name:12!null s_address:13!null s_phone:15!null s_acctbal:16!null s_comment:17!null ps_partkey:19!null ps_suppkey:20!null ps_supplycost:22!null n_name:26!null min:57!null + ├── internal-ordering: -16,+26,+12,+(1|19) ├── cardinality: [0 - 100] - ├── key: (17,18) - ├── fd: (1)-->(3), (17,18)-->(1,3,11,12,14-16,20,23,48), (1)==(17), (17)==(1), (18)-->(11,12,14-16,23), (20)==(48), (48)==(20) - ├── ordering: -15,+23,+11,+(1|17) [actual: -15,+23,+11,+1] + ├── key: (19,20) + ├── fd: (1)-->(3), (19,20)-->(1,3,12,13,15-17,22,26,57), (1)==(19), (19)==(1), (20)-->(12,13,15-17,26), (22)==(57), (57)==(22) + ├── ordering: -16,+26,+12,+(1|19) [actual: -16,+26,+12,+1] ├── sort - │ ├── columns: p_partkey:1!null p_mfgr:3!null s_name:11!null s_address:12!null s_phone:14!null s_acctbal:15!null s_comment:16!null ps_partkey:17!null ps_suppkey:18!null ps_supplycost:20!null n_name:23!null min:48!null - │ ├── key: (17,18) - │ ├── fd: (1)-->(3), (17,18)-->(1,3,11,12,14-16,20,23,48), (1)==(17), (17)==(1), (18)-->(11,12,14-16,23), (20)==(48), (48)==(20) - │ ├── ordering: -15,+23,+11,+(1|17) [actual: -15,+23,+11,+1] + │ ├── columns: p_partkey:1!null p_mfgr:3!null s_name:12!null s_address:13!null s_phone:15!null s_acctbal:16!null s_comment:17!null ps_partkey:19!null ps_suppkey:20!null ps_supplycost:22!null n_name:26!null min:57!null + │ ├── key: (19,20) + │ ├── fd: (1)-->(3), (19,20)-->(1,3,12,13,15-17,22,26,57), (1)==(19), (19)==(1), (20)-->(12,13,15-17,26), (22)==(57), (57)==(22) + │ ├── ordering: -16,+26,+12,+(1|19) [actual: -16,+26,+12,+1] │ ├── limit hint: 100.00 │ └── select - │ ├── columns: p_partkey:1!null p_mfgr:3!null s_name:11!null s_address:12!null s_phone:14!null s_acctbal:15!null s_comment:16!null ps_partkey:17!null ps_suppkey:18!null ps_supplycost:20!null n_name:23!null min:48!null - │ ├── key: (17,18) - │ ├── fd: (1)-->(3), (17,18)-->(1,3,11,12,14-16,20,23,48), (1)==(17), (17)==(1), (18)-->(11,12,14-16,23), (20)==(48), (48)==(20) + │ ├── columns: p_partkey:1!null p_mfgr:3!null s_name:12!null s_address:13!null s_phone:15!null s_acctbal:16!null s_comment:17!null ps_partkey:19!null ps_suppkey:20!null ps_supplycost:22!null n_name:26!null min:57!null + │ ├── key: (19,20) + │ ├── fd: (1)-->(3), (19,20)-->(1,3,12,13,15-17,22,26,57), (1)==(19), (19)==(1), (20)-->(12,13,15-17,26), (22)==(57), (57)==(22) │ ├── group-by - │ │ ├── columns: p_partkey:1!null p_mfgr:3!null s_name:11!null s_address:12!null s_phone:14!null s_acctbal:15!null s_comment:16!null ps_partkey:17!null ps_suppkey:18!null ps_supplycost:20!null n_name:23!null min:48!null - │ │ ├── grouping columns: ps_partkey:17!null ps_suppkey:18!null - │ │ ├── internal-ordering: +(1|17|29),+(10|18) opt(6,27,46) - │ │ ├── key: (17,18) - │ │ ├── fd: (1)-->(3), (17,18)-->(1,3,11,12,14-16,20,23,48), (1)==(17), (17)==(1), (18)-->(11,12,14-16,23) + │ │ ├── columns: p_partkey:1!null p_mfgr:3!null s_name:12!null s_address:13!null s_phone:15!null s_acctbal:16!null s_comment:17!null ps_partkey:19!null ps_suppkey:20!null ps_supplycost:22!null n_name:26!null min:57!null + │ │ ├── grouping columns: ps_partkey:19!null ps_suppkey:20!null + │ │ ├── internal-ordering: +(1|19|34),+(11|20) opt(6,31,54) + │ │ ├── key: (19,20) + │ │ ├── fd: (1)-->(3), (19,20)-->(1,3,12,13,15-17,22,26,57), (1)==(19), (19)==(1), (20)-->(12,13,15-17,26) │ │ ├── inner-join (lookup region) - │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_suppkey:10!null s_name:11!null s_address:12!null s_nationkey:13!null s_phone:14!null s_acctbal:15!null s_comment:16!null ps_partkey:17!null ps_suppkey:18!null ps_supplycost:20!null n_nationkey:22!null n_name:23!null n_regionkey:24!null r_regionkey:26!null r_name:27!null ps_partkey:29!null ps_suppkey:30!null ps_supplycost:32!null s_suppkey:34!null s_nationkey:37!null n_nationkey:41!null n_regionkey:43!null r_regionkey:45!null r_name:46!null - │ │ │ ├── key columns: [43] = [45] + │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_suppkey:11!null s_name:12!null s_address:13!null s_nationkey:14!null s_phone:15!null s_acctbal:16!null s_comment:17!null ps_partkey:19!null ps_suppkey:20!null ps_supplycost:22!null n_nationkey:25!null n_name:26!null n_regionkey:27!null r_regionkey:30!null r_name:31!null ps_partkey:34!null ps_suppkey:35!null ps_supplycost:37!null s_suppkey:40!null s_nationkey:43!null n_nationkey:48!null n_regionkey:50!null r_regionkey:53!null r_name:54!null + │ │ │ ├── key columns: [50] = [53] │ │ │ ├── lookup columns are key - │ │ │ ├── key: (18,29,34) - │ │ │ ├── fd: ()-->(6,27,46), (1)-->(3,5), (10)-->(11-16), (17,18)-->(20), (10)==(18), (18)==(10), (22)-->(23,24), (24)==(26), (26)==(24), (13)==(22), (22)==(13), (1)==(17,29), (17)==(1,29), (29,30)-->(32), (34)-->(37), (41)-->(43), (43)==(45), (45)==(43), (37)==(41), (41)==(37), (30)==(34), (34)==(30), (29)==(1,17) - │ │ │ ├── ordering: +(1|17|29),+(10|18) opt(6,27,46) [actual: +1,+18] + │ │ │ ├── key: (20,34,40) + │ │ │ ├── fd: ()-->(6,31,54), (1)-->(3,5), (11)-->(12-17), (19,20)-->(22), (11)==(20), (20)==(11), (25)-->(26,27), (27)==(30), (30)==(27), (14)==(25), (25)==(14), (1)==(19,34), (19)==(1,34), (34,35)-->(37), (40)-->(43), (48)-->(50), (50)==(53), (53)==(50), (43)==(48), (48)==(43), (35)==(40), (40)==(35), (34)==(1,19) + │ │ │ ├── ordering: +(1|19|34),+(11|20) opt(6,31,54) [actual: +1,+20] │ │ │ ├── inner-join (lookup nation) - │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_suppkey:10!null s_name:11!null s_address:12!null s_nationkey:13!null s_phone:14!null s_acctbal:15!null s_comment:16!null ps_partkey:17!null ps_suppkey:18!null ps_supplycost:20!null n_nationkey:22!null n_name:23!null n_regionkey:24!null r_regionkey:26!null r_name:27!null ps_partkey:29!null ps_suppkey:30!null ps_supplycost:32!null s_suppkey:34!null s_nationkey:37!null n_nationkey:41!null n_regionkey:43!null - │ │ │ │ ├── key columns: [37] = [41] + │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_suppkey:11!null s_name:12!null s_address:13!null s_nationkey:14!null s_phone:15!null s_acctbal:16!null s_comment:17!null ps_partkey:19!null ps_suppkey:20!null ps_supplycost:22!null n_nationkey:25!null n_name:26!null n_regionkey:27!null r_regionkey:30!null r_name:31!null ps_partkey:34!null ps_suppkey:35!null ps_supplycost:37!null s_suppkey:40!null s_nationkey:43!null n_nationkey:48!null n_regionkey:50!null + │ │ │ │ ├── key columns: [43] = [48] │ │ │ │ ├── lookup columns are key - │ │ │ │ ├── key: (18,29,34) - │ │ │ │ ├── fd: ()-->(6,27), (1)-->(3,5), (22)-->(23,24), (10)-->(11-16), (17,18)-->(20), (29,30)-->(32), (34)-->(37), (41)-->(43), (37)==(41), (41)==(37), (30)==(34), (34)==(30), (17)==(1,29), (29)==(1,17), (10)==(18), (18)==(10), (13)==(22), (22)==(13), (24)==(26), (26)==(24), (1)==(17,29) - │ │ │ │ ├── ordering: +(1|17|29),+(10|18) opt(6,27) [actual: +1,+18] + │ │ │ │ ├── key: (20,34,40) + │ │ │ │ ├── fd: ()-->(6,31), (1)-->(3,5), (25)-->(26,27), (11)-->(12-17), (19,20)-->(22), (34,35)-->(37), (40)-->(43), (48)-->(50), (43)==(48), (48)==(43), (35)==(40), (40)==(35), (19)==(1,34), (34)==(1,19), (11)==(20), (20)==(11), (14)==(25), (25)==(14), (27)==(30), (30)==(27), (1)==(19,34) + │ │ │ │ ├── ordering: +(1|19|34),+(11|20) opt(6,31) [actual: +1,+20] │ │ │ │ ├── inner-join (lookup supplier) - │ │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_suppkey:10!null s_name:11!null s_address:12!null s_nationkey:13!null s_phone:14!null s_acctbal:15!null s_comment:16!null ps_partkey:17!null ps_suppkey:18!null ps_supplycost:20!null n_nationkey:22!null n_name:23!null n_regionkey:24!null r_regionkey:26!null r_name:27!null ps_partkey:29!null ps_suppkey:30!null ps_supplycost:32!null s_suppkey:34!null s_nationkey:37!null - │ │ │ │ │ ├── key columns: [30] = [34] + │ │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_suppkey:11!null s_name:12!null s_address:13!null s_nationkey:14!null s_phone:15!null s_acctbal:16!null s_comment:17!null ps_partkey:19!null ps_suppkey:20!null ps_supplycost:22!null n_nationkey:25!null n_name:26!null n_regionkey:27!null r_regionkey:30!null r_name:31!null ps_partkey:34!null ps_suppkey:35!null ps_supplycost:37!null s_suppkey:40!null s_nationkey:43!null + │ │ │ │ │ ├── key columns: [35] = [40] │ │ │ │ │ ├── lookup columns are key - │ │ │ │ │ ├── key: (18,29,34) - │ │ │ │ │ ├── fd: ()-->(6,27), (1)-->(3,5), (22)-->(23,24), (10)-->(11-16), (17,18)-->(20), (29,30)-->(32), (34)-->(37), (30)==(34), (34)==(30), (17)==(1,29), (29)==(1,17), (10)==(18), (18)==(10), (13)==(22), (22)==(13), (24)==(26), (26)==(24), (1)==(17,29) - │ │ │ │ │ ├── ordering: +(1|17|29),+(10|18) opt(6,27) [actual: +1,+18] + │ │ │ │ │ ├── key: (20,34,40) + │ │ │ │ │ ├── fd: ()-->(6,31), (1)-->(3,5), (25)-->(26,27), (11)-->(12-17), (19,20)-->(22), (34,35)-->(37), (40)-->(43), (35)==(40), (40)==(35), (19)==(1,34), (34)==(1,19), (11)==(20), (20)==(11), (14)==(25), (25)==(14), (27)==(30), (30)==(27), (1)==(19,34) + │ │ │ │ │ ├── ordering: +(1|19|34),+(11|20) opt(6,31) [actual: +1,+20] │ │ │ │ │ ├── inner-join (lookup partsupp) - │ │ │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_suppkey:10!null s_name:11!null s_address:12!null s_nationkey:13!null s_phone:14!null s_acctbal:15!null s_comment:16!null ps_partkey:17!null ps_suppkey:18!null ps_supplycost:20!null n_nationkey:22!null n_name:23!null n_regionkey:24!null r_regionkey:26!null r_name:27!null ps_partkey:29!null ps_suppkey:30!null ps_supplycost:32!null - │ │ │ │ │ │ ├── key columns: [1] = [29] - │ │ │ │ │ │ ├── key: (18,29,30) - │ │ │ │ │ │ ├── fd: ()-->(6,27), (1)-->(3,5), (22)-->(23,24), (10)-->(11-16), (17,18)-->(20), (29,30)-->(32), (17)==(1,29), (29)==(1,17), (10)==(18), (18)==(10), (13)==(22), (22)==(13), (24)==(26), (26)==(24), (1)==(17,29) - │ │ │ │ │ │ ├── ordering: +(1|17|29),+(10|18) opt(6,27) [actual: +1,+18] + │ │ │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_suppkey:11!null s_name:12!null s_address:13!null s_nationkey:14!null s_phone:15!null s_acctbal:16!null s_comment:17!null ps_partkey:19!null ps_suppkey:20!null ps_supplycost:22!null n_nationkey:25!null n_name:26!null n_regionkey:27!null r_regionkey:30!null r_name:31!null ps_partkey:34!null ps_suppkey:35!null ps_supplycost:37!null + │ │ │ │ │ │ ├── key columns: [1] = [34] + │ │ │ │ │ │ ├── key: (20,34,35) + │ │ │ │ │ │ ├── fd: ()-->(6,31), (1)-->(3,5), (25)-->(26,27), (11)-->(12-17), (19,20)-->(22), (34,35)-->(37), (19)==(1,34), (34)==(1,19), (11)==(20), (20)==(11), (14)==(25), (25)==(14), (27)==(30), (30)==(27), (1)==(19,34) + │ │ │ │ │ │ ├── ordering: +(1|19|34),+(11|20) opt(6,31) [actual: +1,+20] │ │ │ │ │ │ ├── inner-join (lookup region) - │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_suppkey:10!null s_name:11!null s_address:12!null s_nationkey:13!null s_phone:14!null s_acctbal:15!null s_comment:16!null ps_partkey:17!null ps_suppkey:18!null ps_supplycost:20!null n_nationkey:22!null n_name:23!null n_regionkey:24!null r_regionkey:26!null r_name:27!null - │ │ │ │ │ │ │ ├── key columns: [24] = [26] + │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_suppkey:11!null s_name:12!null s_address:13!null s_nationkey:14!null s_phone:15!null s_acctbal:16!null s_comment:17!null ps_partkey:19!null ps_suppkey:20!null ps_supplycost:22!null n_nationkey:25!null n_name:26!null n_regionkey:27!null r_regionkey:30!null r_name:31!null + │ │ │ │ │ │ │ ├── key columns: [27] = [30] │ │ │ │ │ │ │ ├── lookup columns are key - │ │ │ │ │ │ │ ├── key: (17,18) - │ │ │ │ │ │ │ ├── fd: ()-->(6,27), (1)-->(3,5), (10)-->(11-16), (17,18)-->(20), (10)==(18), (18)==(10), (22)-->(23,24), (24)==(26), (26)==(24), (13)==(22), (22)==(13), (1)==(17), (17)==(1) - │ │ │ │ │ │ │ ├── ordering: +(1|17),+(10|18) opt(6,27) [actual: +1,+18] + │ │ │ │ │ │ │ ├── key: (19,20) + │ │ │ │ │ │ │ ├── fd: ()-->(6,31), (1)-->(3,5), (11)-->(12-17), (19,20)-->(22), (11)==(20), (20)==(11), (25)-->(26,27), (27)==(30), (30)==(27), (14)==(25), (25)==(14), (1)==(19), (19)==(1) + │ │ │ │ │ │ │ ├── ordering: +(1|19),+(11|20) opt(6,31) [actual: +1,+20] │ │ │ │ │ │ │ ├── inner-join (lookup nation) - │ │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_suppkey:10!null s_name:11!null s_address:12!null s_nationkey:13!null s_phone:14!null s_acctbal:15!null s_comment:16!null ps_partkey:17!null ps_suppkey:18!null ps_supplycost:20!null n_nationkey:22!null n_name:23!null n_regionkey:24!null - │ │ │ │ │ │ │ │ ├── key columns: [13] = [22] + │ │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_suppkey:11!null s_name:12!null s_address:13!null s_nationkey:14!null s_phone:15!null s_acctbal:16!null s_comment:17!null ps_partkey:19!null ps_suppkey:20!null ps_supplycost:22!null n_nationkey:25!null n_name:26!null n_regionkey:27!null + │ │ │ │ │ │ │ │ ├── key columns: [14] = [25] │ │ │ │ │ │ │ │ ├── lookup columns are key - │ │ │ │ │ │ │ │ ├── key: (17,18) - │ │ │ │ │ │ │ │ ├── fd: ()-->(6), (1)-->(3,5), (17,18)-->(20), (10)-->(11-16), (22)-->(23,24), (13)==(22), (22)==(13), (10)==(18), (18)==(10), (1)==(17), (17)==(1) - │ │ │ │ │ │ │ │ ├── ordering: +(1|17),+(10|18) opt(6) [actual: +1,+18] + │ │ │ │ │ │ │ │ ├── key: (19,20) + │ │ │ │ │ │ │ │ ├── fd: ()-->(6), (1)-->(3,5), (19,20)-->(22), (11)-->(12-17), (25)-->(26,27), (14)==(25), (25)==(14), (11)==(20), (20)==(11), (1)==(19), (19)==(1) + │ │ │ │ │ │ │ │ ├── ordering: +(1|19),+(11|20) opt(6) [actual: +1,+20] │ │ │ │ │ │ │ │ ├── inner-join (lookup supplier) - │ │ │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_suppkey:10!null s_name:11!null s_address:12!null s_nationkey:13!null s_phone:14!null s_acctbal:15!null s_comment:16!null ps_partkey:17!null ps_suppkey:18!null ps_supplycost:20!null - │ │ │ │ │ │ │ │ │ ├── key columns: [18] = [10] + │ │ │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null s_suppkey:11!null s_name:12!null s_address:13!null s_nationkey:14!null s_phone:15!null s_acctbal:16!null s_comment:17!null ps_partkey:19!null ps_suppkey:20!null ps_supplycost:22!null + │ │ │ │ │ │ │ │ │ ├── key columns: [20] = [11] │ │ │ │ │ │ │ │ │ ├── lookup columns are key - │ │ │ │ │ │ │ │ │ ├── key: (17,18) - │ │ │ │ │ │ │ │ │ ├── fd: ()-->(6), (1)-->(3,5), (10)-->(11-16), (17,18)-->(20), (10)==(18), (18)==(10), (1)==(17), (17)==(1) - │ │ │ │ │ │ │ │ │ ├── ordering: +(1|17),+(10|18) opt(6) [actual: +1,+18] + │ │ │ │ │ │ │ │ │ ├── key: (19,20) + │ │ │ │ │ │ │ │ │ ├── fd: ()-->(6), (1)-->(3,5), (11)-->(12-17), (19,20)-->(22), (11)==(20), (20)==(11), (1)==(19), (19)==(1) + │ │ │ │ │ │ │ │ │ ├── ordering: +(1|19),+(11|20) opt(6) [actual: +1,+20] │ │ │ │ │ │ │ │ │ ├── sort - │ │ │ │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null ps_partkey:17!null ps_suppkey:18!null ps_supplycost:20!null - │ │ │ │ │ │ │ │ │ │ ├── key: (17,18) - │ │ │ │ │ │ │ │ │ │ ├── fd: ()-->(6), (1)-->(3,5), (17,18)-->(20), (1)==(17), (17)==(1) - │ │ │ │ │ │ │ │ │ │ ├── ordering: +(1|17),+18 opt(6) [actual: +1,+18] + │ │ │ │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null ps_partkey:19!null ps_suppkey:20!null ps_supplycost:22!null + │ │ │ │ │ │ │ │ │ │ ├── key: (19,20) + │ │ │ │ │ │ │ │ │ │ ├── fd: ()-->(6), (1)-->(3,5), (19,20)-->(22), (1)==(19), (19)==(1) + │ │ │ │ │ │ │ │ │ │ ├── ordering: +(1|19),+20 opt(6) [actual: +1,+20] │ │ │ │ │ │ │ │ │ │ └── inner-join (lookup partsupp) - │ │ │ │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null ps_partkey:17!null ps_suppkey:18!null ps_supplycost:20!null - │ │ │ │ │ │ │ │ │ │ ├── key columns: [1] = [17] - │ │ │ │ │ │ │ │ │ │ ├── key: (17,18) - │ │ │ │ │ │ │ │ │ │ ├── fd: ()-->(6), (1)-->(3,5), (17,18)-->(20), (1)==(17), (17)==(1) + │ │ │ │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null ps_partkey:19!null ps_suppkey:20!null ps_supplycost:22!null + │ │ │ │ │ │ │ │ │ │ ├── key columns: [1] = [19] + │ │ │ │ │ │ │ │ │ │ ├── key: (19,20) + │ │ │ │ │ │ │ │ │ │ ├── fd: ()-->(6), (1)-->(3,5), (19,20)-->(22), (1)==(19), (19)==(1) │ │ │ │ │ │ │ │ │ │ ├── select │ │ │ │ │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_mfgr:3!null p_type:5!null p_size:6!null │ │ │ │ │ │ │ │ │ │ │ ├── key: (1) @@ -242,35 +242,35 @@ project │ │ │ │ │ │ │ │ │ └── filters (true) │ │ │ │ │ │ │ │ └── filters (true) │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── r_name:27 = 'EUROPE' [outer=(27), constraints=(/27: [/'EUROPE' - /'EUROPE']; tight), fd=()-->(27)] + │ │ │ │ │ │ │ └── r_name:31 = 'EUROPE' [outer=(31), constraints=(/31: [/'EUROPE' - /'EUROPE']; tight), fd=()-->(31)] │ │ │ │ │ │ └── filters (true) │ │ │ │ │ └── filters (true) │ │ │ │ └── filters (true) │ │ │ └── filters - │ │ │ └── r_name:46 = 'EUROPE' [outer=(46), constraints=(/46: [/'EUROPE' - /'EUROPE']; tight), fd=()-->(46)] + │ │ │ └── r_name:54 = 'EUROPE' [outer=(54), constraints=(/54: [/'EUROPE' - /'EUROPE']; tight), fd=()-->(54)] │ │ └── aggregations - │ │ ├── min [as=min:48, outer=(32)] - │ │ │ └── ps_supplycost:32 - │ │ ├── const-agg [as=s_name:11, outer=(11)] - │ │ │ └── s_name:11 - │ │ ├── const-agg [as=s_address:12, outer=(12)] - │ │ │ └── s_address:12 - │ │ ├── const-agg [as=s_phone:14, outer=(14)] - │ │ │ └── s_phone:14 - │ │ ├── const-agg [as=s_acctbal:15, outer=(15)] - │ │ │ └── s_acctbal:15 - │ │ ├── const-agg [as=s_comment:16, outer=(16)] - │ │ │ └── s_comment:16 - │ │ ├── const-agg [as=ps_supplycost:20, outer=(20)] - │ │ │ └── ps_supplycost:20 - │ │ ├── const-agg [as=n_name:23, outer=(23)] - │ │ │ └── n_name:23 + │ │ ├── min [as=min:57, outer=(37)] + │ │ │ └── ps_supplycost:37 + │ │ ├── const-agg [as=s_name:12, outer=(12)] + │ │ │ └── s_name:12 + │ │ ├── const-agg [as=s_address:13, outer=(13)] + │ │ │ └── s_address:13 + │ │ ├── const-agg [as=s_phone:15, outer=(15)] + │ │ │ └── s_phone:15 + │ │ ├── const-agg [as=s_acctbal:16, outer=(16)] + │ │ │ └── s_acctbal:16 + │ │ ├── const-agg [as=s_comment:17, outer=(17)] + │ │ │ └── s_comment:17 + │ │ ├── const-agg [as=ps_supplycost:22, outer=(22)] + │ │ │ └── ps_supplycost:22 + │ │ ├── const-agg [as=n_name:26, outer=(26)] + │ │ │ └── n_name:26 │ │ ├── const-agg [as=p_mfgr:3, outer=(3)] │ │ │ └── p_mfgr:3 │ │ └── const-agg [as=p_partkey:1, outer=(1)] │ │ └── p_partkey:1 │ └── filters - │ └── ps_supplycost:20 = min:48 [outer=(20,48), constraints=(/20: (/NULL - ]; /48: (/NULL - ]), fd=(20)==(48), (48)==(20)] + │ └── ps_supplycost:22 = min:57 [outer=(22,57), constraints=(/22: (/NULL - ]; /57: (/NULL - ]), fd=(22)==(57), (57)==(22)] └── 100 # -------------------------------------------------- @@ -310,56 +310,56 @@ ORDER BY LIMIT 10; ---- limit - ├── columns: l_orderkey:18!null revenue:35!null o_orderdate:13!null o_shippriority:16!null - ├── internal-ordering: -35,+13 + ├── columns: l_orderkey:20!null revenue:38!null o_orderdate:14!null o_shippriority:17!null + ├── internal-ordering: -38,+14 ├── cardinality: [0 - 10] ├── immutable - ├── key: (18) - ├── fd: (18)-->(13,16,35) - ├── ordering: -35,+13 + ├── key: (20) + ├── fd: (20)-->(14,17,38) + ├── ordering: -38,+14 ├── sort - │ ├── columns: o_orderdate:13!null o_shippriority:16!null l_orderkey:18!null sum:35!null + │ ├── columns: o_orderdate:14!null o_shippriority:17!null l_orderkey:20!null sum:38!null │ ├── immutable - │ ├── key: (18) - │ ├── fd: (18)-->(13,16,35) - │ ├── ordering: -35,+13 + │ ├── key: (20) + │ ├── fd: (20)-->(14,17,38) + │ ├── ordering: -38,+14 │ ├── limit hint: 10.00 │ └── group-by - │ ├── columns: o_orderdate:13!null o_shippriority:16!null l_orderkey:18!null sum:35!null - │ ├── grouping columns: l_orderkey:18!null + │ ├── columns: o_orderdate:14!null o_shippriority:17!null l_orderkey:20!null sum:38!null + │ ├── grouping columns: l_orderkey:20!null │ ├── immutable - │ ├── key: (18) - │ ├── fd: (18)-->(13,16,35) + │ ├── key: (20) + │ ├── fd: (20)-->(14,17,38) │ ├── project - │ │ ├── columns: column34:34!null o_orderdate:13!null o_shippriority:16!null l_orderkey:18!null + │ │ ├── columns: column37:37!null o_orderdate:14!null o_shippriority:17!null l_orderkey:20!null │ │ ├── immutable - │ │ ├── fd: (18)-->(13,16) + │ │ ├── fd: (20)-->(14,17) │ │ ├── inner-join (lookup lineitem) - │ │ │ ├── columns: c_custkey:1!null c_mktsegment:7!null o_orderkey:9!null o_custkey:10!null o_orderdate:13!null o_shippriority:16!null l_orderkey:18!null l_extendedprice:23!null l_discount:24!null l_shipdate:28!null - │ │ │ ├── key columns: [18 21] = [18 21] + │ │ │ ├── columns: c_custkey:1!null c_mktsegment:7!null o_orderkey:10!null o_custkey:11!null o_orderdate:14!null o_shippriority:17!null l_orderkey:20!null l_extendedprice:25!null l_discount:26!null l_shipdate:30!null + │ │ │ ├── key columns: [20 23] = [20 23] │ │ │ ├── lookup columns are key - │ │ │ ├── fd: ()-->(7), (9)-->(10,13,16), (9)==(18), (18)==(9), (1)==(10), (10)==(1) + │ │ │ ├── fd: ()-->(7), (10)-->(11,14,17), (10)==(20), (20)==(10), (1)==(11), (11)==(1) │ │ │ ├── inner-join (hash) - │ │ │ │ ├── columns: c_custkey:1!null c_mktsegment:7!null o_orderkey:9!null o_custkey:10!null o_orderdate:13!null o_shippriority:16!null l_orderkey:18!null l_linenumber:21!null l_shipdate:28!null + │ │ │ │ ├── columns: c_custkey:1!null c_mktsegment:7!null o_orderkey:10!null o_custkey:11!null o_orderdate:14!null o_shippriority:17!null l_orderkey:20!null l_linenumber:23!null l_shipdate:30!null │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ │ ├── key: (18,21) - │ │ │ │ ├── fd: ()-->(7), (18,21)-->(28), (9)-->(10,13,16), (1)==(10), (10)==(1), (9)==(18), (18)==(9) + │ │ │ │ ├── key: (20,23) + │ │ │ │ ├── fd: ()-->(7), (20,23)-->(30), (10)-->(11,14,17), (1)==(11), (11)==(1), (10)==(20), (20)==(10) │ │ │ │ ├── scan lineitem@l_sd - │ │ │ │ │ ├── columns: l_orderkey:18!null l_linenumber:21!null l_shipdate:28!null - │ │ │ │ │ ├── constraint: /28/18/21: [/'1995-03-16' - ] - │ │ │ │ │ ├── key: (18,21) - │ │ │ │ │ └── fd: (18,21)-->(28) + │ │ │ │ │ ├── columns: l_orderkey:20!null l_linenumber:23!null l_shipdate:30!null + │ │ │ │ │ ├── constraint: /30/20/23: [/'1995-03-16' - ] + │ │ │ │ │ ├── key: (20,23) + │ │ │ │ │ └── fd: (20,23)-->(30) │ │ │ │ ├── inner-join (lookup orders) - │ │ │ │ │ ├── columns: c_custkey:1!null c_mktsegment:7!null o_orderkey:9!null o_custkey:10!null o_orderdate:13!null o_shippriority:16!null - │ │ │ │ │ ├── key columns: [9] = [9] + │ │ │ │ │ ├── columns: c_custkey:1!null c_mktsegment:7!null o_orderkey:10!null o_custkey:11!null o_orderdate:14!null o_shippriority:17!null + │ │ │ │ │ ├── key columns: [10] = [10] │ │ │ │ │ ├── lookup columns are key - │ │ │ │ │ ├── key: (9) - │ │ │ │ │ ├── fd: ()-->(7), (9)-->(10,13,16), (1)==(10), (10)==(1) + │ │ │ │ │ ├── key: (10) + │ │ │ │ │ ├── fd: ()-->(7), (10)-->(11,14,17), (1)==(11), (11)==(1) │ │ │ │ │ ├── inner-join (lookup orders@o_ck) - │ │ │ │ │ │ ├── columns: c_custkey:1!null c_mktsegment:7!null o_orderkey:9!null o_custkey:10!null - │ │ │ │ │ │ ├── key columns: [1] = [10] - │ │ │ │ │ │ ├── key: (9) - │ │ │ │ │ │ ├── fd: ()-->(7), (9)-->(10), (1)==(10), (10)==(1) + │ │ │ │ │ │ ├── columns: c_custkey:1!null c_mktsegment:7!null o_orderkey:10!null o_custkey:11!null + │ │ │ │ │ │ ├── key columns: [1] = [11] + │ │ │ │ │ │ ├── key: (10) + │ │ │ │ │ │ ├── fd: ()-->(7), (10)-->(11), (1)==(11), (11)==(1) │ │ │ │ │ │ ├── select │ │ │ │ │ │ │ ├── columns: c_custkey:1!null c_mktsegment:7!null │ │ │ │ │ │ │ ├── key: (1) @@ -372,19 +372,19 @@ limit │ │ │ │ │ │ │ └── c_mktsegment:7 = 'BUILDING' [outer=(7), constraints=(/7: [/'BUILDING' - /'BUILDING']; tight), fd=()-->(7)] │ │ │ │ │ │ └── filters (true) │ │ │ │ │ └── filters - │ │ │ │ │ └── o_orderdate:13 < '1995-03-15' [outer=(13), constraints=(/13: (/NULL - /'1995-03-14']; tight)] + │ │ │ │ │ └── o_orderdate:14 < '1995-03-15' [outer=(14), constraints=(/14: (/NULL - /'1995-03-14']; tight)] │ │ │ │ └── filters - │ │ │ │ └── l_orderkey:18 = o_orderkey:9 [outer=(9,18), constraints=(/9: (/NULL - ]; /18: (/NULL - ]), fd=(9)==(18), (18)==(9)] + │ │ │ │ └── l_orderkey:20 = o_orderkey:10 [outer=(10,20), constraints=(/10: (/NULL - ]; /20: (/NULL - ]), fd=(10)==(20), (20)==(10)] │ │ │ └── filters (true) │ │ └── projections - │ │ └── l_extendedprice:23 * (1.0 - l_discount:24) [as=column34:34, outer=(23,24), immutable] + │ │ └── l_extendedprice:25 * (1.0 - l_discount:26) [as=column37:37, outer=(25,26), immutable] │ └── aggregations - │ ├── sum [as=sum:35, outer=(34)] - │ │ └── column34:34 - │ ├── const-agg [as=o_orderdate:13, outer=(13)] - │ │ └── o_orderdate:13 - │ └── const-agg [as=o_shippriority:16, outer=(16)] - │ └── o_shippriority:16 + │ ├── sum [as=sum:38, outer=(37)] + │ │ └── column37:37 + │ ├── const-agg [as=o_orderdate:14, outer=(14)] + │ │ └── o_orderdate:14 + │ └── const-agg [as=o_shippriority:17, outer=(17)] + │ └── o_shippriority:17 └── 10 # -------------------------------------------------- @@ -422,42 +422,42 @@ ORDER BY o_orderpriority; ---- sort - ├── columns: o_orderpriority:6!null order_count:26!null + ├── columns: o_orderpriority:6!null order_count:28!null ├── key: (6) - ├── fd: (6)-->(26) + ├── fd: (6)-->(28) ├── ordering: +6 └── group-by - ├── columns: o_orderpriority:6!null count_rows:26!null + ├── columns: o_orderpriority:6!null count_rows:28!null ├── grouping columns: o_orderpriority:6!null ├── key: (6) - ├── fd: (6)-->(26) + ├── fd: (6)-->(28) ├── project │ ├── columns: o_orderkey:1!null o_orderdate:5!null o_orderpriority:6!null │ ├── key: (1) │ ├── fd: (1)-->(5,6) │ └── inner-join (lookup orders) - │ ├── columns: o_orderkey:1!null o_orderdate:5!null o_orderpriority:6!null l_orderkey:10!null - │ ├── key columns: [10] = [1] + │ ├── columns: o_orderkey:1!null o_orderdate:5!null o_orderpriority:6!null l_orderkey:11!null + │ ├── key columns: [11] = [1] │ ├── lookup columns are key - │ ├── key: (10) - │ ├── fd: (1)-->(5,6), (1)==(10), (10)==(1) + │ ├── key: (11) + │ ├── fd: (1)-->(5,6), (1)==(11), (11)==(1) │ ├── distinct-on - │ │ ├── columns: l_orderkey:10!null - │ │ ├── grouping columns: l_orderkey:10!null - │ │ ├── internal-ordering: +10 - │ │ ├── key: (10) + │ │ ├── columns: l_orderkey:11!null + │ │ ├── grouping columns: l_orderkey:11!null + │ │ ├── internal-ordering: +11 + │ │ ├── key: (11) │ │ └── select - │ │ ├── columns: l_orderkey:10!null l_commitdate:21!null l_receiptdate:22!null - │ │ ├── ordering: +10 + │ │ ├── columns: l_orderkey:11!null l_commitdate:22!null l_receiptdate:23!null + │ │ ├── ordering: +11 │ │ ├── scan lineitem - │ │ │ ├── columns: l_orderkey:10!null l_commitdate:21!null l_receiptdate:22!null - │ │ │ └── ordering: +10 + │ │ │ ├── columns: l_orderkey:11!null l_commitdate:22!null l_receiptdate:23!null + │ │ │ └── ordering: +11 │ │ └── filters - │ │ └── l_commitdate:21 < l_receiptdate:22 [outer=(21,22), constraints=(/21: (/NULL - ]; /22: (/NULL - ])] + │ │ └── l_commitdate:22 < l_receiptdate:23 [outer=(22,23), constraints=(/22: (/NULL - ]; /23: (/NULL - ])] │ └── filters │ └── (o_orderdate:5 >= '1993-07-01') AND (o_orderdate:5 < '1993-10-01') [outer=(5), constraints=(/5: [/'1993-07-01' - /'1993-09-30']; tight)] └── aggregations - └── count-rows [as=count_rows:26] + └── count-rows [as=count_rows:28] # -------------------------------------------------- # Q5 @@ -500,80 +500,80 @@ ORDER BY revenue DESC; ---- sort - ├── columns: n_name:42!null revenue:49!null + ├── columns: n_name:46!null revenue:55!null ├── immutable - ├── key: (42) - ├── fd: (42)-->(49) - ├── ordering: -49 + ├── key: (46) + ├── fd: (46)-->(55) + ├── ordering: -55 └── group-by - ├── columns: n_name:42!null sum:49!null - ├── grouping columns: n_name:42!null + ├── columns: n_name:46!null sum:55!null + ├── grouping columns: n_name:46!null ├── immutable - ├── key: (42) - ├── fd: (42)-->(49) + ├── key: (46) + ├── fd: (46)-->(55) ├── project - │ ├── columns: column48:48!null n_name:42!null + │ ├── columns: column54:54!null n_name:46!null │ ├── immutable │ ├── inner-join (lookup region) - │ │ ├── columns: c_custkey:1!null c_nationkey:4!null o_orderkey:9!null o_custkey:10!null o_orderdate:13!null l_orderkey:18!null l_suppkey:20!null l_extendedprice:23!null l_discount:24!null s_suppkey:34!null s_nationkey:37!null n_nationkey:41!null n_name:42!null n_regionkey:43!null r_regionkey:45!null r_name:46!null - │ │ ├── key columns: [43] = [45] + │ │ ├── columns: c_custkey:1!null c_nationkey:4!null o_orderkey:10!null o_custkey:11!null o_orderdate:14!null l_orderkey:20!null l_suppkey:22!null l_extendedprice:25!null l_discount:26!null s_suppkey:37!null s_nationkey:40!null n_nationkey:45!null n_name:46!null n_regionkey:47!null r_regionkey:50!null r_name:51!null + │ │ ├── key columns: [47] = [50] │ │ ├── lookup columns are key - │ │ ├── fd: ()-->(46), (1)-->(4), (9)-->(10,13), (34)-->(37), (41)-->(42,43), (43)==(45), (45)==(43), (37)==(4,41), (41)==(4,37), (20)==(34), (34)==(20), (9)==(18), (18)==(9), (1)==(10), (10)==(1), (4)==(37,41) + │ │ ├── fd: ()-->(51), (1)-->(4), (10)-->(11,14), (37)-->(40), (45)-->(46,47), (47)==(50), (50)==(47), (40)==(4,45), (45)==(4,40), (22)==(37), (37)==(22), (10)==(20), (20)==(10), (1)==(11), (11)==(1), (4)==(40,45) │ │ ├── inner-join (lookup nation) - │ │ │ ├── columns: c_custkey:1!null c_nationkey:4!null o_orderkey:9!null o_custkey:10!null o_orderdate:13!null l_orderkey:18!null l_suppkey:20!null l_extendedprice:23!null l_discount:24!null s_suppkey:34!null s_nationkey:37!null n_nationkey:41!null n_name:42!null n_regionkey:43!null - │ │ │ ├── key columns: [37] = [41] + │ │ │ ├── columns: c_custkey:1!null c_nationkey:4!null o_orderkey:10!null o_custkey:11!null o_orderdate:14!null l_orderkey:20!null l_suppkey:22!null l_extendedprice:25!null l_discount:26!null s_suppkey:37!null s_nationkey:40!null n_nationkey:45!null n_name:46!null n_regionkey:47!null + │ │ │ ├── key columns: [40] = [45] │ │ │ ├── lookup columns are key - │ │ │ ├── fd: (1)-->(4), (9)-->(10,13), (34)-->(37), (41)-->(42,43), (37)==(4,41), (41)==(4,37), (20)==(34), (34)==(20), (9)==(18), (18)==(9), (1)==(10), (10)==(1), (4)==(37,41) + │ │ │ ├── fd: (1)-->(4), (10)-->(11,14), (37)-->(40), (45)-->(46,47), (40)==(4,45), (45)==(4,40), (22)==(37), (37)==(22), (10)==(20), (20)==(10), (1)==(11), (11)==(1), (4)==(40,45) │ │ │ ├── inner-join (hash) - │ │ │ │ ├── columns: c_custkey:1!null c_nationkey:4!null o_orderkey:9!null o_custkey:10!null o_orderdate:13!null l_orderkey:18!null l_suppkey:20!null l_extendedprice:23!null l_discount:24!null s_suppkey:34!null s_nationkey:37!null + │ │ │ │ ├── columns: c_custkey:1!null c_nationkey:4!null o_orderkey:10!null o_custkey:11!null o_orderdate:14!null l_orderkey:20!null l_suppkey:22!null l_extendedprice:25!null l_discount:26!null s_suppkey:37!null s_nationkey:40!null │ │ │ │ ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) - │ │ │ │ ├── fd: (1)-->(4), (9)-->(10,13), (34)-->(37), (20)==(34), (34)==(20), (9)==(18), (18)==(9), (1)==(10), (10)==(1), (4)==(37), (37)==(4) + │ │ │ │ ├── fd: (1)-->(4), (10)-->(11,14), (37)-->(40), (22)==(37), (37)==(22), (10)==(20), (20)==(10), (1)==(11), (11)==(1), (4)==(40), (40)==(4) │ │ │ │ ├── scan customer@c_nk │ │ │ │ │ ├── columns: c_custkey:1!null c_nationkey:4!null │ │ │ │ │ ├── key: (1) │ │ │ │ │ └── fd: (1)-->(4) │ │ │ │ ├── inner-join (hash) - │ │ │ │ │ ├── columns: o_orderkey:9!null o_custkey:10!null o_orderdate:13!null l_orderkey:18!null l_suppkey:20!null l_extendedprice:23!null l_discount:24!null s_suppkey:34!null s_nationkey:37!null + │ │ │ │ │ ├── columns: o_orderkey:10!null o_custkey:11!null o_orderdate:14!null l_orderkey:20!null l_suppkey:22!null l_extendedprice:25!null l_discount:26!null s_suppkey:37!null s_nationkey:40!null │ │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ │ │ │ │ ├── fd: (9)-->(10,13), (34)-->(37), (20)==(34), (34)==(20), (9)==(18), (18)==(9) + │ │ │ │ │ ├── fd: (10)-->(11,14), (37)-->(40), (22)==(37), (37)==(22), (10)==(20), (20)==(10) │ │ │ │ │ ├── inner-join (merge) - │ │ │ │ │ │ ├── columns: o_orderkey:9!null o_custkey:10!null o_orderdate:13!null l_orderkey:18!null l_suppkey:20!null l_extendedprice:23!null l_discount:24!null - │ │ │ │ │ │ ├── left ordering: +9 - │ │ │ │ │ │ ├── right ordering: +18 - │ │ │ │ │ │ ├── fd: (9)-->(10,13), (9)==(18), (18)==(9) + │ │ │ │ │ │ ├── columns: o_orderkey:10!null o_custkey:11!null o_orderdate:14!null l_orderkey:20!null l_suppkey:22!null l_extendedprice:25!null l_discount:26!null + │ │ │ │ │ │ ├── left ordering: +10 + │ │ │ │ │ │ ├── right ordering: +20 + │ │ │ │ │ │ ├── fd: (10)-->(11,14), (10)==(20), (20)==(10) │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: o_orderkey:9!null o_custkey:10!null o_orderdate:13!null - │ │ │ │ │ │ │ ├── key: (9) - │ │ │ │ │ │ │ ├── fd: (9)-->(10,13) - │ │ │ │ │ │ │ ├── ordering: +9 + │ │ │ │ │ │ │ ├── columns: o_orderkey:10!null o_custkey:11!null o_orderdate:14!null + │ │ │ │ │ │ │ ├── key: (10) + │ │ │ │ │ │ │ ├── fd: (10)-->(11,14) + │ │ │ │ │ │ │ ├── ordering: +10 │ │ │ │ │ │ │ ├── scan orders - │ │ │ │ │ │ │ │ ├── columns: o_orderkey:9!null o_custkey:10!null o_orderdate:13!null - │ │ │ │ │ │ │ │ ├── key: (9) - │ │ │ │ │ │ │ │ ├── fd: (9)-->(10,13) - │ │ │ │ │ │ │ │ └── ordering: +9 + │ │ │ │ │ │ │ │ ├── columns: o_orderkey:10!null o_custkey:11!null o_orderdate:14!null + │ │ │ │ │ │ │ │ ├── key: (10) + │ │ │ │ │ │ │ │ ├── fd: (10)-->(11,14) + │ │ │ │ │ │ │ │ └── ordering: +10 │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── (o_orderdate:13 >= '1994-01-01') AND (o_orderdate:13 < '1995-01-01') [outer=(13), constraints=(/13: [/'1994-01-01' - /'1994-12-31']; tight)] + │ │ │ │ │ │ │ └── (o_orderdate:14 >= '1994-01-01') AND (o_orderdate:14 < '1995-01-01') [outer=(14), constraints=(/14: [/'1994-01-01' - /'1994-12-31']; tight)] │ │ │ │ │ │ ├── scan lineitem - │ │ │ │ │ │ │ ├── columns: l_orderkey:18!null l_suppkey:20!null l_extendedprice:23!null l_discount:24!null - │ │ │ │ │ │ │ └── ordering: +18 + │ │ │ │ │ │ │ ├── columns: l_orderkey:20!null l_suppkey:22!null l_extendedprice:25!null l_discount:26!null + │ │ │ │ │ │ │ └── ordering: +20 │ │ │ │ │ │ └── filters (true) │ │ │ │ │ ├── scan supplier@s_nk - │ │ │ │ │ │ ├── columns: s_suppkey:34!null s_nationkey:37!null - │ │ │ │ │ │ ├── key: (34) - │ │ │ │ │ │ └── fd: (34)-->(37) + │ │ │ │ │ │ ├── columns: s_suppkey:37!null s_nationkey:40!null + │ │ │ │ │ │ ├── key: (37) + │ │ │ │ │ │ └── fd: (37)-->(40) │ │ │ │ │ └── filters - │ │ │ │ │ └── l_suppkey:20 = s_suppkey:34 [outer=(20,34), constraints=(/20: (/NULL - ]; /34: (/NULL - ]), fd=(20)==(34), (34)==(20)] + │ │ │ │ │ └── l_suppkey:22 = s_suppkey:37 [outer=(22,37), constraints=(/22: (/NULL - ]; /37: (/NULL - ]), fd=(22)==(37), (37)==(22)] │ │ │ │ └── filters - │ │ │ │ ├── c_custkey:1 = o_custkey:10 [outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] - │ │ │ │ └── c_nationkey:4 = s_nationkey:37 [outer=(4,37), constraints=(/4: (/NULL - ]; /37: (/NULL - ]), fd=(4)==(37), (37)==(4)] + │ │ │ │ ├── c_custkey:1 = o_custkey:11 [outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ]), fd=(1)==(11), (11)==(1)] + │ │ │ │ └── c_nationkey:4 = s_nationkey:40 [outer=(4,40), constraints=(/4: (/NULL - ]; /40: (/NULL - ]), fd=(4)==(40), (40)==(4)] │ │ │ └── filters (true) │ │ └── filters - │ │ └── r_name:46 = 'ASIA' [outer=(46), constraints=(/46: [/'ASIA' - /'ASIA']; tight), fd=()-->(46)] + │ │ └── r_name:51 = 'ASIA' [outer=(51), constraints=(/51: [/'ASIA' - /'ASIA']; tight), fd=()-->(51)] │ └── projections - │ └── l_extendedprice:23 * (1.0 - l_discount:24) [as=column48:48, outer=(23,24), immutable] + │ └── l_extendedprice:25 * (1.0 - l_discount:26) [as=column54:54, outer=(25,26), immutable] └── aggregations - └── sum [as=sum:49, outer=(48)] - └── column48:48 + └── sum [as=sum:55, outer=(54)] + └── column54:54 # -------------------------------------------------- # Q6 @@ -602,13 +602,13 @@ WHERE AND l_quantity < 24; ---- scalar-group-by - ├── columns: revenue:18 + ├── columns: revenue:19 ├── cardinality: [1 - 1] ├── immutable ├── key: () - ├── fd: ()-->(18) + ├── fd: ()-->(19) ├── project - │ ├── columns: column17:17!null + │ ├── columns: column18:18!null │ ├── immutable │ ├── select │ │ ├── columns: l_quantity:5!null l_extendedprice:6!null l_discount:7!null l_shipdate:11!null @@ -619,10 +619,10 @@ scalar-group-by │ │ ├── (l_shipdate:11 >= '1994-01-01') AND (l_shipdate:11 < '1995-01-01') [outer=(11), constraints=(/11: [/'1994-01-01' - /'1994-12-31']; tight)] │ │ └── l_quantity:5 < 24.0 [outer=(5), constraints=(/5: (/NULL - /23.999999999999996]; tight)] │ └── projections - │ └── l_extendedprice:6 * l_discount:7 [as=column17:17, outer=(6,7), immutable] + │ └── l_extendedprice:6 * l_discount:7 [as=column18:18, outer=(6,7), immutable] └── aggregations - └── sum [as=sum:18, outer=(17)] - └── column17:17 + └── sum [as=sum:19, outer=(18)] + └── column18:18 # -------------------------------------------------- # Q7 @@ -677,88 +677,88 @@ ORDER BY l_year; ---- group-by - ├── columns: supp_nation:42!null cust_nation:46!null l_year:49 revenue:51!null - ├── grouping columns: n1.n_name:42!null n2.n_name:46!null l_year:49 + ├── columns: supp_nation:46!null cust_nation:51!null l_year:55 revenue:57!null + ├── grouping columns: n1.n_name:46!null n2.n_name:51!null l_year:55 ├── immutable - ├── key: (42,46,49) - ├── fd: (42,46,49)-->(51) - ├── ordering: +42,+46,+49 + ├── key: (46,51,55) + ├── fd: (46,51,55)-->(57) + ├── ordering: +46,+51,+55 ├── sort - │ ├── columns: n1.n_name:42!null n2.n_name:46!null l_year:49 volume:50!null + │ ├── columns: n1.n_name:46!null n2.n_name:51!null l_year:55 volume:56!null │ ├── immutable - │ ├── ordering: +42,+46,+49 + │ ├── ordering: +46,+51,+55 │ └── project - │ ├── columns: l_year:49 volume:50!null n1.n_name:42!null n2.n_name:46!null + │ ├── columns: l_year:55 volume:56!null n1.n_name:46!null n2.n_name:51!null │ ├── immutable │ ├── inner-join (hash) - │ │ ├── columns: s_suppkey:1!null s_nationkey:4!null l_orderkey:8!null l_suppkey:10!null l_extendedprice:13!null l_discount:14!null l_shipdate:18!null o_orderkey:24!null o_custkey:25!null c_custkey:33!null c_nationkey:36!null n1.n_nationkey:41!null n1.n_name:42!null n2.n_nationkey:45!null n2.n_name:46!null + │ │ ├── columns: s_suppkey:1!null s_nationkey:4!null l_orderkey:9!null l_suppkey:11!null l_extendedprice:14!null l_discount:15!null l_shipdate:19!null o_orderkey:26!null o_custkey:27!null c_custkey:36!null c_nationkey:39!null n1.n_nationkey:45!null n1.n_name:46!null n2.n_nationkey:50!null n2.n_name:51!null │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ ├── fd: (1)-->(4), (24)-->(25), (33)-->(36), (41)-->(42), (45)-->(46), (36)==(45), (45)==(36), (25)==(33), (33)==(25), (8)==(24), (24)==(8), (1)==(10), (10)==(1), (4)==(41), (41)==(4) + │ │ ├── fd: (1)-->(4), (26)-->(27), (36)-->(39), (45)-->(46), (50)-->(51), (39)==(50), (50)==(39), (27)==(36), (36)==(27), (9)==(26), (26)==(9), (1)==(11), (11)==(1), (4)==(45), (45)==(4) │ │ ├── inner-join (hash) - │ │ │ ├── columns: s_suppkey:1!null s_nationkey:4!null l_orderkey:8!null l_suppkey:10!null l_extendedprice:13!null l_discount:14!null l_shipdate:18!null o_orderkey:24!null o_custkey:25!null n1.n_nationkey:41!null n1.n_name:42!null + │ │ │ ├── columns: s_suppkey:1!null s_nationkey:4!null l_orderkey:9!null l_suppkey:11!null l_extendedprice:14!null l_discount:15!null l_shipdate:19!null o_orderkey:26!null o_custkey:27!null n1.n_nationkey:45!null n1.n_name:46!null │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ │ │ ├── fd: (24)-->(25), (1)-->(4), (41)-->(42), (4)==(41), (41)==(4), (1)==(10), (10)==(1), (8)==(24), (24)==(8) + │ │ │ ├── fd: (26)-->(27), (1)-->(4), (45)-->(46), (4)==(45), (45)==(4), (1)==(11), (11)==(1), (9)==(26), (26)==(9) │ │ │ ├── inner-join (hash) - │ │ │ │ ├── columns: l_orderkey:8!null l_suppkey:10!null l_extendedprice:13!null l_discount:14!null l_shipdate:18!null o_orderkey:24!null o_custkey:25!null + │ │ │ │ ├── columns: l_orderkey:9!null l_suppkey:11!null l_extendedprice:14!null l_discount:15!null l_shipdate:19!null o_orderkey:26!null o_custkey:27!null │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ │ │ │ ├── fd: (24)-->(25), (8)==(24), (24)==(8) + │ │ │ │ ├── fd: (26)-->(27), (9)==(26), (26)==(9) │ │ │ │ ├── select - │ │ │ │ │ ├── columns: l_orderkey:8!null l_suppkey:10!null l_extendedprice:13!null l_discount:14!null l_shipdate:18!null + │ │ │ │ │ ├── columns: l_orderkey:9!null l_suppkey:11!null l_extendedprice:14!null l_discount:15!null l_shipdate:19!null │ │ │ │ │ ├── scan lineitem - │ │ │ │ │ │ └── columns: l_orderkey:8!null l_suppkey:10!null l_extendedprice:13!null l_discount:14!null l_shipdate:18!null + │ │ │ │ │ │ └── columns: l_orderkey:9!null l_suppkey:11!null l_extendedprice:14!null l_discount:15!null l_shipdate:19!null │ │ │ │ │ └── filters - │ │ │ │ │ └── (l_shipdate:18 >= '1995-01-01') AND (l_shipdate:18 <= '1996-12-31') [outer=(18), constraints=(/18: [/'1995-01-01' - /'1996-12-31']; tight)] + │ │ │ │ │ └── (l_shipdate:19 >= '1995-01-01') AND (l_shipdate:19 <= '1996-12-31') [outer=(19), constraints=(/19: [/'1995-01-01' - /'1996-12-31']; tight)] │ │ │ │ ├── scan orders@o_ck - │ │ │ │ │ ├── columns: o_orderkey:24!null o_custkey:25!null - │ │ │ │ │ ├── key: (24) - │ │ │ │ │ └── fd: (24)-->(25) + │ │ │ │ │ ├── columns: o_orderkey:26!null o_custkey:27!null + │ │ │ │ │ ├── key: (26) + │ │ │ │ │ └── fd: (26)-->(27) │ │ │ │ └── filters - │ │ │ │ └── o_orderkey:24 = l_orderkey:8 [outer=(8,24), constraints=(/8: (/NULL - ]; /24: (/NULL - ]), fd=(8)==(24), (24)==(8)] + │ │ │ │ └── o_orderkey:26 = l_orderkey:9 [outer=(9,26), constraints=(/9: (/NULL - ]; /26: (/NULL - ]), fd=(9)==(26), (26)==(9)] │ │ │ ├── inner-join (merge) - │ │ │ │ ├── columns: s_suppkey:1!null s_nationkey:4!null n1.n_nationkey:41!null n1.n_name:42!null + │ │ │ │ ├── columns: s_suppkey:1!null s_nationkey:4!null n1.n_nationkey:45!null n1.n_name:46!null │ │ │ │ ├── left ordering: +4 - │ │ │ │ ├── right ordering: +41 + │ │ │ │ ├── right ordering: +45 │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(4), (41)-->(42), (4)==(41), (41)==(4) + │ │ │ │ ├── fd: (1)-->(4), (45)-->(46), (4)==(45), (45)==(4) │ │ │ │ ├── scan supplier@s_nk │ │ │ │ │ ├── columns: s_suppkey:1!null s_nationkey:4!null │ │ │ │ │ ├── key: (1) │ │ │ │ │ ├── fd: (1)-->(4) │ │ │ │ │ └── ordering: +4 │ │ │ │ ├── scan n1 - │ │ │ │ │ ├── columns: n1.n_nationkey:41!null n1.n_name:42!null - │ │ │ │ │ ├── key: (41) - │ │ │ │ │ ├── fd: (41)-->(42) - │ │ │ │ │ └── ordering: +41 + │ │ │ │ │ ├── columns: n1.n_nationkey:45!null n1.n_name:46!null + │ │ │ │ │ ├── key: (45) + │ │ │ │ │ ├── fd: (45)-->(46) + │ │ │ │ │ └── ordering: +45 │ │ │ │ └── filters (true) │ │ │ └── filters - │ │ │ └── s_suppkey:1 = l_suppkey:10 [outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] + │ │ │ └── s_suppkey:1 = l_suppkey:11 [outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ]), fd=(1)==(11), (11)==(1)] │ │ ├── inner-join (merge) - │ │ │ ├── columns: c_custkey:33!null c_nationkey:36!null n2.n_nationkey:45!null n2.n_name:46!null - │ │ │ ├── left ordering: +36 - │ │ │ ├── right ordering: +45 - │ │ │ ├── key: (33) - │ │ │ ├── fd: (33)-->(36), (45)-->(46), (36)==(45), (45)==(36) + │ │ │ ├── columns: c_custkey:36!null c_nationkey:39!null n2.n_nationkey:50!null n2.n_name:51!null + │ │ │ ├── left ordering: +39 + │ │ │ ├── right ordering: +50 + │ │ │ ├── key: (36) + │ │ │ ├── fd: (36)-->(39), (50)-->(51), (39)==(50), (50)==(39) │ │ │ ├── scan customer@c_nk - │ │ │ │ ├── columns: c_custkey:33!null c_nationkey:36!null - │ │ │ │ ├── key: (33) - │ │ │ │ ├── fd: (33)-->(36) - │ │ │ │ └── ordering: +36 + │ │ │ │ ├── columns: c_custkey:36!null c_nationkey:39!null + │ │ │ │ ├── key: (36) + │ │ │ │ ├── fd: (36)-->(39) + │ │ │ │ └── ordering: +39 │ │ │ ├── scan n2 - │ │ │ │ ├── columns: n2.n_nationkey:45!null n2.n_name:46!null - │ │ │ │ ├── key: (45) - │ │ │ │ ├── fd: (45)-->(46) - │ │ │ │ └── ordering: +45 + │ │ │ │ ├── columns: n2.n_nationkey:50!null n2.n_name:51!null + │ │ │ │ ├── key: (50) + │ │ │ │ ├── fd: (50)-->(51) + │ │ │ │ └── ordering: +50 │ │ │ └── filters (true) │ │ └── filters - │ │ ├── c_custkey:33 = o_custkey:25 [outer=(25,33), constraints=(/25: (/NULL - ]; /33: (/NULL - ]), fd=(25)==(33), (33)==(25)] - │ │ └── ((n1.n_name:42 = 'FRANCE') AND (n2.n_name:46 = 'GERMANY')) OR ((n1.n_name:42 = 'GERMANY') AND (n2.n_name:46 = 'FRANCE')) [outer=(42,46), constraints=(/42: [/'FRANCE' - /'FRANCE'] [/'GERMANY' - /'GERMANY']; /46: [/'FRANCE' - /'FRANCE'] [/'GERMANY' - /'GERMANY'])] + │ │ ├── c_custkey:36 = o_custkey:27 [outer=(27,36), constraints=(/27: (/NULL - ]; /36: (/NULL - ]), fd=(27)==(36), (36)==(27)] + │ │ └── ((n1.n_name:46 = 'FRANCE') AND (n2.n_name:51 = 'GERMANY')) OR ((n1.n_name:46 = 'GERMANY') AND (n2.n_name:51 = 'FRANCE')) [outer=(46,51), constraints=(/46: [/'FRANCE' - /'FRANCE'] [/'GERMANY' - /'GERMANY']; /51: [/'FRANCE' - /'FRANCE'] [/'GERMANY' - /'GERMANY'])] │ └── projections - │ ├── extract('year', l_shipdate:18) [as=l_year:49, outer=(18), immutable] - │ └── l_extendedprice:13 * (1.0 - l_discount:14) [as=volume:50, outer=(13,14), immutable] + │ ├── extract('year', l_shipdate:19) [as=l_year:55, outer=(19), immutable] + │ └── l_extendedprice:14 * (1.0 - l_discount:15) [as=volume:56, outer=(14,15), immutable] └── aggregations - └── sum [as=sum:51, outer=(50)] - └── volume:50 + └── sum [as=sum:57, outer=(56)] + └── volume:56 # -------------------------------------------------- # Q8 @@ -812,62 +812,62 @@ ORDER BY o_year; ---- sort - ├── columns: o_year:61 mkt_share:66!null + ├── columns: o_year:69 mkt_share:74!null ├── immutable - ├── key: (61) - ├── fd: (61)-->(66) - ├── ordering: +61 + ├── key: (69) + ├── fd: (69)-->(74) + ├── ordering: +69 └── project - ├── columns: mkt_share:66!null o_year:61 + ├── columns: mkt_share:74!null o_year:69 ├── immutable - ├── key: (61) - ├── fd: (61)-->(66) + ├── key: (69) + ├── fd: (69)-->(74) ├── group-by - │ ├── columns: o_year:61 sum:64!null sum:65!null - │ ├── grouping columns: o_year:61 + │ ├── columns: o_year:69 sum:72!null sum:73!null + │ ├── grouping columns: o_year:69 │ ├── immutable - │ ├── key: (61) - │ ├── fd: (61)-->(64,65) + │ ├── key: (69) + │ ├── fd: (69)-->(72,73) │ ├── project - │ │ ├── columns: column63:63!null o_year:61 volume:62!null + │ │ ├── columns: column71:71!null o_year:69 volume:70!null │ │ ├── immutable │ │ ├── project - │ │ │ ├── columns: o_year:61 volume:62!null n2.n_name:55!null + │ │ │ ├── columns: o_year:69 volume:70!null n2.n_name:61!null │ │ │ ├── immutable │ │ │ ├── inner-join (hash) - │ │ │ │ ├── columns: p_partkey:1!null p_type:5!null s_suppkey:10!null s_nationkey:13!null l_orderkey:17!null l_partkey:18!null l_suppkey:19!null l_extendedprice:22!null l_discount:23!null o_orderkey:33!null o_custkey:34!null o_orderdate:37!null c_custkey:42!null c_nationkey:45!null n1.n_nationkey:50!null n1.n_regionkey:52!null n2.n_nationkey:54!null n2.n_name:55!null r_regionkey:58!null r_name:59!null + │ │ │ │ ├── columns: p_partkey:1!null p_type:5!null s_suppkey:11!null s_nationkey:14!null l_orderkey:19!null l_partkey:20!null l_suppkey:21!null l_extendedprice:24!null l_discount:25!null o_orderkey:36!null o_custkey:37!null o_orderdate:40!null c_custkey:46!null c_nationkey:49!null n1.n_nationkey:55!null n1.n_regionkey:57!null n2.n_nationkey:60!null n2.n_name:61!null r_regionkey:65!null r_name:66!null │ │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ │ ├── fd: ()-->(5,59), (10)-->(13), (33)-->(34,37), (42)-->(45), (50)-->(52), (52)==(58), (58)==(52), (45)==(50), (50)==(45), (34)==(42), (42)==(34), (17)==(33), (33)==(17), (10)==(19), (19)==(10), (54)-->(55), (13)==(54), (54)==(13), (1)==(18), (18)==(1) + │ │ │ │ ├── fd: ()-->(5,66), (11)-->(14), (36)-->(37,40), (46)-->(49), (55)-->(57), (57)==(65), (65)==(57), (49)==(55), (55)==(49), (37)==(46), (46)==(37), (19)==(36), (36)==(19), (11)==(21), (21)==(11), (60)-->(61), (14)==(60), (60)==(14), (1)==(20), (20)==(1) │ │ │ │ ├── inner-join (lookup nation) - │ │ │ │ │ ├── columns: p_partkey:1!null p_type:5!null s_suppkey:10!null s_nationkey:13!null l_orderkey:17!null l_partkey:18!null l_suppkey:19!null l_extendedprice:22!null l_discount:23!null o_orderkey:33!null o_custkey:34!null o_orderdate:37!null c_custkey:42!null c_nationkey:45!null n2.n_nationkey:54!null n2.n_name:55!null - │ │ │ │ │ ├── key columns: [13] = [54] + │ │ │ │ │ ├── columns: p_partkey:1!null p_type:5!null s_suppkey:11!null s_nationkey:14!null l_orderkey:19!null l_partkey:20!null l_suppkey:21!null l_extendedprice:24!null l_discount:25!null o_orderkey:36!null o_custkey:37!null o_orderdate:40!null c_custkey:46!null c_nationkey:49!null n2.n_nationkey:60!null n2.n_name:61!null + │ │ │ │ │ ├── key columns: [14] = [60] │ │ │ │ │ ├── lookup columns are key - │ │ │ │ │ ├── fd: ()-->(5), (42)-->(45), (33)-->(34,37), (10)-->(13), (54)-->(55), (13)==(54), (54)==(13), (10)==(19), (19)==(10), (17)==(33), (33)==(17), (34)==(42), (42)==(34), (1)==(18), (18)==(1) + │ │ │ │ │ ├── fd: ()-->(5), (46)-->(49), (36)-->(37,40), (11)-->(14), (60)-->(61), (14)==(60), (60)==(14), (11)==(21), (21)==(11), (19)==(36), (36)==(19), (37)==(46), (46)==(37), (1)==(20), (20)==(1) │ │ │ │ │ ├── inner-join (lookup customer) - │ │ │ │ │ │ ├── columns: p_partkey:1!null p_type:5!null s_suppkey:10!null s_nationkey:13!null l_orderkey:17!null l_partkey:18!null l_suppkey:19!null l_extendedprice:22!null l_discount:23!null o_orderkey:33!null o_custkey:34!null o_orderdate:37!null c_custkey:42!null c_nationkey:45!null - │ │ │ │ │ │ ├── key columns: [34] = [42] + │ │ │ │ │ │ ├── columns: p_partkey:1!null p_type:5!null s_suppkey:11!null s_nationkey:14!null l_orderkey:19!null l_partkey:20!null l_suppkey:21!null l_extendedprice:24!null l_discount:25!null o_orderkey:36!null o_custkey:37!null o_orderdate:40!null c_custkey:46!null c_nationkey:49!null + │ │ │ │ │ │ ├── key columns: [37] = [46] │ │ │ │ │ │ ├── lookup columns are key - │ │ │ │ │ │ ├── fd: ()-->(5), (10)-->(13), (33)-->(34,37), (42)-->(45), (34)==(42), (42)==(34), (17)==(33), (33)==(17), (10)==(19), (19)==(10), (1)==(18), (18)==(1) + │ │ │ │ │ │ ├── fd: ()-->(5), (11)-->(14), (36)-->(37,40), (46)-->(49), (37)==(46), (46)==(37), (19)==(36), (36)==(19), (11)==(21), (21)==(11), (1)==(20), (20)==(1) │ │ │ │ │ │ ├── inner-join (lookup supplier) - │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_type:5!null s_suppkey:10!null s_nationkey:13!null l_orderkey:17!null l_partkey:18!null l_suppkey:19!null l_extendedprice:22!null l_discount:23!null o_orderkey:33!null o_custkey:34!null o_orderdate:37!null - │ │ │ │ │ │ │ ├── key columns: [19] = [10] + │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_type:5!null s_suppkey:11!null s_nationkey:14!null l_orderkey:19!null l_partkey:20!null l_suppkey:21!null l_extendedprice:24!null l_discount:25!null o_orderkey:36!null o_custkey:37!null o_orderdate:40!null + │ │ │ │ │ │ │ ├── key columns: [21] = [11] │ │ │ │ │ │ │ ├── lookup columns are key - │ │ │ │ │ │ │ ├── fd: ()-->(5), (10)-->(13), (33)-->(34,37), (17)==(33), (33)==(17), (10)==(19), (19)==(10), (1)==(18), (18)==(1) + │ │ │ │ │ │ │ ├── fd: ()-->(5), (11)-->(14), (36)-->(37,40), (19)==(36), (36)==(19), (11)==(21), (21)==(11), (1)==(20), (20)==(1) │ │ │ │ │ │ │ ├── inner-join (lookup orders) - │ │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_type:5!null l_orderkey:17!null l_partkey:18!null l_suppkey:19!null l_extendedprice:22!null l_discount:23!null o_orderkey:33!null o_custkey:34!null o_orderdate:37!null - │ │ │ │ │ │ │ │ ├── key columns: [17] = [33] + │ │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_type:5!null l_orderkey:19!null l_partkey:20!null l_suppkey:21!null l_extendedprice:24!null l_discount:25!null o_orderkey:36!null o_custkey:37!null o_orderdate:40!null + │ │ │ │ │ │ │ │ ├── key columns: [19] = [36] │ │ │ │ │ │ │ │ ├── lookup columns are key - │ │ │ │ │ │ │ │ ├── fd: ()-->(5), (33)-->(34,37), (17)==(33), (33)==(17), (1)==(18), (18)==(1) + │ │ │ │ │ │ │ │ ├── fd: ()-->(5), (36)-->(37,40), (19)==(36), (36)==(19), (1)==(20), (20)==(1) │ │ │ │ │ │ │ │ ├── inner-join (lookup lineitem) - │ │ │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_type:5!null l_orderkey:17!null l_partkey:18!null l_suppkey:19!null l_extendedprice:22!null l_discount:23!null - │ │ │ │ │ │ │ │ │ ├── key columns: [17 20] = [17 20] + │ │ │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_type:5!null l_orderkey:19!null l_partkey:20!null l_suppkey:21!null l_extendedprice:24!null l_discount:25!null + │ │ │ │ │ │ │ │ │ ├── key columns: [19 22] = [19 22] │ │ │ │ │ │ │ │ │ ├── lookup columns are key - │ │ │ │ │ │ │ │ │ ├── fd: ()-->(5), (1)==(18), (18)==(1) + │ │ │ │ │ │ │ │ │ ├── fd: ()-->(5), (1)==(20), (20)==(1) │ │ │ │ │ │ │ │ │ ├── inner-join (lookup lineitem@l_pk) - │ │ │ │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_type:5!null l_orderkey:17!null l_partkey:18!null l_linenumber:20!null - │ │ │ │ │ │ │ │ │ │ ├── key columns: [1] = [18] - │ │ │ │ │ │ │ │ │ │ ├── key: (17,20) - │ │ │ │ │ │ │ │ │ │ ├── fd: ()-->(5), (17,20)-->(18), (1)==(18), (18)==(1) + │ │ │ │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_type:5!null l_orderkey:19!null l_partkey:20!null l_linenumber:22!null + │ │ │ │ │ │ │ │ │ │ ├── key columns: [1] = [20] + │ │ │ │ │ │ │ │ │ │ ├── key: (19,22) + │ │ │ │ │ │ │ │ │ │ ├── fd: ()-->(5), (19,22)-->(20), (1)==(20), (20)==(1) │ │ │ │ │ │ │ │ │ │ ├── select │ │ │ │ │ │ │ │ │ │ │ ├── columns: p_partkey:1!null p_type:5!null │ │ │ │ │ │ │ │ │ │ │ ├── key: (1) @@ -881,40 +881,40 @@ sort │ │ │ │ │ │ │ │ │ │ └── filters (true) │ │ │ │ │ │ │ │ │ └── filters (true) │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ └── (o_orderdate:37 >= '1995-01-01') AND (o_orderdate:37 <= '1996-12-31') [outer=(37), constraints=(/37: [/'1995-01-01' - /'1996-12-31']; tight)] + │ │ │ │ │ │ │ │ └── (o_orderdate:40 >= '1995-01-01') AND (o_orderdate:40 <= '1996-12-31') [outer=(40), constraints=(/40: [/'1995-01-01' - /'1996-12-31']; tight)] │ │ │ │ │ │ │ └── filters (true) │ │ │ │ │ │ └── filters (true) │ │ │ │ │ └── filters (true) │ │ │ │ ├── inner-join (lookup nation@n_rk) - │ │ │ │ │ ├── columns: n1.n_nationkey:50!null n1.n_regionkey:52!null r_regionkey:58!null r_name:59!null - │ │ │ │ │ ├── key columns: [58] = [52] - │ │ │ │ │ ├── key: (50) - │ │ │ │ │ ├── fd: ()-->(59), (50)-->(52), (52)==(58), (58)==(52) + │ │ │ │ │ ├── columns: n1.n_nationkey:55!null n1.n_regionkey:57!null r_regionkey:65!null r_name:66!null + │ │ │ │ │ ├── key columns: [65] = [57] + │ │ │ │ │ ├── key: (55) + │ │ │ │ │ ├── fd: ()-->(66), (55)-->(57), (57)==(65), (65)==(57) │ │ │ │ │ ├── select - │ │ │ │ │ │ ├── columns: r_regionkey:58!null r_name:59!null - │ │ │ │ │ │ ├── key: (58) - │ │ │ │ │ │ ├── fd: ()-->(59) + │ │ │ │ │ │ ├── columns: r_regionkey:65!null r_name:66!null + │ │ │ │ │ │ ├── key: (65) + │ │ │ │ │ │ ├── fd: ()-->(66) │ │ │ │ │ │ ├── scan region - │ │ │ │ │ │ │ ├── columns: r_regionkey:58!null r_name:59!null - │ │ │ │ │ │ │ ├── key: (58) - │ │ │ │ │ │ │ └── fd: (58)-->(59) + │ │ │ │ │ │ │ ├── columns: r_regionkey:65!null r_name:66!null + │ │ │ │ │ │ │ ├── key: (65) + │ │ │ │ │ │ │ └── fd: (65)-->(66) │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── r_name:59 = 'AMERICA' [outer=(59), constraints=(/59: [/'AMERICA' - /'AMERICA']; tight), fd=()-->(59)] + │ │ │ │ │ │ └── r_name:66 = 'AMERICA' [outer=(66), constraints=(/66: [/'AMERICA' - /'AMERICA']; tight), fd=()-->(66)] │ │ │ │ │ └── filters (true) │ │ │ │ └── filters - │ │ │ │ └── c_nationkey:45 = n1.n_nationkey:50 [outer=(45,50), constraints=(/45: (/NULL - ]; /50: (/NULL - ]), fd=(45)==(50), (50)==(45)] + │ │ │ │ └── c_nationkey:49 = n1.n_nationkey:55 [outer=(49,55), constraints=(/49: (/NULL - ]; /55: (/NULL - ]), fd=(49)==(55), (55)==(49)] │ │ │ └── projections - │ │ │ ├── extract('year', o_orderdate:37) [as=o_year:61, outer=(37), immutable] - │ │ │ └── l_extendedprice:22 * (1.0 - l_discount:23) [as=volume:62, outer=(22,23), immutable] + │ │ │ ├── extract('year', o_orderdate:40) [as=o_year:69, outer=(40), immutable] + │ │ │ └── l_extendedprice:24 * (1.0 - l_discount:25) [as=volume:70, outer=(24,25), immutable] │ │ └── projections - │ │ └── CASE WHEN n2.n_name:55 = 'BRAZIL' THEN volume:62 ELSE 0.0 END [as=column63:63, outer=(55,62)] + │ │ └── CASE WHEN n2.n_name:61 = 'BRAZIL' THEN volume:70 ELSE 0.0 END [as=column71:71, outer=(61,70)] │ └── aggregations - │ ├── sum [as=sum:64, outer=(63)] - │ │ └── column63:63 - │ └── sum [as=sum:65, outer=(62)] - │ └── volume:62 + │ ├── sum [as=sum:72, outer=(71)] + │ │ └── column71:71 + │ └── sum [as=sum:73, outer=(70)] + │ └── volume:70 └── projections - └── sum:64 / sum:65 [as=mkt_share:66, outer=(64,65), immutable] + └── sum:72 / sum:73 [as=mkt_share:74, outer=(72,73), immutable] # -------------------------------------------------- # Q9 @@ -966,64 +966,64 @@ ORDER BY o_year DESC; ---- sort - ├── columns: nation:48!null o_year:51 sum_profit:53!null + ├── columns: nation:53!null o_year:57 sum_profit:59!null ├── immutable - ├── key: (48,51) - ├── fd: (48,51)-->(53) - ├── ordering: +48,-51 + ├── key: (53,57) + ├── fd: (53,57)-->(59) + ├── ordering: +53,-57 └── group-by - ├── columns: n_name:48!null o_year:51 sum:53!null - ├── grouping columns: n_name:48!null o_year:51 + ├── columns: n_name:53!null o_year:57 sum:59!null + ├── grouping columns: n_name:53!null o_year:57 ├── immutable - ├── key: (48,51) - ├── fd: (48,51)-->(53) + ├── key: (53,57) + ├── fd: (53,57)-->(59) ├── project - │ ├── columns: o_year:51 amount:52!null n_name:48!null + │ ├── columns: o_year:57 amount:58!null n_name:53!null │ ├── immutable │ ├── inner-join (lookup nation) - │ │ ├── columns: p_partkey:1!null p_name:2!null s_suppkey:10!null s_nationkey:13!null l_orderkey:17!null l_partkey:18!null l_suppkey:19!null l_quantity:21!null l_extendedprice:22!null l_discount:23!null ps_partkey:33!null ps_suppkey:34!null ps_supplycost:36!null o_orderkey:38!null o_orderdate:42!null n_nationkey:47!null n_name:48!null - │ │ ├── key columns: [13] = [47] + │ │ ├── columns: p_partkey:1!null p_name:2!null s_suppkey:11!null s_nationkey:14!null l_orderkey:19!null l_partkey:20!null l_suppkey:21!null l_quantity:23!null l_extendedprice:24!null l_discount:25!null ps_partkey:36!null ps_suppkey:37!null ps_supplycost:39!null o_orderkey:42!null o_orderdate:46!null n_nationkey:52!null n_name:53!null + │ │ ├── key columns: [14] = [52] │ │ ├── lookup columns are key - │ │ ├── fd: (1)-->(2), (10)-->(13), (33,34)-->(36), (19)==(10,34), (34)==(10,19), (18)==(1,33), (33)==(1,18), (38)-->(42), (17)==(38), (38)==(17), (10)==(19,34), (47)-->(48), (13)==(47), (47)==(13), (1)==(18,33) + │ │ ├── fd: (1)-->(2), (11)-->(14), (36,37)-->(39), (21)==(11,37), (37)==(11,21), (20)==(1,36), (36)==(1,20), (42)-->(46), (19)==(42), (42)==(19), (11)==(21,37), (52)-->(53), (14)==(52), (52)==(14), (1)==(20,36) │ │ ├── inner-join (lookup orders) - │ │ │ ├── columns: p_partkey:1!null p_name:2!null s_suppkey:10!null s_nationkey:13!null l_orderkey:17!null l_partkey:18!null l_suppkey:19!null l_quantity:21!null l_extendedprice:22!null l_discount:23!null ps_partkey:33!null ps_suppkey:34!null ps_supplycost:36!null o_orderkey:38!null o_orderdate:42!null - │ │ │ ├── key columns: [17] = [38] + │ │ │ ├── columns: p_partkey:1!null p_name:2!null s_suppkey:11!null s_nationkey:14!null l_orderkey:19!null l_partkey:20!null l_suppkey:21!null l_quantity:23!null l_extendedprice:24!null l_discount:25!null ps_partkey:36!null ps_suppkey:37!null ps_supplycost:39!null o_orderkey:42!null o_orderdate:46!null + │ │ │ ├── key columns: [19] = [42] │ │ │ ├── lookup columns are key - │ │ │ ├── fd: (1)-->(2), (10)-->(13), (33,34)-->(36), (19)==(10,34), (34)==(10,19), (18)==(1,33), (33)==(1,18), (38)-->(42), (17)==(38), (38)==(17), (10)==(19,34), (1)==(18,33) + │ │ │ ├── fd: (1)-->(2), (11)-->(14), (36,37)-->(39), (21)==(11,37), (37)==(11,21), (20)==(1,36), (36)==(1,20), (42)-->(46), (19)==(42), (42)==(19), (11)==(21,37), (1)==(20,36) │ │ │ ├── inner-join (lookup part) - │ │ │ │ ├── columns: p_partkey:1!null p_name:2!null s_suppkey:10!null s_nationkey:13!null l_orderkey:17!null l_partkey:18!null l_suppkey:19!null l_quantity:21!null l_extendedprice:22!null l_discount:23!null ps_partkey:33!null ps_suppkey:34!null ps_supplycost:36!null - │ │ │ │ ├── key columns: [18] = [1] + │ │ │ │ ├── columns: p_partkey:1!null p_name:2!null s_suppkey:11!null s_nationkey:14!null l_orderkey:19!null l_partkey:20!null l_suppkey:21!null l_quantity:23!null l_extendedprice:24!null l_discount:25!null ps_partkey:36!null ps_suppkey:37!null ps_supplycost:39!null + │ │ │ │ ├── key columns: [20] = [1] │ │ │ │ ├── lookup columns are key - │ │ │ │ ├── fd: (1)-->(2), (10)-->(13), (33,34)-->(36), (19)==(10,34), (34)==(10,19), (18)==(1,33), (33)==(1,18), (10)==(19,34), (1)==(18,33) + │ │ │ │ ├── fd: (1)-->(2), (11)-->(14), (36,37)-->(39), (21)==(11,37), (37)==(11,21), (20)==(1,36), (36)==(1,20), (11)==(21,37), (1)==(20,36) │ │ │ │ ├── inner-join (lookup supplier) - │ │ │ │ │ ├── columns: s_suppkey:10!null s_nationkey:13!null l_orderkey:17!null l_partkey:18!null l_suppkey:19!null l_quantity:21!null l_extendedprice:22!null l_discount:23!null ps_partkey:33!null ps_suppkey:34!null ps_supplycost:36!null - │ │ │ │ │ ├── key columns: [19] = [10] + │ │ │ │ │ ├── columns: s_suppkey:11!null s_nationkey:14!null l_orderkey:19!null l_partkey:20!null l_suppkey:21!null l_quantity:23!null l_extendedprice:24!null l_discount:25!null ps_partkey:36!null ps_suppkey:37!null ps_supplycost:39!null + │ │ │ │ │ ├── key columns: [21] = [11] │ │ │ │ │ ├── lookup columns are key - │ │ │ │ │ ├── fd: (10)-->(13), (33,34)-->(36), (19)==(10,34), (34)==(10,19), (18)==(33), (33)==(18), (10)==(19,34) + │ │ │ │ │ ├── fd: (11)-->(14), (36,37)-->(39), (21)==(11,37), (37)==(11,21), (20)==(36), (36)==(20), (11)==(21,37) │ │ │ │ │ ├── inner-join (hash) - │ │ │ │ │ │ ├── columns: l_orderkey:17!null l_partkey:18!null l_suppkey:19!null l_quantity:21!null l_extendedprice:22!null l_discount:23!null ps_partkey:33!null ps_suppkey:34!null ps_supplycost:36!null + │ │ │ │ │ │ ├── columns: l_orderkey:19!null l_partkey:20!null l_suppkey:21!null l_quantity:23!null l_extendedprice:24!null l_discount:25!null ps_partkey:36!null ps_suppkey:37!null ps_supplycost:39!null │ │ │ │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-more) - │ │ │ │ │ │ ├── fd: (33,34)-->(36), (19)==(34), (34)==(19), (18)==(33), (33)==(18) + │ │ │ │ │ │ ├── fd: (36,37)-->(39), (21)==(37), (37)==(21), (20)==(36), (36)==(20) │ │ │ │ │ │ ├── scan lineitem - │ │ │ │ │ │ │ └── columns: l_orderkey:17!null l_partkey:18!null l_suppkey:19!null l_quantity:21!null l_extendedprice:22!null l_discount:23!null + │ │ │ │ │ │ │ └── columns: l_orderkey:19!null l_partkey:20!null l_suppkey:21!null l_quantity:23!null l_extendedprice:24!null l_discount:25!null │ │ │ │ │ │ ├── scan partsupp - │ │ │ │ │ │ │ ├── columns: ps_partkey:33!null ps_suppkey:34!null ps_supplycost:36!null - │ │ │ │ │ │ │ ├── key: (33,34) - │ │ │ │ │ │ │ └── fd: (33,34)-->(36) + │ │ │ │ │ │ │ ├── columns: ps_partkey:36!null ps_suppkey:37!null ps_supplycost:39!null + │ │ │ │ │ │ │ ├── key: (36,37) + │ │ │ │ │ │ │ └── fd: (36,37)-->(39) │ │ │ │ │ │ └── filters - │ │ │ │ │ │ ├── ps_suppkey:34 = l_suppkey:19 [outer=(19,34), constraints=(/19: (/NULL - ]; /34: (/NULL - ]), fd=(19)==(34), (34)==(19)] - │ │ │ │ │ │ └── ps_partkey:33 = l_partkey:18 [outer=(18,33), constraints=(/18: (/NULL - ]; /33: (/NULL - ]), fd=(18)==(33), (33)==(18)] + │ │ │ │ │ │ ├── ps_suppkey:37 = l_suppkey:21 [outer=(21,37), constraints=(/21: (/NULL - ]; /37: (/NULL - ]), fd=(21)==(37), (37)==(21)] + │ │ │ │ │ │ └── ps_partkey:36 = l_partkey:20 [outer=(20,36), constraints=(/20: (/NULL - ]; /36: (/NULL - ]), fd=(20)==(36), (36)==(20)] │ │ │ │ │ └── filters (true) │ │ │ │ └── filters │ │ │ │ └── p_name:2 LIKE '%green%' [outer=(2), constraints=(/2: (/NULL - ])] │ │ │ └── filters (true) │ │ └── filters (true) │ └── projections - │ ├── extract('year', o_orderdate:42) [as=o_year:51, outer=(42), immutable] - │ └── (l_extendedprice:22 * (1.0 - l_discount:23)) - (ps_supplycost:36 * l_quantity:21) [as=amount:52, outer=(21-23,36), immutable] + │ ├── extract('year', o_orderdate:46) [as=o_year:57, outer=(46), immutable] + │ └── (l_extendedprice:24 * (1.0 - l_discount:25)) - (ps_supplycost:39 * l_quantity:23) [as=amount:58, outer=(23-25,39), immutable] └── aggregations - └── sum [as=sum:53, outer=(52)] - └── amount:52 + └── sum [as=sum:59, outer=(58)] + └── amount:58 # -------------------------------------------------- # Q10 @@ -1074,61 +1074,61 @@ ORDER BY LIMIT 20; ---- limit - ├── columns: c_custkey:1!null c_name:2!null revenue:39!null c_acctbal:6!null n_name:35!null c_address:3!null c_phone:5!null c_comment:8!null - ├── internal-ordering: -39 + ├── columns: c_custkey:1!null c_name:2!null revenue:43!null c_acctbal:6!null n_name:38!null c_address:3!null c_phone:5!null c_comment:8!null + ├── internal-ordering: -43 ├── cardinality: [0 - 20] ├── immutable ├── key: (1) - ├── fd: (1)-->(2,3,5,6,8,35,39) - ├── ordering: -39 + ├── fd: (1)-->(2,3,5,6,8,38,43) + ├── ordering: -43 ├── sort - │ ├── columns: c_custkey:1!null c_name:2!null c_address:3!null c_phone:5!null c_acctbal:6!null c_comment:8!null n_name:35!null sum:39!null + │ ├── columns: c_custkey:1!null c_name:2!null c_address:3!null c_phone:5!null c_acctbal:6!null c_comment:8!null n_name:38!null sum:43!null │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(2,3,5,6,8,35,39) - │ ├── ordering: -39 + │ ├── fd: (1)-->(2,3,5,6,8,38,43) + │ ├── ordering: -43 │ ├── limit hint: 20.00 │ └── group-by - │ ├── columns: c_custkey:1!null c_name:2!null c_address:3!null c_phone:5!null c_acctbal:6!null c_comment:8!null n_name:35!null sum:39!null + │ ├── columns: c_custkey:1!null c_name:2!null c_address:3!null c_phone:5!null c_acctbal:6!null c_comment:8!null n_name:38!null sum:43!null │ ├── grouping columns: c_custkey:1!null │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(2,3,5,6,8,35,39) + │ ├── fd: (1)-->(2,3,5,6,8,38,43) │ ├── project - │ │ ├── columns: column38:38!null c_custkey:1!null c_name:2!null c_address:3!null c_phone:5!null c_acctbal:6!null c_comment:8!null n_name:35!null + │ │ ├── columns: column42:42!null c_custkey:1!null c_name:2!null c_address:3!null c_phone:5!null c_acctbal:6!null c_comment:8!null n_name:38!null │ │ ├── immutable - │ │ ├── fd: (1)-->(2,3,5,6,8,35) + │ │ ├── fd: (1)-->(2,3,5,6,8,38) │ │ ├── inner-join (lookup nation) - │ │ │ ├── columns: c_custkey:1!null c_name:2!null c_address:3!null c_nationkey:4!null c_phone:5!null c_acctbal:6!null c_comment:8!null o_orderkey:9!null o_custkey:10!null o_orderdate:13!null l_orderkey:18!null l_extendedprice:23!null l_discount:24!null l_returnflag:26!null n_nationkey:34!null n_name:35!null - │ │ │ ├── key columns: [4] = [34] + │ │ │ ├── columns: c_custkey:1!null c_name:2!null c_address:3!null c_nationkey:4!null c_phone:5!null c_acctbal:6!null c_comment:8!null o_orderkey:10!null o_custkey:11!null o_orderdate:14!null l_orderkey:20!null l_extendedprice:25!null l_discount:26!null l_returnflag:28!null n_nationkey:37!null n_name:38!null + │ │ │ ├── key columns: [4] = [37] │ │ │ ├── lookup columns are key - │ │ │ ├── fd: ()-->(26), (1)-->(2-6,8), (9)-->(10,13), (9)==(18), (18)==(9), (1)==(10), (10)==(1), (34)-->(35), (4)==(34), (34)==(4) + │ │ │ ├── fd: ()-->(28), (1)-->(2-6,8), (10)-->(11,14), (10)==(20), (20)==(10), (1)==(11), (11)==(1), (37)-->(38), (4)==(37), (37)==(4) │ │ │ ├── inner-join (lookup customer) - │ │ │ │ ├── columns: c_custkey:1!null c_name:2!null c_address:3!null c_nationkey:4!null c_phone:5!null c_acctbal:6!null c_comment:8!null o_orderkey:9!null o_custkey:10!null o_orderdate:13!null l_orderkey:18!null l_extendedprice:23!null l_discount:24!null l_returnflag:26!null - │ │ │ │ ├── key columns: [10] = [1] + │ │ │ │ ├── columns: c_custkey:1!null c_name:2!null c_address:3!null c_nationkey:4!null c_phone:5!null c_acctbal:6!null c_comment:8!null o_orderkey:10!null o_custkey:11!null o_orderdate:14!null l_orderkey:20!null l_extendedprice:25!null l_discount:26!null l_returnflag:28!null + │ │ │ │ ├── key columns: [11] = [1] │ │ │ │ ├── lookup columns are key - │ │ │ │ ├── fd: ()-->(26), (1)-->(2-6,8), (9)-->(10,13), (9)==(18), (18)==(9), (1)==(10), (10)==(1) + │ │ │ │ ├── fd: ()-->(28), (1)-->(2-6,8), (10)-->(11,14), (10)==(20), (20)==(10), (1)==(11), (11)==(1) │ │ │ │ ├── inner-join (lookup orders) - │ │ │ │ │ ├── columns: o_orderkey:9!null o_custkey:10!null o_orderdate:13!null l_orderkey:18!null l_extendedprice:23!null l_discount:24!null l_returnflag:26!null - │ │ │ │ │ ├── key columns: [18] = [9] + │ │ │ │ │ ├── columns: o_orderkey:10!null o_custkey:11!null o_orderdate:14!null l_orderkey:20!null l_extendedprice:25!null l_discount:26!null l_returnflag:28!null + │ │ │ │ │ ├── key columns: [20] = [10] │ │ │ │ │ ├── lookup columns are key - │ │ │ │ │ ├── fd: ()-->(26), (9)-->(10,13), (9)==(18), (18)==(9) + │ │ │ │ │ ├── fd: ()-->(28), (10)-->(11,14), (10)==(20), (20)==(10) │ │ │ │ │ ├── select - │ │ │ │ │ │ ├── columns: l_orderkey:18!null l_extendedprice:23!null l_discount:24!null l_returnflag:26!null - │ │ │ │ │ │ ├── fd: ()-->(26) + │ │ │ │ │ │ ├── columns: l_orderkey:20!null l_extendedprice:25!null l_discount:26!null l_returnflag:28!null + │ │ │ │ │ │ ├── fd: ()-->(28) │ │ │ │ │ │ ├── scan lineitem - │ │ │ │ │ │ │ └── columns: l_orderkey:18!null l_extendedprice:23!null l_discount:24!null l_returnflag:26!null + │ │ │ │ │ │ │ └── columns: l_orderkey:20!null l_extendedprice:25!null l_discount:26!null l_returnflag:28!null │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── l_returnflag:26 = 'R' [outer=(26), constraints=(/26: [/'R' - /'R']; tight), fd=()-->(26)] + │ │ │ │ │ │ └── l_returnflag:28 = 'R' [outer=(28), constraints=(/28: [/'R' - /'R']; tight), fd=()-->(28)] │ │ │ │ │ └── filters - │ │ │ │ │ └── (o_orderdate:13 >= '1993-10-01') AND (o_orderdate:13 < '1994-01-01') [outer=(13), constraints=(/13: [/'1993-10-01' - /'1993-12-31']; tight)] + │ │ │ │ │ └── (o_orderdate:14 >= '1993-10-01') AND (o_orderdate:14 < '1994-01-01') [outer=(14), constraints=(/14: [/'1993-10-01' - /'1993-12-31']; tight)] │ │ │ │ └── filters (true) │ │ │ └── filters (true) │ │ └── projections - │ │ └── l_extendedprice:23 * (1.0 - l_discount:24) [as=column38:38, outer=(23,24), immutable] + │ │ └── l_extendedprice:25 * (1.0 - l_discount:26) [as=column42:42, outer=(25,26), immutable] │ └── aggregations - │ ├── sum [as=sum:39, outer=(38)] - │ │ └── column38:38 + │ ├── sum [as=sum:43, outer=(42)] + │ │ └── column42:42 │ ├── const-agg [as=c_name:2, outer=(2)] │ │ └── c_name:2 │ ├── const-agg [as=c_address:3, outer=(3)] @@ -1139,8 +1139,8 @@ limit │ │ └── c_acctbal:6 │ ├── const-agg [as=c_comment:8, outer=(8)] │ │ └── c_comment:8 - │ └── const-agg [as=n_name:35, outer=(35)] - │ └── n_name:35 + │ └── const-agg [as=n_name:38, outer=(38)] + │ └── n_name:38 └── 20 # -------------------------------------------------- @@ -1183,107 +1183,107 @@ ORDER BY value DESC; ---- sort - ├── columns: ps_partkey:1!null value:18!null + ├── columns: ps_partkey:1!null value:21!null ├── immutable ├── key: (1) - ├── fd: (1)-->(18) - ├── ordering: -18 + ├── fd: (1)-->(21) + ├── ordering: -21 └── select - ├── columns: ps_partkey:1!null sum:18!null + ├── columns: ps_partkey:1!null sum:21!null ├── immutable ├── key: (1) - ├── fd: (1)-->(18) + ├── fd: (1)-->(21) ├── group-by - │ ├── columns: ps_partkey:1!null sum:18!null + │ ├── columns: ps_partkey:1!null sum:21!null │ ├── grouping columns: ps_partkey:1!null │ ├── immutable │ ├── key: (1) - │ ├── fd: (1)-->(18) + │ ├── fd: (1)-->(21) │ ├── project - │ │ ├── columns: column17:17!null ps_partkey:1!null + │ │ ├── columns: column20:20!null ps_partkey:1!null │ │ ├── immutable │ │ ├── inner-join (hash) - │ │ │ ├── columns: ps_partkey:1!null ps_suppkey:2!null ps_availqty:3!null ps_supplycost:4!null s_suppkey:6!null s_nationkey:9!null n_nationkey:13!null n_name:14!null + │ │ │ ├── columns: ps_partkey:1!null ps_suppkey:2!null ps_availqty:3!null ps_supplycost:4!null s_suppkey:7!null s_nationkey:10!null n_nationkey:15!null n_name:16!null │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ ├── key: (1,6) - │ │ │ ├── fd: ()-->(14), (1,2)-->(3,4), (6)-->(9), (9)==(13), (13)==(9), (2)==(6), (6)==(2) + │ │ │ ├── key: (1,7) + │ │ │ ├── fd: ()-->(16), (1,2)-->(3,4), (7)-->(10), (10)==(15), (15)==(10), (2)==(7), (7)==(2) │ │ │ ├── scan partsupp │ │ │ │ ├── columns: ps_partkey:1!null ps_suppkey:2!null ps_availqty:3!null ps_supplycost:4!null │ │ │ │ ├── key: (1,2) │ │ │ │ └── fd: (1,2)-->(3,4) │ │ │ ├── inner-join (lookup supplier@s_nk) - │ │ │ │ ├── columns: s_suppkey:6!null s_nationkey:9!null n_nationkey:13!null n_name:14!null - │ │ │ │ ├── key columns: [13] = [9] - │ │ │ │ ├── key: (6) - │ │ │ │ ├── fd: ()-->(14), (6)-->(9), (9)==(13), (13)==(9) + │ │ │ │ ├── columns: s_suppkey:7!null s_nationkey:10!null n_nationkey:15!null n_name:16!null + │ │ │ │ ├── key columns: [15] = [10] + │ │ │ │ ├── key: (7) + │ │ │ │ ├── fd: ()-->(16), (7)-->(10), (10)==(15), (15)==(10) │ │ │ │ ├── select - │ │ │ │ │ ├── columns: n_nationkey:13!null n_name:14!null - │ │ │ │ │ ├── key: (13) - │ │ │ │ │ ├── fd: ()-->(14) + │ │ │ │ │ ├── columns: n_nationkey:15!null n_name:16!null + │ │ │ │ │ ├── key: (15) + │ │ │ │ │ ├── fd: ()-->(16) │ │ │ │ │ ├── scan nation - │ │ │ │ │ │ ├── columns: n_nationkey:13!null n_name:14!null - │ │ │ │ │ │ ├── key: (13) - │ │ │ │ │ │ └── fd: (13)-->(14) + │ │ │ │ │ │ ├── columns: n_nationkey:15!null n_name:16!null + │ │ │ │ │ │ ├── key: (15) + │ │ │ │ │ │ └── fd: (15)-->(16) │ │ │ │ │ └── filters - │ │ │ │ │ └── n_name:14 = 'GERMANY' [outer=(14), constraints=(/14: [/'GERMANY' - /'GERMANY']; tight), fd=()-->(14)] + │ │ │ │ │ └── n_name:16 = 'GERMANY' [outer=(16), constraints=(/16: [/'GERMANY' - /'GERMANY']; tight), fd=()-->(16)] │ │ │ │ └── filters (true) │ │ │ └── filters - │ │ │ └── ps_suppkey:2 = s_suppkey:6 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + │ │ │ └── ps_suppkey:2 = s_suppkey:7 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] │ │ └── projections - │ │ └── ps_supplycost:4 * ps_availqty:3::FLOAT8 [as=column17:17, outer=(3,4), immutable] + │ │ └── ps_supplycost:4 * ps_availqty:3::FLOAT8 [as=column20:20, outer=(3,4), immutable] │ └── aggregations - │ └── sum [as=sum:18, outer=(17)] - │ └── column17:17 + │ └── sum [as=sum:21, outer=(20)] + │ └── column20:20 └── filters - └── gt [outer=(18), immutable, subquery, constraints=(/18: (/NULL - ])] - ├── sum:18 + └── gt [outer=(21), immutable, subquery, constraints=(/21: (/NULL - ])] + ├── sum:21 └── subquery └── project - ├── columns: "?column?":37 + ├── columns: "?column?":43 ├── cardinality: [1 - 1] ├── immutable ├── key: () - ├── fd: ()-->(37) + ├── fd: ()-->(43) ├── scalar-group-by - │ ├── columns: sum:36 + │ ├── columns: sum:42 │ ├── cardinality: [1 - 1] │ ├── immutable │ ├── key: () - │ ├── fd: ()-->(36) + │ ├── fd: ()-->(42) │ ├── project - │ │ ├── columns: column35:35!null + │ │ ├── columns: column41:41!null │ │ ├── immutable │ │ ├── inner-join (hash) - │ │ │ ├── columns: ps_suppkey:20!null ps_availqty:21!null ps_supplycost:22!null s_suppkey:24!null s_nationkey:27!null n_nationkey:31!null n_name:32!null + │ │ │ ├── columns: ps_suppkey:23!null ps_availqty:24!null ps_supplycost:25!null s_suppkey:28!null s_nationkey:31!null n_nationkey:36!null n_name:37!null │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ ├── fd: ()-->(32), (24)-->(27), (27)==(31), (31)==(27), (20)==(24), (24)==(20) + │ │ │ ├── fd: ()-->(37), (28)-->(31), (31)==(36), (36)==(31), (23)==(28), (28)==(23) │ │ │ ├── scan partsupp - │ │ │ │ └── columns: ps_suppkey:20!null ps_availqty:21!null ps_supplycost:22!null + │ │ │ │ └── columns: ps_suppkey:23!null ps_availqty:24!null ps_supplycost:25!null │ │ │ ├── inner-join (lookup supplier@s_nk) - │ │ │ │ ├── columns: s_suppkey:24!null s_nationkey:27!null n_nationkey:31!null n_name:32!null - │ │ │ │ ├── key columns: [31] = [27] - │ │ │ │ ├── key: (24) - │ │ │ │ ├── fd: ()-->(32), (24)-->(27), (27)==(31), (31)==(27) + │ │ │ │ ├── columns: s_suppkey:28!null s_nationkey:31!null n_nationkey:36!null n_name:37!null + │ │ │ │ ├── key columns: [36] = [31] + │ │ │ │ ├── key: (28) + │ │ │ │ ├── fd: ()-->(37), (28)-->(31), (31)==(36), (36)==(31) │ │ │ │ ├── select - │ │ │ │ │ ├── columns: n_nationkey:31!null n_name:32!null - │ │ │ │ │ ├── key: (31) - │ │ │ │ │ ├── fd: ()-->(32) + │ │ │ │ │ ├── columns: n_nationkey:36!null n_name:37!null + │ │ │ │ │ ├── key: (36) + │ │ │ │ │ ├── fd: ()-->(37) │ │ │ │ │ ├── scan nation - │ │ │ │ │ │ ├── columns: n_nationkey:31!null n_name:32!null - │ │ │ │ │ │ ├── key: (31) - │ │ │ │ │ │ └── fd: (31)-->(32) + │ │ │ │ │ │ ├── columns: n_nationkey:36!null n_name:37!null + │ │ │ │ │ │ ├── key: (36) + │ │ │ │ │ │ └── fd: (36)-->(37) │ │ │ │ │ └── filters - │ │ │ │ │ └── n_name:32 = 'GERMANY' [outer=(32), constraints=(/32: [/'GERMANY' - /'GERMANY']; tight), fd=()-->(32)] + │ │ │ │ │ └── n_name:37 = 'GERMANY' [outer=(37), constraints=(/37: [/'GERMANY' - /'GERMANY']; tight), fd=()-->(37)] │ │ │ │ └── filters (true) │ │ │ └── filters - │ │ │ └── ps_suppkey:20 = s_suppkey:24 [outer=(20,24), constraints=(/20: (/NULL - ]; /24: (/NULL - ]), fd=(20)==(24), (24)==(20)] + │ │ │ └── ps_suppkey:23 = s_suppkey:28 [outer=(23,28), constraints=(/23: (/NULL - ]; /28: (/NULL - ]), fd=(23)==(28), (28)==(23)] │ │ └── projections - │ │ └── ps_supplycost:22 * ps_availqty:21::FLOAT8 [as=column35:35, outer=(21,22), immutable] + │ │ └── ps_supplycost:25 * ps_availqty:24::FLOAT8 [as=column41:41, outer=(24,25), immutable] │ └── aggregations - │ └── sum [as=sum:36, outer=(35)] - │ └── column35:35 + │ └── sum [as=sum:42, outer=(41)] + │ └── column41:41 └── projections - └── sum:36 * 0.0001 [as="?column?":37, outer=(36), immutable] + └── sum:42 * 0.0001 [as="?column?":43, outer=(42), immutable] # -------------------------------------------------- # Q12 @@ -1330,40 +1330,40 @@ ORDER BY l_shipmode; ---- sort - ├── columns: l_shipmode:24!null high_line_count:27!null low_line_count:29!null - ├── key: (24) - ├── fd: (24)-->(27,29) - ├── ordering: +24 + ├── columns: l_shipmode:25!null high_line_count:29!null low_line_count:31!null + ├── key: (25) + ├── fd: (25)-->(29,31) + ├── ordering: +25 └── group-by - ├── columns: l_shipmode:24!null sum:27!null sum:29!null - ├── grouping columns: l_shipmode:24!null - ├── key: (24) - ├── fd: (24)-->(27,29) + ├── columns: l_shipmode:25!null sum:29!null sum:31!null + ├── grouping columns: l_shipmode:25!null + ├── key: (25) + ├── fd: (25)-->(29,31) ├── project - │ ├── columns: column26:26!null column28:28!null l_shipmode:24!null + │ ├── columns: column28:28!null column30:30!null l_shipmode:25!null │ ├── inner-join (lookup orders) - │ │ ├── columns: o_orderkey:1!null o_orderpriority:6!null l_orderkey:10!null l_shipdate:20!null l_commitdate:21!null l_receiptdate:22!null l_shipmode:24!null - │ │ ├── key columns: [10] = [1] + │ │ ├── columns: o_orderkey:1!null o_orderpriority:6!null l_orderkey:11!null l_shipdate:21!null l_commitdate:22!null l_receiptdate:23!null l_shipmode:25!null + │ │ ├── key columns: [11] = [1] │ │ ├── lookup columns are key - │ │ ├── fd: (1)-->(6), (1)==(10), (10)==(1) + │ │ ├── fd: (1)-->(6), (1)==(11), (11)==(1) │ │ ├── select - │ │ │ ├── columns: l_orderkey:10!null l_shipdate:20!null l_commitdate:21!null l_receiptdate:22!null l_shipmode:24!null + │ │ │ ├── columns: l_orderkey:11!null l_shipdate:21!null l_commitdate:22!null l_receiptdate:23!null l_shipmode:25!null │ │ │ ├── scan lineitem - │ │ │ │ └── columns: l_orderkey:10!null l_shipdate:20!null l_commitdate:21!null l_receiptdate:22!null l_shipmode:24!null + │ │ │ │ └── columns: l_orderkey:11!null l_shipdate:21!null l_commitdate:22!null l_receiptdate:23!null l_shipmode:25!null │ │ │ └── filters - │ │ │ ├── (l_receiptdate:22 >= '1994-01-01') AND (l_receiptdate:22 < '1995-01-01') [outer=(22), constraints=(/22: [/'1994-01-01' - /'1994-12-31']; tight)] - │ │ │ ├── l_shipmode:24 IN ('MAIL', 'SHIP') [outer=(24), constraints=(/24: [/'MAIL' - /'MAIL'] [/'SHIP' - /'SHIP']; tight)] - │ │ │ ├── l_commitdate:21 < l_receiptdate:22 [outer=(21,22), constraints=(/21: (/NULL - ]; /22: (/NULL - ])] - │ │ │ └── l_shipdate:20 < l_commitdate:21 [outer=(20,21), constraints=(/20: (/NULL - ]; /21: (/NULL - ])] + │ │ │ ├── (l_receiptdate:23 >= '1994-01-01') AND (l_receiptdate:23 < '1995-01-01') [outer=(23), constraints=(/23: [/'1994-01-01' - /'1994-12-31']; tight)] + │ │ │ ├── l_shipmode:25 IN ('MAIL', 'SHIP') [outer=(25), constraints=(/25: [/'MAIL' - /'MAIL'] [/'SHIP' - /'SHIP']; tight)] + │ │ │ ├── l_commitdate:22 < l_receiptdate:23 [outer=(22,23), constraints=(/22: (/NULL - ]; /23: (/NULL - ])] + │ │ │ └── l_shipdate:21 < l_commitdate:22 [outer=(21,22), constraints=(/21: (/NULL - ]; /22: (/NULL - ])] │ │ └── filters (true) │ └── projections - │ ├── CASE WHEN (o_orderpriority:6 = '1-URGENT') OR (o_orderpriority:6 = '2-HIGH') THEN 1 ELSE 0 END [as=column26:26, outer=(6)] - │ └── CASE WHEN (o_orderpriority:6 != '1-URGENT') AND (o_orderpriority:6 != '2-HIGH') THEN 1 ELSE 0 END [as=column28:28, outer=(6)] + │ ├── CASE WHEN (o_orderpriority:6 = '1-URGENT') OR (o_orderpriority:6 = '2-HIGH') THEN 1 ELSE 0 END [as=column28:28, outer=(6)] + │ └── CASE WHEN (o_orderpriority:6 != '1-URGENT') AND (o_orderpriority:6 != '2-HIGH') THEN 1 ELSE 0 END [as=column30:30, outer=(6)] └── aggregations - ├── sum [as=sum:27, outer=(26)] - │ └── column26:26 - └── sum [as=sum:29, outer=(28)] - └── column28:28 + ├── sum [as=sum:29, outer=(28)] + │ └── column28:28 + └── sum [as=sum:31, outer=(30)] + └── column30:30 # -------------------------------------------------- # Q13 @@ -1398,45 +1398,45 @@ ORDER BY c_count DESC; ---- sort - ├── columns: c_count:18!null custdist:19!null - ├── key: (18) - ├── fd: (18)-->(19) - ├── ordering: -19,-18 + ├── columns: c_count:20!null custdist:21!null + ├── key: (20) + ├── fd: (20)-->(21) + ├── ordering: -21,-20 └── group-by - ├── columns: count:18!null count_rows:19!null - ├── grouping columns: count:18!null - ├── key: (18) - ├── fd: (18)-->(19) + ├── columns: count:20!null count_rows:21!null + ├── grouping columns: count:20!null + ├── key: (20) + ├── fd: (20)-->(21) ├── group-by - │ ├── columns: c_custkey:1!null count:18!null + │ ├── columns: c_custkey:1!null count:20!null │ ├── grouping columns: c_custkey:1!null │ ├── key: (1) - │ ├── fd: (1)-->(18) + │ ├── fd: (1)-->(20) │ ├── left-join (hash) - │ │ ├── columns: c_custkey:1!null o_orderkey:9 o_custkey:10 o_comment:17 + │ │ ├── columns: c_custkey:1!null o_orderkey:10 o_custkey:11 o_comment:18 │ │ ├── multiplicity: left-rows(one-or-more), right-rows(exactly-one) - │ │ ├── key: (1,9) - │ │ ├── fd: (9)-->(10,17) + │ │ ├── key: (1,10) + │ │ ├── fd: (10)-->(11,18) │ │ ├── scan customer@c_nk │ │ │ ├── columns: c_custkey:1!null │ │ │ └── key: (1) │ │ ├── select - │ │ │ ├── columns: o_orderkey:9!null o_custkey:10!null o_comment:17!null - │ │ │ ├── key: (9) - │ │ │ ├── fd: (9)-->(10,17) + │ │ │ ├── columns: o_orderkey:10!null o_custkey:11!null o_comment:18!null + │ │ │ ├── key: (10) + │ │ │ ├── fd: (10)-->(11,18) │ │ │ ├── scan orders - │ │ │ │ ├── columns: o_orderkey:9!null o_custkey:10!null o_comment:17!null - │ │ │ │ ├── key: (9) - │ │ │ │ └── fd: (9)-->(10,17) + │ │ │ │ ├── columns: o_orderkey:10!null o_custkey:11!null o_comment:18!null + │ │ │ │ ├── key: (10) + │ │ │ │ └── fd: (10)-->(11,18) │ │ │ └── filters - │ │ │ └── o_comment:17 NOT LIKE '%special%requests%' [outer=(17), constraints=(/17: (/NULL - ])] + │ │ │ └── o_comment:18 NOT LIKE '%special%requests%' [outer=(18), constraints=(/18: (/NULL - ])] │ │ └── filters - │ │ └── c_custkey:1 = o_custkey:10 [outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] + │ │ └── c_custkey:1 = o_custkey:11 [outer=(1,11), constraints=(/1: (/NULL - ]; /11: (/NULL - ]), fd=(1)==(11), (11)==(1)] │ └── aggregations - │ └── count [as=count:18, outer=(9)] - │ └── o_orderkey:9 + │ └── count [as=count:20, outer=(10)] + │ └── o_orderkey:10 └── aggregations - └── count-rows [as=count_rows:19] + └── count-rows [as=count_rows:21] # -------------------------------------------------- # Q14 @@ -1465,28 +1465,28 @@ WHERE AND l_shipdate < DATE '1995-09-01' + INTERVAL '1' MONTH; ---- project - ├── columns: promo_revenue:30 + ├── columns: promo_revenue:32 ├── cardinality: [1 - 1] ├── immutable ├── key: () - ├── fd: ()-->(30) + ├── fd: ()-->(32) ├── scalar-group-by - │ ├── columns: sum:27 sum:29 + │ ├── columns: sum:29 sum:31 │ ├── cardinality: [1 - 1] │ ├── immutable │ ├── key: () - │ ├── fd: ()-->(27,29) + │ ├── fd: ()-->(29,31) │ ├── project - │ │ ├── columns: column26:26!null column28:28!null + │ │ ├── columns: column28:28!null column30:30!null │ │ ├── immutable │ │ ├── inner-join (hash) - │ │ │ ├── columns: l_partkey:2!null l_extendedprice:6!null l_discount:7!null l_shipdate:11!null p_partkey:17!null p_type:21!null + │ │ │ ├── columns: l_partkey:2!null l_extendedprice:6!null l_discount:7!null l_shipdate:11!null p_partkey:18!null p_type:22!null │ │ │ ├── multiplicity: left-rows(zero-or-more), right-rows(exactly-one) - │ │ │ ├── fd: (17)-->(21), (2)==(17), (17)==(2) + │ │ │ ├── fd: (18)-->(22), (2)==(18), (18)==(2) │ │ │ ├── scan part - │ │ │ │ ├── columns: p_partkey:17!null p_type:21!null - │ │ │ │ ├── key: (17) - │ │ │ │ └── fd: (17)-->(21) + │ │ │ │ ├── columns: p_partkey:18!null p_type:22!null + │ │ │ │ ├── key: (18) + │ │ │ │ └── fd: (18)-->(22) │ │ │ ├── select │ │ │ │ ├── columns: l_partkey:2!null l_extendedprice:6!null l_discount:7!null l_shipdate:11!null │ │ │ │ ├── scan lineitem @@ -1494,17 +1494,17 @@ project │ │ │ │ └── filters │ │ │ │ └── (l_shipdate:11 >= '1995-09-01') AND (l_shipdate:11 < '1995-10-01') [outer=(11), constraints=(/11: [/'1995-09-01' - /'1995-09-30']; tight)] │ │ │ └── filters - │ │ │ └── l_partkey:2 = p_partkey:17 [outer=(2,17), constraints=(/2: (/NULL - ]; /17: (/NULL - ]), fd=(2)==(17), (17)==(2)] + │ │ │ └── l_partkey:2 = p_partkey:18 [outer=(2,18), constraints=(/2: (/NULL - ]; /18: (/NULL - ]), fd=(2)==(18), (18)==(2)] │ │ └── projections - │ │ ├── CASE WHEN p_type:21 LIKE 'PROMO%' THEN l_extendedprice:6 * (1.0 - l_discount:7) ELSE 0.0 END [as=column26:26, outer=(6,7,21), immutable] - │ │ └── l_extendedprice:6 * (1.0 - l_discount:7) [as=column28:28, outer=(6,7), immutable] + │ │ ├── CASE WHEN p_type:22 LIKE 'PROMO%' THEN l_extendedprice:6 * (1.0 - l_discount:7) ELSE 0.0 END [as=column28:28, outer=(6,7,22), immutable] + │ │ └── l_extendedprice:6 * (1.0 - l_discount:7) [as=column30:30, outer=(6,7), immutable] │ └── aggregations - │ ├── sum [as=sum:27, outer=(26)] - │ │ └── column26:26 - │ └── sum [as=sum:29, outer=(28)] - │ └── column28:28 + │ ├── sum [as=sum:29, outer=(28)] + │ │ └── column28:28 + │ └── sum [as=sum:31, outer=(30)] + │ └── column30:30 └── projections - └── (sum:27 * 100.0) / sum:29 [as=promo_revenue:30, outer=(27,29), immutable] + └── (sum:29 * 100.0) / sum:31 [as=promo_revenue:32, outer=(29,31), immutable] # -------------------------------------------------- # Q15 @@ -1553,81 +1553,81 @@ ORDER BY s_suppkey; ---- sort - ├── columns: s_suppkey:1!null s_name:2!null s_address:3!null s_phone:5!null total_revenue:25!null + ├── columns: s_suppkey:1!null s_name:2!null s_address:3!null s_phone:5!null total_revenue:27!null ├── immutable ├── key: (1) - ├── fd: (1)-->(2,3,5,25) + ├── fd: (1)-->(2,3,5,27) ├── ordering: +1 └── project - ├── columns: s_suppkey:1!null s_name:2!null s_address:3!null s_phone:5!null sum:25!null + ├── columns: s_suppkey:1!null s_name:2!null s_address:3!null s_phone:5!null sum:27!null ├── immutable ├── key: (1) - ├── fd: (1)-->(2,3,5,25) + ├── fd: (1)-->(2,3,5,27) └── inner-join (lookup supplier) - ├── columns: s_suppkey:1!null s_name:2!null s_address:3!null s_phone:5!null l_suppkey:10!null sum:25!null - ├── key columns: [10] = [1] + ├── columns: s_suppkey:1!null s_name:2!null s_address:3!null s_phone:5!null l_suppkey:11!null sum:27!null + ├── key columns: [11] = [1] ├── lookup columns are key ├── immutable - ├── key: (10) - ├── fd: (1)-->(2,3,5), (10)-->(25), (1)==(10), (10)==(1) + ├── key: (11) + ├── fd: (1)-->(2,3,5), (11)-->(27), (1)==(11), (11)==(1) ├── select - │ ├── columns: l_suppkey:10!null sum:25!null + │ ├── columns: l_suppkey:11!null sum:27!null │ ├── immutable - │ ├── key: (10) - │ ├── fd: (10)-->(25) + │ ├── key: (11) + │ ├── fd: (11)-->(27) │ ├── group-by - │ │ ├── columns: l_suppkey:10!null sum:25!null - │ │ ├── grouping columns: l_suppkey:10!null + │ │ ├── columns: l_suppkey:11!null sum:27!null + │ │ ├── grouping columns: l_suppkey:11!null │ │ ├── immutable - │ │ ├── key: (10) - │ │ ├── fd: (10)-->(25) + │ │ ├── key: (11) + │ │ ├── fd: (11)-->(27) │ │ ├── project - │ │ │ ├── columns: column24:24!null l_suppkey:10!null + │ │ │ ├── columns: column26:26!null l_suppkey:11!null │ │ │ ├── immutable │ │ │ ├── select - │ │ │ │ ├── columns: l_suppkey:10!null l_extendedprice:13!null l_discount:14!null l_shipdate:18!null + │ │ │ │ ├── columns: l_suppkey:11!null l_extendedprice:14!null l_discount:15!null l_shipdate:19!null │ │ │ │ ├── scan lineitem - │ │ │ │ │ └── columns: l_suppkey:10!null l_extendedprice:13!null l_discount:14!null l_shipdate:18!null + │ │ │ │ │ └── columns: l_suppkey:11!null l_extendedprice:14!null l_discount:15!null l_shipdate:19!null │ │ │ │ └── filters - │ │ │ │ └── (l_shipdate:18 >= '1996-01-01') AND (l_shipdate:18 < '1996-04-01') [outer=(18), constraints=(/18: [/'1996-01-01' - /'1996-03-31']; tight)] + │ │ │ │ └── (l_shipdate:19 >= '1996-01-01') AND (l_shipdate:19 < '1996-04-01') [outer=(19), constraints=(/19: [/'1996-01-01' - /'1996-03-31']; tight)] │ │ │ └── projections - │ │ │ └── l_extendedprice:13 * (1.0 - l_discount:14) [as=column24:24, outer=(13,14), immutable] + │ │ │ └── l_extendedprice:14 * (1.0 - l_discount:15) [as=column26:26, outer=(14,15), immutable] │ │ └── aggregations - │ │ └── sum [as=sum:25, outer=(24)] - │ │ └── column24:24 + │ │ └── sum [as=sum:27, outer=(26)] + │ │ └── column26:26 │ └── filters - │ └── eq [outer=(25), immutable, subquery, constraints=(/25: (/NULL - ])] - │ ├── sum:25 + │ └── eq [outer=(27), immutable, subquery, constraints=(/27: (/NULL - ])] + │ ├── sum:27 │ └── subquery │ └── scalar-group-by - │ ├── columns: max:44 + │ ├── columns: max:47 │ ├── cardinality: [1 - 1] │ ├── immutable │ ├── key: () - │ ├── fd: ()-->(44) + │ ├── fd: ()-->(47) │ ├── group-by - │ │ ├── columns: l_suppkey:28!null sum:43!null - │ │ ├── grouping columns: l_suppkey:28!null + │ │ ├── columns: l_suppkey:30!null sum:46!null + │ │ ├── grouping columns: l_suppkey:30!null │ │ ├── immutable - │ │ ├── key: (28) - │ │ ├── fd: (28)-->(43) + │ │ ├── key: (30) + │ │ ├── fd: (30)-->(46) │ │ ├── project - │ │ │ ├── columns: column42:42!null l_suppkey:28!null + │ │ │ ├── columns: column45:45!null l_suppkey:30!null │ │ │ ├── immutable │ │ │ ├── select - │ │ │ │ ├── columns: l_suppkey:28!null l_extendedprice:31!null l_discount:32!null l_shipdate:36!null + │ │ │ │ ├── columns: l_suppkey:30!null l_extendedprice:33!null l_discount:34!null l_shipdate:38!null │ │ │ │ ├── scan lineitem - │ │ │ │ │ └── columns: l_suppkey:28!null l_extendedprice:31!null l_discount:32!null l_shipdate:36!null + │ │ │ │ │ └── columns: l_suppkey:30!null l_extendedprice:33!null l_discount:34!null l_shipdate:38!null │ │ │ │ └── filters - │ │ │ │ └── (l_shipdate:36 >= '1996-01-01') AND (l_shipdate:36 < '1996-04-01') [outer=(36), constraints=(/36: [/'1996-01-01' - /'1996-03-31']; tight)] + │ │ │ │ └── (l_shipdate:38 >= '1996-01-01') AND (l_shipdate:38 < '1996-04-01') [outer=(38), constraints=(/38: [/'1996-01-01' - /'1996-03-31']; tight)] │ │ │ └── projections - │ │ │ └── l_extendedprice:31 * (1.0 - l_discount:32) [as=column42:42, outer=(31,32), immutable] + │ │ │ └── l_extendedprice:33 * (1.0 - l_discount:34) [as=column45:45, outer=(33,34), immutable] │ │ └── aggregations - │ │ └── sum [as=sum:43, outer=(42)] - │ │ └── column42:42 + │ │ └── sum [as=sum:46, outer=(45)] + │ │ └── column45:45 │ └── aggregations - │ └── max [as=max:44, outer=(43)] - │ └── sum:43 + │ └── max [as=max:47, outer=(46)] + │ └── sum:46 └── filters (true) # -------------------------------------------------- @@ -1677,53 +1677,53 @@ ORDER BY p_size; ---- sort - ├── columns: p_brand:9!null p_type:10!null p_size:11!null supplier_cnt:22!null - ├── key: (9-11) - ├── fd: (9-11)-->(22) - ├── ordering: -22,+9,+10,+11 + ├── columns: p_brand:10!null p_type:11!null p_size:12!null supplier_cnt:25!null + ├── key: (10-12) + ├── fd: (10-12)-->(25) + ├── ordering: -25,+10,+11,+12 └── group-by - ├── columns: p_brand:9!null p_type:10!null p_size:11!null count:22!null - ├── grouping columns: p_brand:9!null p_type:10!null p_size:11!null - ├── key: (9-11) - ├── fd: (9-11)-->(22) + ├── columns: p_brand:10!null p_type:11!null p_size:12!null count:25!null + ├── grouping columns: p_brand:10!null p_type:11!null p_size:12!null + ├── key: (10-12) + ├── fd: (10-12)-->(25) ├── distinct-on - │ ├── columns: ps_suppkey:2!null p_brand:9!null p_type:10!null p_size:11!null - │ ├── grouping columns: ps_suppkey:2!null p_brand:9!null p_type:10!null p_size:11!null - │ ├── key: (2,9-11) + │ ├── columns: ps_suppkey:2!null p_brand:10!null p_type:11!null p_size:12!null + │ ├── grouping columns: ps_suppkey:2!null p_brand:10!null p_type:11!null p_size:12!null + │ ├── key: (2,10-12) │ └── inner-join (lookup part) - │ ├── columns: ps_partkey:1!null ps_suppkey:2!null p_partkey:6!null p_brand:9!null p_type:10!null p_size:11!null - │ ├── key columns: [1] = [6] + │ ├── columns: ps_partkey:1!null ps_suppkey:2!null p_partkey:7!null p_brand:10!null p_type:11!null p_size:12!null + │ ├── key columns: [1] = [7] │ ├── lookup columns are key - │ ├── key: (2,6) - │ ├── fd: (6)-->(9-11), (1)==(6), (6)==(1) + │ ├── key: (2,7) + │ ├── fd: (7)-->(10-12), (1)==(7), (7)==(1) │ ├── anti-join (merge) │ │ ├── columns: ps_partkey:1!null ps_suppkey:2!null │ │ ├── left ordering: +2 - │ │ ├── right ordering: +15 + │ │ ├── right ordering: +17 │ │ ├── key: (1,2) │ │ ├── scan partsupp@ps_sk │ │ │ ├── columns: ps_partkey:1!null ps_suppkey:2!null │ │ │ ├── key: (1,2) │ │ │ └── ordering: +2 │ │ ├── select - │ │ │ ├── columns: s_suppkey:15!null s_comment:21!null - │ │ │ ├── key: (15) - │ │ │ ├── fd: (15)-->(21) - │ │ │ ├── ordering: +15 + │ │ │ ├── columns: s_suppkey:17!null s_comment:23!null + │ │ │ ├── key: (17) + │ │ │ ├── fd: (17)-->(23) + │ │ │ ├── ordering: +17 │ │ │ ├── scan supplier - │ │ │ │ ├── columns: s_suppkey:15!null s_comment:21!null - │ │ │ │ ├── key: (15) - │ │ │ │ ├── fd: (15)-->(21) - │ │ │ │ └── ordering: +15 + │ │ │ │ ├── columns: s_suppkey:17!null s_comment:23!null + │ │ │ │ ├── key: (17) + │ │ │ │ ├── fd: (17)-->(23) + │ │ │ │ └── ordering: +17 │ │ │ └── filters - │ │ │ └── s_comment:21 LIKE '%Customer%Complaints%' [outer=(21), constraints=(/21: (/NULL - ])] + │ │ │ └── s_comment:23 LIKE '%Customer%Complaints%' [outer=(23), constraints=(/23: (/NULL - ])] │ │ └── filters (true) │ └── filters - │ ├── p_brand:9 != 'Brand#45' [outer=(9), constraints=(/9: (/NULL - /'Brand#45') [/e'Brand#45\x00' - ]; tight)] - │ ├── p_type:10 NOT LIKE 'MEDIUM POLISHED %' [outer=(10), constraints=(/10: (/NULL - ])] - │ └── p_size:11 IN (3, 9, 14, 19, 23, 36, 45, 49) [outer=(11), constraints=(/11: [/3 - /3] [/9 - /9] [/14 - /14] [/19 - /19] [/23 - /23] [/36 - /36] [/45 - /45] [/49 - /49]; tight)] + │ ├── p_brand:10 != 'Brand#45' [outer=(10), constraints=(/10: (/NULL - /'Brand#45') [/e'Brand#45\x00' - ]; tight)] + │ ├── p_type:11 NOT LIKE 'MEDIUM POLISHED %' [outer=(11), constraints=(/11: (/NULL - ])] + │ └── p_size:12 IN (3, 9, 14, 19, 23, 36, 45, 49) [outer=(12), constraints=(/12: [/3 - /3] [/9 - /9] [/14 - /14] [/19 - /19] [/23 - /23] [/36 - /36] [/45 - /45] [/49 - /49]; tight)] └── aggregations - └── count-rows [as=count:22] + └── count-rows [as=count:25] # -------------------------------------------------- # Q17 @@ -1761,80 +1761,80 @@ WHERE ); ---- project - ├── columns: avg_yearly:45 + ├── columns: avg_yearly:48 ├── cardinality: [1 - 1] ├── immutable ├── key: () - ├── fd: ()-->(45) + ├── fd: ()-->(48) ├── scalar-group-by - │ ├── columns: sum:44 + │ ├── columns: sum:47 │ ├── cardinality: [1 - 1] │ ├── immutable │ ├── key: () - │ ├── fd: ()-->(44) + │ ├── fd: ()-->(47) │ ├── inner-join (lookup lineitem) - │ │ ├── columns: l_partkey:2!null l_quantity:5!null l_extendedprice:6!null p_partkey:17!null "?column?":43!null + │ │ ├── columns: l_partkey:2!null l_quantity:5!null l_extendedprice:6!null p_partkey:18!null "?column?":46!null │ │ ├── key columns: [1 4] = [1 4] │ │ ├── lookup columns are key │ │ ├── immutable - │ │ ├── fd: (17)-->(43), (2)==(17), (17)==(2) + │ │ ├── fd: (18)-->(46), (2)==(18), (18)==(2) │ │ ├── inner-join (lookup lineitem@l_pk) - │ │ │ ├── columns: l_orderkey:1!null l_partkey:2!null l_linenumber:4!null p_partkey:17!null "?column?":43 - │ │ │ ├── key columns: [17] = [2] + │ │ │ ├── columns: l_orderkey:1!null l_partkey:2!null l_linenumber:4!null p_partkey:18!null "?column?":46 + │ │ │ ├── key columns: [18] = [2] │ │ │ ├── immutable │ │ │ ├── key: (1,4) - │ │ │ ├── fd: (17)-->(43), (1,4)-->(2), (2)==(17), (17)==(2) + │ │ │ ├── fd: (18)-->(46), (1,4)-->(2), (2)==(18), (18)==(2) │ │ │ ├── project - │ │ │ │ ├── columns: "?column?":43 p_partkey:17!null + │ │ │ │ ├── columns: "?column?":46 p_partkey:18!null │ │ │ │ ├── immutable - │ │ │ │ ├── key: (17) - │ │ │ │ ├── fd: (17)-->(43) + │ │ │ │ ├── key: (18) + │ │ │ │ ├── fd: (18)-->(46) │ │ │ │ ├── group-by - │ │ │ │ │ ├── columns: p_partkey:17!null avg:42 - │ │ │ │ │ ├── grouping columns: p_partkey:17!null - │ │ │ │ │ ├── internal-ordering: +17 opt(20,23) - │ │ │ │ │ ├── key: (17) - │ │ │ │ │ ├── fd: (17)-->(42) + │ │ │ │ │ ├── columns: p_partkey:18!null avg:45 + │ │ │ │ │ ├── grouping columns: p_partkey:18!null + │ │ │ │ │ ├── internal-ordering: +18 opt(21,24) + │ │ │ │ │ ├── key: (18) + │ │ │ │ │ ├── fd: (18)-->(45) │ │ │ │ │ ├── left-join (lookup lineitem) - │ │ │ │ │ │ ├── columns: p_partkey:17!null p_brand:20!null p_container:23!null l_partkey:27 l_quantity:30 - │ │ │ │ │ │ ├── key columns: [26 29] = [26 29] + │ │ │ │ │ │ ├── columns: p_partkey:18!null p_brand:21!null p_container:24!null l_partkey:29 l_quantity:32 + │ │ │ │ │ │ ├── key columns: [28 31] = [28 31] │ │ │ │ │ │ ├── lookup columns are key - │ │ │ │ │ │ ├── fd: ()-->(20,23) - │ │ │ │ │ │ ├── ordering: +17 opt(20,23) [actual: +17] + │ │ │ │ │ │ ├── fd: ()-->(21,24) + │ │ │ │ │ │ ├── ordering: +18 opt(21,24) [actual: +18] │ │ │ │ │ │ ├── left-join (lookup lineitem@l_pk) - │ │ │ │ │ │ │ ├── columns: p_partkey:17!null p_brand:20!null p_container:23!null l_orderkey:26 l_partkey:27 l_linenumber:29 - │ │ │ │ │ │ │ ├── key columns: [17] = [27] - │ │ │ │ │ │ │ ├── key: (17,26,29) - │ │ │ │ │ │ │ ├── fd: ()-->(20,23), (26,29)-->(27) - │ │ │ │ │ │ │ ├── ordering: +17 opt(20,23) [actual: +17] + │ │ │ │ │ │ │ ├── columns: p_partkey:18!null p_brand:21!null p_container:24!null l_orderkey:28 l_partkey:29 l_linenumber:31 + │ │ │ │ │ │ │ ├── key columns: [18] = [29] + │ │ │ │ │ │ │ ├── key: (18,28,31) + │ │ │ │ │ │ │ ├── fd: ()-->(21,24), (28,31)-->(29) + │ │ │ │ │ │ │ ├── ordering: +18 opt(21,24) [actual: +18] │ │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ │ ├── columns: p_partkey:17!null p_brand:20!null p_container:23!null - │ │ │ │ │ │ │ │ ├── key: (17) - │ │ │ │ │ │ │ │ ├── fd: ()-->(20,23) - │ │ │ │ │ │ │ │ ├── ordering: +17 opt(20,23) [actual: +17] + │ │ │ │ │ │ │ │ ├── columns: p_partkey:18!null p_brand:21!null p_container:24!null + │ │ │ │ │ │ │ │ ├── key: (18) + │ │ │ │ │ │ │ │ ├── fd: ()-->(21,24) + │ │ │ │ │ │ │ │ ├── ordering: +18 opt(21,24) [actual: +18] │ │ │ │ │ │ │ │ ├── scan part - │ │ │ │ │ │ │ │ │ ├── columns: p_partkey:17!null p_brand:20!null p_container:23!null - │ │ │ │ │ │ │ │ │ ├── key: (17) - │ │ │ │ │ │ │ │ │ ├── fd: (17)-->(20,23) - │ │ │ │ │ │ │ │ │ └── ordering: +17 opt(20,23) [actual: +17] + │ │ │ │ │ │ │ │ │ ├── columns: p_partkey:18!null p_brand:21!null p_container:24!null + │ │ │ │ │ │ │ │ │ ├── key: (18) + │ │ │ │ │ │ │ │ │ ├── fd: (18)-->(21,24) + │ │ │ │ │ │ │ │ │ └── ordering: +18 opt(21,24) [actual: +18] │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ ├── p_brand:20 = 'Brand#23' [outer=(20), constraints=(/20: [/'Brand#23' - /'Brand#23']; tight), fd=()-->(20)] - │ │ │ │ │ │ │ │ └── p_container:23 = 'MED BOX' [outer=(23), constraints=(/23: [/'MED BOX' - /'MED BOX']; tight), fd=()-->(23)] + │ │ │ │ │ │ │ │ ├── p_brand:21 = 'Brand#23' [outer=(21), constraints=(/21: [/'Brand#23' - /'Brand#23']; tight), fd=()-->(21)] + │ │ │ │ │ │ │ │ └── p_container:24 = 'MED BOX' [outer=(24), constraints=(/24: [/'MED BOX' - /'MED BOX']; tight), fd=()-->(24)] │ │ │ │ │ │ │ └── filters (true) │ │ │ │ │ │ └── filters (true) │ │ │ │ │ └── aggregations - │ │ │ │ │ └── avg [as=avg:42, outer=(30)] - │ │ │ │ │ └── l_quantity:30 + │ │ │ │ │ └── avg [as=avg:45, outer=(32)] + │ │ │ │ │ └── l_quantity:32 │ │ │ │ └── projections - │ │ │ │ └── avg:42 * 0.2 [as="?column?":43, outer=(42), immutable] + │ │ │ │ └── avg:45 * 0.2 [as="?column?":46, outer=(45), immutable] │ │ │ └── filters (true) │ │ └── filters - │ │ └── l_quantity:5 < "?column?":43 [outer=(5,43), constraints=(/5: (/NULL - ]; /43: (/NULL - ])] + │ │ └── l_quantity:5 < "?column?":46 [outer=(5,46), constraints=(/5: (/NULL - ]; /46: (/NULL - ])] │ └── aggregations - │ └── sum [as=sum:44, outer=(6)] + │ └── sum [as=sum:47, outer=(6)] │ └── l_extendedprice:6 └── projections - └── sum:44 / 7.0 [as=avg_yearly:45, outer=(44)] + └── sum:47 / 7.0 [as=avg_yearly:48, outer=(47)] # -------------------------------------------------- # Q18 @@ -1883,90 +1883,90 @@ ORDER BY LIMIT 100; ---- limit - ├── columns: c_name:2!null c_custkey:1!null o_orderkey:9!null o_orderdate:13!null o_totalprice:12!null sum:51!null - ├── internal-ordering: -12,+13 + ├── columns: c_name:2!null c_custkey:1!null o_orderkey:10!null o_orderdate:14!null o_totalprice:13!null sum:55!null + ├── internal-ordering: -13,+14 ├── cardinality: [0 - 100] - ├── key: (9) - ├── fd: (1)-->(2), (9)-->(1,2,12,13,51) - ├── ordering: -12,+13 + ├── key: (10) + ├── fd: (1)-->(2), (10)-->(1,2,13,14,55) + ├── ordering: -13,+14 ├── sort - │ ├── columns: c_custkey:1!null c_name:2!null o_orderkey:9!null o_totalprice:12!null o_orderdate:13!null sum:51!null - │ ├── key: (9) - │ ├── fd: (1)-->(2), (9)-->(1,2,12,13,51) - │ ├── ordering: -12,+13 + │ ├── columns: c_custkey:1!null c_name:2!null o_orderkey:10!null o_totalprice:13!null o_orderdate:14!null sum:55!null + │ ├── key: (10) + │ ├── fd: (1)-->(2), (10)-->(1,2,13,14,55) + │ ├── ordering: -13,+14 │ ├── limit hint: 100.00 │ └── group-by - │ ├── columns: c_custkey:1!null c_name:2!null o_orderkey:9!null o_totalprice:12!null o_orderdate:13!null sum:51!null - │ ├── grouping columns: o_orderkey:9!null - │ ├── internal-ordering: +(9|18) - │ ├── key: (9) - │ ├── fd: (1)-->(2), (9)-->(1,2,12,13,51) + │ ├── columns: c_custkey:1!null c_name:2!null o_orderkey:10!null o_totalprice:13!null o_orderdate:14!null sum:55!null + │ ├── grouping columns: o_orderkey:10!null + │ ├── internal-ordering: +(10|20) + │ ├── key: (10) + │ ├── fd: (1)-->(2), (10)-->(1,2,13,14,55) │ ├── inner-join (merge) - │ │ ├── columns: c_custkey:1!null c_name:2!null o_orderkey:9!null o_custkey:10!null o_totalprice:12!null o_orderdate:13!null l_orderkey:18!null l_quantity:22!null - │ │ ├── left ordering: +9 - │ │ ├── right ordering: +18 - │ │ ├── fd: (1)-->(2), (9)-->(10,12,13), (9)==(18), (18)==(9), (1)==(10), (10)==(1) - │ │ ├── ordering: +(9|18) [actual: +9] + │ │ ├── columns: c_custkey:1!null c_name:2!null o_orderkey:10!null o_custkey:11!null o_totalprice:13!null o_orderdate:14!null l_orderkey:20!null l_quantity:24!null + │ │ ├── left ordering: +10 + │ │ ├── right ordering: +20 + │ │ ├── fd: (1)-->(2), (10)-->(11,13,14), (10)==(20), (20)==(10), (1)==(11), (11)==(1) + │ │ ├── ordering: +(10|20) [actual: +10] │ │ ├── inner-join (lookup customer) - │ │ │ ├── columns: c_custkey:1!null c_name:2!null o_orderkey:9!null o_custkey:10!null o_totalprice:12!null o_orderdate:13!null - │ │ │ ├── key columns: [10] = [1] + │ │ │ ├── columns: c_custkey:1!null c_name:2!null o_orderkey:10!null o_custkey:11!null o_totalprice:13!null o_orderdate:14!null + │ │ │ ├── key columns: [11] = [1] │ │ │ ├── lookup columns are key - │ │ │ ├── key: (9) - │ │ │ ├── fd: (1)-->(2), (9)-->(10,12,13), (1)==(10), (10)==(1) - │ │ │ ├── ordering: +9 + │ │ │ ├── key: (10) + │ │ │ ├── fd: (1)-->(2), (10)-->(11,13,14), (1)==(11), (11)==(1) + │ │ │ ├── ordering: +10 │ │ │ ├── project - │ │ │ │ ├── columns: o_orderkey:9!null o_custkey:10!null o_totalprice:12!null o_orderdate:13!null - │ │ │ │ ├── key: (9) - │ │ │ │ ├── fd: (9)-->(10,12,13) - │ │ │ │ ├── ordering: +9 + │ │ │ │ ├── columns: o_orderkey:10!null o_custkey:11!null o_totalprice:13!null o_orderdate:14!null + │ │ │ │ ├── key: (10) + │ │ │ │ ├── fd: (10)-->(11,13,14) + │ │ │ │ ├── ordering: +10 │ │ │ │ └── project - │ │ │ │ ├── columns: o_orderkey:9!null o_custkey:10!null o_totalprice:12!null o_orderdate:13!null l_orderkey:34!null - │ │ │ │ ├── key: (34) - │ │ │ │ ├── fd: (9)-->(10,12,13), (9)==(34), (34)==(9) - │ │ │ │ ├── ordering: +(9|34) [actual: +34] + │ │ │ │ ├── columns: o_orderkey:10!null o_custkey:11!null o_totalprice:13!null o_orderdate:14!null l_orderkey:37!null + │ │ │ │ ├── key: (37) + │ │ │ │ ├── fd: (10)-->(11,13,14), (10)==(37), (37)==(10) + │ │ │ │ ├── ordering: +(10|37) [actual: +37] │ │ │ │ └── inner-join (lookup orders) - │ │ │ │ ├── columns: o_orderkey:9!null o_custkey:10!null o_totalprice:12!null o_orderdate:13!null l_orderkey:34!null sum:50!null - │ │ │ │ ├── key columns: [34] = [9] + │ │ │ │ ├── columns: o_orderkey:10!null o_custkey:11!null o_totalprice:13!null o_orderdate:14!null l_orderkey:37!null sum:54!null + │ │ │ │ ├── key columns: [37] = [10] │ │ │ │ ├── lookup columns are key - │ │ │ │ ├── key: (34) - │ │ │ │ ├── fd: (9)-->(10,12,13), (34)-->(50), (9)==(34), (34)==(9) - │ │ │ │ ├── ordering: +(9|34) [actual: +34] + │ │ │ │ ├── key: (37) + │ │ │ │ ├── fd: (10)-->(11,13,14), (37)-->(54), (10)==(37), (37)==(10) + │ │ │ │ ├── ordering: +(10|37) [actual: +37] │ │ │ │ ├── select - │ │ │ │ │ ├── columns: l_orderkey:34!null sum:50!null - │ │ │ │ │ ├── key: (34) - │ │ │ │ │ ├── fd: (34)-->(50) - │ │ │ │ │ ├── ordering: +34 + │ │ │ │ │ ├── columns: l_orderkey:37!null sum:54!null + │ │ │ │ │ ├── key: (37) + │ │ │ │ │ ├── fd: (37)-->(54) + │ │ │ │ │ ├── ordering: +37 │ │ │ │ │ ├── group-by - │ │ │ │ │ │ ├── columns: l_orderkey:34!null sum:50!null - │ │ │ │ │ │ ├── grouping columns: l_orderkey:34!null - │ │ │ │ │ │ ├── key: (34) - │ │ │ │ │ │ ├── fd: (34)-->(50) - │ │ │ │ │ │ ├── ordering: +34 + │ │ │ │ │ │ ├── columns: l_orderkey:37!null sum:54!null + │ │ │ │ │ │ ├── grouping columns: l_orderkey:37!null + │ │ │ │ │ │ ├── key: (37) + │ │ │ │ │ │ ├── fd: (37)-->(54) + │ │ │ │ │ │ ├── ordering: +37 │ │ │ │ │ │ ├── scan lineitem - │ │ │ │ │ │ │ ├── columns: l_orderkey:34!null l_quantity:38!null - │ │ │ │ │ │ │ └── ordering: +34 + │ │ │ │ │ │ │ ├── columns: l_orderkey:37!null l_quantity:41!null + │ │ │ │ │ │ │ └── ordering: +37 │ │ │ │ │ │ └── aggregations - │ │ │ │ │ │ └── sum [as=sum:50, outer=(38)] - │ │ │ │ │ │ └── l_quantity:38 + │ │ │ │ │ │ └── sum [as=sum:54, outer=(41)] + │ │ │ │ │ │ └── l_quantity:41 │ │ │ │ │ └── filters - │ │ │ │ │ └── sum:50 > 300.0 [outer=(50), constraints=(/50: [/300.00000000000006 - ]; tight)] + │ │ │ │ │ └── sum:54 > 300.0 [outer=(54), constraints=(/54: [/300.00000000000006 - ]; tight)] │ │ │ │ └── filters (true) │ │ │ └── filters (true) │ │ ├── scan lineitem - │ │ │ ├── columns: l_orderkey:18!null l_quantity:22!null - │ │ │ └── ordering: +18 + │ │ │ ├── columns: l_orderkey:20!null l_quantity:24!null + │ │ │ └── ordering: +20 │ │ └── filters (true) │ └── aggregations - │ ├── sum [as=sum:51, outer=(22)] - │ │ └── l_quantity:22 + │ ├── sum [as=sum:55, outer=(24)] + │ │ └── l_quantity:24 │ ├── const-agg [as=c_custkey:1, outer=(1)] │ │ └── c_custkey:1 │ ├── const-agg [as=c_name:2, outer=(2)] │ │ └── c_name:2 - │ ├── const-agg [as=o_totalprice:12, outer=(12)] - │ │ └── o_totalprice:12 - │ └── const-agg [as=o_orderdate:13, outer=(13)] - │ └── o_orderdate:13 + │ ├── const-agg [as=o_totalprice:13, outer=(13)] + │ │ └── o_totalprice:13 + │ └── const-agg [as=o_orderdate:14, outer=(14)] + │ └── o_orderdate:14 └── 100 # -------------------------------------------------- @@ -2019,19 +2019,19 @@ WHERE ); ---- scalar-group-by - ├── columns: revenue:27 + ├── columns: revenue:29 ├── cardinality: [1 - 1] ├── immutable ├── key: () - ├── fd: ()-->(27) + ├── fd: ()-->(29) ├── project - │ ├── columns: column26:26!null + │ ├── columns: column28:28!null │ ├── immutable │ ├── inner-join (lookup part) - │ │ ├── columns: l_partkey:2!null l_quantity:5!null l_extendedprice:6!null l_discount:7!null l_shipinstruct:14!null l_shipmode:15!null p_partkey:17!null p_brand:20!null p_size:22!null p_container:23!null - │ │ ├── key columns: [2] = [17] + │ │ ├── columns: l_partkey:2!null l_quantity:5!null l_extendedprice:6!null l_discount:7!null l_shipinstruct:14!null l_shipmode:15!null p_partkey:18!null p_brand:21!null p_size:23!null p_container:24!null + │ │ ├── key columns: [2] = [18] │ │ ├── lookup columns are key - │ │ ├── fd: ()-->(14), (17)-->(20,22,23), (2)==(17), (17)==(2) + │ │ ├── fd: ()-->(14), (18)-->(21,23,24), (2)==(18), (18)==(2) │ │ ├── select │ │ │ ├── columns: l_partkey:2!null l_quantity:5!null l_extendedprice:6!null l_discount:7!null l_shipinstruct:14!null l_shipmode:15!null │ │ │ ├── fd: ()-->(14) @@ -2041,13 +2041,13 @@ scalar-group-by │ │ │ ├── l_shipmode:15 IN ('AIR', 'AIR REG') [outer=(15), constraints=(/15: [/'AIR' - /'AIR'] [/'AIR REG' - /'AIR REG']; tight)] │ │ │ └── l_shipinstruct:14 = 'DELIVER IN PERSON' [outer=(14), constraints=(/14: [/'DELIVER IN PERSON' - /'DELIVER IN PERSON']; tight), fd=()-->(14)] │ │ └── filters - │ │ ├── ((((((p_brand:20 = 'Brand#12') AND (p_container:23 IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG'))) AND (l_quantity:5 >= 1.0)) AND (l_quantity:5 <= 11.0)) AND (p_size:22 <= 5)) OR (((((p_brand:20 = 'Brand#23') AND (p_container:23 IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG'))) AND (l_quantity:5 >= 10.0)) AND (l_quantity:5 <= 20.0)) AND (p_size:22 <= 10))) OR (((((p_brand:20 = 'Brand#34') AND (p_container:23 IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG'))) AND (l_quantity:5 >= 20.0)) AND (l_quantity:5 <= 30.0)) AND (p_size:22 <= 15)) [outer=(5,20,22,23), constraints=(/5: [/1.0 - /30.0]; /20: [/'Brand#12' - /'Brand#12'] [/'Brand#23' - /'Brand#23'] [/'Brand#34' - /'Brand#34']; /22: (/NULL - /15]; /23: [/'LG BOX' - /'LG BOX'] [/'LG CASE' - /'LG CASE'] [/'LG PACK' - /'LG PACK'] [/'LG PKG' - /'LG PKG'] [/'MED BAG' - /'MED BAG'] [/'MED BOX' - /'MED BOX'] [/'MED PACK' - /'MED PACK'] [/'MED PKG' - /'MED PKG'] [/'SM BOX' - /'SM BOX'] [/'SM CASE' - /'SM CASE'] [/'SM PACK' - /'SM PACK'] [/'SM PKG' - /'SM PKG'])] - │ │ └── p_size:22 >= 1 [outer=(22), constraints=(/22: [/1 - ]; tight)] + │ │ ├── ((((((p_brand:21 = 'Brand#12') AND (p_container:24 IN ('SM BOX', 'SM CASE', 'SM PACK', 'SM PKG'))) AND (l_quantity:5 >= 1.0)) AND (l_quantity:5 <= 11.0)) AND (p_size:23 <= 5)) OR (((((p_brand:21 = 'Brand#23') AND (p_container:24 IN ('MED BAG', 'MED BOX', 'MED PACK', 'MED PKG'))) AND (l_quantity:5 >= 10.0)) AND (l_quantity:5 <= 20.0)) AND (p_size:23 <= 10))) OR (((((p_brand:21 = 'Brand#34') AND (p_container:24 IN ('LG BOX', 'LG CASE', 'LG PACK', 'LG PKG'))) AND (l_quantity:5 >= 20.0)) AND (l_quantity:5 <= 30.0)) AND (p_size:23 <= 15)) [outer=(5,21,23,24), constraints=(/5: [/1.0 - /30.0]; /21: [/'Brand#12' - /'Brand#12'] [/'Brand#23' - /'Brand#23'] [/'Brand#34' - /'Brand#34']; /23: (/NULL - /15]; /24: [/'LG BOX' - /'LG BOX'] [/'LG CASE' - /'LG CASE'] [/'LG PACK' - /'LG PACK'] [/'LG PKG' - /'LG PKG'] [/'MED BAG' - /'MED BAG'] [/'MED BOX' - /'MED BOX'] [/'MED PACK' - /'MED PACK'] [/'MED PKG' - /'MED PKG'] [/'SM BOX' - /'SM BOX'] [/'SM CASE' - /'SM CASE'] [/'SM PACK' - /'SM PACK'] [/'SM PKG' - /'SM PKG'])] + │ │ └── p_size:23 >= 1 [outer=(23), constraints=(/23: [/1 - ]; tight)] │ └── projections - │ └── l_extendedprice:6 * (1.0 - l_discount:7) [as=column26:26, outer=(6,7), immutable] + │ └── l_extendedprice:6 * (1.0 - l_discount:7) [as=column28:28, outer=(6,7), immutable] └── aggregations - └── sum [as=sum:27, outer=(26)] - └── column26:26 + └── sum [as=sum:29, outer=(28)] + └── column28:28 # -------------------------------------------------- # Q20 @@ -2110,86 +2110,86 @@ sort ├── columns: s_name:2!null s_address:3!null ├── immutable └── inner-join (lookup nation) - ├── columns: s_suppkey:1!null s_name:2!null s_address:3!null s_nationkey:4!null n_nationkey:8!null n_name:9!null - ├── key columns: [4] = [8] + ├── columns: s_suppkey:1!null s_name:2!null s_address:3!null s_nationkey:4!null n_nationkey:9!null n_name:10!null + ├── key columns: [4] = [9] ├── lookup columns are key ├── immutable ├── key: (1) - ├── fd: ()-->(9), (1)-->(2-4), (4)==(8), (8)==(4) + ├── fd: ()-->(10), (1)-->(2-4), (4)==(9), (9)==(4) ├── project │ ├── columns: s_suppkey:1!null s_name:2!null s_address:3!null s_nationkey:4!null │ ├── immutable │ ├── key: (1) │ ├── fd: (1)-->(2-4) │ └── inner-join (lookup supplier) - │ ├── columns: s_suppkey:1!null s_name:2!null s_address:3!null s_nationkey:4!null ps_suppkey:13!null - │ ├── key columns: [13] = [1] + │ ├── columns: s_suppkey:1!null s_name:2!null s_address:3!null s_nationkey:4!null ps_suppkey:15!null + │ ├── key columns: [15] = [1] │ ├── lookup columns are key │ ├── immutable - │ ├── key: (13) - │ ├── fd: (1)-->(2-4), (1)==(13), (13)==(1) + │ ├── key: (15) + │ ├── fd: (1)-->(2-4), (1)==(15), (15)==(1) │ ├── distinct-on - │ │ ├── columns: ps_suppkey:13!null - │ │ ├── grouping columns: ps_suppkey:13!null + │ │ ├── columns: ps_suppkey:15!null + │ │ ├── grouping columns: ps_suppkey:15!null │ │ ├── immutable - │ │ ├── key: (13) + │ │ ├── key: (15) │ │ └── semi-join (hash) - │ │ ├── columns: ps_partkey:12!null ps_suppkey:13!null + │ │ ├── columns: ps_partkey:14!null ps_suppkey:15!null │ │ ├── immutable - │ │ ├── key: (12,13) + │ │ ├── key: (14,15) │ │ ├── project - │ │ │ ├── columns: ps_partkey:12!null ps_suppkey:13!null + │ │ │ ├── columns: ps_partkey:14!null ps_suppkey:15!null │ │ │ ├── immutable - │ │ │ ├── key: (12,13) + │ │ │ ├── key: (14,15) │ │ │ └── select - │ │ │ ├── columns: ps_partkey:12!null ps_suppkey:13!null ps_availqty:14!null sum:42 + │ │ │ ├── columns: ps_partkey:14!null ps_suppkey:15!null ps_availqty:16!null sum:47 │ │ │ ├── immutable - │ │ │ ├── key: (12,13) - │ │ │ ├── fd: (12,13)-->(14,42) + │ │ │ ├── key: (14,15) + │ │ │ ├── fd: (14,15)-->(16,47) │ │ │ ├── group-by - │ │ │ │ ├── columns: ps_partkey:12!null ps_suppkey:13!null ps_availqty:14!null sum:42 - │ │ │ │ ├── grouping columns: ps_partkey:12!null ps_suppkey:13!null - │ │ │ │ ├── key: (12,13) - │ │ │ │ ├── fd: (12,13)-->(14,42) + │ │ │ │ ├── columns: ps_partkey:14!null ps_suppkey:15!null ps_availqty:16!null sum:47 + │ │ │ │ ├── grouping columns: ps_partkey:14!null ps_suppkey:15!null + │ │ │ │ ├── key: (14,15) + │ │ │ │ ├── fd: (14,15)-->(16,47) │ │ │ │ ├── left-join (hash) - │ │ │ │ │ ├── columns: ps_partkey:12!null ps_suppkey:13!null ps_availqty:14!null l_partkey:27 l_suppkey:28 l_quantity:30 l_shipdate:36 + │ │ │ │ │ ├── columns: ps_partkey:14!null ps_suppkey:15!null ps_availqty:16!null l_partkey:31 l_suppkey:32 l_quantity:34 l_shipdate:40 │ │ │ │ │ ├── multiplicity: left-rows(one-or-more), right-rows(exactly-one) - │ │ │ │ │ ├── fd: (12,13)-->(14) + │ │ │ │ │ ├── fd: (14,15)-->(16) │ │ │ │ │ ├── scan partsupp - │ │ │ │ │ │ ├── columns: ps_partkey:12!null ps_suppkey:13!null ps_availqty:14!null - │ │ │ │ │ │ ├── key: (12,13) - │ │ │ │ │ │ └── fd: (12,13)-->(14) + │ │ │ │ │ │ ├── columns: ps_partkey:14!null ps_suppkey:15!null ps_availqty:16!null + │ │ │ │ │ │ ├── key: (14,15) + │ │ │ │ │ │ └── fd: (14,15)-->(16) │ │ │ │ │ ├── select - │ │ │ │ │ │ ├── columns: l_partkey:27!null l_suppkey:28!null l_quantity:30!null l_shipdate:36!null + │ │ │ │ │ │ ├── columns: l_partkey:31!null l_suppkey:32!null l_quantity:34!null l_shipdate:40!null │ │ │ │ │ │ ├── scan lineitem - │ │ │ │ │ │ │ └── columns: l_partkey:27!null l_suppkey:28!null l_quantity:30!null l_shipdate:36!null + │ │ │ │ │ │ │ └── columns: l_partkey:31!null l_suppkey:32!null l_quantity:34!null l_shipdate:40!null │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── (l_shipdate:36 >= '1994-01-01') AND (l_shipdate:36 < '1995-01-01') [outer=(36), constraints=(/36: [/'1994-01-01' - /'1994-12-31']; tight)] + │ │ │ │ │ │ └── (l_shipdate:40 >= '1994-01-01') AND (l_shipdate:40 < '1995-01-01') [outer=(40), constraints=(/40: [/'1994-01-01' - /'1994-12-31']; tight)] │ │ │ │ │ └── filters - │ │ │ │ │ ├── l_partkey:27 = ps_partkey:12 [outer=(12,27), constraints=(/12: (/NULL - ]; /27: (/NULL - ]), fd=(12)==(27), (27)==(12)] - │ │ │ │ │ └── l_suppkey:28 = ps_suppkey:13 [outer=(13,28), constraints=(/13: (/NULL - ]; /28: (/NULL - ]), fd=(13)==(28), (28)==(13)] + │ │ │ │ │ ├── l_partkey:31 = ps_partkey:14 [outer=(14,31), constraints=(/14: (/NULL - ]; /31: (/NULL - ]), fd=(14)==(31), (31)==(14)] + │ │ │ │ │ └── l_suppkey:32 = ps_suppkey:15 [outer=(15,32), constraints=(/15: (/NULL - ]; /32: (/NULL - ]), fd=(15)==(32), (32)==(15)] │ │ │ │ └── aggregations - │ │ │ │ ├── sum [as=sum:42, outer=(30)] - │ │ │ │ │ └── l_quantity:30 - │ │ │ │ └── const-agg [as=ps_availqty:14, outer=(14)] - │ │ │ │ └── ps_availqty:14 + │ │ │ │ ├── sum [as=sum:47, outer=(34)] + │ │ │ │ │ └── l_quantity:34 + │ │ │ │ └── const-agg [as=ps_availqty:16, outer=(16)] + │ │ │ │ └── ps_availqty:16 │ │ │ └── filters - │ │ │ └── ps_availqty:14 > (sum:42 * 0.5) [outer=(14,42), immutable, constraints=(/14: (/NULL - ])] + │ │ │ └── ps_availqty:16 > (sum:47 * 0.5) [outer=(16,47), immutable, constraints=(/16: (/NULL - ])] │ │ ├── select - │ │ │ ├── columns: p_partkey:17!null p_name:18!null - │ │ │ ├── key: (17) - │ │ │ ├── fd: (17)-->(18) + │ │ │ ├── columns: p_partkey:20!null p_name:21!null + │ │ │ ├── key: (20) + │ │ │ ├── fd: (20)-->(21) │ │ │ ├── scan part - │ │ │ │ ├── columns: p_partkey:17!null p_name:18!null - │ │ │ │ ├── key: (17) - │ │ │ │ └── fd: (17)-->(18) + │ │ │ │ ├── columns: p_partkey:20!null p_name:21!null + │ │ │ │ ├── key: (20) + │ │ │ │ └── fd: (20)-->(21) │ │ │ └── filters - │ │ │ └── p_name:18 LIKE 'forest%' [outer=(18), constraints=(/18: [/'forest' - /'foresu'); tight)] + │ │ │ └── p_name:21 LIKE 'forest%' [outer=(21), constraints=(/21: [/'forest' - /'foresu'); tight)] │ │ └── filters - │ │ └── ps_partkey:12 = p_partkey:17 [outer=(12,17), constraints=(/12: (/NULL - ]; /17: (/NULL - ]), fd=(12)==(17), (17)==(12)] + │ │ └── ps_partkey:14 = p_partkey:20 [outer=(14,20), constraints=(/14: (/NULL - ]; /20: (/NULL - ]), fd=(14)==(20), (20)==(14)] │ └── filters (true) └── filters - └── n_name:9 = 'CANADA' [outer=(9), constraints=(/9: [/'CANADA' - /'CANADA']; tight), fd=()-->(9)] + └── n_name:10 = 'CANADA' [outer=(10), constraints=(/10: [/'CANADA' - /'CANADA']; tight), fd=()-->(10)] # -------------------------------------------------- # Q21 @@ -2244,74 +2244,74 @@ ORDER BY LIMIT 100; ---- limit - ├── columns: s_name:2!null numwait:69!null - ├── internal-ordering: -69,+2 + ├── columns: s_name:2!null numwait:75!null + ├── internal-ordering: -75,+2 ├── cardinality: [0 - 100] ├── key: (2) - ├── fd: (2)-->(69) - ├── ordering: -69,+2 + ├── fd: (2)-->(75) + ├── ordering: -75,+2 ├── sort - │ ├── columns: s_name:2!null count_rows:69!null + │ ├── columns: s_name:2!null count_rows:75!null │ ├── key: (2) - │ ├── fd: (2)-->(69) - │ ├── ordering: -69,+2 + │ ├── fd: (2)-->(75) + │ ├── ordering: -75,+2 │ ├── limit hint: 100.00 │ └── group-by - │ ├── columns: s_name:2!null count_rows:69!null + │ ├── columns: s_name:2!null count_rows:75!null │ ├── grouping columns: s_name:2!null │ ├── key: (2) - │ ├── fd: (2)-->(69) + │ ├── fd: (2)-->(75) │ ├── inner-join (lookup nation) - │ │ ├── columns: s_suppkey:1!null s_name:2!null s_nationkey:4!null l1.l_orderkey:8!null l1.l_suppkey:10!null l1.l_commitdate:19!null l1.l_receiptdate:20!null o_orderkey:24!null o_orderstatus:26!null n_nationkey:33!null n_name:34!null - │ │ ├── key columns: [4] = [33] + │ │ ├── columns: s_suppkey:1!null s_name:2!null s_nationkey:4!null l1.l_orderkey:9!null l1.l_suppkey:11!null l1.l_commitdate:20!null l1.l_receiptdate:21!null o_orderkey:26!null o_orderstatus:28!null n_nationkey:36!null n_name:37!null + │ │ ├── key columns: [4] = [36] │ │ ├── lookup columns are key - │ │ ├── fd: ()-->(26,34), (1)-->(2,4), (8)==(24), (24)==(8), (1)==(10), (10)==(1), (4)==(33), (33)==(4) + │ │ ├── fd: ()-->(28,37), (1)-->(2,4), (9)==(26), (26)==(9), (1)==(11), (11)==(1), (4)==(36), (36)==(4) │ │ ├── inner-join (lookup supplier) - │ │ │ ├── columns: s_suppkey:1!null s_name:2!null s_nationkey:4!null l1.l_orderkey:8!null l1.l_suppkey:10!null l1.l_commitdate:19!null l1.l_receiptdate:20!null o_orderkey:24!null o_orderstatus:26!null - │ │ │ ├── key columns: [10] = [1] + │ │ │ ├── columns: s_suppkey:1!null s_name:2!null s_nationkey:4!null l1.l_orderkey:9!null l1.l_suppkey:11!null l1.l_commitdate:20!null l1.l_receiptdate:21!null o_orderkey:26!null o_orderstatus:28!null + │ │ │ ├── key columns: [11] = [1] │ │ │ ├── lookup columns are key - │ │ │ ├── fd: ()-->(26), (1)-->(2,4), (8)==(24), (24)==(8), (1)==(10), (10)==(1) + │ │ │ ├── fd: ()-->(28), (1)-->(2,4), (9)==(26), (26)==(9), (1)==(11), (11)==(1) │ │ │ ├── inner-join (lookup orders) - │ │ │ │ ├── columns: l1.l_orderkey:8!null l1.l_suppkey:10!null l1.l_commitdate:19!null l1.l_receiptdate:20!null o_orderkey:24!null o_orderstatus:26!null - │ │ │ │ ├── key columns: [8] = [24] + │ │ │ │ ├── columns: l1.l_orderkey:9!null l1.l_suppkey:11!null l1.l_commitdate:20!null l1.l_receiptdate:21!null o_orderkey:26!null o_orderstatus:28!null + │ │ │ │ ├── key columns: [9] = [26] │ │ │ │ ├── lookup columns are key - │ │ │ │ ├── fd: ()-->(26), (8)==(24), (24)==(8) + │ │ │ │ ├── fd: ()-->(28), (9)==(26), (26)==(9) │ │ │ │ ├── semi-join (hash) - │ │ │ │ │ ├── columns: l1.l_orderkey:8!null l1.l_suppkey:10!null l1.l_commitdate:19!null l1.l_receiptdate:20!null + │ │ │ │ │ ├── columns: l1.l_orderkey:9!null l1.l_suppkey:11!null l1.l_commitdate:20!null l1.l_receiptdate:21!null │ │ │ │ │ ├── anti-join (merge) - │ │ │ │ │ │ ├── columns: l1.l_orderkey:8!null l1.l_suppkey:10!null l1.l_commitdate:19!null l1.l_receiptdate:20!null - │ │ │ │ │ │ ├── left ordering: +8 - │ │ │ │ │ │ ├── right ordering: +53 + │ │ │ │ │ │ ├── columns: l1.l_orderkey:9!null l1.l_suppkey:11!null l1.l_commitdate:20!null l1.l_receiptdate:21!null + │ │ │ │ │ │ ├── left ordering: +9 + │ │ │ │ │ │ ├── right ordering: +58 │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: l1.l_orderkey:8!null l1.l_suppkey:10!null l1.l_commitdate:19!null l1.l_receiptdate:20!null - │ │ │ │ │ │ │ ├── ordering: +8 + │ │ │ │ │ │ │ ├── columns: l1.l_orderkey:9!null l1.l_suppkey:11!null l1.l_commitdate:20!null l1.l_receiptdate:21!null + │ │ │ │ │ │ │ ├── ordering: +9 │ │ │ │ │ │ │ ├── scan l1 - │ │ │ │ │ │ │ │ ├── columns: l1.l_orderkey:8!null l1.l_suppkey:10!null l1.l_commitdate:19!null l1.l_receiptdate:20!null - │ │ │ │ │ │ │ │ └── ordering: +8 + │ │ │ │ │ │ │ │ ├── columns: l1.l_orderkey:9!null l1.l_suppkey:11!null l1.l_commitdate:20!null l1.l_receiptdate:21!null + │ │ │ │ │ │ │ │ └── ordering: +9 │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── l1.l_receiptdate:20 > l1.l_commitdate:19 [outer=(19,20), constraints=(/19: (/NULL - ]; /20: (/NULL - ])] + │ │ │ │ │ │ │ └── l1.l_receiptdate:21 > l1.l_commitdate:20 [outer=(20,21), constraints=(/20: (/NULL - ]; /21: (/NULL - ])] │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: l3.l_orderkey:53!null l3.l_suppkey:55!null l3.l_commitdate:64!null l3.l_receiptdate:65!null - │ │ │ │ │ │ │ ├── ordering: +53 + │ │ │ │ │ │ │ ├── columns: l3.l_orderkey:58!null l3.l_suppkey:60!null l3.l_commitdate:69!null l3.l_receiptdate:70!null + │ │ │ │ │ │ │ ├── ordering: +58 │ │ │ │ │ │ │ ├── scan l3 - │ │ │ │ │ │ │ │ ├── columns: l3.l_orderkey:53!null l3.l_suppkey:55!null l3.l_commitdate:64!null l3.l_receiptdate:65!null - │ │ │ │ │ │ │ │ └── ordering: +53 + │ │ │ │ │ │ │ │ ├── columns: l3.l_orderkey:58!null l3.l_suppkey:60!null l3.l_commitdate:69!null l3.l_receiptdate:70!null + │ │ │ │ │ │ │ │ └── ordering: +58 │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── l3.l_receiptdate:65 > l3.l_commitdate:64 [outer=(64,65), constraints=(/64: (/NULL - ]; /65: (/NULL - ])] + │ │ │ │ │ │ │ └── l3.l_receiptdate:70 > l3.l_commitdate:69 [outer=(69,70), constraints=(/69: (/NULL - ]; /70: (/NULL - ])] │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── l3.l_suppkey:55 != l1.l_suppkey:10 [outer=(10,55), constraints=(/10: (/NULL - ]; /55: (/NULL - ])] + │ │ │ │ │ │ └── l3.l_suppkey:60 != l1.l_suppkey:11 [outer=(11,60), constraints=(/11: (/NULL - ]; /60: (/NULL - ])] │ │ │ │ │ ├── scan l2@l_sk - │ │ │ │ │ │ └── columns: l2.l_orderkey:37!null l2.l_suppkey:39!null + │ │ │ │ │ │ └── columns: l2.l_orderkey:41!null l2.l_suppkey:43!null │ │ │ │ │ └── filters - │ │ │ │ │ ├── l2.l_orderkey:37 = l1.l_orderkey:8 [outer=(8,37), constraints=(/8: (/NULL - ]; /37: (/NULL - ]), fd=(8)==(37), (37)==(8)] - │ │ │ │ │ └── l2.l_suppkey:39 != l1.l_suppkey:10 [outer=(10,39), constraints=(/10: (/NULL - ]; /39: (/NULL - ])] + │ │ │ │ │ ├── l2.l_orderkey:41 = l1.l_orderkey:9 [outer=(9,41), constraints=(/9: (/NULL - ]; /41: (/NULL - ]), fd=(9)==(41), (41)==(9)] + │ │ │ │ │ └── l2.l_suppkey:43 != l1.l_suppkey:11 [outer=(11,43), constraints=(/11: (/NULL - ]; /43: (/NULL - ])] │ │ │ │ └── filters - │ │ │ │ └── o_orderstatus:26 = 'F' [outer=(26), constraints=(/26: [/'F' - /'F']; tight), fd=()-->(26)] + │ │ │ │ └── o_orderstatus:28 = 'F' [outer=(28), constraints=(/28: [/'F' - /'F']; tight), fd=()-->(28)] │ │ │ └── filters (true) │ │ └── filters - │ │ └── n_name:34 = 'SAUDI ARABIA' [outer=(34), constraints=(/34: [/'SAUDI ARABIA' - /'SAUDI ARABIA']; tight), fd=()-->(34)] + │ │ └── n_name:37 = 'SAUDI ARABIA' [outer=(37), constraints=(/37: [/'SAUDI ARABIA' - /'SAUDI ARABIA']; tight), fd=()-->(37)] │ └── aggregations - │ └── count-rows [as=count_rows:69] + │ └── count-rows [as=count_rows:75] └── 100 # -------------------------------------------------- @@ -2364,23 +2364,23 @@ ORDER BY cntrycode; ---- group-by - ├── columns: cntrycode:27 numcust:28!null totacctbal:29!null - ├── grouping columns: cntrycode:27 + ├── columns: cntrycode:30 numcust:31!null totacctbal:32!null + ├── grouping columns: cntrycode:30 ├── immutable - ├── key: (27) - ├── fd: (27)-->(28,29) - ├── ordering: +27 + ├── key: (30) + ├── fd: (30)-->(31,32) + ├── ordering: +30 ├── sort - │ ├── columns: c_acctbal:6!null cntrycode:27 + │ ├── columns: c_acctbal:6!null cntrycode:30 │ ├── immutable - │ ├── ordering: +27 + │ ├── ordering: +30 │ └── project - │ ├── columns: cntrycode:27 c_acctbal:6!null + │ ├── columns: cntrycode:30 c_acctbal:6!null │ ├── immutable │ ├── anti-join (merge) │ │ ├── columns: c_custkey:1!null c_phone:5!null c_acctbal:6!null │ │ ├── left ordering: +1 - │ │ ├── right ordering: +19 + │ │ ├── right ordering: +21 │ │ ├── immutable │ │ ├── key: (1) │ │ ├── fd: (1)-->(5,6) @@ -2401,29 +2401,29 @@ group-by │ │ │ ├── c_acctbal:6 │ │ │ └── subquery │ │ │ └── scalar-group-by - │ │ │ ├── columns: avg:17 + │ │ │ ├── columns: avg:19 │ │ │ ├── cardinality: [1 - 1] │ │ │ ├── immutable │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(17) + │ │ │ ├── fd: ()-->(19) │ │ │ ├── select - │ │ │ │ ├── columns: c_phone:13!null c_acctbal:14!null + │ │ │ │ ├── columns: c_phone:14!null c_acctbal:15!null │ │ │ │ ├── immutable │ │ │ │ ├── scan customer - │ │ │ │ │ └── columns: c_phone:13!null c_acctbal:14!null + │ │ │ │ │ └── columns: c_phone:14!null c_acctbal:15!null │ │ │ │ └── filters - │ │ │ │ ├── c_acctbal:14 > 0.0 [outer=(14), constraints=(/14: [/5e-324 - ]; tight)] - │ │ │ │ └── substring(c_phone:13, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31') [outer=(13), immutable] + │ │ │ │ ├── c_acctbal:15 > 0.0 [outer=(15), constraints=(/15: [/5e-324 - ]; tight)] + │ │ │ │ └── substring(c_phone:14, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31') [outer=(14), immutable] │ │ │ └── aggregations - │ │ │ └── avg [as=avg:17, outer=(14)] - │ │ │ └── c_acctbal:14 + │ │ │ └── avg [as=avg:19, outer=(15)] + │ │ │ └── c_acctbal:15 │ │ ├── scan orders@o_ck - │ │ │ ├── columns: o_custkey:19!null - │ │ │ └── ordering: +19 + │ │ │ ├── columns: o_custkey:21!null + │ │ │ └── ordering: +21 │ │ └── filters (true) │ └── projections - │ └── substring(c_phone:5, 1, 2) [as=cntrycode:27, outer=(5), immutable] + │ └── substring(c_phone:5, 1, 2) [as=cntrycode:30, outer=(5), immutable] └── aggregations - ├── count-rows [as=count_rows:28] - └── sum [as=sum:29, outer=(6)] + ├── count-rows [as=count_rows:31] + └── sum [as=sum:32, outer=(6)] └── c_acctbal:6 diff --git a/pkg/sql/opt/xform/testdata/external/trading b/pkg/sql/opt/xform/testdata/external/trading index 89fbc4bc8636..a2e26eff68c9 100644 --- a/pkg/sql/opt/xform/testdata/external/trading +++ b/pkg/sql/opt/xform/testdata/external/trading @@ -548,35 +548,35 @@ SELECT FROM CardsView WHERE Version > 1584421773604892000.0000000000 ---- project - ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null buyprice:9!null sellprice:10!null desiredinventory:12!null actualinventory:13!null version:15!null discount:11!null maxinventory:14!null + ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null buyprice:10!null sellprice:11!null desiredinventory:13!null actualinventory:14!null version:16!null discount:12!null maxinventory:15!null ├── immutable ├── stats: [rows=1] - ├── key: (15) - ├── fd: (1)-->(2-6,9-15), (2,4,5)~~>(1,3,6), (15)-->(1-6,9-14) + ├── key: (16) + ├── fd: (1)-->(2-6,10-16), (2,4,5)~~>(1,3,6), (16)-->(1-6,10-15) └── inner-join (lookup cards) - ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null dealerid:7!null cardid:8!null buyprice:9!null sellprice:10!null discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null version:15!null - ├── key columns: [8] = [1] + ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null dealerid:8!null cardid:9!null buyprice:10!null sellprice:11!null discount:12!null desiredinventory:13!null actualinventory:14!null maxinventory:15!null version:16!null + ├── key columns: [9] = [1] ├── lookup columns are key ├── immutable - ├── stats: [rows=1, distinct(1)=0.0201621393, null(1)=0, distinct(8)=0.0201621393, null(8)=0] - ├── key: (8) - ├── fd: ()-->(7), (1)-->(2-6), (2,4,5)~~>(1,3,6), (8)-->(9-15), (15)-->(8-14), (1)==(8), (8)==(1) + ├── stats: [rows=1, distinct(1)=0.0201621393, null(1)=0, distinct(9)=0.0201621393, null(9)=0] + ├── key: (9) + ├── fd: ()-->(8), (1)-->(2-6), (2,4,5)~~>(1,3,6), (9)-->(10-16), (16)-->(9-15), (1)==(9), (9)==(1) ├── index-join cardsinfo - │ ├── columns: dealerid:7!null cardid:8!null buyprice:9!null sellprice:10!null discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null version:15!null + │ ├── columns: dealerid:8!null cardid:9!null buyprice:10!null sellprice:11!null discount:12!null desiredinventory:13!null actualinventory:14!null maxinventory:15!null version:16!null │ ├── immutable - │ ├── stats: [rows=0.0201621426, distinct(7)=0.0201621426, null(7)=0, distinct(8)=0.0201621393, null(8)=0, distinct(9)=0.02016214, null(9)=0, distinct(10)=0.02016214, null(10)=0, distinct(11)=0.02016214, null(11)=0, distinct(12)=0.02016214, null(12)=0, distinct(13)=0.02016214, null(13)=0, distinct(14)=0.02016214, null(14)=0, distinct(15)=0.0201621426, null(15)=0, distinct(7,15)=0.0201621426, null(7,15)=0] - │ │ histogram(15)= 0 0 0.020162 0 + │ ├── stats: [rows=0.0201621426, distinct(8)=0.0201621426, null(8)=0, distinct(9)=0.0201621393, null(9)=0, distinct(10)=0.02016214, null(10)=0, distinct(11)=0.02016214, null(11)=0, distinct(12)=0.02016214, null(12)=0, distinct(13)=0.02016214, null(13)=0, distinct(14)=0.02016214, null(14)=0, distinct(15)=0.02016214, null(15)=0, distinct(16)=0.0201621426, null(16)=0, distinct(8,16)=0.0201621426, null(8,16)=0] + │ │ histogram(16)= 0 0 0.020162 0 │ │ <--- 1584421773604892000.0000000000 ---------- 1584421778604892000 - │ ├── key: (8) - │ ├── fd: ()-->(7), (8)-->(9-15), (15)-->(8-14) + │ ├── key: (9) + │ ├── fd: ()-->(8), (9)-->(10-16), (16)-->(9-15) │ └── scan cardsinfo@cardsinfoversionindex - │ ├── columns: dealerid:7!null cardid:8!null version:15!null - │ ├── constraint: /7/15: (/1/1584421773604892000.0000000000 - /1] - │ ├── stats: [rows=0.0201621426, distinct(7)=0.0201621426, null(7)=0, distinct(8)=0.0201621393, null(8)=0, distinct(15)=0.0201621426, null(15)=0, distinct(7,15)=0.0201621426, null(7,15)=0] - │ │ histogram(15)= 0 0 0.020162 0 + │ ├── columns: dealerid:8!null cardid:9!null version:16!null + │ ├── constraint: /8/16: (/1/1584421773604892000.0000000000 - /1] + │ ├── stats: [rows=0.0201621426, distinct(8)=0.0201621426, null(8)=0, distinct(9)=0.0201621393, null(9)=0, distinct(16)=0.0201621426, null(16)=0, distinct(8,16)=0.0201621426, null(8,16)=0] + │ │ histogram(16)= 0 0 0.020162 0 │ │ <--- 1584421773604892000.0000000000 ---------- 1584421778604892000 - │ ├── key: (8) - │ └── fd: ()-->(7), (8)-->(15), (15)-->(8) + │ ├── key: (9) + │ └── fd: ()-->(8), (9)-->(16), (16)-->(9) └── filters (true) # Get version of last card that was changed. @@ -585,86 +585,86 @@ opt format=show-stats SELECT coalesce(max(Version), 0) FROM GlobalCardsView ---- project - ├── columns: coalesce:31 + ├── columns: coalesce:34 ├── cardinality: [1 - 1] ├── stats: [rows=1] ├── key: () - ├── fd: ()-->(31) + ├── fd: ()-->(34) ├── scalar-group-by - │ ├── columns: max:30 + │ ├── columns: max:33 │ ├── cardinality: [1 - 1] │ ├── stats: [rows=1] │ ├── key: () - │ ├── fd: ()-->(30) + │ ├── fd: ()-->(33) │ ├── limit - │ │ ├── columns: dealerid:7!null version:15!null - │ │ ├── internal-ordering: -15 + │ │ ├── columns: dealerid:8!null version:16!null + │ │ ├── internal-ordering: -16 │ │ ├── cardinality: [0 - 1] │ │ ├── stats: [rows=1] │ │ ├── key: () - │ │ ├── fd: ()-->(7,15) + │ │ ├── fd: ()-->(8,16) │ │ ├── sort - │ │ │ ├── columns: dealerid:7!null version:15!null + │ │ │ ├── columns: dealerid:8!null version:16!null │ │ │ ├── cardinality: [0 - 4] - │ │ │ ├── stats: [rows=4, distinct(7,15)=4, null(7,15)=0] - │ │ │ ├── key: (7,15) - │ │ │ ├── ordering: -15 + │ │ │ ├── stats: [rows=4, distinct(8,16)=4, null(8,16)=0] + │ │ │ ├── key: (8,16) + │ │ │ ├── ordering: -16 │ │ │ ├── limit hint: 1.00 │ │ │ └── union - │ │ │ ├── columns: dealerid:7!null version:15!null - │ │ │ ├── left columns: dealerid:7!null version:15!null - │ │ │ ├── right columns: dealerid:59 version:67 + │ │ │ ├── columns: dealerid:8!null version:16!null + │ │ │ ├── left columns: dealerid:8!null version:16!null + │ │ │ ├── right columns: dealerid:65 version:73 │ │ │ ├── cardinality: [0 - 4] - │ │ │ ├── stats: [rows=4, distinct(7,15)=4, null(7,15)=0] - │ │ │ ├── key: (7,15) + │ │ │ ├── stats: [rows=4, distinct(8,16)=4, null(8,16)=0] + │ │ │ ├── key: (8,16) │ │ │ ├── union - │ │ │ │ ├── columns: dealerid:7!null version:15!null - │ │ │ │ ├── left columns: dealerid:7!null version:15!null - │ │ │ │ ├── right columns: dealerid:50 version:58 + │ │ │ │ ├── columns: dealerid:8!null version:16!null + │ │ │ │ ├── left columns: dealerid:8!null version:16!null + │ │ │ │ ├── right columns: dealerid:55 version:63 │ │ │ │ ├── cardinality: [0 - 3] - │ │ │ │ ├── stats: [rows=3, distinct(7,15)=3, null(7,15)=0] - │ │ │ │ ├── key: (7,15) + │ │ │ │ ├── stats: [rows=3, distinct(8,16)=3, null(8,16)=0] + │ │ │ │ ├── key: (8,16) │ │ │ │ ├── union - │ │ │ │ │ ├── columns: dealerid:7!null version:15!null - │ │ │ │ │ ├── left columns: dealerid:32 version:40 - │ │ │ │ │ ├── right columns: dealerid:41 version:49 + │ │ │ │ │ ├── columns: dealerid:8!null version:16!null + │ │ │ │ │ ├── left columns: dealerid:35 version:43 + │ │ │ │ │ ├── right columns: dealerid:45 version:53 │ │ │ │ │ ├── cardinality: [0 - 2] - │ │ │ │ │ ├── stats: [rows=2, distinct(7,15)=2, null(7,15)=0] - │ │ │ │ │ ├── key: (7,15) + │ │ │ │ │ ├── stats: [rows=2, distinct(8,16)=2, null(8,16)=0] + │ │ │ │ │ ├── key: (8,16) │ │ │ │ │ ├── scan cardsinfo@cardsinfoversionindex,rev - │ │ │ │ │ │ ├── columns: dealerid:32!null version:40!null - │ │ │ │ │ │ ├── constraint: /32/40: [/1 - /1] + │ │ │ │ │ │ ├── columns: dealerid:35!null version:43!null + │ │ │ │ │ │ ├── constraint: /35/43: [/1 - /1] │ │ │ │ │ │ ├── limit: 1(rev) - │ │ │ │ │ │ ├── stats: [rows=1, distinct(32)=1, null(32)=0, distinct(32,40)=1, null(32,40)=0] + │ │ │ │ │ │ ├── stats: [rows=1, distinct(35)=1, null(35)=0, distinct(35,43)=1, null(35,43)=0] │ │ │ │ │ │ ├── key: () - │ │ │ │ │ │ └── fd: ()-->(32,40) + │ │ │ │ │ │ └── fd: ()-->(35,43) │ │ │ │ │ └── scan cardsinfo@cardsinfoversionindex,rev - │ │ │ │ │ ├── columns: dealerid:41!null version:49!null - │ │ │ │ │ ├── constraint: /41/49: [/2 - /2] + │ │ │ │ │ ├── columns: dealerid:45!null version:53!null + │ │ │ │ │ ├── constraint: /45/53: [/2 - /2] │ │ │ │ │ ├── limit: 1(rev) - │ │ │ │ │ ├── stats: [rows=1, distinct(41)=1, null(41)=0, distinct(41,49)=1, null(41,49)=0] + │ │ │ │ │ ├── stats: [rows=1, distinct(45)=1, null(45)=0, distinct(45,53)=1, null(45,53)=0] │ │ │ │ │ ├── key: () - │ │ │ │ │ └── fd: ()-->(41,49) + │ │ │ │ │ └── fd: ()-->(45,53) │ │ │ │ └── scan cardsinfo@cardsinfoversionindex,rev - │ │ │ │ ├── columns: dealerid:50!null version:58!null - │ │ │ │ ├── constraint: /50/58: [/3 - /3] + │ │ │ │ ├── columns: dealerid:55!null version:63!null + │ │ │ │ ├── constraint: /55/63: [/3 - /3] │ │ │ │ ├── limit: 1(rev) - │ │ │ │ ├── stats: [rows=1, distinct(50)=1, null(50)=0, distinct(50,58)=1, null(50,58)=0] + │ │ │ │ ├── stats: [rows=1, distinct(55)=1, null(55)=0, distinct(55,63)=1, null(55,63)=0] │ │ │ │ ├── key: () - │ │ │ │ └── fd: ()-->(50,58) + │ │ │ │ └── fd: ()-->(55,63) │ │ │ └── scan cardsinfo@cardsinfoversionindex,rev - │ │ │ ├── columns: dealerid:59!null version:67!null - │ │ │ ├── constraint: /59/67: [/4 - /4] + │ │ │ ├── columns: dealerid:65!null version:73!null + │ │ │ ├── constraint: /65/73: [/4 - /4] │ │ │ ├── limit: 1(rev) - │ │ │ ├── stats: [rows=1, distinct(59)=1, null(59)=0, distinct(59,67)=1, null(59,67)=0] + │ │ │ ├── stats: [rows=1, distinct(65)=1, null(65)=0, distinct(65,73)=1, null(65,73)=0] │ │ │ ├── key: () - │ │ │ └── fd: ()-->(59,67) + │ │ │ └── fd: ()-->(65,73) │ │ └── 1 │ └── aggregations - │ └── const-agg [as=max:30, outer=(15)] - │ └── version:15 + │ └── const-agg [as=max:33, outer=(16)] + │ └── version:16 └── projections - └── COALESCE(max:30, 0) [as=coalesce:31, outer=(30)] + └── COALESCE(max:33, 0) [as=coalesce:34, outer=(33)] # Show last 20 transactions for a particular card. # @@ -686,43 +686,43 @@ ORDER BY TransactionDate DESC LIMIT 20 ---- project - ├── columns: isbuy:2!null transactiondate:3!null cardid:4!null quantity:5!null sellprice:6!null buyprice:7!null isbuy2:10!null date:11!null accountname:12!null customername:13!null + ├── columns: isbuy:2!null transactiondate:3!null cardid:4!null quantity:5!null sellprice:6!null buyprice:7!null isbuy2:11!null date:12!null accountname:13!null customername:14!null ├── cardinality: [0 - 20] ├── stats: [rows=20] - ├── key: (5,11) - ├── fd: ()-->(2,4,10), (3,5)-->(6,7), (11)-->(12,13), (3)==(11), (11)==(3) - ├── ordering: -(3|11) opt(2,4,10) [actual: -3] + ├── key: (5,12) + ├── fd: ()-->(2,4,11), (3,5)-->(6,7), (12)-->(13,14), (3)==(12), (12)==(3) + ├── ordering: -(3|12) opt(2,4,11) [actual: -3] └── limit - ├── columns: transactiondetails.dealerid:1!null transactiondetails.isbuy:2!null transactiondate:3!null cardid:4!null quantity:5!null sellprice:6!null buyprice:7!null transactions.dealerid:9!null transactions.isbuy:10!null date:11!null accountname:12!null customername:13!null - ├── internal-ordering: -(3|11) opt(1,2,4,9,10) + ├── columns: transactiondetails.dealerid:1!null transactiondetails.isbuy:2!null transactiondate:3!null cardid:4!null quantity:5!null sellprice:6!null buyprice:7!null transactions.dealerid:10!null transactions.isbuy:11!null date:12!null accountname:13!null customername:14!null + ├── internal-ordering: -(3|12) opt(1,2,4,10,11) ├── cardinality: [0 - 20] ├── stats: [rows=20] - ├── key: (5,11) - ├── fd: ()-->(1,2,4,9,10), (3,5)-->(6,7), (11)-->(12,13), (3)==(11), (11)==(3) - ├── ordering: -(3|11) opt(1,2,4,9,10) [actual: -3] + ├── key: (5,12) + ├── fd: ()-->(1,2,4,10,11), (3,5)-->(6,7), (12)-->(13,14), (3)==(12), (12)==(3) + ├── ordering: -(3|12) opt(1,2,4,10,11) [actual: -3] ├── inner-join (lookup transactiondetails) - │ ├── columns: transactiondetails.dealerid:1!null transactiondetails.isbuy:2!null transactiondate:3!null cardid:4!null quantity:5!null sellprice:6!null buyprice:7!null transactions.dealerid:9!null transactions.isbuy:10!null date:11!null accountname:12!null customername:13!null + │ ├── columns: transactiondetails.dealerid:1!null transactiondetails.isbuy:2!null transactiondate:3!null cardid:4!null quantity:5!null sellprice:6!null buyprice:7!null transactions.dealerid:10!null transactions.isbuy:11!null date:12!null accountname:13!null customername:14!null │ ├── key columns: [1 2 3 4 5] = [1 2 3 4 5] │ ├── lookup columns are key - │ ├── stats: [rows=478.646617, distinct(3)=478.646617, null(3)=0, distinct(11)=478.646617, null(11)=0] - │ ├── key: (5,11) - │ ├── fd: ()-->(1,2,4,9,10), (3,5)-->(6,7), (11)-->(12,13), (3)==(11), (11)==(3) - │ ├── ordering: -(3|11) opt(1,2,4,9,10) [actual: -3] + │ ├── stats: [rows=478.646617, distinct(3)=478.646617, null(3)=0, distinct(12)=478.646617, null(12)=0] + │ ├── key: (5,12) + │ ├── fd: ()-->(1,2,4,10,11), (3,5)-->(6,7), (12)-->(13,14), (3)==(12), (12)==(3) + │ ├── ordering: -(3|12) opt(1,2,4,10,11) [actual: -3] │ ├── limit hint: 20.00 │ ├── inner-join (lookup transactions) - │ │ ├── columns: transactiondetails.dealerid:1!null transactiondetails.isbuy:2!null transactiondate:3!null cardid:4!null quantity:5!null transactions.dealerid:9!null transactions.isbuy:10!null date:11!null accountname:12!null customername:13!null - │ │ ├── key columns: [19 20 3] = [9 10 11] + │ │ ├── columns: transactiondetails.dealerid:1!null transactiondetails.isbuy:2!null transactiondate:3!null cardid:4!null quantity:5!null transactions.dealerid:10!null transactions.isbuy:11!null date:12!null accountname:13!null customername:14!null + │ │ ├── key columns: [21 22 3] = [10 11 12] │ │ ├── lookup columns are key - │ │ ├── stats: [rows=478.646617, distinct(3)=478.646617, null(3)=0, distinct(11)=478.646617, null(11)=0] - │ │ ├── key: (5,11) - │ │ ├── fd: ()-->(1,2,4,9,10), (11)-->(12,13), (3)==(11), (11)==(3) - │ │ ├── ordering: -(3|11) opt(1,2,4,9,10) [actual: -3] + │ │ ├── stats: [rows=478.646617, distinct(3)=478.646617, null(3)=0, distinct(12)=478.646617, null(12)=0] + │ │ ├── key: (5,12) + │ │ ├── fd: ()-->(1,2,4,10,11), (12)-->(13,14), (3)==(12), (12)==(3) + │ │ ├── ordering: -(3|12) opt(1,2,4,10,11) [actual: -3] │ │ ├── limit hint: 100.00 │ │ ├── project - │ │ │ ├── columns: "project_const_col_@9":19!null "project_const_col_@10":20!null transactiondetails.dealerid:1!null transactiondetails.isbuy:2!null transactiondate:3!null cardid:4!null quantity:5!null + │ │ │ ├── columns: "project_const_col_@10":21!null "project_const_col_@11":22!null transactiondetails.dealerid:1!null transactiondetails.isbuy:2!null transactiondate:3!null cardid:4!null quantity:5!null │ │ │ ├── stats: [rows=478.646617] │ │ │ ├── key: (3,5) - │ │ │ ├── fd: ()-->(1,2,4,19,20) + │ │ │ ├── fd: ()-->(1,2,4,21,22) │ │ │ ├── ordering: -3 opt(1,2,4) [actual: -3] │ │ │ ├── limit hint: 100.00 │ │ │ ├── scan transactiondetails@detailscardidindex,rev @@ -734,8 +734,8 @@ project │ │ │ │ ├── ordering: -3 opt(1,2,4) [actual: -3] │ │ │ │ └── limit hint: 100.00 │ │ │ └── projections - │ │ │ ├── 1 [as="project_const_col_@9":19] - │ │ │ └── false [as="project_const_col_@10":20] + │ │ │ ├── 1 [as="project_const_col_@10":21] + │ │ │ └── false [as="project_const_col_@11":22] │ │ └── filters (true) │ └── filters (true) └── 20 @@ -798,57 +798,57 @@ ORDER BY Name, SetName, Number LIMIT 50 ---- project - ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null buyprice:9!null sellprice:10!null desiredinventory:12!null actualinventory:13!null version:15!null discount:11!null maxinventory:14!null twodaysales:25 + ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null buyprice:10!null sellprice:11!null desiredinventory:13!null actualinventory:14!null version:16!null discount:12!null maxinventory:15!null twodaysales:28 ├── cardinality: [0 - 50] ├── immutable ├── stats: [rows=50] - ├── key: (15,25) - ├── fd: (1)-->(2-6,9-15), (2,4,5)~~>(1,3,6), (15)-->(1-6,9-14) + ├── key: (16,28) + ├── fd: (1)-->(2-6,10-16), (2,4,5)~~>(1,3,6), (16)-->(1-6,10-15) ├── ordering: +2,+4,+5 ├── limit - │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null cardsinfo.cardid:8!null cardsinfo.buyprice:9!null cardsinfo.sellprice:10!null discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null cardsinfo.version:15!null sum:24 + │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null cardsinfo.cardid:9!null cardsinfo.buyprice:10!null cardsinfo.sellprice:11!null discount:12!null desiredinventory:13!null actualinventory:14!null maxinventory:15!null cardsinfo.version:16!null sum:27 │ ├── internal-ordering: +2,+4,+5 │ ├── cardinality: [0 - 50] │ ├── immutable │ ├── stats: [rows=50] - │ ├── key: (8) - │ ├── fd: (1)-->(2-6), (2,4,5)~~>(1,3,6), (8)-->(1-6,9-15,24), (15)-->(8-14), (1)==(8), (8)==(1) + │ ├── key: (9) + │ ├── fd: (1)-->(2-6), (2,4,5)~~>(1,3,6), (9)-->(1-6,10-16,27), (16)-->(9-15), (1)==(9), (9)==(1) │ ├── ordering: +2,+4,+5 │ ├── sort - │ │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null cardsinfo.cardid:8!null cardsinfo.buyprice:9!null cardsinfo.sellprice:10!null discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null cardsinfo.version:15!null sum:24 + │ │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null cardsinfo.cardid:9!null cardsinfo.buyprice:10!null cardsinfo.sellprice:11!null discount:12!null desiredinventory:13!null actualinventory:14!null maxinventory:15!null cardsinfo.version:16!null sum:27 │ │ ├── immutable - │ │ ├── stats: [rows=19000, distinct(8)=19000, null(8)=0] - │ │ ├── key: (8) - │ │ ├── fd: (1)-->(2-6), (2,4,5)~~>(1,3,6), (8)-->(1-6,9-15,24), (15)-->(8-14), (1)==(8), (8)==(1) + │ │ ├── stats: [rows=19000, distinct(9)=19000, null(9)=0] + │ │ ├── key: (9) + │ │ ├── fd: (1)-->(2-6), (2,4,5)~~>(1,3,6), (9)-->(1-6,10-16,27), (16)-->(9-15), (1)==(9), (9)==(1) │ │ ├── ordering: +2,+4,+5 │ │ ├── limit hint: 50.00 │ │ └── group-by - │ │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null cardsinfo.cardid:8!null cardsinfo.buyprice:9!null cardsinfo.sellprice:10!null discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null cardsinfo.version:15!null sum:24 - │ │ ├── grouping columns: cardsinfo.cardid:8!null + │ │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null cardsinfo.cardid:9!null cardsinfo.buyprice:10!null cardsinfo.sellprice:11!null discount:12!null desiredinventory:13!null actualinventory:14!null maxinventory:15!null cardsinfo.version:16!null sum:27 + │ │ ├── grouping columns: cardsinfo.cardid:9!null │ │ ├── immutable - │ │ ├── stats: [rows=19000, distinct(8)=19000, null(8)=0] - │ │ ├── key: (8) - │ │ ├── fd: (1)-->(2-6), (2,4,5)~~>(1,3,6), (8)-->(1-6,9-15,24), (15)-->(8-14), (1)==(8), (8)==(1) + │ │ ├── stats: [rows=19000, distinct(9)=19000, null(9)=0] + │ │ ├── key: (9) + │ │ ├── fd: (1)-->(2-6), (2,4,5)~~>(1,3,6), (9)-->(1-6,10-16,27), (16)-->(9-15), (1)==(9), (9)==(1) │ │ ├── right-join (hash) - │ │ │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null cardsinfo.dealerid:7!null cardsinfo.cardid:8!null cardsinfo.buyprice:9!null cardsinfo.sellprice:10!null discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null cardsinfo.version:15!null transactiondetails.dealerid:16 isbuy:17 transactiondate:18 transactiondetails.cardid:19 quantity:20 + │ │ │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null cardsinfo.dealerid:8!null cardsinfo.cardid:9!null cardsinfo.buyprice:10!null cardsinfo.sellprice:11!null discount:12!null desiredinventory:13!null actualinventory:14!null maxinventory:15!null cardsinfo.version:16!null transactiondetails.dealerid:18 isbuy:19 transactiondate:20 transactiondetails.cardid:21 quantity:22 │ │ │ ├── immutable - │ │ │ ├── stats: [rows=5523583.18, distinct(8)=19000, null(8)=0, distinct(19)=19000, null(19)=0] - │ │ │ ├── key: (8,18-20) - │ │ │ ├── fd: ()-->(7), (1)-->(2-6), (2,4,5)~~>(1,3,6), (8)-->(9-15), (15)-->(8-14), (1)==(8), (8)==(1), (8,18-20)-->(16,17) + │ │ │ ├── stats: [rows=5523583.18, distinct(9)=19000, null(9)=0, distinct(21)=19000, null(21)=0] + │ │ │ ├── key: (9,20-22) + │ │ │ ├── fd: ()-->(8), (1)-->(2-6), (2,4,5)~~>(1,3,6), (9)-->(10-16), (16)-->(9-15), (1)==(9), (9)==(1), (9,20-22)-->(18,19) │ │ │ ├── scan transactiondetails - │ │ │ │ ├── columns: transactiondetails.dealerid:16!null isbuy:17!null transactiondate:18!null transactiondetails.cardid:19!null quantity:20!null - │ │ │ │ ├── constraint: /16/17/18/19/20: [/1/false/'2020-02-28 00:00:00+00:00' - /1/false/'2020-03-01 00:00:00+00:00'] - │ │ │ │ ├── stats: [rows=10630000, distinct(16)=1, null(16)=0, distinct(17)=1, null(17)=0, distinct(18)=10630000, null(18)=0, distinct(19)=57000, null(19)=0, distinct(16,17)=1, null(16,17)=0, distinct(16-18)=10630000, null(16-18)=0] - │ │ │ │ ├── key: (18-20) - │ │ │ │ └── fd: ()-->(16,17) + │ │ │ │ ├── columns: transactiondetails.dealerid:18!null isbuy:19!null transactiondate:20!null transactiondetails.cardid:21!null quantity:22!null + │ │ │ │ ├── constraint: /18/19/20/21/22: [/1/false/'2020-02-28 00:00:00+00:00' - /1/false/'2020-03-01 00:00:00+00:00'] + │ │ │ │ ├── stats: [rows=10630000, distinct(18)=1, null(18)=0, distinct(19)=1, null(19)=0, distinct(20)=10630000, null(20)=0, distinct(21)=57000, null(21)=0, distinct(18,19)=1, null(18,19)=0, distinct(18-20)=10630000, null(18-20)=0] + │ │ │ │ ├── key: (20-22) + │ │ │ │ └── fd: ()-->(18,19) │ │ │ ├── inner-join (merge) - │ │ │ │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null cardsinfo.dealerid:7!null cardsinfo.cardid:8!null cardsinfo.buyprice:9!null cardsinfo.sellprice:10!null discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null cardsinfo.version:15!null + │ │ │ │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null cardsinfo.dealerid:8!null cardsinfo.cardid:9!null cardsinfo.buyprice:10!null cardsinfo.sellprice:11!null discount:12!null desiredinventory:13!null actualinventory:14!null maxinventory:15!null cardsinfo.version:16!null │ │ │ │ ├── left ordering: +1 - │ │ │ │ ├── right ordering: +8 + │ │ │ │ ├── right ordering: +9 │ │ │ │ ├── immutable - │ │ │ │ ├── stats: [rows=29618.4611, distinct(1)=19000, null(1)=0, distinct(2)=11668.1409, null(2)=0, distinct(5)=829, null(5)=0, distinct(6)=5572.85686, null(6)=0, distinct(7)=1, null(7)=0, distinct(8)=19000, null(8)=0, distinct(9)=21037.9959, null(9)=0, distinct(10)=21037.9959, null(10)=0, distinct(11)=21037.9959, null(11)=0, distinct(12)=21037.9959, null(12)=0, distinct(13)=21037.9959, null(13)=0, distinct(14)=21037.9959, null(14)=0, distinct(15)=23225.5851, null(15)=0] - │ │ │ │ ├── key: (8) - │ │ │ │ ├── fd: ()-->(7), (1)-->(2-6), (2,4,5)~~>(1,3,6), (8)-->(9-15), (15)-->(8-14), (1)==(8), (8)==(1) + │ │ │ │ ├── stats: [rows=29618.4611, distinct(1)=19000, null(1)=0, distinct(2)=11668.1409, null(2)=0, distinct(5)=829, null(5)=0, distinct(6)=5572.85686, null(6)=0, distinct(8)=1, null(8)=0, distinct(9)=19000, null(9)=0, distinct(10)=21037.9959, null(10)=0, distinct(11)=21037.9959, null(11)=0, distinct(12)=21037.9959, null(12)=0, distinct(13)=21037.9959, null(13)=0, distinct(14)=21037.9959, null(14)=0, distinct(15)=21037.9959, null(15)=0, distinct(16)=23225.5851, null(16)=0] + │ │ │ │ ├── key: (9) + │ │ │ │ ├── fd: ()-->(8), (1)-->(2-6), (2,4,5)~~>(1,3,6), (9)-->(10-16), (16)-->(9-15), (1)==(9), (9)==(1) │ │ │ │ ├── select │ │ │ │ │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null │ │ │ │ │ ├── immutable @@ -865,18 +865,18 @@ project │ │ │ │ │ └── filters │ │ │ │ │ └── (name:2, setname:4, number:5) > ('Shock', '7E', 248) [outer=(2,4,5), immutable, constraints=(/2/4/5: [/'Shock'/'7E'/249 - ]; tight)] │ │ │ │ ├── scan cardsinfo - │ │ │ │ │ ├── columns: cardsinfo.dealerid:7!null cardsinfo.cardid:8!null cardsinfo.buyprice:9!null cardsinfo.sellprice:10!null discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null cardsinfo.version:15!null - │ │ │ │ │ ├── constraint: /7/8: [/1 - /1] - │ │ │ │ │ ├── stats: [rows=58333.3333, distinct(7)=1, null(7)=0, distinct(8)=37420.3552, null(8)=0, distinct(9)=40676.7278, null(9)=0, distinct(10)=40676.7278, null(10)=0, distinct(11)=40676.7278, null(11)=0, distinct(12)=40676.7278, null(12)=0, distinct(13)=40676.7278, null(13)=0, distinct(14)=40676.7278, null(14)=0, distinct(15)=58333.3333, null(15)=0] - │ │ │ │ │ ├── key: (8) - │ │ │ │ │ ├── fd: ()-->(7), (8)-->(9-15), (15)-->(8-14) - │ │ │ │ │ └── ordering: +8 opt(7) [actual: +8] + │ │ │ │ │ ├── columns: cardsinfo.dealerid:8!null cardsinfo.cardid:9!null cardsinfo.buyprice:10!null cardsinfo.sellprice:11!null discount:12!null desiredinventory:13!null actualinventory:14!null maxinventory:15!null cardsinfo.version:16!null + │ │ │ │ │ ├── constraint: /8/9: [/1 - /1] + │ │ │ │ │ ├── stats: [rows=58333.3333, distinct(8)=1, null(8)=0, distinct(9)=37420.3552, null(9)=0, distinct(10)=40676.7278, null(10)=0, distinct(11)=40676.7278, null(11)=0, distinct(12)=40676.7278, null(12)=0, distinct(13)=40676.7278, null(13)=0, distinct(14)=40676.7278, null(14)=0, distinct(15)=40676.7278, null(15)=0, distinct(16)=58333.3333, null(16)=0] + │ │ │ │ │ ├── key: (9) + │ │ │ │ │ ├── fd: ()-->(8), (9)-->(10-16), (16)-->(9-15) + │ │ │ │ │ └── ordering: +9 opt(8) [actual: +9] │ │ │ │ └── filters (true) │ │ │ └── filters - │ │ │ └── transactiondetails.cardid:19 = id:1 [outer=(1,19), constraints=(/1: (/NULL - ]; /19: (/NULL - ]), fd=(1)==(19), (19)==(1)] + │ │ │ └── transactiondetails.cardid:21 = id:1 [outer=(1,21), constraints=(/1: (/NULL - ]; /21: (/NULL - ]), fd=(1)==(21), (21)==(1)] │ │ └── aggregations - │ │ ├── sum [as=sum:24, outer=(20)] - │ │ │ └── quantity:20 + │ │ ├── sum [as=sum:27, outer=(22)] + │ │ │ └── quantity:22 │ │ ├── const-agg [as=id:1, outer=(1)] │ │ │ └── id:1 │ │ ├── const-agg [as=name:2, outer=(2)] @@ -889,23 +889,23 @@ project │ │ │ └── number:5 │ │ ├── const-agg [as=isfoil:6, outer=(6)] │ │ │ └── isfoil:6 - │ │ ├── const-agg [as=cardsinfo.buyprice:9, outer=(9)] - │ │ │ └── cardsinfo.buyprice:9 - │ │ ├── const-agg [as=cardsinfo.sellprice:10, outer=(10)] - │ │ │ └── cardsinfo.sellprice:10 - │ │ ├── const-agg [as=discount:11, outer=(11)] - │ │ │ └── discount:11 - │ │ ├── const-agg [as=desiredinventory:12, outer=(12)] - │ │ │ └── desiredinventory:12 - │ │ ├── const-agg [as=actualinventory:13, outer=(13)] - │ │ │ └── actualinventory:13 - │ │ ├── const-agg [as=maxinventory:14, outer=(14)] - │ │ │ └── maxinventory:14 - │ │ └── const-agg [as=cardsinfo.version:15, outer=(15)] - │ │ └── cardsinfo.version:15 + │ │ ├── const-agg [as=cardsinfo.buyprice:10, outer=(10)] + │ │ │ └── cardsinfo.buyprice:10 + │ │ ├── const-agg [as=cardsinfo.sellprice:11, outer=(11)] + │ │ │ └── cardsinfo.sellprice:11 + │ │ ├── const-agg [as=discount:12, outer=(12)] + │ │ │ └── discount:12 + │ │ ├── const-agg [as=desiredinventory:13, outer=(13)] + │ │ │ └── desiredinventory:13 + │ │ ├── const-agg [as=actualinventory:14, outer=(14)] + │ │ │ └── actualinventory:14 + │ │ ├── const-agg [as=maxinventory:15, outer=(15)] + │ │ │ └── maxinventory:15 + │ │ └── const-agg [as=cardsinfo.version:16, outer=(16)] + │ │ └── cardsinfo.version:16 │ └── 50 └── projections - └── COALESCE(sum:24, 0) [as=value:25, outer=(24)] + └── COALESCE(sum:27, 0) [as=value:28, outer=(27)] # Daily transaction query. # @@ -942,35 +942,35 @@ GROUP BY extract(day from d.TransactionDate) ORDER BY extract(day from d.TransactionDate) ---- sort - ├── columns: extract:37 totalsell:32!null totalbuy:34!null totalprofit:36!null + ├── columns: extract:41 totalsell:36!null totalbuy:38!null totalprofit:40!null ├── stable - ├── stats: [rows=750327.164, distinct(37)=750327.164, null(37)=0] - ├── key: (37) - ├── fd: (37)-->(32,34,36) - ├── ordering: +37 + ├── stats: [rows=750327.164, distinct(41)=750327.164, null(41)=0] + ├── key: (41) + ├── fd: (41)-->(36,38,40) + ├── ordering: +41 └── group-by - ├── columns: sum:32!null sum:34!null sum:36!null column37:37 - ├── grouping columns: column37:37 + ├── columns: sum:36!null sum:38!null sum:40!null column41:41 + ├── grouping columns: column41:41 ├── stable - ├── stats: [rows=750327.164, distinct(37)=750327.164, null(37)=0] - ├── key: (37) - ├── fd: (37)-->(32,34,36) + ├── stats: [rows=750327.164, distinct(41)=750327.164, null(41)=0] + ├── key: (41) + ├── fd: (41)-->(36,38,40) ├── project - │ ├── columns: column31:31!null column33:33!null column35:35!null column37:37 + │ ├── columns: column35:35!null column37:37!null column39:39!null column41:41 │ ├── stable - │ ├── stats: [rows=1198631.87, distinct(37)=750327.164, null(37)=0] + │ ├── stats: [rows=1198631.87, distinct(41)=750327.164, null(41)=0] │ ├── inner-join (hash) - │ │ ├── columns: transactiondetails.dealerid:1!null transactiondetails.isbuy:2!null transactiondate:3!null transactiondetails.cardid:4!null quantity:5!null transactiondetails.sellprice:6!null transactiondetails.buyprice:7!null transactions.dealerid:9!null transactions.isbuy:10!null date:11!null accountname:12!null customername:13!null id:16!null cardsinfo.dealerid:22!null cardsinfo.cardid:23!null + │ │ ├── columns: transactiondetails.dealerid:1!null transactiondetails.isbuy:2!null transactiondate:3!null transactiondetails.cardid:4!null quantity:5!null transactiondetails.sellprice:6!null transactiondetails.buyprice:7!null transactions.dealerid:10!null transactions.isbuy:11!null date:12!null accountname:13!null customername:14!null id:18!null cardsinfo.dealerid:25!null cardsinfo.cardid:26!null │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ ├── stats: [rows=1198631.87, distinct(3)=750327.164, null(3)=0, distinct(4)=37420.3552, null(4)=0, distinct(16)=37420.3552, null(16)=0] - │ │ ├── key: (5,11,23) - │ │ ├── fd: ()-->(1,2,9,10,22), (3-5)-->(6,7), (11)-->(12,13), (3)==(11), (11)==(3), (16)==(4,23), (23)==(4,16), (4)==(16,23) + │ │ ├── stats: [rows=1198631.87, distinct(3)=750327.164, null(3)=0, distinct(4)=37420.3552, null(4)=0, distinct(18)=37420.3552, null(18)=0] + │ │ ├── key: (5,12,26) + │ │ ├── fd: ()-->(1,2,10,11,25), (3-5)-->(6,7), (12)-->(13,14), (3)==(12), (12)==(3), (18)==(4,26), (26)==(4,18), (4)==(18,26) │ │ ├── inner-join (hash) - │ │ │ ├── columns: transactiondetails.dealerid:1!null transactiondetails.isbuy:2!null transactiondate:3!null transactiondetails.cardid:4!null quantity:5!null transactiondetails.sellprice:6!null transactiondetails.buyprice:7!null transactions.dealerid:9!null transactions.isbuy:10!null date:11!null accountname:12!null customername:13!null + │ │ │ ├── columns: transactiondetails.dealerid:1!null transactiondetails.isbuy:2!null transactiondate:3!null transactiondetails.cardid:4!null quantity:5!null transactiondetails.sellprice:6!null transactiondetails.buyprice:7!null transactions.dealerid:10!null transactions.isbuy:11!null date:12!null accountname:13!null customername:14!null │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ ├── stats: [rows=1171234.57, distinct(1)=1, null(1)=0, distinct(2)=1, null(2)=0, distinct(3)=1171234.57, null(3)=0, distinct(4)=56999.9999, null(4)=0, distinct(5)=1091498.71, null(5)=0, distinct(6)=1091498.71, null(6)=0, distinct(7)=1091498.71, null(7)=0, distinct(9)=1, null(9)=0, distinct(10)=1, null(10)=0, distinct(11)=1171234.57, null(11)=0, distinct(12)=551608.449, null(12)=0, distinct(13)=551608.449, null(13)=0] - │ │ │ ├── key: (4,5,11) - │ │ │ ├── fd: ()-->(1,2,9,10), (3-5)-->(6,7), (11)-->(12,13), (3)==(11), (11)==(3) + │ │ │ ├── stats: [rows=1171234.57, distinct(1)=1, null(1)=0, distinct(2)=1, null(2)=0, distinct(3)=1171234.57, null(3)=0, distinct(4)=56999.9999, null(4)=0, distinct(5)=1091498.71, null(5)=0, distinct(6)=1091498.71, null(6)=0, distinct(7)=1091498.71, null(7)=0, distinct(10)=1, null(10)=0, distinct(11)=1, null(11)=0, distinct(12)=1171234.57, null(12)=0, distinct(13)=551608.449, null(13)=0, distinct(14)=551608.449, null(14)=0] + │ │ │ ├── key: (4,5,12) + │ │ │ ├── fd: ()-->(1,2,10,11), (3-5)-->(6,7), (12)-->(13,14), (3)==(12), (12)==(3) │ │ │ ├── scan transactiondetails │ │ │ │ ├── columns: transactiondetails.dealerid:1!null transactiondetails.isbuy:2!null transactiondate:3!null transactiondetails.cardid:4!null quantity:5!null transactiondetails.sellprice:6!null transactiondetails.buyprice:7!null │ │ │ │ ├── constraint: /1/2/3/4/5: [/1/false/'2020-02-23 00:00:00+00:00' - /1/false/'2020-03-01 00:00:00+00:00'] @@ -978,53 +978,53 @@ sort │ │ │ │ ├── key: (3-5) │ │ │ │ └── fd: ()-->(1,2), (3-5)-->(6,7) │ │ │ ├── select - │ │ │ │ ├── columns: transactions.dealerid:9!null transactions.isbuy:10!null date:11!null accountname:12!null customername:13!null - │ │ │ │ ├── stats: [rows=1171234.57, distinct(9)=1, null(9)=0, distinct(10)=1, null(10)=0, distinct(11)=1171234.57, null(11)=0, distinct(12)=666666.667, null(12)=0, distinct(13)=666666.667, null(13)=0, distinct(9,10)=1, null(9,10)=0, distinct(9-13)=1171234.57, null(9-13)=0] - │ │ │ │ ├── key: (11) - │ │ │ │ ├── fd: ()-->(9,10), (11)-->(12,13) + │ │ │ │ ├── columns: transactions.dealerid:10!null transactions.isbuy:11!null date:12!null accountname:13!null customername:14!null + │ │ │ │ ├── stats: [rows=1171234.57, distinct(10)=1, null(10)=0, distinct(11)=1, null(11)=0, distinct(12)=1171234.57, null(12)=0, distinct(13)=666666.667, null(13)=0, distinct(14)=666666.667, null(14)=0, distinct(10,11)=1, null(10,11)=0, distinct(10-14)=1171234.57, null(10-14)=0] + │ │ │ │ ├── key: (12) + │ │ │ │ ├── fd: ()-->(10,11), (12)-->(13,14) │ │ │ │ ├── scan transactions - │ │ │ │ │ ├── columns: transactions.dealerid:9!null transactions.isbuy:10!null date:11!null accountname:12!null customername:13!null - │ │ │ │ │ ├── constraint: /9/10/11: [/1/false/'2020-02-23 00:00:00+00:00' - /1/false/'2020-03-01 00:00:00+00:00'] - │ │ │ │ │ ├── stats: [rows=1181111.11, distinct(9)=1, null(9)=0, distinct(10)=1, null(10)=0, distinct(11)=1181111.11, null(11)=0, distinct(9,10)=1, null(9,10)=0, distinct(9-11)=1181111.11, null(9-11)=0] - │ │ │ │ │ ├── key: (11) - │ │ │ │ │ └── fd: ()-->(9,10), (11)-->(12,13) + │ │ │ │ │ ├── columns: transactions.dealerid:10!null transactions.isbuy:11!null date:12!null accountname:13!null customername:14!null + │ │ │ │ │ ├── constraint: /10/11/12: [/1/false/'2020-02-23 00:00:00+00:00' - /1/false/'2020-03-01 00:00:00+00:00'] + │ │ │ │ │ ├── stats: [rows=1181111.11, distinct(10)=1, null(10)=0, distinct(11)=1, null(11)=0, distinct(12)=1181111.11, null(12)=0, distinct(10,11)=1, null(10,11)=0, distinct(10-12)=1181111.11, null(10-12)=0] + │ │ │ │ │ ├── key: (12) + │ │ │ │ │ └── fd: ()-->(10,11), (12)-->(13,14) │ │ │ │ └── filters - │ │ │ │ ├── accountname:12 != 'someaccount' [outer=(12), constraints=(/12: (/NULL - /'someaccount') [/e'someaccount\x00' - ]; tight)] - │ │ │ │ └── customername:13 != 'somecustomer' [outer=(13), constraints=(/13: (/NULL - /'somecustomer') [/e'somecustomer\x00' - ]; tight)] + │ │ │ │ ├── accountname:13 != 'someaccount' [outer=(13), constraints=(/13: (/NULL - /'someaccount') [/e'someaccount\x00' - ]; tight)] + │ │ │ │ └── customername:14 != 'somecustomer' [outer=(14), constraints=(/14: (/NULL - /'somecustomer') [/e'somecustomer\x00' - ]; tight)] │ │ │ └── filters - │ │ │ └── transactiondate:3 = date:11 [outer=(3,11), constraints=(/3: (/NULL - ]; /11: (/NULL - ]), fd=(3)==(11), (11)==(3)] + │ │ │ └── transactiondate:3 = date:12 [outer=(3,12), constraints=(/3: (/NULL - ]; /12: (/NULL - ]), fd=(3)==(12), (12)==(3)] │ │ ├── inner-join (hash) - │ │ │ ├── columns: id:16!null cardsinfo.dealerid:22!null cardsinfo.cardid:23!null + │ │ │ ├── columns: id:18!null cardsinfo.dealerid:25!null cardsinfo.cardid:26!null │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) - │ │ │ ├── stats: [rows=58333.3333, distinct(16)=37420.3552, null(16)=0, distinct(22)=1, null(22)=0, distinct(23)=37420.3552, null(23)=0] - │ │ │ ├── key: (23) - │ │ │ ├── fd: ()-->(22), (16)==(23), (23)==(16) + │ │ │ ├── stats: [rows=58333.3333, distinct(18)=37420.3552, null(18)=0, distinct(25)=1, null(25)=0, distinct(26)=37420.3552, null(26)=0] + │ │ │ ├── key: (26) + │ │ │ ├── fd: ()-->(25), (18)==(26), (26)==(18) │ │ │ ├── scan cardsinfo@cardsinfoversionindex - │ │ │ │ ├── columns: cardsinfo.dealerid:22!null cardsinfo.cardid:23!null - │ │ │ │ ├── constraint: /22/30: [/1 - /1] - │ │ │ │ ├── stats: [rows=58333.3333, distinct(22)=1, null(22)=0, distinct(23)=37420.3552, null(23)=0] - │ │ │ │ ├── key: (23) - │ │ │ │ └── fd: ()-->(22) + │ │ │ │ ├── columns: cardsinfo.dealerid:25!null cardsinfo.cardid:26!null + │ │ │ │ ├── constraint: /25/33: [/1 - /1] + │ │ │ │ ├── stats: [rows=58333.3333, distinct(25)=1, null(25)=0, distinct(26)=37420.3552, null(26)=0] + │ │ │ │ ├── key: (26) + │ │ │ │ └── fd: ()-->(25) │ │ │ ├── scan cards@cardsnamesetnumber - │ │ │ │ ├── columns: id:16!null - │ │ │ │ ├── stats: [rows=57000, distinct(16)=57000, null(16)=0] - │ │ │ │ └── key: (16) + │ │ │ │ ├── columns: id:18!null + │ │ │ │ ├── stats: [rows=57000, distinct(18)=57000, null(18)=0] + │ │ │ │ └── key: (18) │ │ │ └── filters - │ │ │ └── id:16 = cardsinfo.cardid:23 [outer=(16,23), constraints=(/16: (/NULL - ]; /23: (/NULL - ]), fd=(16)==(23), (23)==(16)] + │ │ │ └── id:18 = cardsinfo.cardid:26 [outer=(18,26), constraints=(/18: (/NULL - ]; /26: (/NULL - ]), fd=(18)==(26), (26)==(18)] │ │ └── filters - │ │ └── id:16 = transactiondetails.cardid:4 [outer=(4,16), constraints=(/4: (/NULL - ]; /16: (/NULL - ]), fd=(4)==(16), (16)==(4)] + │ │ └── id:18 = transactiondetails.cardid:4 [outer=(4,18), constraints=(/4: (/NULL - ]; /18: (/NULL - ]), fd=(4)==(18), (18)==(4)] │ └── projections - │ ├── quantity:5 * transactiondetails.sellprice:6::DECIMAL [as=column31:31, outer=(5,6), immutable] - │ ├── quantity:5 * transactiondetails.buyprice:7::DECIMAL [as=column33:33, outer=(5,7), immutable] - │ ├── quantity:5 * (transactiondetails.sellprice:6::DECIMAL - transactiondetails.buyprice:7::DECIMAL) [as=column35:35, outer=(5-7), immutable] - │ └── extract('day', transactiondate:3) [as=column37:37, outer=(3), stable] + │ ├── quantity:5 * transactiondetails.sellprice:6::DECIMAL [as=column35:35, outer=(5,6), immutable] + │ ├── quantity:5 * transactiondetails.buyprice:7::DECIMAL [as=column37:37, outer=(5,7), immutable] + │ ├── quantity:5 * (transactiondetails.sellprice:6::DECIMAL - transactiondetails.buyprice:7::DECIMAL) [as=column39:39, outer=(5-7), immutable] + │ └── extract('day', transactiondate:3) [as=column41:41, outer=(3), stable] └── aggregations - ├── sum [as=sum:32, outer=(31)] - │ └── column31:31 - ├── sum [as=sum:34, outer=(33)] - │ └── column33:33 - └── sum [as=sum:36, outer=(35)] - └── column35:35 + ├── sum [as=sum:36, outer=(35)] + │ └── column35:35 + ├── sum [as=sum:38, outer=(37)] + │ └── column37:37 + └── sum [as=sum:40, outer=(39)] + └── column39:39 # Check if transaction was already inserted, for idempotency. # @@ -1045,21 +1045,21 @@ SELECT ) AS AlreadyInserted ---- values - ├── columns: alreadyinserted:9 + ├── columns: alreadyinserted:10 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(9) + ├── fd: ()-->(10) └── tuple └── any: eq ├── project - │ ├── columns: coalesce:8 + │ ├── columns: coalesce:9 │ ├── scan transactions │ │ ├── columns: dealerid:1!null isbuy:2!null operationid:6 │ │ ├── constraint: /1/2/3: [/1/false - /1/false] │ │ ├── lax-key: (6) │ │ └── fd: ()-->(1,2) │ └── projections - │ └── COALESCE(operationid:6, '00000000-0000-0000-0000-000000000000') [as=coalesce:8, outer=(6)] + │ └── COALESCE(operationid:6, '00000000-0000-0000-0000-000000000000') [as=coalesce:9, outer=(6)] └── '70f03eb1-4f58-4c26-b72d-c524a9d537dd' # Get account locations of a list of cards. @@ -1088,25 +1088,25 @@ ORDER BY sum(Quantity) DESC LIMIT 1000 ---- limit - ├── columns: accountname:10!null quantity:14!null - ├── internal-ordering: -14 + ├── columns: accountname:10!null quantity:15!null + ├── internal-ordering: -15 ├── cardinality: [0 - 1000] ├── key: (10) - ├── fd: (10)-->(14) - ├── ordering: -14 + ├── fd: (10)-->(15) + ├── ordering: -15 ├── sort - │ ├── columns: accountname:10!null sum:14!null + │ ├── columns: accountname:10!null sum:15!null │ ├── key: (10) - │ ├── fd: (10)-->(14) - │ ├── ordering: -14 + │ ├── fd: (10)-->(15) + │ ├── ordering: -15 │ ├── limit hint: 1000.00 │ └── group-by - │ ├── columns: accountname:10!null sum:14!null + │ ├── columns: accountname:10!null sum:15!null │ ├── grouping columns: accountname:10!null │ ├── key: (10) - │ ├── fd: (10)-->(14) + │ ├── fd: (10)-->(15) │ ├── project - │ │ ├── columns: quantity:13!null accountname:10!null + │ │ ├── columns: quantity:14!null accountname:10!null │ │ ├── inner-join (lookup inventorydetails) │ │ │ ├── columns: id:6!null quantity:7!null dealerid:8!null cardid:9!null accountname:10!null inventorydetails.quantity:11!null │ │ │ ├── key columns: [8 9 10] = [8 9 10] @@ -1126,10 +1126,10 @@ limit │ │ │ │ └── accountname:10 IN ('account-1', 'account-2', 'account-3') [outer=(10), constraints=(/10: [/'account-1' - /'account-1'] [/'account-2' - /'account-2'] [/'account-3' - /'account-3']; tight)] │ │ │ └── filters (true) │ │ └── projections - │ │ └── CASE WHEN quantity:7 < inventorydetails.quantity:11 THEN quantity:7 ELSE inventorydetails.quantity:11 END [as=quantity:13, outer=(7,11)] + │ │ └── CASE WHEN quantity:7 < inventorydetails.quantity:11 THEN quantity:7 ELSE inventorydetails.quantity:11 END [as=quantity:14, outer=(7,11)] │ └── aggregations - │ └── sum [as=sum:14, outer=(13)] - │ └── quantity:13 + │ └── sum [as=sum:15, outer=(14)] + │ └── quantity:14 └── 1000 # Get buy/sell volume of a particular card in the last 2 days. @@ -1158,31 +1158,31 @@ SELECT coalesce(( ), 0) ---- values - ├── columns: coalesce:17 + ├── columns: coalesce:19 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(17) + ├── fd: ()-->(19) └── tuple └── coalesce ├── subquery │ └── scalar-group-by - │ ├── columns: sum:16 + │ ├── columns: sum:18 │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(16) + │ ├── fd: ()-->(18) │ ├── limit - │ │ ├── columns: d.dealerid:1!null d.isbuy:2!null transactiondate:3!null cardid:4!null quantity:5!null t.dealerid:9!null t.isbuy:10!null date:11!null - │ │ ├── internal-ordering: -(3|11) opt(4) + │ │ ├── columns: d.dealerid:1!null d.isbuy:2!null transactiondate:3!null cardid:4!null quantity:5!null t.dealerid:10!null t.isbuy:11!null date:12!null + │ │ ├── internal-ordering: -(3|12) opt(4) │ │ ├── cardinality: [0 - 100] - │ │ ├── key: (5,9-11) - │ │ ├── fd: ()-->(4), (1)==(9), (9)==(1), (2)==(10), (10)==(2), (3)==(11), (11)==(3) + │ │ ├── key: (5,10-12) + │ │ ├── fd: ()-->(4), (1)==(10), (10)==(1), (2)==(11), (11)==(2), (3)==(12), (12)==(3) │ │ ├── inner-join (lookup transactions) - │ │ │ ├── columns: d.dealerid:1!null d.isbuy:2!null transactiondate:3!null cardid:4!null quantity:5!null t.dealerid:9!null t.isbuy:10!null date:11!null - │ │ │ ├── key columns: [1 2 3] = [9 10 11] + │ │ │ ├── columns: d.dealerid:1!null d.isbuy:2!null transactiondate:3!null cardid:4!null quantity:5!null t.dealerid:10!null t.isbuy:11!null date:12!null + │ │ │ ├── key columns: [1 2 3] = [10 11 12] │ │ │ ├── lookup columns are key - │ │ │ ├── key: (5,9-11) - │ │ │ ├── fd: ()-->(4), (1)==(9), (9)==(1), (2)==(10), (10)==(2), (3)==(11), (11)==(3) - │ │ │ ├── ordering: -(3|11) opt(4) [actual: -3] + │ │ │ ├── key: (5,10-12) + │ │ │ ├── fd: ()-->(4), (1)==(10), (10)==(1), (2)==(11), (11)==(2), (3)==(12), (12)==(3) + │ │ │ ├── ordering: -(3|12) opt(4) [actual: -3] │ │ │ ├── limit hint: 100.00 │ │ │ ├── sort │ │ │ │ ├── columns: d.dealerid:1!null d.isbuy:2!null transactiondate:3!null cardid:4!null quantity:5!null @@ -1206,12 +1206,12 @@ values │ │ │ │ ├── key: (1-3,5) │ │ │ │ └── fd: ()-->(4) │ │ │ └── filters - │ │ │ ├── (date:11 >= '2020-02-28 00:00:00+00:00') AND (date:11 <= '2020-03-01 00:00:00+00:00') [outer=(11), constraints=(/11: [/'2020-02-28 00:00:00+00:00' - /'2020-03-01 00:00:00+00:00']; tight)] - │ │ │ ├── ((((t.dealerid:9 = 1) OR (t.dealerid:9 = 2)) OR (t.dealerid:9 = 3)) OR (t.dealerid:9 = 4)) OR (t.dealerid:9 = 5) [outer=(9), constraints=(/9: [/1 - /1] [/2 - /2] [/3 - /3] [/4 - /4] [/5 - /5]; tight)] - │ │ │ └── t.isbuy:10 IN (false, true) [outer=(10), constraints=(/10: [/false - /false] [/true - /true]; tight)] + │ │ │ ├── (date:12 >= '2020-02-28 00:00:00+00:00') AND (date:12 <= '2020-03-01 00:00:00+00:00') [outer=(12), constraints=(/12: [/'2020-02-28 00:00:00+00:00' - /'2020-03-01 00:00:00+00:00']; tight)] + │ │ │ ├── ((((t.dealerid:10 = 1) OR (t.dealerid:10 = 2)) OR (t.dealerid:10 = 3)) OR (t.dealerid:10 = 4)) OR (t.dealerid:10 = 5) [outer=(10), constraints=(/10: [/1 - /1] [/2 - /2] [/3 - /3] [/4 - /4] [/5 - /5]; tight)] + │ │ │ └── t.isbuy:11 IN (false, true) [outer=(11), constraints=(/11: [/false - /false] [/true - /true]; tight)] │ │ └── 100 │ └── aggregations - │ └── sum [as=sum:16, outer=(5)] + │ └── sum [as=sum:18, outer=(5)] │ └── quantity:5 └── 0 @@ -1227,21 +1227,21 @@ VALUES (1, FALSE, '2020-03-01', 'the-account', 'the-customer', '70F03EB1-4F58-4C insert transactions ├── columns: ├── insert-mapping: - │ ├── column1:8 => dealerid:1 - │ ├── column2:9 => isbuy:2 - │ ├── column3:10 => date:3 - │ ├── column4:11 => accountname:4 - │ ├── column5:12 => customername:5 - │ ├── column6:13 => operationid:6 - │ └── column14:14 => version:7 + │ ├── column1:9 => dealerid:1 + │ ├── column2:10 => isbuy:2 + │ ├── column3:11 => date:3 + │ ├── column4:12 => accountname:4 + │ ├── column5:13 => customername:5 + │ ├── column6:14 => operationid:6 + │ └── column15:15 => version:7 ├── cardinality: [0 - 0] ├── volatile, mutations └── values - ├── columns: column1:8!null column2:9!null column3:10!null column4:11!null column5:12!null column6:13!null column14:14 + ├── columns: column1:9!null column2:10!null column3:11!null column4:12!null column5:13!null column6:14!null column15:15 ├── cardinality: [1 - 1] ├── volatile ├── key: () - ├── fd: ()-->(8-14) + ├── fd: ()-->(9-15) └── (1, false, '2020-03-01 00:00:00+00:00', 'the-account', 'the-customer', '70f03eb1-4f58-4c26-b72d-c524a9d537dd', cluster_logical_timestamp()) # Upsert buy or sell transaction. @@ -1251,42 +1251,42 @@ VALUES (1, FALSE, '2020-03-01', 'the-account', 'the-customer', '70F03EB1-4F58-4C ---- upsert transactions ├── columns: - ├── canary column: 15 - ├── fetch columns: dealerid:15 isbuy:16 date:17 accountname:18 customername:19 operationid:20 version:21 + ├── canary column: 16 + ├── fetch columns: dealerid:16 isbuy:17 date:18 accountname:19 customername:20 operationid:21 version:22 ├── insert-mapping: - │ ├── column1:8 => dealerid:1 - │ ├── column2:9 => isbuy:2 - │ ├── column3:10 => date:3 - │ ├── column4:11 => accountname:4 - │ ├── column5:12 => customername:5 - │ ├── column6:13 => operationid:6 - │ └── column14:14 => version:7 + │ ├── column1:9 => dealerid:1 + │ ├── column2:10 => isbuy:2 + │ ├── column3:11 => date:3 + │ ├── column4:12 => accountname:4 + │ ├── column5:13 => customername:5 + │ ├── column6:14 => operationid:6 + │ └── column15:15 => version:7 ├── update-mapping: - │ ├── column4:11 => accountname:4 - │ ├── column5:12 => customername:5 - │ └── column6:13 => operationid:6 + │ ├── column4:12 => accountname:4 + │ ├── column5:13 => customername:5 + │ └── column6:14 => operationid:6 ├── cardinality: [0 - 0] ├── volatile, mutations └── left-join (cross) - ├── columns: column1:8!null column2:9!null column3:10!null column4:11!null column5:12!null column6:13!null column14:14 dealerid:15 isbuy:16 date:17 accountname:18 customername:19 operationid:20 version:21 + ├── columns: column1:9!null column2:10!null column3:11!null column4:12!null column5:13!null column6:14!null column15:15 dealerid:16 isbuy:17 date:18 accountname:19 customername:20 operationid:21 version:22 ├── cardinality: [1 - 1] ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) ├── volatile ├── key: () - ├── fd: ()-->(8-21) + ├── fd: ()-->(9-22) ├── values - │ ├── columns: column1:8!null column2:9!null column3:10!null column4:11!null column5:12!null column6:13!null column14:14 + │ ├── columns: column1:9!null column2:10!null column3:11!null column4:12!null column5:13!null column6:14!null column15:15 │ ├── cardinality: [1 - 1] │ ├── volatile │ ├── key: () - │ ├── fd: ()-->(8-14) + │ ├── fd: ()-->(9-15) │ └── (1, false, '2020-03-01 00:00:00+00:00', 'the-account', 'the-customer', '70f03eb1-4f58-4c26-b72d-c524a9d537dd', cluster_logical_timestamp()) ├── scan transactions - │ ├── columns: dealerid:15!null isbuy:16!null date:17!null accountname:18!null customername:19!null operationid:20 version:21!null - │ ├── constraint: /15/16/17: [/1/false/'2020-03-01 00:00:00+00:00' - /1/false/'2020-03-01 00:00:00+00:00'] + │ ├── columns: dealerid:16!null isbuy:17!null date:18!null accountname:19!null customername:20!null operationid:21 version:22!null + │ ├── constraint: /16/17/18: [/1/false/'2020-03-01 00:00:00+00:00' - /1/false/'2020-03-01 00:00:00+00:00'] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(15-21) + │ └── fd: ()-->(16-22) └── filters (true) # Insert structured data (c=CardId, q=Quantity, s=SellPrice, b=BuyPrice) @@ -1306,108 +1306,108 @@ FROM updates ---- upsert transactiondetails ├── columns: - ├── canary column: 21 - ├── fetch columns: transactiondetails.dealerid:21 transactiondetails.isbuy:22 transactiondate:23 cardid:24 quantity:25 transactiondetails.sellprice:26 transactiondetails.buyprice:27 transactiondetails.version:28 + ├── canary column: 22 + ├── fetch columns: transactiondetails.dealerid:22 transactiondetails.isbuy:23 transactiondate:24 cardid:25 quantity:26 transactiondetails.sellprice:27 transactiondetails.buyprice:28 transactiondetails.version:29 ├── insert-mapping: - │ ├── "?column?":11 => transactiondetails.dealerid:2 - │ ├── bool:12 => transactiondetails.isbuy:3 - │ ├── current_timestamp:13 => transactiondate:4 - │ ├── int8:14 => cardid:5 - │ ├── int8:15 => quantity:6 - │ ├── sellprice:19 => transactiondetails.sellprice:7 - │ ├── buyprice:20 => transactiondetails.buyprice:8 - │ └── column18:18 => transactiondetails.version:9 + │ ├── "?column?":12 => transactiondetails.dealerid:2 + │ ├── bool:13 => transactiondetails.isbuy:3 + │ ├── current_timestamp:14 => transactiondate:4 + │ ├── int8:15 => cardid:5 + │ ├── int8:16 => quantity:6 + │ ├── sellprice:20 => transactiondetails.sellprice:7 + │ ├── buyprice:21 => transactiondetails.buyprice:8 + │ └── column19:19 => transactiondetails.version:9 ├── update-mapping: - │ ├── sellprice:19 => transactiondetails.sellprice:7 - │ └── buyprice:20 => transactiondetails.buyprice:8 + │ ├── sellprice:20 => transactiondetails.sellprice:7 + │ └── buyprice:21 => transactiondetails.buyprice:8 ├── input binding: &2 ├── cardinality: [0 - 0] ├── volatile, mutations ├── project - │ ├── columns: upsert_dealerid:29 upsert_isbuy:30 upsert_transactiondate:31 upsert_cardid:32 "?column?":11!null bool:12!null current_timestamp:13!null int8:14!null int8:15!null column18:18 sellprice:19 buyprice:20 transactiondetails.dealerid:21 transactiondetails.isbuy:22 transactiondate:23 cardid:24 quantity:25 transactiondetails.sellprice:26 transactiondetails.buyprice:27 transactiondetails.version:28 + │ ├── columns: upsert_dealerid:31 upsert_isbuy:32 upsert_transactiondate:33 upsert_cardid:34 "?column?":12!null bool:13!null current_timestamp:14!null int8:15!null int8:16!null column19:19 sellprice:20 buyprice:21 transactiondetails.dealerid:22 transactiondetails.isbuy:23 transactiondate:24 cardid:25 quantity:26 transactiondetails.sellprice:27 transactiondetails.buyprice:28 transactiondetails.version:29 │ ├── cardinality: [1 - 2] │ ├── volatile - │ ├── key: (14,15) - │ ├── fd: ()-->(11-13), (14,15)-->(18-28), (21-25)-->(26-28), (21)-->(29), (21,22)-->(30), (21,23)-->(31), (14,21,24)-->(32) + │ ├── key: (15,16) + │ ├── fd: ()-->(12-14), (15,16)-->(19-29), (22-26)-->(27-29), (22)-->(31), (22,23)-->(32), (22,24)-->(33), (15,22,25)-->(34) │ ├── left-join (lookup transactiondetails) - │ │ ├── columns: "?column?":11!null bool:12!null current_timestamp:13!null int8:14!null int8:15!null column18:18 sellprice:19 buyprice:20 transactiondetails.dealerid:21 transactiondetails.isbuy:22 transactiondate:23 cardid:24 quantity:25 transactiondetails.sellprice:26 transactiondetails.buyprice:27 transactiondetails.version:28 - │ │ ├── key columns: [11 12 13 14 15] = [21 22 23 24 25] + │ │ ├── columns: "?column?":12!null bool:13!null current_timestamp:14!null int8:15!null int8:16!null column19:19 sellprice:20 buyprice:21 transactiondetails.dealerid:22 transactiondetails.isbuy:23 transactiondate:24 cardid:25 quantity:26 transactiondetails.sellprice:27 transactiondetails.buyprice:28 transactiondetails.version:29 + │ │ ├── key columns: [12 13 14 15 16] = [22 23 24 25 26] │ │ ├── lookup columns are key │ │ ├── cardinality: [1 - 2] │ │ ├── volatile - │ │ ├── key: (14,15) - │ │ ├── fd: ()-->(11-13), (14,15)-->(18-28), (21-25)-->(26-28) + │ │ ├── key: (15,16) + │ │ ├── fd: ()-->(12-14), (15,16)-->(19-29), (22-26)-->(27-29) │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: "?column?":11!null bool:12!null current_timestamp:13!null int8:14!null int8:15!null column18:18 sellprice:19 buyprice:20 - │ │ │ ├── grouping columns: int8:14!null int8:15!null + │ │ │ ├── columns: "?column?":12!null bool:13!null current_timestamp:14!null int8:15!null int8:16!null column19:19 sellprice:20 buyprice:21 + │ │ │ ├── grouping columns: int8:15!null int8:16!null │ │ │ ├── error: "UPSERT or INSERT...ON CONFLICT command cannot affect row a second time" │ │ │ ├── cardinality: [1 - 2] │ │ │ ├── volatile - │ │ │ ├── key: (14,15) - │ │ │ ├── fd: ()-->(11-13), (14,15)-->(11-13,18-20) + │ │ │ ├── key: (15,16) + │ │ │ ├── fd: ()-->(12-14), (15,16)-->(12-14,19-21) │ │ │ ├── project - │ │ │ │ ├── columns: sellprice:19 buyprice:20 column18:18 "?column?":11!null bool:12!null current_timestamp:13!null int8:14!null int8:15!null + │ │ │ │ ├── columns: sellprice:20 buyprice:21 column19:19 "?column?":12!null bool:13!null current_timestamp:14!null int8:15!null int8:16!null │ │ │ │ ├── cardinality: [2 - 2] │ │ │ │ ├── volatile - │ │ │ │ ├── fd: ()-->(11-13) + │ │ │ │ ├── fd: ()-->(12-14) │ │ │ │ ├── values - │ │ │ │ │ ├── columns: detail_b:52!null detail_c:53!null detail_q:54!null detail_s:55!null + │ │ │ │ │ ├── columns: detail_b:56!null detail_c:57!null detail_q:58!null detail_s:59!null │ │ │ │ │ ├── cardinality: [2 - 2] │ │ │ │ │ ├── ('2.29', '49833', '4', '2.89') │ │ │ │ │ └── ('17.59', '29483', '2', '18.93') │ │ │ │ └── projections - │ │ │ │ ├── crdb_internal.round_decimal_values(detail_s:55::STRING::DECIMAL(10,4), 4) [as=sellprice:19, outer=(55), immutable] - │ │ │ │ ├── crdb_internal.round_decimal_values(detail_b:52::STRING::DECIMAL(10,4), 4) [as=buyprice:20, outer=(52), immutable] - │ │ │ │ ├── cluster_logical_timestamp() [as=column18:18, volatile] - │ │ │ │ ├── 1 [as="?column?":11] - │ │ │ │ ├── false [as=bool:12] - │ │ │ │ ├── '2017-05-10 13:00:00+00:00' [as=current_timestamp:13] - │ │ │ │ ├── detail_c:53::STRING::INT8 [as=int8:14, outer=(53), immutable] - │ │ │ │ └── detail_q:54::STRING::INT8 [as=int8:15, outer=(54), immutable] + │ │ │ │ ├── crdb_internal.round_decimal_values(detail_s:59::STRING::DECIMAL(10,4), 4) [as=sellprice:20, outer=(59), immutable] + │ │ │ │ ├── crdb_internal.round_decimal_values(detail_b:56::STRING::DECIMAL(10,4), 4) [as=buyprice:21, outer=(56), immutable] + │ │ │ │ ├── cluster_logical_timestamp() [as=column19:19, volatile] + │ │ │ │ ├── 1 [as="?column?":12] + │ │ │ │ ├── false [as=bool:13] + │ │ │ │ ├── '2017-05-10 13:00:00+00:00' [as=current_timestamp:14] + │ │ │ │ ├── detail_c:57::STRING::INT8 [as=int8:15, outer=(57), immutable] + │ │ │ │ └── detail_q:58::STRING::INT8 [as=int8:16, outer=(58), immutable] │ │ │ └── aggregations - │ │ │ ├── first-agg [as=sellprice:19, outer=(19)] - │ │ │ │ └── sellprice:19 - │ │ │ ├── first-agg [as=buyprice:20, outer=(20)] - │ │ │ │ └── buyprice:20 - │ │ │ ├── first-agg [as=column18:18, outer=(18)] - │ │ │ │ └── column18:18 - │ │ │ ├── const-agg [as="?column?":11, outer=(11)] - │ │ │ │ └── "?column?":11 - │ │ │ ├── const-agg [as=bool:12, outer=(12)] - │ │ │ │ └── bool:12 - │ │ │ └── const-agg [as=current_timestamp:13, outer=(13)] - │ │ │ └── current_timestamp:13 + │ │ │ ├── first-agg [as=sellprice:20, outer=(20)] + │ │ │ │ └── sellprice:20 + │ │ │ ├── first-agg [as=buyprice:21, outer=(21)] + │ │ │ │ └── buyprice:21 + │ │ │ ├── first-agg [as=column19:19, outer=(19)] + │ │ │ │ └── column19:19 + │ │ │ ├── const-agg [as="?column?":12, outer=(12)] + │ │ │ │ └── "?column?":12 + │ │ │ ├── const-agg [as=bool:13, outer=(13)] + │ │ │ │ └── bool:13 + │ │ │ └── const-agg [as=current_timestamp:14, outer=(14)] + │ │ │ └── current_timestamp:14 │ │ └── filters (true) │ └── projections - │ ├── CASE WHEN transactiondetails.dealerid:21 IS NULL THEN "?column?":11 ELSE transactiondetails.dealerid:21 END [as=upsert_dealerid:29, outer=(11,21)] - │ ├── CASE WHEN transactiondetails.dealerid:21 IS NULL THEN bool:12 ELSE transactiondetails.isbuy:22 END [as=upsert_isbuy:30, outer=(12,21,22)] - │ ├── CASE WHEN transactiondetails.dealerid:21 IS NULL THEN current_timestamp:13 ELSE transactiondate:23 END [as=upsert_transactiondate:31, outer=(13,21,23)] - │ └── CASE WHEN transactiondetails.dealerid:21 IS NULL THEN int8:14 ELSE cardid:24 END [as=upsert_cardid:32, outer=(14,21,24)] + │ ├── CASE WHEN transactiondetails.dealerid:22 IS NULL THEN "?column?":12 ELSE transactiondetails.dealerid:22 END [as=upsert_dealerid:31, outer=(12,22)] + │ ├── CASE WHEN transactiondetails.dealerid:22 IS NULL THEN bool:13 ELSE transactiondetails.isbuy:23 END [as=upsert_isbuy:32, outer=(13,22,23)] + │ ├── CASE WHEN transactiondetails.dealerid:22 IS NULL THEN current_timestamp:14 ELSE transactiondate:24 END [as=upsert_transactiondate:33, outer=(14,22,24)] + │ └── CASE WHEN transactiondetails.dealerid:22 IS NULL THEN int8:15 ELSE cardid:25 END [as=upsert_cardid:34, outer=(15,22,25)] └── f-k-checks ├── f-k-checks-item: transactiondetails(dealerid,isbuy,transactiondate) -> transactions(dealerid,isbuy,date) │ └── anti-join (lookup transactions) - │ ├── columns: upsert_dealerid:35 upsert_isbuy:36 upsert_transactiondate:37 - │ ├── key columns: [35 36 37] = [38 39 40] + │ ├── columns: upsert_dealerid:37 upsert_isbuy:38 upsert_transactiondate:39 + │ ├── key columns: [37 38 39] = [40 41 42] │ ├── lookup columns are key │ ├── cardinality: [0 - 2] │ ├── with-scan &2 - │ │ ├── columns: upsert_dealerid:35 upsert_isbuy:36 upsert_transactiondate:37 + │ │ ├── columns: upsert_dealerid:37 upsert_isbuy:38 upsert_transactiondate:39 │ │ ├── mapping: - │ │ │ ├── upsert_dealerid:29 => upsert_dealerid:35 - │ │ │ ├── upsert_isbuy:30 => upsert_isbuy:36 - │ │ │ └── upsert_transactiondate:31 => upsert_transactiondate:37 + │ │ │ ├── upsert_dealerid:31 => upsert_dealerid:37 + │ │ │ ├── upsert_isbuy:32 => upsert_isbuy:38 + │ │ │ └── upsert_transactiondate:33 => upsert_transactiondate:39 │ │ └── cardinality: [1 - 2] │ └── filters (true) └── f-k-checks-item: transactiondetails(cardid) -> cards(id) └── anti-join (lookup cards) - ├── columns: upsert_cardid:45 - ├── key columns: [45] = [46] + ├── columns: upsert_cardid:48 + ├── key columns: [48] = [49] ├── lookup columns are key ├── cardinality: [0 - 2] ├── with-scan &2 - │ ├── columns: upsert_cardid:45 + │ ├── columns: upsert_cardid:48 │ ├── mapping: - │ │ └── upsert_cardid:32 => upsert_cardid:45 + │ │ └── upsert_cardid:34 => upsert_cardid:48 │ └── cardinality: [1 - 2] └── filters (true) @@ -1418,18 +1418,18 @@ WHERE dealerid = 1 AND accountname = 'some-account' AND cardid = ANY ARRAY[29483 ---- delete inventorydetails ├── columns: - ├── fetch columns: dealerid:6 cardid:7 accountname:8 + ├── fetch columns: dealerid:7 cardid:8 accountname:9 ├── cardinality: [0 - 0] ├── volatile, mutations └── scan inventorydetails@inventorydetails_auto_index_inventorydetailscardidkey - ├── columns: dealerid:6!null cardid:7!null accountname:8!null - ├── constraint: /7/6/8 + ├── columns: dealerid:7!null cardid:8!null accountname:9!null + ├── constraint: /8/7/9 │ ├── [/294/1/'some-account' - /294/1/'some-account'] │ ├── [/1793/1/'some-account' - /1793/1/'some-account'] │ └── [/29483/1/'some-account' - /29483/1/'some-account'] ├── cardinality: [0 - 3] - ├── key: (7) - └── fd: ()-->(6,8) + ├── key: (8) + └── fd: ()-->(7,9) # Update CardsInfo inventory numbers (by CardId, Quantity) to reflect card # transfers. @@ -1448,99 +1448,99 @@ WHERE ci.cardid = Updates.c AND ci.dealerid = 1 ---- update ci ├── columns: - ├── fetch columns: ci.dealerid:15 ci.cardid:16 buyprice:17 sellprice:18 discount:19 desiredinventory:20 actualinventory:21 maxinventory:22 ci.version:23 + ├── fetch columns: ci.dealerid:16 ci.cardid:17 buyprice:18 sellprice:19 discount:20 desiredinventory:21 actualinventory:22 maxinventory:23 ci.version:24 ├── update-mapping: - │ └── actualinventory_new:33 => actualinventory:12 + │ └── actualinventory_new:36 => actualinventory:12 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: actualinventory_new:33 ci.dealerid:15!null ci.cardid:16!null buyprice:17!null sellprice:18!null discount:19!null desiredinventory:20!null actualinventory:21!null maxinventory:22!null ci.version:23!null c:24!null q:25!null - ├── key: (16) - ├── fd: ()-->(15), (16)-->(17-25,33), (23)-->(16-22), (16)==(24), (24)==(16) + ├── columns: actualinventory_new:36 ci.dealerid:16!null ci.cardid:17!null buyprice:18!null sellprice:19!null discount:20!null desiredinventory:21!null actualinventory:22!null maxinventory:23!null ci.version:24!null c:26!null q:27!null + ├── key: (17) + ├── fd: ()-->(16), (17)-->(18-24,26,27,36), (24)-->(17-23), (17)==(26), (26)==(17) ├── group-by - │ ├── columns: ci.dealerid:15!null ci.cardid:16!null buyprice:17!null sellprice:18!null discount:19!null desiredinventory:20!null actualinventory:21!null maxinventory:22!null ci.version:23!null c:24!null q:25!null sum_int:31 - │ ├── grouping columns: ci.cardid:16!null - │ ├── key: (16) - │ ├── fd: ()-->(15), (16)-->(15,17-25,31), (23)-->(16-22), (16)==(24), (24)==(16) + │ ├── columns: ci.dealerid:16!null ci.cardid:17!null buyprice:18!null sellprice:19!null discount:20!null desiredinventory:21!null actualinventory:22!null maxinventory:23!null ci.version:24!null c:26!null q:27!null sum_int:34 + │ ├── grouping columns: ci.cardid:17!null + │ ├── key: (17) + │ ├── fd: ()-->(16), (17)-->(16,18-24,26,27,34), (24)-->(17-23), (17)==(26), (26)==(17) │ ├── left-join (lookup inventorydetails) - │ │ ├── columns: ci.dealerid:15!null ci.cardid:16!null buyprice:17!null sellprice:18!null discount:19!null desiredinventory:20!null actualinventory:21!null maxinventory:22!null ci.version:23!null c:24!null q:25!null id.dealerid:26 id.cardid:27 quantity:29 - │ │ ├── key columns: [37 16] = [26 27] - │ │ ├── fd: ()-->(15), (16)-->(17-25), (23)-->(16-22), (16)==(24), (24)==(16) + │ │ ├── columns: ci.dealerid:16!null ci.cardid:17!null buyprice:18!null sellprice:19!null discount:20!null desiredinventory:21!null actualinventory:22!null maxinventory:23!null ci.version:24!null c:26!null q:27!null id.dealerid:28 id.cardid:29 quantity:31 + │ │ ├── key columns: [40 17] = [28 29] + │ │ ├── fd: ()-->(16), (17)-->(18-24,26,27), (24)-->(17-23), (17)==(26), (26)==(17) │ │ ├── project - │ │ │ ├── columns: "project_const_col_@26":37!null ci.dealerid:15!null ci.cardid:16!null buyprice:17!null sellprice:18!null discount:19!null desiredinventory:20!null actualinventory:21!null maxinventory:22!null ci.version:23!null c:24!null q:25!null + │ │ │ ├── columns: "project_const_col_@28":40!null ci.dealerid:16!null ci.cardid:17!null buyprice:18!null sellprice:19!null discount:20!null desiredinventory:21!null actualinventory:22!null maxinventory:23!null ci.version:24!null c:26!null q:27!null │ │ │ ├── cardinality: [0 - 2] - │ │ │ ├── key: (16) - │ │ │ ├── fd: ()-->(15,37), (16)-->(17-25), (23)-->(16-22), (16)==(24), (24)==(16) + │ │ │ ├── key: (17) + │ │ │ ├── fd: ()-->(16,40), (17)-->(18-24,26,27), (24)-->(17-23), (17)==(26), (26)==(17) │ │ │ ├── distinct-on - │ │ │ │ ├── columns: ci.dealerid:15!null ci.cardid:16!null buyprice:17!null sellprice:18!null discount:19!null desiredinventory:20!null actualinventory:21!null maxinventory:22!null ci.version:23!null c:24!null q:25!null - │ │ │ │ ├── grouping columns: ci.cardid:16!null + │ │ │ │ ├── columns: ci.dealerid:16!null ci.cardid:17!null buyprice:18!null sellprice:19!null discount:20!null desiredinventory:21!null actualinventory:22!null maxinventory:23!null ci.version:24!null c:26!null q:27!null + │ │ │ │ ├── grouping columns: ci.cardid:17!null │ │ │ │ ├── cardinality: [0 - 2] - │ │ │ │ ├── key: (16) - │ │ │ │ ├── fd: ()-->(15), (16)-->(15,17-25), (23)-->(16-22), (16)==(24), (24)==(16) + │ │ │ │ ├── key: (17) + │ │ │ │ ├── fd: ()-->(16), (17)-->(16,18-24,26,27), (24)-->(17-23), (17)==(26), (26)==(17) │ │ │ │ ├── inner-join (lookup cardsinfo) - │ │ │ │ │ ├── columns: ci.dealerid:15!null ci.cardid:16!null buyprice:17!null sellprice:18!null discount:19!null desiredinventory:20!null actualinventory:21!null maxinventory:22!null ci.version:23!null c:24!null q:25!null - │ │ │ │ │ ├── key columns: [34 24] = [15 16] + │ │ │ │ │ ├── columns: ci.dealerid:16!null ci.cardid:17!null buyprice:18!null sellprice:19!null discount:20!null desiredinventory:21!null actualinventory:22!null maxinventory:23!null ci.version:24!null c:26!null q:27!null + │ │ │ │ │ ├── key columns: [37 26] = [16 17] │ │ │ │ │ ├── lookup columns are key │ │ │ │ │ ├── cardinality: [0 - 2] - │ │ │ │ │ ├── fd: ()-->(15), (16)-->(17-23), (23)-->(16-22), (16)==(24), (24)==(16) + │ │ │ │ │ ├── fd: ()-->(16), (17)-->(18-24), (24)-->(17-23), (17)==(26), (26)==(17) │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: "project_const_col_@15":34!null c:24!null q:25!null + │ │ │ │ │ │ ├── columns: "project_const_col_@16":37!null c:26!null q:27!null │ │ │ │ │ │ ├── cardinality: [2 - 2] - │ │ │ │ │ │ ├── fd: ()-->(34) + │ │ │ │ │ │ ├── fd: ()-->(37) │ │ │ │ │ │ ├── values - │ │ │ │ │ │ │ ├── columns: c:24!null q:25!null + │ │ │ │ │ │ │ ├── columns: c:26!null q:27!null │ │ │ │ │ │ │ ├── cardinality: [2 - 2] │ │ │ │ │ │ │ ├── (42948, 3) │ │ │ │ │ │ │ └── (24924, 4) │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── 1 [as="project_const_col_@15":34] + │ │ │ │ │ │ └── 1 [as="project_const_col_@16":37] │ │ │ │ │ └── filters (true) │ │ │ │ └── aggregations - │ │ │ │ ├── first-agg [as=buyprice:17, outer=(17)] - │ │ │ │ │ └── buyprice:17 - │ │ │ │ ├── first-agg [as=sellprice:18, outer=(18)] - │ │ │ │ │ └── sellprice:18 - │ │ │ │ ├── first-agg [as=discount:19, outer=(19)] - │ │ │ │ │ └── discount:19 - │ │ │ │ ├── first-agg [as=desiredinventory:20, outer=(20)] - │ │ │ │ │ └── desiredinventory:20 - │ │ │ │ ├── first-agg [as=actualinventory:21, outer=(21)] - │ │ │ │ │ └── actualinventory:21 - │ │ │ │ ├── first-agg [as=maxinventory:22, outer=(22)] - │ │ │ │ │ └── maxinventory:22 - │ │ │ │ ├── first-agg [as=ci.version:23, outer=(23)] - │ │ │ │ │ └── ci.version:23 - │ │ │ │ ├── first-agg [as=c:24, outer=(24)] - │ │ │ │ │ └── c:24 - │ │ │ │ ├── first-agg [as=q:25, outer=(25)] - │ │ │ │ │ └── q:25 - │ │ │ │ └── const-agg [as=ci.dealerid:15, outer=(15)] - │ │ │ │ └── ci.dealerid:15 + │ │ │ │ ├── first-agg [as=buyprice:18, outer=(18)] + │ │ │ │ │ └── buyprice:18 + │ │ │ │ ├── first-agg [as=sellprice:19, outer=(19)] + │ │ │ │ │ └── sellprice:19 + │ │ │ │ ├── first-agg [as=discount:20, outer=(20)] + │ │ │ │ │ └── discount:20 + │ │ │ │ ├── first-agg [as=desiredinventory:21, outer=(21)] + │ │ │ │ │ └── desiredinventory:21 + │ │ │ │ ├── first-agg [as=actualinventory:22, outer=(22)] + │ │ │ │ │ └── actualinventory:22 + │ │ │ │ ├── first-agg [as=maxinventory:23, outer=(23)] + │ │ │ │ │ └── maxinventory:23 + │ │ │ │ ├── first-agg [as=ci.version:24, outer=(24)] + │ │ │ │ │ └── ci.version:24 + │ │ │ │ ├── first-agg [as=c:26, outer=(26)] + │ │ │ │ │ └── c:26 + │ │ │ │ ├── first-agg [as=q:27, outer=(27)] + │ │ │ │ │ └── q:27 + │ │ │ │ └── const-agg [as=ci.dealerid:16, outer=(16)] + │ │ │ │ └── ci.dealerid:16 │ │ │ └── projections - │ │ │ └── 1 [as="project_const_col_@26":37] + │ │ │ └── 1 [as="project_const_col_@28":40] │ │ └── filters (true) │ └── aggregations - │ ├── sum-int [as=sum_int:31, outer=(29)] - │ │ └── quantity:29 - │ ├── const-agg [as=ci.dealerid:15, outer=(15)] - │ │ └── ci.dealerid:15 - │ ├── const-agg [as=buyprice:17, outer=(17)] - │ │ └── buyprice:17 - │ ├── const-agg [as=sellprice:18, outer=(18)] - │ │ └── sellprice:18 - │ ├── const-agg [as=discount:19, outer=(19)] - │ │ └── discount:19 - │ ├── const-agg [as=desiredinventory:20, outer=(20)] - │ │ └── desiredinventory:20 - │ ├── const-agg [as=actualinventory:21, outer=(21)] - │ │ └── actualinventory:21 - │ ├── const-agg [as=maxinventory:22, outer=(22)] - │ │ └── maxinventory:22 - │ ├── const-agg [as=ci.version:23, outer=(23)] - │ │ └── ci.version:23 - │ ├── const-agg [as=c:24, outer=(24)] - │ │ └── c:24 - │ └── const-agg [as=q:25, outer=(25)] - │ └── q:25 + │ ├── sum-int [as=sum_int:34, outer=(31)] + │ │ └── quantity:31 + │ ├── const-agg [as=ci.dealerid:16, outer=(16)] + │ │ └── ci.dealerid:16 + │ ├── const-agg [as=buyprice:18, outer=(18)] + │ │ └── buyprice:18 + │ ├── const-agg [as=sellprice:19, outer=(19)] + │ │ └── sellprice:19 + │ ├── const-agg [as=discount:20, outer=(20)] + │ │ └── discount:20 + │ ├── const-agg [as=desiredinventory:21, outer=(21)] + │ │ └── desiredinventory:21 + │ ├── const-agg [as=actualinventory:22, outer=(22)] + │ │ └── actualinventory:22 + │ ├── const-agg [as=maxinventory:23, outer=(23)] + │ │ └── maxinventory:23 + │ ├── const-agg [as=ci.version:24, outer=(24)] + │ │ └── ci.version:24 + │ ├── const-agg [as=c:26, outer=(26)] + │ │ └── c:26 + │ └── const-agg [as=q:27, outer=(27)] + │ └── q:27 └── projections - └── COALESCE(sum_int:31, 0) [as=actualinventory_new:33, outer=(31)] + └── COALESCE(sum_int:34, 0) [as=actualinventory_new:36, outer=(34)] diff --git a/pkg/sql/opt/xform/testdata/external/trading-mutation b/pkg/sql/opt/xform/testdata/external/trading-mutation index 07dc7bcebc2d..ef35b1232a41 100644 --- a/pkg/sql/opt/xform/testdata/external/trading-mutation +++ b/pkg/sql/opt/xform/testdata/external/trading-mutation @@ -556,33 +556,33 @@ SELECT FROM CardsView WHERE Version > 1584421773604892000.0000000000 ---- project - ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null buyprice:9!null sellprice:10!null desiredinventory:12!null actualinventory:13!null version:15!null discount:11!null maxinventory:14!null + ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null buyprice:10!null sellprice:11!null desiredinventory:13!null actualinventory:14!null version:16!null discount:12!null maxinventory:15!null ├── immutable ├── stats: [rows=1] - ├── key: (15) - ├── fd: (1)-->(2-6,9-15), (2,4,5)~~>(1,3,6), (15)-->(1-6,9-14) + ├── key: (16) + ├── fd: (1)-->(2-6,10-16), (2,4,5)~~>(1,3,6), (16)-->(1-6,10-15) └── inner-join (lookup cards) - ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null dealerid:7!null cardid:8!null buyprice:9!null sellprice:10!null discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null version:15!null - ├── key columns: [8] = [1] + ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null dealerid:8!null cardid:9!null buyprice:10!null sellprice:11!null discount:12!null desiredinventory:13!null actualinventory:14!null maxinventory:15!null version:16!null + ├── key columns: [9] = [1] ├── lookup columns are key ├── immutable - ├── stats: [rows=1, distinct(1)=6.35833333e-05, null(1)=0, distinct(8)=6.35833333e-05, null(8)=0] - ├── key: (8) - ├── fd: ()-->(7), (1)-->(2-6), (2,4,5)~~>(1,3,6), (8)-->(9-15), (15)-->(8-14), (1)==(8), (8)==(1) + ├── stats: [rows=1, distinct(1)=6.35833333e-05, null(1)=0, distinct(9)=6.35833333e-05, null(9)=0] + ├── key: (9) + ├── fd: ()-->(8), (1)-->(2-6), (2,4,5)~~>(1,3,6), (9)-->(10-16), (16)-->(9-15), (1)==(9), (9)==(1) ├── index-join cardsinfo - │ ├── columns: dealerid:7!null cardid:8!null buyprice:9!null sellprice:10!null discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null version:15!null + │ ├── columns: dealerid:8!null cardid:9!null buyprice:10!null sellprice:11!null discount:12!null desiredinventory:13!null actualinventory:14!null maxinventory:15!null version:16!null │ ├── immutable - │ ├── stats: [rows=6.35833333e-05, distinct(7)=6.35833333e-05, null(7)=0, distinct(8)=6.35833333e-05, null(8)=0, distinct(9)=6.35833333e-05, null(9)=0, distinct(10)=6.35833333e-05, null(10)=0, distinct(11)=6.35833333e-05, null(11)=0, distinct(12)=6.35833333e-05, null(12)=0, distinct(13)=6.35833333e-05, null(13)=0, distinct(14)=6.35833333e-05, null(14)=0, distinct(15)=6.35833333e-05, null(15)=0, distinct(7,15)=6.35833333e-05, null(7,15)=0] - │ │ histogram(15)= - │ ├── key: (8) - │ ├── fd: ()-->(7), (8)-->(9-15), (15)-->(8-14) + │ ├── stats: [rows=6.35833333e-05, distinct(8)=6.35833333e-05, null(8)=0, distinct(9)=6.35833333e-05, null(9)=0, distinct(10)=6.35833333e-05, null(10)=0, distinct(11)=6.35833333e-05, null(11)=0, distinct(12)=6.35833333e-05, null(12)=0, distinct(13)=6.35833333e-05, null(13)=0, distinct(14)=6.35833333e-05, null(14)=0, distinct(15)=6.35833333e-05, null(15)=0, distinct(16)=6.35833333e-05, null(16)=0, distinct(8,16)=6.35833333e-05, null(8,16)=0] + │ │ histogram(16)= + │ ├── key: (9) + │ ├── fd: ()-->(8), (9)-->(10-16), (16)-->(9-15) │ └── scan cardsinfo@cardsinfoversionindex - │ ├── columns: dealerid:7!null cardid:8!null version:15!null - │ ├── constraint: /7/15: (/1/1584421773604892000.0000000000 - /1] - │ ├── stats: [rows=6.35833333e-05, distinct(7)=6.35833333e-05, null(7)=0, distinct(8)=6.35833333e-05, null(8)=0, distinct(15)=6.35833333e-05, null(15)=0, distinct(7,15)=6.35833333e-05, null(7,15)=0] - │ │ histogram(15)= - │ ├── key: (8) - │ └── fd: ()-->(7), (8)-->(15), (15)-->(8) + │ ├── columns: dealerid:8!null cardid:9!null version:16!null + │ ├── constraint: /8/16: (/1/1584421773604892000.0000000000 - /1] + │ ├── stats: [rows=6.35833333e-05, distinct(8)=6.35833333e-05, null(8)=0, distinct(9)=6.35833333e-05, null(9)=0, distinct(16)=6.35833333e-05, null(16)=0, distinct(8,16)=6.35833333e-05, null(8,16)=0] + │ │ histogram(16)= + │ ├── key: (9) + │ └── fd: ()-->(8), (9)-->(16), (16)-->(9) └── filters (true) # Get version of last card that was changed. @@ -591,86 +591,86 @@ opt format=show-stats SELECT coalesce(max(Version), 0) FROM GlobalCardsView ---- project - ├── columns: coalesce:35 + ├── columns: coalesce:38 ├── cardinality: [1 - 1] ├── stats: [rows=1] ├── key: () - ├── fd: ()-->(35) + ├── fd: ()-->(38) ├── scalar-group-by - │ ├── columns: max:34 + │ ├── columns: max:37 │ ├── cardinality: [1 - 1] │ ├── stats: [rows=1] │ ├── key: () - │ ├── fd: ()-->(34) + │ ├── fd: ()-->(37) │ ├── limit - │ │ ├── columns: dealerid:7!null version:15!null - │ │ ├── internal-ordering: -15 + │ │ ├── columns: dealerid:8!null version:16!null + │ │ ├── internal-ordering: -16 │ │ ├── cardinality: [0 - 1] │ │ ├── stats: [rows=1] │ │ ├── key: () - │ │ ├── fd: ()-->(7,15) + │ │ ├── fd: ()-->(8,16) │ │ ├── sort - │ │ │ ├── columns: dealerid:7!null version:15!null + │ │ │ ├── columns: dealerid:8!null version:16!null │ │ │ ├── cardinality: [0 - 4] - │ │ │ ├── stats: [rows=4, distinct(7,15)=4, null(7,15)=0] - │ │ │ ├── key: (7,15) - │ │ │ ├── ordering: -15 + │ │ │ ├── stats: [rows=4, distinct(8,16)=4, null(8,16)=0] + │ │ │ ├── key: (8,16) + │ │ │ ├── ordering: -16 │ │ │ ├── limit hint: 1.00 │ │ │ └── union - │ │ │ ├── columns: dealerid:7!null version:15!null - │ │ │ ├── left columns: dealerid:7!null version:15!null - │ │ │ ├── right columns: dealerid:75 version:83 + │ │ │ ├── columns: dealerid:8!null version:16!null + │ │ │ ├── left columns: dealerid:8!null version:16!null + │ │ │ ├── right columns: dealerid:81 version:89 │ │ │ ├── cardinality: [0 - 4] - │ │ │ ├── stats: [rows=4, distinct(7,15)=4, null(7,15)=0] - │ │ │ ├── key: (7,15) + │ │ │ ├── stats: [rows=4, distinct(8,16)=4, null(8,16)=0] + │ │ │ ├── key: (8,16) │ │ │ ├── union - │ │ │ │ ├── columns: dealerid:7!null version:15!null - │ │ │ │ ├── left columns: dealerid:7!null version:15!null - │ │ │ │ ├── right columns: dealerid:62 version:70 + │ │ │ │ ├── columns: dealerid:8!null version:16!null + │ │ │ │ ├── left columns: dealerid:8!null version:16!null + │ │ │ │ ├── right columns: dealerid:67 version:75 │ │ │ │ ├── cardinality: [0 - 3] - │ │ │ │ ├── stats: [rows=3, distinct(7,15)=3, null(7,15)=0] - │ │ │ │ ├── key: (7,15) + │ │ │ │ ├── stats: [rows=3, distinct(8,16)=3, null(8,16)=0] + │ │ │ │ ├── key: (8,16) │ │ │ │ ├── union - │ │ │ │ │ ├── columns: dealerid:7!null version:15!null - │ │ │ │ │ ├── left columns: dealerid:36 version:44 - │ │ │ │ │ ├── right columns: dealerid:49 version:57 + │ │ │ │ │ ├── columns: dealerid:8!null version:16!null + │ │ │ │ │ ├── left columns: dealerid:39 version:47 + │ │ │ │ │ ├── right columns: dealerid:53 version:61 │ │ │ │ │ ├── cardinality: [0 - 2] - │ │ │ │ │ ├── stats: [rows=2, distinct(7,15)=2, null(7,15)=0] - │ │ │ │ │ ├── key: (7,15) + │ │ │ │ │ ├── stats: [rows=2, distinct(8,16)=2, null(8,16)=0] + │ │ │ │ │ ├── key: (8,16) │ │ │ │ │ ├── scan cardsinfo@cardsinfoversionindex,rev - │ │ │ │ │ │ ├── columns: dealerid:36!null version:44!null - │ │ │ │ │ │ ├── constraint: /36/44: [/1 - /1] + │ │ │ │ │ │ ├── columns: dealerid:39!null version:47!null + │ │ │ │ │ │ ├── constraint: /39/47: [/1 - /1] │ │ │ │ │ │ ├── limit: 1(rev) - │ │ │ │ │ │ ├── stats: [rows=1, distinct(36)=1, null(36)=0, distinct(36,44)=1, null(36,44)=0] + │ │ │ │ │ │ ├── stats: [rows=1, distinct(39)=1, null(39)=0, distinct(39,47)=1, null(39,47)=0] │ │ │ │ │ │ ├── key: () - │ │ │ │ │ │ └── fd: ()-->(36,44) + │ │ │ │ │ │ └── fd: ()-->(39,47) │ │ │ │ │ └── scan cardsinfo@cardsinfoversionindex,rev - │ │ │ │ │ ├── columns: dealerid:49!null version:57!null - │ │ │ │ │ ├── constraint: /49/57: [/2 - /2] + │ │ │ │ │ ├── columns: dealerid:53!null version:61!null + │ │ │ │ │ ├── constraint: /53/61: [/2 - /2] │ │ │ │ │ ├── limit: 1(rev) - │ │ │ │ │ ├── stats: [rows=1, distinct(49)=1, null(49)=0, distinct(49,57)=1, null(49,57)=0] + │ │ │ │ │ ├── stats: [rows=1, distinct(53)=1, null(53)=0, distinct(53,61)=1, null(53,61)=0] │ │ │ │ │ ├── key: () - │ │ │ │ │ └── fd: ()-->(49,57) + │ │ │ │ │ └── fd: ()-->(53,61) │ │ │ │ └── scan cardsinfo@cardsinfoversionindex,rev - │ │ │ │ ├── columns: dealerid:62!null version:70!null - │ │ │ │ ├── constraint: /62/70: [/3 - /3] + │ │ │ │ ├── columns: dealerid:67!null version:75!null + │ │ │ │ ├── constraint: /67/75: [/3 - /3] │ │ │ │ ├── limit: 1(rev) - │ │ │ │ ├── stats: [rows=1, distinct(62)=1, null(62)=0, distinct(62,70)=1, null(62,70)=0] + │ │ │ │ ├── stats: [rows=1, distinct(67)=1, null(67)=0, distinct(67,75)=1, null(67,75)=0] │ │ │ │ ├── key: () - │ │ │ │ └── fd: ()-->(62,70) + │ │ │ │ └── fd: ()-->(67,75) │ │ │ └── scan cardsinfo@cardsinfoversionindex,rev - │ │ │ ├── columns: dealerid:75!null version:83!null - │ │ │ ├── constraint: /75/83: [/4 - /4] + │ │ │ ├── columns: dealerid:81!null version:89!null + │ │ │ ├── constraint: /81/89: [/4 - /4] │ │ │ ├── limit: 1(rev) - │ │ │ ├── stats: [rows=1, distinct(75)=1, null(75)=0, distinct(75,83)=1, null(75,83)=0] + │ │ │ ├── stats: [rows=1, distinct(81)=1, null(81)=0, distinct(81,89)=1, null(81,89)=0] │ │ │ ├── key: () - │ │ │ └── fd: ()-->(75,83) + │ │ │ └── fd: ()-->(81,89) │ │ └── 1 │ └── aggregations - │ └── const-agg [as=max:34, outer=(15)] - │ └── version:15 + │ └── const-agg [as=max:37, outer=(16)] + │ └── version:16 └── projections - └── COALESCE(max:34, 0) [as=coalesce:35, outer=(34)] + └── COALESCE(max:37, 0) [as=coalesce:38, outer=(37)] # Show last 20 transactions for a particular card. # @@ -690,43 +690,43 @@ ORDER BY TransactionDate DESC LIMIT 20 ---- project - ├── columns: isbuy:2!null transactiondate:3!null cardid:4!null quantity:5!null sellprice:6!null buyprice:7!null isbuy2:12!null date:13!null accountname:14!null customername:15!null + ├── columns: isbuy:2!null transactiondate:3!null cardid:4!null quantity:5!null sellprice:6!null buyprice:7!null isbuy2:13!null date:14!null accountname:15!null customername:16!null ├── cardinality: [0 - 20] ├── stats: [rows=20] - ├── key: (5,13) - ├── fd: ()-->(2,4,12), (3,5)-->(6,7), (13)-->(14,15), (3)==(13), (13)==(3) - ├── ordering: -(3|13) opt(2,4,12) [actual: -3] + ├── key: (5,14) + ├── fd: ()-->(2,4,13), (3,5)-->(6,7), (14)-->(15,16), (3)==(14), (14)==(3) + ├── ordering: -(3|14) opt(2,4,13) [actual: -3] └── limit - ├── columns: transactiondetails.dealerid:1!null transactiondetails.isbuy:2!null transactiondate:3!null cardid:4!null quantity:5!null sellprice:6!null buyprice:7!null transactions.dealerid:11!null transactions.isbuy:12!null date:13!null accountname:14!null customername:15!null - ├── internal-ordering: -(3|13) opt(1,2,4,11,12) + ├── columns: transactiondetails.dealerid:1!null transactiondetails.isbuy:2!null transactiondate:3!null cardid:4!null quantity:5!null sellprice:6!null buyprice:7!null transactions.dealerid:12!null transactions.isbuy:13!null date:14!null accountname:15!null customername:16!null + ├── internal-ordering: -(3|14) opt(1,2,4,12,13) ├── cardinality: [0 - 20] ├── stats: [rows=20] - ├── key: (5,13) - ├── fd: ()-->(1,2,4,11,12), (3,5)-->(6,7), (13)-->(14,15), (3)==(13), (13)==(3) - ├── ordering: -(3|13) opt(1,2,4,11,12) [actual: -3] + ├── key: (5,14) + ├── fd: ()-->(1,2,4,12,13), (3,5)-->(6,7), (14)-->(15,16), (3)==(14), (14)==(3) + ├── ordering: -(3|14) opt(1,2,4,12,13) [actual: -3] ├── inner-join (lookup transactiondetails) - │ ├── columns: transactiondetails.dealerid:1!null transactiondetails.isbuy:2!null transactiondate:3!null cardid:4!null quantity:5!null sellprice:6!null buyprice:7!null transactions.dealerid:11!null transactions.isbuy:12!null date:13!null accountname:14!null customername:15!null + │ ├── columns: transactiondetails.dealerid:1!null transactiondetails.isbuy:2!null transactiondate:3!null cardid:4!null quantity:5!null sellprice:6!null buyprice:7!null transactions.dealerid:12!null transactions.isbuy:13!null date:14!null accountname:15!null customername:16!null │ ├── key columns: [1 2 3 4 5] = [1 2 3 4 5] │ ├── lookup columns are key - │ ├── stats: [rows=478.646617, distinct(3)=478.646617, null(3)=0, distinct(13)=478.646617, null(13)=0] - │ ├── key: (5,13) - │ ├── fd: ()-->(1,2,4,11,12), (3,5)-->(6,7), (13)-->(14,15), (3)==(13), (13)==(3) - │ ├── ordering: -(3|13) opt(1,2,4,11,12) [actual: -3] + │ ├── stats: [rows=478.646617, distinct(3)=478.646617, null(3)=0, distinct(14)=478.646617, null(14)=0] + │ ├── key: (5,14) + │ ├── fd: ()-->(1,2,4,12,13), (3,5)-->(6,7), (14)-->(15,16), (3)==(14), (14)==(3) + │ ├── ordering: -(3|14) opt(1,2,4,12,13) [actual: -3] │ ├── limit hint: 20.00 │ ├── inner-join (lookup transactions) - │ │ ├── columns: transactiondetails.dealerid:1!null transactiondetails.isbuy:2!null transactiondate:3!null cardid:4!null quantity:5!null transactions.dealerid:11!null transactions.isbuy:12!null date:13!null accountname:14!null customername:15!null - │ │ ├── key columns: [23 24 3] = [11 12 13] + │ │ ├── columns: transactiondetails.dealerid:1!null transactiondetails.isbuy:2!null transactiondate:3!null cardid:4!null quantity:5!null transactions.dealerid:12!null transactions.isbuy:13!null date:14!null accountname:15!null customername:16!null + │ │ ├── key columns: [25 26 3] = [12 13 14] │ │ ├── lookup columns are key - │ │ ├── stats: [rows=478.646617, distinct(3)=478.646617, null(3)=0, distinct(13)=478.646617, null(13)=0] - │ │ ├── key: (5,13) - │ │ ├── fd: ()-->(1,2,4,11,12), (13)-->(14,15), (3)==(13), (13)==(3) - │ │ ├── ordering: -(3|13) opt(1,2,4,11,12) [actual: -3] + │ │ ├── stats: [rows=478.646617, distinct(3)=478.646617, null(3)=0, distinct(14)=478.646617, null(14)=0] + │ │ ├── key: (5,14) + │ │ ├── fd: ()-->(1,2,4,12,13), (14)-->(15,16), (3)==(14), (14)==(3) + │ │ ├── ordering: -(3|14) opt(1,2,4,12,13) [actual: -3] │ │ ├── limit hint: 100.00 │ │ ├── project - │ │ │ ├── columns: "project_const_col_@11":23!null "project_const_col_@12":24!null transactiondetails.dealerid:1!null transactiondetails.isbuy:2!null transactiondate:3!null cardid:4!null quantity:5!null + │ │ │ ├── columns: "project_const_col_@12":25!null "project_const_col_@13":26!null transactiondetails.dealerid:1!null transactiondetails.isbuy:2!null transactiondate:3!null cardid:4!null quantity:5!null │ │ │ ├── stats: [rows=478.646617] │ │ │ ├── key: (3,5) - │ │ │ ├── fd: ()-->(1,2,4,23,24) + │ │ │ ├── fd: ()-->(1,2,4,25,26) │ │ │ ├── ordering: -3 opt(1,2,4) [actual: -3] │ │ │ ├── limit hint: 100.00 │ │ │ ├── scan transactiondetails@detailscardidindex,rev @@ -738,8 +738,8 @@ project │ │ │ │ ├── ordering: -3 opt(1,2,4) [actual: -3] │ │ │ │ └── limit hint: 100.00 │ │ │ └── projections - │ │ │ ├── 1 [as="project_const_col_@11":23] - │ │ │ └── false [as="project_const_col_@12":24] + │ │ │ ├── 1 [as="project_const_col_@12":25] + │ │ │ └── false [as="project_const_col_@13":26] │ │ └── filters (true) │ └── filters (true) └── 20 @@ -802,57 +802,57 @@ ORDER BY Name, SetName, Number LIMIT 50 ---- project - ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null buyprice:9!null sellprice:10!null desiredinventory:12!null actualinventory:13!null version:15!null discount:11!null maxinventory:14!null twodaysales:31 + ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null buyprice:10!null sellprice:11!null desiredinventory:13!null actualinventory:14!null version:16!null discount:12!null maxinventory:15!null twodaysales:34 ├── cardinality: [0 - 50] ├── immutable ├── stats: [rows=50] - ├── key: (15,31) - ├── fd: (1)-->(2-6,9-15), (2,4,5)~~>(1,3,6), (15)-->(1-6,9-14) + ├── key: (16,34) + ├── fd: (1)-->(2-6,10-16), (2,4,5)~~>(1,3,6), (16)-->(1-6,10-15) ├── ordering: +2,+4,+5 ├── limit - │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null cardsinfo.cardid:8!null cardsinfo.buyprice:9!null cardsinfo.sellprice:10!null cardsinfo.discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null cardsinfo.version:15!null sum:30 + │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null cardsinfo.cardid:9!null cardsinfo.buyprice:10!null cardsinfo.sellprice:11!null cardsinfo.discount:12!null desiredinventory:13!null actualinventory:14!null maxinventory:15!null cardsinfo.version:16!null sum:33 │ ├── internal-ordering: +2,+4,+5 │ ├── cardinality: [0 - 50] │ ├── immutable │ ├── stats: [rows=50] - │ ├── key: (8) - │ ├── fd: (1)-->(2-6), (2,4,5)~~>(1,3,6), (8)-->(1-6,9-15,30), (15)-->(8-14), (1)==(8), (8)==(1) + │ ├── key: (9) + │ ├── fd: (1)-->(2-6), (2,4,5)~~>(1,3,6), (9)-->(1-6,10-16,33), (16)-->(9-15), (1)==(9), (9)==(1) │ ├── ordering: +2,+4,+5 │ ├── sort - │ │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null cardsinfo.cardid:8!null cardsinfo.buyprice:9!null cardsinfo.sellprice:10!null cardsinfo.discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null cardsinfo.version:15!null sum:30 + │ │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null cardsinfo.cardid:9!null cardsinfo.buyprice:10!null cardsinfo.sellprice:11!null cardsinfo.discount:12!null desiredinventory:13!null actualinventory:14!null maxinventory:15!null cardsinfo.version:16!null sum:33 │ │ ├── immutable - │ │ ├── stats: [rows=19000, distinct(8)=19000, null(8)=0] - │ │ ├── key: (8) - │ │ ├── fd: (1)-->(2-6), (2,4,5)~~>(1,3,6), (8)-->(1-6,9-15,30), (15)-->(8-14), (1)==(8), (8)==(1) + │ │ ├── stats: [rows=19000, distinct(9)=19000, null(9)=0] + │ │ ├── key: (9) + │ │ ├── fd: (1)-->(2-6), (2,4,5)~~>(1,3,6), (9)-->(1-6,10-16,33), (16)-->(9-15), (1)==(9), (9)==(1) │ │ ├── ordering: +2,+4,+5 │ │ ├── limit hint: 50.00 │ │ └── group-by - │ │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null cardsinfo.cardid:8!null cardsinfo.buyprice:9!null cardsinfo.sellprice:10!null cardsinfo.discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null cardsinfo.version:15!null sum:30 - │ │ ├── grouping columns: cardsinfo.cardid:8!null + │ │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null cardsinfo.cardid:9!null cardsinfo.buyprice:10!null cardsinfo.sellprice:11!null cardsinfo.discount:12!null desiredinventory:13!null actualinventory:14!null maxinventory:15!null cardsinfo.version:16!null sum:33 + │ │ ├── grouping columns: cardsinfo.cardid:9!null │ │ ├── immutable - │ │ ├── stats: [rows=19000, distinct(8)=19000, null(8)=0] - │ │ ├── key: (8) - │ │ ├── fd: (1)-->(2-6), (2,4,5)~~>(1,3,6), (8)-->(1-6,9-15,30), (15)-->(8-14), (1)==(8), (8)==(1) + │ │ ├── stats: [rows=19000, distinct(9)=19000, null(9)=0] + │ │ ├── key: (9) + │ │ ├── fd: (1)-->(2-6), (2,4,5)~~>(1,3,6), (9)-->(1-6,10-16,33), (16)-->(9-15), (1)==(9), (9)==(1) │ │ ├── right-join (hash) - │ │ │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null cardsinfo.dealerid:7!null cardsinfo.cardid:8!null cardsinfo.buyprice:9!null cardsinfo.sellprice:10!null cardsinfo.discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null cardsinfo.version:15!null transactiondetails.dealerid:20 isbuy:21 transactiondate:22 transactiondetails.cardid:23 quantity:24 + │ │ │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null cardsinfo.dealerid:8!null cardsinfo.cardid:9!null cardsinfo.buyprice:10!null cardsinfo.sellprice:11!null cardsinfo.discount:12!null desiredinventory:13!null actualinventory:14!null maxinventory:15!null cardsinfo.version:16!null transactiondetails.dealerid:22 isbuy:23 transactiondate:24 transactiondetails.cardid:25 quantity:26 │ │ │ ├── immutable - │ │ │ ├── stats: [rows=5523583.18, distinct(8)=19000, null(8)=0, distinct(23)=19000, null(23)=0] - │ │ │ ├── key: (8,22-24) - │ │ │ ├── fd: ()-->(7), (1)-->(2-6), (2,4,5)~~>(1,3,6), (8)-->(9-15), (15)-->(8-14), (1)==(8), (8)==(1), (8,22-24)-->(20,21) + │ │ │ ├── stats: [rows=5523583.18, distinct(9)=19000, null(9)=0, distinct(25)=19000, null(25)=0] + │ │ │ ├── key: (9,24-26) + │ │ │ ├── fd: ()-->(8), (1)-->(2-6), (2,4,5)~~>(1,3,6), (9)-->(10-16), (16)-->(9-15), (1)==(9), (9)==(1), (9,24-26)-->(22,23) │ │ │ ├── scan transactiondetails - │ │ │ │ ├── columns: transactiondetails.dealerid:20!null isbuy:21!null transactiondate:22!null transactiondetails.cardid:23!null quantity:24!null - │ │ │ │ ├── constraint: /20/21/22/23/24: [/1/false/'2020-02-28 00:00:00+00:00' - /1/false/'2020-03-01 00:00:00+00:00'] - │ │ │ │ ├── stats: [rows=10630000, distinct(20)=1, null(20)=0, distinct(21)=1, null(21)=0, distinct(22)=10630000, null(22)=0, distinct(23)=57000, null(23)=0, distinct(20,21)=1, null(20,21)=0, distinct(20-22)=10630000, null(20-22)=0] - │ │ │ │ ├── key: (22-24) - │ │ │ │ └── fd: ()-->(20,21) + │ │ │ │ ├── columns: transactiondetails.dealerid:22!null isbuy:23!null transactiondate:24!null transactiondetails.cardid:25!null quantity:26!null + │ │ │ │ ├── constraint: /22/23/24/25/26: [/1/false/'2020-02-28 00:00:00+00:00' - /1/false/'2020-03-01 00:00:00+00:00'] + │ │ │ │ ├── stats: [rows=10630000, distinct(22)=1, null(22)=0, distinct(23)=1, null(23)=0, distinct(24)=10630000, null(24)=0, distinct(25)=57000, null(25)=0, distinct(22,23)=1, null(22,23)=0, distinct(22-24)=10630000, null(22-24)=0] + │ │ │ │ ├── key: (24-26) + │ │ │ │ └── fd: ()-->(22,23) │ │ │ ├── inner-join (merge) - │ │ │ │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null cardsinfo.dealerid:7!null cardsinfo.cardid:8!null cardsinfo.buyprice:9!null cardsinfo.sellprice:10!null cardsinfo.discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null cardsinfo.version:15!null + │ │ │ │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null cardsinfo.dealerid:8!null cardsinfo.cardid:9!null cardsinfo.buyprice:10!null cardsinfo.sellprice:11!null cardsinfo.discount:12!null desiredinventory:13!null actualinventory:14!null maxinventory:15!null cardsinfo.version:16!null │ │ │ │ ├── left ordering: +1 - │ │ │ │ ├── right ordering: +8 + │ │ │ │ ├── right ordering: +9 │ │ │ │ ├── immutable - │ │ │ │ ├── stats: [rows=29618.4611, distinct(1)=19000, null(1)=0, distinct(2)=11668.1409, null(2)=0, distinct(5)=829, null(5)=0, distinct(6)=5572.85686, null(6)=0, distinct(7)=1, null(7)=0, distinct(8)=19000, null(8)=0, distinct(9)=21037.9959, null(9)=0, distinct(10)=21037.9959, null(10)=0, distinct(11)=21037.9959, null(11)=0, distinct(12)=21037.9959, null(12)=0, distinct(13)=21037.9959, null(13)=0, distinct(14)=21037.9959, null(14)=0, distinct(15)=23225.5851, null(15)=0] - │ │ │ │ ├── key: (8) - │ │ │ │ ├── fd: ()-->(7), (1)-->(2-6), (2,4,5)~~>(1,3,6), (8)-->(9-15), (15)-->(8-14), (1)==(8), (8)==(1) + │ │ │ │ ├── stats: [rows=29618.4611, distinct(1)=19000, null(1)=0, distinct(2)=11668.1409, null(2)=0, distinct(5)=829, null(5)=0, distinct(6)=5572.85686, null(6)=0, distinct(8)=1, null(8)=0, distinct(9)=19000, null(9)=0, distinct(10)=21037.9959, null(10)=0, distinct(11)=21037.9959, null(11)=0, distinct(12)=21037.9959, null(12)=0, distinct(13)=21037.9959, null(13)=0, distinct(14)=21037.9959, null(14)=0, distinct(15)=21037.9959, null(15)=0, distinct(16)=23225.5851, null(16)=0] + │ │ │ │ ├── key: (9) + │ │ │ │ ├── fd: ()-->(8), (1)-->(2-6), (2,4,5)~~>(1,3,6), (9)-->(10-16), (16)-->(9-15), (1)==(9), (9)==(1) │ │ │ │ ├── select │ │ │ │ │ ├── columns: id:1!null name:2!null rarity:3 setname:4 number:5!null isfoil:6!null │ │ │ │ │ ├── immutable @@ -869,18 +869,18 @@ project │ │ │ │ │ └── filters │ │ │ │ │ └── (name:2, setname:4, number:5) > ('Shock', '7E', 248) [outer=(2,4,5), immutable, constraints=(/2/4/5: [/'Shock'/'7E'/249 - ]; tight)] │ │ │ │ ├── scan cardsinfo - │ │ │ │ │ ├── columns: cardsinfo.dealerid:7!null cardsinfo.cardid:8!null cardsinfo.buyprice:9!null cardsinfo.sellprice:10!null cardsinfo.discount:11!null desiredinventory:12!null actualinventory:13!null maxinventory:14!null cardsinfo.version:15!null - │ │ │ │ │ ├── constraint: /7/8: [/1 - /1] - │ │ │ │ │ ├── stats: [rows=58333.3333, distinct(7)=1, null(7)=0, distinct(8)=37420.3552, null(8)=0, distinct(9)=40676.7278, null(9)=0, distinct(10)=40676.7278, null(10)=0, distinct(11)=40676.7278, null(11)=0, distinct(12)=40676.7278, null(12)=0, distinct(13)=40676.7278, null(13)=0, distinct(14)=40676.7278, null(14)=0, distinct(15)=58333.3333, null(15)=0] - │ │ │ │ │ ├── key: (8) - │ │ │ │ │ ├── fd: ()-->(7), (8)-->(9-15), (15)-->(8-14) - │ │ │ │ │ └── ordering: +8 opt(7) [actual: +8] + │ │ │ │ │ ├── columns: cardsinfo.dealerid:8!null cardsinfo.cardid:9!null cardsinfo.buyprice:10!null cardsinfo.sellprice:11!null cardsinfo.discount:12!null desiredinventory:13!null actualinventory:14!null maxinventory:15!null cardsinfo.version:16!null + │ │ │ │ │ ├── constraint: /8/9: [/1 - /1] + │ │ │ │ │ ├── stats: [rows=58333.3333, distinct(8)=1, null(8)=0, distinct(9)=37420.3552, null(9)=0, distinct(10)=40676.7278, null(10)=0, distinct(11)=40676.7278, null(11)=0, distinct(12)=40676.7278, null(12)=0, distinct(13)=40676.7278, null(13)=0, distinct(14)=40676.7278, null(14)=0, distinct(15)=40676.7278, null(15)=0, distinct(16)=58333.3333, null(16)=0] + │ │ │ │ │ ├── key: (9) + │ │ │ │ │ ├── fd: ()-->(8), (9)-->(10-16), (16)-->(9-15) + │ │ │ │ │ └── ordering: +9 opt(8) [actual: +9] │ │ │ │ └── filters (true) │ │ │ └── filters - │ │ │ └── transactiondetails.cardid:23 = id:1 [outer=(1,23), constraints=(/1: (/NULL - ]; /23: (/NULL - ]), fd=(1)==(23), (23)==(1)] + │ │ │ └── transactiondetails.cardid:25 = id:1 [outer=(1,25), constraints=(/1: (/NULL - ]; /25: (/NULL - ]), fd=(1)==(25), (25)==(1)] │ │ └── aggregations - │ │ ├── sum [as=sum:30, outer=(24)] - │ │ │ └── quantity:24 + │ │ ├── sum [as=sum:33, outer=(26)] + │ │ │ └── quantity:26 │ │ ├── const-agg [as=id:1, outer=(1)] │ │ │ └── id:1 │ │ ├── const-agg [as=name:2, outer=(2)] @@ -893,23 +893,23 @@ project │ │ │ └── number:5 │ │ ├── const-agg [as=isfoil:6, outer=(6)] │ │ │ └── isfoil:6 - │ │ ├── const-agg [as=cardsinfo.buyprice:9, outer=(9)] - │ │ │ └── cardsinfo.buyprice:9 - │ │ ├── const-agg [as=cardsinfo.sellprice:10, outer=(10)] - │ │ │ └── cardsinfo.sellprice:10 - │ │ ├── const-agg [as=cardsinfo.discount:11, outer=(11)] - │ │ │ └── cardsinfo.discount:11 - │ │ ├── const-agg [as=desiredinventory:12, outer=(12)] - │ │ │ └── desiredinventory:12 - │ │ ├── const-agg [as=actualinventory:13, outer=(13)] - │ │ │ └── actualinventory:13 - │ │ ├── const-agg [as=maxinventory:14, outer=(14)] - │ │ │ └── maxinventory:14 - │ │ └── const-agg [as=cardsinfo.version:15, outer=(15)] - │ │ └── cardsinfo.version:15 + │ │ ├── const-agg [as=cardsinfo.buyprice:10, outer=(10)] + │ │ │ └── cardsinfo.buyprice:10 + │ │ ├── const-agg [as=cardsinfo.sellprice:11, outer=(11)] + │ │ │ └── cardsinfo.sellprice:11 + │ │ ├── const-agg [as=cardsinfo.discount:12, outer=(12)] + │ │ │ └── cardsinfo.discount:12 + │ │ ├── const-agg [as=desiredinventory:13, outer=(13)] + │ │ │ └── desiredinventory:13 + │ │ ├── const-agg [as=actualinventory:14, outer=(14)] + │ │ │ └── actualinventory:14 + │ │ ├── const-agg [as=maxinventory:15, outer=(15)] + │ │ │ └── maxinventory:15 + │ │ └── const-agg [as=cardsinfo.version:16, outer=(16)] + │ │ └── cardsinfo.version:16 │ └── 50 └── projections - └── COALESCE(sum:30, 0) [as=value:31, outer=(30)] + └── COALESCE(sum:33, 0) [as=value:34, outer=(33)] # Daily transaction query. # @@ -946,35 +946,35 @@ GROUP BY extract(day from d.TransactionDate) ORDER BY extract(day from d.TransactionDate) ---- sort - ├── columns: extract:45 totalsell:40!null totalbuy:42!null totalprofit:44!null + ├── columns: extract:49 totalsell:44!null totalbuy:46!null totalprofit:48!null ├── stable - ├── stats: [rows=750327.164, distinct(45)=750327.164, null(45)=0] - ├── key: (45) - ├── fd: (45)-->(40,42,44) - ├── ordering: +45 + ├── stats: [rows=750327.164, distinct(49)=750327.164, null(49)=0] + ├── key: (49) + ├── fd: (49)-->(44,46,48) + ├── ordering: +49 └── group-by - ├── columns: sum:40!null sum:42!null sum:44!null column45:45 - ├── grouping columns: column45:45 + ├── columns: sum:44!null sum:46!null sum:48!null column49:49 + ├── grouping columns: column49:49 ├── stable - ├── stats: [rows=750327.164, distinct(45)=750327.164, null(45)=0] - ├── key: (45) - ├── fd: (45)-->(40,42,44) + ├── stats: [rows=750327.164, distinct(49)=750327.164, null(49)=0] + ├── key: (49) + ├── fd: (49)-->(44,46,48) ├── project - │ ├── columns: column39:39!null column41:41!null column43:43!null column45:45 + │ ├── columns: column43:43!null column45:45!null column47:47!null column49:49 │ ├── stable - │ ├── stats: [rows=1198631.87, distinct(45)=750327.164, null(45)=0] + │ ├── stats: [rows=1198631.87, distinct(49)=750327.164, null(49)=0] │ ├── inner-join (hash) - │ │ ├── columns: transactiondetails.dealerid:1!null transactiondetails.isbuy:2!null transactiondate:3!null transactiondetails.cardid:4!null quantity:5!null transactiondetails.sellprice:6!null transactiondetails.buyprice:7!null transactions.dealerid:11!null transactions.isbuy:12!null date:13!null accountname:14!null customername:15!null id:20!null cardsinfo.dealerid:26!null cardsinfo.cardid:27!null + │ │ ├── columns: transactiondetails.dealerid:1!null transactiondetails.isbuy:2!null transactiondate:3!null transactiondetails.cardid:4!null quantity:5!null transactiondetails.sellprice:6!null transactiondetails.buyprice:7!null transactions.dealerid:12!null transactions.isbuy:13!null date:14!null accountname:15!null customername:16!null id:22!null cardsinfo.dealerid:29!null cardsinfo.cardid:30!null │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ ├── stats: [rows=1198631.87, distinct(3)=750327.164, null(3)=0, distinct(4)=37420.3552, null(4)=0, distinct(20)=37420.3552, null(20)=0] - │ │ ├── key: (5,13,27) - │ │ ├── fd: ()-->(1,2,11,12,26), (3-5)-->(6,7), (13)-->(14,15), (3)==(13), (13)==(3), (20)==(4,27), (27)==(4,20), (4)==(20,27) + │ │ ├── stats: [rows=1198631.87, distinct(3)=750327.164, null(3)=0, distinct(4)=37420.3552, null(4)=0, distinct(22)=37420.3552, null(22)=0] + │ │ ├── key: (5,14,30) + │ │ ├── fd: ()-->(1,2,12,13,29), (3-5)-->(6,7), (14)-->(15,16), (3)==(14), (14)==(3), (22)==(4,30), (30)==(4,22), (4)==(22,30) │ │ ├── inner-join (hash) - │ │ │ ├── columns: transactiondetails.dealerid:1!null transactiondetails.isbuy:2!null transactiondate:3!null transactiondetails.cardid:4!null quantity:5!null transactiondetails.sellprice:6!null transactiondetails.buyprice:7!null transactions.dealerid:11!null transactions.isbuy:12!null date:13!null accountname:14!null customername:15!null + │ │ │ ├── columns: transactiondetails.dealerid:1!null transactiondetails.isbuy:2!null transactiondate:3!null transactiondetails.cardid:4!null quantity:5!null transactiondetails.sellprice:6!null transactiondetails.buyprice:7!null transactions.dealerid:12!null transactions.isbuy:13!null date:14!null accountname:15!null customername:16!null │ │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ │ ├── stats: [rows=1171234.57, distinct(1)=1, null(1)=0, distinct(2)=1, null(2)=0, distinct(3)=1171234.57, null(3)=0, distinct(4)=56999.9999, null(4)=0, distinct(5)=1091498.71, null(5)=0, distinct(6)=1091498.71, null(6)=0, distinct(7)=1091498.71, null(7)=0, distinct(11)=1, null(11)=0, distinct(12)=1, null(12)=0, distinct(13)=1171234.57, null(13)=0, distinct(14)=551608.449, null(14)=0, distinct(15)=551608.449, null(15)=0] - │ │ │ ├── key: (4,5,13) - │ │ │ ├── fd: ()-->(1,2,11,12), (3-5)-->(6,7), (13)-->(14,15), (3)==(13), (13)==(3) + │ │ │ ├── stats: [rows=1171234.57, distinct(1)=1, null(1)=0, distinct(2)=1, null(2)=0, distinct(3)=1171234.57, null(3)=0, distinct(4)=56999.9999, null(4)=0, distinct(5)=1091498.71, null(5)=0, distinct(6)=1091498.71, null(6)=0, distinct(7)=1091498.71, null(7)=0, distinct(12)=1, null(12)=0, distinct(13)=1, null(13)=0, distinct(14)=1171234.57, null(14)=0, distinct(15)=551608.449, null(15)=0, distinct(16)=551608.449, null(16)=0] + │ │ │ ├── key: (4,5,14) + │ │ │ ├── fd: ()-->(1,2,12,13), (3-5)-->(6,7), (14)-->(15,16), (3)==(14), (14)==(3) │ │ │ ├── scan transactiondetails │ │ │ │ ├── columns: transactiondetails.dealerid:1!null transactiondetails.isbuy:2!null transactiondate:3!null transactiondetails.cardid:4!null quantity:5!null transactiondetails.sellprice:6!null transactiondetails.buyprice:7!null │ │ │ │ ├── constraint: /1/2/3/4/5: [/1/false/'2020-02-23 00:00:00+00:00' - /1/false/'2020-03-01 00:00:00+00:00'] @@ -982,53 +982,53 @@ sort │ │ │ │ ├── key: (3-5) │ │ │ │ └── fd: ()-->(1,2), (3-5)-->(6,7) │ │ │ ├── select - │ │ │ │ ├── columns: transactions.dealerid:11!null transactions.isbuy:12!null date:13!null accountname:14!null customername:15!null - │ │ │ │ ├── stats: [rows=1171234.57, distinct(11)=1, null(11)=0, distinct(12)=1, null(12)=0, distinct(13)=1171234.57, null(13)=0, distinct(14)=666666.667, null(14)=0, distinct(15)=666666.667, null(15)=0, distinct(11,12)=1, null(11,12)=0, distinct(11-15)=1171234.57, null(11-15)=0] - │ │ │ │ ├── key: (13) - │ │ │ │ ├── fd: ()-->(11,12), (13)-->(14,15) + │ │ │ │ ├── columns: transactions.dealerid:12!null transactions.isbuy:13!null date:14!null accountname:15!null customername:16!null + │ │ │ │ ├── stats: [rows=1171234.57, distinct(12)=1, null(12)=0, distinct(13)=1, null(13)=0, distinct(14)=1171234.57, null(14)=0, distinct(15)=666666.667, null(15)=0, distinct(16)=666666.667, null(16)=0, distinct(12,13)=1, null(12,13)=0, distinct(12-16)=1171234.57, null(12-16)=0] + │ │ │ │ ├── key: (14) + │ │ │ │ ├── fd: ()-->(12,13), (14)-->(15,16) │ │ │ │ ├── scan transactions - │ │ │ │ │ ├── columns: transactions.dealerid:11!null transactions.isbuy:12!null date:13!null accountname:14!null customername:15!null - │ │ │ │ │ ├── constraint: /11/12/13: [/1/false/'2020-02-23 00:00:00+00:00' - /1/false/'2020-03-01 00:00:00+00:00'] - │ │ │ │ │ ├── stats: [rows=1181111.11, distinct(11)=1, null(11)=0, distinct(12)=1, null(12)=0, distinct(13)=1181111.11, null(13)=0, distinct(11,12)=1, null(11,12)=0, distinct(11-13)=1181111.11, null(11-13)=0] - │ │ │ │ │ ├── key: (13) - │ │ │ │ │ └── fd: ()-->(11,12), (13)-->(14,15) + │ │ │ │ │ ├── columns: transactions.dealerid:12!null transactions.isbuy:13!null date:14!null accountname:15!null customername:16!null + │ │ │ │ │ ├── constraint: /12/13/14: [/1/false/'2020-02-23 00:00:00+00:00' - /1/false/'2020-03-01 00:00:00+00:00'] + │ │ │ │ │ ├── stats: [rows=1181111.11, distinct(12)=1, null(12)=0, distinct(13)=1, null(13)=0, distinct(14)=1181111.11, null(14)=0, distinct(12,13)=1, null(12,13)=0, distinct(12-14)=1181111.11, null(12-14)=0] + │ │ │ │ │ ├── key: (14) + │ │ │ │ │ └── fd: ()-->(12,13), (14)-->(15,16) │ │ │ │ └── filters - │ │ │ │ ├── accountname:14 != 'someaccount' [outer=(14), constraints=(/14: (/NULL - /'someaccount') [/e'someaccount\x00' - ]; tight)] - │ │ │ │ └── customername:15 != 'somecustomer' [outer=(15), constraints=(/15: (/NULL - /'somecustomer') [/e'somecustomer\x00' - ]; tight)] + │ │ │ │ ├── accountname:15 != 'someaccount' [outer=(15), constraints=(/15: (/NULL - /'someaccount') [/e'someaccount\x00' - ]; tight)] + │ │ │ │ └── customername:16 != 'somecustomer' [outer=(16), constraints=(/16: (/NULL - /'somecustomer') [/e'somecustomer\x00' - ]; tight)] │ │ │ └── filters - │ │ │ └── transactiondate:3 = date:13 [outer=(3,13), constraints=(/3: (/NULL - ]; /13: (/NULL - ]), fd=(3)==(13), (13)==(3)] + │ │ │ └── transactiondate:3 = date:14 [outer=(3,14), constraints=(/3: (/NULL - ]; /14: (/NULL - ]), fd=(3)==(14), (14)==(3)] │ │ ├── inner-join (hash) - │ │ │ ├── columns: id:20!null cardsinfo.dealerid:26!null cardsinfo.cardid:27!null + │ │ │ ├── columns: id:22!null cardsinfo.dealerid:29!null cardsinfo.cardid:30!null │ │ │ ├── multiplicity: left-rows(exactly-one), right-rows(zero-or-one) - │ │ │ ├── stats: [rows=58333.3333, distinct(20)=37420.3552, null(20)=0, distinct(26)=1, null(26)=0, distinct(27)=37420.3552, null(27)=0] - │ │ │ ├── key: (27) - │ │ │ ├── fd: ()-->(26), (20)==(27), (27)==(20) + │ │ │ ├── stats: [rows=58333.3333, distinct(22)=37420.3552, null(22)=0, distinct(29)=1, null(29)=0, distinct(30)=37420.3552, null(30)=0] + │ │ │ ├── key: (30) + │ │ │ ├── fd: ()-->(29), (22)==(30), (30)==(22) │ │ │ ├── scan cardsinfo@cardsinfoversionindex - │ │ │ │ ├── columns: cardsinfo.dealerid:26!null cardsinfo.cardid:27!null - │ │ │ │ ├── constraint: /26/34: [/1 - /1] - │ │ │ │ ├── stats: [rows=58333.3333, distinct(26)=1, null(26)=0, distinct(27)=37420.3552, null(27)=0] - │ │ │ │ ├── key: (27) - │ │ │ │ └── fd: ()-->(26) + │ │ │ │ ├── columns: cardsinfo.dealerid:29!null cardsinfo.cardid:30!null + │ │ │ │ ├── constraint: /29/37: [/1 - /1] + │ │ │ │ ├── stats: [rows=58333.3333, distinct(29)=1, null(29)=0, distinct(30)=37420.3552, null(30)=0] + │ │ │ │ ├── key: (30) + │ │ │ │ └── fd: ()-->(29) │ │ │ ├── scan cards@cardsnamesetnumber - │ │ │ │ ├── columns: id:20!null - │ │ │ │ ├── stats: [rows=57000, distinct(20)=57000, null(20)=0] - │ │ │ │ └── key: (20) + │ │ │ │ ├── columns: id:22!null + │ │ │ │ ├── stats: [rows=57000, distinct(22)=57000, null(22)=0] + │ │ │ │ └── key: (22) │ │ │ └── filters - │ │ │ └── id:20 = cardsinfo.cardid:27 [outer=(20,27), constraints=(/20: (/NULL - ]; /27: (/NULL - ]), fd=(20)==(27), (27)==(20)] + │ │ │ └── id:22 = cardsinfo.cardid:30 [outer=(22,30), constraints=(/22: (/NULL - ]; /30: (/NULL - ]), fd=(22)==(30), (30)==(22)] │ │ └── filters - │ │ └── id:20 = transactiondetails.cardid:4 [outer=(4,20), constraints=(/4: (/NULL - ]; /20: (/NULL - ]), fd=(4)==(20), (20)==(4)] + │ │ └── id:22 = transactiondetails.cardid:4 [outer=(4,22), constraints=(/4: (/NULL - ]; /22: (/NULL - ]), fd=(4)==(22), (22)==(4)] │ └── projections - │ ├── quantity:5 * transactiondetails.sellprice:6::DECIMAL [as=column39:39, outer=(5,6), immutable] - │ ├── quantity:5 * transactiondetails.buyprice:7::DECIMAL [as=column41:41, outer=(5,7), immutable] - │ ├── quantity:5 * (transactiondetails.sellprice:6::DECIMAL - transactiondetails.buyprice:7::DECIMAL) [as=column43:43, outer=(5-7), immutable] - │ └── extract('day', transactiondate:3) [as=column45:45, outer=(3), stable] + │ ├── quantity:5 * transactiondetails.sellprice:6::DECIMAL [as=column43:43, outer=(5,6), immutable] + │ ├── quantity:5 * transactiondetails.buyprice:7::DECIMAL [as=column45:45, outer=(5,7), immutable] + │ ├── quantity:5 * (transactiondetails.sellprice:6::DECIMAL - transactiondetails.buyprice:7::DECIMAL) [as=column47:47, outer=(5-7), immutable] + │ └── extract('day', transactiondate:3) [as=column49:49, outer=(3), stable] └── aggregations - ├── sum [as=sum:40, outer=(39)] - │ └── column39:39 - ├── sum [as=sum:42, outer=(41)] - │ └── column41:41 - └── sum [as=sum:44, outer=(43)] - └── column43:43 + ├── sum [as=sum:44, outer=(43)] + │ └── column43:43 + ├── sum [as=sum:46, outer=(45)] + │ └── column45:45 + └── sum [as=sum:48, outer=(47)] + └── column47:47 # Check if transaction was already inserted, for idempotency. # @@ -1049,21 +1049,21 @@ SELECT ) AS AlreadyInserted ---- values - ├── columns: alreadyinserted:11 + ├── columns: alreadyinserted:12 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(11) + ├── fd: ()-->(12) └── tuple └── any: eq ├── project - │ ├── columns: coalesce:10 + │ ├── columns: coalesce:11 │ ├── scan transactions │ │ ├── columns: dealerid:1!null isbuy:2!null operationid:6 │ │ ├── constraint: /1/2/3: [/1/false - /1/false] │ │ ├── lax-key: (6) │ │ └── fd: ()-->(1,2) │ └── projections - │ └── COALESCE(operationid:6, '00000000-0000-0000-0000-000000000000') [as=coalesce:10, outer=(6)] + │ └── COALESCE(operationid:6, '00000000-0000-0000-0000-000000000000') [as=coalesce:11, outer=(6)] └── '70f03eb1-4f58-4c26-b72d-c524a9d537dd' # Get account locations of a list of cards. @@ -1092,25 +1092,25 @@ ORDER BY sum(Quantity) DESC LIMIT 1000 ---- limit - ├── columns: accountname:10!null quantity:16!null - ├── internal-ordering: -16 + ├── columns: accountname:10!null quantity:17!null + ├── internal-ordering: -17 ├── cardinality: [0 - 1000] ├── key: (10) - ├── fd: (10)-->(16) - ├── ordering: -16 + ├── fd: (10)-->(17) + ├── ordering: -17 ├── sort - │ ├── columns: accountname:10!null sum:16!null + │ ├── columns: accountname:10!null sum:17!null │ ├── key: (10) - │ ├── fd: (10)-->(16) - │ ├── ordering: -16 + │ ├── fd: (10)-->(17) + │ ├── ordering: -17 │ ├── limit hint: 1000.00 │ └── group-by - │ ├── columns: accountname:10!null sum:16!null + │ ├── columns: accountname:10!null sum:17!null │ ├── grouping columns: accountname:10!null │ ├── key: (10) - │ ├── fd: (10)-->(16) + │ ├── fd: (10)-->(17) │ ├── project - │ │ ├── columns: quantity:15!null accountname:10!null + │ │ ├── columns: quantity:16!null accountname:10!null │ │ ├── inner-join (lookup inventorydetails) │ │ │ ├── columns: id:6!null quantity:7!null dealerid:8!null cardid:9!null accountname:10!null inventorydetails.quantity:11!null │ │ │ ├── key columns: [8 9 10] = [8 9 10] @@ -1130,10 +1130,10 @@ limit │ │ │ │ └── accountname:10 IN ('account-1', 'account-2', 'account-3') [outer=(10), constraints=(/10: [/'account-1' - /'account-1'] [/'account-2' - /'account-2'] [/'account-3' - /'account-3']; tight)] │ │ │ └── filters (true) │ │ └── projections - │ │ └── CASE WHEN quantity:7 < inventorydetails.quantity:11 THEN quantity:7 ELSE inventorydetails.quantity:11 END [as=quantity:15, outer=(7,11)] + │ │ └── CASE WHEN quantity:7 < inventorydetails.quantity:11 THEN quantity:7 ELSE inventorydetails.quantity:11 END [as=quantity:16, outer=(7,11)] │ └── aggregations - │ └── sum [as=sum:16, outer=(15)] - │ └── quantity:15 + │ └── sum [as=sum:17, outer=(16)] + │ └── quantity:16 └── 1000 # Get buy/sell volume of a particular card in the last 2 days. @@ -1162,31 +1162,31 @@ SELECT coalesce(( ), 0) ---- values - ├── columns: coalesce:21 + ├── columns: coalesce:23 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(21) + ├── fd: ()-->(23) └── tuple └── coalesce ├── subquery │ └── scalar-group-by - │ ├── columns: sum:20 + │ ├── columns: sum:22 │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(20) + │ ├── fd: ()-->(22) │ ├── limit - │ │ ├── columns: d.dealerid:1!null d.isbuy:2!null transactiondate:3!null cardid:4!null quantity:5!null t.dealerid:11!null t.isbuy:12!null date:13!null - │ │ ├── internal-ordering: -(3|13) opt(4) + │ │ ├── columns: d.dealerid:1!null d.isbuy:2!null transactiondate:3!null cardid:4!null quantity:5!null t.dealerid:12!null t.isbuy:13!null date:14!null + │ │ ├── internal-ordering: -(3|14) opt(4) │ │ ├── cardinality: [0 - 100] - │ │ ├── key: (5,11-13) - │ │ ├── fd: ()-->(4), (1)==(11), (11)==(1), (2)==(12), (12)==(2), (3)==(13), (13)==(3) + │ │ ├── key: (5,12-14) + │ │ ├── fd: ()-->(4), (1)==(12), (12)==(1), (2)==(13), (13)==(2), (3)==(14), (14)==(3) │ │ ├── inner-join (lookup transactions) - │ │ │ ├── columns: d.dealerid:1!null d.isbuy:2!null transactiondate:3!null cardid:4!null quantity:5!null t.dealerid:11!null t.isbuy:12!null date:13!null - │ │ │ ├── key columns: [1 2 3] = [11 12 13] + │ │ │ ├── columns: d.dealerid:1!null d.isbuy:2!null transactiondate:3!null cardid:4!null quantity:5!null t.dealerid:12!null t.isbuy:13!null date:14!null + │ │ │ ├── key columns: [1 2 3] = [12 13 14] │ │ │ ├── lookup columns are key - │ │ │ ├── key: (5,11-13) - │ │ │ ├── fd: ()-->(4), (1)==(11), (11)==(1), (2)==(12), (12)==(2), (3)==(13), (13)==(3) - │ │ │ ├── ordering: -(3|13) opt(4) [actual: -3] + │ │ │ ├── key: (5,12-14) + │ │ │ ├── fd: ()-->(4), (1)==(12), (12)==(1), (2)==(13), (13)==(2), (3)==(14), (14)==(3) + │ │ │ ├── ordering: -(3|14) opt(4) [actual: -3] │ │ │ ├── limit hint: 100.00 │ │ │ ├── sort │ │ │ │ ├── columns: d.dealerid:1!null d.isbuy:2!null transactiondate:3!null cardid:4!null quantity:5!null @@ -1210,12 +1210,12 @@ values │ │ │ │ ├── key: (1-3,5) │ │ │ │ └── fd: ()-->(4) │ │ │ └── filters - │ │ │ ├── (date:13 >= '2020-02-28 00:00:00+00:00') AND (date:13 <= '2020-03-01 00:00:00+00:00') [outer=(13), constraints=(/13: [/'2020-02-28 00:00:00+00:00' - /'2020-03-01 00:00:00+00:00']; tight)] - │ │ │ ├── ((((t.dealerid:11 = 1) OR (t.dealerid:11 = 2)) OR (t.dealerid:11 = 3)) OR (t.dealerid:11 = 4)) OR (t.dealerid:11 = 5) [outer=(11), constraints=(/11: [/1 - /1] [/2 - /2] [/3 - /3] [/4 - /4] [/5 - /5]; tight)] - │ │ │ └── t.isbuy:12 IN (false, true) [outer=(12), constraints=(/12: [/false - /false] [/true - /true]; tight)] + │ │ │ ├── (date:14 >= '2020-02-28 00:00:00+00:00') AND (date:14 <= '2020-03-01 00:00:00+00:00') [outer=(14), constraints=(/14: [/'2020-02-28 00:00:00+00:00' - /'2020-03-01 00:00:00+00:00']; tight)] + │ │ │ ├── ((((t.dealerid:12 = 1) OR (t.dealerid:12 = 2)) OR (t.dealerid:12 = 3)) OR (t.dealerid:12 = 4)) OR (t.dealerid:12 = 5) [outer=(12), constraints=(/12: [/1 - /1] [/2 - /2] [/3 - /3] [/4 - /4] [/5 - /5]; tight)] + │ │ │ └── t.isbuy:13 IN (false, true) [outer=(13), constraints=(/13: [/false - /false] [/true - /true]; tight)] │ │ └── 100 │ └── aggregations - │ └── sum [as=sum:20, outer=(5)] + │ └── sum [as=sum:22, outer=(5)] │ └── quantity:5 └── 0 @@ -1231,22 +1231,22 @@ VALUES (1, FALSE, '2020-03-01', 'the-account', 'the-customer', '70F03EB1-4F58-4C insert transactions ├── columns: ├── insert-mapping: - │ ├── column1:10 => dealerid:1 - │ ├── column2:11 => isbuy:2 - │ ├── column3:12 => date:3 - │ ├── column4:13 => accountname:4 - │ ├── column5:14 => customername:5 - │ ├── column6:15 => operationid:6 - │ ├── column16:16 => version:7 - │ └── column17:17 => olddate:8 + │ ├── column1:11 => dealerid:1 + │ ├── column2:12 => isbuy:2 + │ ├── column3:13 => date:3 + │ ├── column4:14 => accountname:4 + │ ├── column5:15 => customername:5 + │ ├── column6:16 => operationid:6 + │ ├── column17:17 => version:7 + │ └── column18:18 => olddate:8 ├── cardinality: [0 - 0] ├── volatile, mutations └── values - ├── columns: column1:10!null column2:11!null column3:12!null column4:13!null column5:14!null column6:15!null column16:16 column17:17!null + ├── columns: column1:11!null column2:12!null column3:13!null column4:14!null column5:15!null column6:16!null column17:17 column18:18!null ├── cardinality: [1 - 1] ├── volatile ├── key: () - ├── fd: ()-->(10-17) + ├── fd: ()-->(11-18) └── (1, false, '2020-03-01 00:00:00+00:00', 'the-account', 'the-customer', '70f03eb1-4f58-4c26-b72d-c524a9d537dd', cluster_logical_timestamp(), '0001-01-01 00:00:00+00:00') # Upsert buy or sell transaction. @@ -1256,44 +1256,44 @@ VALUES (1, FALSE, '2020-03-01', 'the-account', 'the-customer', '70F03EB1-4F58-4C ---- upsert transactions ├── columns: - ├── canary column: 18 - ├── fetch columns: dealerid:18 isbuy:19 date:20 accountname:21 customername:22 operationid:23 version:24 olddate:25 extra:26 + ├── canary column: 19 + ├── fetch columns: dealerid:19 isbuy:20 date:21 accountname:22 customername:23 operationid:24 version:25 olddate:26 extra:27 ├── insert-mapping: - │ ├── column1:10 => dealerid:1 - │ ├── column2:11 => isbuy:2 - │ ├── column3:12 => date:3 - │ ├── column4:13 => accountname:4 - │ ├── column5:14 => customername:5 - │ ├── column6:15 => operationid:6 - │ ├── column16:16 => version:7 - │ └── column17:17 => olddate:8 + │ ├── column1:11 => dealerid:1 + │ ├── column2:12 => isbuy:2 + │ ├── column3:13 => date:3 + │ ├── column4:14 => accountname:4 + │ ├── column5:15 => customername:5 + │ ├── column6:16 => operationid:6 + │ ├── column17:17 => version:7 + │ └── column18:18 => olddate:8 ├── update-mapping: - │ ├── column4:13 => accountname:4 - │ ├── column5:14 => customername:5 - │ ├── column6:15 => operationid:6 - │ └── column17:17 => olddate:8 + │ ├── column4:14 => accountname:4 + │ ├── column5:15 => customername:5 + │ ├── column6:16 => operationid:6 + │ └── column18:18 => olddate:8 ├── cardinality: [0 - 0] ├── volatile, mutations └── left-join (cross) - ├── columns: column1:10!null column2:11!null column3:12!null column4:13!null column5:14!null column6:15!null column16:16 column17:17!null dealerid:18 isbuy:19 date:20 accountname:21 customername:22 operationid:23 version:24 olddate:25 extra:26 + ├── columns: column1:11!null column2:12!null column3:13!null column4:14!null column5:15!null column6:16!null column17:17 column18:18!null dealerid:19 isbuy:20 date:21 accountname:22 customername:23 operationid:24 version:25 olddate:26 extra:27 ├── cardinality: [1 - 1] ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) ├── volatile ├── key: () - ├── fd: ()-->(10-26) + ├── fd: ()-->(11-27) ├── values - │ ├── columns: column1:10!null column2:11!null column3:12!null column4:13!null column5:14!null column6:15!null column16:16 column17:17!null + │ ├── columns: column1:11!null column2:12!null column3:13!null column4:14!null column5:15!null column6:16!null column17:17 column18:18!null │ ├── cardinality: [1 - 1] │ ├── volatile │ ├── key: () - │ ├── fd: ()-->(10-17) + │ ├── fd: ()-->(11-18) │ └── (1, false, '2020-03-01 00:00:00+00:00', 'the-account', 'the-customer', '70f03eb1-4f58-4c26-b72d-c524a9d537dd', cluster_logical_timestamp(), '0001-01-01 00:00:00+00:00') ├── scan transactions - │ ├── columns: dealerid:18!null isbuy:19!null date:20!null accountname:21!null customername:22!null operationid:23 version:24!null olddate:25 extra:26 - │ ├── constraint: /18/19/20: [/1/false/'2020-03-01 00:00:00+00:00' - /1/false/'2020-03-01 00:00:00+00:00'] + │ ├── columns: dealerid:19!null isbuy:20!null date:21!null accountname:22!null customername:23!null operationid:24 version:25!null olddate:26 extra:27 + │ ├── constraint: /19/20/21: [/1/false/'2020-03-01 00:00:00+00:00' - /1/false/'2020-03-01 00:00:00+00:00'] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(18-26) + │ └── fd: ()-->(19-27) └── filters (true) # Insert structured data (c=CardId, q=Quantity, s=SellPrice, b=BuyPrice) @@ -1313,113 +1313,113 @@ FROM updates ---- upsert transactiondetails ├── columns: - ├── canary column: 25 - ├── fetch columns: transactiondetails.dealerid:25 transactiondetails.isbuy:26 transactiondate:27 cardid:28 quantity:29 transactiondetails.sellprice:30 transactiondetails.buyprice:31 transactiondetails.version:32 transactiondetails.discount:33 transactiondetails.extra:34 + ├── canary column: 26 + ├── fetch columns: transactiondetails.dealerid:26 transactiondetails.isbuy:27 transactiondate:28 cardid:29 quantity:30 transactiondetails.sellprice:31 transactiondetails.buyprice:32 transactiondetails.version:33 transactiondetails.discount:34 transactiondetails.extra:35 ├── insert-mapping: - │ ├── "?column?":13 => transactiondetails.dealerid:2 - │ ├── bool:14 => transactiondetails.isbuy:3 - │ ├── current_timestamp:15 => transactiondate:4 - │ ├── int8:16 => cardid:5 - │ ├── int8:17 => quantity:6 - │ ├── sellprice:22 => transactiondetails.sellprice:7 - │ ├── buyprice:23 => transactiondetails.buyprice:8 - │ ├── column20:20 => transactiondetails.version:9 - │ └── discount:24 => transactiondetails.discount:10 + │ ├── "?column?":14 => transactiondetails.dealerid:2 + │ ├── bool:15 => transactiondetails.isbuy:3 + │ ├── current_timestamp:16 => transactiondate:4 + │ ├── int8:17 => cardid:5 + │ ├── int8:18 => quantity:6 + │ ├── sellprice:23 => transactiondetails.sellprice:7 + │ ├── buyprice:24 => transactiondetails.buyprice:8 + │ ├── column21:21 => transactiondetails.version:9 + │ └── discount:25 => transactiondetails.discount:10 ├── update-mapping: - │ ├── sellprice:22 => transactiondetails.sellprice:7 - │ ├── buyprice:23 => transactiondetails.buyprice:8 - │ └── discount:24 => transactiondetails.discount:10 + │ ├── sellprice:23 => transactiondetails.sellprice:7 + │ ├── buyprice:24 => transactiondetails.buyprice:8 + │ └── discount:25 => transactiondetails.discount:10 ├── input binding: &2 ├── cardinality: [0 - 0] ├── volatile, mutations ├── project - │ ├── columns: upsert_dealerid:35 upsert_isbuy:36 upsert_transactiondate:37 upsert_cardid:38 "?column?":13!null bool:14!null current_timestamp:15!null int8:16!null int8:17!null column20:20 sellprice:22 buyprice:23 discount:24!null transactiondetails.dealerid:25 transactiondetails.isbuy:26 transactiondate:27 cardid:28 quantity:29 transactiondetails.sellprice:30 transactiondetails.buyprice:31 transactiondetails.version:32 transactiondetails.discount:33 transactiondetails.extra:34 + │ ├── columns: upsert_dealerid:37 upsert_isbuy:38 upsert_transactiondate:39 upsert_cardid:40 "?column?":14!null bool:15!null current_timestamp:16!null int8:17!null int8:18!null column21:21 sellprice:23 buyprice:24 discount:25!null transactiondetails.dealerid:26 transactiondetails.isbuy:27 transactiondate:28 cardid:29 quantity:30 transactiondetails.sellprice:31 transactiondetails.buyprice:32 transactiondetails.version:33 transactiondetails.discount:34 transactiondetails.extra:35 │ ├── cardinality: [1 - 2] │ ├── volatile - │ ├── key: (16,17) - │ ├── fd: ()-->(13-15,24), (16,17)-->(20,22,23,25-34), (25-29)-->(30-34), (25)-->(35), (25,26)-->(36), (25,27)-->(37), (16,25,28)-->(38) + │ ├── key: (17,18) + │ ├── fd: ()-->(14-16,25), (17,18)-->(21,23,24,26-35), (26-30)-->(31-35), (26)-->(37), (26,27)-->(38), (26,28)-->(39), (17,26,29)-->(40) │ ├── left-join (lookup transactiondetails) - │ │ ├── columns: "?column?":13!null bool:14!null current_timestamp:15!null int8:16!null int8:17!null column20:20 sellprice:22 buyprice:23 discount:24!null transactiondetails.dealerid:25 transactiondetails.isbuy:26 transactiondate:27 cardid:28 quantity:29 transactiondetails.sellprice:30 transactiondetails.buyprice:31 transactiondetails.version:32 transactiondetails.discount:33 transactiondetails.extra:34 - │ │ ├── key columns: [13 14 15 16 17] = [25 26 27 28 29] + │ │ ├── columns: "?column?":14!null bool:15!null current_timestamp:16!null int8:17!null int8:18!null column21:21 sellprice:23 buyprice:24 discount:25!null transactiondetails.dealerid:26 transactiondetails.isbuy:27 transactiondate:28 cardid:29 quantity:30 transactiondetails.sellprice:31 transactiondetails.buyprice:32 transactiondetails.version:33 transactiondetails.discount:34 transactiondetails.extra:35 + │ │ ├── key columns: [14 15 16 17 18] = [26 27 28 29 30] │ │ ├── lookup columns are key │ │ ├── cardinality: [1 - 2] │ │ ├── volatile - │ │ ├── key: (16,17) - │ │ ├── fd: ()-->(13-15,24), (16,17)-->(20,22,23,25-34), (25-29)-->(30-34) + │ │ ├── key: (17,18) + │ │ ├── fd: ()-->(14-16,25), (17,18)-->(21,23,24,26-35), (26-30)-->(31-35) │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: "?column?":13!null bool:14!null current_timestamp:15!null int8:16!null int8:17!null column20:20 sellprice:22 buyprice:23 discount:24!null - │ │ │ ├── grouping columns: int8:16!null int8:17!null + │ │ │ ├── columns: "?column?":14!null bool:15!null current_timestamp:16!null int8:17!null int8:18!null column21:21 sellprice:23 buyprice:24 discount:25!null + │ │ │ ├── grouping columns: int8:17!null int8:18!null │ │ │ ├── error: "UPSERT or INSERT...ON CONFLICT command cannot affect row a second time" │ │ │ ├── cardinality: [1 - 2] │ │ │ ├── volatile - │ │ │ ├── key: (16,17) - │ │ │ ├── fd: ()-->(13-15,24), (16,17)-->(13-15,20,22-24) + │ │ │ ├── key: (17,18) + │ │ │ ├── fd: ()-->(14-16,25), (17,18)-->(14-16,21,23-25) │ │ │ ├── project - │ │ │ │ ├── columns: sellprice:22 buyprice:23 discount:24!null column20:20 "?column?":13!null bool:14!null current_timestamp:15!null int8:16!null int8:17!null + │ │ │ │ ├── columns: sellprice:23 buyprice:24 discount:25!null column21:21 "?column?":14!null bool:15!null current_timestamp:16!null int8:17!null int8:18!null │ │ │ │ ├── cardinality: [2 - 2] │ │ │ │ ├── volatile - │ │ │ │ ├── fd: ()-->(13-15,24) + │ │ │ │ ├── fd: ()-->(14-16,25) │ │ │ │ ├── values - │ │ │ │ │ ├── columns: detail_b:60!null detail_c:61!null detail_q:62!null detail_s:63!null + │ │ │ │ │ ├── columns: detail_b:64!null detail_c:65!null detail_q:66!null detail_s:67!null │ │ │ │ │ ├── cardinality: [2 - 2] │ │ │ │ │ ├── ('2.29', '49833', '4', '2.89') │ │ │ │ │ └── ('17.59', '29483', '2', '18.93') │ │ │ │ └── projections - │ │ │ │ ├── crdb_internal.round_decimal_values(detail_s:63::STRING::DECIMAL(10,4), 4) [as=sellprice:22, outer=(63), immutable] - │ │ │ │ ├── crdb_internal.round_decimal_values(detail_b:60::STRING::DECIMAL(10,4), 4) [as=buyprice:23, outer=(60), immutable] - │ │ │ │ ├── 0.0000 [as=discount:24] - │ │ │ │ ├── cluster_logical_timestamp() [as=column20:20, volatile] - │ │ │ │ ├── 1 [as="?column?":13] - │ │ │ │ ├── false [as=bool:14] - │ │ │ │ ├── '2017-05-10 13:00:00+00:00' [as=current_timestamp:15] - │ │ │ │ ├── detail_c:61::STRING::INT8 [as=int8:16, outer=(61), immutable] - │ │ │ │ └── detail_q:62::STRING::INT8 [as=int8:17, outer=(62), immutable] + │ │ │ │ ├── crdb_internal.round_decimal_values(detail_s:67::STRING::DECIMAL(10,4), 4) [as=sellprice:23, outer=(67), immutable] + │ │ │ │ ├── crdb_internal.round_decimal_values(detail_b:64::STRING::DECIMAL(10,4), 4) [as=buyprice:24, outer=(64), immutable] + │ │ │ │ ├── 0.0000 [as=discount:25] + │ │ │ │ ├── cluster_logical_timestamp() [as=column21:21, volatile] + │ │ │ │ ├── 1 [as="?column?":14] + │ │ │ │ ├── false [as=bool:15] + │ │ │ │ ├── '2017-05-10 13:00:00+00:00' [as=current_timestamp:16] + │ │ │ │ ├── detail_c:65::STRING::INT8 [as=int8:17, outer=(65), immutable] + │ │ │ │ └── detail_q:66::STRING::INT8 [as=int8:18, outer=(66), immutable] │ │ │ └── aggregations - │ │ │ ├── first-agg [as=sellprice:22, outer=(22)] - │ │ │ │ └── sellprice:22 - │ │ │ ├── first-agg [as=buyprice:23, outer=(23)] - │ │ │ │ └── buyprice:23 - │ │ │ ├── first-agg [as=column20:20, outer=(20)] - │ │ │ │ └── column20:20 - │ │ │ ├── first-agg [as=discount:24, outer=(24)] - │ │ │ │ └── discount:24 - │ │ │ ├── const-agg [as="?column?":13, outer=(13)] - │ │ │ │ └── "?column?":13 - │ │ │ ├── const-agg [as=bool:14, outer=(14)] - │ │ │ │ └── bool:14 - │ │ │ └── const-agg [as=current_timestamp:15, outer=(15)] - │ │ │ └── current_timestamp:15 + │ │ │ ├── first-agg [as=sellprice:23, outer=(23)] + │ │ │ │ └── sellprice:23 + │ │ │ ├── first-agg [as=buyprice:24, outer=(24)] + │ │ │ │ └── buyprice:24 + │ │ │ ├── first-agg [as=column21:21, outer=(21)] + │ │ │ │ └── column21:21 + │ │ │ ├── first-agg [as=discount:25, outer=(25)] + │ │ │ │ └── discount:25 + │ │ │ ├── const-agg [as="?column?":14, outer=(14)] + │ │ │ │ └── "?column?":14 + │ │ │ ├── const-agg [as=bool:15, outer=(15)] + │ │ │ │ └── bool:15 + │ │ │ └── const-agg [as=current_timestamp:16, outer=(16)] + │ │ │ └── current_timestamp:16 │ │ └── filters (true) │ └── projections - │ ├── CASE WHEN transactiondetails.dealerid:25 IS NULL THEN "?column?":13 ELSE transactiondetails.dealerid:25 END [as=upsert_dealerid:35, outer=(13,25)] - │ ├── CASE WHEN transactiondetails.dealerid:25 IS NULL THEN bool:14 ELSE transactiondetails.isbuy:26 END [as=upsert_isbuy:36, outer=(14,25,26)] - │ ├── CASE WHEN transactiondetails.dealerid:25 IS NULL THEN current_timestamp:15 ELSE transactiondate:27 END [as=upsert_transactiondate:37, outer=(15,25,27)] - │ └── CASE WHEN transactiondetails.dealerid:25 IS NULL THEN int8:16 ELSE cardid:28 END [as=upsert_cardid:38, outer=(16,25,28)] + │ ├── CASE WHEN transactiondetails.dealerid:26 IS NULL THEN "?column?":14 ELSE transactiondetails.dealerid:26 END [as=upsert_dealerid:37, outer=(14,26)] + │ ├── CASE WHEN transactiondetails.dealerid:26 IS NULL THEN bool:15 ELSE transactiondetails.isbuy:27 END [as=upsert_isbuy:38, outer=(15,26,27)] + │ ├── CASE WHEN transactiondetails.dealerid:26 IS NULL THEN current_timestamp:16 ELSE transactiondate:28 END [as=upsert_transactiondate:39, outer=(16,26,28)] + │ └── CASE WHEN transactiondetails.dealerid:26 IS NULL THEN int8:17 ELSE cardid:29 END [as=upsert_cardid:40, outer=(17,26,29)] └── f-k-checks ├── f-k-checks-item: transactiondetails(dealerid,isbuy,transactiondate) -> transactions(dealerid,isbuy,date) │ └── anti-join (lookup transactions) - │ ├── columns: upsert_dealerid:41 upsert_isbuy:42 upsert_transactiondate:43 - │ ├── key columns: [41 42 43] = [44 45 46] + │ ├── columns: upsert_dealerid:43 upsert_isbuy:44 upsert_transactiondate:45 + │ ├── key columns: [43 44 45] = [46 47 48] │ ├── lookup columns are key │ ├── cardinality: [0 - 2] │ ├── with-scan &2 - │ │ ├── columns: upsert_dealerid:41 upsert_isbuy:42 upsert_transactiondate:43 + │ │ ├── columns: upsert_dealerid:43 upsert_isbuy:44 upsert_transactiondate:45 │ │ ├── mapping: - │ │ │ ├── upsert_dealerid:35 => upsert_dealerid:41 - │ │ │ ├── upsert_isbuy:36 => upsert_isbuy:42 - │ │ │ └── upsert_transactiondate:37 => upsert_transactiondate:43 + │ │ │ ├── upsert_dealerid:37 => upsert_dealerid:43 + │ │ │ ├── upsert_isbuy:38 => upsert_isbuy:44 + │ │ │ └── upsert_transactiondate:39 => upsert_transactiondate:45 │ │ └── cardinality: [1 - 2] │ └── filters (true) └── f-k-checks-item: transactiondetails(cardid) -> cards(id) └── anti-join (lookup cards) - ├── columns: upsert_cardid:53 - ├── key columns: [53] = [54] + ├── columns: upsert_cardid:56 + ├── key columns: [56] = [57] ├── lookup columns are key ├── cardinality: [0 - 2] ├── with-scan &2 - │ ├── columns: upsert_cardid:53 + │ ├── columns: upsert_cardid:56 │ ├── mapping: - │ │ └── upsert_cardid:38 => upsert_cardid:53 + │ │ └── upsert_cardid:40 => upsert_cardid:56 │ └── cardinality: [1 - 2] └── filters (true) @@ -1430,18 +1430,18 @@ WHERE dealerid = 1 AND accountname = 'some-account' AND cardid = ANY ARRAY[29483 ---- delete inventorydetails ├── columns: - ├── fetch columns: dealerid:8 cardid:9 accountname:10 + ├── fetch columns: dealerid:9 cardid:10 accountname:11 ├── cardinality: [0 - 0] ├── volatile, mutations └── scan inventorydetails@inventorydetails_auto_index_inventorydetailscardidkey - ├── columns: dealerid:8!null cardid:9!null accountname:10!null - ├── constraint: /9/8/10 + ├── columns: dealerid:9!null cardid:10!null accountname:11!null + ├── constraint: /10/9/11 │ ├── [/294/1/'some-account' - /294/1/'some-account'] │ ├── [/1793/1/'some-account' - /1793/1/'some-account'] │ └── [/29483/1/'some-account' - /29483/1/'some-account'] ├── cardinality: [0 - 3] - ├── key: (9) - └── fd: ()-->(8,10) + ├── key: (10) + └── fd: ()-->(9,11) # Update CardsInfo inventory numbers (by CardId, Quantity) to reflect card # transfers. @@ -1460,122 +1460,122 @@ WHERE ci.cardid = Updates.c AND ci.dealerid = 1 ---- update ci ├── columns: - ├── fetch columns: ci.dealerid:19 ci.cardid:20 buyprice:21 sellprice:22 discount:23 desiredinventory:24 actualinventory:25 maxinventory:26 ci.version:27 ci.discountbuyprice:28 notes:29 oldinventory:30 ci.extra:31 + ├── fetch columns: ci.dealerid:20 ci.cardid:21 buyprice:22 sellprice:23 discount:24 desiredinventory:25 actualinventory:26 maxinventory:27 ci.version:28 ci.discountbuyprice:29 notes:30 oldinventory:31 ci.extra:32 ├── update-mapping: - │ ├── actualinventory_new:43 => actualinventory:12 - │ ├── discountbuyprice:47 => ci.discountbuyprice:15 - │ ├── column44:44 => notes:16 - │ └── column45:45 => oldinventory:17 + │ ├── actualinventory_new:46 => actualinventory:12 + │ ├── discountbuyprice:50 => ci.discountbuyprice:15 + │ ├── column47:47 => notes:16 + │ └── column48:48 => oldinventory:17 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: discountbuyprice:47 column44:44 column45:45!null actualinventory_new:43 ci.dealerid:19!null ci.cardid:20!null buyprice:21!null sellprice:22!null discount:23!null desiredinventory:24!null actualinventory:25!null maxinventory:26!null ci.version:27!null ci.discountbuyprice:28 notes:29 oldinventory:30 ci.extra:31 c:32!null q:33!null + ├── columns: discountbuyprice:50 column47:47 column48:48!null actualinventory_new:46 ci.dealerid:20!null ci.cardid:21!null buyprice:22!null sellprice:23!null discount:24!null desiredinventory:25!null actualinventory:26!null maxinventory:27!null ci.version:28!null ci.discountbuyprice:29 notes:30 oldinventory:31 ci.extra:32 c:34!null q:35!null ├── immutable - ├── key: (20) - ├── fd: ()-->(19,44,45), (20)-->(21-33,43,47), (27)-->(20-26,28-31), (20)==(32), (32)==(20) + ├── key: (21) + ├── fd: ()-->(20,47,48), (21)-->(22-32,34,35,46,50), (28)-->(21-27,29-32), (21)==(34), (34)==(21) ├── group-by - │ ├── columns: ci.dealerid:19!null ci.cardid:20!null buyprice:21!null sellprice:22!null discount:23!null desiredinventory:24!null actualinventory:25!null maxinventory:26!null ci.version:27!null ci.discountbuyprice:28 notes:29 oldinventory:30 ci.extra:31 c:32!null q:33!null sum_int:41 - │ ├── grouping columns: ci.cardid:20!null - │ ├── key: (20) - │ ├── fd: ()-->(19), (20)-->(19,21-33,41), (27)-->(20-26,28-31), (20)==(32), (32)==(20) + │ ├── columns: ci.dealerid:20!null ci.cardid:21!null buyprice:22!null sellprice:23!null discount:24!null desiredinventory:25!null actualinventory:26!null maxinventory:27!null ci.version:28!null ci.discountbuyprice:29 notes:30 oldinventory:31 ci.extra:32 c:34!null q:35!null sum_int:44 + │ ├── grouping columns: ci.cardid:21!null + │ ├── key: (21) + │ ├── fd: ()-->(20), (21)-->(20,22-32,34,35,44), (28)-->(21-27,29-32), (21)==(34), (34)==(21) │ ├── left-join (lookup inventorydetails) - │ │ ├── columns: ci.dealerid:19!null ci.cardid:20!null buyprice:21!null sellprice:22!null discount:23!null desiredinventory:24!null actualinventory:25!null maxinventory:26!null ci.version:27!null ci.discountbuyprice:28 notes:29 oldinventory:30 ci.extra:31 c:32!null q:33!null id.dealerid:34 id.cardid:35 quantity:37 - │ │ ├── key columns: [51 20] = [34 35] - │ │ ├── fd: ()-->(19), (20)-->(21-33), (27)-->(20-26,28-31), (20)==(32), (32)==(20) + │ │ ├── columns: ci.dealerid:20!null ci.cardid:21!null buyprice:22!null sellprice:23!null discount:24!null desiredinventory:25!null actualinventory:26!null maxinventory:27!null ci.version:28!null ci.discountbuyprice:29 notes:30 oldinventory:31 ci.extra:32 c:34!null q:35!null id.dealerid:36 id.cardid:37 quantity:39 + │ │ ├── key columns: [54 21] = [36 37] + │ │ ├── fd: ()-->(20), (21)-->(22-32,34,35), (28)-->(21-27,29-32), (21)==(34), (34)==(21) │ │ ├── project - │ │ │ ├── columns: "project_const_col_@34":51!null ci.dealerid:19!null ci.cardid:20!null buyprice:21!null sellprice:22!null discount:23!null desiredinventory:24!null actualinventory:25!null maxinventory:26!null ci.version:27!null ci.discountbuyprice:28 notes:29 oldinventory:30 ci.extra:31 c:32!null q:33!null + │ │ │ ├── columns: "project_const_col_@36":54!null ci.dealerid:20!null ci.cardid:21!null buyprice:22!null sellprice:23!null discount:24!null desiredinventory:25!null actualinventory:26!null maxinventory:27!null ci.version:28!null ci.discountbuyprice:29 notes:30 oldinventory:31 ci.extra:32 c:34!null q:35!null │ │ │ ├── cardinality: [0 - 2] - │ │ │ ├── key: (20) - │ │ │ ├── fd: ()-->(19,51), (20)-->(21-33), (27)-->(20-26,28-31), (20)==(32), (32)==(20) + │ │ │ ├── key: (21) + │ │ │ ├── fd: ()-->(20,54), (21)-->(22-32,34,35), (28)-->(21-27,29-32), (21)==(34), (34)==(21) │ │ │ ├── distinct-on - │ │ │ │ ├── columns: ci.dealerid:19!null ci.cardid:20!null buyprice:21!null sellprice:22!null discount:23!null desiredinventory:24!null actualinventory:25!null maxinventory:26!null ci.version:27!null ci.discountbuyprice:28 notes:29 oldinventory:30 ci.extra:31 c:32!null q:33!null - │ │ │ │ ├── grouping columns: ci.cardid:20!null + │ │ │ │ ├── columns: ci.dealerid:20!null ci.cardid:21!null buyprice:22!null sellprice:23!null discount:24!null desiredinventory:25!null actualinventory:26!null maxinventory:27!null ci.version:28!null ci.discountbuyprice:29 notes:30 oldinventory:31 ci.extra:32 c:34!null q:35!null + │ │ │ │ ├── grouping columns: ci.cardid:21!null │ │ │ │ ├── cardinality: [0 - 2] - │ │ │ │ ├── key: (20) - │ │ │ │ ├── fd: ()-->(19), (20)-->(19,21-33), (27)-->(20-26,28-31), (20)==(32), (32)==(20) + │ │ │ │ ├── key: (21) + │ │ │ │ ├── fd: ()-->(20), (21)-->(20,22-32,34,35), (28)-->(21-27,29-32), (21)==(34), (34)==(21) │ │ │ │ ├── inner-join (lookup cardsinfo) - │ │ │ │ │ ├── columns: ci.dealerid:19!null ci.cardid:20!null buyprice:21!null sellprice:22!null discount:23!null desiredinventory:24!null actualinventory:25!null maxinventory:26!null ci.version:27!null ci.discountbuyprice:28 notes:29 oldinventory:30 ci.extra:31 c:32!null q:33!null - │ │ │ │ │ ├── key columns: [48 32] = [19 20] + │ │ │ │ │ ├── columns: ci.dealerid:20!null ci.cardid:21!null buyprice:22!null sellprice:23!null discount:24!null desiredinventory:25!null actualinventory:26!null maxinventory:27!null ci.version:28!null ci.discountbuyprice:29 notes:30 oldinventory:31 ci.extra:32 c:34!null q:35!null + │ │ │ │ │ ├── key columns: [51 34] = [20 21] │ │ │ │ │ ├── lookup columns are key │ │ │ │ │ ├── cardinality: [0 - 2] - │ │ │ │ │ ├── fd: ()-->(19), (20)-->(21-31), (27)-->(20-26,28-31), (20)==(32), (32)==(20) + │ │ │ │ │ ├── fd: ()-->(20), (21)-->(22-32), (28)-->(21-27,29-32), (21)==(34), (34)==(21) │ │ │ │ │ ├── project - │ │ │ │ │ │ ├── columns: "project_const_col_@19":48!null c:32!null q:33!null + │ │ │ │ │ │ ├── columns: "project_const_col_@20":51!null c:34!null q:35!null │ │ │ │ │ │ ├── cardinality: [2 - 2] - │ │ │ │ │ │ ├── fd: ()-->(48) + │ │ │ │ │ │ ├── fd: ()-->(51) │ │ │ │ │ │ ├── values - │ │ │ │ │ │ │ ├── columns: c:32!null q:33!null + │ │ │ │ │ │ │ ├── columns: c:34!null q:35!null │ │ │ │ │ │ │ ├── cardinality: [2 - 2] │ │ │ │ │ │ │ ├── (42948, 3) │ │ │ │ │ │ │ └── (24924, 4) │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── 1 [as="project_const_col_@19":48] + │ │ │ │ │ │ └── 1 [as="project_const_col_@20":51] │ │ │ │ │ └── filters (true) │ │ │ │ └── aggregations - │ │ │ │ ├── first-agg [as=buyprice:21, outer=(21)] - │ │ │ │ │ └── buyprice:21 - │ │ │ │ ├── first-agg [as=sellprice:22, outer=(22)] - │ │ │ │ │ └── sellprice:22 - │ │ │ │ ├── first-agg [as=discount:23, outer=(23)] - │ │ │ │ │ └── discount:23 - │ │ │ │ ├── first-agg [as=desiredinventory:24, outer=(24)] - │ │ │ │ │ └── desiredinventory:24 - │ │ │ │ ├── first-agg [as=actualinventory:25, outer=(25)] - │ │ │ │ │ └── actualinventory:25 - │ │ │ │ ├── first-agg [as=maxinventory:26, outer=(26)] - │ │ │ │ │ └── maxinventory:26 - │ │ │ │ ├── first-agg [as=ci.version:27, outer=(27)] - │ │ │ │ │ └── ci.version:27 - │ │ │ │ ├── first-agg [as=ci.discountbuyprice:28, outer=(28)] - │ │ │ │ │ └── ci.discountbuyprice:28 - │ │ │ │ ├── first-agg [as=notes:29, outer=(29)] - │ │ │ │ │ └── notes:29 - │ │ │ │ ├── first-agg [as=oldinventory:30, outer=(30)] - │ │ │ │ │ └── oldinventory:30 - │ │ │ │ ├── first-agg [as=ci.extra:31, outer=(31)] - │ │ │ │ │ └── ci.extra:31 - │ │ │ │ ├── first-agg [as=c:32, outer=(32)] - │ │ │ │ │ └── c:32 - │ │ │ │ ├── first-agg [as=q:33, outer=(33)] - │ │ │ │ │ └── q:33 - │ │ │ │ └── const-agg [as=ci.dealerid:19, outer=(19)] - │ │ │ │ └── ci.dealerid:19 + │ │ │ │ ├── first-agg [as=buyprice:22, outer=(22)] + │ │ │ │ │ └── buyprice:22 + │ │ │ │ ├── first-agg [as=sellprice:23, outer=(23)] + │ │ │ │ │ └── sellprice:23 + │ │ │ │ ├── first-agg [as=discount:24, outer=(24)] + │ │ │ │ │ └── discount:24 + │ │ │ │ ├── first-agg [as=desiredinventory:25, outer=(25)] + │ │ │ │ │ └── desiredinventory:25 + │ │ │ │ ├── first-agg [as=actualinventory:26, outer=(26)] + │ │ │ │ │ └── actualinventory:26 + │ │ │ │ ├── first-agg [as=maxinventory:27, outer=(27)] + │ │ │ │ │ └── maxinventory:27 + │ │ │ │ ├── first-agg [as=ci.version:28, outer=(28)] + │ │ │ │ │ └── ci.version:28 + │ │ │ │ ├── first-agg [as=ci.discountbuyprice:29, outer=(29)] + │ │ │ │ │ └── ci.discountbuyprice:29 + │ │ │ │ ├── first-agg [as=notes:30, outer=(30)] + │ │ │ │ │ └── notes:30 + │ │ │ │ ├── first-agg [as=oldinventory:31, outer=(31)] + │ │ │ │ │ └── oldinventory:31 + │ │ │ │ ├── first-agg [as=ci.extra:32, outer=(32)] + │ │ │ │ │ └── ci.extra:32 + │ │ │ │ ├── first-agg [as=c:34, outer=(34)] + │ │ │ │ │ └── c:34 + │ │ │ │ ├── first-agg [as=q:35, outer=(35)] + │ │ │ │ │ └── q:35 + │ │ │ │ └── const-agg [as=ci.dealerid:20, outer=(20)] + │ │ │ │ └── ci.dealerid:20 │ │ │ └── projections - │ │ │ └── 1 [as="project_const_col_@34":51] + │ │ │ └── 1 [as="project_const_col_@36":54] │ │ └── filters (true) │ └── aggregations - │ ├── sum-int [as=sum_int:41, outer=(37)] - │ │ └── quantity:37 - │ ├── const-agg [as=ci.dealerid:19, outer=(19)] - │ │ └── ci.dealerid:19 - │ ├── const-agg [as=buyprice:21, outer=(21)] - │ │ └── buyprice:21 - │ ├── const-agg [as=sellprice:22, outer=(22)] - │ │ └── sellprice:22 - │ ├── const-agg [as=discount:23, outer=(23)] - │ │ └── discount:23 - │ ├── const-agg [as=desiredinventory:24, outer=(24)] - │ │ └── desiredinventory:24 - │ ├── const-agg [as=actualinventory:25, outer=(25)] - │ │ └── actualinventory:25 - │ ├── const-agg [as=maxinventory:26, outer=(26)] - │ │ └── maxinventory:26 - │ ├── const-agg [as=ci.version:27, outer=(27)] - │ │ └── ci.version:27 - │ ├── const-agg [as=ci.discountbuyprice:28, outer=(28)] - │ │ └── ci.discountbuyprice:28 - │ ├── const-agg [as=notes:29, outer=(29)] - │ │ └── notes:29 - │ ├── const-agg [as=oldinventory:30, outer=(30)] - │ │ └── oldinventory:30 - │ ├── const-agg [as=ci.extra:31, outer=(31)] - │ │ └── ci.extra:31 - │ ├── const-agg [as=c:32, outer=(32)] - │ │ └── c:32 - │ └── const-agg [as=q:33, outer=(33)] - │ └── q:33 + │ ├── sum-int [as=sum_int:44, outer=(39)] + │ │ └── quantity:39 + │ ├── const-agg [as=ci.dealerid:20, outer=(20)] + │ │ └── ci.dealerid:20 + │ ├── const-agg [as=buyprice:22, outer=(22)] + │ │ └── buyprice:22 + │ ├── const-agg [as=sellprice:23, outer=(23)] + │ │ └── sellprice:23 + │ ├── const-agg [as=discount:24, outer=(24)] + │ │ └── discount:24 + │ ├── const-agg [as=desiredinventory:25, outer=(25)] + │ │ └── desiredinventory:25 + │ ├── const-agg [as=actualinventory:26, outer=(26)] + │ │ └── actualinventory:26 + │ ├── const-agg [as=maxinventory:27, outer=(27)] + │ │ └── maxinventory:27 + │ ├── const-agg [as=ci.version:28, outer=(28)] + │ │ └── ci.version:28 + │ ├── const-agg [as=ci.discountbuyprice:29, outer=(29)] + │ │ └── ci.discountbuyprice:29 + │ ├── const-agg [as=notes:30, outer=(30)] + │ │ └── notes:30 + │ ├── const-agg [as=oldinventory:31, outer=(31)] + │ │ └── oldinventory:31 + │ ├── const-agg [as=ci.extra:32, outer=(32)] + │ │ └── ci.extra:32 + │ ├── const-agg [as=c:34, outer=(34)] + │ │ └── c:34 + │ └── const-agg [as=q:35, outer=(35)] + │ └── q:35 └── projections - ├── crdb_internal.round_decimal_values(buyprice:21::DECIMAL - discount:23::DECIMAL, 4) [as=discountbuyprice:47, outer=(21,23), immutable] - ├── CAST(NULL AS STRING) [as=column44:44] - ├── 0 [as=column45:45] - └── COALESCE(sum_int:41, 0) [as=actualinventory_new:43, outer=(41)] + ├── crdb_internal.round_decimal_values(buyprice:22::DECIMAL - discount:24::DECIMAL, 4) [as=discountbuyprice:50, outer=(22,24), immutable] + ├── CAST(NULL AS STRING) [as=column47:47] + ├── 0 [as=column48:48] + └── COALESCE(sum_int:44, 0) [as=actualinventory_new:46, outer=(44)] diff --git a/pkg/sql/opt/xform/testdata/external/ycsb b/pkg/sql/opt/xform/testdata/external/ycsb index 3412e23d76b3..42d11ab401e3 100644 --- a/pkg/sql/opt/xform/testdata/external/ycsb +++ b/pkg/sql/opt/xform/testdata/external/ycsb @@ -25,24 +25,24 @@ UPDATE usertable SET field5 = 'field5data' WHERE ycsb_key = 'user123' ---- update usertable ├── columns: - ├── fetch columns: ycsb_key:12 field5:18 + ├── fetch columns: ycsb_key:13 field5:19 ├── update-mapping: - │ └── field5_new:23 => field5:7 + │ └── field5_new:25 => field5:7 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: field5_new:23!null ycsb_key:12!null field5:18 + ├── columns: field5_new:25!null ycsb_key:13!null field5:19 ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(12,18,23) + ├── fd: ()-->(13,19,25) ├── scan usertable - │ ├── columns: ycsb_key:12!null field5:18 - │ ├── constraint: /12: [/'user123' - /'user123'] + │ ├── columns: ycsb_key:13!null field5:19 + │ ├── constraint: /13: [/'user123' - /'user123'] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(12,18) + │ └── fd: ()-->(13,19) └── projections - └── 'field5data' [as=field5_new:23] + └── 'field5data' [as=field5_new:25] # -------------------------------------------------- # Workload B: Read mostly @@ -93,24 +93,24 @@ INSERT INTO usertable VALUES ( insert usertable ├── columns: ├── insert-mapping: - │ ├── column1:12 => ycsb_key:1 - │ ├── column2:13 => field0:2 - │ ├── column3:14 => field1:3 - │ ├── column4:15 => field2:4 - │ ├── column5:16 => field3:5 - │ ├── column6:17 => field4:6 - │ ├── column7:18 => field5:7 - │ ├── column8:19 => field6:8 - │ ├── column9:20 => field7:9 - │ ├── column10:21 => field8:10 - │ └── column11:22 => field9:11 + │ ├── column1:13 => ycsb_key:1 + │ ├── column2:14 => field0:2 + │ ├── column3:15 => field1:3 + │ ├── column4:16 => field2:4 + │ ├── column5:17 => field3:5 + │ ├── column6:18 => field4:6 + │ ├── column7:19 => field5:7 + │ ├── column8:20 => field6:8 + │ ├── column9:21 => field7:9 + │ ├── column10:22 => field8:10 + │ └── column11:23 => field9:11 ├── cardinality: [0 - 0] ├── volatile, mutations └── values - ├── columns: column1:12!null column2:13!null column3:14!null column4:15!null column5:16!null column6:17!null column7:18!null column8:19!null column9:20!null column10:21!null column11:22!null + ├── columns: column1:13!null column2:14!null column3:15!null column4:16!null column5:17!null column6:18!null column7:19!null column8:20!null column9:21!null column10:22!null column11:23!null ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(12-22) + ├── fd: ()-->(13-23) └── ('user123', 'field0data', 'field1data', 'field2data', 'field3data', 'field4data', 'field5data', 'field6data', 'field7data', 'field8data', 'field9data') # -------------------------------------------------- @@ -169,21 +169,21 @@ UPDATE usertable SET field5 = 'field5data' WHERE ycsb_key = 'user123' ---- update usertable ├── columns: - ├── fetch columns: ycsb_key:12 field5:18 + ├── fetch columns: ycsb_key:13 field5:19 ├── update-mapping: - │ └── field5_new:23 => field5:7 + │ └── field5_new:25 => field5:7 ├── cardinality: [0 - 0] ├── volatile, mutations └── project - ├── columns: field5_new:23!null ycsb_key:12!null field5:18 + ├── columns: field5_new:25!null ycsb_key:13!null field5:19 ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(12,18,23) + ├── fd: ()-->(13,19,25) ├── scan usertable - │ ├── columns: ycsb_key:12!null field5:18 - │ ├── constraint: /12: [/'user123' - /'user123'] + │ ├── columns: ycsb_key:13!null field5:19 + │ ├── constraint: /13: [/'user123' - /'user123'] │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(12,18) + │ └── fd: ()-->(13,19) └── projections - └── 'field5data' [as=field5_new:23] + └── 'field5data' [as=field5_new:25] diff --git a/pkg/sql/opt/xform/testdata/physprops/limit_hint b/pkg/sql/opt/xform/testdata/physprops/limit_hint index 9ce2d062da4c..01193f83bf9d 100644 --- a/pkg/sql/opt/xform/testdata/physprops/limit_hint +++ b/pkg/sql/opt/xform/testdata/physprops/limit_hint @@ -55,17 +55,17 @@ opt SELECT * FROM (SELECT * FROM t OFFSET 5) WITH ORDINALITY ORDER BY ordinality LIMIT 10 ---- limit - ├── columns: x:1!null y:2 z:3 ordinality:4!null - ├── internal-ordering: +4 + ├── columns: x:1!null y:2 z:3 ordinality:5!null + ├── internal-ordering: +5 ├── cardinality: [0 - 10] ├── key: (1) - ├── fd: (1)-->(2-4), (4)-->(1-3) - ├── ordering: +4 + ├── fd: (1)-->(2,3,5), (5)-->(1-3) + ├── ordering: +5 ├── ordinality - │ ├── columns: x:1!null y:2 z:3 ordinality:4!null + │ ├── columns: x:1!null y:2 z:3 ordinality:5!null │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (4)-->(1-3) - │ ├── ordering: +4 + │ ├── fd: (1)-->(2,3,5), (5)-->(1-3) + │ ├── ordering: +5 │ ├── limit hint: 10.00 │ └── offset │ ├── columns: x:1!null y:2 z:3 @@ -89,14 +89,14 @@ opt SELECT * FROM (SELECT * FROM t UNION SELECT * from t) LIMIT 10 ---- limit - ├── columns: x:7!null y:8 z:9 + ├── columns: x:9!null y:10 z:11 ├── cardinality: [0 - 10] - ├── key: (7-9) + ├── key: (9-11) ├── union - │ ├── columns: x:7!null y:8 z:9 + │ ├── columns: x:9!null y:10 z:11 │ ├── left columns: t.x:1 t.y:2 t.z:3 - │ ├── right columns: t.x:4 t.y:5 t.z:6 - │ ├── key: (7-9) + │ ├── right columns: t.x:5 t.y:6 t.z:7 + │ ├── key: (9-11) │ ├── limit hint: 10.00 │ ├── scan t │ │ ├── columns: t.x:1!null t.y:2 t.z:3 @@ -104,9 +104,9 @@ limit │ │ ├── fd: (1)-->(2,3) │ │ └── limit hint: 10.00 │ └── scan t - │ ├── columns: t.x:4!null t.y:5 t.z:6 - │ ├── key: (4) - │ ├── fd: (4)-->(5,6) + │ ├── columns: t.x:5!null t.y:6 t.z:7 + │ ├── key: (5) + │ ├── fd: (5)-->(6,7) │ └── limit hint: 10.00 └── 10 @@ -114,12 +114,12 @@ opt SELECT * FROM (SELECT * FROM t UNION ALL SELECT * from t) LIMIT 10 ---- limit - ├── columns: x:7!null y:8 z:9 + ├── columns: x:9!null y:10 z:11 ├── cardinality: [0 - 10] ├── union-all - │ ├── columns: x:7!null y:8 z:9 + │ ├── columns: x:9!null y:10 z:11 │ ├── left columns: t.x:1 t.y:2 t.z:3 - │ ├── right columns: t.x:4 t.y:5 t.z:6 + │ ├── right columns: t.x:5 t.y:6 t.z:7 │ ├── limit hint: 10.00 │ ├── scan t │ │ ├── columns: t.x:1!null t.y:2 t.z:3 @@ -127,9 +127,9 @@ limit │ │ ├── fd: (1)-->(2,3) │ │ └── limit hint: 10.00 │ └── scan t - │ ├── columns: t.x:4!null t.y:5 t.z:6 - │ ├── key: (4) - │ ├── fd: (4)-->(5,6) + │ ├── columns: t.x:5!null t.y:6 t.z:7 + │ ├── key: (5) + │ ├── fd: (5)-->(6,7) │ └── limit hint: 10.00 └── 10 @@ -143,14 +143,14 @@ limit ├── intersect │ ├── columns: z:3 │ ├── left columns: z:3 - │ ├── right columns: y:5 + │ ├── right columns: y:6 │ ├── key: (3) │ ├── limit hint: 10.00 │ ├── scan t │ │ ├── columns: z:3 │ │ └── limit hint: 10.00 │ └── scan t@y_idx - │ ├── columns: y:5 + │ ├── columns: y:6 │ └── limit hint: 10.00 └── 10 @@ -163,13 +163,13 @@ limit ├── intersect-all │ ├── columns: z:3 │ ├── left columns: z:3 - │ ├── right columns: y:5 + │ ├── right columns: y:6 │ ├── limit hint: 10.00 │ ├── scan t │ │ ├── columns: z:3 │ │ └── limit hint: 10.00 │ └── scan t@y_idx - │ ├── columns: y:5 + │ ├── columns: y:6 │ └── limit hint: 10.00 └── 10 @@ -183,14 +183,14 @@ limit ├── except │ ├── columns: z:3 │ ├── left columns: z:3 - │ ├── right columns: y:5 + │ ├── right columns: y:6 │ ├── key: (3) │ ├── limit hint: 10.00 │ ├── scan t │ │ ├── columns: z:3 │ │ └── limit hint: 10.00 │ └── scan t@y_idx - │ ├── columns: y:5 + │ ├── columns: y:6 │ └── limit hint: 10.00 └── 10 @@ -203,13 +203,13 @@ limit ├── except-all │ ├── columns: z:3 │ ├── left columns: z:3 - │ ├── right columns: y:5 + │ ├── right columns: y:6 │ ├── limit hint: 10.00 │ ├── scan t │ │ ├── columns: z:3 │ │ └── limit hint: 10.00 │ └── scan t@y_idx - │ ├── columns: y:5 + │ ├── columns: y:6 │ └── limit hint: 10.00 └── 10 @@ -307,17 +307,17 @@ opt SELECT z FROM t@y_idx WITH ORDINALITY ORDER BY ordinality LIMIT 10 ---- limit - ├── columns: z:3 [hidden: ordinality:4!null] - ├── internal-ordering: +4 + ├── columns: z:3 [hidden: ordinality:5!null] + ├── internal-ordering: +5 ├── cardinality: [0 - 10] - ├── key: (4) - ├── fd: (4)-->(3) - ├── ordering: +4 + ├── key: (5) + ├── fd: (5)-->(3) + ├── ordering: +5 ├── ordinality - │ ├── columns: z:3 ordinality:4!null - │ ├── key: (4) - │ ├── fd: (4)-->(3) - │ ├── ordering: +4 + │ ├── columns: z:3 ordinality:5!null + │ ├── key: (5) + │ ├── fd: (5)-->(3) + │ ├── ordering: +5 │ ├── limit hint: 10.00 │ └── index-join t │ ├── columns: z:3 @@ -334,17 +334,17 @@ opt SELECT * FROM t WITH ORDINALITY ORDER BY ordinality LIMIT 10 ---- limit - ├── columns: x:1!null y:2 z:3 ordinality:4!null - ├── internal-ordering: +4 + ├── columns: x:1!null y:2 z:3 ordinality:5!null + ├── internal-ordering: +5 ├── cardinality: [0 - 10] ├── key: (1) - ├── fd: (1)-->(2-4), (4)-->(1-3) - ├── ordering: +4 + ├── fd: (1)-->(2,3,5), (5)-->(1-3) + ├── ordering: +5 ├── ordinality - │ ├── columns: x:1!null y:2 z:3 ordinality:4!null + │ ├── columns: x:1!null y:2 z:3 ordinality:5!null │ ├── key: (1) - │ ├── fd: (1)-->(2-4), (4)-->(1-3) - │ ├── ordering: +4 + │ ├── fd: (1)-->(2,3,5), (5)-->(1-3) + │ ├── ordering: +5 │ ├── limit hint: 10.00 │ └── scan t │ ├── columns: x:1!null y:2 z:3 @@ -358,26 +358,26 @@ opt SELECT * FROM (SELECT 1 FROM t) WITH ORDINALITY ORDER BY ordinality LIMIT 10 ---- limit - ├── columns: "?column?":4!null ordinality:5!null - ├── internal-ordering: +5 opt(4) + ├── columns: "?column?":5!null ordinality:6!null + ├── internal-ordering: +6 opt(5) ├── cardinality: [0 - 10] - ├── key: (5) - ├── fd: ()-->(4) - ├── ordering: +5 opt(4) [actual: +5] + ├── key: (6) + ├── fd: ()-->(5) + ├── ordering: +6 opt(5) [actual: +6] ├── ordinality - │ ├── columns: "?column?":4!null ordinality:5!null - │ ├── key: (5) - │ ├── fd: ()-->(4) - │ ├── ordering: +5 opt(4) [actual: +5] + │ ├── columns: "?column?":5!null ordinality:6!null + │ ├── key: (6) + │ ├── fd: ()-->(5) + │ ├── ordering: +6 opt(5) [actual: +6] │ ├── limit hint: 10.00 │ └── project - │ ├── columns: "?column?":4!null - │ ├── fd: ()-->(4) + │ ├── columns: "?column?":5!null + │ ├── fd: ()-->(5) │ ├── limit hint: 10.00 │ ├── scan t@y_idx │ │ └── limit hint: 10.00 │ └── projections - │ └── 1 [as="?column?":4] + │ └── 1 [as="?column?":5] └── 10 # ProjectSet operator. @@ -385,12 +385,12 @@ opt SELECT *, generate_series(1, t.x) FROM t LIMIT 10 ---- limit - ├── columns: x:1!null y:2 z:3 generate_series:4 + ├── columns: x:1!null y:2 z:3 generate_series:5 ├── cardinality: [0 - 10] ├── immutable ├── fd: (1)-->(2,3) ├── project-set - │ ├── columns: x:1!null y:2 z:3 generate_series:4 + │ ├── columns: x:1!null y:2 z:3 generate_series:5 │ ├── immutable │ ├── fd: (1)-->(2,3) │ ├── limit hint: 10.00 @@ -443,23 +443,23 @@ opt SELECT * FROM a JOIN b ON k=z WHERE x > 0 AND x <= 5000 LIMIT 6003 ---- limit - ├── columns: k:1!null i:2 s:3 d:4!null x:5!null z:6!null + ├── columns: k:1!null i:2 s:3 d:4!null x:6!null z:7!null ├── cardinality: [0 - 6003] - ├── fd: (1)-->(2-4), (1)==(6), (6)==(1) + ├── fd: (1)-->(2-4), (1)==(7), (7)==(1) ├── inner-join (lookup a) - │ ├── columns: k:1!null i:2 s:3 d:4!null x:5!null z:6!null - │ ├── key columns: [6] = [1] + │ ├── columns: k:1!null i:2 s:3 d:4!null x:6!null z:7!null + │ ├── key columns: [7] = [1] │ ├── lookup columns are key - │ ├── fd: (1)-->(2-4), (1)==(6), (6)==(1) + │ ├── fd: (1)-->(2-4), (1)==(7), (7)==(1) │ ├── limit hint: 6003.00 │ ├── select - │ │ ├── columns: x:5!null z:6!null + │ │ ├── columns: x:6!null z:7!null │ │ ├── limit hint: 6100.00 │ │ ├── scan b - │ │ │ ├── columns: x:5 z:6!null + │ │ │ ├── columns: x:6 z:7!null │ │ │ └── limit hint: 6100.00 │ │ └── filters - │ │ └── (x:5 > 0) AND (x:5 <= 5000) [outer=(5)] + │ │ └── (x:6 > 0) AND (x:6 <= 5000) [outer=(6)] │ └── filters (true) └── 6003 @@ -468,23 +468,23 @@ opt SELECT * FROM a JOIN b ON k=z WHERE x > 0 AND x <= 5000 LIMIT 3 ---- limit - ├── columns: k:1!null i:2 s:3 d:4!null x:5!null z:6!null + ├── columns: k:1!null i:2 s:3 d:4!null x:6!null z:7!null ├── cardinality: [0 - 3] - ├── fd: (1)-->(2-4), (1)==(6), (6)==(1) + ├── fd: (1)-->(2-4), (1)==(7), (7)==(1) ├── inner-join (lookup a) - │ ├── columns: k:1!null i:2 s:3 d:4!null x:5!null z:6!null - │ ├── key columns: [6] = [1] + │ ├── columns: k:1!null i:2 s:3 d:4!null x:6!null z:7!null + │ ├── key columns: [7] = [1] │ ├── lookup columns are key - │ ├── fd: (1)-->(2-4), (1)==(6), (6)==(1) + │ ├── fd: (1)-->(2-4), (1)==(7), (7)==(1) │ ├── limit hint: 3.00 │ ├── select - │ │ ├── columns: x:5!null z:6!null + │ │ ├── columns: x:6!null z:7!null │ │ ├── limit hint: 100.00 │ │ ├── scan b - │ │ │ ├── columns: x:5 z:6!null + │ │ │ ├── columns: x:6 z:7!null │ │ │ └── limit hint: 100.00 │ │ └── filters - │ │ └── (x:5 > 0) AND (x:5 <= 5000) [outer=(5)] + │ │ └── (x:6 > 0) AND (x:6 <= 5000) [outer=(6)] │ └── filters (true) └── 3 @@ -552,41 +552,41 @@ opt SELECT * FROM v0, t0 NATURAL JOIN t1 LIMIT -1 ---- project - ├── columns: c0:3!null c0:4!null + ├── columns: c0:4!null c0:5!null ├── cardinality: [0 - 0] ├── immutable ├── key: () - ├── fd: ()-->(3,4) + ├── fd: ()-->(4,5) └── limit - ├── columns: "?column?":3!null t0.c0:4!null t1.c0:6!null + ├── columns: "?column?":4!null t0.c0:5!null t1.c0:8!null ├── cardinality: [0 - 0] ├── immutable ├── key: () - ├── fd: ()-->(3,4,6) + ├── fd: ()-->(4,5,8) ├── inner-join (cross) - │ ├── columns: "?column?":3!null t0.c0:4!null t1.c0:6!null + │ ├── columns: "?column?":4!null t0.c0:5!null t1.c0:8!null │ ├── cardinality: [0 - 0] │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) │ ├── immutable - │ ├── fd: ()-->(3), (4)==(6), (6)==(4) + │ ├── fd: ()-->(4), (5)==(8), (8)==(5) │ ├── limit hint: 1.00 │ ├── inner-join (hash) - │ │ ├── columns: t0.c0:4!null t1.c0:6!null + │ │ ├── columns: t0.c0:5!null t1.c0:8!null │ │ ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) - │ │ ├── fd: (4)==(6), (6)==(4) + │ │ ├── fd: (5)==(8), (8)==(5) │ │ ├── scan t0 - │ │ │ ├── columns: t0.c0:4 - │ │ │ └── lax-key: (4) + │ │ │ ├── columns: t0.c0:5 + │ │ │ └── lax-key: (5) │ │ ├── scan t1 - │ │ │ └── columns: t1.c0:6 + │ │ │ └── columns: t1.c0:8 │ │ └── filters - │ │ └── t0.c0:4 = t1.c0:6 [outer=(4,6), fd=(4)==(6), (6)==(4)] + │ │ └── t0.c0:5 = t1.c0:8 [outer=(5,8), fd=(5)==(8), (8)==(5)] │ ├── project - │ │ ├── columns: "?column?":3!null + │ │ ├── columns: "?column?":4!null │ │ ├── cardinality: [0 - 0] │ │ ├── immutable │ │ ├── key: () - │ │ ├── fd: ()-->(3) + │ │ ├── fd: ()-->(4) │ │ ├── limit │ │ │ ├── cardinality: [0 - 0] │ │ │ ├── immutable @@ -595,6 +595,6 @@ project │ │ │ │ └── limit hint: 1.00 │ │ │ └── -1 │ │ └── projections - │ │ └── 0 [as="?column?":3] + │ │ └── 0 [as="?column?":4] │ └── filters (true) └── -1 diff --git a/pkg/sql/opt/xform/testdata/physprops/ordering b/pkg/sql/opt/xform/testdata/physprops/ordering index 857efbf706ef..5c69ec4b6b44 100644 --- a/pkg/sql/opt/xform/testdata/physprops/ordering +++ b/pkg/sql/opt/xform/testdata/physprops/ordering @@ -129,33 +129,33 @@ opt SELECT x+1 AS r, y FROM a ORDER BY x, y DESC ---- project - ├── columns: r:5!null y:2!null [hidden: x:1!null] + ├── columns: r:6!null y:2!null [hidden: x:1!null] ├── immutable ├── key: (1,2) - ├── fd: (1)-->(5) + ├── fd: (1)-->(6) ├── ordering: +1,-2 ├── scan a │ ├── columns: x:1!null y:2!null │ ├── key: (1,2) │ └── ordering: +1,-2 └── projections - └── x:1 + 1 [as=r:5, outer=(1), immutable] + └── x:1 + 1 [as=r:6, outer=(1), immutable] # Pass through ordering to scan operator that can't support it. opt SELECT y, x, z+1 AS r FROM a ORDER BY x, y ---- sort (segmented) - ├── columns: y:2!null x:1!null r:5 + ├── columns: y:2!null x:1!null r:6 ├── immutable ├── key: (1,2) - ├── fd: (1,2)-->(5) + ├── fd: (1,2)-->(6) ├── ordering: +1,+2 └── project - ├── columns: r:5 x:1!null y:2!null + ├── columns: r:6 x:1!null y:2!null ├── immutable ├── key: (1,2) - ├── fd: (1,2)-->(5) + ├── fd: (1,2)-->(6) ├── ordering: +1 ├── scan a │ ├── columns: x:1!null y:2!null z:3 @@ -163,30 +163,30 @@ sort (segmented) │ ├── fd: (1,2)-->(3) │ └── ordering: +1 └── projections - └── z:3 + 1 [as=r:5, outer=(3), immutable] + └── z:3 + 1 [as=r:6, outer=(3), immutable] # Ordering cannot be passed through because it includes computed column. opt SELECT x, y+1 AS computed, y FROM a ORDER BY x, computed ---- sort (segmented) - ├── columns: x:1!null computed:5!null y:2!null + ├── columns: x:1!null computed:6!null y:2!null ├── immutable ├── key: (1,2) - ├── fd: (1,2)-->(5) - ├── ordering: +1,+5 + ├── fd: (1,2)-->(6) + ├── ordering: +1,+6 └── project - ├── columns: computed:5!null x:1!null y:2!null + ├── columns: computed:6!null x:1!null y:2!null ├── immutable ├── key: (1,2) - ├── fd: (1,2)-->(5) + ├── fd: (1,2)-->(6) ├── ordering: +1 ├── scan a │ ├── columns: x:1!null y:2!null │ ├── key: (1,2) │ └── ordering: +1 └── projections - └── y:2 + 1.0 [as=computed:5, outer=(2), immutable] + └── y:2 + 1.0 [as=computed:6, outer=(2), immutable] # Ordering on an expression that gets constant-folded to a simple variable. # Example from #43360: a boolean (possibly a placeholder) indicates the sort @@ -195,48 +195,48 @@ opt SELECT * FROM a ORDER BY CASE WHEN false THEN x END ASC, CASE WHEN NOT false THEN x END DESC ---- project - ├── columns: x:1!null y:2!null z:3 s:4!null [hidden: column6:6!null] + ├── columns: x:1!null y:2!null z:3 s:4!null [hidden: column7:7!null] ├── key: (1,2) - ├── fd: (1,2)-->(3,4), (1)==(6), (6)==(1) - ├── ordering: -(1|6) [actual: -1] + ├── fd: (1,2)-->(3,4), (1)==(7), (7)==(1) + ├── ordering: -(1|7) [actual: -1] ├── scan a,rev │ ├── columns: x:1!null y:2!null z:3 s:4!null │ ├── key: (1,2) │ ├── fd: (1,2)-->(3,4) │ └── ordering: -1 └── projections - └── x:1 [as=column6:6, outer=(1)] + └── x:1 [as=column7:7, outer=(1)] opt SELECT * FROM a ORDER BY CASE WHEN true THEN x END ASC, CASE WHEN NOT true THEN x END DESC ---- project - ├── columns: x:1!null y:2!null z:3 s:4!null [hidden: column5:5!null] + ├── columns: x:1!null y:2!null z:3 s:4!null [hidden: column6:6!null] ├── key: (1,2) - ├── fd: (1,2)-->(3,4), (1)==(5), (5)==(1) - ├── ordering: +(1|5) [actual: +1] + ├── fd: (1,2)-->(3,4), (1)==(6), (6)==(1) + ├── ordering: +(1|6) [actual: +1] ├── scan a │ ├── columns: x:1!null y:2!null z:3 s:4!null │ ├── key: (1,2) │ ├── fd: (1,2)-->(3,4) │ └── ordering: +1 └── projections - └── x:1 [as=column5:5, outer=(1)] + └── x:1 [as=column6:6, outer=(1)] # Similar case, except the equivalent input column is not being projected. opt SELECT 1 FROM a ORDER BY CASE WHEN false THEN x END ASC, CASE WHEN NOT false THEN x END DESC ---- project - ├── columns: "?column?":5!null [hidden: column7:7!null] - ├── fd: ()-->(5) - ├── ordering: -7 opt(5) [actual: -7] + ├── columns: "?column?":6!null [hidden: column8:8!null] + ├── fd: ()-->(6) + ├── ordering: -8 opt(6) [actual: -8] ├── scan a,rev │ ├── columns: x:1!null │ └── ordering: -1 └── projections - ├── 1 [as="?column?":5] - └── x:1 [as=column7:7, outer=(1)] + ├── 1 [as="?column?":6] + └── x:1 [as=column8:8, outer=(1)] # -------------------------------------------------- # Select + Project operators (pass through both). @@ -247,10 +247,10 @@ opt SELECT y, x-1 AS z FROM a WHERE x>y ORDER BY x, y DESC ---- project - ├── columns: y:2!null z:5!null [hidden: x:1!null] + ├── columns: y:2!null z:6!null [hidden: x:1!null] ├── immutable ├── key: (1,2) - ├── fd: (1)-->(5) + ├── fd: (1)-->(6) ├── ordering: +1,-2 ├── select │ ├── columns: x:1!null y:2!null @@ -263,14 +263,14 @@ project │ └── filters │ └── x:1 > y:2 [outer=(1,2)] └── projections - └── x:1 - 1 [as=z:5, outer=(1), immutable] + └── x:1 - 1 [as=z:6, outer=(1), immutable] memo SELECT y, x-1 AS z FROM a WHERE x>y ORDER BY x, y DESC ---- -memo (optimized, ~5KB, required=[presentation: y:2,z:5] [ordering: +1,-2]) +memo (optimized, ~5KB, required=[presentation: y:2,z:6] [ordering: +1,-2]) ├── G1: (project G2 G3 x y) - │ ├── [presentation: y:2,z:5] [ordering: +1,-2] + │ ├── [presentation: y:2,z:6] [ordering: +1,-2] │ │ ├── best: (project G2="[ordering: +1,-2]" G3 x y) │ │ └── cost: 1076.72 │ └── [] @@ -358,35 +358,35 @@ opt SELECT array_agg(z) FROM (SELECT * FROM a ORDER BY y) ---- scalar-group-by - ├── columns: array_agg:5 + ├── columns: array_agg:6 ├── internal-ordering: +2 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(5) + ├── fd: ()-->(6) ├── sort │ ├── columns: y:2!null z:3 │ ├── ordering: +2 │ └── scan a │ └── columns: y:2!null z:3 └── aggregations - └── array-agg [as=array_agg:5, outer=(3)] + └── array-agg [as=array_agg:6, outer=(3)] └── z:3 opt SELECT array_agg(x) FROM (SELECT * FROM a ORDER BY x, y DESC) ---- scalar-group-by - ├── columns: array_agg:5 + ├── columns: array_agg:6 ├── internal-ordering: +1,-2 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(5) + ├── fd: ()-->(6) ├── scan a │ ├── columns: x:1!null y:2!null │ ├── key: (1,2) │ └── ordering: +1,-2 └── aggregations - └── array-agg [as=array_agg:5, outer=(1)] + └── array-agg [as=array_agg:6, outer=(1)] └── x:1 # Pass through ordering on grouping columns. @@ -394,68 +394,68 @@ opt SELECT a, min(b) FROM abc GROUP BY a ORDER BY a ---- group-by - ├── columns: a:1!null min:4!null + ├── columns: a:1!null min:5!null ├── grouping columns: a:1!null ├── key: (1) - ├── fd: (1)-->(4) + ├── fd: (1)-->(5) ├── ordering: +1 ├── scan abc │ ├── columns: a:1!null b:2!null │ └── ordering: +1 └── aggregations - └── min [as=min:4, outer=(2)] + └── min [as=min:5, outer=(2)] └── b:2 opt SELECT a, b, min(c) FROM abc GROUP BY a, b ORDER BY a ---- group-by - ├── columns: a:1!null b:2!null min:4!null + ├── columns: a:1!null b:2!null min:5!null ├── grouping columns: a:1!null b:2!null ├── internal-ordering: +1,+2 ├── key: (1,2) - ├── fd: (1,2)-->(4) + ├── fd: (1,2)-->(5) ├── ordering: +1 ├── scan abc │ ├── columns: a:1!null b:2!null c:3!null │ ├── key: (1-3) │ └── ordering: +1,+2 └── aggregations - └── min [as=min:4, outer=(3)] + └── min [as=min:5, outer=(3)] └── c:3 opt SELECT a, b, min(c) FROM abc GROUP BY a, b ORDER BY a, b ---- group-by - ├── columns: a:1!null b:2!null min:4!null + ├── columns: a:1!null b:2!null min:5!null ├── grouping columns: a:1!null b:2!null ├── key: (1,2) - ├── fd: (1,2)-->(4) + ├── fd: (1,2)-->(5) ├── ordering: +1,+2 ├── scan abc │ ├── columns: a:1!null b:2!null c:3!null │ ├── key: (1-3) │ └── ordering: +1,+2 └── aggregations - └── min [as=min:4, outer=(3)] + └── min [as=min:5, outer=(3)] └── c:3 opt SELECT a, b, min(c) FROM abc GROUP BY b, a ORDER BY a, b ---- group-by - ├── columns: a:1!null b:2!null min:4!null + ├── columns: a:1!null b:2!null min:5!null ├── grouping columns: a:1!null b:2!null ├── key: (1,2) - ├── fd: (1,2)-->(4) + ├── fd: (1,2)-->(5) ├── ordering: +1,+2 ├── scan abc │ ├── columns: a:1!null b:2!null c:3!null │ ├── key: (1-3) │ └── ordering: +1,+2 └── aggregations - └── min [as=min:4, outer=(3)] + └── min [as=min:5, outer=(3)] └── c:3 # We can't pass through the ordering if it refers to aggregation results. @@ -463,23 +463,23 @@ opt SELECT a, b, min(c) AS m FROM abc GROUP BY a, b ORDER BY a, m ---- sort (segmented) - ├── columns: a:1!null b:2!null m:4!null + ├── columns: a:1!null b:2!null m:5!null ├── key: (1,2) - ├── fd: (1,2)-->(4) - ├── ordering: +1,+4 + ├── fd: (1,2)-->(5) + ├── ordering: +1,+5 └── group-by - ├── columns: a:1!null b:2!null min:4!null + ├── columns: a:1!null b:2!null min:5!null ├── grouping columns: a:1!null b:2!null ├── internal-ordering: +1,+2 ├── key: (1,2) - ├── fd: (1,2)-->(4) + ├── fd: (1,2)-->(5) ├── ordering: +1 ├── scan abc │ ├── columns: a:1!null b:2!null c:3!null │ ├── key: (1-3) │ └── ordering: +1,+2 └── aggregations - └── min [as=min:4, outer=(3)] + └── min [as=min:5, outer=(3)] └── c:3 # Satisfy both the required and the internal orderings by requiring a+,b+,c+. @@ -487,54 +487,54 @@ opt SELECT a, b, array_agg(c) FROM (SELECT * FROM abc ORDER BY c) GROUP BY a, b ORDER BY a, b ---- group-by - ├── columns: a:1!null b:2!null array_agg:4!null + ├── columns: a:1!null b:2!null array_agg:5!null ├── grouping columns: a:1!null b:2!null ├── internal-ordering: +3 opt(1,2) ├── key: (1,2) - ├── fd: (1,2)-->(4) + ├── fd: (1,2)-->(5) ├── ordering: +1,+2 ├── scan abc │ ├── columns: a:1!null b:2!null c:3!null │ ├── key: (1-3) │ └── ordering: +1,+2,+3 └── aggregations - └── array-agg [as=array_agg:4, outer=(3)] + └── array-agg [as=array_agg:5, outer=(3)] └── c:3 opt SELECT a, b, array_agg(c) FROM (SELECT * FROM abc ORDER BY a, b, c) GROUP BY a, b ORDER BY a, b ---- group-by - ├── columns: a:1!null b:2!null array_agg:4!null + ├── columns: a:1!null b:2!null array_agg:5!null ├── grouping columns: a:1!null b:2!null ├── internal-ordering: +3 opt(1,2) ├── key: (1,2) - ├── fd: (1,2)-->(4) + ├── fd: (1,2)-->(5) ├── ordering: +1,+2 ├── scan abc │ ├── columns: a:1!null b:2!null c:3!null │ ├── key: (1-3) │ └── ordering: +1,+2,+3 └── aggregations - └── array-agg [as=array_agg:4, outer=(3)] + └── array-agg [as=array_agg:5, outer=(3)] └── c:3 opt SELECT a, b, array_agg(c) FROM (SELECT * FROM abc ORDER BY b, c, a) GROUP BY b, a ORDER BY a, b ---- group-by - ├── columns: a:1!null b:2!null array_agg:4!null + ├── columns: a:1!null b:2!null array_agg:5!null ├── grouping columns: a:1!null b:2!null ├── internal-ordering: +3 opt(1,2) ├── key: (1,2) - ├── fd: (1,2)-->(4) + ├── fd: (1,2)-->(5) ├── ordering: +1,+2 ├── scan abc │ ├── columns: a:1!null b:2!null c:3!null │ ├── key: (1-3) │ └── ordering: +1,+2,+3 └── aggregations - └── array-agg [as=array_agg:4, outer=(3)] + └── array-agg [as=array_agg:5, outer=(3)] └── c:3 # Verify that the GroupBy child ordering is simplified according to the child's @@ -543,10 +543,10 @@ opt SELECT sum(c) FROM abc WHERE a = 1 GROUP BY b ORDER BY b ---- group-by - ├── columns: sum:4!null [hidden: b:2!null] + ├── columns: sum:5!null [hidden: b:2!null] ├── grouping columns: b:2!null ├── key: (2) - ├── fd: (2)-->(4) + ├── fd: (2)-->(5) ├── ordering: +2 ├── scan abc │ ├── columns: a:1!null b:2!null c:3!null @@ -555,7 +555,7 @@ group-by │ ├── fd: ()-->(1) │ └── ordering: +2 opt(1) [actual: +2] └── aggregations - └── sum [as=sum:4, outer=(3)] + └── sum [as=sum:5, outer=(3)] └── c:3 # Verify we do a streaming group-by using the a, b ordering. @@ -563,18 +563,18 @@ opt SELECT sum(d) FROM abcd GROUP BY a, b, c ---- project - ├── columns: sum:6 + ├── columns: sum:7 └── group-by - ├── columns: a:1 b:2 c:3 sum:6 + ├── columns: a:1 b:2 c:3 sum:7 ├── grouping columns: a:1 b:2 c:3 ├── internal-ordering: +1,+2 ├── key: (1-3) - ├── fd: (1-3)-->(6) + ├── fd: (1-3)-->(7) ├── scan abcd@ab │ ├── columns: a:1 b:2 c:3 d:4 │ └── ordering: +1,+2 └── aggregations - └── sum [as=sum:6, outer=(4)] + └── sum [as=sum:7, outer=(4)] └── d:4 # Verify we do a streaming group-by using the c, d ordering. @@ -582,36 +582,36 @@ opt SELECT sum(a) FROM abcd GROUP BY b, c, d ---- project - ├── columns: sum:6 + ├── columns: sum:7 └── group-by - ├── columns: b:2 c:3 d:4 sum:6 + ├── columns: b:2 c:3 d:4 sum:7 ├── grouping columns: b:2 c:3 d:4 ├── internal-ordering: +3,+4 ├── key: (2-4) - ├── fd: (2-4)-->(6) + ├── fd: (2-4)-->(7) ├── scan abcd@cd │ ├── columns: a:1 b:2 c:3 d:4 │ └── ordering: +3,+4 └── aggregations - └── sum [as=sum:6, outer=(1)] + └── sum [as=sum:7, outer=(1)] └── a:1 opt SELECT array_agg(d) FROM (SELECT * FROM abcd ORDER BY c) GROUP BY a, b ---- project - ├── columns: array_agg:6 + ├── columns: array_agg:7 └── group-by - ├── columns: a:1 b:2 array_agg:6 + ├── columns: a:1 b:2 array_agg:7 ├── grouping columns: a:1 b:2 ├── internal-ordering: +3 opt(1,2) ├── key: (1,2) - ├── fd: (1,2)-->(6) + ├── fd: (1,2)-->(7) ├── scan abcd@cd │ ├── columns: a:1 b:2 c:3 d:4 │ └── ordering: +3 opt(1,2) [actual: +3] └── aggregations - └── array-agg [as=array_agg:6, outer=(4)] + └── array-agg [as=array_agg:7, outer=(4)] └── d:4 # -------------------------------------------------- @@ -621,7 +621,7 @@ opt EXPLAIN (VERBOSE) SELECT * FROM a ORDER BY y ---- explain - ├── columns: tree:5 field:8 description:9 columns:10 ordering:11 [hidden: level:6 node_type:7] + ├── columns: tree:6 field:9 description:10 columns:11 ordering:12 [hidden: level:7 node_type:8] ├── mode: verbose └── sort ├── columns: x:1!null y:2!null z:3 s:4!null @@ -636,17 +636,17 @@ explain memo EXPLAIN (VERBOSE) SELECT * FROM a ORDER BY y ---- -memo (optimized, ~2KB, required=[presentation: tree:5,field:8,description:9,columns:10,ordering:11]) +memo (optimized, ~3KB, required=[presentation: tree:6,field:9,description:10,columns:11,ordering:12]) ├── G1: (explain G2 [presentation: x:1,y:2,z:3,s:4] [ordering: +2]) - │ └── [presentation: tree:5,field:8,description:9,columns:10,ordering:11] + │ └── [presentation: tree:6,field:9,description:10,columns:11,ordering:12] │ ├── best: (explain G2="[presentation: x:1,y:2,z:3,s:4] [ordering: +2]" [presentation: x:1,y:2,z:3,s:4] [ordering: +2]) │ └── cost: 1299.36 - └── G2: (scan a) + └── G2: (scan a,cols=(1-4)) ├── [presentation: x:1,y:2,z:3,s:4] [ordering: +2] │ ├── best: (sort G2) │ └── cost: 1299.35 └── [] - ├── best: (scan a) + ├── best: (scan a,cols=(1-4)) └── cost: 1080.02 # -------------------------------------------------- @@ -656,9 +656,9 @@ memo (optimized, ~2KB, required=[presentation: tree:5,field:8,description:9,colu memo SELECT y FROM a WITH ORDINALITY ORDER BY ordinality ---- -memo (optimized, ~4KB, required=[presentation: y:2] [ordering: +5]) +memo (optimized, ~4KB, required=[presentation: y:2] [ordering: +6]) ├── G1: (ordinality G2) - │ ├── [presentation: y:2] [ordering: +5] + │ ├── [presentation: y:2] [ordering: +6] │ │ ├── best: (ordinality G2) │ │ └── cost: 1060.03 │ └── [] @@ -672,9 +672,9 @@ memo (optimized, ~4KB, required=[presentation: y:2] [ordering: +5]) memo SELECT y FROM a WITH ORDINALITY ORDER BY -ordinality ---- -memo (optimized, ~5KB, required=[presentation: y:2] [ordering: +6]) +memo (optimized, ~5KB, required=[presentation: y:2] [ordering: +7]) ├── G1: (project G2 G3 y) - │ ├── [presentation: y:2] [ordering: +6] + │ ├── [presentation: y:2] [ordering: +7] │ │ ├── best: (sort G1) │ │ └── cost: 1299.37 │ └── [] @@ -695,9 +695,9 @@ memo (optimized, ~5KB, required=[presentation: y:2] [ordering: +6]) memo SELECT y FROM a WITH ORDINALITY ORDER BY ordinality, x ---- -memo (optimized, ~6KB, required=[presentation: y:2] [ordering: +5]) +memo (optimized, ~6KB, required=[presentation: y:2] [ordering: +6]) ├── G1: (ordinality G2) - │ ├── [presentation: y:2] [ordering: +5] + │ ├── [presentation: y:2] [ordering: +6] │ │ ├── best: (ordinality G2) │ │ └── cost: 1060.03 │ └── [] @@ -711,9 +711,9 @@ memo (optimized, ~6KB, required=[presentation: y:2] [ordering: +5]) memo SELECT y FROM (SELECT * FROM a ORDER BY y) WITH ORDINALITY ORDER BY y, ordinality ---- -memo (optimized, ~4KB, required=[presentation: y:2] [ordering: +2,+5]) +memo (optimized, ~5KB, required=[presentation: y:2] [ordering: +2,+6]) ├── G1: (ordinality G2 ordering=+2) - │ ├── [presentation: y:2] [ordering: +2,+5] + │ ├── [presentation: y:2] [ordering: +2,+6] │ │ ├── best: (ordinality G2="[ordering: +2]" ordering=+2) │ │ └── cost: 1279.36 │ └── [] @@ -730,9 +730,9 @@ memo (optimized, ~4KB, required=[presentation: y:2] [ordering: +2,+5]) memo SELECT y FROM (SELECT * FROM a ORDER BY y) WITH ORDINALITY ORDER BY ordinality, y ---- -memo (optimized, ~4KB, required=[presentation: y:2] [ordering: +5]) +memo (optimized, ~5KB, required=[presentation: y:2] [ordering: +6]) ├── G1: (ordinality G2 ordering=+2) - │ ├── [presentation: y:2] [ordering: +5] + │ ├── [presentation: y:2] [ordering: +6] │ │ ├── best: (ordinality G2="[ordering: +2]" ordering=+2) │ │ └── cost: 1279.36 │ └── [] @@ -749,9 +749,9 @@ memo (optimized, ~4KB, required=[presentation: y:2] [ordering: +5]) memo SELECT y FROM a WITH ORDINALITY ORDER BY ordinality DESC ---- -memo (optimized, ~4KB, required=[presentation: y:2] [ordering: -5]) +memo (optimized, ~4KB, required=[presentation: y:2] [ordering: -6]) ├── G1: (ordinality G2) - │ ├── [presentation: y:2] [ordering: -5] + │ ├── [presentation: y:2] [ordering: -6] │ │ ├── best: (sort G1) │ │ └── cost: 1279.36 │ └── [] @@ -770,40 +770,40 @@ opt SELECT * FROM abc JOIN xyz ON a=x ORDER BY a ---- inner-join (merge) - ├── columns: a:1!null b:2!null c:3!null x:4!null y:5!null z:6!null + ├── columns: a:1!null b:2!null c:3!null x:5!null y:6!null z:7!null ├── left ordering: +1 - ├── right ordering: +4 - ├── key: (2-6) - ├── fd: (1)==(4), (4)==(1) - ├── ordering: +(1|4) [actual: +1] + ├── right ordering: +5 + ├── key: (2,3,5-7) + ├── fd: (1)==(5), (5)==(1) + ├── ordering: +(1|5) [actual: +1] ├── scan abc │ ├── columns: a:1!null b:2!null c:3!null │ ├── key: (1-3) │ └── ordering: +1 ├── scan xyz - │ ├── columns: x:4!null y:5!null z:6!null - │ ├── key: (4-6) - │ └── ordering: +4 + │ ├── columns: x:5!null y:6!null z:7!null + │ ├── key: (5-7) + │ └── ordering: +5 └── filters (true) opt SELECT * FROM abc JOIN xyz ON a=x ORDER BY x ---- inner-join (merge) - ├── columns: a:1!null b:2!null c:3!null x:4!null y:5!null z:6!null + ├── columns: a:1!null b:2!null c:3!null x:5!null y:6!null z:7!null ├── left ordering: +1 - ├── right ordering: +4 - ├── key: (2-6) - ├── fd: (1)==(4), (4)==(1) - ├── ordering: +(1|4) [actual: +1] + ├── right ordering: +5 + ├── key: (2,3,5-7) + ├── fd: (1)==(5), (5)==(1) + ├── ordering: +(1|5) [actual: +1] ├── scan abc │ ├── columns: a:1!null b:2!null c:3!null │ ├── key: (1-3) │ └── ordering: +1 ├── scan xyz - │ ├── columns: x:4!null y:5!null z:6!null - │ ├── key: (4-6) - │ └── ordering: +4 + │ ├── columns: x:5!null y:6!null z:7!null + │ ├── key: (5-7) + │ └── ordering: +5 └── filters (true) # A left join guarantees an ordering on the left side. @@ -811,19 +811,19 @@ opt SELECT * FROM abc LEFT JOIN xyz ON a=x ORDER BY a ---- left-join (merge) - ├── columns: a:1!null b:2!null c:3!null x:4 y:5 z:6 + ├── columns: a:1!null b:2!null c:3!null x:5 y:6 z:7 ├── left ordering: +1 - ├── right ordering: +4 - ├── key: (1-6) + ├── right ordering: +5 + ├── key: (1-3,5-7) ├── ordering: +1 ├── scan abc │ ├── columns: a:1!null b:2!null c:3!null │ ├── key: (1-3) │ └── ordering: +1 ├── scan xyz - │ ├── columns: x:4!null y:5!null z:6!null - │ ├── key: (4-6) - │ └── ordering: +4 + │ ├── columns: x:5!null y:6!null z:7!null + │ ├── key: (5-7) + │ └── ordering: +5 └── filters (true) # A left join doesn't guarantee an ordering on x (some rows will have NULLs). @@ -831,22 +831,22 @@ opt SELECT * FROM abc LEFT JOIN xyz ON a=x ORDER BY x ---- sort - ├── columns: a:1!null b:2!null c:3!null x:4 y:5 z:6 - ├── key: (1-6) - ├── ordering: +4 + ├── columns: a:1!null b:2!null c:3!null x:5 y:6 z:7 + ├── key: (1-3,5-7) + ├── ordering: +5 └── left-join (merge) - ├── columns: a:1!null b:2!null c:3!null x:4 y:5 z:6 + ├── columns: a:1!null b:2!null c:3!null x:5 y:6 z:7 ├── left ordering: +1 - ├── right ordering: +4 - ├── key: (1-6) + ├── right ordering: +5 + ├── key: (1-3,5-7) ├── scan abc │ ├── columns: a:1!null b:2!null c:3!null │ ├── key: (1-3) │ └── ordering: +1 ├── scan xyz - │ ├── columns: x:4!null y:5!null z:6!null - │ ├── key: (4-6) - │ └── ordering: +4 + │ ├── columns: x:5!null y:6!null z:7!null + │ ├── key: (5-7) + │ └── ordering: +5 └── filters (true) # A right join doesn't guarantee an ordering on a (some rows will have NULLs). @@ -854,18 +854,18 @@ opt SELECT * FROM abc RIGHT JOIN xyz ON a=x ORDER BY a ---- sort - ├── columns: a:1 b:2 c:3 x:4!null y:5!null z:6!null - ├── key: (1-6) + ├── columns: a:1 b:2 c:3 x:5!null y:6!null z:7!null + ├── key: (1-3,5-7) ├── ordering: +1 └── left-join (merge) - ├── columns: a:1 b:2 c:3 x:4!null y:5!null z:6!null - ├── left ordering: +4 + ├── columns: a:1 b:2 c:3 x:5!null y:6!null z:7!null + ├── left ordering: +5 ├── right ordering: +1 - ├── key: (1-6) + ├── key: (1-3,5-7) ├── scan xyz - │ ├── columns: x:4!null y:5!null z:6!null - │ ├── key: (4-6) - │ └── ordering: +4 + │ ├── columns: x:5!null y:6!null z:7!null + │ ├── key: (5-7) + │ └── ordering: +5 ├── scan abc │ ├── columns: a:1!null b:2!null c:3!null │ ├── key: (1-3) @@ -876,15 +876,15 @@ opt SELECT * FROM abc RIGHT JOIN xyz ON a=x ORDER BY x ---- left-join (merge) - ├── columns: a:1 b:2 c:3 x:4!null y:5!null z:6!null - ├── left ordering: +4 + ├── columns: a:1 b:2 c:3 x:5!null y:6!null z:7!null + ├── left ordering: +5 ├── right ordering: +1 - ├── key: (1-6) - ├── ordering: +4 + ├── key: (1-3,5-7) + ├── ordering: +5 ├── scan xyz - │ ├── columns: x:4!null y:5!null z:6!null - │ ├── key: (4-6) - │ └── ordering: +4 + │ ├── columns: x:5!null y:6!null z:7!null + │ ├── key: (5-7) + │ └── ordering: +5 ├── scan abc │ ├── columns: a:1!null b:2!null c:3!null │ ├── key: (1-3) @@ -895,82 +895,82 @@ opt SELECT * FROM abc FULL OUTER JOIN xyz ON a=x ORDER BY a ---- sort - ├── columns: a:1 b:2 c:3 x:4 y:5 z:6 - ├── key: (1-6) + ├── columns: a:1 b:2 c:3 x:5 y:6 z:7 + ├── key: (1-3,5-7) ├── ordering: +1 └── full-join (merge) - ├── columns: a:1 b:2 c:3 x:4 y:5 z:6 + ├── columns: a:1 b:2 c:3 x:5 y:6 z:7 ├── left ordering: +1 - ├── right ordering: +4 - ├── key: (1-6) + ├── right ordering: +5 + ├── key: (1-3,5-7) ├── scan abc │ ├── columns: a:1!null b:2!null c:3!null │ ├── key: (1-3) │ └── ordering: +1 ├── scan xyz - │ ├── columns: x:4!null y:5!null z:6!null - │ ├── key: (4-6) - │ └── ordering: +4 + │ ├── columns: x:5!null y:6!null z:7!null + │ ├── key: (5-7) + │ └── ordering: +5 └── filters (true) opt SELECT * FROM abc JOIN xyz ON a=x AND b=y ORDER BY a ---- inner-join (merge) - ├── columns: a:1!null b:2!null c:3!null x:4!null y:5!null z:6!null + ├── columns: a:1!null b:2!null c:3!null x:5!null y:6!null z:7!null ├── left ordering: +1,+2 - ├── right ordering: +4,+5 - ├── key: (3-6) - ├── fd: (1)==(4), (4)==(1), (2)==(5), (5)==(2) - ├── ordering: +(1|4) [actual: +1] + ├── right ordering: +5,+6 + ├── key: (3,5-7) + ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2) + ├── ordering: +(1|5) [actual: +1] ├── scan abc │ ├── columns: a:1!null b:2!null c:3!null │ ├── key: (1-3) │ └── ordering: +1,+2 ├── scan xyz - │ ├── columns: x:4!null y:5!null z:6!null - │ ├── key: (4-6) - │ └── ordering: +4,+5 + │ ├── columns: x:5!null y:6!null z:7!null + │ ├── key: (5-7) + │ └── ordering: +5,+6 └── filters (true) opt SELECT * FROM abc JOIN xyz ON a=x AND b=y ORDER BY a, b ---- inner-join (merge) - ├── columns: a:1!null b:2!null c:3!null x:4!null y:5!null z:6!null + ├── columns: a:1!null b:2!null c:3!null x:5!null y:6!null z:7!null ├── left ordering: +1,+2 - ├── right ordering: +4,+5 - ├── key: (3-6) - ├── fd: (1)==(4), (4)==(1), (2)==(5), (5)==(2) - ├── ordering: +(1|4),+(2|5) [actual: +1,+2] + ├── right ordering: +5,+6 + ├── key: (3,5-7) + ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2) + ├── ordering: +(1|5),+(2|6) [actual: +1,+2] ├── scan abc │ ├── columns: a:1!null b:2!null c:3!null │ ├── key: (1-3) │ └── ordering: +1,+2 ├── scan xyz - │ ├── columns: x:4!null y:5!null z:6!null - │ ├── key: (4-6) - │ └── ordering: +4,+5 + │ ├── columns: x:5!null y:6!null z:7!null + │ ├── key: (5-7) + │ └── ordering: +5,+6 └── filters (true) opt SELECT * FROM abc JOIN xyz ON a=x AND b=y ORDER BY a, y ---- inner-join (merge) - ├── columns: a:1!null b:2!null c:3!null x:4!null y:5!null z:6!null + ├── columns: a:1!null b:2!null c:3!null x:5!null y:6!null z:7!null ├── left ordering: +1,+2 - ├── right ordering: +4,+5 - ├── key: (3-6) - ├── fd: (1)==(4), (4)==(1), (2)==(5), (5)==(2) - ├── ordering: +(1|4),+(2|5) [actual: +1,+2] + ├── right ordering: +5,+6 + ├── key: (3,5-7) + ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2) + ├── ordering: +(1|5),+(2|6) [actual: +1,+2] ├── scan abc │ ├── columns: a:1!null b:2!null c:3!null │ ├── key: (1-3) │ └── ordering: +1,+2 ├── scan xyz - │ ├── columns: x:4!null y:5!null z:6!null - │ ├── key: (4-6) - │ └── ordering: +4,+5 + │ ├── columns: x:5!null y:6!null z:7!null + │ ├── key: (5-7) + │ └── ordering: +5,+6 └── filters (true) # -------------------------------------------------- @@ -1452,47 +1452,47 @@ opt SELECT * FROM [INSERT INTO abc SELECT * FROM xyz ORDER BY y, z LIMIT 2 RETURNING *] ORDER BY b ---- sort - ├── columns: a:7!null b:8!null c:9!null + ├── columns: a:9!null b:10!null c:11!null ├── cardinality: [0 - 2] ├── volatile, mutations - ├── key: (7-9) - ├── ordering: +8 + ├── key: (9-11) + ├── ordering: +10 └── with &1 - ├── columns: a:7!null b:8!null c:9!null + ├── columns: a:9!null b:10!null c:11!null ├── cardinality: [0 - 2] ├── volatile, mutations - ├── key: (7-9) + ├── key: (9-11) ├── insert abc │ ├── columns: abc.a:1!null abc.b:2!null abc.c:3!null │ ├── insert-mapping: - │ │ ├── x:4 => abc.a:1 - │ │ ├── y:5 => abc.b:2 - │ │ └── z:6 => abc.c:3 + │ │ ├── x:5 => abc.a:1 + │ │ ├── y:6 => abc.b:2 + │ │ └── z:7 => abc.c:3 │ ├── cardinality: [0 - 2] │ ├── volatile, mutations │ ├── key: (1-3) │ └── limit - │ ├── columns: x:4!null y:5!null z:6!null - │ ├── internal-ordering: +5,+6 + │ ├── columns: x:5!null y:6!null z:7!null + │ ├── internal-ordering: +6,+7 │ ├── cardinality: [0 - 2] - │ ├── key: (4-6) + │ ├── key: (5-7) │ ├── sort - │ │ ├── columns: x:4!null y:5!null z:6!null - │ │ ├── key: (4-6) - │ │ ├── ordering: +5,+6 + │ │ ├── columns: x:5!null y:6!null z:7!null + │ │ ├── key: (5-7) + │ │ ├── ordering: +6,+7 │ │ ├── limit hint: 2.00 │ │ └── scan xyz - │ │ ├── columns: x:4!null y:5!null z:6!null - │ │ └── key: (4-6) + │ │ ├── columns: x:5!null y:6!null z:7!null + │ │ └── key: (5-7) │ └── 2 └── with-scan &1 - ├── columns: a:7!null b:8!null c:9!null + ├── columns: a:9!null b:10!null c:11!null ├── mapping: - │ ├── abc.a:1 => a:7 - │ ├── abc.b:2 => b:8 - │ └── abc.c:3 => c:9 + │ ├── abc.a:1 => a:9 + │ ├── abc.b:2 => b:10 + │ └── abc.c:3 => c:11 ├── cardinality: [0 - 2] - └── key: (7-9) + └── key: (9-11) # Verify that provided orderings are derived correctly. opt @@ -1501,31 +1501,31 @@ FROM [INSERT INTO xyz SELECT b, c, d FROM abcd ORDER BY c, d LIMIT 2 RETURNING * ORDER BY y ---- sort - ├── columns: x:9!null y:10!null z:11!null + ├── columns: x:11!null y:12!null z:13!null ├── cardinality: [0 - 2] ├── volatile, mutations - ├── ordering: +10 + ├── ordering: +12 └── with &1 - ├── columns: x:9!null y:10!null z:11!null + ├── columns: x:11!null y:12!null z:13!null ├── cardinality: [0 - 2] ├── volatile, mutations ├── insert xyz │ ├── columns: xyz.x:1!null xyz.y:2!null xyz.z:3!null │ ├── insert-mapping: - │ │ ├── b:5 => xyz.x:1 - │ │ ├── c:6 => xyz.y:2 - │ │ └── d:7 => xyz.z:3 + │ │ ├── b:6 => xyz.x:1 + │ │ ├── c:7 => xyz.y:2 + │ │ └── d:8 => xyz.z:3 │ ├── cardinality: [0 - 2] │ ├── volatile, mutations │ └── scan abcd@cd - │ ├── columns: b:5 c:6 d:7 + │ ├── columns: b:6 c:7 d:8 │ └── limit: 2 └── with-scan &1 - ├── columns: x:9!null y:10!null z:11!null + ├── columns: x:11!null y:12!null z:13!null ├── mapping: - │ ├── xyz.x:1 => x:9 - │ ├── xyz.y:2 => y:10 - │ └── xyz.z:3 => z:11 + │ ├── xyz.x:1 => x:11 + │ ├── xyz.y:2 => y:12 + │ └── xyz.z:3 => z:13 └── cardinality: [0 - 2] # Verify that provided orderings are derived correctly with equivalence FD. @@ -1537,67 +1537,67 @@ WHERE x=y ORDER BY y ---- sort - ├── columns: x:9!null y:10!null z:11!null + ├── columns: x:11!null y:12!null z:13!null ├── cardinality: [0 - 2] ├── volatile, mutations - ├── fd: (9)==(10), (10)==(9) - ├── ordering: +(9|10) [actual: +9] + ├── fd: (11)==(12), (12)==(11) + ├── ordering: +(11|12) [actual: +11] └── with &1 - ├── columns: x:9!null y:10!null z:11!null + ├── columns: x:11!null y:12!null z:13!null ├── cardinality: [0 - 2] ├── volatile, mutations - ├── fd: (9)==(10), (10)==(9) + ├── fd: (11)==(12), (12)==(11) ├── insert xyz │ ├── columns: xyz.x:1!null xyz.y:2!null xyz.z:3!null │ ├── insert-mapping: - │ │ ├── b:5 => xyz.x:1 - │ │ ├── c:6 => xyz.y:2 - │ │ └── d:7 => xyz.z:3 + │ │ ├── b:6 => xyz.x:1 + │ │ ├── c:7 => xyz.y:2 + │ │ └── d:8 => xyz.z:3 │ ├── cardinality: [0 - 2] │ ├── volatile, mutations │ └── scan abcd@cd - │ ├── columns: b:5 c:6 d:7 + │ ├── columns: b:6 c:7 d:8 │ └── limit: 2 └── select - ├── columns: x:9!null y:10!null z:11!null + ├── columns: x:11!null y:12!null z:13!null ├── cardinality: [0 - 2] - ├── fd: (9)==(10), (10)==(9) + ├── fd: (11)==(12), (12)==(11) ├── with-scan &1 - │ ├── columns: x:9!null y:10!null z:11!null + │ ├── columns: x:11!null y:12!null z:13!null │ ├── mapping: - │ │ ├── xyz.x:1 => x:9 - │ │ ├── xyz.y:2 => y:10 - │ │ └── xyz.z:3 => z:11 + │ │ ├── xyz.x:1 => x:11 + │ │ ├── xyz.y:2 => y:12 + │ │ └── xyz.z:3 => z:13 │ └── cardinality: [0 - 2] └── filters - └── x:9 = y:10 [outer=(9,10), fd=(9)==(10), (10)==(9)] + └── x:11 = y:12 [outer=(11,12), fd=(11)==(12), (12)==(11)] # Ignore internal ordering. opt SELECT * FROM [INSERT INTO abc SELECT * FROM xyz ORDER BY y, z RETURNING *] ---- with &1 - ├── columns: a:7!null b:8!null c:9!null + ├── columns: a:9!null b:10!null c:11!null ├── volatile, mutations - ├── key: (7-9) + ├── key: (9-11) ├── insert abc │ ├── columns: abc.a:1!null abc.b:2!null abc.c:3!null │ ├── insert-mapping: - │ │ ├── x:4 => abc.a:1 - │ │ ├── y:5 => abc.b:2 - │ │ └── z:6 => abc.c:3 + │ │ ├── x:5 => abc.a:1 + │ │ ├── y:6 => abc.b:2 + │ │ └── z:7 => abc.c:3 │ ├── volatile, mutations │ ├── key: (1-3) │ └── scan xyz - │ ├── columns: x:4!null y:5!null z:6!null - │ └── key: (4-6) + │ ├── columns: x:5!null y:6!null z:7!null + │ └── key: (5-7) └── with-scan &1 - ├── columns: a:7!null b:8!null c:9!null + ├── columns: a:9!null b:10!null c:11!null ├── mapping: - │ ├── abc.a:1 => a:7 - │ ├── abc.b:2 => b:8 - │ └── abc.c:3 => c:9 - └── key: (7-9) + │ ├── abc.a:1 => a:9 + │ ├── abc.b:2 => b:10 + │ └── abc.c:3 => c:11 + └── key: (9-11) # -------------------------------------------------- # Update operator. @@ -1608,46 +1608,46 @@ opt SELECT * FROM [UPDATE abcd SET (a, b)=(1, 2) RETURNING *] ORDER BY c ---- sort - ├── columns: a:13!null b:14!null c:15 d:16 + ├── columns: a:15!null b:16!null c:17 d:18 ├── volatile, mutations - ├── fd: ()-->(13,14) - ├── ordering: +15 opt(13,14) [actual: +15] + ├── fd: ()-->(15,16) + ├── ordering: +17 opt(15,16) [actual: +17] └── with &1 - ├── columns: a:13!null b:14!null c:15 d:16 + ├── columns: a:15!null b:16!null c:17 d:18 ├── volatile, mutations - ├── fd: ()-->(13,14) + ├── fd: ()-->(15,16) ├── project │ ├── columns: abcd.a:1!null abcd.b:2!null abcd.c:3 abcd.d:4 │ ├── volatile, mutations │ ├── fd: ()-->(1,2) │ └── update abcd │ ├── columns: abcd.a:1!null abcd.b:2!null abcd.c:3 abcd.d:4 rowid:5!null - │ ├── fetch columns: abcd.a:6 abcd.b:7 abcd.c:8 abcd.d:9 rowid:10 + │ ├── fetch columns: abcd.a:7 abcd.b:8 abcd.c:9 abcd.d:10 rowid:11 │ ├── update-mapping: - │ │ ├── a_new:11 => abcd.a:1 - │ │ └── b_new:12 => abcd.b:2 + │ │ ├── a_new:13 => abcd.a:1 + │ │ └── b_new:14 => abcd.b:2 │ ├── volatile, mutations │ ├── key: (5) │ ├── fd: ()-->(1,2), (5)-->(3,4) │ └── project - │ ├── columns: a_new:11!null b_new:12!null abcd.a:6 abcd.b:7 abcd.c:8 abcd.d:9 rowid:10!null - │ ├── key: (10) - │ ├── fd: ()-->(11,12), (10)-->(6-9) + │ ├── columns: a_new:13!null b_new:14!null abcd.a:7 abcd.b:8 abcd.c:9 abcd.d:10 rowid:11!null + │ ├── key: (11) + │ ├── fd: ()-->(13,14), (11)-->(7-10) │ ├── scan abcd - │ │ ├── columns: abcd.a:6 abcd.b:7 abcd.c:8 abcd.d:9 rowid:10!null - │ │ ├── key: (10) - │ │ └── fd: (10)-->(6-9) + │ │ ├── columns: abcd.a:7 abcd.b:8 abcd.c:9 abcd.d:10 rowid:11!null + │ │ ├── key: (11) + │ │ └── fd: (11)-->(7-10) │ └── projections - │ ├── 1 [as=a_new:11] - │ └── 2 [as=b_new:12] + │ ├── 1 [as=a_new:13] + │ └── 2 [as=b_new:14] └── with-scan &1 - ├── columns: a:13!null b:14!null c:15 d:16 + ├── columns: a:15!null b:16!null c:17 d:18 ├── mapping: - │ ├── abcd.a:1 => a:13 - │ ├── abcd.b:2 => b:14 - │ ├── abcd.c:3 => c:15 - │ └── abcd.d:4 => d:16 - └── fd: ()-->(13,14) + │ ├── abcd.a:1 => a:15 + │ ├── abcd.b:2 => b:16 + │ ├── abcd.c:3 => c:17 + │ └── abcd.d:4 => d:18 + └── fd: ()-->(15,16) # Verify that provided orderings are derived correctly. opt @@ -1656,12 +1656,12 @@ FROM [UPDATE abcd SET b=b+1 ORDER BY c LIMIT 10 RETURNING *] ORDER BY c, d ---- sort - ├── columns: a:12 b:13 c:14 d:15 + ├── columns: a:14 b:15 c:16 d:17 ├── cardinality: [0 - 10] ├── volatile, mutations - ├── ordering: +14,+15 + ├── ordering: +16,+17 └── with &1 - ├── columns: a:12 b:13 c:14 d:15 + ├── columns: a:14 b:15 c:16 d:17 ├── cardinality: [0 - 10] ├── volatile, mutations ├── project @@ -1670,33 +1670,33 @@ sort │ ├── volatile, mutations │ └── update abcd │ ├── columns: abcd.a:1 abcd.b:2 abcd.c:3 abcd.d:4 rowid:5!null - │ ├── fetch columns: abcd.a:6 abcd.b:7 abcd.c:8 abcd.d:9 rowid:10 + │ ├── fetch columns: abcd.a:7 abcd.b:8 abcd.c:9 abcd.d:10 rowid:11 │ ├── update-mapping: - │ │ └── b_new:11 => abcd.b:2 + │ │ └── b_new:13 => abcd.b:2 │ ├── cardinality: [0 - 10] │ ├── volatile, mutations │ ├── key: (5) │ ├── fd: (5)-->(1-4) │ └── project - │ ├── columns: b_new:11 abcd.a:6 abcd.b:7 abcd.c:8 abcd.d:9 rowid:10!null + │ ├── columns: b_new:13 abcd.a:7 abcd.b:8 abcd.c:9 abcd.d:10 rowid:11!null │ ├── cardinality: [0 - 10] │ ├── immutable - │ ├── key: (10) - │ ├── fd: (10)-->(6-9), (7)-->(11) + │ ├── key: (11) + │ ├── fd: (11)-->(7-10), (8)-->(13) │ ├── scan abcd@cd - │ │ ├── columns: abcd.a:6 abcd.b:7 abcd.c:8 abcd.d:9 rowid:10!null + │ │ ├── columns: abcd.a:7 abcd.b:8 abcd.c:9 abcd.d:10 rowid:11!null │ │ ├── limit: 10 - │ │ ├── key: (10) - │ │ └── fd: (10)-->(6-9) + │ │ ├── key: (11) + │ │ └── fd: (11)-->(7-10) │ └── projections - │ └── abcd.b:7 + 1 [as=b_new:11, outer=(7), immutable] + │ └── abcd.b:8 + 1 [as=b_new:13, outer=(8), immutable] └── with-scan &1 - ├── columns: a:12 b:13 c:14 d:15 + ├── columns: a:14 b:15 c:16 d:17 ├── mapping: - │ ├── abcd.a:1 => a:12 - │ ├── abcd.b:2 => b:13 - │ ├── abcd.c:3 => c:14 - │ └── abcd.d:4 => d:15 + │ ├── abcd.a:1 => a:14 + │ ├── abcd.b:2 => b:15 + │ ├── abcd.c:3 => c:16 + │ └── abcd.d:4 => d:17 └── cardinality: [0 - 10] # Verify that provided orderings are derived correctly with equivalence FD. @@ -1708,56 +1708,56 @@ WHERE b=c ORDER BY b, d ---- sort - ├── columns: a:12 b:13!null c:14!null d:15 + ├── columns: a:14 b:15!null c:16!null d:17 ├── cardinality: [0 - 10] ├── volatile, mutations - ├── fd: (13)==(14), (14)==(13) - ├── ordering: +(13|14),+15 [actual: +13,+15] + ├── fd: (15)==(16), (16)==(15) + ├── ordering: +(15|16),+17 [actual: +15,+17] └── with &1 - ├── columns: a:12 b:13!null c:14!null d:15 + ├── columns: a:14 b:15!null c:16!null d:17 ├── cardinality: [0 - 10] ├── volatile, mutations - ├── fd: (13)==(14), (14)==(13) + ├── fd: (15)==(16), (16)==(15) ├── project │ ├── columns: abcd.a:1 abcd.b:2 abcd.c:3 abcd.d:4 │ ├── cardinality: [0 - 10] │ ├── volatile, mutations │ └── update abcd │ ├── columns: abcd.a:1 abcd.b:2 abcd.c:3 abcd.d:4 rowid:5!null - │ ├── fetch columns: abcd.a:6 abcd.b:7 abcd.c:8 abcd.d:9 rowid:10 + │ ├── fetch columns: abcd.a:7 abcd.b:8 abcd.c:9 abcd.d:10 rowid:11 │ ├── update-mapping: - │ │ └── b_new:11 => abcd.b:2 + │ │ └── b_new:13 => abcd.b:2 │ ├── cardinality: [0 - 10] │ ├── volatile, mutations │ ├── key: (5) │ ├── fd: (5)-->(1-4) │ └── project - │ ├── columns: b_new:11 abcd.a:6 abcd.b:7 abcd.c:8 abcd.d:9 rowid:10!null + │ ├── columns: b_new:13 abcd.a:7 abcd.b:8 abcd.c:9 abcd.d:10 rowid:11!null │ ├── cardinality: [0 - 10] │ ├── immutable - │ ├── key: (10) - │ ├── fd: (10)-->(6-9), (7)-->(11) + │ ├── key: (11) + │ ├── fd: (11)-->(7-10), (8)-->(13) │ ├── scan abcd@cd - │ │ ├── columns: abcd.a:6 abcd.b:7 abcd.c:8 abcd.d:9 rowid:10!null + │ │ ├── columns: abcd.a:7 abcd.b:8 abcd.c:9 abcd.d:10 rowid:11!null │ │ ├── limit: 10 - │ │ ├── key: (10) - │ │ └── fd: (10)-->(6-9) + │ │ ├── key: (11) + │ │ └── fd: (11)-->(7-10) │ └── projections - │ └── abcd.b:7 + 1 [as=b_new:11, outer=(7), immutable] + │ └── abcd.b:8 + 1 [as=b_new:13, outer=(8), immutable] └── select - ├── columns: a:12 b:13!null c:14!null d:15 + ├── columns: a:14 b:15!null c:16!null d:17 ├── cardinality: [0 - 10] - ├── fd: (13)==(14), (14)==(13) + ├── fd: (15)==(16), (16)==(15) ├── with-scan &1 - │ ├── columns: a:12 b:13 c:14 d:15 + │ ├── columns: a:14 b:15 c:16 d:17 │ ├── mapping: - │ │ ├── abcd.a:1 => a:12 - │ │ ├── abcd.b:2 => b:13 - │ │ ├── abcd.c:3 => c:14 - │ │ └── abcd.d:4 => d:15 + │ │ ├── abcd.a:1 => a:14 + │ │ ├── abcd.b:2 => b:15 + │ │ ├── abcd.c:3 => c:16 + │ │ └── abcd.d:4 => d:17 │ └── cardinality: [0 - 10] └── filters - └── b:13 = c:14 [outer=(13,14), fd=(13)==(14), (14)==(13)] + └── b:15 = c:16 [outer=(15,16), fd=(15)==(16), (16)==(15)] # -------------------------------------------------- # Upsert operator. @@ -1777,75 +1777,75 @@ FROM ORDER BY b ---- sort - ├── columns: a:14!null b:15!null c:16!null + ├── columns: a:17!null b:18!null c:19!null ├── cardinality: [0 - 2] ├── volatile, mutations - ├── ordering: +15 + ├── ordering: +18 └── with &1 - ├── columns: a:14!null b:15!null c:16!null + ├── columns: a:17!null b:18!null c:19!null ├── cardinality: [0 - 2] ├── volatile, mutations ├── upsert abc │ ├── columns: abc.a:1!null abc.b:2!null abc.c:3!null - │ ├── canary column: 7 - │ ├── fetch columns: abc.a:7 abc.b:8 abc.c:9 + │ ├── canary column: 9 + │ ├── fetch columns: abc.a:9 abc.b:10 abc.c:11 │ ├── insert-mapping: - │ │ ├── x:4 => abc.a:1 - │ │ ├── y:5 => abc.b:2 - │ │ └── z:6 => abc.c:3 + │ │ ├── x:5 => abc.a:1 + │ │ ├── y:6 => abc.b:2 + │ │ └── z:7 => abc.c:3 │ ├── update-mapping: - │ │ └── upsert_a:11 => abc.a:1 + │ │ └── upsert_a:14 => abc.a:1 │ ├── return-mapping: - │ │ ├── upsert_a:11 => abc.a:1 - │ │ ├── upsert_b:12 => abc.b:2 - │ │ └── upsert_c:13 => abc.c:3 + │ │ ├── upsert_a:14 => abc.a:1 + │ │ ├── upsert_b:15 => abc.b:2 + │ │ └── upsert_c:16 => abc.c:3 │ ├── cardinality: [0 - 2] │ ├── volatile, mutations │ └── project - │ ├── columns: upsert_a:11!null upsert_b:12 upsert_c:13 x:4!null y:5!null z:6!null abc.a:7 abc.b:8 abc.c:9 + │ ├── columns: upsert_a:14!null upsert_b:15 upsert_c:16 x:5!null y:6!null z:7!null abc.a:9 abc.b:10 abc.c:11 │ ├── cardinality: [0 - 2] - │ ├── key: (4-6) - │ ├── fd: (4-6)-->(7-9), (4,7)-->(11), (5,7,8)-->(12), (6,7,9)-->(13) + │ ├── key: (5-7) + │ ├── fd: (5-7)-->(9-11), (5,9)-->(14), (6,9,10)-->(15), (7,9,11)-->(16) │ ├── left-join (lookup abc) - │ │ ├── columns: x:4!null y:5!null z:6!null abc.a:7 abc.b:8 abc.c:9 - │ │ ├── key columns: [4 5 6] = [7 8 9] + │ │ ├── columns: x:5!null y:6!null z:7!null abc.a:9 abc.b:10 abc.c:11 + │ │ ├── key columns: [5 6 7] = [9 10 11] │ │ ├── lookup columns are key │ │ ├── cardinality: [0 - 2] - │ │ ├── key: (4-6) - │ │ ├── fd: (4-6)-->(7-9) + │ │ ├── key: (5-7) + │ │ ├── fd: (5-7)-->(9-11) │ │ ├── ensure-upsert-distinct-on - │ │ │ ├── columns: x:4!null y:5!null z:6!null - │ │ │ ├── grouping columns: x:4!null y:5!null z:6!null - │ │ │ ├── internal-ordering: +5,+6 + │ │ │ ├── columns: x:5!null y:6!null z:7!null + │ │ │ ├── grouping columns: x:5!null y:6!null z:7!null + │ │ │ ├── internal-ordering: +6,+7 │ │ │ ├── error: "UPSERT or INSERT...ON CONFLICT command cannot affect row a second time" │ │ │ ├── cardinality: [0 - 2] - │ │ │ ├── key: (4-6) + │ │ │ ├── key: (5-7) │ │ │ └── limit - │ │ │ ├── columns: x:4!null y:5!null z:6!null - │ │ │ ├── internal-ordering: +5,+6 + │ │ │ ├── columns: x:5!null y:6!null z:7!null + │ │ │ ├── internal-ordering: +6,+7 │ │ │ ├── cardinality: [0 - 2] - │ │ │ ├── key: (4-6) - │ │ │ ├── ordering: +5,+6 + │ │ │ ├── key: (5-7) + │ │ │ ├── ordering: +6,+7 │ │ │ ├── sort - │ │ │ │ ├── columns: x:4!null y:5!null z:6!null - │ │ │ │ ├── key: (4-6) - │ │ │ │ ├── ordering: +5,+6 + │ │ │ │ ├── columns: x:5!null y:6!null z:7!null + │ │ │ │ ├── key: (5-7) + │ │ │ │ ├── ordering: +6,+7 │ │ │ │ ├── limit hint: 2.00 │ │ │ │ └── scan xyz - │ │ │ │ ├── columns: x:4!null y:5!null z:6!null - │ │ │ │ └── key: (4-6) + │ │ │ │ ├── columns: x:5!null y:6!null z:7!null + │ │ │ │ └── key: (5-7) │ │ │ └── 2 │ │ └── filters (true) │ └── projections - │ ├── CASE WHEN abc.a:7 IS NULL THEN x:4 ELSE 10 END [as=upsert_a:11, outer=(4,7)] - │ ├── CASE WHEN abc.a:7 IS NULL THEN y:5 ELSE abc.b:8 END [as=upsert_b:12, outer=(5,7,8)] - │ └── CASE WHEN abc.a:7 IS NULL THEN z:6 ELSE abc.c:9 END [as=upsert_c:13, outer=(6,7,9)] + │ ├── CASE WHEN abc.a:9 IS NULL THEN x:5 ELSE 10 END [as=upsert_a:14, outer=(5,9)] + │ ├── CASE WHEN abc.a:9 IS NULL THEN y:6 ELSE abc.b:10 END [as=upsert_b:15, outer=(6,9,10)] + │ └── CASE WHEN abc.a:9 IS NULL THEN z:7 ELSE abc.c:11 END [as=upsert_c:16, outer=(7,9,11)] └── with-scan &1 - ├── columns: a:14!null b:15!null c:16!null + ├── columns: a:17!null b:18!null c:19!null ├── mapping: - │ ├── abc.a:1 => a:14 - │ ├── abc.b:2 => b:15 - │ └── abc.c:3 => c:16 + │ ├── abc.a:1 => a:17 + │ ├── abc.b:2 => b:18 + │ └── abc.c:3 => c:19 └── cardinality: [0 - 2] # -------------------------------------------------- @@ -1857,32 +1857,32 @@ opt SELECT * FROM [DELETE FROM abcd RETURNING *] ORDER BY c ---- sort - ├── columns: a:11 b:12 c:13 d:14 + ├── columns: a:13 b:14 c:15 d:16 ├── volatile, mutations - ├── ordering: +13 + ├── ordering: +15 └── with &1 - ├── columns: a:11 b:12 c:13 d:14 + ├── columns: a:13 b:14 c:15 d:16 ├── volatile, mutations ├── project │ ├── columns: abcd.a:1 abcd.b:2 abcd.c:3 abcd.d:4 │ ├── volatile, mutations │ └── delete abcd │ ├── columns: abcd.a:1 abcd.b:2 abcd.c:3 abcd.d:4 rowid:5!null - │ ├── fetch columns: abcd.a:6 abcd.b:7 abcd.c:8 abcd.d:9 rowid:10 + │ ├── fetch columns: abcd.a:7 abcd.b:8 abcd.c:9 abcd.d:10 rowid:11 │ ├── volatile, mutations │ ├── key: (5) │ ├── fd: (5)-->(1-4) │ └── scan abcd - │ ├── columns: abcd.a:6 abcd.b:7 abcd.c:8 abcd.d:9 rowid:10!null - │ ├── key: (10) - │ └── fd: (10)-->(6-9) + │ ├── columns: abcd.a:7 abcd.b:8 abcd.c:9 abcd.d:10 rowid:11!null + │ ├── key: (11) + │ └── fd: (11)-->(7-10) └── with-scan &1 - ├── columns: a:11 b:12 c:13 d:14 + ├── columns: a:13 b:14 c:15 d:16 └── mapping: - ├── abcd.a:1 => a:11 - ├── abcd.b:2 => b:12 - ├── abcd.c:3 => c:13 - └── abcd.d:4 => d:14 + ├── abcd.a:1 => a:13 + ├── abcd.b:2 => b:14 + ├── abcd.c:3 => c:15 + └── abcd.d:4 => d:16 # Verify that provided orderings are derived correctly. opt @@ -1891,12 +1891,12 @@ FROM [DELETE FROM abcd ORDER BY c LIMIT 10 RETURNING *] ORDER BY c, d ---- sort - ├── columns: a:11 b:12 c:13 d:14 + ├── columns: a:13 b:14 c:15 d:16 ├── cardinality: [0 - 10] ├── volatile, mutations - ├── ordering: +13,+14 + ├── ordering: +15,+16 └── with &1 - ├── columns: a:11 b:12 c:13 d:14 + ├── columns: a:13 b:14 c:15 d:16 ├── cardinality: [0 - 10] ├── volatile, mutations ├── project @@ -1905,23 +1905,23 @@ sort │ ├── volatile, mutations │ └── delete abcd │ ├── columns: abcd.a:1 abcd.b:2 abcd.c:3 abcd.d:4 rowid:5!null - │ ├── fetch columns: abcd.a:6 abcd.b:7 abcd.c:8 abcd.d:9 rowid:10 + │ ├── fetch columns: abcd.a:7 abcd.b:8 abcd.c:9 abcd.d:10 rowid:11 │ ├── cardinality: [0 - 10] │ ├── volatile, mutations │ ├── key: (5) │ ├── fd: (5)-->(1-4) │ └── scan abcd@cd - │ ├── columns: abcd.a:6 abcd.b:7 abcd.c:8 abcd.d:9 rowid:10!null + │ ├── columns: abcd.a:7 abcd.b:8 abcd.c:9 abcd.d:10 rowid:11!null │ ├── limit: 10 - │ ├── key: (10) - │ └── fd: (10)-->(6-9) + │ ├── key: (11) + │ └── fd: (11)-->(7-10) └── with-scan &1 - ├── columns: a:11 b:12 c:13 d:14 + ├── columns: a:13 b:14 c:15 d:16 ├── mapping: - │ ├── abcd.a:1 => a:11 - │ ├── abcd.b:2 => b:12 - │ ├── abcd.c:3 => c:13 - │ └── abcd.d:4 => d:14 + │ ├── abcd.a:1 => a:13 + │ ├── abcd.b:2 => b:14 + │ ├── abcd.c:3 => c:15 + │ └── abcd.d:4 => d:16 └── cardinality: [0 - 10] # Verify that provided orderings are derived correctly with equivalence FD. @@ -1933,46 +1933,46 @@ WHERE b=c ORDER BY b, d ---- sort - ├── columns: a:11 b:12!null c:13!null d:14 + ├── columns: a:13 b:14!null c:15!null d:16 ├── cardinality: [0 - 10] ├── volatile, mutations - ├── fd: (12)==(13), (13)==(12) - ├── ordering: +(12|13),+14 [actual: +12,+14] + ├── fd: (14)==(15), (15)==(14) + ├── ordering: +(14|15),+16 [actual: +14,+16] └── with &1 - ├── columns: a:11 b:12!null c:13!null d:14 + ├── columns: a:13 b:14!null c:15!null d:16 ├── cardinality: [0 - 10] ├── volatile, mutations - ├── fd: (12)==(13), (13)==(12) + ├── fd: (14)==(15), (15)==(14) ├── project │ ├── columns: abcd.a:1 abcd.b:2 abcd.c:3 abcd.d:4 │ ├── cardinality: [0 - 10] │ ├── volatile, mutations │ └── delete abcd │ ├── columns: abcd.a:1 abcd.b:2 abcd.c:3 abcd.d:4 rowid:5!null - │ ├── fetch columns: abcd.a:6 abcd.b:7 abcd.c:8 abcd.d:9 rowid:10 + │ ├── fetch columns: abcd.a:7 abcd.b:8 abcd.c:9 abcd.d:10 rowid:11 │ ├── cardinality: [0 - 10] │ ├── volatile, mutations │ ├── key: (5) │ ├── fd: (5)-->(1-4) │ └── scan abcd@cd - │ ├── columns: abcd.a:6 abcd.b:7 abcd.c:8 abcd.d:9 rowid:10!null + │ ├── columns: abcd.a:7 abcd.b:8 abcd.c:9 abcd.d:10 rowid:11!null │ ├── limit: 10 - │ ├── key: (10) - │ └── fd: (10)-->(6-9) + │ ├── key: (11) + │ └── fd: (11)-->(7-10) └── select - ├── columns: a:11 b:12!null c:13!null d:14 + ├── columns: a:13 b:14!null c:15!null d:16 ├── cardinality: [0 - 10] - ├── fd: (12)==(13), (13)==(12) + ├── fd: (14)==(15), (15)==(14) ├── with-scan &1 - │ ├── columns: a:11 b:12 c:13 d:14 + │ ├── columns: a:13 b:14 c:15 d:16 │ ├── mapping: - │ │ ├── abcd.a:1 => a:11 - │ │ ├── abcd.b:2 => b:12 - │ │ ├── abcd.c:3 => c:13 - │ │ └── abcd.d:4 => d:14 + │ │ ├── abcd.a:1 => a:13 + │ │ ├── abcd.b:2 => b:14 + │ │ ├── abcd.c:3 => c:15 + │ │ └── abcd.d:4 => d:16 │ └── cardinality: [0 - 10] └── filters - └── b:12 = c:13 [outer=(12,13), fd=(12)==(13), (13)==(12)] + └── b:14 = c:15 [outer=(14,15), fd=(14)==(15), (15)==(14)] # Regression test for #36219: lookup join with ON condition that imposes an @@ -1981,22 +1981,22 @@ opt disable=(PushFilterIntoJoinLeftAndRight,PushFilterIntoJoinLeft,PushFilterInt SELECT * FROM abc JOIN xyz ON a=x AND x=z ORDER BY z ---- inner-join (merge) - ├── columns: a:1!null b:2!null c:3!null x:4!null y:5!null z:6!null + ├── columns: a:1!null b:2!null c:3!null x:5!null y:6!null z:7!null ├── left ordering: +1 - ├── right ordering: +4 - ├── key: (2,3,5,6) - ├── fd: (1)==(4,6), (4)==(1,6), (6)==(1,4) - ├── ordering: +(1|4|6) [actual: +1] + ├── right ordering: +5 + ├── key: (2,3,6,7) + ├── fd: (1)==(5,7), (5)==(1,7), (7)==(1,5) + ├── ordering: +(1|5|7) [actual: +1] ├── scan abc │ ├── columns: a:1!null b:2!null c:3!null │ ├── key: (1-3) │ └── ordering: +1 ├── scan xyz - │ ├── columns: x:4!null y:5!null z:6!null - │ ├── key: (4-6) - │ └── ordering: +4 + │ ├── columns: x:5!null y:6!null z:7!null + │ ├── key: (5-7) + │ └── ordering: +5 └── filters - └── x:4 = z:6 [outer=(4,6), fd=(4)==(6), (6)==(4)] + └── x:5 = z:7 [outer=(5,7), fd=(5)==(7), (7)==(5)] # TODO(justin): figure out when it is that window functions can preserve their # input ordering. @@ -2004,17 +2004,17 @@ opt SELECT *, row_number() OVER() FROM abc ORDER BY a ---- sort - ├── columns: a:1!null b:2!null c:3!null row_number:4 + ├── columns: a:1!null b:2!null c:3!null row_number:5 ├── key: (1-3) ├── ordering: +1 └── window partition=() - ├── columns: a:1!null b:2!null c:3!null row_number:4 + ├── columns: a:1!null b:2!null c:3!null row_number:5 ├── key: (1-3) ├── scan abc │ ├── columns: a:1!null b:2!null c:3!null │ └── key: (1-3) └── windows - └── row-number [as=row_number:4] + └── row-number [as=row_number:5] # Regression test for #44469 (DistinctOn needs to remap the provided ordering). exec-ddl @@ -2035,32 +2035,32 @@ FROM t44469_a INNER LOOKUP JOIN t44469_b ON a = b INNER LOOKUP JOIN t44469_cd ON ORDER BY b ---- distinct-on - ├── columns: b:3!null - ├── grouping columns: b:3!null - ├── key: (3) - ├── ordering: +3 + ├── columns: b:4!null + ├── grouping columns: b:4!null + ├── key: (4) + ├── ordering: +4 └── inner-join (lookup t44469_cd@secondary) - ├── columns: a:1!null b:3!null c:5!null d:6!null + ├── columns: a:1!null b:4!null c:7!null d:8!null ├── flags: force lookup join (into right side) - ├── key columns: [8 1] = [5 6] - ├── fd: ()-->(5), (1)==(3,6), (3)==(1,6), (6)==(1,3) - ├── ordering: +(1|3|6) opt(5) [actual: +1] + ├── key columns: [11 1] = [7 8] + ├── fd: ()-->(7), (1)==(4,8), (4)==(1,8), (8)==(1,4) + ├── ordering: +(1|4|8) opt(7) [actual: +1] ├── project - │ ├── columns: "project_const_col_@5":8!null a:1!null b:3!null - │ ├── fd: ()-->(8), (1)==(3), (3)==(1) - │ ├── ordering: +(1|3) [actual: +1] + │ ├── columns: "project_const_col_@7":11!null a:1!null b:4!null + │ ├── fd: ()-->(11), (1)==(4), (4)==(1) + │ ├── ordering: +(1|4) [actual: +1] │ ├── inner-join (lookup t44469_b@secondary) - │ │ ├── columns: a:1!null b:3!null + │ │ ├── columns: a:1!null b:4!null │ │ ├── flags: force lookup join (into right side) - │ │ ├── key columns: [1] = [3] - │ │ ├── fd: (1)==(3), (3)==(1) - │ │ ├── ordering: +(1|3) [actual: +1] + │ │ ├── key columns: [1] = [4] + │ │ ├── fd: (1)==(4), (4)==(1) + │ │ ├── ordering: +(1|4) [actual: +1] │ │ ├── scan t44469_a@secondary │ │ │ ├── columns: a:1 │ │ │ └── ordering: +1 │ │ └── filters (true) │ └── projections - │ └── 1 [as="project_const_col_@5":8] + │ └── 1 [as="project_const_col_@7":11] └── filters (true) # Regression test for #47041: factor check constraints into the (canonical) @@ -2079,8 +2079,8 @@ opt SELECT 1 FROM t47041 WHERE a > 1 AND k > 0 GROUP BY b, a ORDER BY b ---- project - ├── columns: "?column?":4!null - ├── fd: ()-->(4) + ├── columns: "?column?":5!null + ├── fd: ()-->(5) ├── distinct-on │ ├── columns: a:2!null │ ├── grouping columns: a:2!null @@ -2097,4 +2097,4 @@ project │ └── filters │ └── k:1 > 0 [outer=(1)] └── projections - └── 1 [as="?column?":4] + └── 1 [as="?column?":5] diff --git a/pkg/sql/opt/xform/testdata/physprops/presentation b/pkg/sql/opt/xform/testdata/physprops/presentation index 966100a42bbf..e96ce50b209b 100644 --- a/pkg/sql/opt/xform/testdata/physprops/presentation +++ b/pkg/sql/opt/xform/testdata/physprops/presentation @@ -35,33 +35,33 @@ opt SELECT 1+a.y AS plus, a.x FROM a ---- project - ├── columns: plus:3 x:1!null + ├── columns: plus:4 x:1!null ├── immutable ├── key: (1) - ├── fd: (1)-->(3) + ├── fd: (1)-->(4) ├── scan a │ ├── columns: x:1!null y:2 │ ├── key: (1) │ └── fd: (1)-->(2) └── projections - └── y:2 + 1 [as=plus:3, outer=(2), immutable] + └── y:2 + 1 [as=plus:4, outer=(2), immutable] # Join operator. opt SELECT b.x, rowid, a.y, a.x, a.y y2, b.y FROM a, b ---- inner-join (cross) - ├── columns: x:3 rowid:5!null y:2 x:1!null y2:2 y:4 - ├── key: (1,5) - ├── fd: (1)-->(2), (5)-->(3,4) + ├── columns: x:4 rowid:6!null y:2 x:1!null y2:2 y:5 + ├── key: (1,6) + ├── fd: (1)-->(2), (6)-->(4,5) ├── scan a │ ├── columns: a.x:1!null a.y:2 │ ├── key: (1) │ └── fd: (1)-->(2) ├── scan b - │ ├── columns: b.x:3 b.y:4 rowid:5!null - │ ├── key: (5) - │ └── fd: (5)-->(3,4) + │ ├── columns: b.x:4 b.y:5 rowid:6!null + │ ├── key: (6) + │ └── fd: (6)-->(4,5) └── filters (true) # Groupby operator. @@ -69,18 +69,18 @@ opt SELECT max(y), y, y, x FROM a GROUP BY a.x, a.y ---- group-by - ├── columns: max:3 y:2 y:2 x:1!null + ├── columns: max:4 y:2 y:2 x:1!null ├── grouping columns: x:1!null ├── internal-ordering: +1 ├── key: (1) - ├── fd: (1)-->(2,3) + ├── fd: (1)-->(2,4) ├── scan a │ ├── columns: x:1!null y:2 │ ├── key: (1) │ ├── fd: (1)-->(2) │ └── ordering: +1 └── aggregations - ├── max [as=max:3, outer=(2)] + ├── max [as=max:4, outer=(2)] │ └── y:2 └── const-agg [as=y:2, outer=(2)] └── y:2 diff --git a/pkg/sql/opt/xform/testdata/ruleprops/orderings b/pkg/sql/opt/xform/testdata/ruleprops/orderings index cddfb382b2a9..f4230302c67c 100644 --- a/pkg/sql/opt/xform/testdata/ruleprops/orderings +++ b/pkg/sql/opt/xform/testdata/ruleprops/orderings @@ -47,10 +47,10 @@ project ├── prune: (1,3) ├── interesting orderings: (+1) (+3) └── scan abc - ├── columns: a:1 b:2 c:3 rowid:4!null + ├── columns: a:1 b:2 c:3 rowid:4!null crdb_internal_mvcc_timestamp:5 ├── key: (4) - ├── fd: (4)-->(1-3), (3)~~>(1,2,4) - ├── prune: (1-4) + ├── fd: (4)-->(1-3,5), (3)~~>(1,2,4,5) + ├── prune: (1-5) └── interesting orderings: (+4) (+1,+2,+4) (+3,+4) build @@ -63,10 +63,10 @@ project ├── prune: (2,3) ├── interesting orderings: (+3) └── scan abc - ├── columns: a:1 b:2 c:3 rowid:4!null + ├── columns: a:1 b:2 c:3 rowid:4!null crdb_internal_mvcc_timestamp:5 ├── key: (4) - ├── fd: (4)-->(1-3), (3)~~>(1,2,4) - ├── prune: (1-4) + ├── fd: (4)-->(1-3,5), (3)~~>(1,2,4,5) + ├── prune: (1-5) └── interesting orderings: (+4) (+1,+2,+4) (+3,+4) # GroupBy operator. @@ -74,12 +74,12 @@ opt SELECT min(b), a FROM abc GROUP BY a ---- group-by - ├── columns: min:5 a:1 + ├── columns: min:6 a:1 ├── grouping columns: a:1 ├── internal-ordering: +1 ├── key: (1) - ├── fd: (1)-->(5) - ├── prune: (5) + ├── fd: (1)-->(6) + ├── prune: (6) ├── interesting orderings: (+1) ├── scan abc@secondary │ ├── columns: a:1 b:2 @@ -87,18 +87,18 @@ group-by │ ├── prune: (1,2) │ └── interesting orderings: (+1,+2) └── aggregations - └── min [as=min:5, outer=(2)] + └── min [as=min:6, outer=(2)] └── b:2 opt SELECT min(b), c FROM abc GROUP BY c ---- group-by - ├── columns: min:5 c:3 + ├── columns: min:6 c:3 ├── grouping columns: c:3 ├── key: (3) - ├── fd: (3)-->(5) - ├── prune: (5) + ├── fd: (3)-->(6) + ├── prune: (6) ├── interesting orderings: (+3) ├── scan abc │ ├── columns: b:2 c:3 @@ -107,7 +107,7 @@ group-by │ ├── prune: (2,3) │ └── interesting orderings: (+3) └── aggregations - └── min [as=min:5, outer=(2)] + └── min [as=min:6, outer=(2)] └── b:2 # GroupBy with required ordering. @@ -115,12 +115,12 @@ opt SELECT array_agg(a), b, c FROM (SELECT * FROM abc ORDER BY b, a) GROUP BY b, c ---- group-by - ├── columns: array_agg:5 b:2 c:3 + ├── columns: array_agg:6 b:2 c:3 ├── grouping columns: b:2 c:3 ├── internal-ordering: +1 opt(2,3) ├── key: (2,3) - ├── fd: (3)~~>(2), (2,3)-->(5) - ├── prune: (5) + ├── fd: (3)~~>(2), (2,3)-->(6) + ├── prune: (6) ├── sort │ ├── columns: a:1 b:2 c:3 │ ├── lax-key: (1-3) @@ -135,7 +135,7 @@ group-by │ ├── prune: (1-3) │ └── interesting orderings: (+1,+2) (+3) └── aggregations - └── array-agg [as=array_agg:5, outer=(1)] + └── array-agg [as=array_agg:6, outer=(1)] └── a:1 # Scalar GroupBy case. @@ -143,11 +143,11 @@ opt SELECT max(a), min(b), sum(c) FROM abc ---- scalar-group-by - ├── columns: max:5 min:6 sum:7 + ├── columns: max:6 min:7 sum:8 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(5-7) - ├── prune: (5-7) + ├── fd: ()-->(6-8) + ├── prune: (6-8) ├── scan abc │ ├── columns: a:1 b:2 c:3 │ ├── lax-key: (1-3) @@ -155,11 +155,11 @@ scalar-group-by │ ├── prune: (1-3) │ └── interesting orderings: (+1,+2) (+3) └── aggregations - ├── max [as=max:5, outer=(1)] + ├── max [as=max:6, outer=(1)] │ └── a:1 - ├── min [as=min:6, outer=(2)] + ├── min [as=min:7, outer=(2)] │ └── b:2 - └── sum [as=sum:7, outer=(3)] + └── sum [as=sum:8, outer=(3)] └── c:3 # LookupJoin operator. @@ -264,24 +264,24 @@ opt SELECT * FROM abc JOIN xyz ON a=x ---- inner-join (hash) - ├── columns: a:1!null b:2 c:3 x:5!null y:6 z:7 - ├── lax-key: (2,3,5-7) - ├── fd: (3)~~>(1,2), (5,6)~~>(7), (1)==(5), (5)==(1) - ├── prune: (2,3,6,7) - ├── interesting orderings: (+1,+2) (+3) (+7) (+5,+6) + ├── columns: a:1!null b:2 c:3 x:6!null y:7 z:8 + ├── lax-key: (2,3,6-8) + ├── fd: (3)~~>(1,2), (6,7)~~>(8), (1)==(6), (6)==(1) + ├── prune: (2,3,7,8) + ├── interesting orderings: (+1,+2) (+3) (+8) (+6,+7) ├── scan abc │ ├── columns: a:1 b:2 c:3 │ ├── lax-key: (1-3) │ ├── fd: (3)~~>(1,2) │ ├── prune: (1-3) │ ├── interesting orderings: (+1,+2) (+3) - │ └── unfiltered-cols: (1-4) + │ └── unfiltered-cols: (1-5) ├── scan xyz - │ ├── columns: x:5 y:6 z:7 - │ ├── lax-key: (5-7) - │ ├── fd: (5,6)~~>(7) - │ ├── prune: (5-7) - │ ├── interesting orderings: (+7) (+5,+6) - │ └── unfiltered-cols: (5-8) + │ ├── columns: x:6 y:7 z:8 + │ ├── lax-key: (6-8) + │ ├── fd: (6,7)~~>(8) + │ ├── prune: (6-8) + │ ├── interesting orderings: (+8) (+6,+7) + │ └── unfiltered-cols: (6-10) └── filters - └── a:1 = x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + └── a:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] diff --git a/pkg/sql/opt/xform/testdata/rules/combo b/pkg/sql/opt/xform/testdata/rules/combo index 1a1bfc9f7096..b4af2c2f70bf 100644 --- a/pkg/sql/opt/xform/testdata/rules/combo +++ b/pkg/sql/opt/xform/testdata/rules/combo @@ -30,109 +30,109 @@ GenerateIndexScans ================================================================================ Source expression: inner-join (hash) - ├── columns: a:1!null b:2!null c:3 x:5!null y:6!null z:7 - ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2) + ├── columns: a:1!null b:2!null c:3 x:6!null y:7!null z:8 + ├── fd: (1)==(6), (6)==(1), (2)==(7), (7)==(2) ├── scan abc │ └── columns: a:1 b:2 c:3 ├── scan xyz - │ └── columns: x:5 y:6 z:7 + │ └── columns: x:6 y:7 z:8 └── filters - ├── a:1 = x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - └── b:2 = y:6 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + ├── a:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── b:2 = y:7 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] New expression 1 of 1: inner-join (hash) - ├── columns: a:1!null b:2!null c:3 x:5!null y:6!null z:7 - ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2) + ├── columns: a:1!null b:2!null c:3 x:6!null y:7!null z:8 + ├── fd: (1)==(6), (6)==(1), (2)==(7), (7)==(2) ├── scan abc@ab │ └── columns: a:1 b:2 c:3 ├── scan xyz - │ └── columns: x:5 y:6 z:7 + │ └── columns: x:6 y:7 z:8 └── filters - ├── a:1 = x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - └── b:2 = y:6 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + ├── a:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── b:2 = y:7 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] ================================================================================ GenerateIndexScans ================================================================================ Source expression: inner-join (hash) - ├── columns: a:1!null b:2!null c:3 x:5!null y:6!null z:7 - ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2) + ├── columns: a:1!null b:2!null c:3 x:6!null y:7!null z:8 + ├── fd: (1)==(6), (6)==(1), (2)==(7), (7)==(2) ├── scan abc │ └── columns: a:1 b:2 c:3 ├── scan xyz - │ └── columns: x:5 y:6 z:7 + │ └── columns: x:6 y:7 z:8 └── filters - ├── a:1 = x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - └── b:2 = y:6 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + ├── a:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── b:2 = y:7 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] New expression 1 of 1: inner-join (hash) - ├── columns: a:1!null b:2!null c:3 x:5!null y:6!null z:7 - ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2) + ├── columns: a:1!null b:2!null c:3 x:6!null y:7!null z:8 + ├── fd: (1)==(6), (6)==(1), (2)==(7), (7)==(2) ├── scan abc │ └── columns: a:1 b:2 c:3 ├── scan xyz@xy - │ └── columns: x:5 y:6 z:7 + │ └── columns: x:6 y:7 z:8 └── filters - ├── a:1 = x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - └── b:2 = y:6 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + ├── a:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── b:2 = y:7 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] ================================================================================ CommuteJoin ================================================================================ Source expression: inner-join (hash) - ├── columns: a:1!null b:2!null c:3 x:5!null y:6!null z:7 - ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2) + ├── columns: a:1!null b:2!null c:3 x:6!null y:7!null z:8 + ├── fd: (1)==(6), (6)==(1), (2)==(7), (7)==(2) ├── scan abc │ └── columns: a:1 b:2 c:3 ├── scan xyz - │ └── columns: x:5 y:6 z:7 + │ └── columns: x:6 y:7 z:8 └── filters - ├── a:1 = x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - └── b:2 = y:6 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + ├── a:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── b:2 = y:7 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] New expression 1 of 1: inner-join (hash) - ├── columns: a:1!null b:2!null c:3 x:5!null y:6!null z:7 - ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2) + ├── columns: a:1!null b:2!null c:3 x:6!null y:7!null z:8 + ├── fd: (1)==(6), (6)==(1), (2)==(7), (7)==(2) ├── scan xyz - │ └── columns: x:5 y:6 z:7 + │ └── columns: x:6 y:7 z:8 ├── scan abc │ └── columns: a:1 b:2 c:3 └── filters - ├── a:1 = x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - └── b:2 = y:6 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + ├── a:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── b:2 = y:7 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] ================================================================================ GenerateMergeJoins ================================================================================ Source expression: inner-join (hash) - ├── columns: a:1!null b:2!null c:3 x:5!null y:6!null z:7 - ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2) + ├── columns: a:1!null b:2!null c:3 x:6!null y:7!null z:8 + ├── fd: (1)==(6), (6)==(1), (2)==(7), (7)==(2) ├── scan abc │ └── columns: a:1 b:2 c:3 ├── scan xyz - │ └── columns: x:5 y:6 z:7 + │ └── columns: x:6 y:7 z:8 └── filters - ├── a:1 = x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - └── b:2 = y:6 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + ├── a:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── b:2 = y:7 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] New expression 1 of 1: inner-join (merge) - ├── columns: a:1!null b:2!null c:3 x:5!null y:6!null z:7 + ├── columns: a:1!null b:2!null c:3 x:6!null y:7!null z:8 ├── left ordering: +1,+2 - ├── right ordering: +5,+6 - ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2) + ├── right ordering: +6,+7 + ├── fd: (1)==(6), (6)==(1), (2)==(7), (7)==(2) ├── scan abc@ab │ ├── columns: a:1 b:2 c:3 │ └── ordering: +1,+2 ├── scan xyz@xy - │ ├── columns: x:5 y:6 z:7 - │ └── ordering: +5,+6 + │ ├── columns: x:6 y:7 z:8 + │ └── ordering: +6,+7 └── filters (true) ================================================================================ @@ -140,21 +140,21 @@ GenerateLookupJoins ================================================================================ Source expression: inner-join (hash) - ├── columns: a:1!null b:2!null c:3 x:5!null y:6!null z:7 - ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2) + ├── columns: a:1!null b:2!null c:3 x:6!null y:7!null z:8 + ├── fd: (1)==(6), (6)==(1), (2)==(7), (7)==(2) ├── scan abc │ └── columns: a:1 b:2 c:3 ├── scan xyz - │ └── columns: x:5 y:6 z:7 + │ └── columns: x:6 y:7 z:8 └── filters - ├── a:1 = x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - └── b:2 = y:6 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + ├── a:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── b:2 = y:7 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] New expression 1 of 1: inner-join (lookup xyz@xy) - ├── columns: a:1!null b:2!null c:3 x:5!null y:6!null z:7 - ├── key columns: [1 2] = [5 6] - ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2) + ├── columns: a:1!null b:2!null c:3 x:6!null y:7!null z:8 + ├── key columns: [1 2] = [6 7] + ├── fd: (1)==(6), (6)==(1), (2)==(7), (7)==(2) ├── scan abc │ └── columns: a:1 b:2 c:3 └── filters (true) @@ -164,15 +164,15 @@ CommuteJoin ================================================================================ Source expression: inner-join (hash) - ├── columns: a:1!null b:2!null c:3 x:5!null y:6!null z:7 - ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2) + ├── columns: a:1!null b:2!null c:3 x:6!null y:7!null z:8 + ├── fd: (1)==(6), (6)==(1), (2)==(7), (7)==(2) ├── scan xyz - │ └── columns: x:5 y:6 z:7 + │ └── columns: x:6 y:7 z:8 ├── scan abc │ └── columns: a:1 b:2 c:3 └── filters - ├── a:1 = x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - └── b:2 = y:6 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + ├── a:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── b:2 = y:7 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] No new expressions. @@ -181,25 +181,25 @@ GenerateMergeJoins ================================================================================ Source expression: inner-join (hash) - ├── columns: a:1!null b:2!null c:3 x:5!null y:6!null z:7 - ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2) + ├── columns: a:1!null b:2!null c:3 x:6!null y:7!null z:8 + ├── fd: (1)==(6), (6)==(1), (2)==(7), (7)==(2) ├── scan xyz - │ └── columns: x:5 y:6 z:7 + │ └── columns: x:6 y:7 z:8 ├── scan abc │ └── columns: a:1 b:2 c:3 └── filters - ├── a:1 = x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - └── b:2 = y:6 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + ├── a:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── b:2 = y:7 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] New expression 1 of 1: inner-join (merge) - ├── columns: a:1!null b:2!null c:3 x:5!null y:6!null z:7 - ├── left ordering: +5,+6 + ├── columns: a:1!null b:2!null c:3 x:6!null y:7!null z:8 + ├── left ordering: +6,+7 ├── right ordering: +1,+2 - ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2) + ├── fd: (1)==(6), (6)==(1), (2)==(7), (7)==(2) ├── scan xyz@xy - │ ├── columns: x:5 y:6 z:7 - │ └── ordering: +5,+6 + │ ├── columns: x:6 y:7 z:8 + │ └── ordering: +6,+7 ├── scan abc@ab │ ├── columns: a:1 b:2 c:3 │ └── ordering: +1,+2 @@ -210,23 +210,23 @@ GenerateLookupJoins ================================================================================ Source expression: inner-join (hash) - ├── columns: a:1!null b:2!null c:3 x:5!null y:6!null z:7 - ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2) + ├── columns: a:1!null b:2!null c:3 x:6!null y:7!null z:8 + ├── fd: (1)==(6), (6)==(1), (2)==(7), (7)==(2) ├── scan xyz - │ └── columns: x:5 y:6 z:7 + │ └── columns: x:6 y:7 z:8 ├── scan abc │ └── columns: a:1 b:2 c:3 └── filters - ├── a:1 = x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - └── b:2 = y:6 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + ├── a:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── b:2 = y:7 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] New expression 1 of 1: inner-join (lookup abc@ab) - ├── columns: a:1!null b:2!null c:3 x:5!null y:6!null z:7 - ├── key columns: [5 6] = [1 2] - ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2) + ├── columns: a:1!null b:2!null c:3 x:6!null y:7!null z:8 + ├── key columns: [6 7] = [1 2] + ├── fd: (1)==(6), (6)==(1), (2)==(7), (7)==(2) ├── scan xyz - │ └── columns: x:5 y:6 z:7 + │ └── columns: x:6 y:7 z:8 └── filters (true) ---- ---- diff --git a/pkg/sql/opt/xform/testdata/rules/groupby b/pkg/sql/opt/xform/testdata/rules/groupby index cca4cc769559..9d4c5619b472 100644 --- a/pkg/sql/opt/xform/testdata/rules/groupby +++ b/pkg/sql/opt/xform/testdata/rules/groupby @@ -40,17 +40,17 @@ opt SELECT min(a) FROM abc ---- scalar-group-by - ├── columns: min:5 + ├── columns: min:6 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(5) + ├── fd: ()-->(6) ├── scan abc │ ├── columns: a:1!null │ ├── limit: 1 │ ├── key: () │ └── fd: ()-->(1) └── aggregations - └── const-agg [as=min:5, outer=(1)] + └── const-agg [as=min:6, outer=(1)] └── a:1 # Verify the rule still fires even if DISTINCT is used. @@ -58,17 +58,17 @@ opt SELECT min(DISTINCT a) FROM abc ---- scalar-group-by - ├── columns: min:5 + ├── columns: min:6 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(5) + ├── fd: ()-->(6) ├── scan abc │ ├── columns: a:1!null │ ├── limit: 1 │ ├── key: () │ └── fd: ()-->(1) └── aggregations - └── const-agg [as=min:5, outer=(1)] + └── const-agg [as=min:6, outer=(1)] └── a:1 # Verify the rule does not fire when FILTER is used. @@ -76,10 +76,10 @@ opt SELECT min(a) FILTER (WHERE a > 'a') FROM abc ---- scalar-group-by - ├── columns: min:6 + ├── columns: min:7 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(6) + ├── fd: ()-->(7) ├── scan abc │ ├── columns: a:1!null │ ├── constraint: /1: [/e'a\x00' - ] @@ -87,31 +87,31 @@ scalar-group-by │ ├── key: () │ └── fd: ()-->(1) └── aggregations - └── const-agg [as=min:6, outer=(1)] + └── const-agg [as=min:7, outer=(1)] └── a:1 opt SELECT min(b) FROM abc ---- scalar-group-by - ├── columns: min:5 + ├── columns: min:6 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(5) + ├── fd: ()-->(6) ├── scan abc │ └── columns: b:2 └── aggregations - └── min [as=min:5, outer=(2)] + └── min [as=min:6, outer=(2)] └── b:2 opt SELECT min(y) FROM xyz where z=7 ---- scalar-group-by - ├── columns: min:4 + ├── columns: min:5 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(4) + ├── fd: ()-->(5) ├── scan xyz@zyx │ ├── columns: y:2!null z:3!null │ ├── constraint: /3/2/1: (/7.0/NULL - /7.0] @@ -119,7 +119,7 @@ scalar-group-by │ ├── key: () │ └── fd: ()-->(2,3) └── aggregations - └── const-agg [as=min:4, outer=(2)] + └── const-agg [as=min:5, outer=(2)] └── y:2 # ReplaceScalarMaxWithLimit has the same behavior with max() as @@ -128,10 +128,10 @@ opt SELECT max(y) FROM xyz where z=7 ---- scalar-group-by - ├── columns: max:4 + ├── columns: max:5 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(4) + ├── fd: ()-->(5) ├── scan xyz@zyx,rev │ ├── columns: y:2!null z:3!null │ ├── constraint: /3/2/1: (/7.0/NULL - /7.0] @@ -139,7 +139,7 @@ scalar-group-by │ ├── key: () │ └── fd: ()-->(2,3) └── aggregations - └── const-agg [as=max:4, outer=(2)] + └── const-agg [as=max:5, outer=(2)] └── y:2 # We expect ReplaceScalarMinWithLimit not to be preferred here. @@ -149,10 +149,10 @@ opt SELECT min(y) FROM xyz@xy WHERE x>7 ---- scalar-group-by - ├── columns: min:4 + ├── columns: min:5 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(4) + ├── fd: ()-->(5) ├── scan xyz@xy │ ├── columns: x:1!null y:2 │ ├── constraint: /1/2: [/8 - ] @@ -160,7 +160,7 @@ scalar-group-by │ ├── key: (1) │ └── fd: (1)-->(2) └── aggregations - └── min [as=min:4, outer=(2)] + └── min [as=min:5, outer=(2)] └── y:2 # We expect ReplaceMaxWithLimit not to be preferred here. @@ -170,10 +170,10 @@ opt SELECT max(y) FROM xyz@xy WHERE x>7 ---- scalar-group-by - ├── columns: max:4 + ├── columns: max:5 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(4) + ├── fd: ()-->(5) ├── scan xyz@xy │ ├── columns: x:1!null y:2 │ ├── constraint: /1/2: [/8 - ] @@ -181,51 +181,51 @@ scalar-group-by │ ├── key: (1) │ └── fd: (1)-->(2) └── aggregations - └── max [as=max:4, outer=(2)] + └── max [as=max:5, outer=(2)] └── y:2 opt SELECT max(x) FROM xyz ---- scalar-group-by - ├── columns: max:4 + ├── columns: max:5 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(4) + ├── fd: ()-->(5) ├── scan xyz@xy,rev │ ├── columns: x:1!null │ ├── limit: 1(rev) │ ├── key: () │ └── fd: ()-->(1) └── aggregations - └── const-agg [as=max:4, outer=(1)] + └── const-agg [as=max:5, outer=(1)] └── x:1 opt SELECT min(x) FROM xyz ---- scalar-group-by - ├── columns: min:4 + ├── columns: min:5 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(4) + ├── fd: ()-->(5) ├── scan xyz@xy │ ├── columns: x:1!null │ ├── limit: 1 │ ├── key: () │ └── fd: ()-->(1) └── aggregations - └── const-agg [as=min:4, outer=(1)] + └── const-agg [as=min:5, outer=(1)] └── x:1 opt SELECT min(x) FROM xyz WHERE x in (0, 4, 7) ---- scalar-group-by - ├── columns: min:4 + ├── columns: min:5 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(4) + ├── fd: ()-->(5) ├── scan xyz@xy │ ├── columns: x:1!null │ ├── constraint: /1/2 @@ -236,17 +236,17 @@ scalar-group-by │ ├── key: () │ └── fd: ()-->(1) └── aggregations - └── const-agg [as=min:4, outer=(1)] + └── const-agg [as=min:5, outer=(1)] └── x:1 opt SELECT max(x) FROM xyz WHERE x in (0, 4, 7) ---- scalar-group-by - ├── columns: max:4 + ├── columns: max:5 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(4) + ├── fd: ()-->(5) ├── scan xyz@xy,rev │ ├── columns: x:1!null │ ├── constraint: /1/2 @@ -257,17 +257,17 @@ scalar-group-by │ ├── key: () │ └── fd: ()-->(1) └── aggregations - └── const-agg [as=max:4, outer=(1)] + └── const-agg [as=max:5, outer=(1)] └── x:1 opt SELECT min(y) FROM xyz ---- scalar-group-by - ├── columns: min:4 + ├── columns: min:5 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(4) + ├── fd: ()-->(5) ├── scan xyz@yy │ ├── columns: y:2!null │ ├── constraint: /2/1: (/NULL - ] @@ -275,17 +275,17 @@ scalar-group-by │ ├── key: () │ └── fd: ()-->(2) └── aggregations - └── const-agg [as=min:4, outer=(2)] + └── const-agg [as=min:5, outer=(2)] └── y:2 opt SELECT min(y), min(y) FROM xyz ---- scalar-group-by - ├── columns: min:4 min:4 + ├── columns: min:5 min:5 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(4) + ├── fd: ()-->(5) ├── scan xyz@yy │ ├── columns: y:2!null │ ├── constraint: /2/1: (/NULL - ] @@ -293,7 +293,7 @@ scalar-group-by │ ├── key: () │ └── fd: ()-->(2) └── aggregations - └── const-agg [as=min:4, outer=(2)] + └── const-agg [as=min:5, outer=(2)] └── y:2 # ReplaceScalarMinWithLimit does not apply when there is @@ -302,18 +302,18 @@ opt SELECT min(y) FROM xyz GROUP BY y ---- project - ├── columns: min:4 + ├── columns: min:5 └── group-by - ├── columns: y:2 min:4 + ├── columns: y:2 min:5 ├── grouping columns: y:2 ├── internal-ordering: +2 ├── key: (2) - ├── fd: (2)-->(4) + ├── fd: (2)-->(5) ├── scan xyz@yy │ ├── columns: y:2 │ └── ordering: +2 └── aggregations - └── min [as=min:4, outer=(2)] + └── min [as=min:5, outer=(2)] └── y:2 # ReplaceScalarMaxWithLimit does not apply when there is @@ -322,18 +322,18 @@ opt SELECT max(y) FROM xyz GROUP BY y ---- project - ├── columns: max:4 + ├── columns: max:5 └── group-by - ├── columns: y:2 max:4 + ├── columns: y:2 max:5 ├── grouping columns: y:2 ├── internal-ordering: +2 ├── key: (2) - ├── fd: (2)-->(4) + ├── fd: (2)-->(5) ├── scan xyz@yy │ ├── columns: y:2 │ └── ordering: +2 └── aggregations - └── max [as=max:4, outer=(2)] + └── max [as=max:5, outer=(2)] └── y:2 # ReplaceScalarMinWithLimit does not apply when there is @@ -342,20 +342,20 @@ opt SELECT min(y) FROM xyz GROUP BY x ---- project - ├── columns: min:4 + ├── columns: min:5 └── group-by - ├── columns: x:1!null min:4 + ├── columns: x:1!null min:5 ├── grouping columns: x:1!null ├── internal-ordering: +1 ├── key: (1) - ├── fd: (1)-->(4) + ├── fd: (1)-->(5) ├── scan xyz@xy │ ├── columns: x:1!null y:2 │ ├── key: (1) │ ├── fd: (1)-->(2) │ └── ordering: +1 └── aggregations - └── min [as=min:4, outer=(2)] + └── min [as=min:5, outer=(2)] └── y:2 # ReplaceScalarMinWithLimit does not apply on multiple aggregations @@ -364,18 +364,18 @@ opt SELECT min(y), min(x) FROM xyz ---- scalar-group-by - ├── columns: min:4 min:5 + ├── columns: min:5 min:6 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(4,5) + ├── fd: ()-->(5,6) ├── scan xyz@xy │ ├── columns: x:1!null y:2 │ ├── key: (1) │ └── fd: (1)-->(2) └── aggregations - ├── min [as=min:4, outer=(2)] + ├── min [as=min:5, outer=(2)] │ └── y:2 - └── min [as=min:5, outer=(1)] + └── min [as=min:6, outer=(1)] └── x:1 @@ -385,18 +385,18 @@ opt SELECT max(y), max(x) FROM xyz ---- scalar-group-by - ├── columns: max:4 max:5 + ├── columns: max:5 max:6 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(4,5) + ├── fd: ()-->(5,6) ├── scan xyz@xy │ ├── columns: x:1!null y:2 │ ├── key: (1) │ └── fd: (1)-->(2) └── aggregations - ├── max [as=max:4, outer=(2)] + ├── max [as=max:5, outer=(2)] │ └── y:2 - └── max [as=max:5, outer=(1)] + └── max [as=max:6, outer=(1)] └── x:1 # ReplaceScalarMinWithLimit does not apply with @@ -405,18 +405,18 @@ opt SELECT x,min(y) FROM xyz GROUP BY x,y ---- group-by - ├── columns: x:1!null min:4 + ├── columns: x:1!null min:5 ├── grouping columns: x:1!null ├── internal-ordering: +1 ├── key: (1) - ├── fd: (1)-->(4) + ├── fd: (1)-->(5) ├── scan xyz@xy │ ├── columns: x:1!null y:2 │ ├── key: (1) │ ├── fd: (1)-->(2) │ └── ordering: +1 └── aggregations - └── min [as=min:4, outer=(2)] + └── min [as=min:5, outer=(2)] └── y:2 # ReplaceScalarMaxWithLimit does not apply with @@ -425,18 +425,18 @@ opt SELECT x,max(y) FROM xyz GROUP BY x,y ---- group-by - ├── columns: x:1!null max:4 + ├── columns: x:1!null max:5 ├── grouping columns: x:1!null ├── internal-ordering: +1 ├── key: (1) - ├── fd: (1)-->(4) + ├── fd: (1)-->(5) ├── scan xyz@xy │ ├── columns: x:1!null y:2 │ ├── key: (1) │ ├── fd: (1)-->(2) │ └── ordering: +1 └── aggregations - └── max [as=max:4, outer=(2)] + └── max [as=max:5, outer=(2)] └── y:2 # ReplaceScalarMinWithLimit does not apply to non-scalar @@ -445,22 +445,22 @@ opt SELECT min(x), count(y) FROM xyz GROUP BY x,y ---- project - ├── columns: min:4!null count:5!null + ├── columns: min:5!null count:6!null └── group-by - ├── columns: x:1!null min:4!null count:5!null + ├── columns: x:1!null min:5!null count:6!null ├── grouping columns: x:1!null ├── internal-ordering: +1 ├── key: (1) - ├── fd: (1)-->(4,5) + ├── fd: (1)-->(5,6) ├── scan xyz@xy │ ├── columns: x:1!null y:2 │ ├── key: (1) │ ├── fd: (1)-->(2) │ └── ordering: +1 └── aggregations - ├── min [as=min:4, outer=(1)] + ├── min [as=min:5, outer=(1)] │ └── x:1 - └── count [as=count:5, outer=(2)] + └── count [as=count:6, outer=(2)] └── y:2 # ReplaceScalarMaxWithLimit does not apply to non-scalar @@ -469,30 +469,30 @@ opt SELECT max(x), count(y) FROM xyz GROUP BY x,y ---- project - ├── columns: max:4!null count:5!null + ├── columns: max:5!null count:6!null └── group-by - ├── columns: x:1!null max:4!null count:5!null + ├── columns: x:1!null max:5!null count:6!null ├── grouping columns: x:1!null ├── internal-ordering: +1 ├── key: (1) - ├── fd: (1)-->(4,5) + ├── fd: (1)-->(5,6) ├── scan xyz@xy │ ├── columns: x:1!null y:2 │ ├── key: (1) │ ├── fd: (1)-->(2) │ └── ordering: +1 └── aggregations - ├── max [as=max:4, outer=(1)] + ├── max [as=max:5, outer=(1)] │ └── x:1 - └── count [as=count:5, outer=(2)] + └── count [as=count:6, outer=(2)] └── y:2 memo SELECT min(a) FROM abc ---- -memo (optimized, ~5KB, required=[presentation: min:5]) +memo (optimized, ~5KB, required=[presentation: min:6]) ├── G1: (scalar-group-by G2 G3 cols=()) (scalar-group-by G4 G5 cols=()) - │ └── [presentation: min:5] + │ └── [presentation: min:6] │ ├── best: (scalar-group-by G4 G5 cols=()) │ └── cost: 1.10 ├── G2: (scan abc,cols=(1)) @@ -516,9 +516,9 @@ memo (optimized, ~5KB, required=[presentation: min:5]) memo SELECT min(b) FROM abc ---- -memo (optimized, ~6KB, required=[presentation: min:5]) +memo (optimized, ~6KB, required=[presentation: min:6]) ├── G1: (scalar-group-by G2 G3 cols=()) (scalar-group-by G4 G5 cols=()) - │ └── [presentation: min:5] + │ └── [presentation: min:6] │ ├── best: (scalar-group-by G2 G3 cols=()) │ └── cost: 1060.05 ├── G2: (scan abc,cols=(2)) @@ -552,9 +552,9 @@ memo (optimized, ~6KB, required=[presentation: min:5]) memo SELECT max(a) FROM abc ---- -memo (optimized, ~5KB, required=[presentation: max:5]) +memo (optimized, ~5KB, required=[presentation: max:6]) ├── G1: (scalar-group-by G2 G3 cols=()) (scalar-group-by G4 G5 cols=()) - │ └── [presentation: max:5] + │ └── [presentation: max:6] │ ├── best: (scalar-group-by G4 G5 cols=()) │ └── cost: 1.10 ├── G2: (scan abc,cols=(1)) @@ -578,9 +578,9 @@ memo (optimized, ~5KB, required=[presentation: max:5]) memo SELECT max(b) FROM abc ---- -memo (optimized, ~6KB, required=[presentation: max:5]) +memo (optimized, ~6KB, required=[presentation: max:6]) ├── G1: (scalar-group-by G2 G3 cols=()) (scalar-group-by G4 G5 cols=()) - │ └── [presentation: max:5] + │ └── [presentation: max:6] │ ├── best: (scalar-group-by G2 G3 cols=()) │ └── cost: 1060.05 ├── G2: (scan abc,cols=(2)) @@ -619,17 +619,17 @@ opt SELECT max(a) FROM abc ---- scalar-group-by - ├── columns: max:5 + ├── columns: max:6 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(5) + ├── fd: ()-->(6) ├── scan abc,rev │ ├── columns: a:1!null │ ├── limit: 1(rev) │ ├── key: () │ └── fd: ()-->(1) └── aggregations - └── const-agg [as=max:5, outer=(1)] + └── const-agg [as=max:6, outer=(1)] └── a:1 # Verify the rule still fires even if DISTINCT is used. @@ -637,17 +637,17 @@ opt SELECT max(DISTINCT a) FROM abc ---- scalar-group-by - ├── columns: max:5 + ├── columns: max:6 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(5) + ├── fd: ()-->(6) ├── scan abc,rev │ ├── columns: a:1!null │ ├── limit: 1(rev) │ ├── key: () │ └── fd: ()-->(1) └── aggregations - └── const-agg [as=max:5, outer=(1)] + └── const-agg [as=max:6, outer=(1)] └── a:1 # Verify the rule does not fire when FILTER is used. @@ -655,10 +655,10 @@ opt SELECT max(a) FILTER (WHERE a > 'a') FROM abc ---- scalar-group-by - ├── columns: max:6 + ├── columns: max:7 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(6) + ├── fd: ()-->(7) ├── scan abc,rev │ ├── columns: a:1!null │ ├── constraint: /1: [/e'a\x00' - ] @@ -666,29 +666,29 @@ scalar-group-by │ ├── key: () │ └── fd: ()-->(1) └── aggregations - └── const-agg [as=max:6, outer=(1)] + └── const-agg [as=max:7, outer=(1)] └── a:1 opt SELECT max(b) FROM abc ---- scalar-group-by - ├── columns: max:5 + ├── columns: max:6 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(5) + ├── fd: ()-->(6) ├── scan abc │ └── columns: b:2 └── aggregations - └── max [as=max:5, outer=(2)] + └── max [as=max:6, outer=(2)] └── b:2 memo SELECT max(b) FROM abc ---- -memo (optimized, ~6KB, required=[presentation: max:5]) +memo (optimized, ~6KB, required=[presentation: max:6]) ├── G1: (scalar-group-by G2 G3 cols=()) (scalar-group-by G4 G5 cols=()) - │ └── [presentation: max:5] + │ └── [presentation: max:6] │ ├── best: (scalar-group-by G2 G3 cols=()) │ └── cost: 1060.05 ├── G2: (scan abc,cols=(2)) @@ -728,10 +728,10 @@ opt expect=ReplaceMinWithLimit SELECT min(k) FROM kuvw WHERE w = 5 GROUP BY w ---- project - ├── columns: min:5!null + ├── columns: min:6!null ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(5) + ├── fd: ()-->(6) ├── scan kuvw@w │ ├── columns: k:1!null w:4!null │ ├── constraint: /4/1: [/5 - /5] @@ -739,17 +739,17 @@ project │ ├── key: () │ └── fd: ()-->(1,4) └── projections - └── k:1 [as=min:5, outer=(1)] + └── k:1 [as=min:6, outer=(1)] # Basic max case. opt expect=ReplaceMaxWithLimit SELECT max(w) FROM kuvw WHERE v = 5 GROUP BY v ---- project - ├── columns: max:5 + ├── columns: max:6 ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(5) + ├── fd: ()-->(6) ├── scan kuvw@vw,rev │ ├── columns: v:3!null w:4 │ ├── constraint: /3/4/1: [/5 - /5] @@ -757,23 +757,23 @@ project │ ├── key: () │ └── fd: ()-->(3,4) └── projections - └── w:4 [as=max:5, outer=(4)] + └── w:4 [as=max:6, outer=(4)] # Add const_agg function, as well as min function. opt expect=ReplaceMinWithLimit SELECT v + 1, min(w), v FROM kuvw WHERE v = 5 AND w IS NOT NULL GROUP BY v ---- project - ├── columns: "?column?":6!null min:5!null v:3!null + ├── columns: "?column?":7!null min:6!null v:3!null ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(3,5,6) + ├── fd: ()-->(3,6,7) ├── project - │ ├── columns: min:5!null v:3!null + │ ├── columns: min:6!null v:3!null │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(3,5) + │ ├── fd: ()-->(3,6) │ ├── scan kuvw@vw │ │ ├── columns: v:3!null w:4!null │ │ ├── constraint: /3/4/1: (/5/NULL - /5] @@ -781,25 +781,25 @@ project │ │ ├── key: () │ │ └── fd: ()-->(3,4) │ └── projections - │ └── w:4 [as=min:5, outer=(4)] + │ └── w:4 [as=min:6, outer=(4)] └── projections - └── v:3 + 1 [as="?column?":6, outer=(3), immutable] + └── v:3 + 1 [as="?column?":7, outer=(3), immutable] # Add const_agg function, as well as max function. opt expect=ReplaceMaxWithLimit SELECT v + 1, max(w), v FROM kuvw WHERE v = 5 GROUP BY v ---- project - ├── columns: "?column?":6!null max:5 v:3!null + ├── columns: "?column?":7!null max:6 v:3!null ├── cardinality: [0 - 1] ├── immutable ├── key: () - ├── fd: ()-->(3,5,6) + ├── fd: ()-->(3,6,7) ├── project - │ ├── columns: max:5 v:3!null + │ ├── columns: max:6 v:3!null │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(3,5) + │ ├── fd: ()-->(3,6) │ ├── scan kuvw@vw,rev │ │ ├── columns: v:3!null w:4 │ │ ├── constraint: /3/4/1: [/5 - /5] @@ -807,19 +807,19 @@ project │ │ ├── key: () │ │ └── fd: ()-->(3,4) │ └── projections - │ └── w:4 [as=max:5, outer=(4)] + │ └── w:4 [as=max:6, outer=(4)] └── projections - └── v:3 + 1 [as="?column?":6, outer=(3), immutable] + └── v:3 + 1 [as="?column?":7, outer=(3), immutable] # Use multiple grouping columns with min function. opt expect=ReplaceMinWithLimit SELECT min(k) FROM kuvw WHERE v = 5 AND w = 10 GROUP BY v, w ---- project - ├── columns: min:5!null + ├── columns: min:6!null ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(5) + ├── fd: ()-->(6) ├── scan kuvw@vw │ ├── columns: k:1!null v:3!null w:4!null │ ├── constraint: /3/4/1: [/5/10 - /5/10] @@ -827,17 +827,17 @@ project │ ├── key: () │ └── fd: ()-->(1,3,4) └── projections - └── k:1 [as=min:5, outer=(1)] + └── k:1 [as=min:6, outer=(1)] # Use multiple grouping columns with max function. opt expect=ReplaceMaxWithLimit SELECT max(k) FROM kuvw WHERE v = 5 AND w = 10 GROUP BY v, w ---- project - ├── columns: max:5!null + ├── columns: max:6!null ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(5) + ├── fd: ()-->(6) ├── scan kuvw@vw,rev │ ├── columns: k:1!null v:3!null w:4!null │ ├── constraint: /3/4/1: [/5/10 - /5/10] @@ -845,17 +845,17 @@ project │ ├── key: () │ └── fd: ()-->(1,3,4) └── projections - └── k:1 [as=max:5, outer=(1)] + └── k:1 [as=max:6, outer=(1)] # Use multiple grouping columns with min function, and project them. opt expect=ReplaceMinWithLimit SELECT v, min(k), w FROM kuvw WHERE v = 5 AND w = 10 GROUP BY v, w ---- project - ├── columns: v:3!null min:5!null w:4!null + ├── columns: v:3!null min:6!null w:4!null ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(3-5) + ├── fd: ()-->(3,4,6) ├── scan kuvw@vw │ ├── columns: k:1!null v:3!null w:4!null │ ├── constraint: /3/4/1: [/5/10 - /5/10] @@ -863,17 +863,17 @@ project │ ├── key: () │ └── fd: ()-->(1,3,4) └── projections - └── k:1 [as=min:5, outer=(1)] + └── k:1 [as=min:6, outer=(1)] # Use multiple grouping columns with max function, and project them. opt expect=ReplaceMaxWithLimit SELECT v, max(k), w FROM kuvw WHERE v = 5 AND w = 10 GROUP BY v, w ---- project - ├── columns: v:3!null max:5!null w:4!null + ├── columns: v:3!null max:6!null w:4!null ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(3-5) + ├── fd: ()-->(3,4,6) ├── scan kuvw@vw,rev │ ├── columns: k:1!null v:3!null w:4!null │ ├── constraint: /3/4/1: [/5/10 - /5/10] @@ -881,17 +881,17 @@ project │ ├── key: () │ └── fd: ()-->(1,3,4) └── projections - └── k:1 [as=max:5, outer=(1)] + └── k:1 [as=max:6, outer=(1)] # Multiple grouping columns, but different min column; use different index. opt expect=ReplaceMinWithLimit SELECT min(u) FROM kuvw WHERE v = 5 AND w = 10 AND u > 0 GROUP BY v, w ---- project - ├── columns: min:5!null + ├── columns: min:6!null ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(5) + ├── fd: ()-->(6) ├── scan kuvw@wvu │ ├── columns: u:2!null v:3!null w:4!null │ ├── constraint: /4/3/2/1: [/10/5/1 - /10/5] @@ -899,17 +899,17 @@ project │ ├── key: () │ └── fd: ()-->(2-4) └── projections - └── u:2 [as=min:5, outer=(2)] + └── u:2 [as=min:6, outer=(2)] # Multiple grouping columns, but different max column; use different index. opt expect=ReplaceMaxWithLimit SELECT max(u) FROM kuvw WHERE v = 5 AND w = 10 GROUP BY v, w ---- project - ├── columns: max:5 + ├── columns: max:6 ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(5) + ├── fd: ()-->(6) ├── scan kuvw@wvu,rev │ ├── columns: u:2 v:3!null w:4!null │ ├── constraint: /4/3/2/1: [/10/5 - /10/5] @@ -917,20 +917,20 @@ project │ ├── key: () │ └── fd: ()-->(2-4) └── projections - └── u:2 [as=max:5, outer=(2)] + └── u:2 [as=max:6, outer=(2)] # One of grouping columns is not constant, with min function. opt expect-not=ReplaceMinWithLimit SELECT min(k) FROM kuvw WHERE v = 5 GROUP BY v, w ---- project - ├── columns: min:5!null + ├── columns: min:6!null └── group-by - ├── columns: w:4 min:5!null + ├── columns: w:4 min:6!null ├── grouping columns: w:4 ├── internal-ordering: +4 opt(3) ├── key: (4) - ├── fd: (4)-->(5) + ├── fd: (4)-->(6) ├── scan kuvw@vw │ ├── columns: k:1!null v:3!null w:4 │ ├── constraint: /3/4/1: [/5 - /5] @@ -938,7 +938,7 @@ project │ ├── fd: ()-->(3), (1)-->(4) │ └── ordering: +4 opt(3) [actual: +4] └── aggregations - └── min [as=min:5, outer=(1)] + └── min [as=min:6, outer=(1)] └── k:1 # One of grouping columns is not constant, with max function. @@ -946,13 +946,13 @@ opt expect-not=ReplaceMaxWithLimit SELECT max(k) FROM kuvw WHERE v = 5 GROUP BY v, w ---- project - ├── columns: max:5!null + ├── columns: max:6!null └── group-by - ├── columns: w:4 max:5!null + ├── columns: w:4 max:6!null ├── grouping columns: w:4 ├── internal-ordering: +4 opt(3) ├── key: (4) - ├── fd: (4)-->(5) + ├── fd: (4)-->(6) ├── scan kuvw@vw │ ├── columns: k:1!null v:3!null w:4 │ ├── constraint: /3/4/1: [/5 - /5] @@ -960,7 +960,7 @@ project │ ├── fd: ()-->(3), (1)-->(4) │ └── ordering: +4 opt(3) [actual: +4] └── aggregations - └── max [as=max:5, outer=(1)] + └── max [as=max:6, outer=(1)] └── k:1 # We expect ReplaceMinWithLimit not to be preferred here. @@ -970,13 +970,13 @@ opt expect-not=ReplaceMinWithLimit SELECT min(w) FROM kuvw WHERE v > 5 AND w IS NOT NULL GROUP BY v ---- project - ├── columns: min:5!null + ├── columns: min:6!null └── group-by - ├── columns: v:3!null min:5!null + ├── columns: v:3!null min:6!null ├── grouping columns: v:3!null ├── internal-ordering: +3 ├── key: (3) - ├── fd: (3)-->(5) + ├── fd: (3)-->(6) ├── select │ ├── columns: v:3!null w:4!null │ ├── ordering: +3 @@ -987,7 +987,7 @@ project │ └── filters │ └── w:4 IS NOT NULL [outer=(4), constraints=(/4: (/NULL - ]; tight)] └── aggregations - └── min [as=min:5, outer=(4)] + └── min [as=min:6, outer=(4)] └── w:4 # We expect ReplaceMaxWithLimit not to be preferred here. @@ -997,19 +997,19 @@ opt expect-not=ReplaceMaxWithLimit SELECT max(w) FROM kuvw WHERE v > 5 GROUP BY v ---- project - ├── columns: max:5 + ├── columns: max:6 └── group-by - ├── columns: v:3!null max:5 + ├── columns: v:3!null max:6 ├── grouping columns: v:3!null ├── internal-ordering: +3 ├── key: (3) - ├── fd: (3)-->(5) + ├── fd: (3)-->(6) ├── scan kuvw@vw │ ├── columns: v:3!null w:4 │ ├── constraint: /3/4/1: [/6 - ] │ └── ordering: +3 └── aggregations - └── max [as=max:5, outer=(4)] + └── max [as=max:6, outer=(4)] └── w:4 # ReplaceMinWithLimit does not apply on multiple aggregations @@ -1018,19 +1018,19 @@ opt expect-not=ReplaceMinWithLimit SELECT min(w), min(k) FROM kuvw WHERE v = 5 AND w IS NOT NULL GROUP BY v ---- group-by - ├── columns: min:5!null min:6!null + ├── columns: min:6!null min:7!null ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(5,6) + ├── fd: ()-->(6,7) ├── scan kuvw@vw │ ├── columns: k:1!null v:3!null w:4!null │ ├── constraint: /3/4/1: (/5/NULL - /5] │ ├── key: (1) │ └── fd: ()-->(3), (1)-->(4) └── aggregations - ├── min [as=min:5, outer=(4)] + ├── min [as=min:6, outer=(4)] │ └── w:4 - └── min [as=min:6, outer=(1)] + └── min [as=min:7, outer=(1)] └── k:1 # ReplaceMaxWithLimit does not apply on multiple aggregations @@ -1039,19 +1039,19 @@ opt expect-not=ReplaceMaxWithLimit SELECT max(w), max(k) FROM kuvw WHERE v = 5 GROUP BY v ---- group-by - ├── columns: max:5 max:6!null + ├── columns: max:6 max:7!null ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(5,6) + ├── fd: ()-->(6,7) ├── scan kuvw@vw │ ├── columns: k:1!null v:3!null w:4 │ ├── constraint: /3/4/1: [/5 - /5] │ ├── key: (1) │ └── fd: ()-->(3), (1)-->(4) └── aggregations - ├── max [as=max:5, outer=(4)] + ├── max [as=max:6, outer=(4)] │ └── w:4 - └── max [as=max:6, outer=(1)] + └── max [as=max:7, outer=(1)] └── k:1 # ReplaceMinWithLimit does not apply when other aggregates are present. @@ -1059,19 +1059,19 @@ opt expect-not=ReplaceMinWithLimit SELECT min(k), max(k) FROM kuvw WHERE w = 5 GROUP BY w ---- group-by - ├── columns: min:5!null max:6!null + ├── columns: min:6!null max:7!null ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(5,6) + ├── fd: ()-->(6,7) ├── scan kuvw@wvu │ ├── columns: k:1!null w:4!null │ ├── constraint: /4/3/2/1: [/5 - /5] │ ├── key: (1) │ └── fd: ()-->(4) └── aggregations - ├── min [as=min:5, outer=(1)] + ├── min [as=min:6, outer=(1)] │ └── k:1 - └── max [as=max:6, outer=(1)] + └── max [as=max:7, outer=(1)] └── k:1 # ReplaceMaxWithLimit does not apply when other aggregates are present. @@ -1079,18 +1079,18 @@ opt expect-not=ReplaceMaxWithLimit SELECT max(w), count(w) FROM kuvw WHERE v = 5 GROUP BY v ---- group-by - ├── columns: max:5 count:6!null + ├── columns: max:6 count:7!null ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(5,6) + ├── fd: ()-->(6,7) ├── scan kuvw@vw │ ├── columns: v:3!null w:4 │ ├── constraint: /3/4/1: [/5 - /5] │ └── fd: ()-->(3) └── aggregations - ├── max [as=max:5, outer=(4)] + ├── max [as=max:6, outer=(4)] │ └── w:4 - └── count [as=count:6, outer=(4)] + └── count [as=count:7, outer=(4)] └── w:4 # min/max functions are not symmetric because of NULL ordering (NULL values @@ -1100,16 +1100,16 @@ opt expect-not=ReplaceMinWithLimit SELECT min(w) FROM kuvw WHERE v = 5 GROUP BY v ---- group-by - ├── columns: min:5 + ├── columns: min:6 ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(5) + ├── fd: ()-->(6) ├── scan kuvw@vw │ ├── columns: v:3!null w:4 │ ├── constraint: /3/4/1: [/5 - /5] │ └── fd: ()-->(3) └── aggregations - └── min [as=min:5, outer=(4)] + └── min [as=min:6, outer=(4)] └── w:4 # -------------------------------------------------- @@ -1120,9 +1120,9 @@ group-by memo SELECT array_agg(w) FROM (SELECT * FROM kuvw ORDER BY w) GROUP BY u,v ---- -memo (optimized, ~6KB, required=[presentation: array_agg:5]) +memo (optimized, ~7KB, required=[presentation: array_agg:6]) ├── G1: (project G2 G3 array_agg) - │ └── [presentation: array_agg:5] + │ └── [presentation: array_agg:6] │ ├── best: (project G2 G3 array_agg) │ └── cost: 1120.05 ├── G2: (group-by G4 G5 cols=(2,3),ordering=+4 opt(2,3)) (group-by G4 G5 cols=(2,3),ordering=+2,+3,+4) (group-by G4 G5 cols=(2,3),ordering=+4,+3,+2) (group-by G4 G5 cols=(2,3),ordering=+3,+4) @@ -1154,9 +1154,9 @@ memo (optimized, ~6KB, required=[presentation: array_agg:5]) memo SELECT sum(w) FROM kuvw GROUP BY u,v,w ---- -memo (optimized, ~6KB, required=[presentation: sum:5]) +memo (optimized, ~6KB, required=[presentation: sum:6]) ├── G1: (project G2 G3 sum) - │ └── [presentation: sum:5] + │ └── [presentation: sum:6] │ ├── best: (project G2 G3 sum) │ └── cost: 1130.05 ├── G2: (group-by G4 G5 cols=(2-4)) (group-by G4 G5 cols=(2-4),ordering=+2,+3,+4) (group-by G4 G5 cols=(2-4),ordering=+4,+3,+2) (group-by G4 G5 cols=(2-4),ordering=+3,+4) @@ -1185,9 +1185,9 @@ memo (optimized, ~6KB, required=[presentation: sum:5]) memo SELECT sum(w) FROM kuvw GROUP BY v ---- -memo (optimized, ~6KB, required=[presentation: sum:5]) +memo (optimized, ~6KB, required=[presentation: sum:6]) ├── G1: (project G2 G3 sum) - │ └── [presentation: sum:5] + │ └── [presentation: sum:6] │ ├── best: (project G2 G3 sum) │ └── cost: 1082.05 ├── G2: (group-by G4 G5 cols=(3)) (group-by G4 G5 cols=(3),ordering=+3) @@ -1210,9 +1210,9 @@ memo (optimized, ~6KB, required=[presentation: sum:5]) memo SELECT array_agg(w) FROM (SELECT * FROM kuvw ORDER BY u,w) GROUP BY v ---- -memo (optimized, ~6KB, required=[presentation: array_agg:5]) +memo (optimized, ~7KB, required=[presentation: array_agg:6]) ├── G1: (project G2 G3 array_agg) - │ └── [presentation: array_agg:5] + │ └── [presentation: array_agg:6] │ ├── best: (project G2 G3 array_agg) │ └── cost: 1102.05 ├── G2: (group-by G4 G5 cols=(3),ordering=+2,+4 opt(3)) (group-by G4 G5 cols=(3),ordering=+2,+3,+4) @@ -1241,9 +1241,9 @@ memo (optimized, ~6KB, required=[presentation: array_agg:5]) memo SELECT array_agg(k) FROM (SELECT * FROM kuvw WHERE u=v ORDER BY u) GROUP BY w ---- -memo (optimized, ~10KB, required=[presentation: array_agg:5]) +memo (optimized, ~12KB, required=[presentation: array_agg:6]) ├── G1: (project G2 G3 array_agg) - │ └── [presentation: array_agg:5] + │ └── [presentation: array_agg:6] │ ├── best: (project G2 G3 array_agg) │ └── cost: 1079.64 ├── G2: (group-by G4 G5 cols=(4),ordering=+(2|3) opt(4)) (group-by G4 G5 cols=(4),ordering=+(2|3)) (group-by G4 G5 cols=(4),ordering=+4,+(2|3)) (group-by G4 G5 cols=(4),ordering=+(2|3),+4) @@ -1268,35 +1268,35 @@ memo (optimized, ~10KB, required=[presentation: array_agg:5]) │ ├── best: (select G8 G7) │ └── cost: 1079.13 ├── G5: (aggregations G10) - ├── G6: (scan kuvw) (scan kuvw@uvw) (scan kuvw@wvu) (scan kuvw@vw) (scan kuvw@w) + ├── G6: (scan kuvw,cols=(1-4)) (scan kuvw@uvw,cols=(1-4)) (scan kuvw@wvu,cols=(1-4)) (scan kuvw@vw,cols=(1-4)) (scan kuvw@w,cols=(1-4)) │ ├── [ordering: +2 opt(4)] - │ │ ├── best: (scan kuvw@uvw) + │ │ ├── best: (scan kuvw@uvw,cols=(1-4)) │ │ └── cost: 1080.02 │ ├── [ordering: +2,+4] │ │ ├── best: (sort G6="[ordering: +2]") │ │ └── cost: 1166.67 │ ├── [ordering: +2] - │ │ ├── best: (scan kuvw@uvw) + │ │ ├── best: (scan kuvw@uvw,cols=(1-4)) │ │ └── cost: 1080.02 │ ├── [ordering: +4,+2] │ │ ├── best: (sort G6="[ordering: +4]") │ │ └── cost: 1166.67 │ ├── [ordering: +4] - │ │ ├── best: (scan kuvw@wvu) + │ │ ├── best: (scan kuvw@wvu,cols=(1-4)) │ │ └── cost: 1080.02 │ └── [] - │ ├── best: (scan kuvw) + │ ├── best: (scan kuvw,cols=(1-4)) │ └── cost: 1080.02 ├── G7: (filters G11) - ├── G8: (scan kuvw@uvw,constrained) + ├── G8: (scan kuvw@uvw,cols=(1-4),constrained) │ ├── [ordering: +2 opt(4)] - │ │ ├── best: (scan kuvw@uvw,constrained) + │ │ ├── best: (scan kuvw@uvw,cols=(1-4),constrained) │ │ └── cost: 1069.21 │ ├── [ordering: +2,+4] │ │ ├── best: (sort G8="[ordering: +2]") │ │ └── cost: 1154.71 │ ├── [ordering: +2] - │ │ ├── best: (scan kuvw@uvw,constrained) + │ │ ├── best: (scan kuvw@uvw,cols=(1-4),constrained) │ │ └── cost: 1069.21 │ ├── [ordering: +4,+2] │ │ ├── best: (sort G8) @@ -1305,9 +1305,9 @@ memo (optimized, ~10KB, required=[presentation: array_agg:5]) │ │ ├── best: (sort G8) │ │ └── cost: 1286.06 │ └── [] - │ ├── best: (scan kuvw@uvw,constrained) + │ ├── best: (scan kuvw@uvw,cols=(1-4),constrained) │ └── cost: 1069.21 - ├── G9: (scan kuvw@vw,constrained) + ├── G9: (scan kuvw@vw,cols=(1-4),constrained) │ ├── [ordering: +2 opt(4)] │ │ ├── best: (sort G9) │ │ └── cost: 1286.06 @@ -1324,7 +1324,7 @@ memo (optimized, ~10KB, required=[presentation: array_agg:5]) │ │ ├── best: (sort G9) │ │ └── cost: 1286.06 │ └── [] - │ ├── best: (scan kuvw@vw,constrained) + │ ├── best: (scan kuvw@vw,cols=(1-4),constrained) │ └── cost: 1069.21 ├── G10: (array-agg G12) ├── G11: (eq G13 G14) @@ -1335,9 +1335,9 @@ memo (optimized, ~10KB, required=[presentation: array_agg:5]) memo SELECT sum(k) FROM (SELECT * FROM kuvw WHERE u=v) GROUP BY u,w ---- -memo (optimized, ~10KB, required=[presentation: sum:5]) +memo (optimized, ~12KB, required=[presentation: sum:6]) ├── G1: (project G2 G3 sum) - │ └── [presentation: sum:5] + │ └── [presentation: sum:6] │ ├── best: (project G2 G3 sum) │ └── cost: 1079.70 ├── G2: (group-by G4 G5 cols=(2,4)) (group-by G4 G5 cols=(2,4),ordering=+(2|3)) (group-by G4 G5 cols=(2,4),ordering=+4) @@ -1356,28 +1356,28 @@ memo (optimized, ~10KB, required=[presentation: sum:5]) │ ├── best: (select G8 G7) │ └── cost: 1079.13 ├── G5: (aggregations G10) - ├── G6: (scan kuvw) (scan kuvw@uvw) (scan kuvw@wvu) (scan kuvw@vw) (scan kuvw@w) + ├── G6: (scan kuvw,cols=(1-4)) (scan kuvw@uvw,cols=(1-4)) (scan kuvw@wvu,cols=(1-4)) (scan kuvw@vw,cols=(1-4)) (scan kuvw@w,cols=(1-4)) │ ├── [ordering: +2] - │ │ ├── best: (scan kuvw@uvw) + │ │ ├── best: (scan kuvw@uvw,cols=(1-4)) │ │ └── cost: 1080.02 │ ├── [ordering: +4] - │ │ ├── best: (scan kuvw@wvu) + │ │ ├── best: (scan kuvw@wvu,cols=(1-4)) │ │ └── cost: 1080.02 │ └── [] - │ ├── best: (scan kuvw) + │ ├── best: (scan kuvw,cols=(1-4)) │ └── cost: 1080.02 ├── G7: (filters G11) - ├── G8: (scan kuvw@uvw,constrained) + ├── G8: (scan kuvw@uvw,cols=(1-4),constrained) │ ├── [ordering: +2] - │ │ ├── best: (scan kuvw@uvw,constrained) + │ │ ├── best: (scan kuvw@uvw,cols=(1-4),constrained) │ │ └── cost: 1069.21 │ ├── [ordering: +4] │ │ ├── best: (sort G8) │ │ └── cost: 1286.06 │ └── [] - │ ├── best: (scan kuvw@uvw,constrained) + │ ├── best: (scan kuvw@uvw,cols=(1-4),constrained) │ └── cost: 1069.21 - ├── G9: (scan kuvw@vw,constrained) + ├── G9: (scan kuvw@vw,cols=(1-4),constrained) │ ├── [ordering: +2] │ │ ├── best: (sort G9) │ │ └── cost: 1286.06 @@ -1385,7 +1385,7 @@ memo (optimized, ~10KB, required=[presentation: sum:5]) │ │ ├── best: (sort G9) │ │ └── cost: 1286.06 │ └── [] - │ ├── best: (scan kuvw@vw,constrained) + │ ├── best: (scan kuvw@vw,cols=(1-4),constrained) │ └── cost: 1069.21 ├── G10: (sum G12) ├── G11: (eq G13 G14) @@ -1397,9 +1397,9 @@ memo (optimized, ~10KB, required=[presentation: sum:5]) memo SELECT array_agg(w) FROM (SELECT * FROM kuvw ORDER BY w DESC) GROUP BY u,v ---- -memo (optimized, ~5KB, required=[presentation: array_agg:5]) +memo (optimized, ~7KB, required=[presentation: array_agg:6]) ├── G1: (project G2 G3 array_agg) - │ └── [presentation: array_agg:5] + │ └── [presentation: array_agg:6] │ ├── best: (project G2 G3 array_agg) │ └── cost: 1229.71 ├── G2: (group-by G4 G5 cols=(2,3),ordering=-4 opt(2,3)) @@ -1720,9 +1720,9 @@ memo (optimized, ~4KB, required=[presentation: u:2,v:3,w:4] [ordering: +4]) memo SELECT (SELECT w FROM kuvw WHERE v=1 AND x=u) FROM xyz ORDER BY x+1, x ---- -memo (optimized, ~25KB, required=[presentation: w:8] [ordering: +9,+1]) +memo (optimized, ~25KB, required=[presentation: w:10] [ordering: +11,+1]) ├── G1: (project G2 G3 x) - │ ├── [presentation: w:8] [ordering: +9,+1] + │ ├── [presentation: w:10] [ordering: +11,+1] │ │ ├── best: (sort G1) │ │ └── cost: 1352.04 │ └── [] @@ -1733,12 +1733,12 @@ memo (optimized, ~25KB, required=[presentation: w:8] [ordering: +9,+1]) │ ├── best: (ensure-distinct-on G4="[ordering: +1]" G5 cols=(1),ordering=+1) │ └── cost: 1091.73 ├── G3: (projections G6 G7) - ├── G4: (left-join G8 G9 G10) (right-join G9 G8 G10) (merge-join G8 G9 G11 left-join,+1,+5) (lookup-join G12 G11 kuvw@uvw,keyCols=[1 10],outCols=(1,5-7)) (lookup-join G13 G10 kuvw@vw,keyCols=[11],outCols=(1,5-7)) (merge-join G9 G8 G11 right-join,+5,+1) + ├── G4: (left-join G8 G9 G10) (right-join G9 G8 G10) (merge-join G8 G9 G11 left-join,+1,+6) (lookup-join G12 G11 kuvw@uvw,keyCols=[1 12],outCols=(1,6-8)) (lookup-join G13 G10 kuvw@vw,keyCols=[13],outCols=(1,6-8)) (merge-join G9 G8 G11 right-join,+6,+1) │ ├── [ordering: +1] - │ │ ├── best: (merge-join G8="[ordering: +1]" G9="[ordering: +5 opt(6)]" G11 left-join,+1,+5) + │ │ ├── best: (merge-join G8="[ordering: +1]" G9="[ordering: +6 opt(7)]" G11 left-join,+1,+6) │ │ └── cost: 1061.71 │ └── [] - │ ├── best: (merge-join G8="[ordering: +1]" G9="[ordering: +5 opt(6)]" G11 left-join,+1,+5) + │ ├── best: (merge-join G8="[ordering: +1]" G9="[ordering: +6 opt(7)]" G11 left-join,+1,+6) │ └── cost: 1061.71 ├── G5: (aggregations G14) ├── G6: (variable kuvw.w) @@ -1750,12 +1750,12 @@ memo (optimized, ~25KB, required=[presentation: w:8] [ordering: +9,+1]) │ └── [] │ ├── best: (scan xyz@xy,cols=(1)) │ └── cost: 1030.02 - ├── G9: (select G17 G18) (scan kuvw@vw,cols=(5-7),constrained) - │ ├── [ordering: +5 opt(6)] + ├── G9: (select G17 G18) (scan kuvw@vw,cols=(6-8),constrained) + │ ├── [ordering: +6 opt(7)] │ │ ├── best: (sort G9) │ │ └── cost: 11.58 │ └── [] - │ ├── best: (scan kuvw@vw,cols=(5-7),constrained) + │ ├── best: (scan kuvw@vw,cols=(6-8),constrained) │ └── cost: 10.71 ├── G10: (filters G19) ├── G11: (filters) @@ -1776,12 +1776,12 @@ memo (optimized, ~25KB, required=[presentation: w:8] [ordering: +9,+1]) ├── G14: (const-agg G6) ├── G15: (variable x) ├── G16: (const 1) - ├── G17: (scan kuvw,cols=(5-7)) (scan kuvw@uvw,cols=(5-7)) (scan kuvw@wvu,cols=(5-7)) (scan kuvw@vw,cols=(5-7)) (scan kuvw@w,cols=(5-7)) - │ ├── [ordering: +5 opt(6)] - │ │ ├── best: (scan kuvw@uvw,cols=(5-7)) + ├── G17: (scan kuvw,cols=(6-8)) (scan kuvw@uvw,cols=(6-8)) (scan kuvw@wvu,cols=(6-8)) (scan kuvw@vw,cols=(6-8)) (scan kuvw@w,cols=(6-8)) + │ ├── [ordering: +6 opt(7)] + │ │ ├── best: (scan kuvw@uvw,cols=(6-8)) │ │ └── cost: 1070.02 │ └── [] - │ ├── best: (scan kuvw,cols=(5-7)) + │ ├── best: (scan kuvw,cols=(6-8)) │ └── cost: 1070.02 ├── G18: (filters G21) ├── G19: (eq G15 G22) @@ -1799,54 +1799,54 @@ memo (optimized, ~19KB, required=[]) │ └── [] │ ├── best: (insert G2 G3 xyz) │ └── cost: 2150.51 - ├── G2: (upsert-distinct-on G4 G5 cols=(6)) (upsert-distinct-on G4 G5 cols=(6),ordering=+6 opt(8,9)) + ├── G2: (upsert-distinct-on G4 G5 cols=(7)) (upsert-distinct-on G4 G5 cols=(7),ordering=+7 opt(10,11)) │ └── [] - │ ├── best: (upsert-distinct-on G4="[ordering: +6 opt(8,9)]" G5 cols=(6),ordering=+6 opt(8,9)) + │ ├── best: (upsert-distinct-on G4="[ordering: +7 opt(10,11)]" G5 cols=(7),ordering=+7 opt(10,11)) │ └── cost: 2150.50 ├── G3: (f-k-checks) ├── G4: (select G6 G7) - │ ├── [ordering: +6 opt(8,9)] - │ │ ├── best: (select G6="[ordering: +6 opt(8,9)]" G7) + │ ├── [ordering: +7 opt(10,11)] + │ │ ├── best: (select G6="[ordering: +7 opt(10,11)]" G7) │ │ └── cost: 2150.08 │ └── [] │ ├── best: (select G6 G7) │ └── cost: 2150.08 ├── G5: (aggregations G8 G9) - ├── G6: (left-join G10 G11 G12) (right-join G11 G10 G12) (merge-join G10 G11 G13 left-join,+6,+9) (lookup-join G10 G13 xyz,keyCols=[6],outCols=(6-9)) (lookup-join G10 G13 xyz@xy,keyCols=[6],outCols=(6-9)) (merge-join G11 G10 G13 right-join,+9,+6) - │ ├── [ordering: +6 opt(8,9)] - │ │ ├── best: (merge-join G10="[ordering: +6 opt(8)]" G11="[ordering: +9]" G13 left-join,+6,+9) + ├── G6: (left-join G10 G11 G12) (right-join G11 G10 G12) (merge-join G10 G11 G13 left-join,+7,+11) (lookup-join G10 G13 xyz,keyCols=[7],outCols=(7,8,10,11)) (lookup-join G10 G13 xyz@xy,keyCols=[7],outCols=(7,8,10,11)) (merge-join G11 G10 G13 right-join,+11,+7) + │ ├── [ordering: +7 opt(10,11)] + │ │ ├── best: (merge-join G10="[ordering: +7 opt(10)]" G11="[ordering: +11]" G13 left-join,+7,+11) │ │ └── cost: 2140.06 │ └── [] - │ ├── best: (merge-join G10="[ordering: +6 opt(8)]" G11="[ordering: +9]" G13 left-join,+6,+9) + │ ├── best: (merge-join G10="[ordering: +7 opt(10)]" G11="[ordering: +11]" G13 left-join,+7,+11) │ └── cost: 2140.06 ├── G7: (filters G14) ├── G8: (first-agg G15) ├── G9: (first-agg G16) ├── G10: (project G17 G18 v w) - │ ├── [ordering: +6 opt(8)] - │ │ ├── best: (project G17="[ordering: +6]" G18 v w) + │ ├── [ordering: +7 opt(10)] + │ │ ├── best: (project G17="[ordering: +7]" G18 v w) │ │ └── cost: 1080.03 │ └── [] │ ├── best: (project G17 G18 v w) │ └── cost: 1080.03 - ├── G11: (scan xyz,cols=(9)) (scan xyz@xy,cols=(9)) (scan xyz@zyx,cols=(9)) (scan xyz@yy,cols=(9)) - │ ├── [ordering: +9] - │ │ ├── best: (scan xyz@xy,cols=(9)) + ├── G11: (scan xyz,cols=(11)) (scan xyz@xy,cols=(11)) (scan xyz@zyx,cols=(11)) (scan xyz@yy,cols=(11)) + │ ├── [ordering: +11] + │ │ ├── best: (scan xyz@xy,cols=(11)) │ │ └── cost: 1030.02 │ └── [] - │ ├── best: (scan xyz@xy,cols=(9)) + │ ├── best: (scan xyz@xy,cols=(11)) │ └── cost: 1030.02 ├── G12: (filters G19) ├── G13: (filters) ├── G14: (is G20 G21) ├── G15: (variable w) ├── G16: (variable "?column?") - ├── G17: (scan kuvw,cols=(6,7)) (scan kuvw@uvw,cols=(6,7)) (scan kuvw@wvu,cols=(6,7)) (scan kuvw@vw,cols=(6,7)) (scan kuvw@w,cols=(6,7)) - │ ├── [ordering: +6] - │ │ ├── best: (scan kuvw@vw,cols=(6,7)) + ├── G17: (scan kuvw,cols=(7,8)) (scan kuvw@uvw,cols=(7,8)) (scan kuvw@wvu,cols=(7,8)) (scan kuvw@vw,cols=(7,8)) (scan kuvw@w,cols=(7,8)) + │ ├── [ordering: +7] + │ │ ├── best: (scan kuvw@vw,cols=(7,8)) │ │ └── cost: 1060.02 │ └── [] - │ ├── best: (scan kuvw,cols=(6,7)) + │ ├── best: (scan kuvw,cols=(7,8)) │ └── cost: 1060.02 ├── G18: (projections G22) ├── G19: (eq G23 G20) @@ -1859,7 +1859,7 @@ memo (optimized, ~19KB, required=[]) memo INSERT INTO xyz SELECT v, w, 1.0 FROM kuvw ON CONFLICT (x) DO UPDATE SET z=2.0 ---- -memo (optimized, ~20KB, required=[]) +memo (optimized, ~22KB, required=[]) ├── G1: (upsert G2 G3 xyz) │ └── [] │ ├── best: (upsert G2 G3 xyz) @@ -1869,35 +1869,35 @@ memo (optimized, ~20KB, required=[]) │ ├── best: (project G4 G5 v w ?column? x y z) │ └── cost: 2230.09 ├── G3: (f-k-checks) - ├── G4: (left-join G6 G7 G8) (right-join G7 G6 G8) (lookup-join G6 G9 xyz,keyCols=[6],outCols=(6-11)) (lookup-join G10 G9 xyz,keyCols=[9],outCols=(6-11)) (merge-join G7 G6 G9 right-join,+9,+6) + ├── G4: (left-join G6 G7 G8) (right-join G7 G6 G8) (lookup-join G6 G9 xyz,keyCols=[7],outCols=(7,8,10-13)) (lookup-join G10 G9 xyz,keyCols=[11],outCols=(7,8,10-13)) (merge-join G7 G6 G9 right-join,+11,+7) │ └── [] - │ ├── best: (merge-join G7="[ordering: +9]" G6="[ordering: +6 opt(8)]" G9 right-join,+9,+6) + │ ├── best: (merge-join G7="[ordering: +11]" G6="[ordering: +7 opt(10)]" G9 right-join,+11,+7) │ └── cost: 2210.08 ├── G5: (projections G11) - ├── G6: (ensure-upsert-distinct-on G12 G13 cols=(6)) (ensure-upsert-distinct-on G12 G13 cols=(6),ordering=+6 opt(8)) - │ ├── [ordering: +6 opt(8)] - │ │ ├── best: (ensure-upsert-distinct-on G12="[ordering: +6 opt(8)]" G13 cols=(6)) + ├── G6: (ensure-upsert-distinct-on G12 G13 cols=(7)) (ensure-upsert-distinct-on G12 G13 cols=(7),ordering=+7 opt(10)) + │ ├── [ordering: +7 opt(10)] + │ │ ├── best: (ensure-upsert-distinct-on G12="[ordering: +7 opt(10)]" G13 cols=(7)) │ │ └── cost: 1120.05 │ └── [] - │ ├── best: (ensure-upsert-distinct-on G12="[ordering: +6 opt(8)]" G13 cols=(6),ordering=+6 opt(8)) + │ ├── best: (ensure-upsert-distinct-on G12="[ordering: +7 opt(10)]" G13 cols=(7),ordering=+7 opt(10)) │ └── cost: 1120.05 - ├── G7: (scan xyz) (scan xyz@zyx) - │ ├── [ordering: +9] - │ │ ├── best: (scan xyz) + ├── G7: (scan xyz,cols=(11-13)) (scan xyz@zyx,cols=(11-13)) + │ ├── [ordering: +11] + │ │ ├── best: (scan xyz,cols=(11-13)) │ │ └── cost: 1060.02 │ └── [] - │ ├── best: (scan xyz) + │ ├── best: (scan xyz,cols=(11-13)) │ └── cost: 1060.02 ├── G8: (filters G14) ├── G9: (filters) - ├── G10: (lookup-join G6 G9 xyz@xy,keyCols=[6],outCols=(6-10)) + ├── G10: (lookup-join G6 G9 xyz@xy,keyCols=[7],outCols=(7,8,10-12)) │ └── [] - │ ├── best: (lookup-join G6 G9 xyz@xy,keyCols=[6],outCols=(6-10)) + │ ├── best: (lookup-join G6 G9 xyz@xy,keyCols=[7],outCols=(7,8,10-12)) │ └── cost: 7170.06 ├── G11: (case G15 G16 G17) ├── G12: (project G18 G19 v w) - │ ├── [ordering: +6 opt(8)] - │ │ ├── best: (project G18="[ordering: +6]" G19 v w) + │ ├── [ordering: +7 opt(10)] + │ │ ├── best: (project G18="[ordering: +7]" G19 v w) │ │ └── cost: 1080.03 │ └── [] │ ├── best: (project G18 G19 v w) @@ -1907,12 +1907,12 @@ memo (optimized, ~20KB, required=[]) ├── G15: (true) ├── G16: (scalar-list G24) ├── G17: (const 2.0) - ├── G18: (scan kuvw,cols=(6,7)) (scan kuvw@uvw,cols=(6,7)) (scan kuvw@wvu,cols=(6,7)) (scan kuvw@vw,cols=(6,7)) (scan kuvw@w,cols=(6,7)) - │ ├── [ordering: +6] - │ │ ├── best: (scan kuvw@vw,cols=(6,7)) + ├── G18: (scan kuvw,cols=(7,8)) (scan kuvw@uvw,cols=(7,8)) (scan kuvw@wvu,cols=(7,8)) (scan kuvw@vw,cols=(7,8)) (scan kuvw@w,cols=(7,8)) + │ ├── [ordering: +7] + │ │ ├── best: (scan kuvw@vw,cols=(7,8)) │ │ └── cost: 1060.02 │ └── [] - │ ├── best: (scan kuvw,cols=(6,7)) + │ ├── best: (scan kuvw,cols=(7,8)) │ └── cost: 1060.02 ├── G19: (projections G25) ├── G20: (first-agg G26) diff --git a/pkg/sql/opt/xform/testdata/rules/join b/pkg/sql/opt/xform/testdata/rules/join index ed8c867350ef..2910d7b01ee8 100644 --- a/pkg/sql/opt/xform/testdata/rules/join +++ b/pkg/sql/opt/xform/testdata/rules/join @@ -109,36 +109,36 @@ opt expect=ReorderJoins SELECT * FROM abc, stu, xyz WHERE abc.a=stu.s AND stu.s=xyz.x ---- inner-join (merge) - ├── columns: a:1!null b:2 c:3 s:5!null t:6!null u:7!null x:8!null y:9 z:10 - ├── left ordering: +5 + ├── columns: a:1!null b:2 c:3 s:6!null t:7!null u:8!null x:10!null y:11 z:12 + ├── left ordering: +6 ├── right ordering: +1 - ├── fd: (5)==(1,8), (8)==(1,5), (1)==(5,8) + ├── fd: (6)==(1,10), (10)==(1,6), (1)==(6,10) ├── scan stu - │ ├── columns: s:5!null t:6!null u:7!null - │ ├── key: (5-7) - │ └── ordering: +5 + │ ├── columns: s:6!null t:7!null u:8!null + │ ├── key: (6-8) + │ └── ordering: +6 ├── inner-join (merge) - │ ├── columns: a:1!null b:2 c:3 x:8!null y:9 z:10 + │ ├── columns: a:1!null b:2 c:3 x:10!null y:11 z:12 │ ├── left ordering: +1 - │ ├── right ordering: +8 - │ ├── fd: (1)==(8), (8)==(1) - │ ├── ordering: +(1|8) [actual: +1] + │ ├── right ordering: +10 + │ ├── fd: (1)==(10), (10)==(1) + │ ├── ordering: +(1|10) [actual: +1] │ ├── scan abc@ab │ │ ├── columns: a:1 b:2 c:3 │ │ └── ordering: +1 │ ├── scan xyz@xy - │ │ ├── columns: x:8 y:9 z:10 - │ │ └── ordering: +8 + │ │ ├── columns: x:10 y:11 z:12 + │ │ └── ordering: +10 │ └── filters (true) └── filters (true) memo expect=ReorderJoins SELECT * FROM abc, stu, xyz WHERE abc.a=stu.s AND stu.s=xyz.x ---- -memo (optimized, ~34KB, required=[presentation: a:1,b:2,c:3,s:5,t:6,u:7,x:8,y:9,z:10]) - ├── G1: (inner-join G2 G3 G4) (inner-join G5 G6 G4) (inner-join G7 G8 G9) (inner-join G3 G2 G4) (merge-join G2 G3 G10 inner-join,+1,+5) (inner-join G6 G5 G4) (merge-join G5 G6 G10 inner-join,+5,+1) (inner-join G8 G7 G9) (merge-join G7 G8 G10 inner-join,+5,+8) (lookup-join G7 G10 xyz@xy,keyCols=[5],outCols=(1-3,5-10)) (merge-join G3 G2 G10 inner-join,+5,+1) (lookup-join G3 G10 abc@ab,keyCols=[5],outCols=(1-3,5-10)) (merge-join G6 G5 G10 inner-join,+1,+5) (lookup-join G6 G10 stu,keyCols=[1],outCols=(1-3,5-10)) (merge-join G8 G7 G10 inner-join,+8,+5) - │ └── [presentation: a:1,b:2,c:3,s:5,t:6,u:7,x:8,y:9,z:10] - │ ├── best: (merge-join G5="[ordering: +5]" G6="[ordering: +(1|8)]" G10 inner-join,+5,+1) +memo (optimized, ~36KB, required=[presentation: a:1,b:2,c:3,s:6,t:7,u:8,x:10,y:11,z:12]) + ├── G1: (inner-join G2 G3 G4) (inner-join G5 G6 G4) (inner-join G7 G8 G9) (inner-join G3 G2 G4) (merge-join G2 G3 G10 inner-join,+1,+6) (inner-join G6 G5 G4) (merge-join G5 G6 G10 inner-join,+6,+1) (inner-join G8 G7 G9) (merge-join G7 G8 G10 inner-join,+6,+10) (lookup-join G7 G10 xyz@xy,keyCols=[6],outCols=(1-3,6-8,10-12)) (merge-join G3 G2 G10 inner-join,+6,+1) (lookup-join G3 G10 abc@ab,keyCols=[6],outCols=(1-3,6-8,10-12)) (merge-join G6 G5 G10 inner-join,+1,+6) (lookup-join G6 G10 stu,keyCols=[1],outCols=(1-3,6-8,10-12)) (merge-join G8 G7 G10 inner-join,+10,+6) + │ └── [presentation: a:1,b:2,c:3,s:6,t:7,u:8,x:10,y:11,z:12] + │ ├── best: (merge-join G5="[ordering: +6]" G6="[ordering: +(1|10)]" G10 inner-join,+6,+1) │ └── cost: 12980.08 ├── G2: (scan abc,cols=(1-3)) (scan abc@ab,cols=(1-3)) (scan abc@bc,cols=(1-3)) │ ├── [ordering: +1] @@ -147,41 +147,41 @@ memo (optimized, ~34KB, required=[presentation: a:1,b:2,c:3,s:5,t:6,u:7,x:8,y:9, │ └── [] │ ├── best: (scan abc,cols=(1-3)) │ └── cost: 1070.02 - ├── G3: (inner-join G5 G8 G9) (inner-join G8 G5 G9) (merge-join G5 G8 G10 inner-join,+5,+8) (lookup-join G5 G10 xyz@xy,keyCols=[5],outCols=(5-10)) (merge-join G8 G5 G10 inner-join,+8,+5) (lookup-join G8 G10 stu,keyCols=[8],outCols=(5-10)) - │ ├── [ordering: +(5|8)] - │ │ ├── best: (merge-join G5="[ordering: +5]" G8="[ordering: +8]" G10 inner-join,+5,+8) + ├── G3: (inner-join G5 G8 G9) (inner-join G8 G5 G9) (merge-join G5 G8 G10 inner-join,+6,+10) (lookup-join G5 G10 xyz@xy,keyCols=[6],outCols=(6-8,10-12)) (merge-join G8 G5 G10 inner-join,+10,+6) (lookup-join G8 G10 stu,keyCols=[10],outCols=(6-8,10-12)) + │ ├── [ordering: +(6|10)] + │ │ ├── best: (merge-join G5="[ordering: +6]" G8="[ordering: +10]" G10 inner-join,+6,+10) │ │ └── cost: 11880.05 │ └── [] - │ ├── best: (merge-join G5="[ordering: +5]" G8="[ordering: +8]" G10 inner-join,+5,+8) + │ ├── best: (merge-join G5="[ordering: +6]" G8="[ordering: +10]" G10 inner-join,+6,+10) │ └── cost: 11880.05 ├── G4: (filters G11) - ├── G5: (scan stu) (scan stu@uts) - │ ├── [ordering: +5] - │ │ ├── best: (scan stu) + ├── G5: (scan stu,cols=(6-8)) (scan stu@uts,cols=(6-8)) + │ ├── [ordering: +6] + │ │ ├── best: (scan stu,cols=(6-8)) │ │ └── cost: 10600.02 │ └── [] - │ ├── best: (scan stu) + │ ├── best: (scan stu,cols=(6-8)) │ └── cost: 10600.02 - ├── G6: (inner-join G2 G8 G12) (inner-join G8 G2 G12) (merge-join G2 G8 G10 inner-join,+1,+8) (lookup-join G2 G10 xyz@xy,keyCols=[1],outCols=(1-3,8-10)) (merge-join G8 G2 G10 inner-join,+8,+1) (lookup-join G8 G10 abc@ab,keyCols=[8],outCols=(1-3,8-10)) - │ ├── [ordering: +(1|8)] - │ │ ├── best: (merge-join G2="[ordering: +1]" G8="[ordering: +8]" G10 inner-join,+1,+8) + ├── G6: (inner-join G2 G8 G12) (inner-join G8 G2 G12) (merge-join G2 G8 G10 inner-join,+1,+10) (lookup-join G2 G10 xyz@xy,keyCols=[1],outCols=(1-3,10-12)) (merge-join G8 G2 G10 inner-join,+10,+1) (lookup-join G8 G10 abc@ab,keyCols=[10],outCols=(1-3,10-12)) + │ ├── [ordering: +(1|10)] + │ │ ├── best: (merge-join G2="[ordering: +1]" G8="[ordering: +10]" G10 inner-join,+1,+10) │ │ └── cost: 2170.05 │ └── [] - │ ├── best: (merge-join G2="[ordering: +1]" G8="[ordering: +8]" G10 inner-join,+1,+8) + │ ├── best: (merge-join G2="[ordering: +1]" G8="[ordering: +10]" G10 inner-join,+1,+10) │ └── cost: 2170.05 - ├── G7: (inner-join G2 G5 G4) (inner-join G5 G2 G4) (merge-join G2 G5 G10 inner-join,+1,+5) (lookup-join G2 G10 stu,keyCols=[1],outCols=(1-3,5-7)) (merge-join G5 G2 G10 inner-join,+5,+1) (lookup-join G5 G10 abc@ab,keyCols=[5],outCols=(1-3,5-7)) - │ ├── [ordering: +(1|5)] - │ │ ├── best: (merge-join G2="[ordering: +1]" G5="[ordering: +5]" G10 inner-join,+1,+5) + ├── G7: (inner-join G2 G5 G4) (inner-join G5 G2 G4) (merge-join G2 G5 G10 inner-join,+1,+6) (lookup-join G2 G10 stu,keyCols=[1],outCols=(1-3,6-8)) (merge-join G5 G2 G10 inner-join,+6,+1) (lookup-join G5 G10 abc@ab,keyCols=[6],outCols=(1-3,6-8)) + │ ├── [ordering: +(1|6)] + │ │ ├── best: (merge-join G2="[ordering: +1]" G5="[ordering: +6]" G10 inner-join,+1,+6) │ │ └── cost: 11880.05 │ └── [] - │ ├── best: (merge-join G2="[ordering: +1]" G5="[ordering: +5]" G10 inner-join,+1,+5) + │ ├── best: (merge-join G2="[ordering: +1]" G5="[ordering: +6]" G10 inner-join,+1,+6) │ └── cost: 11880.05 - ├── G8: (scan xyz,cols=(8-10)) (scan xyz@xy,cols=(8-10)) (scan xyz@yz,cols=(8-10)) - │ ├── [ordering: +8] - │ │ ├── best: (scan xyz@xy,cols=(8-10)) + ├── G8: (scan xyz,cols=(10-12)) (scan xyz@xy,cols=(10-12)) (scan xyz@yz,cols=(10-12)) + │ ├── [ordering: +10] + │ │ ├── best: (scan xyz@xy,cols=(10-12)) │ │ └── cost: 1070.02 │ └── [] - │ ├── best: (scan xyz,cols=(8-10)) + │ ├── best: (scan xyz,cols=(10-12)) │ └── cost: 1070.02 ├── G9: (filters G13) ├── G10: (filters) @@ -226,34 +226,34 @@ JOIN parent1 USING (pid1) ORDER BY pid1 ---- project - ├── columns: pid1:1!null cid1:2!null gcid1:3!null gca1:4 ca1:7 pa1:9 + ├── columns: pid1:1!null cid1:2!null gcid1:3!null gca1:4 ca1:8 pa1:11 ├── key: (1-3) - ├── fd: (1-3)-->(4), (1,2)-->(7), (1)-->(9) + ├── fd: (1-3)-->(4), (1,2)-->(8), (1)-->(11) ├── ordering: +1 └── inner-join (lookup parent1) - ├── columns: grandchild1.pid1:1!null grandchild1.cid1:2!null gcid1:3!null gca1:4 child1.pid1:5!null child1.cid1:6!null ca1:7 parent1.pid1:8!null pa1:9 - ├── key columns: [1] = [8] + ├── columns: grandchild1.pid1:1!null grandchild1.cid1:2!null gcid1:3!null gca1:4 child1.pid1:6!null child1.cid1:7!null ca1:8 parent1.pid1:10!null pa1:11 + ├── key columns: [1] = [10] ├── lookup columns are key - ├── key: (3,6,8) - ├── fd: (1-3)-->(4), (5,6)-->(7), (1)==(5,8), (5)==(1,8), (2)==(6), (6)==(2), (8)-->(9), (8)==(1,5) - ├── ordering: +(1|5|8) [actual: +1] + ├── key: (3,7,10) + ├── fd: (1-3)-->(4), (6,7)-->(8), (1)==(6,10), (6)==(1,10), (2)==(7), (7)==(2), (10)-->(11), (10)==(1,6) + ├── ordering: +(1|6|10) [actual: +1] ├── inner-join (merge) - │ ├── columns: grandchild1.pid1:1!null grandchild1.cid1:2!null gcid1:3!null gca1:4 child1.pid1:5!null child1.cid1:6!null ca1:7 + │ ├── columns: grandchild1.pid1:1!null grandchild1.cid1:2!null gcid1:3!null gca1:4 child1.pid1:6!null child1.cid1:7!null ca1:8 │ ├── left ordering: +1,+2 - │ ├── right ordering: +5,+6 - │ ├── key: (3,5,6) - │ ├── fd: (1-3)-->(4), (5,6)-->(7), (1)==(5), (5)==(1), (2)==(6), (6)==(2) - │ ├── ordering: +(1|5) [actual: +1] + │ ├── right ordering: +6,+7 + │ ├── key: (3,6,7) + │ ├── fd: (1-3)-->(4), (6,7)-->(8), (1)==(6), (6)==(1), (2)==(7), (7)==(2) + │ ├── ordering: +(1|6) [actual: +1] │ ├── scan grandchild1 │ │ ├── columns: grandchild1.pid1:1!null grandchild1.cid1:2!null gcid1:3!null gca1:4 │ │ ├── key: (1-3) │ │ ├── fd: (1-3)-->(4) │ │ └── ordering: +1,+2 │ ├── scan child1 - │ │ ├── columns: child1.pid1:5!null child1.cid1:6!null ca1:7 - │ │ ├── key: (5,6) - │ │ ├── fd: (5,6)-->(7) - │ │ └── ordering: +5,+6 + │ │ ├── columns: child1.pid1:6!null child1.cid1:7!null ca1:8 + │ │ ├── key: (6,7) + │ │ ├── fd: (6,7)-->(8) + │ │ └── ordering: +6,+7 │ └── filters (true) └── filters (true) @@ -263,77 +263,77 @@ JOIN child1 USING (pid1, cid1) JOIN parent1 USING (pid1) ORDER BY pid1 ---- -memo (optimized, ~26KB, required=[presentation: pid1:1,cid1:2,gcid1:3,gca1:4,ca1:7,pa1:9] [ordering: +1]) +memo (optimized, ~34KB, required=[presentation: pid1:1,cid1:2,gcid1:3,gca1:4,ca1:8,pa1:11] [ordering: +1]) ├── G1: (project G2 G3 pid1 cid1 gcid1 gca1 ca1 pa1) - │ ├── [presentation: pid1:1,cid1:2,gcid1:3,gca1:4,ca1:7,pa1:9] [ordering: +1] - │ │ ├── best: (project G2="[ordering: +(1|5|8)]" G3 pid1 cid1 gcid1 gca1 ca1 pa1) + │ ├── [presentation: pid1:1,cid1:2,gcid1:3,gca1:4,ca1:8,pa1:11] [ordering: +1] + │ │ ├── best: (project G2="[ordering: +(1|6|10)]" G3 pid1 cid1 gcid1 gca1 ca1 pa1) │ │ └── cost: 2767.07 │ └── [] │ ├── best: (project G2 G3 pid1 cid1 gcid1 gca1 ca1 pa1) │ └── cost: 2767.07 - ├── G2: (inner-join G4 G5 G6) (inner-join G7 G8 G9) (inner-join G10 G11 G9) (inner-join G5 G4 G6) (merge-join G4 G5 G12 inner-join,+1,+8) (lookup-join G4 G12 parent1,keyCols=[1],outCols=(1-9)) (inner-join G8 G7 G9) (merge-join G7 G8 G12 inner-join,+1,+2,+5,+6) (inner-join G11 G10 G9) (merge-join G10 G11 G12 inner-join,+5,+6,+1,+2) (merge-join G5 G4 G12 inner-join,+8,+1) (merge-join G8 G7 G12 inner-join,+5,+6,+1,+2) (lookup-join G8 G12 grandchild1,keyCols=[5 6],outCols=(1-9)) (merge-join G11 G10 G12 inner-join,+1,+2,+5,+6) (lookup-join G11 G12 child1,keyCols=[1 2],outCols=(1-9)) - │ ├── [ordering: +(1|5|8)] - │ │ ├── best: (lookup-join G4="[ordering: +(1|5)]" G12 parent1,keyCols=[1],outCols=(1-9)) + ├── G2: (inner-join G4 G5 G6) (inner-join G7 G8 G9) (inner-join G10 G11 G9) (inner-join G5 G4 G6) (merge-join G4 G5 G12 inner-join,+1,+10) (lookup-join G4 G12 parent1,keyCols=[1],outCols=(1-4,6-8,10,11)) (inner-join G8 G7 G9) (merge-join G7 G8 G12 inner-join,+1,+2,+6,+7) (inner-join G11 G10 G9) (merge-join G10 G11 G12 inner-join,+6,+7,+1,+2) (merge-join G5 G4 G12 inner-join,+10,+1) (merge-join G8 G7 G12 inner-join,+6,+7,+1,+2) (lookup-join G8 G12 grandchild1,keyCols=[6 7],outCols=(1-4,6-8,10,11)) (merge-join G11 G10 G12 inner-join,+1,+2,+6,+7) (lookup-join G11 G12 child1,keyCols=[1 2],outCols=(1-4,6-8,10,11)) + │ ├── [ordering: +(1|6|10)] + │ │ ├── best: (lookup-join G4="[ordering: +(1|6)]" G12 parent1,keyCols=[1],outCols=(1-4,6-8,10,11)) │ │ └── cost: 2766.06 │ └── [] - │ ├── best: (lookup-join G4 G12 parent1,keyCols=[1],outCols=(1-9)) + │ ├── best: (lookup-join G4 G12 parent1,keyCols=[1],outCols=(1-4,6-8,10,11)) │ └── cost: 2766.06 ├── G3: (projections) - ├── G4: (inner-join G7 G10 G9) (inner-join G10 G7 G9) (merge-join G7 G10 G12 inner-join,+1,+2,+5,+6) (lookup-join G7 G12 child1,keyCols=[1 2],outCols=(1-7)) (merge-join G10 G7 G12 inner-join,+5,+6,+1,+2) (lookup-join G10 G12 grandchild1,keyCols=[5 6],outCols=(1-7)) - │ ├── [ordering: +(1|5)] - │ │ ├── best: (merge-join G7="[ordering: +1,+2]" G10="[ordering: +5,+6]" G12 inner-join,+1,+2,+5,+6) + ├── G4: (inner-join G7 G10 G9) (inner-join G10 G7 G9) (merge-join G7 G10 G12 inner-join,+1,+2,+6,+7) (lookup-join G7 G12 child1,keyCols=[1 2],outCols=(1-4,6-8)) (merge-join G10 G7 G12 inner-join,+6,+7,+1,+2) (lookup-join G10 G12 grandchild1,keyCols=[6 7],outCols=(1-4,6-8)) + │ ├── [ordering: +(1|6)] + │ │ ├── best: (merge-join G7="[ordering: +1,+2]" G10="[ordering: +6,+7]" G12 inner-join,+1,+2,+6,+7) │ │ └── cost: 2161.05 │ └── [] - │ ├── best: (merge-join G7="[ordering: +1,+2]" G10="[ordering: +5,+6]" G12 inner-join,+1,+2,+5,+6) + │ ├── best: (merge-join G7="[ordering: +1,+2]" G10="[ordering: +6,+7]" G12 inner-join,+1,+2,+6,+7) │ └── cost: 2161.05 - ├── G5: (scan parent1) - │ ├── [ordering: +8] - │ │ ├── best: (scan parent1) + ├── G5: (scan parent1,cols=(10,11)) + │ ├── [ordering: +10] + │ │ ├── best: (scan parent1,cols=(10,11)) │ │ └── cost: 1040.02 │ └── [] - │ ├── best: (scan parent1) + │ ├── best: (scan parent1,cols=(10,11)) │ └── cost: 1040.02 ├── G6: (filters G13) - ├── G7: (scan grandchild1) + ├── G7: (scan grandchild1,cols=(1-4)) │ ├── [ordering: +1,+2] - │ │ ├── best: (scan grandchild1) + │ │ ├── best: (scan grandchild1,cols=(1-4)) │ │ └── cost: 1080.02 │ ├── [ordering: +1] - │ │ ├── best: (scan grandchild1) + │ │ ├── best: (scan grandchild1,cols=(1-4)) │ │ └── cost: 1080.02 │ └── [] - │ ├── best: (scan grandchild1) + │ ├── best: (scan grandchild1,cols=(1-4)) │ └── cost: 1080.02 - ├── G8: (inner-join G10 G5 G14) (inner-join G5 G10 G14) (merge-join G10 G5 G12 inner-join,+5,+8) (lookup-join G10 G12 parent1,keyCols=[5],outCols=(5-9)) (merge-join G5 G10 G12 inner-join,+8,+5) (lookup-join G5 G12 child1,keyCols=[8],outCols=(5-9)) - │ ├── [ordering: +(5|8),+6] + ├── G8: (inner-join G10 G5 G14) (inner-join G5 G10 G14) (merge-join G10 G5 G12 inner-join,+6,+10) (lookup-join G10 G12 parent1,keyCols=[6],outCols=(6-8,10,11)) (merge-join G5 G10 G12 inner-join,+10,+6) (lookup-join G5 G12 child1,keyCols=[10],outCols=(6-8,10,11)) + │ ├── [ordering: +(6|10),+7] │ │ ├── best: (sort G8) │ │ └── cost: 2360.34 - │ ├── [ordering: +(5|8)] - │ │ ├── best: (merge-join G10="[ordering: +5]" G5="[ordering: +8]" G12 inner-join,+5,+8) + │ ├── [ordering: +(6|10)] + │ │ ├── best: (merge-join G10="[ordering: +6]" G5="[ordering: +10]" G12 inner-join,+6,+10) │ │ └── cost: 2130.05 │ └── [] - │ ├── best: (merge-join G10="[ordering: +5]" G5="[ordering: +8]" G12 inner-join,+5,+8) + │ ├── best: (merge-join G10="[ordering: +6]" G5="[ordering: +10]" G12 inner-join,+6,+10) │ └── cost: 2130.05 ├── G9: (filters G15 G16) - ├── G10: (scan child1) - │ ├── [ordering: +5,+6] - │ │ ├── best: (scan child1) + ├── G10: (scan child1,cols=(6-8)) + │ ├── [ordering: +6,+7] + │ │ ├── best: (scan child1,cols=(6-8)) │ │ └── cost: 1060.02 - │ ├── [ordering: +5] - │ │ ├── best: (scan child1) + │ ├── [ordering: +6] + │ │ ├── best: (scan child1,cols=(6-8)) │ │ └── cost: 1060.02 │ └── [] - │ ├── best: (scan child1) + │ ├── best: (scan child1,cols=(6-8)) │ └── cost: 1060.02 - ├── G11: (inner-join G7 G5 G6) (inner-join G5 G7 G6) (merge-join G7 G5 G12 inner-join,+1,+8) (lookup-join G7 G12 parent1,keyCols=[1],outCols=(1-4,8,9)) (merge-join G5 G7 G12 inner-join,+8,+1) (lookup-join G5 G12 grandchild1,keyCols=[8],outCols=(1-4,8,9)) - │ ├── [ordering: +(1|8),+2] + ├── G11: (inner-join G7 G5 G6) (inner-join G5 G7 G6) (merge-join G7 G5 G12 inner-join,+1,+10) (lookup-join G7 G12 parent1,keyCols=[1],outCols=(1-4,10,11)) (merge-join G5 G7 G12 inner-join,+10,+1) (lookup-join G5 G12 grandchild1,keyCols=[10],outCols=(1-4,10,11)) + │ ├── [ordering: +(1|10),+2] │ │ ├── best: (sort G11) │ │ └── cost: 2380.34 - │ ├── [ordering: +(1|8)] - │ │ ├── best: (merge-join G7="[ordering: +1]" G5="[ordering: +8]" G12 inner-join,+1,+8) + │ ├── [ordering: +(1|10)] + │ │ ├── best: (merge-join G7="[ordering: +1]" G5="[ordering: +10]" G12 inner-join,+1,+10) │ │ └── cost: 2150.05 │ └── [] - │ ├── best: (merge-join G7="[ordering: +1]" G5="[ordering: +8]" G12 inner-join,+1,+8) + │ ├── best: (merge-join G7="[ordering: +1]" G5="[ordering: +10]" G12 inner-join,+1,+10) │ └── cost: 2150.05 ├── G12: (filters) ├── G13: (eq G17 G18) @@ -351,9 +351,9 @@ memo (optimized, ~26KB, required=[presentation: pid1:1,cid1:2,gcid1:3,gca1:4,ca1 memo SELECT * FROM abc, stu, xyz, pqr WHERE a = 1 ---- -memo (optimized, ~19KB, required=[presentation: a:1,b:2,c:3,s:5,t:6,u:7,x:8,y:9,z:10,p:12,q:13,r:14,s:15,t:16]) +memo (optimized, ~24KB, required=[presentation: a:1,b:2,c:3,s:6,t:7,u:8,x:10,y:11,z:12,p:15,q:16,r:17,s:18,t:19]) ├── G1: (inner-join G2 G3 G4) (inner-join G3 G2 G4) - │ └── [presentation: a:1,b:2,c:3,s:5,t:6,u:7,x:8,y:9,z:10,p:12,q:13,r:14,s:15,t:16] + │ └── [presentation: a:1,b:2,c:3,s:6,t:7,u:8,x:10,y:11,z:12,p:15,q:16,r:17,s:18,t:19] │ ├── best: (inner-join G3 G2 G4) │ └── cost: 325035476.19 ├── G2: (select G5 G6) (scan abc@ab,cols=(1-3),constrained) @@ -370,22 +370,22 @@ memo (optimized, ~19KB, required=[presentation: a:1,b:2,c:3,s:5,t:6,u:7,x:8,y:9, │ ├── best: (scan abc,cols=(1-3)) │ └── cost: 1070.02 ├── G6: (filters G9) - ├── G7: (scan stu) (scan stu@uts) + ├── G7: (scan stu,cols=(6-8)) (scan stu@uts,cols=(6-8)) │ └── [] - │ ├── best: (scan stu) + │ ├── best: (scan stu,cols=(6-8)) │ └── cost: 10600.02 ├── G8: (inner-join G10 G11 G4) (inner-join G11 G10 G4) │ └── [] │ ├── best: (inner-join G10 G11 G4) │ └── cost: 12200.05 ├── G9: (eq G12 G13) - ├── G10: (scan xyz,cols=(8-10)) (scan xyz@xy,cols=(8-10)) (scan xyz@yz,cols=(8-10)) + ├── G10: (scan xyz,cols=(10-12)) (scan xyz@xy,cols=(10-12)) (scan xyz@yz,cols=(10-12)) │ └── [] - │ ├── best: (scan xyz,cols=(8-10)) + │ ├── best: (scan xyz,cols=(10-12)) │ └── cost: 1070.02 - ├── G11: (scan pqr) + ├── G11: (scan pqr,cols=(15-19)) │ └── [] - │ ├── best: (scan pqr) + │ ├── best: (scan pqr,cols=(15-19)) │ └── cost: 1100.02 ├── G12: (variable a) └── G13: (const 1) @@ -397,56 +397,56 @@ SELECT * FROM stu, abc, xyz, pqr WHERE u = a AND a = x AND x = p ---- -memo (optimized, ~27KB, required=[presentation: s:1,t:2,u:3,a:4,b:5,c:6,x:8,y:9,z:10,p:12,q:13,r:14,s:15,t:16]) - ├── G1: (inner-join G2 G3 G4) (inner-join G3 G2 G4) (merge-join G2 G3 G5 inner-join,+3,+4) (merge-join G3 G2 G5 inner-join,+4,+3) (lookup-join G3 G5 stu@uts,keyCols=[4],outCols=(1-6,8-10,12-16)) - │ └── [presentation: s:1,t:2,u:3,a:4,b:5,c:6,x:8,y:9,z:10,p:12,q:13,r:14,s:15,t:16] - │ ├── best: (merge-join G2="[ordering: +3]" G3="[ordering: +(4|8|12)]" G5 inner-join,+3,+4) +memo (optimized, ~31KB, required=[presentation: s:1,t:2,u:3,a:5,b:6,c:7,x:10,y:11,z:12,p:15,q:16,r:17,s:18,t:19]) + ├── G1: (inner-join G2 G3 G4) (inner-join G3 G2 G4) (merge-join G2 G3 G5 inner-join,+3,+5) (merge-join G3 G2 G5 inner-join,+5,+3) (lookup-join G3 G5 stu@uts,keyCols=[5],outCols=(1-3,5-7,10-12,15-19)) + │ └── [presentation: s:1,t:2,u:3,a:5,b:6,c:7,x:10,y:11,z:12,p:15,q:16,r:17,s:18,t:19] + │ ├── best: (merge-join G2="[ordering: +3]" G3="[ordering: +(5|10|15)]" G5 inner-join,+3,+5) │ └── cost: 14110.11 - ├── G2: (scan stu) (scan stu@uts) + ├── G2: (scan stu,cols=(1-3)) (scan stu@uts,cols=(1-3)) │ ├── [ordering: +3] - │ │ ├── best: (scan stu@uts) + │ │ ├── best: (scan stu@uts,cols=(1-3)) │ │ └── cost: 10600.02 │ └── [] - │ ├── best: (scan stu) + │ ├── best: (scan stu,cols=(1-3)) │ └── cost: 10600.02 - ├── G3: (inner-join G6 G7 G8) (inner-join G7 G6 G8) (merge-join G6 G7 G5 inner-join,+4,+8) (merge-join G7 G6 G5 inner-join,+8,+4) (lookup-join G7 G5 abc@ab,keyCols=[8],outCols=(4-6,8-10,12-16)) - │ ├── [ordering: +(4|8|12)] - │ │ ├── best: (merge-join G6="[ordering: +4]" G7="[ordering: +(8|12)]" G5 inner-join,+4,+8) + ├── G3: (inner-join G6 G7 G8) (inner-join G7 G6 G8) (merge-join G6 G7 G5 inner-join,+5,+10) (merge-join G7 G6 G5 inner-join,+10,+5) (lookup-join G7 G5 abc@ab,keyCols=[10],outCols=(5-7,10-12,15-19)) + │ ├── [ordering: +(5|10|15)] + │ │ ├── best: (merge-join G6="[ordering: +5]" G7="[ordering: +(10|15)]" G5 inner-join,+5,+10) │ │ └── cost: 3300.08 │ └── [] - │ ├── best: (merge-join G6="[ordering: +4]" G7="[ordering: +(8|12)]" G5 inner-join,+4,+8) + │ ├── best: (merge-join G6="[ordering: +5]" G7="[ordering: +(10|15)]" G5 inner-join,+5,+10) │ └── cost: 3300.08 ├── G4: (filters G9) ├── G5: (filters) - ├── G6: (scan abc,cols=(4-6)) (scan abc@ab,cols=(4-6)) (scan abc@bc,cols=(4-6)) - │ ├── [ordering: +4] - │ │ ├── best: (scan abc@ab,cols=(4-6)) + ├── G6: (scan abc,cols=(5-7)) (scan abc@ab,cols=(5-7)) (scan abc@bc,cols=(5-7)) + │ ├── [ordering: +5] + │ │ ├── best: (scan abc@ab,cols=(5-7)) │ │ └── cost: 1070.02 │ └── [] - │ ├── best: (scan abc,cols=(4-6)) + │ ├── best: (scan abc,cols=(5-7)) │ └── cost: 1070.02 - ├── G7: (inner-join G10 G11 G12) (inner-join G11 G10 G12) (merge-join G10 G11 G5 inner-join,+8,+12) (lookup-join G10 G5 pqr,keyCols=[8],outCols=(8-10,12-16)) (merge-join G11 G10 G5 inner-join,+12,+8) (lookup-join G11 G5 xyz@xy,keyCols=[12],outCols=(8-10,12-16)) - │ ├── [ordering: +(8|12)] - │ │ ├── best: (merge-join G10="[ordering: +8]" G11="[ordering: +12]" G5 inner-join,+8,+12) + ├── G7: (inner-join G10 G11 G12) (inner-join G11 G10 G12) (merge-join G10 G11 G5 inner-join,+10,+15) (lookup-join G10 G5 pqr,keyCols=[10],outCols=(10-12,15-19)) (merge-join G11 G10 G5 inner-join,+15,+10) (lookup-join G11 G5 xyz@xy,keyCols=[15],outCols=(10-12,15-19)) + │ ├── [ordering: +(10|15)] + │ │ ├── best: (merge-join G10="[ordering: +10]" G11="[ordering: +15]" G5 inner-join,+10,+15) │ │ └── cost: 2200.05 │ └── [] - │ ├── best: (merge-join G10="[ordering: +8]" G11="[ordering: +12]" G5 inner-join,+8,+12) + │ ├── best: (merge-join G10="[ordering: +10]" G11="[ordering: +15]" G5 inner-join,+10,+15) │ └── cost: 2200.05 ├── G8: (filters G13) ├── G9: (eq G14 G15) - ├── G10: (scan xyz,cols=(8-10)) (scan xyz@xy,cols=(8-10)) (scan xyz@yz,cols=(8-10)) - │ ├── [ordering: +8] - │ │ ├── best: (scan xyz@xy,cols=(8-10)) + ├── G10: (scan xyz,cols=(10-12)) (scan xyz@xy,cols=(10-12)) (scan xyz@yz,cols=(10-12)) + │ ├── [ordering: +10] + │ │ ├── best: (scan xyz@xy,cols=(10-12)) │ │ └── cost: 1070.02 │ └── [] - │ ├── best: (scan xyz,cols=(8-10)) + │ ├── best: (scan xyz,cols=(10-12)) │ └── cost: 1070.02 - ├── G11: (scan pqr) - │ ├── [ordering: +12] - │ │ ├── best: (scan pqr) + ├── G11: (scan pqr,cols=(15-19)) + │ ├── [ordering: +15] + │ │ ├── best: (scan pqr,cols=(15-19)) │ │ └── cost: 1100.02 │ └── [] - │ ├── best: (scan pqr) + │ ├── best: (scan pqr,cols=(15-19)) │ └── cost: 1100.02 ├── G12: (filters G16) ├── G13: (eq G15 G17) @@ -472,7 +472,7 @@ inner-join (lookup stu) ├── inner-join (lookup xyz@yz) │ ├── inner-join (lookup abc@ab) │ │ ├── scan pqr - │ │ │ └── constraint: /12: [/5 - /5] + │ │ │ └── constraint: /15: [/5 - /5] │ │ └── filters (true) │ └── filters (true) └── filters (true) @@ -491,33 +491,33 @@ INNER JOIN LATERAL ( ) ON a = v ---- -memo (optimized, ~11KB, required=[presentation: a:1,b:2,c:3,v:5,s:6,t:7,u:8]) +memo (optimized, ~13KB, required=[presentation: a:1,b:2,c:3,v:6,s:7,t:8,u:9]) ├── G1: (inner-join-apply G2 G3 G4) - │ └── [presentation: a:1,b:2,c:3,v:5,s:6,t:7,u:8] + │ └── [presentation: a:1,b:2,c:3,v:6,s:7,t:8,u:9] │ ├── best: (inner-join-apply G2 G3 G4) │ └── cost: 1164.53 ├── G2: (scan abc,cols=(1-3)) (scan abc@ab,cols=(1-3)) (scan abc@bc,cols=(1-3)) │ └── [] │ ├── best: (scan abc,cols=(1-3)) │ └── cost: 1070.02 - ├── G3: (inner-join G5 G6 G7) (inner-join G6 G5 G7) (lookup-join G5 G8 stu,keyCols=[5],outCols=(5-8)) (merge-join G6 G5 G8 inner-join,+6,+5) + ├── G3: (inner-join G5 G6 G7) (inner-join G6 G5 G7) (lookup-join G5 G8 stu,keyCols=[6],outCols=(6-9)) (merge-join G6 G5 G8 inner-join,+7,+6) │ └── [] - │ ├── best: (lookup-join G5 G8 stu,keyCols=[5],outCols=(5-8)) + │ ├── best: (lookup-join G5 G8 stu,keyCols=[6],outCols=(6-9)) │ └── cost: 81.44 ├── G4: (filters G9) ├── G5: (values G10 id=v1) - │ ├── [ordering: +5] + │ ├── [ordering: +6] │ │ ├── best: (sort G5) │ │ └── cost: 0.12 │ └── [] │ ├── best: (values G10 id=v1) │ └── cost: 0.03 - ├── G6: (scan stu) (scan stu@uts) - │ ├── [ordering: +6] - │ │ ├── best: (scan stu) + ├── G6: (scan stu,cols=(7-9)) (scan stu@uts,cols=(7-9)) + │ ├── [ordering: +7] + │ │ ├── best: (scan stu,cols=(7-9)) │ │ └── cost: 10600.02 │ └── [] - │ ├── best: (scan stu) + │ ├── best: (scan stu,cols=(7-9)) │ └── cost: 10600.02 ├── G7: (filters G11) ├── G8: (filters) @@ -605,51 +605,51 @@ ON s = a INNER JOIN xyz ON b = y ---- -memo (optimized, ~25KB, required=[presentation: s:1,t:2,u:3,a:4,b:5,c:6,x:8,y:9,z:10]) - ├── G1: (inner-join G2 G3 G4) (inner-join G5 G6 G7) (inner-join G3 G2 G4) (merge-join G2 G3 G8 inner-join,+5,+9) (lookup-join G2 G8 xyz@yz,keyCols=[5],outCols=(1-6,8-10)) (inner-join G6 G5 G7) (merge-join G5 G6 G8 inner-join,+1,+4) (merge-join G3 G2 G8 inner-join,+9,+5) (merge-join G6 G5 G8 inner-join,+4,+1) (lookup-join G6 G8 stu,keyCols=[4],outCols=(1-6,8-10)) - │ └── [presentation: s:1,t:2,u:3,a:4,b:5,c:6,x:8,y:9,z:10] +memo (optimized, ~27KB, required=[presentation: s:1,t:2,u:3,a:5,b:6,c:7,x:10,y:11,z:12]) + ├── G1: (inner-join G2 G3 G4) (inner-join G5 G6 G7) (inner-join G3 G2 G4) (merge-join G2 G3 G8 inner-join,+6,+11) (lookup-join G2 G8 xyz@yz,keyCols=[6],outCols=(1-3,5-7,10-12)) (inner-join G6 G5 G7) (merge-join G5 G6 G8 inner-join,+1,+5) (merge-join G3 G2 G8 inner-join,+11,+6) (merge-join G6 G5 G8 inner-join,+5,+1) (lookup-join G6 G8 stu,keyCols=[5],outCols=(1-3,5-7,10-12)) + │ └── [presentation: s:1,t:2,u:3,a:5,b:6,c:7,x:10,y:11,z:12] │ ├── best: (inner-join G2 G3 G4) │ └── cost: 14072.69 - ├── G2: (inner-join G5 G9 G7) (inner-join G9 G5 G7) (merge-join G5 G9 G8 inner-join,+1,+4) (lookup-join G5 G8 abc@ab,keyCols=[1],outCols=(1-6)) (merge-join G9 G5 G8 inner-join,+4,+1) (lookup-join G9 G8 stu,keyCols=[4],outCols=(1-6)) - │ ├── [ordering: +5] + ├── G2: (inner-join G5 G9 G7) (inner-join G9 G5 G7) (merge-join G5 G9 G8 inner-join,+1,+5) (lookup-join G5 G8 abc@ab,keyCols=[1],outCols=(1-3,5-7)) (merge-join G9 G5 G8 inner-join,+5,+1) (lookup-join G9 G8 stu,keyCols=[5],outCols=(1-3,5-7)) + │ ├── [ordering: +6] │ │ ├── best: (sort G2) │ │ └── cost: 14737.60 │ └── [] - │ ├── best: (merge-join G5="[ordering: +1]" G9="[ordering: +4]" G8 inner-join,+1,+4) + │ ├── best: (merge-join G5="[ordering: +1]" G9="[ordering: +5]" G8 inner-join,+1,+5) │ └── cost: 11880.05 - ├── G3: (scan xyz,cols=(8-10)) (scan xyz@xy,cols=(8-10)) (scan xyz@yz,cols=(8-10)) - │ ├── [ordering: +9] - │ │ ├── best: (scan xyz@yz,cols=(8-10)) + ├── G3: (scan xyz,cols=(10-12)) (scan xyz@xy,cols=(10-12)) (scan xyz@yz,cols=(10-12)) + │ ├── [ordering: +11] + │ │ ├── best: (scan xyz@yz,cols=(10-12)) │ │ └── cost: 1070.02 │ └── [] - │ ├── best: (scan xyz,cols=(8-10)) + │ ├── best: (scan xyz,cols=(10-12)) │ └── cost: 1070.02 ├── G4: (filters G10) - ├── G5: (scan stu) (scan stu@uts) + ├── G5: (scan stu,cols=(1-3)) (scan stu@uts,cols=(1-3)) │ ├── [ordering: +1] - │ │ ├── best: (scan stu) + │ │ ├── best: (scan stu,cols=(1-3)) │ │ └── cost: 10600.02 │ └── [] - │ ├── best: (scan stu) + │ ├── best: (scan stu,cols=(1-3)) │ └── cost: 10600.02 - ├── G6: (inner-join G9 G3 G4) (inner-join G3 G9 G4) (merge-join G9 G3 G8 inner-join,+5,+9) (lookup-join G9 G8 xyz@yz,keyCols=[5],outCols=(4-6,8-10)) (merge-join G3 G9 G8 inner-join,+9,+5) (lookup-join G3 G8 abc@bc,keyCols=[9],outCols=(4-6,8-10)) - │ ├── [ordering: +4] + ├── G6: (inner-join G9 G3 G4) (inner-join G3 G9 G4) (merge-join G9 G3 G8 inner-join,+6,+11) (lookup-join G9 G8 xyz@yz,keyCols=[6],outCols=(5-7,10-12)) (merge-join G3 G9 G8 inner-join,+11,+6) (lookup-join G3 G8 abc@bc,keyCols=[11],outCols=(5-7,10-12)) + │ ├── [ordering: +5] │ │ ├── best: (sort G6) │ │ └── cost: 5053.06 │ └── [] - │ ├── best: (merge-join G9="[ordering: +5]" G3="[ordering: +9]" G8 inner-join,+5,+9) + │ ├── best: (merge-join G9="[ordering: +6]" G3="[ordering: +11]" G8 inner-join,+6,+11) │ └── cost: 2258.06 ├── G7: (filters G11) ├── G8: (filters) - ├── G9: (scan abc,cols=(4-6)) (scan abc@ab,cols=(4-6)) (scan abc@bc,cols=(4-6)) - │ ├── [ordering: +4] - │ │ ├── best: (scan abc@ab,cols=(4-6)) - │ │ └── cost: 1070.02 + ├── G9: (scan abc,cols=(5-7)) (scan abc@ab,cols=(5-7)) (scan abc@bc,cols=(5-7)) │ ├── [ordering: +5] - │ │ ├── best: (scan abc@bc,cols=(4-6)) + │ │ ├── best: (scan abc@ab,cols=(5-7)) + │ │ └── cost: 1070.02 + │ ├── [ordering: +6] + │ │ ├── best: (scan abc@bc,cols=(5-7)) │ │ └── cost: 1070.02 │ └── [] - │ ├── best: (scan abc,cols=(4-6)) + │ ├── best: (scan abc,cols=(5-7)) │ └── cost: 1070.02 ├── G10: (eq G12 G13) ├── G11: (eq G14 G15) @@ -667,9 +667,9 @@ memo (optimized, ~25KB, required=[presentation: s:1,t:2,u:3,a:4,b:5,c:6,x:8,y:9, memo SELECT * FROM abc JOIN xyz ON a=z ---- -memo (optimized, ~10KB, required=[presentation: a:1,b:2,c:3,x:5,y:6,z:7]) - ├── G1: (inner-join G2 G3 G4) (inner-join G3 G2 G4) (merge-join G2 G3 G5 inner-join,+1,+7) (lookup-join G3 G5 abc@ab,keyCols=[7],outCols=(1-3,5-7)) - │ └── [presentation: a:1,b:2,c:3,x:5,y:6,z:7] +memo (optimized, ~10KB, required=[presentation: a:1,b:2,c:3,x:6,y:7,z:8]) + ├── G1: (inner-join G2 G3 G4) (inner-join G3 G2 G4) (merge-join G2 G3 G5 inner-join,+1,+8) (lookup-join G3 G5 abc@ab,keyCols=[8],outCols=(1-3,6-8)) + │ └── [presentation: a:1,b:2,c:3,x:6,y:7,z:8] │ ├── best: (inner-join G2 G3 G4) │ └── cost: 2179.96 ├── G2: (scan abc,cols=(1-3)) (scan abc@ab,cols=(1-3)) (scan abc@bc,cols=(1-3)) @@ -679,12 +679,12 @@ memo (optimized, ~10KB, required=[presentation: a:1,b:2,c:3,x:5,y:6,z:7]) │ └── [] │ ├── best: (scan abc,cols=(1-3)) │ └── cost: 1070.02 - ├── G3: (scan xyz,cols=(5-7)) (scan xyz@xy,cols=(5-7)) (scan xyz@yz,cols=(5-7)) - │ ├── [ordering: +7] + ├── G3: (scan xyz,cols=(6-8)) (scan xyz@xy,cols=(6-8)) (scan xyz@yz,cols=(6-8)) + │ ├── [ordering: +8] │ │ ├── best: (sort G3) │ │ └── cost: 1289.35 │ └── [] - │ ├── best: (scan xyz,cols=(5-7)) + │ ├── best: (scan xyz,cols=(6-8)) │ └── cost: 1070.02 ├── G4: (filters G6) ├── G5: (filters) @@ -695,9 +695,9 @@ memo (optimized, ~10KB, required=[presentation: a:1,b:2,c:3,x:5,y:6,z:7]) memo SELECT * FROM abc FULL OUTER JOIN xyz ON a=z ---- -memo (optimized, ~9KB, required=[presentation: a:1,b:2,c:3,x:5,y:6,z:7]) - ├── G1: (full-join G2 G3 G4) (full-join G3 G2 G4) (merge-join G2 G3 G5 full-join,+1,+7) - │ └── [presentation: a:1,b:2,c:3,x:5,y:6,z:7] +memo (optimized, ~9KB, required=[presentation: a:1,b:2,c:3,x:6,y:7,z:8]) + ├── G1: (full-join G2 G3 G4) (full-join G3 G2 G4) (merge-join G2 G3 G5 full-join,+1,+8) + │ └── [presentation: a:1,b:2,c:3,x:6,y:7,z:8] │ ├── best: (full-join G2 G3 G4) │ └── cost: 2180.06 ├── G2: (scan abc,cols=(1-3)) (scan abc@ab,cols=(1-3)) (scan abc@bc,cols=(1-3)) @@ -707,12 +707,12 @@ memo (optimized, ~9KB, required=[presentation: a:1,b:2,c:3,x:5,y:6,z:7]) │ └── [] │ ├── best: (scan abc,cols=(1-3)) │ └── cost: 1070.02 - ├── G3: (scan xyz,cols=(5-7)) (scan xyz@xy,cols=(5-7)) (scan xyz@yz,cols=(5-7)) - │ ├── [ordering: +7] + ├── G3: (scan xyz,cols=(6-8)) (scan xyz@xy,cols=(6-8)) (scan xyz@yz,cols=(6-8)) + │ ├── [ordering: +8] │ │ ├── best: (sort G3) │ │ └── cost: 1289.35 │ └── [] - │ ├── best: (scan xyz,cols=(5-7)) + │ ├── best: (scan xyz,cols=(6-8)) │ └── cost: 1070.02 ├── G4: (filters G6) ├── G5: (filters) @@ -725,10 +725,10 @@ opt SELECT * FROM abc INNER JOIN xyz ON a=c WHERE b=1 ---- inner-join (cross) - ├── columns: a:1!null b:2!null c:3!null x:5 y:6 z:7 + ├── columns: a:1!null b:2!null c:3!null x:6 y:7 z:8 ├── fd: ()-->(2), (1)==(3), (3)==(1) ├── scan xyz - │ └── columns: x:5 y:6 z:7 + │ └── columns: x:6 y:7 z:8 ├── select │ ├── columns: a:1!null b:2!null c:3!null │ ├── fd: ()-->(2), (1)==(3), (3)==(1) @@ -745,7 +745,7 @@ opt SELECT * FROM abc INNER HASH JOIN xyz ON a=c WHERE b=1 ---- inner-join (cross) - ├── columns: a:1!null b:2!null c:3!null x:5 y:6 z:7 + ├── columns: a:1!null b:2!null c:3!null x:6 y:7 z:8 ├── flags: force hash join (store right side) ├── fd: ()-->(2), (1)==(3), (3)==(1) ├── select @@ -758,22 +758,22 @@ inner-join (cross) │ └── filters │ └── a:1 = c:3 [outer=(1,3), constraints=(/1: (/NULL - ]; /3: (/NULL - ]), fd=(1)==(3), (3)==(1)] ├── scan xyz - │ └── columns: x:5 y:6 z:7 + │ └── columns: x:6 y:7 z:8 └── filters (true) opt SELECT * FROM (SELECT * FROM abc WHERE b=1) FULL OUTER JOIN xyz ON a=z ---- full-join (hash) - ├── columns: a:1 b:2 c:3 x:5 y:6 z:7 + ├── columns: a:1 b:2 c:3 x:6 y:7 z:8 ├── scan xyz - │ └── columns: x:5 y:6 z:7 + │ └── columns: x:6 y:7 z:8 ├── scan abc@bc │ ├── columns: a:1 b:2!null c:3 │ ├── constraint: /2/3/4: [/1 - /1] │ └── fd: ()-->(2) └── filters - └── a:1 = z:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + └── a:1 = z:8 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] # Verify that commuting works correctly when there is a lookup join hint # (specifically that it returns the original expression and flags when applied @@ -781,18 +781,18 @@ full-join (hash) memo SELECT * FROM abc INNER LOOKUP JOIN xyz ON a=x ---- -memo (optimized, ~10KB, required=[presentation: a:1,b:2,c:3,x:5,y:6,z:7]) - ├── G1: (inner-join G2 G3 G4) (inner-join G3 G2 G4) (lookup-join G2 G5 xyz@xy,keyCols=[1],outCols=(1-3,5-7)) - │ └── [presentation: a:1,b:2,c:3,x:5,y:6,z:7] - │ ├── best: (lookup-join G2 G5 xyz@xy,keyCols=[1],outCols=(1-3,5-7)) +memo (optimized, ~10KB, required=[presentation: a:1,b:2,c:3,x:6,y:7,z:8]) + ├── G1: (inner-join G2 G3 G4) (inner-join G3 G2 G4) (lookup-join G2 G5 xyz@xy,keyCols=[1],outCols=(1-3,6-8)) + │ └── [presentation: a:1,b:2,c:3,x:6,y:7,z:8] + │ ├── best: (lookup-join G2 G5 xyz@xy,keyCols=[1],outCols=(1-3,6-8)) │ └── cost: 23150.03 ├── G2: (scan abc,cols=(1-3)) (scan abc@ab,cols=(1-3)) (scan abc@bc,cols=(1-3)) │ └── [] │ ├── best: (scan abc,cols=(1-3)) │ └── cost: 1070.02 - ├── G3: (scan xyz,cols=(5-7)) (scan xyz@xy,cols=(5-7)) (scan xyz@yz,cols=(5-7)) + ├── G3: (scan xyz,cols=(6-8)) (scan xyz@xy,cols=(6-8)) (scan xyz@yz,cols=(6-8)) │ └── [] - │ ├── best: (scan xyz,cols=(5-7)) + │ ├── best: (scan xyz,cols=(6-8)) │ └── cost: 1070.02 ├── G4: (filters G6) ├── G5: (filters) @@ -807,9 +807,9 @@ memo (optimized, ~10KB, required=[presentation: a:1,b:2,c:3,x:5,y:6,z:7]) memo SELECT * FROM abc LEFT OUTER JOIN xyz ON a=z ---- -memo (optimized, ~9KB, required=[presentation: a:1,b:2,c:3,x:5,y:6,z:7]) - ├── G1: (left-join G2 G3 G4) (right-join G3 G2 G4) (merge-join G2 G3 G5 left-join,+1,+7) - │ └── [presentation: a:1,b:2,c:3,x:5,y:6,z:7] +memo (optimized, ~9KB, required=[presentation: a:1,b:2,c:3,x:6,y:7,z:8]) + ├── G1: (left-join G2 G3 G4) (right-join G3 G2 G4) (merge-join G2 G3 G5 left-join,+1,+8) + │ └── [presentation: a:1,b:2,c:3,x:6,y:7,z:8] │ ├── best: (left-join G2 G3 G4) │ └── cost: 2180.06 ├── G2: (scan abc,cols=(1-3)) (scan abc@ab,cols=(1-3)) (scan abc@bc,cols=(1-3)) @@ -819,12 +819,12 @@ memo (optimized, ~9KB, required=[presentation: a:1,b:2,c:3,x:5,y:6,z:7]) │ └── [] │ ├── best: (scan abc,cols=(1-3)) │ └── cost: 1070.02 - ├── G3: (scan xyz,cols=(5-7)) (scan xyz@xy,cols=(5-7)) (scan xyz@yz,cols=(5-7)) - │ ├── [ordering: +7] + ├── G3: (scan xyz,cols=(6-8)) (scan xyz@xy,cols=(6-8)) (scan xyz@yz,cols=(6-8)) + │ ├── [ordering: +8] │ │ ├── best: (sort G3) │ │ └── cost: 1289.35 │ └── [] - │ ├── best: (scan xyz,cols=(5-7)) + │ ├── best: (scan xyz,cols=(6-8)) │ └── cost: 1070.02 ├── G4: (filters G6) ├── G5: (filters) @@ -836,16 +836,16 @@ opt SELECT * FROM abc LEFT OUTER JOIN xyz ON a=z WHERE b=1 ---- right-join (hash) - ├── columns: a:1 b:2!null c:3 x:5 y:6 z:7 + ├── columns: a:1 b:2!null c:3 x:6 y:7 z:8 ├── fd: ()-->(2) ├── scan xyz - │ └── columns: x:5 y:6 z:7 + │ └── columns: x:6 y:7 z:8 ├── scan abc@bc │ ├── columns: a:1 b:2!null c:3 │ ├── constraint: /2/3/4: [/1 - /1] │ └── fd: ()-->(2) └── filters - └── a:1 = z:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + └── a:1 = z:8 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] # -------------------------------------------------- # CommuteRightJoin @@ -854,17 +854,17 @@ right-join (hash) memo SELECT * FROM abc RIGHT OUTER JOIN xyz ON a=z ---- -memo (optimized, ~10KB, required=[presentation: a:1,b:2,c:3,x:5,y:6,z:7]) - ├── G1: (left-join G2 G3 G4) (right-join G3 G2 G4) (lookup-join G2 G5 abc@ab,keyCols=[7],outCols=(1-3,5-7)) (merge-join G3 G2 G5 right-join,+1,+7) - │ └── [presentation: a:1,b:2,c:3,x:5,y:6,z:7] +memo (optimized, ~10KB, required=[presentation: a:1,b:2,c:3,x:6,y:7,z:8]) + ├── G1: (left-join G2 G3 G4) (right-join G3 G2 G4) (lookup-join G2 G5 abc@ab,keyCols=[8],outCols=(1-3,6-8)) (merge-join G3 G2 G5 right-join,+1,+8) + │ └── [presentation: a:1,b:2,c:3,x:6,y:7,z:8] │ ├── best: (left-join G2 G3 G4) │ └── cost: 2180.06 - ├── G2: (scan xyz,cols=(5-7)) (scan xyz@xy,cols=(5-7)) (scan xyz@yz,cols=(5-7)) - │ ├── [ordering: +7] + ├── G2: (scan xyz,cols=(6-8)) (scan xyz@xy,cols=(6-8)) (scan xyz@yz,cols=(6-8)) + │ ├── [ordering: +8] │ │ ├── best: (sort G2) │ │ └── cost: 1289.35 │ └── [] - │ ├── best: (scan xyz,cols=(5-7)) + │ ├── best: (scan xyz,cols=(6-8)) │ └── cost: 1070.02 ├── G3: (scan abc,cols=(1-3)) (scan abc@ab,cols=(1-3)) (scan abc@bc,cols=(1-3)) │ ├── [ordering: +1] @@ -883,15 +883,15 @@ opt SELECT * FROM (SELECT * FROM abc WHERE b=1) RIGHT OUTER JOIN xyz ON a=z ---- left-join (hash) - ├── columns: a:1 b:2 c:3 x:5 y:6 z:7 + ├── columns: a:1 b:2 c:3 x:6 y:7 z:8 ├── scan xyz - │ └── columns: x:5 y:6 z:7 + │ └── columns: x:6 y:7 z:8 ├── scan abc@bc │ ├── columns: a:1 b:2!null c:3 │ ├── constraint: /2/3/4: [/1 - /1] │ └── fd: ()-->(2) └── filters - └── a:1 = z:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + └── a:1 = z:8 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] # -------------------------------------------------- # GenerateMergeJoins @@ -901,25 +901,25 @@ opt SELECT * FROM abc JOIN xyz ON a=x ---- inner-join (merge) - ├── columns: a:1!null b:2 c:3 x:5!null y:6 z:7 + ├── columns: a:1!null b:2 c:3 x:6!null y:7 z:8 ├── left ordering: +1 - ├── right ordering: +5 - ├── fd: (1)==(5), (5)==(1) + ├── right ordering: +6 + ├── fd: (1)==(6), (6)==(1) ├── scan abc@ab │ ├── columns: a:1 b:2 c:3 │ └── ordering: +1 ├── scan xyz@xy - │ ├── columns: x:5 y:6 z:7 - │ └── ordering: +5 + │ ├── columns: x:6 y:7 z:8 + │ └── ordering: +6 └── filters (true) memo SELECT * FROM abc JOIN xyz ON a=x ---- -memo (optimized, ~12KB, required=[presentation: a:1,b:2,c:3,x:5,y:6,z:7]) - ├── G1: (inner-join G2 G3 G4) (inner-join G3 G2 G4) (merge-join G2 G3 G5 inner-join,+1,+5) (lookup-join G2 G5 xyz@xy,keyCols=[1],outCols=(1-3,5-7)) (merge-join G3 G2 G5 inner-join,+5,+1) (lookup-join G3 G5 abc@ab,keyCols=[5],outCols=(1-3,5-7)) - │ └── [presentation: a:1,b:2,c:3,x:5,y:6,z:7] - │ ├── best: (merge-join G2="[ordering: +1]" G3="[ordering: +5]" G5 inner-join,+1,+5) +memo (optimized, ~12KB, required=[presentation: a:1,b:2,c:3,x:6,y:7,z:8]) + ├── G1: (inner-join G2 G3 G4) (inner-join G3 G2 G4) (merge-join G2 G3 G5 inner-join,+1,+6) (lookup-join G2 G5 xyz@xy,keyCols=[1],outCols=(1-3,6-8)) (merge-join G3 G2 G5 inner-join,+6,+1) (lookup-join G3 G5 abc@ab,keyCols=[6],outCols=(1-3,6-8)) + │ └── [presentation: a:1,b:2,c:3,x:6,y:7,z:8] + │ ├── best: (merge-join G2="[ordering: +1]" G3="[ordering: +6]" G5 inner-join,+1,+6) │ └── cost: 2170.05 ├── G2: (scan abc,cols=(1-3)) (scan abc@ab,cols=(1-3)) (scan abc@bc,cols=(1-3)) │ ├── [ordering: +1] @@ -928,12 +928,12 @@ memo (optimized, ~12KB, required=[presentation: a:1,b:2,c:3,x:5,y:6,z:7]) │ └── [] │ ├── best: (scan abc,cols=(1-3)) │ └── cost: 1070.02 - ├── G3: (scan xyz,cols=(5-7)) (scan xyz@xy,cols=(5-7)) (scan xyz@yz,cols=(5-7)) - │ ├── [ordering: +5] - │ │ ├── best: (scan xyz@xy,cols=(5-7)) + ├── G3: (scan xyz,cols=(6-8)) (scan xyz@xy,cols=(6-8)) (scan xyz@yz,cols=(6-8)) + │ ├── [ordering: +6] + │ │ ├── best: (scan xyz@xy,cols=(6-8)) │ │ └── cost: 1070.02 │ └── [] - │ ├── best: (scan xyz,cols=(5-7)) + │ ├── best: (scan xyz,cols=(6-8)) │ └── cost: 1070.02 ├── G4: (filters G6) ├── G5: (filters) @@ -945,18 +945,18 @@ memo (optimized, ~12KB, required=[presentation: a:1,b:2,c:3,x:5,y:6,z:7]) memo SELECT * FROM abc INNER HASH JOIN xyz ON a=x ---- -memo (optimized, ~9KB, required=[presentation: a:1,b:2,c:3,x:5,y:6,z:7]) +memo (optimized, ~9KB, required=[presentation: a:1,b:2,c:3,x:6,y:7,z:8]) ├── G1: (inner-join G2 G3 G4) (inner-join G3 G2 G4) - │ └── [presentation: a:1,b:2,c:3,x:5,y:6,z:7] + │ └── [presentation: a:1,b:2,c:3,x:6,y:7,z:8] │ ├── best: (inner-join G2 G3 G4) │ └── cost: 2180.06 ├── G2: (scan abc,cols=(1-3)) (scan abc@ab,cols=(1-3)) (scan abc@bc,cols=(1-3)) │ └── [] │ ├── best: (scan abc,cols=(1-3)) │ └── cost: 1070.02 - ├── G3: (scan xyz,cols=(5-7)) (scan xyz@xy,cols=(5-7)) (scan xyz@yz,cols=(5-7)) + ├── G3: (scan xyz,cols=(6-8)) (scan xyz@xy,cols=(6-8)) (scan xyz@yz,cols=(6-8)) │ └── [] - │ ├── best: (scan xyz,cols=(5-7)) + │ ├── best: (scan xyz,cols=(6-8)) │ └── cost: 1070.02 ├── G4: (filters G5) ├── G5: (eq G6 G7) @@ -967,32 +967,32 @@ opt SELECT * FROM abc JOIN xyz ON x=a ---- inner-join (merge) - ├── columns: a:1!null b:2 c:3 x:5!null y:6 z:7 + ├── columns: a:1!null b:2 c:3 x:6!null y:7 z:8 ├── left ordering: +1 - ├── right ordering: +5 - ├── fd: (1)==(5), (5)==(1) + ├── right ordering: +6 + ├── fd: (1)==(6), (6)==(1) ├── scan abc@ab │ ├── columns: a:1 b:2 c:3 │ └── ordering: +1 ├── scan xyz@xy - │ ├── columns: x:5 y:6 z:7 - │ └── ordering: +5 + │ ├── columns: x:6 y:7 z:8 + │ └── ordering: +6 └── filters (true) opt SELECT * FROM abc JOIN xyz ON a=x AND a=x AND x=a ---- inner-join (merge) - ├── columns: a:1!null b:2 c:3 x:5!null y:6 z:7 + ├── columns: a:1!null b:2 c:3 x:6!null y:7 z:8 ├── left ordering: +1 - ├── right ordering: +5 - ├── fd: (1)==(5), (5)==(1) + ├── right ordering: +6 + ├── fd: (1)==(6), (6)==(1) ├── scan abc@ab │ ├── columns: a:1 b:2 c:3 │ └── ordering: +1 ├── scan xyz@xy - │ ├── columns: x:5 y:6 z:7 - │ └── ordering: +5 + │ ├── columns: x:6 y:7 z:8 + │ └── ordering: +6 └── filters (true) # Use constraints to force the choice of an index which doesn't help, and @@ -1001,48 +1001,48 @@ opt SELECT * FROM abc JOIN xyz ON a=x AND b=y WHERE b=1 AND y=1 ---- inner-join (hash) - ├── columns: a:1!null b:2!null c:3 x:5!null y:6!null z:7 - ├── fd: ()-->(2,6), (1)==(5), (5)==(1), (2)==(6), (6)==(2) + ├── columns: a:1!null b:2!null c:3 x:6!null y:7!null z:8 + ├── fd: ()-->(2,7), (1)==(6), (6)==(1), (2)==(7), (7)==(2) ├── scan abc@bc │ ├── columns: a:1 b:2!null c:3 │ ├── constraint: /2/3/4: [/1 - /1] │ └── fd: ()-->(2) ├── scan xyz@yz - │ ├── columns: x:5 y:6!null z:7 - │ ├── constraint: /6/7/8: [/1 - /1] - │ └── fd: ()-->(6) + │ ├── columns: x:6 y:7!null z:8 + │ ├── constraint: /7/8/9: [/1 - /1] + │ └── fd: ()-->(7) └── filters - ├── a:1 = x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] - └── b:2 = y:6 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + ├── a:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── b:2 = y:7 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] # Verify case where we generate multiple merge-joins. memo SELECT * FROM stu AS l JOIN stu AS r ON (l.s, l.t, l.u) = (r.s, r.t, r.u) ---- -memo (optimized, ~11KB, required=[presentation: s:1,t:2,u:3,s:4,t:5,u:6]) - ├── G1: (inner-join G2 G3 G4) (inner-join G3 G2 G4) (merge-join G2 G3 G5 inner-join,+1,+2,+3,+4,+5,+6) (merge-join G2 G3 G5 inner-join,+3,+2,+1,+6,+5,+4) (lookup-join G2 G5 stu,keyCols=[1 2 3],outCols=(1-6)) (lookup-join G2 G5 stu@uts,keyCols=[3 2 1],outCols=(1-6)) (merge-join G3 G2 G5 inner-join,+4,+5,+6,+1,+2,+3) (merge-join G3 G2 G5 inner-join,+6,+5,+4,+3,+2,+1) (lookup-join G3 G5 stu,keyCols=[4 5 6],outCols=(1-6)) (lookup-join G3 G5 stu@uts,keyCols=[6 5 4],outCols=(1-6)) - │ └── [presentation: s:1,t:2,u:3,s:4,t:5,u:6] - │ ├── best: (merge-join G2="[ordering: +1,+2,+3]" G3="[ordering: +4,+5,+6]" G5 inner-join,+1,+2,+3,+4,+5,+6) +memo (optimized, ~15KB, required=[presentation: s:1,t:2,u:3,s:5,t:6,u:7]) + ├── G1: (inner-join G2 G3 G4) (inner-join G3 G2 G4) (merge-join G2 G3 G5 inner-join,+1,+2,+3,+5,+6,+7) (merge-join G2 G3 G5 inner-join,+3,+2,+1,+7,+6,+5) (lookup-join G2 G5 stu,keyCols=[1 2 3],outCols=(1-3,5-7)) (lookup-join G2 G5 stu@uts,keyCols=[3 2 1],outCols=(1-3,5-7)) (merge-join G3 G2 G5 inner-join,+5,+6,+7,+1,+2,+3) (merge-join G3 G2 G5 inner-join,+7,+6,+5,+3,+2,+1) (lookup-join G3 G5 stu,keyCols=[5 6 7],outCols=(1-3,5-7)) (lookup-join G3 G5 stu@uts,keyCols=[7 6 5],outCols=(1-3,5-7)) + │ └── [presentation: s:1,t:2,u:3,s:5,t:6,u:7] + │ ├── best: (merge-join G2="[ordering: +1,+2,+3]" G3="[ordering: +5,+6,+7]" G5 inner-join,+1,+2,+3,+5,+6,+7) │ └── cost: 21400.05 - ├── G2: (scan l) (scan l@uts) + ├── G2: (scan l,cols=(1-3)) (scan l@uts,cols=(1-3)) │ ├── [ordering: +1,+2,+3] - │ │ ├── best: (scan l) + │ │ ├── best: (scan l,cols=(1-3)) │ │ └── cost: 10600.02 │ ├── [ordering: +3,+2,+1] - │ │ ├── best: (scan l@uts) + │ │ ├── best: (scan l@uts,cols=(1-3)) │ │ └── cost: 10600.02 │ └── [] - │ ├── best: (scan l) + │ ├── best: (scan l,cols=(1-3)) │ └── cost: 10600.02 - ├── G3: (scan r) (scan r@uts) - │ ├── [ordering: +4,+5,+6] - │ │ ├── best: (scan r) + ├── G3: (scan r,cols=(5-7)) (scan r@uts,cols=(5-7)) + │ ├── [ordering: +5,+6,+7] + │ │ ├── best: (scan r,cols=(5-7)) │ │ └── cost: 10600.02 - │ ├── [ordering: +6,+5,+4] - │ │ ├── best: (scan r@uts) + │ ├── [ordering: +7,+6,+5] + │ │ ├── best: (scan r@uts,cols=(5-7)) │ │ └── cost: 10600.02 │ └── [] - │ ├── best: (scan r) + │ ├── best: (scan r,cols=(5-7)) │ └── cost: 10600.02 ├── G4: (filters G6 G7 G8) ├── G5: (filters) @@ -1065,53 +1065,53 @@ GenerateMergeJoins ================================================================================ Source expression: inner-join (hash) - ├── columns: s:1!null t:2!null u:3!null s:4!null t:5!null u:6!null + ├── columns: s:1!null t:2!null u:3!null s:5!null t:6!null u:7!null ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) - ├── key: (4-6) - ├── fd: (1)==(4), (4)==(1), (2)==(5), (5)==(2), (3)==(6), (6)==(3) + ├── key: (5-7) + ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2), (3)==(7), (7)==(3) ├── scan l │ ├── columns: l.s:1!null l.t:2!null l.u:3!null │ └── key: (1-3) ├── scan r - │ ├── columns: r.s:4!null r.t:5!null r.u:6!null - │ └── key: (4-6) + │ ├── columns: r.s:5!null r.t:6!null r.u:7!null + │ └── key: (5-7) └── filters - ├── l.s:1 = r.s:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] - ├── l.t:2 = r.t:5 [outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ]), fd=(2)==(5), (5)==(2)] - └── l.u:3 = r.u:6 [outer=(3,6), constraints=(/3: (/NULL - ]; /6: (/NULL - ]), fd=(3)==(6), (6)==(3)] + ├── l.s:1 = r.s:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + ├── l.t:2 = r.t:6 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + └── l.u:3 = r.u:7 [outer=(3,7), constraints=(/3: (/NULL - ]; /7: (/NULL - ]), fd=(3)==(7), (7)==(3)] New expression 1 of 2: inner-join (merge) - ├── columns: s:1!null t:2!null u:3!null s:4!null t:5!null u:6!null + ├── columns: s:1!null t:2!null u:3!null s:5!null t:6!null u:7!null ├── left ordering: +1,+2,+3 - ├── right ordering: +4,+5,+6 - ├── key: (4-6) - ├── fd: (1)==(4), (4)==(1), (2)==(5), (5)==(2), (3)==(6), (6)==(3) + ├── right ordering: +5,+6,+7 + ├── key: (5-7) + ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2), (3)==(7), (7)==(3) ├── scan l │ ├── columns: l.s:1!null l.t:2!null l.u:3!null │ ├── key: (1-3) │ └── ordering: +1,+2,+3 ├── scan r - │ ├── columns: r.s:4!null r.t:5!null r.u:6!null - │ ├── key: (4-6) - │ └── ordering: +4,+5,+6 + │ ├── columns: r.s:5!null r.t:6!null r.u:7!null + │ ├── key: (5-7) + │ └── ordering: +5,+6,+7 └── filters (true) New expression 2 of 2: inner-join (merge) - ├── columns: s:1!null t:2!null u:3!null s:4!null t:5!null u:6!null + ├── columns: s:1!null t:2!null u:3!null s:5!null t:6!null u:7!null ├── left ordering: +3,+2,+1 - ├── right ordering: +6,+5,+4 - ├── key: (4-6) - ├── fd: (1)==(4), (4)==(1), (2)==(5), (5)==(2), (3)==(6), (6)==(3) + ├── right ordering: +7,+6,+5 + ├── key: (5-7) + ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2), (3)==(7), (7)==(3) ├── scan l@uts │ ├── columns: l.s:1!null l.t:2!null l.u:3!null │ ├── key: (1-3) │ └── ordering: +3,+2,+1 ├── scan r@uts - │ ├── columns: r.s:4!null r.t:5!null r.u:6!null - │ ├── key: (4-6) - │ └── ordering: +6,+5,+4 + │ ├── columns: r.s:5!null r.t:6!null r.u:7!null + │ ├── key: (5-7) + │ └── ordering: +7,+6,+5 └── filters (true) ================================================================================ @@ -1119,32 +1119,32 @@ GenerateMergeJoins ================================================================================ Source expression: inner-join (hash) - ├── columns: s:1!null t:2!null u:3!null s:4!null t:5!null u:6!null + ├── columns: s:1!null t:2!null u:3!null s:5!null t:6!null u:7!null ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) - ├── key: (4-6) - ├── fd: (1)==(4), (4)==(1), (2)==(5), (5)==(2), (3)==(6), (6)==(3) + ├── key: (5-7) + ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2), (3)==(7), (7)==(3) ├── scan r - │ ├── columns: r.s:4!null r.t:5!null r.u:6!null - │ └── key: (4-6) + │ ├── columns: r.s:5!null r.t:6!null r.u:7!null + │ └── key: (5-7) ├── scan l │ ├── columns: l.s:1!null l.t:2!null l.u:3!null │ └── key: (1-3) └── filters - ├── l.s:1 = r.s:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] - ├── l.t:2 = r.t:5 [outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ]), fd=(2)==(5), (5)==(2)] - └── l.u:3 = r.u:6 [outer=(3,6), constraints=(/3: (/NULL - ]; /6: (/NULL - ]), fd=(3)==(6), (6)==(3)] + ├── l.s:1 = r.s:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + ├── l.t:2 = r.t:6 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + └── l.u:3 = r.u:7 [outer=(3,7), constraints=(/3: (/NULL - ]; /7: (/NULL - ]), fd=(3)==(7), (7)==(3)] New expression 1 of 2: inner-join (merge) - ├── columns: s:1!null t:2!null u:3!null s:4!null t:5!null u:6!null - ├── left ordering: +4,+5,+6 + ├── columns: s:1!null t:2!null u:3!null s:5!null t:6!null u:7!null + ├── left ordering: +5,+6,+7 ├── right ordering: +1,+2,+3 - ├── key: (4-6) - ├── fd: (1)==(4), (4)==(1), (2)==(5), (5)==(2), (3)==(6), (6)==(3) + ├── key: (5-7) + ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2), (3)==(7), (7)==(3) ├── scan r - │ ├── columns: r.s:4!null r.t:5!null r.u:6!null - │ ├── key: (4-6) - │ └── ordering: +4,+5,+6 + │ ├── columns: r.s:5!null r.t:6!null r.u:7!null + │ ├── key: (5-7) + │ └── ordering: +5,+6,+7 ├── scan l │ ├── columns: l.s:1!null l.t:2!null l.u:3!null │ ├── key: (1-3) @@ -1153,15 +1153,15 @@ New expression 1 of 2: New expression 2 of 2: inner-join (merge) - ├── columns: s:1!null t:2!null u:3!null s:4!null t:5!null u:6!null - ├── left ordering: +6,+5,+4 + ├── columns: s:1!null t:2!null u:3!null s:5!null t:6!null u:7!null + ├── left ordering: +7,+6,+5 ├── right ordering: +3,+2,+1 - ├── key: (4-6) - ├── fd: (1)==(4), (4)==(1), (2)==(5), (5)==(2), (3)==(6), (6)==(3) + ├── key: (5-7) + ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2), (3)==(7), (7)==(3) ├── scan r@uts - │ ├── columns: r.s:4!null r.t:5!null r.u:6!null - │ ├── key: (4-6) - │ └── ordering: +6,+5,+4 + │ ├── columns: r.s:5!null r.t:6!null r.u:7!null + │ ├── key: (5-7) + │ └── ordering: +7,+6,+5 ├── scan l@uts │ ├── columns: l.s:1!null l.t:2!null l.u:3!null │ ├── key: (1-3) @@ -1187,19 +1187,19 @@ opt SELECT * FROM stu LEFT OUTER JOIN abc ON (c,b,a) = (s,t,u) ---- left-join (merge) - ├── columns: s:1!null t:2!null u:3!null a:4 b:5 c:6 + ├── columns: s:1!null t:2!null u:3!null a:5 b:6 c:7 ├── left ordering: +3,+2,+1 - ├── right ordering: +4,+5,+6 + ├── right ordering: +5,+6,+7 ├── scan stu@uts │ ├── columns: s:1!null t:2!null u:3!null │ ├── key: (1-3) │ └── ordering: +3,+2,+1 ├── sort (segmented) - │ ├── columns: a:4 b:5 c:6 - │ ├── ordering: +4,+5,+6 + │ ├── columns: a:5 b:6 c:7 + │ ├── ordering: +5,+6,+7 │ └── scan abc@ab - │ ├── columns: a:4 b:5 c:6 - │ └── ordering: +4,+5 + │ ├── columns: a:5 b:6 c:7 + │ └── ordering: +5,+6 └── filters (true) # The ordering is coming from the right side. @@ -1207,13 +1207,13 @@ opt SELECT * FROM abc RIGHT OUTER JOIN stu ON (c,b,a) = (s,t,u) ---- left-join (merge) - ├── columns: a:1 b:2 c:3 s:5!null t:6!null u:7!null - ├── left ordering: +7,+6,+5 + ├── columns: a:1 b:2 c:3 s:6!null t:7!null u:8!null + ├── left ordering: +8,+7,+6 ├── right ordering: +1,+2,+3 ├── scan stu@uts - │ ├── columns: s:5!null t:6!null u:7!null - │ ├── key: (5-7) - │ └── ordering: +7,+6,+5 + │ ├── columns: s:6!null t:7!null u:8!null + │ ├── key: (6-8) + │ └── ordering: +8,+7,+6 ├── sort (segmented) │ ├── columns: a:1 b:2 c:3 │ ├── ordering: +1,+2,+3 @@ -1226,18 +1226,18 @@ left-join (merge) memo SELECT * FROM abc JOIN xyz ON a=b ---- -memo (optimized, ~13KB, required=[presentation: a:1,b:2,c:3,x:5,y:6,z:7]) +memo (optimized, ~13KB, required=[presentation: a:1,b:2,c:3,x:6,y:7,z:8]) ├── G1: (inner-join G2 G3 G4) (inner-join G3 G2 G4) - │ └── [presentation: a:1,b:2,c:3,x:5,y:6,z:7] + │ └── [presentation: a:1,b:2,c:3,x:6,y:7,z:8] │ ├── best: (inner-join G3 G2 G4) │ └── cost: 2161.68 ├── G2: (select G5 G6) (select G7 G6) (select G8 G6) │ └── [] │ ├── best: (select G8 G6) │ └── cost: 1069.23 - ├── G3: (scan xyz,cols=(5-7)) (scan xyz@xy,cols=(5-7)) (scan xyz@yz,cols=(5-7)) + ├── G3: (scan xyz,cols=(6-8)) (scan xyz@xy,cols=(6-8)) (scan xyz@yz,cols=(6-8)) │ └── [] - │ ├── best: (scan xyz,cols=(5-7)) + │ ├── best: (scan xyz,cols=(6-8)) │ └── cost: 1070.02 ├── G4: (filters) ├── G5: (scan abc,cols=(1-3)) (scan abc@ab,cols=(1-3)) (scan abc@bc,cols=(1-3)) @@ -1264,18 +1264,18 @@ CREATE TABLE kfloat (k FLOAT PRIMARY KEY) memo SELECT * FROM abc JOIN kfloat ON a=k ---- -memo (optimized, ~8KB, required=[presentation: a:1,b:2,c:3,k:5]) +memo (optimized, ~10KB, required=[presentation: a:1,b:2,c:3,k:6]) ├── G1: (inner-join G2 G3 G4) (inner-join G3 G2 G4) - │ └── [presentation: a:1,b:2,c:3,k:5] + │ └── [presentation: a:1,b:2,c:3,k:6] │ ├── best: (inner-join G2 G3 G4) │ └── cost: 12120.06 ├── G2: (scan abc,cols=(1-3)) (scan abc@ab,cols=(1-3)) (scan abc@bc,cols=(1-3)) │ └── [] │ ├── best: (scan abc,cols=(1-3)) │ └── cost: 1070.02 - ├── G3: (scan kfloat) + ├── G3: (scan kfloat,cols=(6)) │ └── [] - │ ├── best: (scan kfloat) + │ ├── best: (scan kfloat,cols=(6)) │ └── cost: 1020.02 ├── G4: (filters G5) ├── G5: (eq G6 G7) @@ -1287,17 +1287,17 @@ opt SELECT * FROM abc JOIN xyz ON a=x AND a=y ---- inner-join (lookup abc@ab) - ├── columns: a:1!null b:2 c:3 x:5!null y:6!null z:7 - ├── key columns: [5] = [1] - ├── fd: (5)==(1,6), (6)==(1,5), (1)==(5,6) + ├── columns: a:1!null b:2 c:3 x:6!null y:7!null z:8 + ├── key columns: [6] = [1] + ├── fd: (6)==(1,7), (7)==(1,6), (1)==(6,7) ├── select - │ ├── columns: x:5!null y:6!null z:7 - │ ├── fd: (5)==(6), (6)==(5) + │ ├── columns: x:6!null y:7!null z:8 + │ ├── fd: (6)==(7), (7)==(6) │ ├── scan xyz@yz - │ │ ├── columns: x:5 y:6!null z:7 - │ │ └── constraint: /6/7/8: (/NULL - ] + │ │ ├── columns: x:6 y:7!null z:8 + │ │ └── constraint: /7/8/9: (/NULL - ] │ └── filters - │ └── x:5 = y:6 [outer=(5,6), constraints=(/5: (/NULL - ]; /6: (/NULL - ]), fd=(5)==(6), (6)==(5)] + │ └── x:6 = y:7 [outer=(6,7), constraints=(/6: (/NULL - ]; /7: (/NULL - ]), fd=(6)==(7), (7)==(6)] └── filters (true) # Verify multiple merge-joins can be chained. @@ -1305,26 +1305,26 @@ opt SELECT * FROM abc JOIN xyz ON a=x AND b=y RIGHT OUTER JOIN stu ON a=s ---- left-join (merge) - ├── columns: a:1 b:2 c:3 x:5 y:6 z:7 s:9!null t:10!null u:11!null - ├── left ordering: +9 + ├── columns: a:1 b:2 c:3 x:6 y:7 z:8 s:11!null t:12!null u:13!null + ├── left ordering: +11 ├── right ordering: +1 - ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2) + ├── fd: (1)==(6), (6)==(1), (2)==(7), (7)==(2) ├── scan stu - │ ├── columns: s:9!null t:10!null u:11!null - │ ├── key: (9-11) - │ └── ordering: +9 + │ ├── columns: s:11!null t:12!null u:13!null + │ ├── key: (11-13) + │ └── ordering: +11 ├── inner-join (merge) - │ ├── columns: a:1!null b:2!null c:3 x:5!null y:6!null z:7 + │ ├── columns: a:1!null b:2!null c:3 x:6!null y:7!null z:8 │ ├── left ordering: +1,+2 - │ ├── right ordering: +5,+6 - │ ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2) - │ ├── ordering: +(1|5) [actual: +1] + │ ├── right ordering: +6,+7 + │ ├── fd: (1)==(6), (6)==(1), (2)==(7), (7)==(2) + │ ├── ordering: +(1|6) [actual: +1] │ ├── scan abc@ab │ │ ├── columns: a:1 b:2 c:3 │ │ └── ordering: +1,+2 │ ├── scan xyz@xy - │ │ ├── columns: x:5 y:6 z:7 - │ │ └── ordering: +5,+6 + │ │ ├── columns: x:6 y:7 z:8 + │ │ └── ordering: +6,+7 │ └── filters (true) └── filters (true) @@ -1332,26 +1332,26 @@ opt SELECT * FROM abc JOIN xyz ON a=x AND b=y RIGHT OUTER JOIN stu ON a=u AND y=t ---- left-join (merge) - ├── columns: a:1 b:2 c:3 x:5 y:6 z:7 s:9!null t:10!null u:11!null - ├── left ordering: +11,+10 - ├── right ordering: +1,+6 - ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2) + ├── columns: a:1 b:2 c:3 x:6 y:7 z:8 s:11!null t:12!null u:13!null + ├── left ordering: +13,+12 + ├── right ordering: +1,+7 + ├── fd: (1)==(6), (6)==(1), (2)==(7), (7)==(2) ├── scan stu@uts - │ ├── columns: s:9!null t:10!null u:11!null - │ ├── key: (9-11) - │ └── ordering: +11,+10 + │ ├── columns: s:11!null t:12!null u:13!null + │ ├── key: (11-13) + │ └── ordering: +13,+12 ├── inner-join (merge) - │ ├── columns: a:1!null b:2!null c:3 x:5!null y:6!null z:7 + │ ├── columns: a:1!null b:2!null c:3 x:6!null y:7!null z:8 │ ├── left ordering: +1,+2 - │ ├── right ordering: +5,+6 - │ ├── fd: (1)==(5), (5)==(1), (2)==(6), (6)==(2) - │ ├── ordering: +(1|5),+(2|6) [actual: +1,+2] + │ ├── right ordering: +6,+7 + │ ├── fd: (1)==(6), (6)==(1), (2)==(7), (7)==(2) + │ ├── ordering: +(1|6),+(2|7) [actual: +1,+2] │ ├── scan abc@ab │ │ ├── columns: a:1 b:2 c:3 │ │ └── ordering: +1,+2 │ ├── scan xyz@xy - │ │ ├── columns: x:5 y:6 z:7 - │ │ └── ordering: +5,+6 + │ │ ├── columns: x:6 y:7 z:8 + │ │ └── ordering: +6,+7 │ └── filters (true) └── filters (true) @@ -1387,9 +1387,9 @@ opt SELECT a,b,n,m FROM small JOIN abcd ON a=m ---- inner-join (lookup abcd@secondary) - ├── columns: a:4!null b:5 n:2 m:1!null - ├── key columns: [1] = [4] - ├── fd: (1)==(4), (4)==(1) + ├── columns: a:5!null b:6 n:2 m:1!null + ├── key columns: [1] = [5] + ├── fd: (1)==(5), (5)==(1) ├── scan small │ └── columns: m:1 n:2 └── filters (true) @@ -1399,8 +1399,8 @@ opt SELECT a,b,n,m FROM small LEFT JOIN abcd ON a=m ---- left-join (lookup abcd@secondary) - ├── columns: a:4 b:5 n:2 m:1 - ├── key columns: [1] = [4] + ├── columns: a:5 b:6 n:2 m:1 + ├── key columns: [1] = [5] ├── scan small │ └── columns: m:1 n:2 └── filters (true) @@ -1410,14 +1410,14 @@ opt SELECT * FROM small JOIN abcd ON a=m ---- inner-join (lookup abcd) - ├── columns: m:1!null n:2 a:4!null b:5 c:6 - ├── key columns: [7] = [7] + ├── columns: m:1!null n:2 a:5!null b:6 c:7 + ├── key columns: [8] = [8] ├── lookup columns are key - ├── fd: (1)==(4), (4)==(1) + ├── fd: (1)==(5), (5)==(1) ├── inner-join (lookup abcd@secondary) - │ ├── columns: m:1!null n:2 a:4!null b:5 abcd.rowid:7!null - │ ├── key columns: [1] = [4] - │ ├── fd: (7)-->(4,5), (1)==(4), (4)==(1) + │ ├── columns: m:1!null n:2 a:5!null b:6 abcd.rowid:8!null + │ ├── key columns: [1] = [5] + │ ├── fd: (8)-->(5,6), (1)==(5), (5)==(1) │ ├── scan small │ │ └── columns: m:1 n:2 │ └── filters (true) @@ -1428,13 +1428,13 @@ opt SELECT * FROM small LEFT JOIN abcd ON a=m ---- left-join (lookup abcd) - ├── columns: m:1 n:2 a:4 b:5 c:6 - ├── key columns: [7] = [7] + ├── columns: m:1 n:2 a:5 b:6 c:7 + ├── key columns: [8] = [8] ├── lookup columns are key ├── left-join (lookup abcd@secondary) - │ ├── columns: m:1 n:2 a:4 b:5 abcd.rowid:7 - │ ├── key columns: [1] = [4] - │ ├── fd: (7)-->(4,5) + │ ├── columns: m:1 n:2 a:5 b:6 abcd.rowid:8 + │ ├── key columns: [1] = [5] + │ ├── fd: (8)-->(5,6) │ ├── scan small │ │ └── columns: m:1 n:2 │ └── filters (true) @@ -1445,18 +1445,18 @@ opt SELECT * FROM small JOIN abcd ON a=m AND b>n ---- inner-join (lookup abcd) - ├── columns: m:1!null n:2!null a:4!null b:5!null c:6 - ├── key columns: [7] = [7] + ├── columns: m:1!null n:2!null a:5!null b:6!null c:7 + ├── key columns: [8] = [8] ├── lookup columns are key - ├── fd: (1)==(4), (4)==(1) + ├── fd: (1)==(5), (5)==(1) ├── inner-join (lookup abcd@secondary) - │ ├── columns: m:1!null n:2!null a:4!null b:5!null abcd.rowid:7!null - │ ├── key columns: [1] = [4] - │ ├── fd: (7)-->(4,5), (1)==(4), (4)==(1) + │ ├── columns: m:1!null n:2!null a:5!null b:6!null abcd.rowid:8!null + │ ├── key columns: [1] = [5] + │ ├── fd: (8)-->(5,6), (1)==(5), (5)==(1) │ ├── scan small │ │ └── columns: m:1 n:2 │ └── filters - │ └── b:5 > n:2 [outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ])] + │ └── b:6 > n:2 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] └── filters (true) # Non-covering case, extra filter bound by index, left join. @@ -1464,17 +1464,17 @@ opt SELECT * FROM small LEFT JOIN abcd ON a=m AND b>n ---- left-join (lookup abcd) - ├── columns: m:1 n:2 a:4 b:5 c:6 - ├── key columns: [7] = [7] + ├── columns: m:1 n:2 a:5 b:6 c:7 + ├── key columns: [8] = [8] ├── lookup columns are key ├── left-join (lookup abcd@secondary) - │ ├── columns: m:1 n:2 a:4 b:5 abcd.rowid:7 - │ ├── key columns: [1] = [4] - │ ├── fd: (7)-->(4,5) + │ ├── columns: m:1 n:2 a:5 b:6 abcd.rowid:8 + │ ├── key columns: [1] = [5] + │ ├── fd: (8)-->(5,6) │ ├── scan small │ │ └── columns: m:1 n:2 │ └── filters - │ └── b:5 > n:2 [outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ])] + │ └── b:6 > n:2 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] └── filters (true) # Non-covering case, extra filter not bound by index. @@ -1482,19 +1482,19 @@ opt SELECT * FROM small JOIN abcd ON a=m AND c>n ---- inner-join (lookup abcd) - ├── columns: m:1!null n:2!null a:4!null b:5 c:6!null - ├── key columns: [7] = [7] + ├── columns: m:1!null n:2!null a:5!null b:6 c:7!null + ├── key columns: [8] = [8] ├── lookup columns are key - ├── fd: (1)==(4), (4)==(1) + ├── fd: (1)==(5), (5)==(1) ├── inner-join (lookup abcd@secondary) - │ ├── columns: m:1!null n:2 a:4!null b:5 abcd.rowid:7!null - │ ├── key columns: [1] = [4] - │ ├── fd: (7)-->(4,5), (1)==(4), (4)==(1) + │ ├── columns: m:1!null n:2 a:5!null b:6 abcd.rowid:8!null + │ ├── key columns: [1] = [5] + │ ├── fd: (8)-->(5,6), (1)==(5), (5)==(1) │ ├── scan small │ │ └── columns: m:1 n:2 │ └── filters (true) └── filters - └── c:6 > n:2 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + └── c:7 > n:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] # Non-covering case, extra filter not bound by index, left join. # In this case, we can't yet convert to a lookup join (see @@ -1503,14 +1503,14 @@ opt SELECT * FROM small LEFT JOIN abcd ON a=m AND c>n ---- right-join (hash) - ├── columns: m:1 n:2 a:4 b:5 c:6 + ├── columns: m:1 n:2 a:5 b:6 c:7 ├── scan abcd - │ └── columns: a:4 b:5 c:6 + │ └── columns: a:5 b:6 c:7 ├── scan small │ └── columns: m:1 n:2 └── filters - ├── a:4 = m:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] - └── c:6 > n:2 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + ├── a:5 = m:1 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + └── c:7 > n:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] # Verify rule application when we can do a lookup join on both sides. @@ -1523,34 +1523,34 @@ GenerateLookupJoins ================================================================================ Source expression: inner-join (hash) - ├── columns: a:1!null b:2 c:3 x:5!null y:6!null z:7 - ├── fd: (5)==(1,6), (6)==(1,5), (1)==(5,6) + ├── columns: a:1!null b:2 c:3 x:6!null y:7!null z:8 + ├── fd: (6)==(1,7), (7)==(1,6), (1)==(6,7) ├── select - │ ├── columns: x:5!null y:6!null z:7 - │ ├── fd: (5)==(6), (6)==(5) + │ ├── columns: x:6!null y:7!null z:8 + │ ├── fd: (6)==(7), (7)==(6) │ ├── scan xyz@yz - │ │ ├── columns: x:5 y:6!null z:7 - │ │ └── constraint: /6/7/8: (/NULL - ] + │ │ ├── columns: x:6 y:7!null z:8 + │ │ └── constraint: /7/8/9: (/NULL - ] │ └── filters - │ └── x:5 = y:6 [outer=(5,6), constraints=(/5: (/NULL - ]; /6: (/NULL - ]), fd=(5)==(6), (6)==(5)] + │ └── x:6 = y:7 [outer=(6,7), constraints=(/6: (/NULL - ]; /7: (/NULL - ]), fd=(6)==(7), (7)==(6)] ├── scan abc │ └── columns: a:1 b:2 c:3 └── filters - └── a:1 = x:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + └── a:1 = x:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] New expression 1 of 1: inner-join (lookup abc@ab) - ├── columns: a:1!null b:2 c:3 x:5!null y:6!null z:7 - ├── key columns: [5] = [1] - ├── fd: (5)==(1,6), (6)==(1,5), (1)==(5,6) + ├── columns: a:1!null b:2 c:3 x:6!null y:7!null z:8 + ├── key columns: [6] = [1] + ├── fd: (6)==(1,7), (7)==(1,6), (1)==(6,7) ├── select - │ ├── columns: x:5!null y:6!null z:7 - │ ├── fd: (5)==(6), (6)==(5) + │ ├── columns: x:6!null y:7!null z:8 + │ ├── fd: (6)==(7), (7)==(6) │ ├── scan xyz@yz - │ │ ├── columns: x:5 y:6!null z:7 - │ │ └── constraint: /6/7/8: (/NULL - ] + │ │ ├── columns: x:6 y:7!null z:8 + │ │ └── constraint: /7/8/9: (/NULL - ] │ └── filters - │ └── x:5 = y:6 [outer=(5,6), constraints=(/5: (/NULL - ]; /6: (/NULL - ]), fd=(5)==(6), (6)==(5)] + │ └── x:6 = y:7 [outer=(6,7), constraints=(/6: (/NULL - ]; /7: (/NULL - ]), fd=(6)==(7), (7)==(6)] └── filters (true) ---- ---- @@ -1565,14 +1565,14 @@ GenerateLookupJoins ================================================================================ Source expression: inner-join (hash) - ├── columns: a:1!null b:2 c:3 x:5 y:6 z:7!null - ├── fd: (1)==(7), (7)==(1) + ├── columns: a:1!null b:2 c:3 x:6 y:7 z:8!null + ├── fd: (1)==(8), (8)==(1) ├── scan abc │ └── columns: a:1 b:2 c:3 ├── scan xyz - │ └── columns: x:5 y:6 z:7 + │ └── columns: x:6 y:7 z:8 └── filters - └── a:1 = z:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + └── a:1 = z:8 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] No new expressions. @@ -1581,22 +1581,22 @@ GenerateLookupJoins ================================================================================ Source expression: inner-join (hash) - ├── columns: a:1!null b:2 c:3 x:5 y:6 z:7!null - ├── fd: (1)==(7), (7)==(1) + ├── columns: a:1!null b:2 c:3 x:6 y:7 z:8!null + ├── fd: (1)==(8), (8)==(1) ├── scan xyz - │ └── columns: x:5 y:6 z:7 + │ └── columns: x:6 y:7 z:8 ├── scan abc │ └── columns: a:1 b:2 c:3 └── filters - └── a:1 = z:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + └── a:1 = z:8 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] New expression 1 of 1: inner-join (lookup abc@ab) - ├── columns: a:1!null b:2 c:3 x:5 y:6 z:7!null - ├── key columns: [7] = [1] - ├── fd: (1)==(7), (7)==(1) + ├── columns: a:1!null b:2 c:3 x:6 y:7 z:8!null + ├── key columns: [8] = [1] + ├── fd: (1)==(8), (8)==(1) ├── scan xyz - │ └── columns: x:5 y:6 z:7 + │ └── columns: x:6 y:7 z:8 └── filters (true) ---- ---- @@ -1610,20 +1610,20 @@ GenerateLookupJoins ================================================================================ Source expression: left-join (hash) - ├── columns: a:1 b:2 c:3 x:5 y:6 z:7 + ├── columns: a:1 b:2 c:3 x:6 y:7 z:8 ├── scan xyz - │ └── columns: x:5 y:6 z:7 + │ └── columns: x:6 y:7 z:8 ├── scan abc │ └── columns: a:1 b:2 c:3 └── filters - └── a:1 = z:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + └── a:1 = z:8 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] New expression 1 of 1: left-join (lookup abc@ab) - ├── columns: a:1 b:2 c:3 x:5 y:6 z:7 - ├── key columns: [7] = [1] + ├── columns: a:1 b:2 c:3 x:6 y:7 z:8 + ├── key columns: [8] = [1] ├── scan xyz - │ └── columns: x:5 y:6 z:7 + │ └── columns: x:6 y:7 z:8 └── filters (true) ---- ---- @@ -1638,20 +1638,20 @@ GenerateLookupJoins ================================================================================ Source expression: inner-join (hash) - ├── columns: a:1 b:2 c:3!null x:5!null y:6 z:7 - ├── fd: (3)==(5), (5)==(3) + ├── columns: a:1 b:2 c:3!null x:6!null y:7 z:8 + ├── fd: (3)==(6), (6)==(3) ├── scan abc │ └── columns: a:1 b:2 c:3 ├── scan xyz - │ └── columns: x:5 y:6 z:7 + │ └── columns: x:6 y:7 z:8 └── filters - └── c:3 = x:5 [outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ]), fd=(3)==(5), (5)==(3)] + └── c:3 = x:6 [outer=(3,6), constraints=(/3: (/NULL - ]; /6: (/NULL - ]), fd=(3)==(6), (6)==(3)] New expression 1 of 1: inner-join (lookup xyz@xy) - ├── columns: a:1 b:2 c:3!null x:5!null y:6 z:7 - ├── key columns: [3] = [5] - ├── fd: (3)==(5), (5)==(3) + ├── columns: a:1 b:2 c:3!null x:6!null y:7 z:8 + ├── key columns: [3] = [6] + ├── fd: (3)==(6), (6)==(3) ├── scan abc │ └── columns: a:1 b:2 c:3 └── filters (true) @@ -1661,14 +1661,14 @@ GenerateLookupJoins ================================================================================ Source expression: inner-join (hash) - ├── columns: a:1 b:2 c:3!null x:5!null y:6 z:7 - ├── fd: (3)==(5), (5)==(3) + ├── columns: a:1 b:2 c:3!null x:6!null y:7 z:8 + ├── fd: (3)==(6), (6)==(3) ├── scan xyz - │ └── columns: x:5 y:6 z:7 + │ └── columns: x:6 y:7 z:8 ├── scan abc │ └── columns: a:1 b:2 c:3 └── filters - └── c:3 = x:5 [outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ]), fd=(3)==(5), (5)==(3)] + └── c:3 = x:6 [outer=(3,6), constraints=(/3: (/NULL - ]; /6: (/NULL - ]), fd=(3)==(6), (6)==(3)] No new expressions. ---- @@ -1683,18 +1683,18 @@ GenerateLookupJoins ================================================================================ Source expression: left-join (hash) - ├── columns: a:1 b:2 c:3 x:5 y:6 z:7 + ├── columns: a:1 b:2 c:3 x:6 y:7 z:8 ├── scan abc │ └── columns: a:1 b:2 c:3 ├── scan xyz - │ └── columns: x:5 y:6 z:7 + │ └── columns: x:6 y:7 z:8 └── filters - └── c:3 = x:5 [outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ]), fd=(3)==(5), (5)==(3)] + └── c:3 = x:6 [outer=(3,6), constraints=(/3: (/NULL - ]; /6: (/NULL - ]), fd=(3)==(6), (6)==(3)] New expression 1 of 1: left-join (lookup xyz@xy) - ├── columns: a:1 b:2 c:3 x:5 y:6 z:7 - ├── key columns: [3] = [5] + ├── columns: a:1 b:2 c:3 x:6 y:7 z:8 + ├── key columns: [3] = [6] ├── scan abc │ └── columns: a:1 b:2 c:3 └── filters (true) @@ -1711,13 +1711,13 @@ GenerateLookupJoins ================================================================================ Source expression: left-join (hash) - ├── columns: a:1 b:2 c:3 x:5 y:6 z:7 + ├── columns: a:1 b:2 c:3 x:6 y:7 z:8 ├── scan xyz - │ └── columns: x:5 y:6 z:7 + │ └── columns: x:6 y:7 z:8 ├── scan abc │ └── columns: a:1 b:2 c:3 └── filters - └── c:3 = x:5 [outer=(3,5), constraints=(/3: (/NULL - ]; /5: (/NULL - ]), fd=(3)==(5), (5)==(3)] + └── c:3 = x:6 [outer=(3,6), constraints=(/3: (/NULL - ]; /6: (/NULL - ]), fd=(3)==(6), (6)==(3)] No new expressions. ---- @@ -1727,18 +1727,18 @@ No new expressions. memo SELECT a,b,n,m FROM small INNER HASH JOIN abcd ON a=m ---- -memo (optimized, ~8KB, required=[presentation: a:4,b:5,n:2,m:1]) +memo (optimized, ~8KB, required=[presentation: a:5,b:6,n:2,m:1]) ├── G1: (inner-join G2 G3 G4) (inner-join G3 G2 G4) - │ └── [presentation: a:4,b:5,n:2,m:1] + │ └── [presentation: a:5,b:6,n:2,m:1] │ ├── best: (inner-join G2 G3 G4) │ └── cost: 1079.17 ├── G2: (scan small,cols=(1,2)) │ └── [] │ ├── best: (scan small,cols=(1,2)) │ └── cost: 10.52 - ├── G3: (scan abcd,cols=(4,5)) (scan abcd@secondary,cols=(4,5)) + ├── G3: (scan abcd,cols=(5,6)) (scan abcd@secondary,cols=(5,6)) │ └── [] - │ ├── best: (scan abcd@secondary,cols=(4,5)) + │ ├── best: (scan abcd@secondary,cols=(5,6)) │ └── cost: 1050.02 ├── G4: (filters G5) ├── G5: (eq G6 G7) @@ -1751,7 +1751,7 @@ SELECT m, n FROM small WHERE EXISTS(SELECT 1 FROM abcd WHERE m = a) ---- semi-join (lookup abcd@secondary) ├── columns: m:1 n:2 - ├── key columns: [1] = [4] + ├── key columns: [1] = [5] ├── scan small │ └── columns: m:1 n:2 └── filters (true) @@ -1766,10 +1766,10 @@ semi-join (hash) ├── scan small │ └── columns: m:1 n:2 ├── scan abcd - │ └── columns: a:4 c:6 + │ └── columns: a:5 c:7 └── filters - ├── m:1 = a:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] - └── n:2 = c:6 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + ├── m:1 = a:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + └── n:2 = c:7 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] # Lookup anti-join with index that contains all columns in the join condition. opt @@ -1777,7 +1777,7 @@ SELECT m, n FROM small WHERE NOT EXISTS(SELECT 1 FROM abcd WHERE m = a) ---- anti-join (lookup abcd@secondary) ├── columns: m:1 n:2 - ├── key columns: [1] = [4] + ├── key columns: [1] = [5] ├── scan small │ └── columns: m:1 n:2 └── filters (true) @@ -1792,10 +1792,10 @@ anti-join (hash) ├── scan small │ └── columns: m:1 n:2 ├── scan abcd - │ └── columns: a:4 c:6 + │ └── columns: a:5 c:7 └── filters - ├── m:1 = a:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] - └── n:2 = c:6 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + ├── m:1 = a:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + └── n:2 = c:7 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] # -------------------------------------------------- # GenerateLookupJoinsWithFilter @@ -1810,43 +1810,43 @@ opt SELECT a,b,n,m FROM small JOIN abcd ON a=m AND b>1 ---- inner-join (lookup abcd@secondary) - ├── columns: a:4!null b:5!null n:2 m:1!null - ├── key columns: [1] = [4] - ├── fd: (1)==(4), (4)==(1) + ├── columns: a:5!null b:6!null n:2 m:1!null + ├── key columns: [1] = [5] + ├── fd: (1)==(5), (5)==(1) ├── scan small │ └── columns: m:1 n:2 └── filters - └── b:5 > 1 [outer=(5), constraints=(/5: [/2 - ]; tight)] + └── b:6 > 1 [outer=(6), constraints=(/6: [/2 - ]; tight)] # Covering case, left-join. opt SELECT a,b,n,m FROM small LEFT JOIN abcd ON a=m AND b>1 ---- left-join (lookup abcd@secondary) - ├── columns: a:4 b:5 n:2 m:1 - ├── key columns: [1] = [4] + ├── columns: a:5 b:6 n:2 m:1 + ├── key columns: [1] = [5] ├── scan small │ └── columns: m:1 n:2 └── filters - └── b:5 > 1 [outer=(5), constraints=(/5: [/2 - ]; tight)] + └── b:6 > 1 [outer=(6), constraints=(/6: [/2 - ]; tight)] # Non-covering case. opt SELECT * FROM small JOIN abcd ON a=m AND b>1 ---- inner-join (lookup abcd) - ├── columns: m:1!null n:2 a:4!null b:5!null c:6 - ├── key columns: [7] = [7] + ├── columns: m:1!null n:2 a:5!null b:6!null c:7 + ├── key columns: [8] = [8] ├── lookup columns are key - ├── fd: (1)==(4), (4)==(1) + ├── fd: (1)==(5), (5)==(1) ├── inner-join (lookup abcd@secondary) - │ ├── columns: m:1!null n:2 a:4!null b:5!null abcd.rowid:7!null - │ ├── key columns: [1] = [4] - │ ├── fd: (7)-->(4,5), (1)==(4), (4)==(1) + │ ├── columns: m:1!null n:2 a:5!null b:6!null abcd.rowid:8!null + │ ├── key columns: [1] = [5] + │ ├── fd: (8)-->(5,6), (1)==(5), (5)==(1) │ ├── scan small │ │ └── columns: m:1 n:2 │ └── filters - │ └── b:5 > 1 [outer=(5), constraints=(/5: [/2 - ]; tight)] + │ └── b:6 > 1 [outer=(6), constraints=(/6: [/2 - ]; tight)] └── filters (true) # Non-covering case, left join. @@ -1854,17 +1854,17 @@ opt SELECT * FROM small LEFT JOIN abcd ON a=m AND b>1 ---- left-join (lookup abcd) - ├── columns: m:1 n:2 a:4 b:5 c:6 - ├── key columns: [7] = [7] + ├── columns: m:1 n:2 a:5 b:6 c:7 + ├── key columns: [8] = [8] ├── lookup columns are key ├── left-join (lookup abcd@secondary) - │ ├── columns: m:1 n:2 a:4 b:5 abcd.rowid:7 - │ ├── key columns: [1] = [4] - │ ├── fd: (7)-->(4,5) + │ ├── columns: m:1 n:2 a:5 b:6 abcd.rowid:8 + │ ├── key columns: [1] = [5] + │ ├── fd: (8)-->(5,6) │ ├── scan small │ │ └── columns: m:1 n:2 │ └── filters - │ └── b:5 > 1 [outer=(5), constraints=(/5: [/2 - ]; tight)] + │ └── b:6 > 1 [outer=(6), constraints=(/6: [/2 - ]; tight)] └── filters (true) # Non-covering case, extra filter bound by index. @@ -1872,19 +1872,19 @@ opt SELECT * FROM small JOIN abcd ON a=m AND b>n AND b>1 ---- inner-join (lookup abcd) - ├── columns: m:1!null n:2!null a:4!null b:5!null c:6 - ├── key columns: [7] = [7] + ├── columns: m:1!null n:2!null a:5!null b:6!null c:7 + ├── key columns: [8] = [8] ├── lookup columns are key - ├── fd: (1)==(4), (4)==(1) + ├── fd: (1)==(5), (5)==(1) ├── inner-join (lookup abcd@secondary) - │ ├── columns: m:1!null n:2!null a:4!null b:5!null abcd.rowid:7!null - │ ├── key columns: [1] = [4] - │ ├── fd: (7)-->(4,5), (1)==(4), (4)==(1) + │ ├── columns: m:1!null n:2!null a:5!null b:6!null abcd.rowid:8!null + │ ├── key columns: [1] = [5] + │ ├── fd: (8)-->(5,6), (1)==(5), (5)==(1) │ ├── scan small │ │ └── columns: m:1 n:2 │ └── filters - │ ├── b:5 > n:2 [outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ])] - │ └── b:5 > 1 [outer=(5), constraints=(/5: [/2 - ]; tight)] + │ ├── b:6 > n:2 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + │ └── b:6 > 1 [outer=(6), constraints=(/6: [/2 - ]; tight)] └── filters (true) # Non-covering case, extra filter bound by index, left join. @@ -1892,18 +1892,18 @@ opt SELECT * FROM small LEFT JOIN abcd ON a=m AND b>n AND b>1 ---- left-join (lookup abcd) - ├── columns: m:1 n:2 a:4 b:5 c:6 - ├── key columns: [7] = [7] + ├── columns: m:1 n:2 a:5 b:6 c:7 + ├── key columns: [8] = [8] ├── lookup columns are key ├── left-join (lookup abcd@secondary) - │ ├── columns: m:1 n:2 a:4 b:5 abcd.rowid:7 - │ ├── key columns: [1] = [4] - │ ├── fd: (7)-->(4,5) + │ ├── columns: m:1 n:2 a:5 b:6 abcd.rowid:8 + │ ├── key columns: [1] = [5] + │ ├── fd: (8)-->(5,6) │ ├── scan small │ │ └── columns: m:1 n:2 │ └── filters - │ ├── b:5 > n:2 [outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ])] - │ └── b:5 > 1 [outer=(5), constraints=(/5: [/2 - ]; tight)] + │ ├── b:6 > n:2 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + │ └── b:6 > 1 [outer=(6), constraints=(/6: [/2 - ]; tight)] └── filters (true) # Non-covering case, extra filter not bound by index. @@ -1911,20 +1911,20 @@ opt SELECT * FROM small JOIN abcd ON a=m AND c>n AND b>1 ---- inner-join (lookup abcd) - ├── columns: m:1!null n:2!null a:4!null b:5!null c:6!null - ├── key columns: [7] = [7] + ├── columns: m:1!null n:2!null a:5!null b:6!null c:7!null + ├── key columns: [8] = [8] ├── lookup columns are key - ├── fd: (1)==(4), (4)==(1) + ├── fd: (1)==(5), (5)==(1) ├── inner-join (lookup abcd@secondary) - │ ├── columns: m:1!null n:2 a:4!null b:5!null abcd.rowid:7!null - │ ├── key columns: [1] = [4] - │ ├── fd: (7)-->(4,5), (1)==(4), (4)==(1) + │ ├── columns: m:1!null n:2 a:5!null b:6!null abcd.rowid:8!null + │ ├── key columns: [1] = [5] + │ ├── fd: (8)-->(5,6), (1)==(5), (5)==(1) │ ├── scan small │ │ └── columns: m:1 n:2 │ └── filters - │ └── b:5 > 1 [outer=(5), constraints=(/5: [/2 - ]; tight)] + │ └── b:6 > 1 [outer=(6), constraints=(/6: [/2 - ]; tight)] └── filters - └── c:6 > n:2 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + └── c:7 > n:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] # Non-covering case, extra filter not bound by index, left join. # In this case, we can't yet convert to a lookup join (see @@ -1933,39 +1933,39 @@ opt SELECT * FROM small LEFT JOIN abcd ON a=m AND c>n AND b>1 ---- right-join (hash) - ├── columns: m:1 n:2 a:4 b:5 c:6 + ├── columns: m:1 n:2 a:5 b:6 c:7 ├── select - │ ├── columns: a:4 b:5!null c:6 + │ ├── columns: a:5 b:6!null c:7 │ ├── scan abcd - │ │ └── columns: a:4 b:5 c:6 + │ │ └── columns: a:5 b:6 c:7 │ └── filters - │ └── b:5 > 1 [outer=(5), constraints=(/5: [/2 - ]; tight)] + │ └── b:6 > 1 [outer=(6), constraints=(/6: [/2 - ]; tight)] ├── scan small │ └── columns: m:1 n:2 └── filters - ├── a:4 = m:1 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] - └── c:6 > n:2 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ])] + ├── a:5 = m:1 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + └── c:7 > n:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ])] # Constant columns are projected and used by lookup joiner. opt SELECT * FROM small INNER JOIN abcde ON a=m AND b=10 ---- inner-join (lookup abcde) - ├── columns: m:1!null n:2 a:4!null b:5!null c:6 d:7 e:8 - ├── key columns: [9] = [9] + ├── columns: m:1!null n:2 a:5!null b:6!null c:7 d:8 e:9 + ├── key columns: [10] = [10] ├── lookup columns are key - ├── fd: ()-->(5), (1)==(4), (4)==(1) + ├── fd: ()-->(6), (1)==(5), (5)==(1) ├── inner-join (lookup abcde@secondary) - │ ├── columns: m:1!null n:2 a:4!null b:5!null c:6 abcde.rowid:9!null - │ ├── key columns: [1 10] = [4 5] - │ ├── fd: ()-->(5), (9)-->(4,6), (1)==(4), (4)==(1) + │ ├── columns: m:1!null n:2 a:5!null b:6!null c:7 abcde.rowid:10!null + │ ├── key columns: [1 12] = [5 6] + │ ├── fd: ()-->(6), (10)-->(5,7), (1)==(5), (5)==(1) │ ├── project - │ │ ├── columns: "project_const_col_@5":10!null m:1 n:2 - │ │ ├── fd: ()-->(10) + │ │ ├── columns: "project_const_col_@6":12!null m:1 n:2 + │ │ ├── fd: ()-->(12) │ │ ├── scan small │ │ │ └── columns: m:1 n:2 │ │ └── projections - │ │ └── 10 [as="project_const_col_@5":10] + │ │ └── 10 [as="project_const_col_@6":12] │ └── filters (true) └── filters (true) @@ -1974,18 +1974,18 @@ opt SELECT * FROM small INNER JOIN abcde ON a=m AND c=10 ---- inner-join (lookup abcde) - ├── columns: m:1!null n:2 a:4!null b:5 c:6!null d:7 e:8 - ├── key columns: [9] = [9] + ├── columns: m:1!null n:2 a:5!null b:6 c:7!null d:8 e:9 + ├── key columns: [10] = [10] ├── lookup columns are key - ├── fd: ()-->(6), (1)==(4), (4)==(1) + ├── fd: ()-->(7), (1)==(5), (5)==(1) ├── inner-join (lookup abcde@secondary) - │ ├── columns: m:1!null n:2 a:4!null b:5 c:6!null abcde.rowid:9!null - │ ├── key columns: [1] = [4] - │ ├── fd: ()-->(6), (9)-->(4,5), (1)==(4), (4)==(1) + │ ├── columns: m:1!null n:2 a:5!null b:6 c:7!null abcde.rowid:10!null + │ ├── key columns: [1] = [5] + │ ├── fd: ()-->(7), (10)-->(5,6), (1)==(5), (5)==(1) │ ├── scan small │ │ └── columns: m:1 n:2 │ └── filters - │ └── c:6 = 10 [outer=(6), constraints=(/6: [/10 - /10]; tight), fd=()-->(6)] + │ └── c:7 = 10 [outer=(7), constraints=(/7: [/10 - /10]; tight), fd=()-->(7)] └── filters (true) # Multiple constant columns projected and used by lookup joiner. @@ -1993,22 +1993,22 @@ opt SELECT * FROM small INNER JOIN abcde ON a=m AND b=10 AND c=10 ---- inner-join (lookup abcde) - ├── columns: m:1!null n:2 a:4!null b:5!null c:6!null d:7 e:8 - ├── key columns: [9] = [9] + ├── columns: m:1!null n:2 a:5!null b:6!null c:7!null d:8 e:9 + ├── key columns: [10] = [10] ├── lookup columns are key - ├── fd: ()-->(5,6), (1)==(4), (4)==(1) + ├── fd: ()-->(6,7), (1)==(5), (5)==(1) ├── inner-join (lookup abcde@secondary) - │ ├── columns: m:1!null n:2 a:4!null b:5!null c:6!null abcde.rowid:9!null - │ ├── key columns: [1 10 11] = [4 5 6] - │ ├── fd: ()-->(5,6), (9)-->(4), (1)==(4), (4)==(1) + │ ├── columns: m:1!null n:2 a:5!null b:6!null c:7!null abcde.rowid:10!null + │ ├── key columns: [1 12 13] = [5 6 7] + │ ├── fd: ()-->(6,7), (10)-->(5), (1)==(5), (5)==(1) │ ├── project - │ │ ├── columns: "project_const_col_@5":10!null "project_const_col_@6":11!null m:1 n:2 - │ │ ├── fd: ()-->(10,11) + │ │ ├── columns: "project_const_col_@6":12!null "project_const_col_@7":13!null m:1 n:2 + │ │ ├── fd: ()-->(12,13) │ │ ├── scan small │ │ │ └── columns: m:1 n:2 │ │ └── projections - │ │ ├── 10 [as="project_const_col_@5":10] - │ │ └── 10 [as="project_const_col_@6":11] + │ │ ├── 10 [as="project_const_col_@6":12] + │ │ └── 10 [as="project_const_col_@7":13] │ └── filters (true) └── filters (true) @@ -2017,43 +2017,43 @@ opt SELECT * FROM small INNER JOIN abcde ON a=m AND b=10 AND c=10 AND d=10 ---- inner-join (lookup abcde) - ├── columns: m:1!null n:2 a:4!null b:5!null c:6!null d:7!null e:8 - ├── key columns: [9] = [9] + ├── columns: m:1!null n:2 a:5!null b:6!null c:7!null d:8!null e:9 + ├── key columns: [10] = [10] ├── lookup columns are key - ├── fd: ()-->(5-7), (1)==(4), (4)==(1) + ├── fd: ()-->(6-8), (1)==(5), (5)==(1) ├── inner-join (lookup abcde@secondary) - │ ├── columns: m:1!null n:2 a:4!null b:5!null c:6!null abcde.rowid:9!null - │ ├── key columns: [1 10 11] = [4 5 6] - │ ├── fd: ()-->(5,6), (9)-->(4), (1)==(4), (4)==(1) + │ ├── columns: m:1!null n:2 a:5!null b:6!null c:7!null abcde.rowid:10!null + │ ├── key columns: [1 12 13] = [5 6 7] + │ ├── fd: ()-->(6,7), (10)-->(5), (1)==(5), (5)==(1) │ ├── project - │ │ ├── columns: "project_const_col_@5":10!null "project_const_col_@6":11!null m:1 n:2 - │ │ ├── fd: ()-->(10,11) + │ │ ├── columns: "project_const_col_@6":12!null "project_const_col_@7":13!null m:1 n:2 + │ │ ├── fd: ()-->(12,13) │ │ ├── scan small │ │ │ └── columns: m:1 n:2 │ │ └── projections - │ │ ├── 10 [as="project_const_col_@5":10] - │ │ └── 10 [as="project_const_col_@6":11] + │ │ ├── 10 [as="project_const_col_@6":12] + │ │ └── 10 [as="project_const_col_@7":13] │ └── filters (true) └── filters - └── d:7 = 10 [outer=(7), constraints=(/7: [/10 - /10]; tight), fd=()-->(7)] + └── d:8 = 10 [outer=(8), constraints=(/8: [/10 - /10]; tight), fd=()-->(8)] # Non equality filters don't trigger constant projection. opt SELECT * FROM small INNER JOIN abcde ON a=m AND b<10 ---- inner-join (lookup abcde) - ├── columns: m:1!null n:2 a:4!null b:5!null c:6 d:7 e:8 - ├── key columns: [9] = [9] + ├── columns: m:1!null n:2 a:5!null b:6!null c:7 d:8 e:9 + ├── key columns: [10] = [10] ├── lookup columns are key - ├── fd: (1)==(4), (4)==(1) + ├── fd: (1)==(5), (5)==(1) ├── inner-join (lookup abcde@secondary) - │ ├── columns: m:1!null n:2 a:4!null b:5!null c:6 abcde.rowid:9!null - │ ├── key columns: [1] = [4] - │ ├── fd: (9)-->(4-6), (1)==(4), (4)==(1) + │ ├── columns: m:1!null n:2 a:5!null b:6!null c:7 abcde.rowid:10!null + │ ├── key columns: [1] = [5] + │ ├── fd: (10)-->(5-7), (1)==(5), (5)==(1) │ ├── scan small │ │ └── columns: m:1 n:2 │ └── filters - │ └── b:5 < 10 [outer=(5), constraints=(/5: (/NULL - /9]; tight)] + │ └── b:6 < 10 [outer=(6), constraints=(/6: (/NULL - /9]; tight)] └── filters (true) # Lookup Joiner uses the constant equality columns at the same time as the explicit @@ -2062,21 +2062,21 @@ opt SELECT a, b, c FROM small INNER LOOKUP JOIN abcde ON m=b AND a=10 AND c=10 ---- project - ├── columns: a:4!null b:5!null c:6!null - ├── fd: ()-->(4,6) + ├── columns: a:5!null b:6!null c:7!null + ├── fd: ()-->(5,7) └── inner-join (lookup abcde@secondary) - ├── columns: m:1!null a:4!null b:5!null c:6!null + ├── columns: m:1!null a:5!null b:6!null c:7!null ├── flags: force lookup join (into right side) - ├── key columns: [10 1 11] = [4 5 6] - ├── fd: ()-->(4,6), (1)==(5), (5)==(1) + ├── key columns: [12 1 13] = [5 6 7] + ├── fd: ()-->(5,7), (1)==(6), (6)==(1) ├── project - │ ├── columns: "project_const_col_@4":10!null "project_const_col_@6":11!null m:1 - │ ├── fd: ()-->(10,11) + │ ├── columns: "project_const_col_@5":12!null "project_const_col_@7":13!null m:1 + │ ├── fd: ()-->(12,13) │ ├── scan small │ │ └── columns: m:1 │ └── projections - │ ├── 10 [as="project_const_col_@4":10] - │ └── 10 [as="project_const_col_@6":11] + │ ├── 10 [as="project_const_col_@5":12] + │ └── 10 [as="project_const_col_@7":13] └── filters (true) # Projection of constant columns work with non const expressions as well. @@ -2089,22 +2089,22 @@ opt SELECT * FROM small INNER JOIN bool_col ON a=m AND b=10 AND c=true ---- inner-join (lookup bool_col) - ├── columns: m:1!null n:2 a:4!null b:5!null c:6!null d:7 e:8 - ├── key columns: [9] = [9] + ├── columns: m:1!null n:2 a:5!null b:6!null c:7!null d:8 e:9 + ├── key columns: [10] = [10] ├── lookup columns are key - ├── fd: ()-->(5,6), (1)==(4), (4)==(1) + ├── fd: ()-->(6,7), (1)==(5), (5)==(1) ├── inner-join (lookup bool_col@secondary) - │ ├── columns: m:1!null n:2 a:4!null b:5!null c:6!null bool_col.rowid:9!null - │ ├── key columns: [1 10 11] = [4 5 6] - │ ├── fd: ()-->(5,6), (9)-->(4), (1)==(4), (4)==(1) + │ ├── columns: m:1!null n:2 a:5!null b:6!null c:7!null bool_col.rowid:10!null + │ ├── key columns: [1 12 13] = [5 6 7] + │ ├── fd: ()-->(6,7), (10)-->(5), (1)==(5), (5)==(1) │ ├── project - │ │ ├── columns: "project_const_col_@5":10!null "project_const_col_@6":11!null m:1 n:2 - │ │ ├── fd: ()-->(10,11) + │ │ ├── columns: "project_const_col_@6":12!null "project_const_col_@7":13!null m:1 n:2 + │ │ ├── fd: ()-->(12,13) │ │ ├── scan small │ │ │ └── columns: m:1 n:2 │ │ └── projections - │ │ ├── 10 [as="project_const_col_@5":10] - │ │ └── true [as="project_const_col_@6":11] + │ │ ├── 10 [as="project_const_col_@6":12] + │ │ └── true [as="project_const_col_@7":13] │ └── filters (true) └── filters (true) @@ -2112,22 +2112,22 @@ opt SELECT * FROM small INNER JOIN bool_col ON a=m AND b=10 AND c ---- inner-join (lookup bool_col) - ├── columns: m:1!null n:2 a:4!null b:5!null c:6!null d:7 e:8 - ├── key columns: [9] = [9] + ├── columns: m:1!null n:2 a:5!null b:6!null c:7!null d:8 e:9 + ├── key columns: [10] = [10] ├── lookup columns are key - ├── fd: ()-->(5,6), (1)==(4), (4)==(1) + ├── fd: ()-->(6,7), (1)==(5), (5)==(1) ├── inner-join (lookup bool_col@secondary) - │ ├── columns: m:1!null n:2 a:4!null b:5!null c:6!null bool_col.rowid:9!null - │ ├── key columns: [1 10 11] = [4 5 6] - │ ├── fd: ()-->(5,6), (9)-->(4), (1)==(4), (4)==(1) + │ ├── columns: m:1!null n:2 a:5!null b:6!null c:7!null bool_col.rowid:10!null + │ ├── key columns: [1 12 13] = [5 6 7] + │ ├── fd: ()-->(6,7), (10)-->(5), (1)==(5), (5)==(1) │ ├── project - │ │ ├── columns: "project_const_col_@5":10!null "project_const_col_@6":11!null m:1 n:2 - │ │ ├── fd: ()-->(10,11) + │ │ ├── columns: "project_const_col_@6":12!null "project_const_col_@7":13!null m:1 n:2 + │ │ ├── fd: ()-->(12,13) │ │ ├── scan small │ │ │ └── columns: m:1 n:2 │ │ └── projections - │ │ ├── 10 [as="project_const_col_@5":10] - │ │ └── true [as="project_const_col_@6":11] + │ │ ├── 10 [as="project_const_col_@6":12] + │ │ └── true [as="project_const_col_@7":13] │ └── filters (true) └── filters (true) @@ -2135,22 +2135,22 @@ opt SELECT * FROM small INNER JOIN bool_col ON a=m AND b=10 AND NOT c ---- inner-join (lookup bool_col) - ├── columns: m:1!null n:2 a:4!null b:5!null c:6!null d:7 e:8 - ├── key columns: [9] = [9] + ├── columns: m:1!null n:2 a:5!null b:6!null c:7!null d:8 e:9 + ├── key columns: [10] = [10] ├── lookup columns are key - ├── fd: ()-->(5,6), (1)==(4), (4)==(1) + ├── fd: ()-->(6,7), (1)==(5), (5)==(1) ├── inner-join (lookup bool_col@secondary) - │ ├── columns: m:1!null n:2 a:4!null b:5!null c:6!null bool_col.rowid:9!null - │ ├── key columns: [1 10 11] = [4 5 6] - │ ├── fd: ()-->(5,6), (9)-->(4), (1)==(4), (4)==(1) + │ ├── columns: m:1!null n:2 a:5!null b:6!null c:7!null bool_col.rowid:10!null + │ ├── key columns: [1 12 13] = [5 6 7] + │ ├── fd: ()-->(6,7), (10)-->(5), (1)==(5), (5)==(1) │ ├── project - │ │ ├── columns: "project_const_col_@5":10!null "project_const_col_@6":11!null m:1 n:2 - │ │ ├── fd: ()-->(10,11) + │ │ ├── columns: "project_const_col_@6":12!null "project_const_col_@7":13!null m:1 n:2 + │ │ ├── fd: ()-->(12,13) │ │ ├── scan small │ │ │ └── columns: m:1 n:2 │ │ └── projections - │ │ ├── 10 [as="project_const_col_@5":10] - │ │ └── false [as="project_const_col_@6":11] + │ │ ├── 10 [as="project_const_col_@6":12] + │ │ └── false [as="project_const_col_@7":13] │ └── filters (true) └── filters (true) @@ -2192,20 +2192,20 @@ project │ ├── col4:5 = 1 [outer=(5), constraints=(/5: [/1 - /1]; tight), fd=()-->(5)] │ └── col0:2 = 1 [outer=(2), constraints=(/2: [/1 - /1]; tight), fd=()-->(2)] ├── select - │ ├── columns: col0:7!null col2:9 - │ ├── lax-key: (9) - │ ├── fd: ()-->(7,9) + │ ├── columns: col0:8!null col2:10 + │ ├── lax-key: (10) + │ ├── fd: ()-->(8,10) │ ├── index-join t - │ │ ├── columns: col0:7 col2:9 - │ │ ├── lax-key: (7,9) - │ │ ├── fd: ()-->(9), (9)~~>(7) + │ │ ├── columns: col0:8 col2:10 + │ │ ├── lax-key: (8,10) + │ │ ├── fd: ()-->(10), (10)~~>(8) │ │ └── scan t@secondary - │ │ ├── columns: pk:6!null col2:9 - │ │ ├── constraint: /9: [/NULL - /NULL] - │ │ ├── key: (6) - │ │ └── fd: ()-->(9), (9)~~>(6) + │ │ ├── columns: pk:7!null col2:10 + │ │ ├── constraint: /10: [/NULL - /NULL] + │ │ ├── key: (7) + │ │ └── fd: ()-->(10), (10)~~>(7) │ └── filters - │ └── col0:7 = 1 [outer=(7), constraints=(/7: [/1 - /1]; tight), fd=()-->(7)] + │ └── col0:8 = 1 [outer=(8), constraints=(/8: [/1 - /1]; tight), fd=()-->(8)] └── filters (true) # Lookup semi-join with covering index. @@ -2214,11 +2214,11 @@ SELECT m, n FROM small WHERE EXISTS(SELECT 1 FROM abcd WHERE m = a AND a > b) ---- semi-join (lookup abcd@secondary) ├── columns: m:1 n:2 - ├── key columns: [1] = [4] + ├── key columns: [1] = [5] ├── scan small │ └── columns: m:1 n:2 └── filters - └── a:4 > b:5 [outer=(4,5), constraints=(/4: (/NULL - ]; /5: (/NULL - ])] + └── a:5 > b:6 [outer=(5,6), constraints=(/5: (/NULL - ]; /6: (/NULL - ])] # We should not generate a lookup semi-join when the index is non-covering. opt @@ -2229,14 +2229,14 @@ semi-join (hash) ├── scan small │ └── columns: m:1 n:2 ├── select - │ ├── columns: a:4!null b:5!null c:6 + │ ├── columns: a:5!null b:6!null c:7 │ ├── scan abcd - │ │ └── columns: a:4 b:5 c:6 + │ │ └── columns: a:5 b:6 c:7 │ └── filters - │ └── a:4 > b:5 [outer=(4,5), constraints=(/4: (/NULL - ]; /5: (/NULL - ])] + │ └── a:5 > b:6 [outer=(5,6), constraints=(/5: (/NULL - ]; /6: (/NULL - ])] └── filters - ├── m:1 = a:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] - └── n:2 = c:6 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + ├── m:1 = a:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + └── n:2 = c:7 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] # Lookup anti-join with covering index. opt @@ -2244,11 +2244,11 @@ SELECT m, n FROM small WHERE NOT EXISTS(SELECT 1 FROM abcd WHERE m = a AND a > b ---- anti-join (lookup abcd@secondary) ├── columns: m:1 n:2 - ├── key columns: [1] = [4] + ├── key columns: [1] = [5] ├── scan small │ └── columns: m:1 n:2 └── filters - └── a:4 > b:5 [outer=(4,5), constraints=(/4: (/NULL - ]; /5: (/NULL - ])] + └── a:5 > b:6 [outer=(5,6), constraints=(/5: (/NULL - ]; /6: (/NULL - ])] # We should not generate a lookup semi-join when the index is non-covering. opt @@ -2259,14 +2259,14 @@ anti-join (hash) ├── scan small │ └── columns: m:1 n:2 ├── select - │ ├── columns: a:4!null b:5!null c:6 + │ ├── columns: a:5!null b:6!null c:7 │ ├── scan abcd - │ │ └── columns: a:4 b:5 c:6 + │ │ └── columns: a:5 b:6 c:7 │ └── filters - │ └── a:4 > b:5 [outer=(4,5), constraints=(/4: (/NULL - ]; /5: (/NULL - ])] + │ └── a:5 > b:6 [outer=(5,6), constraints=(/5: (/NULL - ]; /6: (/NULL - ])] └── filters - ├── m:1 = a:4 [outer=(1,4), constraints=(/1: (/NULL - ]; /4: (/NULL - ]), fd=(1)==(4), (4)==(1)] - └── n:2 = c:6 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + ├── m:1 = a:5 [outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + └── n:2 = c:7 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] # -------------------------------------------------- # GenerateGeospatialLookupJoins @@ -2352,39 +2352,39 @@ OR n.name = 'Upper East Side' GROUP BY n.name, n.geom ---- project - ├── columns: name:13!null popn_per_sqkm:16 + ├── columns: name:14!null popn_per_sqkm:18 ├── immutable ├── group-by - │ ├── columns: name:13!null n.geom:14 sum:15 - │ ├── grouping columns: name:13!null n.geom:14 + │ ├── columns: name:14!null n.geom:15 sum:17 + │ ├── grouping columns: name:14!null n.geom:15 │ ├── immutable - │ ├── key: (13,14) - │ ├── fd: (13,14)-->(15) + │ ├── key: (14,15) + │ ├── fd: (14,15)-->(17) │ ├── inner-join (lookup nyc_census_blocks) - │ │ ├── columns: popn_total:3 c.boroname:9!null c.geom:10 n.boroname:12!null name:13!null n.geom:14 + │ │ ├── columns: popn_total:3 c.boroname:9!null c.geom:10 n.boroname:13!null name:14!null n.geom:15 │ │ ├── key columns: [1] = [1] │ │ ├── lookup columns are key │ │ ├── immutable - │ │ ├── fd: (9)==(12), (12)==(9) + │ │ ├── fd: (9)==(13), (13)==(9) │ │ ├── inner-join (inverted-lookup nyc_census_blocks@nyc_census_blocks_geo_idx) - │ │ │ ├── columns: c.gid:1!null n.boroname:12 name:13!null n.geom:14 + │ │ │ ├── columns: c.gid:1!null n.boroname:13 name:14!null n.geom:15 │ │ │ ├── inverted-expr - │ │ │ │ └── st_intersects(n.geom:14, c.geom:10) + │ │ │ │ └── st_intersects(n.geom:15, c.geom:10) │ │ │ ├── select - │ │ │ │ ├── columns: n.boroname:12 name:13!null n.geom:14 + │ │ │ │ ├── columns: n.boroname:13 name:14!null n.geom:15 │ │ │ │ ├── scan n - │ │ │ │ │ └── columns: n.boroname:12 name:13 n.geom:14 + │ │ │ │ │ └── columns: n.boroname:13 name:14 n.geom:15 │ │ │ │ └── filters - │ │ │ │ └── (name:13 = 'Upper West Side') OR (name:13 = 'Upper East Side') [outer=(13), constraints=(/13: [/'Upper East Side' - /'Upper East Side'] [/'Upper West Side' - /'Upper West Side']; tight)] + │ │ │ │ └── (name:14 = 'Upper West Side') OR (name:14 = 'Upper East Side') [outer=(14), constraints=(/14: [/'Upper East Side' - /'Upper East Side'] [/'Upper West Side' - /'Upper West Side']; tight)] │ │ │ └── filters (true) │ │ └── filters - │ │ ├── st_intersects(n.geom:14, c.geom:10) [outer=(10,14), immutable] - │ │ └── c.boroname:9 = n.boroname:12 [outer=(9,12), constraints=(/9: (/NULL - ]; /12: (/NULL - ]), fd=(9)==(12), (12)==(9)] + │ │ ├── st_intersects(n.geom:15, c.geom:10) [outer=(10,15), immutable] + │ │ └── c.boroname:9 = n.boroname:13 [outer=(9,13), constraints=(/9: (/NULL - ]; /13: (/NULL - ]), fd=(9)==(13), (13)==(9)] │ └── aggregations - │ └── sum [as=sum:15, outer=(3)] + │ └── sum [as=sum:17, outer=(3)] │ └── popn_total:3 └── projections - └── sum:15 / (st_area(n.geom:14) / 1e+06) [as=popn_per_sqkm:16, outer=(14,15), immutable] + └── sum:17 / (st_area(n.geom:15) / 1e+06) [as=popn_per_sqkm:18, outer=(15,17), immutable] memo expect=GenerateGeoLookupJoins SELECT @@ -2397,19 +2397,19 @@ WHERE n.name = 'Upper West Side' OR n.name = 'Upper East Side' GROUP BY n.name, n.geom ---- -memo (optimized, ~23KB, required=[presentation: name:13,popn_per_sqkm:16]) +memo (optimized, ~23KB, required=[presentation: name:14,popn_per_sqkm:18]) ├── G1: (project G2 G3 name) - │ └── [presentation: name:13,popn_per_sqkm:16] + │ └── [presentation: name:14,popn_per_sqkm:18] │ ├── best: (project G2 G3 name) │ └── cost: 7508.97 - ├── G2: (group-by G4 G5 cols=(13,14)) + ├── G2: (group-by G4 G5 cols=(14,15)) │ └── [] - │ ├── best: (group-by G4 G5 cols=(13,14)) + │ ├── best: (group-by G4 G5 cols=(14,15)) │ └── cost: 7508.92 ├── G3: (projections G6) - ├── G4: (inner-join G7 G8 G9) (inner-join G8 G7 G9) (lookup-join G10 G9 nyc_census_blocks,keyCols=[1],outCols=(3,9,10,12-14)) + ├── G4: (inner-join G7 G8 G9) (inner-join G8 G7 G9) (lookup-join G10 G9 nyc_census_blocks,keyCols=[1],outCols=(3,9,10,13-15)) │ └── [] - │ ├── best: (lookup-join G10 G9 nyc_census_blocks,keyCols=[1],outCols=(3,9,10,12-14)) + │ ├── best: (lookup-join G10 G9 nyc_census_blocks,keyCols=[1],outCols=(3,9,10,13-15)) │ └── cost: 7301.98 ├── G5: (aggregations G11) ├── G6: (div G12 G13) @@ -2429,9 +2429,9 @@ memo (optimized, ~23KB, required=[presentation: name:13,popn_per_sqkm:16]) ├── G11: (sum G19) ├── G12: (variable sum) ├── G13: (div G20 G21) - ├── G14: (scan n,cols=(12-14)) + ├── G14: (scan n,cols=(13-15)) │ └── [] - │ ├── best: (scan n,cols=(12-14)) + │ ├── best: (scan n,cols=(13-15)) │ └── cost: 138.05 ├── G15: (filters G22) ├── G16: (function G23 st_intersects) @@ -2461,22 +2461,22 @@ JOIN nyc_neighborhoods@nyc_neighborhoods_geo_idx AS n ON ST_DWithin(c.geom, n.geom, 50) ---- project - ├── columns: name:13 boroname:9 + ├── columns: name:14 boroname:9 ├── immutable └── inner-join (lookup nyc_neighborhoods) - ├── columns: c.boroname:9 c.geom:10 name:13 n.geom:14 - ├── key columns: [11] = [11] + ├── columns: c.boroname:9 c.geom:10 name:14 n.geom:15 + ├── key columns: [12] = [12] ├── lookup columns are key ├── immutable ├── inner-join (inverted-lookup nyc_neighborhoods@nyc_neighborhoods_geo_idx) - │ ├── columns: c.boroname:9 c.geom:10 n.gid:11!null + │ ├── columns: c.boroname:9 c.geom:10 n.gid:12!null │ ├── inverted-expr - │ │ └── st_dwithin(c.geom:10, n.geom:14, 50.0) + │ │ └── st_dwithin(c.geom:10, n.geom:15, 50.0) │ ├── scan c │ │ └── columns: c.boroname:9 c.geom:10 │ └── filters (true) └── filters - └── st_dwithin(c.geom:10, n.geom:14, 50.0) [outer=(10,14), immutable] + └── st_dwithin(c.geom:10, n.geom:15, 50.0) [outer=(10,15), immutable] # Same test as above, but with arguments commuted. opt expect=GenerateGeoLookupJoins @@ -2487,22 +2487,22 @@ JOIN nyc_neighborhoods@nyc_neighborhoods_geo_idx AS n ON ST_DWithin(n.geom, c.geom, 50) ---- project - ├── columns: name:13 boroname:9 + ├── columns: name:14 boroname:9 ├── immutable └── inner-join (lookup nyc_neighborhoods) - ├── columns: c.boroname:9 c.geom:10 name:13 n.geom:14 - ├── key columns: [11] = [11] + ├── columns: c.boroname:9 c.geom:10 name:14 n.geom:15 + ├── key columns: [12] = [12] ├── lookup columns are key ├── immutable ├── inner-join (inverted-lookup nyc_neighborhoods@nyc_neighborhoods_geo_idx) - │ ├── columns: c.boroname:9 c.geom:10 n.gid:11!null + │ ├── columns: c.boroname:9 c.geom:10 n.gid:12!null │ ├── inverted-expr - │ │ └── st_dwithin(n.geom:14, c.geom:10, 50.0) + │ │ └── st_dwithin(n.geom:15, c.geom:10, 50.0) │ ├── scan c │ │ └── columns: c.boroname:9 c.geom:10 │ └── filters (true) └── filters - └── st_dwithin(n.geom:14, c.geom:10, 50.0) [outer=(10,14), immutable] + └── st_dwithin(n.geom:15, c.geom:10, 50.0) [outer=(10,15), immutable] opt expect=GenerateGeoLookupJoins SELECT @@ -2512,22 +2512,22 @@ JOIN nyc_neighborhoods@nyc_neighborhoods_geo_idx AS n ON ST_Covers(c.geom, n.geom) ---- project - ├── columns: name:13 boroname:9 + ├── columns: name:14 boroname:9 ├── immutable └── inner-join (lookup nyc_neighborhoods) - ├── columns: c.boroname:9 c.geom:10 name:13 n.geom:14 - ├── key columns: [11] = [11] + ├── columns: c.boroname:9 c.geom:10 name:14 n.geom:15 + ├── key columns: [12] = [12] ├── lookup columns are key ├── immutable ├── inner-join (inverted-lookup nyc_neighborhoods@nyc_neighborhoods_geo_idx) - │ ├── columns: c.boroname:9 c.geom:10 n.gid:11!null + │ ├── columns: c.boroname:9 c.geom:10 n.gid:12!null │ ├── inverted-expr - │ │ └── st_covers(c.geom:10, n.geom:14) + │ │ └── st_covers(c.geom:10, n.geom:15) │ ├── scan c │ │ └── columns: c.boroname:9 c.geom:10 │ └── filters (true) └── filters - └── st_covers(c.geom:10, n.geom:14) [outer=(10,14), immutable] + └── st_covers(c.geom:10, n.geom:15) [outer=(10,15), immutable] # Same test as above, but with arguments commuted. opt expect=GenerateGeoLookupJoins @@ -2538,22 +2538,22 @@ JOIN nyc_neighborhoods@nyc_neighborhoods_geo_idx AS n ON ST_Covers(n.geom, c.geom) ---- project - ├── columns: name:13 boroname:9 + ├── columns: name:14 boroname:9 ├── immutable └── inner-join (lookup nyc_neighborhoods) - ├── columns: c.boroname:9 c.geom:10 name:13 n.geom:14 - ├── key columns: [11] = [11] + ├── columns: c.boroname:9 c.geom:10 name:14 n.geom:15 + ├── key columns: [12] = [12] ├── lookup columns are key ├── immutable ├── inner-join (inverted-lookup nyc_neighborhoods@nyc_neighborhoods_geo_idx) - │ ├── columns: c.boroname:9 c.geom:10 n.gid:11!null + │ ├── columns: c.boroname:9 c.geom:10 n.gid:12!null │ ├── inverted-expr - │ │ └── st_coveredby(c.geom:10, n.geom:14) + │ │ └── st_coveredby(c.geom:10, n.geom:15) │ ├── scan c │ │ └── columns: c.boroname:9 c.geom:10 │ └── filters (true) └── filters - └── st_covers(n.geom:14, c.geom:10) [outer=(10,14), immutable] + └── st_covers(n.geom:15, c.geom:10) [outer=(10,15), immutable] # -------------------------------------------------- # GenerateZigZagJoins @@ -3276,39 +3276,39 @@ FROM abc AS x JOIN [INSERT INTO abc (a) SELECT 1 FROM abc RETURNING 1] JOIN abc AS y ON true ON false ---- with &1 - ├── columns: bool:22!null + ├── columns: bool:26!null ├── cardinality: [0 - 0] ├── volatile, mutations ├── key: () - ├── fd: ()-->(22) + ├── fd: ()-->(26) ├── project - │ ├── columns: "?column?":16!null + │ ├── columns: "?column?":19!null │ ├── volatile, mutations - │ ├── fd: ()-->(16) + │ ├── fd: ()-->(19) │ ├── insert abc - │ │ ├── columns: abc.rowid:8!null + │ │ ├── columns: abc.rowid:9!null │ │ ├── insert-mapping: - │ │ │ ├── "?column?":13 => abc.a:5 - │ │ │ ├── column14:14 => abc.b:6 - │ │ │ ├── column14:14 => abc.c:7 - │ │ │ └── column15:15 => abc.rowid:8 + │ │ │ ├── "?column?":16 => abc.a:6 + │ │ │ ├── column17:17 => abc.b:7 + │ │ │ ├── column17:17 => abc.c:8 + │ │ │ └── column18:18 => abc.rowid:9 │ │ ├── volatile, mutations │ │ └── project - │ │ ├── columns: column14:14 column15:15 "?column?":13!null + │ │ ├── columns: column17:17 column18:18 "?column?":16!null │ │ ├── volatile - │ │ ├── fd: ()-->(13,14) + │ │ ├── fd: ()-->(16,17) │ │ ├── scan abc │ │ └── projections - │ │ ├── CAST(NULL AS INT8) [as=column14:14] - │ │ ├── unique_rowid() [as=column15:15, volatile] - │ │ └── 1 [as="?column?":13] + │ │ ├── CAST(NULL AS INT8) [as=column17:17] + │ │ ├── unique_rowid() [as=column18:18, volatile] + │ │ └── 1 [as="?column?":16] │ └── projections - │ └── 1 [as="?column?":16] + │ └── 1 [as="?column?":19] └── values - ├── columns: bool:22!null + ├── columns: bool:26!null ├── cardinality: [0 - 0] ├── key: () - └── fd: ()-->(22) + └── fd: ()-->(26) opt join-limit=3 SELECT 1 FROM ((VALUES (1), (1)) JOIN ((VALUES (1), (1), (1)) JOIN (VALUES (1), (1), (1), (1)) ON true) ON true) @@ -3490,9 +3490,9 @@ semi-join (hash) ├── scan abc │ └── columns: a:1 b:2 c:3 ├── scan def - │ └── columns: f:7 + │ └── columns: f:8 └── filters - └── a:1 = f:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ]), fd=(1)==(7), (7)==(1)] + └── a:1 = f:8 [outer=(1,8), constraints=(/1: (/NULL - ]; /8: (/NULL - ]), fd=(1)==(8), (8)==(1)] # Test that we don't commute a SemiJoin when the On conditions are not # equalities. For example, in this test we have a Lt condition. @@ -3504,9 +3504,9 @@ semi-join (cross) ├── scan abc │ └── columns: a:1 b:2 c:3 ├── scan def - │ └── columns: e:6!null + │ └── columns: e:7!null └── filters - └── a:1 < e:6 [outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] + └── a:1 < e:7 [outer=(1,7), constraints=(/1: (/NULL - ]; /7: (/NULL - ])] # Test that we don't commute a SemiJoin when the On conditions are not # equalities. For example, in this test we have an Or condition. @@ -3518,17 +3518,17 @@ semi-join (cross) ├── scan abc │ └── columns: a:1 b:2 c:3 ├── scan def - │ ├── columns: d:5!null e:6!null - │ └── key: (5,6) + │ ├── columns: d:6!null e:7!null + │ └── key: (6,7) └── filters - └── (a:1 = d:5) OR (c:3 = e:6) [outer=(1,3,5,6)] + └── (a:1 = d:6) OR (c:3 = e:7) [outer=(1,3,6,7)] opt disable=CommuteSemiJoin format=show-all SELECT * from abc WHERE EXISTS (SELECT * FROM def WHERE a=d AND c=e) ---- semi-join (lookup def) ├── columns: a:1(int) b:2(int) c:3(int) - ├── key columns: [1 3] = [5 6] + ├── key columns: [1 3] = [6 7] ├── lookup columns are key ├── stats: [rows=100, distinct(1)=100, null(1)=0, distinct(3)=10, null(3)=0] ├── cost: 507.050394 @@ -3540,7 +3540,7 @@ semi-join (lookup def) │ ├── cost: 107.02 │ ├── prune: (1-3) │ ├── interesting orderings: (+1,+2) (+2,+3) - │ └── unfiltered-cols: (1-4) + │ └── unfiltered-cols: (1-5) └── filters (true) # TODO(rytaft): See stats/join tests. Since we don't collect the stats properly @@ -3551,7 +3551,7 @@ SELECT * from abc WHERE EXISTS (SELECT * FROM def WHERE a=d AND c=e) ---- semi-join (lookup def) ├── columns: a:1(int) b:2(int) c:3(int) - ├── key columns: [1 3] = [5 6] + ├── key columns: [1 3] = [6 7] ├── lookup columns are key ├── stats: [rows=100, distinct(1)=100, null(1)=0, distinct(3)=10, null(3)=0] ├── cost: 507.050394 @@ -3563,7 +3563,7 @@ semi-join (lookup def) │ ├── cost: 107.02 │ ├── prune: (1-3) │ ├── interesting orderings: (+1,+2) (+2,+3) - │ └── unfiltered-cols: (1-4) + │ └── unfiltered-cols: (1-5) └── filters (true) exec-ddl @@ -3616,7 +3616,7 @@ WHERE EXISTS(SELECT * FROM orders o WHERE o.cust_id=c.id AND o.order_date='2019- semi-join (merge) ├── columns: id:1!null name:2 ├── left ordering: +1 - ├── right ordering: +4 + ├── right ordering: +5 ├── key: (1) ├── fd: (1)-->(2) ├── scan c @@ -3625,13 +3625,13 @@ semi-join (merge) │ ├── fd: (1)-->(2) │ └── ordering: +1 ├── sort - │ ├── columns: cust_id:4 order_date:5!null - │ ├── fd: ()-->(5) - │ ├── ordering: +4 opt(5) [actual: +4] + │ ├── columns: cust_id:5 order_date:6!null + │ ├── fd: ()-->(6) + │ ├── ordering: +5 opt(6) [actual: +5] │ └── scan o@secondary - │ ├── columns: cust_id:4 order_date:5!null - │ ├── constraint: /5/3: [/'2019-01-01' - /'2019-01-01'] - │ └── fd: ()-->(5) + │ ├── columns: cust_id:5 order_date:6!null + │ ├── constraint: /6/4: [/'2019-01-01' - /'2019-01-01'] + │ └── fd: ()-->(6) └── filters (true) # The CommuteSemiJoin rule allows a much better plan because we can use @@ -3646,19 +3646,19 @@ project ├── key: (1) ├── fd: (1)-->(2) └── inner-join (lookup customers) - ├── columns: c.id:1!null name:2 cust_id:4!null - ├── key columns: [4] = [1] + ├── columns: c.id:1!null name:2 cust_id:5!null + ├── key columns: [5] = [1] ├── lookup columns are key - ├── key: (4) - ├── fd: (1)-->(2), (1)==(4), (4)==(1) + ├── key: (5) + ├── fd: (1)-->(2), (1)==(5), (5)==(1) ├── distinct-on - │ ├── columns: cust_id:4 - │ ├── grouping columns: cust_id:4 - │ ├── key: (4) + │ ├── columns: cust_id:5 + │ ├── grouping columns: cust_id:5 + │ ├── key: (5) │ └── scan o@secondary - │ ├── columns: cust_id:4 order_date:5!null - │ ├── constraint: /5/3: [/'2019-01-01' - /'2019-01-01'] - │ └── fd: ()-->(5) + │ ├── columns: cust_id:5 order_date:6!null + │ ├── constraint: /6/4: [/'2019-01-01' - /'2019-01-01'] + │ └── fd: ()-->(6) └── filters (true) # GenerateInvertedIndexZigzagJoins is disabled in the presence of a row-level @@ -3693,19 +3693,19 @@ opt expect=PushJoinIntoIndexJoin SELECT * FROM abc INNER JOIN (SELECT * FROM pqr ORDER BY q LIMIT 5) ON a=q ---- inner-join (lookup pqr) - ├── columns: a:1!null b:2 c:3 p:5!null q:6!null r:7 s:8 t:9 - ├── key columns: [5] = [5] + ├── columns: a:1!null b:2 c:3 p:6!null q:7!null r:8 s:9 t:10 + ├── key columns: [6] = [6] ├── lookup columns are key - ├── fd: (5)-->(6-9), (1)==(6), (6)==(1) + ├── fd: (6)-->(7-10), (1)==(7), (7)==(1) ├── inner-join (lookup abc@ab) - │ ├── columns: a:1!null b:2 c:3 p:5!null q:6!null - │ ├── key columns: [6] = [1] - │ ├── fd: (5)-->(6), (1)==(6), (6)==(1) + │ ├── columns: a:1!null b:2 c:3 p:6!null q:7!null + │ ├── key columns: [7] = [1] + │ ├── fd: (6)-->(7), (1)==(7), (7)==(1) │ ├── scan pqr@q - │ │ ├── columns: p:5!null q:6 + │ │ ├── columns: p:6!null q:7 │ │ ├── limit: 5 - │ │ ├── key: (5) - │ │ └── fd: (5)-->(6) + │ │ ├── key: (6) + │ │ └── fd: (6)-->(7) │ └── filters (true) └── filters (true) @@ -3715,20 +3715,20 @@ opt expect=PushJoinIntoIndexJoin SELECT * FROM abc CROSS JOIN (SELECT * FROM pqr ORDER BY q LIMIT 5) ---- inner-join (cross) - ├── columns: a:1 b:2 c:3 p:5!null q:6 r:7 s:8 t:9 - ├── fd: (5)-->(6-9) + ├── columns: a:1 b:2 c:3 p:6!null q:7 r:8 s:9 t:10 + ├── fd: (6)-->(7-10) ├── scan abc │ └── columns: a:1 b:2 c:3 ├── index-join pqr - │ ├── columns: p:5!null q:6 r:7 s:8 t:9 + │ ├── columns: p:6!null q:7 r:8 s:9 t:10 │ ├── cardinality: [0 - 5] - │ ├── key: (5) - │ ├── fd: (5)-->(6-9) + │ ├── key: (6) + │ ├── fd: (6)-->(7-10) │ └── scan pqr@q - │ ├── columns: p:5!null q:6 + │ ├── columns: p:6!null q:7 │ ├── limit: 5 - │ ├── key: (5) - │ └── fd: (5)-->(6) + │ ├── key: (6) + │ └── fd: (6)-->(7) └── filters (true) # No-op case because the index join is the right input of the LeftJoin. @@ -3736,25 +3736,25 @@ opt expect-not=PushJoinIntoIndexJoin SELECT * FROM abc LEFT JOIN (SELECT * FROM pqr ORDER BY q LIMIT 5) ON a=q ---- left-join (merge) - ├── columns: a:1 b:2 c:3 p:5 q:6 r:7 s:8 t:9 + ├── columns: a:1 b:2 c:3 p:6 q:7 r:8 s:9 t:10 ├── left ordering: +1 - ├── right ordering: +6 - ├── fd: (5)-->(6-9) + ├── right ordering: +7 + ├── fd: (6)-->(7-10) ├── scan abc@ab │ ├── columns: a:1 b:2 c:3 │ └── ordering: +1 ├── index-join pqr - │ ├── columns: p:5!null q:6 r:7 s:8 t:9 + │ ├── columns: p:6!null q:7 r:8 s:9 t:10 │ ├── cardinality: [0 - 5] - │ ├── key: (5) - │ ├── fd: (5)-->(6-9) - │ ├── ordering: +6 + │ ├── key: (6) + │ ├── fd: (6)-->(7-10) + │ ├── ordering: +7 │ └── scan pqr@q - │ ├── columns: p:5!null q:6 + │ ├── columns: p:6!null q:7 │ ├── limit: 5 - │ ├── key: (5) - │ ├── fd: (5)-->(6) - │ └── ordering: +6 + │ ├── key: (6) + │ ├── fd: (6)-->(7) + │ └── ordering: +7 └── filters (true) # No-op case because the InnerJoin has join hints. @@ -3762,9 +3762,9 @@ opt expect-not=PushJoinIntoIndexJoin SELECT * FROM (SELECT * FROM pqr ORDER BY q LIMIT 5) INNER HASH JOIN abc ON a=q ---- inner-join (hash) - ├── columns: p:1!null q:2!null r:3 s:4 t:5 a:6!null b:7 c:8 + ├── columns: p:1!null q:2!null r:3 s:4 t:5 a:7!null b:8 c:9 ├── flags: force hash join (store right side) - ├── fd: (1)-->(2-5), (2)==(6), (6)==(2) + ├── fd: (1)-->(2-5), (2)==(7), (7)==(2) ├── index-join pqr │ ├── columns: p:1!null q:2 r:3 s:4 t:5 │ ├── cardinality: [0 - 5] @@ -3776,9 +3776,9 @@ inner-join (hash) │ ├── key: (1) │ └── fd: (1)-->(2) ├── scan abc - │ └── columns: a:6 b:7 c:8 + │ └── columns: a:7 b:8 c:9 └── filters - └── a:6 = q:2 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + └── a:7 = q:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] # No-op case because the ON condition references a column that doesn't come from # the input of the index join or the right side of the InnerJoin. @@ -3786,10 +3786,10 @@ opt expect-not=PushJoinIntoIndexJoin SELECT * FROM (SELECT * FROM pqr ORDER BY q LIMIT 5) INNER JOIN abc ON a=r ---- inner-join (hash) - ├── columns: p:1!null q:2 r:3!null s:4 t:5 a:6!null b:7 c:8 - ├── fd: (1)-->(2-5), (3)==(6), (6)==(3) + ├── columns: p:1!null q:2 r:3!null s:4 t:5 a:7!null b:8 c:9 + ├── fd: (1)-->(2-5), (3)==(7), (7)==(3) ├── scan abc - │ └── columns: a:6 b:7 c:8 + │ └── columns: a:7 b:8 c:9 ├── index-join pqr │ ├── columns: p:1!null q:2 r:3 s:4 t:5 │ ├── cardinality: [0 - 5] @@ -3801,7 +3801,7 @@ inner-join (hash) │ ├── key: (1) │ └── fd: (1)-->(2) └── filters - └── a:6 = r:3 [outer=(3,6), constraints=(/3: (/NULL - ]; /6: (/NULL - ]), fd=(3)==(6), (6)==(3)] + └── a:7 = r:3 [outer=(3,7), constraints=(/3: (/NULL - ]; /7: (/NULL - ]), fd=(3)==(7), (7)==(3)] # No-op case because the right side of the InnerJoin has outer columns. opt expect-not=PushJoinIntoIndexJoin disable=(TryDecorrelateProject) @@ -3816,41 +3816,41 @@ INNER JOIN LATERAL ( ON True ---- inner-join-apply - ├── columns: s:1!null t:2!null u:3!null p:4!null q:5!null r:6 s:7 t:8 a:9!null b:10 c:11 "?column?":13 + ├── columns: s:1!null t:2!null u:3!null p:5!null q:6!null r:7 s:8 t:9 a:11!null b:12 c:13 "?column?":16 ├── immutable - ├── fd: (1-4)-->(5-8), (1-3,9)-->(13), (5)==(9), (9)==(5) + ├── fd: (1-3,5)-->(6-9), (1-3,11)-->(16), (6)==(11), (11)==(6) ├── scan stu │ ├── columns: stu.s:1!null stu.t:2!null u:3!null │ └── key: (1-3) ├── inner-join (merge) - │ ├── columns: p:4!null q:5!null r:6 pqr.s:7 pqr.t:8 a:9!null b:10 c:11 "?column?":13 - │ ├── left ordering: +5 - │ ├── right ordering: +9 + │ ├── columns: p:5!null q:6!null r:7 pqr.s:8 pqr.t:9 a:11!null b:12 c:13 "?column?":16 + │ ├── left ordering: +6 + │ ├── right ordering: +11 │ ├── outer: (1) │ ├── immutable - │ ├── fd: (4)-->(5-8), (9)-->(13), (5)==(9), (9)==(5) + │ ├── fd: (5)-->(6-9), (11)-->(16), (6)==(11), (11)==(6) │ ├── index-join pqr - │ │ ├── columns: p:4!null q:5 r:6 pqr.s:7 pqr.t:8 + │ │ ├── columns: p:5!null q:6 r:7 pqr.s:8 pqr.t:9 │ │ ├── cardinality: [0 - 5] - │ │ ├── key: (4) - │ │ ├── fd: (4)-->(5-8) - │ │ ├── ordering: +5 + │ │ ├── key: (5) + │ │ ├── fd: (5)-->(6-9) + │ │ ├── ordering: +6 │ │ └── scan pqr@q - │ │ ├── columns: p:4!null q:5 + │ │ ├── columns: p:5!null q:6 │ │ ├── limit: 5 - │ │ ├── key: (4) - │ │ ├── fd: (4)-->(5) - │ │ └── ordering: +5 + │ │ ├── key: (5) + │ │ ├── fd: (5)-->(6) + │ │ └── ordering: +6 │ ├── project - │ │ ├── columns: "?column?":13 a:9 b:10 c:11 + │ │ ├── columns: "?column?":16 a:11 b:12 c:13 │ │ ├── outer: (1) │ │ ├── immutable - │ │ ├── fd: (9)-->(13) - │ │ ├── ordering: +9 + │ │ ├── fd: (11)-->(16) + │ │ ├── ordering: +11 │ │ ├── scan abc@ab - │ │ │ ├── columns: a:9 b:10 c:11 - │ │ │ └── ordering: +9 + │ │ │ ├── columns: a:11 b:12 c:13 + │ │ │ └── ordering: +11 │ │ └── projections - │ │ └── a:9 * stu.s:1 [as="?column?":13, outer=(1,9), immutable] + │ │ └── a:11 * stu.s:1 [as="?column?":16, outer=(1,11), immutable] │ └── filters (true) └── filters (true) diff --git a/pkg/sql/opt/xform/testdata/rules/join_order b/pkg/sql/opt/xform/testdata/rules/join_order index 9d46d13b0d0a..ceb58f998979 100644 --- a/pkg/sql/opt/xform/testdata/rules/join_order +++ b/pkg/sql/opt/xform/testdata/rules/join_order @@ -32,19 +32,19 @@ opt join-limit=2 expect=ReorderJoins SELECT * FROM abc, bx, cy WHERE a = 1 AND abc.b = bx.b AND abc.c = cy.c ---- inner-join (lookup cy) - ├── columns: a:1!null b:2!null c:3!null d:4 b:5!null x:6 c:7!null y:8 - ├── key columns: [3] = [7] + ├── columns: a:1!null b:2!null c:3!null d:4 b:6!null x:7 c:9!null y:10 + ├── key columns: [3] = [9] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(1-8) + ├── fd: ()-->(1-4,6,7,9,10) ├── inner-join (lookup bx) - │ ├── columns: a:1!null abc.b:2!null abc.c:3 d:4 bx.b:5!null x:6 - │ ├── key columns: [2] = [5] + │ ├── columns: a:1!null abc.b:2!null abc.c:3 d:4 bx.b:6!null x:7 + │ ├── key columns: [2] = [6] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(1-6) + │ ├── fd: ()-->(1-4,6,7) │ ├── scan abc │ │ ├── columns: a:1!null abc.b:2 abc.c:3 d:4 │ │ ├── constraint: /1: [/1 - /1] @@ -58,25 +58,25 @@ opt join-limit=2 expect=ReorderJoins SELECT * FROM bx, abc, cy WHERE a = 1 AND abc.b = bx.b AND abc.c = cy.c ---- inner-join (lookup cy) - ├── columns: b:1!null x:2 a:3!null b:4!null c:5!null d:6 c:7!null y:8 - ├── key columns: [5] = [7] + ├── columns: b:1!null x:2 a:4!null b:5!null c:6!null d:7 c:9!null y:10 + ├── key columns: [6] = [9] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(1-8) + ├── fd: ()-->(1,2,4-7,9,10) ├── inner-join (lookup bx) - │ ├── columns: bx.b:1!null x:2 a:3!null abc.b:4!null abc.c:5 d:6 - │ ├── key columns: [4] = [1] + │ ├── columns: bx.b:1!null x:2 a:4!null abc.b:5!null abc.c:6 d:7 + │ ├── key columns: [5] = [1] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(1-6) + │ ├── fd: ()-->(1,2,4-7) │ ├── scan abc - │ │ ├── columns: a:3!null abc.b:4 abc.c:5 d:6 - │ │ ├── constraint: /3: [/1 - /1] + │ │ ├── columns: a:4!null abc.b:5 abc.c:6 d:7 + │ │ ├── constraint: /4: [/1 - /1] │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(3-6) + │ │ └── fd: ()-->(4-7) │ └── filters (true) └── filters (true) @@ -84,25 +84,25 @@ opt join-limit=2 expect=ReorderJoins SELECT * FROM bx, cy, abc WHERE a = 1 AND abc.b = bx.b AND abc.c = cy.c ---- inner-join (lookup bx) - ├── columns: b:1!null x:2 c:3!null y:4 a:5!null b:6!null c:7!null d:8 - ├── key columns: [6] = [1] + ├── columns: b:1!null x:2 c:4!null y:5 a:7!null b:8!null c:9!null d:10 + ├── key columns: [8] = [1] ├── lookup columns are key ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(1-8) + ├── fd: ()-->(1,2,4,5,7-10) ├── inner-join (lookup cy) - │ ├── columns: cy.c:3!null y:4 a:5!null abc.b:6 abc.c:7!null d:8 - │ ├── key columns: [7] = [3] + │ ├── columns: cy.c:4!null y:5 a:7!null abc.b:8 abc.c:9!null d:10 + │ ├── key columns: [9] = [4] │ ├── lookup columns are key │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(3-8) + │ ├── fd: ()-->(4,5,7-10) │ ├── scan abc - │ │ ├── columns: a:5!null abc.b:6 abc.c:7 d:8 - │ │ ├── constraint: /5: [/1 - /1] + │ │ ├── columns: a:7!null abc.b:8 abc.c:9 d:10 + │ │ ├── constraint: /7: [/1 - /1] │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(5-8) + │ │ └── fd: ()-->(7-10) │ └── filters (true) └── filters (true) @@ -113,46 +113,46 @@ SELECT * FROM bx, cy, dz, abc WHERE a = 1 AND abc.b = bx.b AND abc.c = cy.c AND cy.c = dz.d ---- -memo (optimized, ~33KB, required=[presentation: b:1,x:2,c:3,y:4,d:5,z:6,a:7,b:8,c:9,d:10]) - ├── G1: (inner-join G2 G3 G4) (inner-join G3 G2 G4) (merge-join G2 G3 G5 inner-join,+1,+8) (lookup-join G3 G5 bx,keyCols=[8],outCols=(1-10)) - │ └── [presentation: b:1,x:2,c:3,y:4,d:5,z:6,a:7,b:8,c:9,d:10] - │ ├── best: (lookup-join G3 G5 bx,keyCols=[8],outCols=(1-10)) +memo (optimized, ~45KB, required=[presentation: b:1,x:2,c:4,y:5,d:7,z:8,a:10,b:11,c:12,d:13]) + ├── G1: (inner-join G2 G3 G4) (inner-join G3 G2 G4) (merge-join G2 G3 G5 inner-join,+1,+11) (lookup-join G3 G5 bx,keyCols=[11],outCols=(1,2,4,5,7,8,10-13)) + │ └── [presentation: b:1,x:2,c:4,y:5,d:7,z:8,a:10,b:11,c:12,d:13] + │ ├── best: (lookup-join G3 G5 bx,keyCols=[11],outCols=(1,2,4,5,7,8,10-13)) │ └── cost: 19.19 - ├── G2: (scan bx) + ├── G2: (scan bx,cols=(1,2)) │ ├── [ordering: +1] - │ │ ├── best: (scan bx) + │ │ ├── best: (scan bx,cols=(1,2)) │ │ └── cost: 1040.02 │ └── [] - │ ├── best: (scan bx) + │ ├── best: (scan bx,cols=(1,2)) │ └── cost: 1040.02 - ├── G3: (inner-join G6 G7 G8) (inner-join G9 G10 G11) (inner-join G7 G6 G8) (merge-join G6 G7 G5 inner-join,+3,+5) (inner-join G10 G9 G11) (merge-join G9 G10 G5 inner-join,+5,+9) (lookup-join G12 G11 abc,keyCols=[12],outCols=(3-10)) (merge-join G7 G6 G5 inner-join,+5,+3) (lookup-join G7 G5 cy,keyCols=[5],outCols=(3-10)) + ├── G3: (inner-join G6 G7 G8) (inner-join G9 G10 G11) (inner-join G7 G6 G8) (merge-join G6 G7 G5 inner-join,+4,+7) (inner-join G10 G9 G11) (merge-join G9 G10 G5 inner-join,+7,+12) (lookup-join G12 G11 abc,keyCols=[16],outCols=(4,5,7,8,10-13)) (merge-join G7 G6 G5 inner-join,+7,+4) (lookup-join G7 G5 cy,keyCols=[7],outCols=(4,5,7,8,10-13)) │ └── [] - │ ├── best: (lookup-join G7 G5 cy,keyCols=[5],outCols=(3-10)) + │ ├── best: (lookup-join G7 G5 cy,keyCols=[7],outCols=(4,5,7,8,10-13)) │ └── cost: 13.15 ├── G4: (filters G13) ├── G5: (filters) - ├── G6: (scan cy) - │ ├── [ordering: +3] - │ │ ├── best: (scan cy) + ├── G6: (scan cy,cols=(4,5)) + │ ├── [ordering: +4] + │ │ ├── best: (scan cy,cols=(4,5)) │ │ └── cost: 1040.02 │ └── [] - │ ├── best: (scan cy) + │ ├── best: (scan cy,cols=(4,5)) │ └── cost: 1040.02 - ├── G7: (inner-join G14 G10 G11) (inner-join G10 G14 G11) (merge-join G14 G10 G5 inner-join,+5,+9) (lookup-join G15 G11 abc,keyCols=[11],outCols=(5-10)) (lookup-join G10 G5 dz,keyCols=[9],outCols=(5-10)) + ├── G7: (inner-join G14 G10 G11) (inner-join G10 G14 G11) (merge-join G14 G10 G5 inner-join,+7,+12) (lookup-join G15 G11 abc,keyCols=[15],outCols=(7,8,10-13)) (lookup-join G10 G5 dz,keyCols=[12],outCols=(7,8,10-13)) │ └── [] - │ ├── best: (lookup-join G10 G5 dz,keyCols=[9],outCols=(5-10)) + │ ├── best: (lookup-join G10 G5 dz,keyCols=[12],outCols=(7,8,10-13)) │ └── cost: 7.13 ├── G8: (filters G16) - ├── G9: (inner-join G6 G14 G8) (inner-join G14 G6 G8) (merge-join G6 G14 G5 inner-join,+3,+5) (lookup-join G6 G5 dz,keyCols=[3],outCols=(3-6)) (merge-join G14 G6 G5 inner-join,+5,+3) (lookup-join G14 G5 cy,keyCols=[5],outCols=(3-6)) - │ ├── [ordering: +(3|5)] - │ │ ├── best: (merge-join G6="[ordering: +3]" G14="[ordering: +5]" G5 inner-join,+3,+5) + ├── G9: (inner-join G6 G14 G8) (inner-join G14 G6 G8) (merge-join G6 G14 G5 inner-join,+4,+7) (lookup-join G6 G5 dz,keyCols=[4],outCols=(4,5,7,8)) (merge-join G14 G6 G5 inner-join,+7,+4) (lookup-join G14 G5 cy,keyCols=[7],outCols=(4,5,7,8)) + │ ├── [ordering: +(4|7)] + │ │ ├── best: (merge-join G6="[ordering: +4]" G14="[ordering: +7]" G5 inner-join,+4,+7) │ │ └── cost: 2110.05 │ └── [] - │ ├── best: (merge-join G6="[ordering: +3]" G14="[ordering: +5]" G5 inner-join,+3,+5) + │ ├── best: (merge-join G6="[ordering: +4]" G14="[ordering: +7]" G5 inner-join,+4,+7) │ └── cost: 2110.05 - ├── G10: (select G17 G18) (scan abc,constrained) + ├── G10: (select G17 G18) (scan abc,cols=(10-13),constrained) │ └── [] - │ ├── best: (scan abc,constrained) + │ ├── best: (scan abc,cols=(10-13),constrained) │ └── cost: 1.09 ├── G11: (filters G19) ├── G12: (project G9 G20 c y d z) @@ -160,21 +160,21 @@ memo (optimized, ~33KB, required=[presentation: b:1,x:2,c:3,y:4,d:5,z:6,a:7,b:8, │ ├── best: (project G9 G20 c y d z) │ └── cost: 2130.06 ├── G13: (eq G21 G22) - ├── G14: (scan dz) - │ ├── [ordering: +5] - │ │ ├── best: (scan dz) + ├── G14: (scan dz,cols=(7,8)) + │ ├── [ordering: +7] + │ │ ├── best: (scan dz,cols=(7,8)) │ │ └── cost: 1040.02 │ └── [] - │ ├── best: (scan dz) + │ ├── best: (scan dz,cols=(7,8)) │ └── cost: 1040.02 ├── G15: (project G14 G20 d z) │ └── [] │ ├── best: (project G14 G20 d z) │ └── cost: 1060.03 ├── G16: (eq G23 G24) - ├── G17: (scan abc) + ├── G17: (scan abc,cols=(10-13)) │ └── [] - │ ├── best: (scan abc) + │ ├── best: (scan abc,cols=(10-13)) │ └── cost: 1080.02 ├── G18: (filters G25) ├── G19: (eq G24 G26) @@ -192,34 +192,34 @@ memo (optimized, ~33KB, required=[presentation: b:1,x:2,c:3,y:4,d:5,z:6,a:7,b:8, memo join-limit=0 expect-not=ReorderJoins SELECT * FROM bx, cy, abc WHERE a = 1 AND abc.b = bx.b AND abc.c = cy.c ---- -memo (optimized, ~17KB, required=[presentation: b:1,x:2,c:3,y:4,a:5,b:6,c:7,d:8]) - ├── G1: (inner-join G2 G3 G4) (inner-join G3 G2 G4) (merge-join G2 G3 G5 inner-join,+1,+6) (lookup-join G3 G5 bx,keyCols=[6],outCols=(1-8)) - │ └── [presentation: b:1,x:2,c:3,y:4,a:5,b:6,c:7,d:8] - │ ├── best: (lookup-join G3 G5 bx,keyCols=[6],outCols=(1-8)) +memo (optimized, ~26KB, required=[presentation: b:1,x:2,c:4,y:5,a:7,b:8,c:9,d:10]) + ├── G1: (inner-join G2 G3 G4) (inner-join G3 G2 G4) (merge-join G2 G3 G5 inner-join,+1,+8) (lookup-join G3 G5 bx,keyCols=[8],outCols=(1,2,4,5,7-10)) + │ └── [presentation: b:1,x:2,c:4,y:5,a:7,b:8,c:9,d:10] + │ ├── best: (lookup-join G3 G5 bx,keyCols=[8],outCols=(1,2,4,5,7-10)) │ └── cost: 13.13 - ├── G2: (scan bx) + ├── G2: (scan bx,cols=(1,2)) │ ├── [ordering: +1] - │ │ ├── best: (scan bx) + │ │ ├── best: (scan bx,cols=(1,2)) │ │ └── cost: 1040.02 │ └── [] - │ ├── best: (scan bx) + │ ├── best: (scan bx,cols=(1,2)) │ └── cost: 1040.02 - ├── G3: (inner-join G6 G7 G8) (inner-join G7 G6 G8) (merge-join G6 G7 G5 inner-join,+3,+7) (lookup-join G9 G8 abc,keyCols=[9],outCols=(3-8)) (lookup-join G7 G5 cy,keyCols=[7],outCols=(3-8)) + ├── G3: (inner-join G6 G7 G8) (inner-join G7 G6 G8) (merge-join G6 G7 G5 inner-join,+4,+9) (lookup-join G9 G8 abc,keyCols=[12],outCols=(4,5,7-10)) (lookup-join G7 G5 cy,keyCols=[9],outCols=(4,5,7-10)) │ └── [] - │ ├── best: (lookup-join G7 G5 cy,keyCols=[7],outCols=(3-8)) + │ ├── best: (lookup-join G7 G5 cy,keyCols=[9],outCols=(4,5,7-10)) │ └── cost: 7.13 ├── G4: (filters G10) ├── G5: (filters) - ├── G6: (scan cy) - │ ├── [ordering: +3] - │ │ ├── best: (scan cy) + ├── G6: (scan cy,cols=(4,5)) + │ ├── [ordering: +4] + │ │ ├── best: (scan cy,cols=(4,5)) │ │ └── cost: 1040.02 │ └── [] - │ ├── best: (scan cy) + │ ├── best: (scan cy,cols=(4,5)) │ └── cost: 1040.02 - ├── G7: (select G11 G12) (scan abc,constrained) + ├── G7: (select G11 G12) (scan abc,cols=(7-10),constrained) │ └── [] - │ ├── best: (scan abc,constrained) + │ ├── best: (scan abc,cols=(7-10),constrained) │ └── cost: 1.09 ├── G8: (filters G13) ├── G9: (project G6 G14 c y) @@ -227,9 +227,9 @@ memo (optimized, ~17KB, required=[presentation: b:1,x:2,c:3,y:4,a:5,b:6,c:7,d:8] │ ├── best: (project G6 G14 c y) │ └── cost: 1060.03 ├── G10: (eq G15 G16) - ├── G11: (scan abc) + ├── G11: (scan abc,cols=(7-10)) │ └── [] - │ ├── best: (scan abc) + │ ├── best: (scan abc,cols=(7-10)) │ └── cost: 1080.02 ├── G12: (filters G17) ├── G13: (eq G18 G19) @@ -245,39 +245,39 @@ memo (optimized, ~17KB, required=[presentation: b:1,x:2,c:3,y:4,a:5,b:6,c:7,d:8] memo join-limit=2 SELECT * FROM bx, cy, abc WHERE a = 1 AND abc.b = bx.b AND abc.c = cy.c ---- -memo (optimized, ~26KB, required=[presentation: b:1,x:2,c:3,y:4,a:5,b:6,c:7,d:8]) - ├── G1: (inner-join G2 G3 G4) (inner-join G5 G6 G7) (inner-join G3 G2 G4) (merge-join G2 G3 G8 inner-join,+1,+6) (inner-join G6 G5 G7) (merge-join G5 G6 G8 inner-join,+3,+7) (lookup-join G3 G8 bx,keyCols=[6],outCols=(1-8)) (lookup-join G6 G8 cy,keyCols=[7],outCols=(1-8)) - │ └── [presentation: b:1,x:2,c:3,y:4,a:5,b:6,c:7,d:8] - │ ├── best: (lookup-join G3 G8 bx,keyCols=[6],outCols=(1-8)) +memo (optimized, ~35KB, required=[presentation: b:1,x:2,c:4,y:5,a:7,b:8,c:9,d:10]) + ├── G1: (inner-join G2 G3 G4) (inner-join G5 G6 G7) (inner-join G3 G2 G4) (merge-join G2 G3 G8 inner-join,+1,+8) (inner-join G6 G5 G7) (merge-join G5 G6 G8 inner-join,+4,+9) (lookup-join G3 G8 bx,keyCols=[8],outCols=(1,2,4,5,7-10)) (lookup-join G6 G8 cy,keyCols=[9],outCols=(1,2,4,5,7-10)) + │ └── [presentation: b:1,x:2,c:4,y:5,a:7,b:8,c:9,d:10] + │ ├── best: (lookup-join G3 G8 bx,keyCols=[8],outCols=(1,2,4,5,7-10)) │ └── cost: 13.13 - ├── G2: (scan bx) + ├── G2: (scan bx,cols=(1,2)) │ ├── [ordering: +1] - │ │ ├── best: (scan bx) + │ │ ├── best: (scan bx,cols=(1,2)) │ │ └── cost: 1040.02 │ └── [] - │ ├── best: (scan bx) + │ ├── best: (scan bx,cols=(1,2)) │ └── cost: 1040.02 - ├── G3: (inner-join G5 G9 G7) (inner-join G9 G5 G7) (merge-join G5 G9 G8 inner-join,+3,+7) (lookup-join G10 G7 abc,keyCols=[9],outCols=(3-8)) (lookup-join G9 G8 cy,keyCols=[7],outCols=(3-8)) + ├── G3: (inner-join G5 G9 G7) (inner-join G9 G5 G7) (merge-join G5 G9 G8 inner-join,+4,+9) (lookup-join G10 G7 abc,keyCols=[12],outCols=(4,5,7-10)) (lookup-join G9 G8 cy,keyCols=[9],outCols=(4,5,7-10)) │ └── [] - │ ├── best: (lookup-join G9 G8 cy,keyCols=[7],outCols=(3-8)) + │ ├── best: (lookup-join G9 G8 cy,keyCols=[9],outCols=(4,5,7-10)) │ └── cost: 7.13 ├── G4: (filters G11) - ├── G5: (scan cy) - │ ├── [ordering: +3] - │ │ ├── best: (scan cy) + ├── G5: (scan cy,cols=(4,5)) + │ ├── [ordering: +4] + │ │ ├── best: (scan cy,cols=(4,5)) │ │ └── cost: 1040.02 │ └── [] - │ ├── best: (scan cy) + │ ├── best: (scan cy,cols=(4,5)) │ └── cost: 1040.02 - ├── G6: (inner-join G2 G9 G4) (inner-join G9 G2 G4) (merge-join G2 G9 G8 inner-join,+1,+6) (lookup-join G12 G4 abc,keyCols=[10],outCols=(1,2,5-8)) (lookup-join G9 G8 bx,keyCols=[6],outCols=(1,2,5-8)) + ├── G6: (inner-join G2 G9 G4) (inner-join G9 G2 G4) (merge-join G2 G9 G8 inner-join,+1,+8) (lookup-join G12 G4 abc,keyCols=[13],outCols=(1,2,7-10)) (lookup-join G9 G8 bx,keyCols=[8],outCols=(1,2,7-10)) │ └── [] - │ ├── best: (lookup-join G9 G8 bx,keyCols=[6],outCols=(1,2,5-8)) + │ ├── best: (lookup-join G9 G8 bx,keyCols=[8],outCols=(1,2,7-10)) │ └── cost: 7.13 ├── G7: (filters G13) ├── G8: (filters) - ├── G9: (select G14 G15) (scan abc,constrained) + ├── G9: (select G14 G15) (scan abc,cols=(7-10),constrained) │ └── [] - │ ├── best: (scan abc,constrained) + │ ├── best: (scan abc,cols=(7-10),constrained) │ └── cost: 1.09 ├── G10: (project G5 G16 c y) │ └── [] @@ -289,9 +289,9 @@ memo (optimized, ~26KB, required=[presentation: b:1,x:2,c:3,y:4,a:5,b:6,c:7,d:8] │ ├── best: (project G2 G16 b x) │ └── cost: 1060.03 ├── G13: (eq G19 G20) - ├── G14: (scan abc) + ├── G14: (scan abc,cols=(7-10)) │ └── [] - │ ├── best: (scan abc) + │ ├── best: (scan abc,cols=(7-10)) │ └── cost: 1080.02 ├── G15: (filters G21) ├── G16: (projections G22) @@ -307,32 +307,32 @@ opt join-limit=3 expect=ReorderJoins SELECT * FROM bx, cy, dz, abc WHERE a = 1 ---- inner-join (cross) - ├── columns: b:1!null x:2 c:3!null y:4 d:5!null z:6 a:7!null b:8 c:9 d:10 - ├── key: (1,3,5) - ├── fd: ()-->(7-10), (1)-->(2), (3)-->(4), (5)-->(6) + ├── columns: b:1!null x:2 c:4!null y:5 d:7!null z:8 a:10!null b:11 c:12 d:13 + ├── key: (1,4,7) + ├── fd: ()-->(10-13), (1)-->(2), (4)-->(5), (7)-->(8) ├── inner-join (cross) - │ ├── columns: cy.c:3!null y:4 dz.d:5!null z:6 a:7!null abc.b:8 abc.c:9 abc.d:10 - │ ├── key: (3,5) - │ ├── fd: ()-->(7-10), (3)-->(4), (5)-->(6) + │ ├── columns: cy.c:4!null y:5 dz.d:7!null z:8 a:10!null abc.b:11 abc.c:12 abc.d:13 + │ ├── key: (4,7) + │ ├── fd: ()-->(10-13), (4)-->(5), (7)-->(8) │ ├── scan cy - │ │ ├── columns: cy.c:3!null y:4 - │ │ ├── key: (3) - │ │ └── fd: (3)-->(4) + │ │ ├── columns: cy.c:4!null y:5 + │ │ ├── key: (4) + │ │ └── fd: (4)-->(5) │ ├── inner-join (cross) - │ │ ├── columns: dz.d:5!null z:6 a:7!null abc.b:8 abc.c:9 abc.d:10 + │ │ ├── columns: dz.d:7!null z:8 a:10!null abc.b:11 abc.c:12 abc.d:13 │ │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ │ ├── key: (5) - │ │ ├── fd: ()-->(7-10), (5)-->(6) + │ │ ├── key: (7) + │ │ ├── fd: ()-->(10-13), (7)-->(8) │ │ ├── scan dz - │ │ │ ├── columns: dz.d:5!null z:6 - │ │ │ ├── key: (5) - │ │ │ └── fd: (5)-->(6) + │ │ │ ├── columns: dz.d:7!null z:8 + │ │ │ ├── key: (7) + │ │ │ └── fd: (7)-->(8) │ │ ├── scan abc - │ │ │ ├── columns: a:7!null abc.b:8 abc.c:9 abc.d:10 - │ │ │ ├── constraint: /7: [/1 - /1] + │ │ │ ├── columns: a:10!null abc.b:11 abc.c:12 abc.d:13 + │ │ │ ├── constraint: /10: [/1 - /1] │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ └── fd: ()-->(7-10) + │ │ │ └── fd: ()-->(10-13) │ │ └── filters (true) │ └── filters (true) ├── scan bx @@ -345,60 +345,60 @@ opt join-limit=2 format=show-all SELECT * FROM abc, bx, cy, dz WHERE a = 1 ---- inner-join (cross) - ├── columns: a:1(int!null) b:2(int) c:3(int) d:4(int) b:5(int!null) x:6(int) c:7(int!null) y:8(int) d:9(int!null) z:10(int) + ├── columns: a:1(int!null) b:2(int) c:3(int) d:4(int) b:6(int!null) x:7(int) c:9(int!null) y:10(int) d:12(int!null) z:13(int) ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) ├── stats: [rows=1e+09] ├── cost: 32525668.7 - ├── key: (5,7,9) - ├── fd: ()-->(1-4), (5)-->(6), (7)-->(8), (9)-->(10) - ├── prune: (2-10) - ├── interesting orderings: (+1) (+5) (+7) (+9) + ├── key: (6,9,12) + ├── fd: ()-->(1-4), (6)-->(7), (9)-->(10), (12)-->(13) + ├── prune: (2-4,6,7,9,10,12,13) + ├── interesting orderings: (+1) (+6) (+9) (+12) ├── join-size: 3 ├── inner-join (cross) - │ ├── columns: t.public.bx.b:5(int!null) t.public.bx.x:6(int) t.public.cy.c:7(int!null) t.public.cy.y:8(int) t.public.dz.d:9(int!null) t.public.dz.z:10(int) + │ ├── columns: t.public.bx.b:6(int!null) t.public.bx.x:7(int) t.public.cy.c:9(int!null) t.public.cy.y:10(int) t.public.dz.d:12(int!null) t.public.dz.z:13(int) │ ├── stats: [rows=1e+09] │ ├── cost: 10025667.6 - │ ├── key: (5,7,9) - │ ├── fd: (5)-->(6), (7)-->(8), (9)-->(10) - │ ├── prune: (5-10) - │ ├── interesting orderings: (+5) (+7) (+9) + │ ├── key: (6,9,12) + │ ├── fd: (6)-->(7), (9)-->(10), (12)-->(13) + │ ├── prune: (6,7,9,10,12,13) + │ ├── interesting orderings: (+6) (+9) (+12) │ ├── join-size: 2 │ ├── inner-join (cross) - │ │ ├── columns: t.public.cy.c:7(int!null) t.public.cy.y:8(int) t.public.dz.d:9(int!null) t.public.dz.z:10(int) + │ │ ├── columns: t.public.cy.c:9(int!null) t.public.cy.y:10(int) t.public.dz.d:12(int!null) t.public.dz.z:13(int) │ │ ├── stats: [rows=1000000] │ │ ├── cost: 12110.05 - │ │ ├── key: (7,9) - │ │ ├── fd: (7)-->(8), (9)-->(10) - │ │ ├── prune: (7-10) - │ │ ├── interesting orderings: (+7) (+9) + │ │ ├── key: (9,12) + │ │ ├── fd: (9)-->(10), (12)-->(13) + │ │ ├── prune: (9,10,12,13) + │ │ ├── interesting orderings: (+9) (+12) │ │ ├── scan t.public.cy - │ │ │ ├── columns: t.public.cy.c:7(int!null) t.public.cy.y:8(int) - │ │ │ ├── stats: [rows=1000] - │ │ │ ├── cost: 1040.02 - │ │ │ ├── key: (7) - │ │ │ ├── fd: (7)-->(8) - │ │ │ ├── prune: (7,8) - │ │ │ ├── interesting orderings: (+7) - │ │ │ └── unfiltered-cols: (7,8) - │ │ ├── scan t.public.dz - │ │ │ ├── columns: t.public.dz.d:9(int!null) t.public.dz.z:10(int) + │ │ │ ├── columns: t.public.cy.c:9(int!null) t.public.cy.y:10(int) │ │ │ ├── stats: [rows=1000] │ │ │ ├── cost: 1040.02 │ │ │ ├── key: (9) │ │ │ ├── fd: (9)-->(10) │ │ │ ├── prune: (9,10) │ │ │ ├── interesting orderings: (+9) - │ │ │ └── unfiltered-cols: (9,10) + │ │ │ └── unfiltered-cols: (9-11) + │ │ ├── scan t.public.dz + │ │ │ ├── columns: t.public.dz.d:12(int!null) t.public.dz.z:13(int) + │ │ │ ├── stats: [rows=1000] + │ │ │ ├── cost: 1040.02 + │ │ │ ├── key: (12) + │ │ │ ├── fd: (12)-->(13) + │ │ │ ├── prune: (12,13) + │ │ │ ├── interesting orderings: (+12) + │ │ │ └── unfiltered-cols: (12-14) │ │ └── filters (true) │ ├── scan t.public.bx - │ │ ├── columns: t.public.bx.b:5(int!null) t.public.bx.x:6(int) + │ │ ├── columns: t.public.bx.b:6(int!null) t.public.bx.x:7(int) │ │ ├── stats: [rows=1000] │ │ ├── cost: 1040.02 - │ │ ├── key: (5) - │ │ ├── fd: (5)-->(6) - │ │ ├── prune: (5,6) - │ │ ├── interesting orderings: (+5) - │ │ └── unfiltered-cols: (5,6) + │ │ ├── key: (6) + │ │ ├── fd: (6)-->(7) + │ │ ├── prune: (6,7) + │ │ ├── interesting orderings: (+6) + │ │ └── unfiltered-cols: (6-8) │ └── filters (true) ├── scan t.public.abc │ ├── columns: t.public.abc.a:1(int!null) t.public.abc.b:2(int) t.public.abc.c:3(int) t.public.abc.d:4(int) @@ -418,43 +418,43 @@ inner-join (cross) memo join-limit=0 SELECT * FROM bx, cy, dz, abc WHERE x = y AND y = z AND z = a ---- -memo (optimized, ~13KB, required=[presentation: b:1,x:2,c:3,y:4,d:5,z:6,a:7,b:8,c:9,d:10]) +memo (optimized, ~24KB, required=[presentation: b:1,x:2,c:4,y:5,d:7,z:8,a:10,b:11,c:12,d:13]) ├── G1: (inner-join G2 G3 G4) (inner-join G3 G2 G4) - │ └── [presentation: b:1,x:2,c:3,y:4,d:5,z:6,a:7,b:8,c:9,d:10] + │ └── [presentation: b:1,x:2,c:4,y:5,d:7,z:8,a:10,b:11,c:12,d:13] │ ├── best: (inner-join G3 G2 G4) │ └── cost: 5478.19 - ├── G2: (scan bx) + ├── G2: (scan bx,cols=(1,2)) │ └── [] - │ ├── best: (scan bx) + │ ├── best: (scan bx,cols=(1,2)) │ └── cost: 1040.02 ├── G3: (inner-join G5 G6 G7) (inner-join G6 G5 G7) │ └── [] │ ├── best: (inner-join G5 G6 G7) │ └── cost: 3327.84 ├── G4: (filters G8) - ├── G5: (scan cy) + ├── G5: (scan cy,cols=(4,5)) │ └── [] - │ ├── best: (scan cy) + │ ├── best: (scan cy,cols=(4,5)) │ └── cost: 1040.02 - ├── G6: (inner-join G9 G10 G11) (inner-join G10 G9 G11) (lookup-join G9 G12 abc,keyCols=[6],outCols=(5-10)) (merge-join G10 G9 G12 inner-join,+7,+6) + ├── G6: (inner-join G9 G10 G11) (inner-join G10 G9 G11) (lookup-join G9 G12 abc,keyCols=[8],outCols=(7,8,10-13)) (merge-join G10 G9 G12 inner-join,+10,+8) │ └── [] │ ├── best: (inner-join G9 G10 G11) │ └── cost: 2159.96 ├── G7: (filters G13) ├── G8: (eq G14 G15) - ├── G9: (scan dz) - │ ├── [ordering: +6] + ├── G9: (scan dz,cols=(7,8)) + │ ├── [ordering: +8] │ │ ├── best: (sort G9) │ │ └── cost: 1259.35 │ └── [] - │ ├── best: (scan dz) + │ ├── best: (scan dz,cols=(7,8)) │ └── cost: 1040.02 - ├── G10: (scan abc) - │ ├── [ordering: +7] - │ │ ├── best: (scan abc) + ├── G10: (scan abc,cols=(10-13)) + │ ├── [ordering: +10] + │ │ ├── best: (scan abc,cols=(10-13)) │ │ └── cost: 1080.02 │ └── [] - │ ├── best: (scan abc) + │ ├── best: (scan abc,cols=(10-13)) │ └── cost: 1080.02 ├── G11: (filters G16) ├── G12: (filters) @@ -472,95 +472,95 @@ memo (optimized, ~13KB, required=[presentation: b:1,x:2,c:3,y:4,d:5,z:6,a:7,b:8, memo join-limit=3 SELECT * FROM bx, cy, dz, abc WHERE x = y AND y = z AND z = a ---- -memo (optimized, ~39KB, required=[presentation: b:1,x:2,c:3,y:4,d:5,z:6,a:7,b:8,c:9,d:10]) - ├── G1: (inner-join G2 G3 G4) (inner-join G5 G6 G4) (inner-join G7 G8 G9) (inner-join G10 G11 G9) (inner-join G12 G13 G4) (inner-join G14 G15 G4) (inner-join G16 G17 G18) (inner-join G3 G2 G4) (inner-join G6 G5 G4) (inner-join G8 G7 G9) (inner-join G11 G10 G9) (inner-join G13 G12 G4) (inner-join G15 G14 G4) (inner-join G17 G16 G18) (lookup-join G16 G19 abc,keyCols=[6],outCols=(1-10)) (merge-join G17 G16 G19 inner-join,+7,+6) - │ └── [presentation: b:1,x:2,c:3,y:4,d:5,z:6,a:7,b:8,c:9,d:10] +memo (optimized, ~50KB, required=[presentation: b:1,x:2,c:4,y:5,d:7,z:8,a:10,b:11,c:12,d:13]) + ├── G1: (inner-join G2 G3 G4) (inner-join G5 G6 G4) (inner-join G7 G8 G9) (inner-join G10 G11 G9) (inner-join G12 G13 G4) (inner-join G14 G15 G4) (inner-join G16 G17 G18) (inner-join G3 G2 G4) (inner-join G6 G5 G4) (inner-join G8 G7 G9) (inner-join G11 G10 G9) (inner-join G13 G12 G4) (inner-join G15 G14 G4) (inner-join G17 G16 G18) (lookup-join G16 G19 abc,keyCols=[8],outCols=(1,2,4,5,7,8,10-13)) (merge-join G17 G16 G19 inner-join,+10,+8) + │ └── [presentation: b:1,x:2,c:4,y:5,d:7,z:8,a:10,b:11,c:12,d:13] │ ├── best: (inner-join G7 G8 G9) │ └── cost: 5478.19 - ├── G2: (scan bx) + ├── G2: (scan bx,cols=(1,2)) │ ├── [ordering: +2] │ │ ├── best: (sort G2) │ │ └── cost: 1259.35 │ └── [] - │ ├── best: (scan bx) + │ ├── best: (scan bx,cols=(1,2)) │ └── cost: 1040.02 - ├── G3: (inner-join G5 G8 G9) (inner-join G10 G13 G9) (inner-join G14 G17 G18) (inner-join G8 G5 G9) (inner-join G13 G10 G9) (inner-join G17 G14 G18) (lookup-join G14 G19 abc,keyCols=[6],outCols=(3-10)) (merge-join G17 G14 G19 inner-join,+7,+6) + ├── G3: (inner-join G5 G8 G9) (inner-join G10 G13 G9) (inner-join G14 G17 G18) (inner-join G8 G5 G9) (inner-join G13 G10 G9) (inner-join G17 G14 G18) (lookup-join G14 G19 abc,keyCols=[8],outCols=(4,5,7,8,10-13)) (merge-join G17 G14 G19 inner-join,+10,+8) │ └── [] │ ├── best: (inner-join G5 G8 G9) │ └── cost: 3327.84 ├── G4: (filters G20) - ├── G5: (scan cy) - │ ├── [ordering: +4] + ├── G5: (scan cy,cols=(4,5)) + │ ├── [ordering: +5] │ │ ├── best: (sort G5) │ │ └── cost: 1259.35 │ └── [] - │ ├── best: (scan cy) + │ ├── best: (scan cy,cols=(4,5)) │ └── cost: 1040.02 - ├── G6: (inner-join G2 G8 G21) (inner-join G10 G15 G18) (inner-join G12 G17 G18) (inner-join G8 G2 G21) (inner-join G15 G10 G18) (inner-join G17 G12 G18) (lookup-join G12 G19 abc,keyCols=[6],outCols=(1,2,5-10)) (merge-join G15 G10 G19 inner-join,+7,+6) (merge-join G17 G12 G19 inner-join,+7,+6) + ├── G6: (inner-join G2 G8 G21) (inner-join G10 G15 G18) (inner-join G12 G17 G18) (inner-join G8 G2 G21) (inner-join G15 G10 G18) (inner-join G17 G12 G18) (lookup-join G12 G19 abc,keyCols=[8],outCols=(1,2,7,8,10-13)) (merge-join G15 G10 G19 inner-join,+10,+8) (merge-join G17 G12 G19 inner-join,+10,+8) │ └── [] │ ├── best: (inner-join G2 G8 G21) │ └── cost: 3327.84 ├── G7: (inner-join G2 G5 G4) (inner-join G5 G2 G4) - │ ├── [ordering: +(2|4)] + │ ├── [ordering: +(2|5)] │ │ ├── best: (sort G7) │ │ └── cost: 5003.07 │ └── [] │ ├── best: (inner-join G2 G5 G4) │ └── cost: 2208.07 - ├── G8: (inner-join G10 G17 G18) (inner-join G17 G10 G18) (lookup-join G10 G19 abc,keyCols=[6],outCols=(5-10)) (merge-join G17 G10 G19 inner-join,+7,+6) + ├── G8: (inner-join G10 G17 G18) (inner-join G17 G10 G18) (lookup-join G10 G19 abc,keyCols=[8],outCols=(7,8,10-13)) (merge-join G17 G10 G19 inner-join,+10,+8) │ └── [] │ ├── best: (inner-join G10 G17 G18) │ └── cost: 2159.96 ├── G9: (filters G22) - ├── G10: (scan dz) - │ ├── [ordering: +6] + ├── G10: (scan dz,cols=(7,8)) + │ ├── [ordering: +8] │ │ ├── best: (sort G10) │ │ └── cost: 1259.35 │ └── [] - │ ├── best: (scan dz) + │ ├── best: (scan dz,cols=(7,8)) │ └── cost: 1040.02 - ├── G11: (inner-join G2 G13 G4) (inner-join G5 G15 G4) (inner-join G7 G17 G23) (inner-join G13 G2 G4) (inner-join G15 G5 G4) (inner-join G17 G7 G23) (lookup-join G7 G19 abc,keyCols=[4],outCols=(1-4,7-10)) (merge-join G17 G7 G19 inner-join,+7,+4) + ├── G11: (inner-join G2 G13 G4) (inner-join G5 G15 G4) (inner-join G7 G17 G23) (inner-join G13 G2 G4) (inner-join G15 G5 G4) (inner-join G17 G7 G23) (lookup-join G7 G19 abc,keyCols=[5],outCols=(1,2,4,5,10-13)) (merge-join G17 G7 G19 inner-join,+10,+5) │ └── [] │ ├── best: (inner-join G2 G13 G4) │ └── cost: 3327.84 ├── G12: (inner-join G2 G10 G21) (inner-join G10 G2 G21) - │ ├── [ordering: +(2|6)] + │ ├── [ordering: +(2|8)] │ │ ├── best: (sort G12) │ │ └── cost: 5003.07 │ └── [] │ ├── best: (inner-join G2 G10 G21) │ └── cost: 2208.07 - ├── G13: (inner-join G5 G17 G23) (inner-join G17 G5 G23) (lookup-join G5 G19 abc,keyCols=[4],outCols=(3,4,7-10)) (merge-join G17 G5 G19 inner-join,+7,+4) + ├── G13: (inner-join G5 G17 G23) (inner-join G17 G5 G23) (lookup-join G5 G19 abc,keyCols=[5],outCols=(4,5,10-13)) (merge-join G17 G5 G19 inner-join,+10,+5) │ └── [] │ ├── best: (inner-join G5 G17 G23) │ └── cost: 2159.96 ├── G14: (inner-join G5 G10 G9) (inner-join G10 G5 G9) - │ ├── [ordering: +(4|6)] + │ ├── [ordering: +(5|8)] │ │ ├── best: (sort G14) │ │ └── cost: 5003.07 │ └── [] │ ├── best: (inner-join G5 G10 G9) │ └── cost: 2208.07 - ├── G15: (inner-join G2 G17 G24) (inner-join G17 G2 G24) (lookup-join G2 G19 abc,keyCols=[2],outCols=(1,2,7-10)) (merge-join G17 G2 G19 inner-join,+7,+2) - │ ├── [ordering: +(2|7)] - │ │ ├── best: (merge-join G17="[ordering: +7]" G2="[ordering: +2]" G19 inner-join,+7,+2) + ├── G15: (inner-join G2 G17 G24) (inner-join G17 G2 G24) (lookup-join G2 G19 abc,keyCols=[2],outCols=(1,2,10-13)) (merge-join G17 G2 G19 inner-join,+10,+2) + │ ├── [ordering: +(2|10)] + │ │ ├── best: (merge-join G17="[ordering: +10]" G2="[ordering: +2]" G19 inner-join,+10,+2) │ │ └── cost: 2369.28 │ └── [] │ ├── best: (inner-join G2 G17 G24) │ └── cost: 2159.96 ├── G16: (inner-join G2 G14 G4) (inner-join G5 G12 G4) (inner-join G7 G10 G9) (inner-join G14 G2 G4) (inner-join G12 G5 G4) (inner-join G10 G7 G9) - │ ├── [ordering: +(2|4|6)] + │ ├── [ordering: +(2|5|8)] │ │ ├── best: (sort G16) │ │ └── cost: 38447.25 │ └── [] │ ├── best: (inner-join G7 G10 G9) │ └── cost: 4358.42 - ├── G17: (scan abc) - │ ├── [ordering: +7] - │ │ ├── best: (scan abc) + ├── G17: (scan abc,cols=(10-13)) + │ ├── [ordering: +10] + │ │ ├── best: (scan abc,cols=(10-13)) │ │ └── cost: 1080.02 │ └── [] - │ ├── best: (scan abc) + │ ├── best: (scan abc,cols=(10-13)) │ └── cost: 1080.02 ├── G18: (filters G25) ├── G19: (filters) @@ -582,40 +582,40 @@ opt SELECT * FROM bx, cy, dz, abc WHERE x = y AND y = z AND z = a ---- inner-join (hash) - ├── columns: b:1!null x:2!null c:3!null y:4!null d:5!null z:6!null a:7!null b:8 c:9 d:10 - ├── key: (1,3,5) - ├── fd: (1)-->(2), (3)-->(4), (5)-->(6), (7)-->(8-10), (6)==(2,4,7), (7)==(2,4,6), (4)==(2,6,7), (2)==(4,6,7) + ├── columns: b:1!null x:2!null c:4!null y:5!null d:7!null z:8!null a:10!null b:11 c:12 d:13 + ├── key: (1,4,7) + ├── fd: (1)-->(2), (4)-->(5), (7)-->(8), (10)-->(11-13), (8)==(2,5,10), (10)==(2,5,8), (5)==(2,8,10), (2)==(5,8,10) ├── inner-join (hash) - │ ├── columns: bx.b:1!null x:2!null cy.c:3!null y:4!null - │ ├── key: (1,3) - │ ├── fd: (1)-->(2), (3)-->(4), (2)==(4), (4)==(2) + │ ├── columns: bx.b:1!null x:2!null cy.c:4!null y:5!null + │ ├── key: (1,4) + │ ├── fd: (1)-->(2), (4)-->(5), (2)==(5), (5)==(2) │ ├── scan bx │ │ ├── columns: bx.b:1!null x:2 │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ ├── scan cy - │ │ ├── columns: cy.c:3!null y:4 - │ │ ├── key: (3) - │ │ └── fd: (3)-->(4) + │ │ ├── columns: cy.c:4!null y:5 + │ │ ├── key: (4) + │ │ └── fd: (4)-->(5) │ └── filters - │ └── x:2 = y:4 [outer=(2,4), constraints=(/2: (/NULL - ]; /4: (/NULL - ]), fd=(2)==(4), (4)==(2)] + │ └── x:2 = y:5 [outer=(2,5), constraints=(/2: (/NULL - ]; /5: (/NULL - ]), fd=(2)==(5), (5)==(2)] ├── inner-join (hash) - │ ├── columns: dz.d:5!null z:6!null a:7!null abc.b:8 abc.c:9 abc.d:10 + │ ├── columns: dz.d:7!null z:8!null a:10!null abc.b:11 abc.c:12 abc.d:13 │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ ├── key: (5) - │ ├── fd: (5)-->(6), (7)-->(8-10), (6)==(7), (7)==(6) + │ ├── key: (7) + │ ├── fd: (7)-->(8), (10)-->(11-13), (8)==(10), (10)==(8) │ ├── scan dz - │ │ ├── columns: dz.d:5!null z:6 - │ │ ├── key: (5) - │ │ └── fd: (5)-->(6) - │ ├── scan abc - │ │ ├── columns: a:7!null abc.b:8 abc.c:9 abc.d:10 + │ │ ├── columns: dz.d:7!null z:8 │ │ ├── key: (7) - │ │ └── fd: (7)-->(8-10) + │ │ └── fd: (7)-->(8) + │ ├── scan abc + │ │ ├── columns: a:10!null abc.b:11 abc.c:12 abc.d:13 + │ │ ├── key: (10) + │ │ └── fd: (10)-->(11-13) │ └── filters - │ └── z:6 = a:7 [outer=(6,7), constraints=(/6: (/NULL - ]; /7: (/NULL - ]), fd=(6)==(7), (7)==(6)] + │ └── z:8 = a:10 [outer=(8,10), constraints=(/8: (/NULL - ]; /10: (/NULL - ]), fd=(8)==(10), (10)==(8)] └── filters - └── y:4 = z:6 [outer=(4,6), constraints=(/4: (/NULL - ]; /6: (/NULL - ]), fd=(4)==(6), (6)==(4)] + └── y:5 = z:8 [outer=(5,8), constraints=(/5: (/NULL - ]; /8: (/NULL - ]), fd=(5)==(8), (8)==(5)] # Regression test for #34795. exec-ddl @@ -634,69 +634,69 @@ WHERE a4.id = 1 AND (SELECT true FROM a WHERE a1.id = 1) ---- project - ├── columns: "?column?":7!null - ├── fd: ()-->(7) + ├── columns: "?column?":12!null + ├── fd: ()-->(12) ├── inner-join (cross) - │ ├── columns: a1.id:1!null a2.id:2!null a3.id:3!null a4.id:4!null bool:6!null + │ ├── columns: a1.id:1!null a2.id:3!null a3.id:5!null a4.id:7!null bool:11!null │ ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - │ ├── key: (3) - │ ├── fd: ()-->(2,4,6), (1)==(3), (3)==(1) + │ ├── key: (5) + │ ├── fd: ()-->(3,7,11), (1)==(5), (5)==(1) │ ├── inner-join (cross) - │ │ ├── columns: a1.id:1!null a2.id:2!null a3.id:3!null bool:6!null + │ │ ├── columns: a1.id:1!null a2.id:3!null a3.id:5!null bool:11!null │ │ ├── multiplicity: left-rows(zero-or-more), right-rows(zero-or-one) - │ │ ├── key: (3) - │ │ ├── fd: ()-->(2,6), (1)==(3), (3)==(1) + │ │ ├── key: (5) + │ │ ├── fd: ()-->(3,11), (1)==(5), (5)==(1) │ │ ├── scan a2 - │ │ │ ├── columns: a2.id:2!null - │ │ │ ├── constraint: /2: [/1 - /1] + │ │ │ ├── columns: a2.id:3!null + │ │ │ ├── constraint: /3: [/1 - /1] │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ └── fd: ()-->(2) + │ │ │ └── fd: ()-->(3) │ │ ├── inner-join (lookup a) - │ │ │ ├── columns: a1.id:1!null a3.id:3!null bool:6!null - │ │ │ ├── key columns: [1] = [3] + │ │ │ ├── columns: a1.id:1!null a3.id:5!null bool:11!null + │ │ │ ├── key columns: [1] = [5] │ │ │ ├── lookup columns are key - │ │ │ ├── key: (3) - │ │ │ ├── fd: ()-->(6), (1)==(3), (3)==(1) + │ │ │ ├── key: (5) + │ │ │ ├── fd: ()-->(11), (1)==(5), (5)==(1) │ │ │ ├── select - │ │ │ │ ├── columns: a1.id:1!null bool:6!null + │ │ │ │ ├── columns: a1.id:1!null bool:11!null │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: ()-->(6) + │ │ │ │ ├── fd: ()-->(11) │ │ │ │ ├── ensure-distinct-on - │ │ │ │ │ ├── columns: a1.id:1!null bool:6 + │ │ │ │ │ ├── columns: a1.id:1!null bool:11 │ │ │ │ │ ├── grouping columns: a1.id:1!null │ │ │ │ │ ├── error: "more than one row returned by a subquery used as an expression" │ │ │ │ │ ├── key: (1) - │ │ │ │ │ ├── fd: (1)-->(6) + │ │ │ │ │ ├── fd: (1)-->(11) │ │ │ │ │ ├── left-join (cross) - │ │ │ │ │ │ ├── columns: a1.id:1!null bool:6 + │ │ │ │ │ │ ├── columns: a1.id:1!null bool:11 │ │ │ │ │ │ ├── scan a1 │ │ │ │ │ │ │ ├── columns: a1.id:1!null │ │ │ │ │ │ │ └── key: (1) │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ ├── columns: bool:6!null - │ │ │ │ │ │ │ ├── fd: ()-->(6) + │ │ │ │ │ │ │ ├── columns: bool:11!null + │ │ │ │ │ │ │ ├── fd: ()-->(11) │ │ │ │ │ │ │ ├── scan a │ │ │ │ │ │ │ └── projections - │ │ │ │ │ │ │ └── true [as=bool:6] + │ │ │ │ │ │ │ └── true [as=bool:11] │ │ │ │ │ │ └── filters │ │ │ │ │ │ └── a1.id:1 = 1 [outer=(1), constraints=(/1: [/1 - /1]; tight), fd=()-->(1)] │ │ │ │ │ └── aggregations - │ │ │ │ │ └── const-agg [as=bool:6, outer=(6)] - │ │ │ │ │ └── bool:6 + │ │ │ │ │ └── const-agg [as=bool:11, outer=(11)] + │ │ │ │ │ └── bool:11 │ │ │ │ └── filters - │ │ │ │ └── bool:6 [outer=(6), constraints=(/6: [/true - /true]; tight), fd=()-->(6)] + │ │ │ │ └── bool:11 [outer=(11), constraints=(/11: [/true - /true]; tight), fd=()-->(11)] │ │ │ └── filters (true) │ │ └── filters (true) │ ├── scan a4 - │ │ ├── columns: a4.id:4!null - │ │ ├── constraint: /4: [/1 - /1] + │ │ ├── columns: a4.id:7!null + │ │ ├── constraint: /7: [/1 - /1] │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ └── fd: ()-->(4) + │ │ └── fd: ()-->(7) │ └── filters (true) └── projections - └── 1 [as="?column?":7] + └── 1 [as="?column?":12] # An edge for b = d should be added to the graph. reorderjoins format=hide-all diff --git a/pkg/sql/opt/xform/testdata/rules/limit b/pkg/sql/opt/xform/testdata/rules/limit index 107308defc78..652dc76be489 100644 --- a/pkg/sql/opt/xform/testdata/rules/limit +++ b/pkg/sql/opt/xform/testdata/rules/limit @@ -181,10 +181,10 @@ limit │ └── columns: s:4 └── subquery └── scan a@s_idx - ├── columns: k:6!null + ├── columns: k:7!null ├── limit: 1 ├── key: () - └── fd: ()-->(6) + └── fd: ()-->(7) memo SELECT s FROM a WHERE s='foo' LIMIT 1 @@ -306,7 +306,7 @@ opt EXPLAIN SELECT * FROM abcd@b WHERE a >= 20 AND a <= 30 ORDER BY b DESC LIMIT 5 ---- explain - ├── columns: tree:5 field:6 description:7 + ├── columns: tree:6 field:7 description:8 └── limit ├── columns: a:1!null b:2 c:3 d:4 ├── internal-ordering: -2 @@ -346,7 +346,7 @@ Initial expression Cost: 10000000000000000159028911097599180468360808563945281389781327557747838772170381060813469985856815104.00 ================================================================================ explain - ├── columns: tree:5 field:6 description:7 + ├── columns: tree:6 field:7 description:8 └── sort ├── columns: a:1!null b:2 c:3 d:4 ├── cardinality: [0 - 5] @@ -366,25 +366,30 @@ Initial expression │ ├── fd: (1)-->(2-4), (2-4)~~>(1) │ ├── ordering: -2 │ ├── limit hint: 5.00 - │ └── select + │ └── project │ ├── columns: a:1!null b:2 c:3 d:4 │ ├── cardinality: [0 - 11] │ ├── key: (1) │ ├── fd: (1)-->(2-4), (2-4)~~>(1) - │ ├── scan abcd - │ │ ├── columns: a:1!null b:2 c:3 d:4 - │ │ ├── flags: force-index=b - │ │ ├── key: (1) - │ │ └── fd: (1)-->(2-4), (2-4)~~>(1) - │ └── filters - │ └── (a:1 >= 20) AND (a:1 <= 30) [outer=(1), constraints=(/1: [/20 - /30]; tight)] + │ └── select + │ ├── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 + │ ├── cardinality: [0 - 11] + │ ├── key: (1) + │ ├── fd: (1)-->(2-5), (2-4)~~>(1,5) + │ ├── scan abcd + │ │ ├── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 + │ │ ├── flags: force-index=b + │ │ ├── key: (1) + │ │ └── fd: (1)-->(2-5), (2-4)~~>(1,5) + │ └── filters + │ └── (a:1 >= 20) AND (a:1 <= 30) [outer=(1), constraints=(/1: [/20 - /30]; tight)] └── 5 ================================================================================ SimplifySelectFilters Cost: 10000000000000000159028911097599180468360808563945281389781327557747838772170381060813469985856815104.00 ================================================================================ explain - ├── columns: tree:5 field:6 description:7 + ├── columns: tree:6 field:7 description:8 └── sort ├── columns: a:1!null b:2 c:3 d:4 ├── cardinality: [0 - 5] @@ -404,27 +409,32 @@ SimplifySelectFilters │ ├── fd: (1)-->(2-4), (2-4)~~>(1) │ ├── ordering: -2 │ ├── limit hint: 5.00 - │ └── select + │ └── project │ ├── columns: a:1!null b:2 c:3 d:4 - │ ├── cardinality: [0 - 11] │ ├── key: (1) │ ├── fd: (1)-->(2-4), (2-4)~~>(1) - │ ├── scan abcd - │ │ ├── columns: a:1!null b:2 c:3 d:4 - │ │ ├── flags: force-index=b - │ │ ├── key: (1) - │ │ └── fd: (1)-->(2-4), (2-4)~~>(1) - │ └── filters - - │ └── (a:1 >= 20) AND (a:1 <= 30) [outer=(1), constraints=(/1: [/20 - /30]; tight)] - + │ ├── a:1 >= 20 [outer=(1), constraints=(/1: [/20 - ]; tight)] - + │ └── a:1 <= 30 [outer=(1), constraints=(/1: (/NULL - /30]; tight)] + │ └── select + │ ├── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 + - │ ├── cardinality: [0 - 11] + │ ├── key: (1) + │ ├── fd: (1)-->(2-5), (2-4)~~>(1,5) + │ ├── scan abcd + │ │ ├── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 + │ │ ├── flags: force-index=b + │ │ ├── key: (1) + │ │ └── fd: (1)-->(2-5), (2-4)~~>(1,5) + │ └── filters + - │ └── (a:1 >= 20) AND (a:1 <= 30) [outer=(1), constraints=(/1: [/20 - /30]; tight)] + + │ ├── a:1 >= 20 [outer=(1), constraints=(/1: [/20 - ]; tight)] + + │ └── a:1 <= 30 [outer=(1), constraints=(/1: (/NULL - /30]; tight)] └── 5 ================================================================================ ConsolidateSelectFilters Cost: 10000000000000000159028911097599180468360808563945281389781327557747838772170381060813469985856815104.00 ================================================================================ explain - ├── columns: tree:5 field:6 description:7 + ├── columns: tree:6 field:7 description:8 └── sort ├── columns: a:1!null b:2 c:3 d:4 ├── cardinality: [0 - 5] @@ -444,19 +454,122 @@ ConsolidateSelectFilters │ ├── fd: (1)-->(2-4), (2-4)~~>(1) │ ├── ordering: -2 │ ├── limit hint: 5.00 - │ └── select + │ └── project │ ├── columns: a:1!null b:2 c:3 d:4 + │ ├── cardinality: [0 - 11] │ ├── key: (1) │ ├── fd: (1)-->(2-4), (2-4)~~>(1) - │ ├── scan abcd - │ │ ├── columns: a:1!null b:2 c:3 d:4 - │ │ ├── flags: force-index=b - │ │ ├── key: (1) - │ │ └── fd: (1)-->(2-4), (2-4)~~>(1) - │ └── filters - - │ ├── a:1 >= 20 [outer=(1), constraints=(/1: [/20 - ]; tight)] - - │ └── a:1 <= 30 [outer=(1), constraints=(/1: (/NULL - /30]; tight)] + │ └── select + │ ├── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 + + │ ├── cardinality: [0 - 11] + │ ├── key: (1) + │ ├── fd: (1)-->(2-5), (2-4)~~>(1,5) + │ ├── scan abcd + │ │ ├── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 + │ │ ├── flags: force-index=b + │ │ ├── key: (1) + │ │ └── fd: (1)-->(2-5), (2-4)~~>(1,5) + │ └── filters + - │ ├── a:1 >= 20 [outer=(1), constraints=(/1: [/20 - ]; tight)] + - │ └── a:1 <= 30 [outer=(1), constraints=(/1: (/NULL - /30]; tight)] + + │ └── (a:1 >= 20) AND (a:1 <= 30) [outer=(1), constraints=(/1: [/20 - /30]; tight)] + └── 5 +================================================================================ +PruneSelectCols + Cost: 10000000000000000159028911097599180468360808563945281389781327557747838772170381060813469985856815104.00 +================================================================================ + explain + ├── columns: tree:6 field:7 description:8 + └── sort + ├── columns: a:1!null b:2 c:3 d:4 + ├── cardinality: [0 - 5] + ├── key: (1) + ├── fd: (1)-->(2-4), (2-4)~~>(1) + ├── ordering: -2 + └── limit + ├── columns: a:1!null b:2 c:3 d:4 + ├── internal-ordering: -2 + ├── cardinality: [0 - 5] + ├── key: (1) + ├── fd: (1)-->(2-4), (2-4)~~>(1) + ├── sort + │ ├── columns: a:1!null b:2 c:3 d:4 + │ ├── cardinality: [0 - 11] + │ ├── key: (1) + │ ├── fd: (1)-->(2-4), (2-4)~~>(1) + │ ├── ordering: -2 + │ ├── limit hint: 5.00 + │ └── project + │ ├── columns: a:1!null b:2 c:3 d:4 + │ ├── cardinality: [0 - 11] + │ ├── key: (1) + │ ├── fd: (1)-->(2-4), (2-4)~~>(1) + │ └── select + - │ ├── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 + + │ ├── columns: a:1!null b:2 c:3 d:4 + │ ├── cardinality: [0 - 11] + │ ├── key: (1) + - │ ├── fd: (1)-->(2-5), (2-4)~~>(1,5) + + │ ├── fd: (1)-->(2-4), (2-4)~~>(1) + │ ├── scan abcd + - │ │ ├── columns: a:1!null b:2 c:3 d:4 crdb_internal_mvcc_timestamp:5 + + │ │ ├── columns: a:1!null b:2 c:3 d:4 + │ │ ├── flags: force-index=b + │ │ ├── key: (1) + - │ │ └── fd: (1)-->(2-5), (2-4)~~>(1,5) + + │ │ └── fd: (1)-->(2-4), (2-4)~~>(1) + │ └── filters + │ └── (a:1 >= 20) AND (a:1 <= 30) [outer=(1), constraints=(/1: [/20 - /30]; tight)] + └── 5 +================================================================================ +EliminateProject + Cost: 10000000000000000159028911097599180468360808563945281389781327557747838772170381060813469985856815104.00 +================================================================================ + explain + ├── columns: tree:6 field:7 description:8 + └── sort + ├── columns: a:1!null b:2 c:3 d:4 + ├── cardinality: [0 - 5] + ├── key: (1) + ├── fd: (1)-->(2-4), (2-4)~~>(1) + ├── ordering: -2 + └── limit + ├── columns: a:1!null b:2 c:3 d:4 + ├── internal-ordering: -2 + ├── cardinality: [0 - 5] + ├── key: (1) + ├── fd: (1)-->(2-4), (2-4)~~>(1) + ├── sort + │ ├── columns: a:1!null b:2 c:3 d:4 + │ ├── cardinality: [0 - 11] + │ ├── key: (1) + │ ├── fd: (1)-->(2-4), (2-4)~~>(1) + │ ├── ordering: -2 + │ ├── limit hint: 5.00 + - │ └── project + + │ └── select + │ ├── columns: a:1!null b:2 c:3 d:4 + │ ├── cardinality: [0 - 11] + │ ├── key: (1) + │ ├── fd: (1)-->(2-4), (2-4)~~>(1) + - │ └── select + - │ ├── columns: a:1!null b:2 c:3 d:4 + - │ ├── cardinality: [0 - 11] + - │ ├── key: (1) + - │ ├── fd: (1)-->(2-4), (2-4)~~>(1) + - │ ├── scan abcd + - │ │ ├── columns: a:1!null b:2 c:3 d:4 + - │ │ ├── flags: force-index=b + - │ │ ├── key: (1) + - │ │ └── fd: (1)-->(2-4), (2-4)~~>(1) + - │ └── filters + - │ └── (a:1 >= 20) AND (a:1 <= 30) [outer=(1), constraints=(/1: [/20 - /30]; tight)] + + │ ├── scan abcd + + │ │ ├── columns: a:1!null b:2 c:3 d:4 + + │ │ ├── flags: force-index=b + + │ │ ├── key: (1) + + │ │ └── fd: (1)-->(2-4), (2-4)~~>(1) + + │ └── filters + │ └── (a:1 >= 20) AND (a:1 <= 30) [outer=(1), constraints=(/1: [/20 - /30]; tight)] └── 5 ================================================================================ @@ -464,7 +577,7 @@ GenerateIndexScans Cost: 5134.92 ================================================================================ explain - ├── columns: tree:5 field:6 description:7 + ├── columns: tree:6 field:7 description:8 - └── sort + └── limit ├── columns: a:1!null b:2 c:3 d:4 @@ -536,7 +649,7 @@ Final best expression Cost: 5134.92 ================================================================================ explain - ├── columns: tree:5 field:6 description:7 + ├── columns: tree:6 field:7 description:8 └── limit ├── columns: a:1!null b:2 c:3 d:4 ├── internal-ordering: -2 @@ -667,30 +780,30 @@ limit │ └── union │ ├── columns: val:2!null data1:6!null │ ├── left columns: val:2!null data1:6!null - │ ├── right columns: val:26 data1:30 + │ ├── right columns: val:29 data1:33 │ ├── cardinality: [0 - 30] │ ├── key: (2,6) │ ├── union │ │ ├── columns: val:2!null data1:6!null - │ │ ├── left columns: val:10 data1:14 - │ │ ├── right columns: val:18 data1:22 + │ │ ├── left columns: val:11 data1:15 + │ │ ├── right columns: val:20 data1:24 │ │ ├── cardinality: [0 - 20] │ │ ├── key: (2,6) │ │ ├── scan index_tab@b - │ │ │ ├── columns: val:10!null data1:14!null - │ │ │ ├── constraint: /10/14/15/9: [/1 - /1] + │ │ │ ├── columns: val:11!null data1:15!null + │ │ │ ├── constraint: /11/15/16/10: [/1 - /1] │ │ │ ├── limit: 10 - │ │ │ └── fd: ()-->(10) + │ │ │ └── fd: ()-->(11) │ │ └── scan index_tab@b - │ │ ├── columns: val:18!null data1:22!null - │ │ ├── constraint: /18/22/23/17: [/2 - /2] + │ │ ├── columns: val:20!null data1:24!null + │ │ ├── constraint: /20/24/25/19: [/2 - /2] │ │ ├── limit: 10 - │ │ └── fd: ()-->(18) + │ │ └── fd: ()-->(20) │ └── scan index_tab@b - │ ├── columns: val:26!null data1:30!null - │ ├── constraint: /26/30/31/25: [/3 - /3] + │ ├── columns: val:29!null data1:33!null + │ ├── constraint: /29/33/34/28: [/3 - /3] │ ├── limit: 10 - │ └── fd: ()-->(26) + │ └── fd: ()-->(29) └── 10 # Case with single-key spans. @@ -698,10 +811,10 @@ opt expect=SplitScanIntoUnionScans SELECT max(data1) FROM index_tab WHERE region = 'US_EAST' OR region = 'US_WEST' ---- scalar-group-by - ├── columns: max:9 + ├── columns: max:10 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(9) + ├── fd: ()-->(10) ├── limit │ ├── columns: region:3!null data1:6!null │ ├── internal-ordering: -6 @@ -716,25 +829,25 @@ scalar-group-by │ │ ├── limit hint: 1.00 │ │ └── union │ │ ├── columns: region:3!null data1:6!null - │ │ ├── left columns: region:12 data1:15 - │ │ ├── right columns: region:20 data1:23 + │ │ ├── left columns: region:13 data1:16 + │ │ ├── right columns: region:22 data1:25 │ │ ├── cardinality: [0 - 2] │ │ ├── key: (3,6) │ │ ├── scan index_tab@c,rev - │ │ │ ├── columns: region:12!null data1:15!null - │ │ │ ├── constraint: /12/15/16/10: [/'US_EAST' - /'US_EAST'] + │ │ │ ├── columns: region:13!null data1:16!null + │ │ │ ├── constraint: /13/16/17/11: [/'US_EAST' - /'US_EAST'] │ │ │ ├── limit: 1(rev) │ │ │ ├── key: () - │ │ │ └── fd: ()-->(12,15) + │ │ │ └── fd: ()-->(13,16) │ │ └── scan index_tab@c,rev - │ │ ├── columns: region:20!null data1:23!null - │ │ ├── constraint: /20/23/24/18: [/'US_WEST' - /'US_WEST'] + │ │ ├── columns: region:22!null data1:25!null + │ │ ├── constraint: /22/25/26/20: [/'US_WEST' - /'US_WEST'] │ │ ├── limit: 1(rev) │ │ ├── key: () - │ │ └── fd: ()-->(20,23) + │ │ └── fd: ()-->(22,25) │ └── 1 └── aggregations - └── const-agg [as=max:9, outer=(6)] + └── const-agg [as=max:10, outer=(6)] └── data1:6 # Case with multi-column keys in single-key spans. @@ -744,10 +857,10 @@ FROM index_tab WHERE (latitude, longitude) = (1, 2) OR (latitude, longitude) = (4, 5) ---- scalar-group-by - ├── columns: max:9 + ├── columns: max:10 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(9) + ├── fd: ()-->(10) ├── limit │ ├── columns: latitude:4!null longitude:5!null data1:6!null │ ├── internal-ordering: -6 @@ -762,25 +875,25 @@ scalar-group-by │ │ ├── limit hint: 1.00 │ │ └── union │ │ ├── columns: latitude:4!null longitude:5!null data1:6!null - │ │ ├── left columns: latitude:13 longitude:14 data1:15 - │ │ ├── right columns: latitude:21 longitude:22 data1:23 + │ │ ├── left columns: latitude:14 longitude:15 data1:16 + │ │ ├── right columns: latitude:23 longitude:24 data1:25 │ │ ├── cardinality: [0 - 2] │ │ ├── key: (4-6) │ │ ├── scan index_tab@d,rev - │ │ │ ├── columns: latitude:13!null longitude:14!null data1:15!null - │ │ │ ├── constraint: /13/14/15/16/10: [/1/2 - /1/2] + │ │ │ ├── columns: latitude:14!null longitude:15!null data1:16!null + │ │ │ ├── constraint: /14/15/16/17/11: [/1/2 - /1/2] │ │ │ ├── limit: 1(rev) │ │ │ ├── key: () - │ │ │ └── fd: ()-->(13-15) + │ │ │ └── fd: ()-->(14-16) │ │ └── scan index_tab@d,rev - │ │ ├── columns: latitude:21!null longitude:22!null data1:23!null - │ │ ├── constraint: /21/22/23/24/18: [/4/5 - /4/5] + │ │ ├── columns: latitude:23!null longitude:24!null data1:25!null + │ │ ├── constraint: /23/24/25/26/20: [/4/5 - /4/5] │ │ ├── limit: 1(rev) │ │ ├── key: () - │ │ └── fd: ()-->(21-23) + │ │ └── fd: ()-->(23-25) │ └── 1 └── aggregations - └── const-agg [as=max:9, outer=(6)] + └── const-agg [as=max:10, outer=(6)] └── data1:6 # Case with countable multi-key spans. @@ -788,10 +901,10 @@ opt expect=SplitScanIntoUnionScans SELECT max(data1) FROM index_tab WHERE val > 0 AND val < 4 ---- scalar-group-by - ├── columns: max:9 + ├── columns: max:10 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(9) + ├── fd: ()-->(10) ├── limit │ ├── columns: val:2!null data1:6!null │ ├── internal-ordering: -6 @@ -807,36 +920,36 @@ scalar-group-by │ │ └── union │ │ ├── columns: val:2!null data1:6!null │ │ ├── left columns: val:2!null data1:6!null - │ │ ├── right columns: val:27 data1:31 + │ │ ├── right columns: val:30 data1:34 │ │ ├── cardinality: [0 - 3] │ │ ├── key: (2,6) │ │ ├── union │ │ │ ├── columns: val:2!null data1:6!null - │ │ │ ├── left columns: val:11 data1:15 - │ │ │ ├── right columns: val:19 data1:23 + │ │ │ ├── left columns: val:12 data1:16 + │ │ │ ├── right columns: val:21 data1:25 │ │ │ ├── cardinality: [0 - 2] │ │ │ ├── key: (2,6) │ │ │ ├── scan index_tab@b,rev - │ │ │ │ ├── columns: val:11!null data1:15!null - │ │ │ │ ├── constraint: /11/15/16/10: [/1 - /1] + │ │ │ │ ├── columns: val:12!null data1:16!null + │ │ │ │ ├── constraint: /12/16/17/11: [/1 - /1] │ │ │ │ ├── limit: 1(rev) │ │ │ │ ├── key: () - │ │ │ │ └── fd: ()-->(11,15) + │ │ │ │ └── fd: ()-->(12,16) │ │ │ └── scan index_tab@b,rev - │ │ │ ├── columns: val:19!null data1:23!null - │ │ │ ├── constraint: /19/23/24/18: [/2 - /2] + │ │ │ ├── columns: val:21!null data1:25!null + │ │ │ ├── constraint: /21/25/26/20: [/2 - /2] │ │ │ ├── limit: 1(rev) │ │ │ ├── key: () - │ │ │ └── fd: ()-->(19,23) + │ │ │ └── fd: ()-->(21,25) │ │ └── scan index_tab@b,rev - │ │ ├── columns: val:27!null data1:31!null - │ │ ├── constraint: /27/31/32/26: [/3 - /3] + │ │ ├── columns: val:30!null data1:34!null + │ │ ├── constraint: /30/34/35/29: [/3 - /3] │ │ ├── limit: 1(rev) │ │ ├── key: () - │ │ └── fd: ()-->(27,31) + │ │ └── fd: ()-->(30,34) │ └── 1 └── aggregations - └── const-agg [as=max:9, outer=(6)] + └── const-agg [as=max:10, outer=(6)] └── data1:6 # Case with limit ordering on more than one column. @@ -860,20 +973,20 @@ limit │ ├── limit hint: 10.00 │ └── union │ ├── columns: region:3!null data1:6!null data2:7!null - │ ├── left columns: region:11 data1:14 data2:15 - │ ├── right columns: region:19 data1:22 data2:23 + │ ├── left columns: region:12 data1:15 data2:16 + │ ├── right columns: region:21 data1:24 data2:25 │ ├── cardinality: [0 - 20] │ ├── key: (3,6,7) │ ├── scan index_tab@c - │ │ ├── columns: region:11!null data1:14!null data2:15!null - │ │ ├── constraint: /11/14/15/9: [/'US_EAST' - /'US_EAST'] + │ │ ├── columns: region:12!null data1:15!null data2:16!null + │ │ ├── constraint: /12/15/16/10: [/'US_EAST' - /'US_EAST'] │ │ ├── limit: 10 - │ │ └── fd: ()-->(11) + │ │ └── fd: ()-->(12) │ └── scan index_tab@c - │ ├── columns: region:19!null data1:22!null data2:23!null - │ ├── constraint: /19/22/23/17: [/'US_WEST' - /'US_WEST'] + │ ├── columns: region:21!null data1:24!null data2:25!null + │ ├── constraint: /21/24/25/19: [/'US_WEST' - /'US_WEST'] │ ├── limit: 10 - │ └── fd: ()-->(19) + │ └── fd: ()-->(21) └── 10 # Case with index join. @@ -903,22 +1016,22 @@ index-join index_tab │ ├── limit hint: 10.00 │ └── union │ ├── columns: id:1!null region:3!null data1:6!null data2:7!null - │ ├── left columns: id:9 region:11 data1:14 data2:15 - │ ├── right columns: id:17 region:19 data1:22 data2:23 + │ ├── left columns: id:10 region:12 data1:15 data2:16 + │ ├── right columns: id:19 region:21 data1:24 data2:25 │ ├── cardinality: [0 - 20] │ ├── key: (1,3,6,7) │ ├── scan index_tab@c - │ │ ├── columns: id:9!null region:11!null data1:14!null data2:15!null - │ │ ├── constraint: /11/14/15/9: [/'US_EAST' - /'US_EAST'] + │ │ ├── columns: id:10!null region:12!null data1:15!null data2:16!null + │ │ ├── constraint: /12/15/16/10: [/'US_EAST' - /'US_EAST'] │ │ ├── limit: 10 - │ │ ├── key: (9) - │ │ └── fd: ()-->(11), (9)-->(14,15) + │ │ ├── key: (10) + │ │ └── fd: ()-->(12), (10)-->(15,16) │ └── scan index_tab@c - │ ├── columns: id:17!null region:19!null data1:22!null data2:23!null - │ ├── constraint: /19/22/23/17: [/'US_WEST' - /'US_WEST'] + │ ├── columns: id:19!null region:21!null data1:24!null data2:25!null + │ ├── constraint: /21/24/25/19: [/'US_WEST' - /'US_WEST'] │ ├── limit: 10 - │ ├── key: (17) - │ └── fd: ()-->(19), (17)-->(22,23) + │ ├── key: (19) + │ └── fd: ()-->(21), (19)-->(24,25) └── 10 # Case where check constraints are used. @@ -941,33 +1054,33 @@ limit │ └── union │ ├── columns: p:1!null q:2!null r:3!null s:4!null │ ├── left columns: p:1!null q:2!null r:3!null s:4!null - │ ├── right columns: p:13 q:14 r:15 s:16 + │ ├── right columns: p:16 q:17 r:18 s:19 │ ├── cardinality: [0 - 15] │ ├── key: (1-4) │ ├── union │ │ ├── columns: p:1!null q:2!null r:3!null s:4!null - │ │ ├── left columns: p:5 q:6 r:7 s:8 - │ │ ├── right columns: p:9 q:10 r:11 s:12 + │ │ ├── left columns: p:6 q:7 r:8 s:9 + │ │ ├── right columns: p:11 q:12 r:13 s:14 │ │ ├── cardinality: [0 - 10] │ │ ├── key: (1-4) │ │ ├── scan pqrs - │ │ │ ├── columns: p:5!null q:6!null r:7!null s:8!null - │ │ │ ├── constraint: /5/6: [/1 - /1] + │ │ │ ├── columns: p:6!null q:7!null r:8!null s:9!null + │ │ │ ├── constraint: /6/7: [/1 - /1] │ │ │ ├── limit: 5 - │ │ │ ├── key: (6) - │ │ │ └── fd: ()-->(5), (6)-->(7,8) + │ │ │ ├── key: (7) + │ │ │ └── fd: ()-->(6), (7)-->(8,9) │ │ └── scan pqrs - │ │ ├── columns: p:9!null q:10!null r:11!null s:12!null - │ │ ├── constraint: /9/10: [/5 - /5] + │ │ ├── columns: p:11!null q:12!null r:13!null s:14!null + │ │ ├── constraint: /11/12: [/5 - /5] │ │ ├── limit: 5 - │ │ ├── key: (10) - │ │ └── fd: ()-->(9), (10)-->(11,12) + │ │ ├── key: (12) + │ │ └── fd: ()-->(11), (12)-->(13,14) │ └── scan pqrs - │ ├── columns: p:13!null q:14!null r:15!null s:16!null - │ ├── constraint: /13/14: [/10 - /10] + │ ├── columns: p:16!null q:17!null r:18!null s:19!null + │ ├── constraint: /16/17: [/10 - /10] │ ├── limit: 5 - │ ├── key: (14) - │ └── fd: ()-->(13), (14)-->(15,16) + │ ├── key: (17) + │ └── fd: ()-->(16), (17)-->(18,19) └── 5 # Case where multiple check constraints are combined into one constraint @@ -990,22 +1103,22 @@ limit │ ├── limit hint: 10.00 │ └── union │ ├── columns: p:1!null q:2!null r:3!null s:4!null - │ ├── left columns: p:5 q:6 r:7 s:8 - │ ├── right columns: p:9 q:10 r:11 s:12 + │ ├── left columns: p:6 q:7 r:8 s:9 + │ ├── right columns: p:11 q:12 r:13 s:14 │ ├── cardinality: [0 - 20] │ ├── key: (1-4) │ ├── scan pqrs@secondary - │ │ ├── columns: p:5!null q:6!null r:7!null s:8!null - │ │ ├── constraint: /7/8/5/6: [/1 - /1] + │ │ ├── columns: p:6!null q:7!null r:8!null s:9!null + │ │ ├── constraint: /8/9/6/7: [/1 - /1] │ │ ├── limit: 10 - │ │ ├── key: (5,6) - │ │ └── fd: ()-->(7), (5,6)-->(8) + │ │ ├── key: (6,7) + │ │ └── fd: ()-->(8), (6,7)-->(9) │ └── scan pqrs@secondary - │ ├── columns: p:9!null q:10!null r:11!null s:12!null - │ ├── constraint: /11/12/9/10: [/2 - /2] + │ ├── columns: p:11!null q:12!null r:13!null s:14!null + │ ├── constraint: /13/14/11/12: [/2 - /2] │ ├── limit: 10 - │ ├── key: (9,10) - │ └── fd: ()-->(11), (9,10)-->(12) + │ ├── key: (11,12) + │ └── fd: ()-->(13), (11,12)-->(14) └── 10 # Check constraints are not used because the scan is already constrained (the @@ -1028,22 +1141,22 @@ limit │ ├── limit hint: 5.00 │ └── union │ ├── columns: p:1!null q:2!null r:3!null s:4!null - │ ├── left columns: p:5 q:6 r:7 s:8 - │ ├── right columns: p:9 q:10 r:11 s:12 + │ ├── left columns: p:6 q:7 r:8 s:9 + │ ├── right columns: p:11 q:12 r:13 s:14 │ ├── cardinality: [0 - 10] │ ├── key: (1-4) │ ├── scan pqrs - │ │ ├── columns: p:5!null q:6!null r:7!null s:8!null - │ │ ├── constraint: /5/6: [/1 - /1] + │ │ ├── columns: p:6!null q:7!null r:8!null s:9!null + │ │ ├── constraint: /6/7: [/1 - /1] │ │ ├── limit: 5 - │ │ ├── key: (6) - │ │ └── fd: ()-->(5), (6)-->(7,8) + │ │ ├── key: (7) + │ │ └── fd: ()-->(6), (7)-->(8,9) │ └── scan pqrs - │ ├── columns: p:9!null q:10!null r:11!null s:12!null - │ ├── constraint: /9/10: [/5 - /5] + │ ├── columns: p:11!null q:12!null r:13!null s:14!null + │ ├── constraint: /11/12: [/5 - /5] │ ├── limit: 5 - │ ├── key: (10) - │ └── fd: ()-->(9), (10)-->(11,12) + │ ├── key: (12) + │ └── fd: ()-->(11), (12)-->(13,14) └── 5 # No-op case because the scan has an inverted index. @@ -1081,15 +1194,15 @@ opt expect-not=SplitScanIntoUnionScans SELECT max(data1) FROM index_tab WHERE region > 'US_EAST' AND region < 'US_WEST' ---- scalar-group-by - ├── columns: max:9 + ├── columns: max:10 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(9) + ├── fd: ()-->(10) ├── scan index_tab@c │ ├── columns: region:3!null data1:6!null │ └── constraint: /3/6/7/1: [/e'US_EAST\x00' - /'US_WEST') └── aggregations - └── max [as=max:9, outer=(6)] + └── max [as=max:10, outer=(6)] └── data1:6 # No-op case because the number of keys exceeds maxScanCount. @@ -1097,15 +1210,15 @@ opt expect-not=SplitScanIntoUnionScans SELECT max(data1) FROM index_tab WHERE val > 0 AND val < 20 ---- scalar-group-by - ├── columns: max:9 + ├── columns: max:10 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(9) + ├── fd: ()-->(10) ├── scan index_tab@b │ ├── columns: val:2!null data1:6!null │ └── constraint: /2/6/7/1: [/1 - /19] └── aggregations - └── max [as=max:9, outer=(6)] + └── max [as=max:10, outer=(6)] └── data1:6 # No-op case because the same number of rows would be scanned by the split-up @@ -1114,10 +1227,10 @@ opt expect-not=SplitScanIntoUnionScans SELECT max(data1) FROM index_tab WHERE id > 0 AND id < 4 ---- scalar-group-by - ├── columns: max:9 + ├── columns: max:10 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(9) + ├── fd: ()-->(10) ├── scan index_tab@a │ ├── columns: id:1!null data1:6!null │ ├── constraint: /1/6/7: [/1 - /3] @@ -1125,7 +1238,7 @@ scalar-group-by │ ├── key: (1) │ └── fd: (1)-->(6) └── aggregations - └── max [as=max:9, outer=(6)] + └── max [as=max:10, outer=(6)] └── data1:6 # No-op case because the scan is already limited. @@ -1135,10 +1248,10 @@ FROM (SELECT region, data1 FROM index_tab LIMIT 10) WHERE region='ASIA' OR region='AUSTRALIA' ---- scalar-group-by - ├── columns: max:9 + ├── columns: max:10 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(9) + ├── fd: ()-->(10) ├── select │ ├── columns: region:3!null data1:6!null │ ├── cardinality: [0 - 10] @@ -1148,7 +1261,7 @@ scalar-group-by │ └── filters │ └── (region:3 = 'ASIA') OR (region:3 = 'AUSTRALIA') [outer=(3), constraints=(/3: [/'ASIA' - /'ASIA'] [/'AUSTRALIA' - /'AUSTRALIA']; tight)] └── aggregations - └── max [as=max:9, outer=(6)] + └── max [as=max:10, outer=(6)] └── data1:6 # No-op case because the limit is negative. @@ -1180,13 +1293,13 @@ opt expect-not=SplitScanIntoUnionScans SELECT max(data1) FROM index_tab@b ---- scalar-group-by - ├── columns: max:9 + ├── columns: max:10 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(9) + ├── fd: ()-->(10) ├── scan index_tab@b │ ├── columns: data1:6!null │ └── flags: force-index=b └── aggregations - └── max [as=max:9, outer=(6)] + └── max [as=max:10, outer=(6)] └── data1:6 diff --git a/pkg/sql/opt/xform/testdata/rules/partitioned b/pkg/sql/opt/xform/testdata/rules/partitioned index a611668a76e6..68fd0293b1c8 100644 --- a/pkg/sql/opt/xform/testdata/rules/partitioned +++ b/pkg/sql/opt/xform/testdata/rules/partitioned @@ -33,7 +33,7 @@ EXPLAIN (OPT, VERBOSE) val = 1 ---- explain - ├── columns: text:5 + ├── columns: text:6 ├── mode: opt, verbose └── select ├── columns: planet:1!null region:2!null subregion:3!null val:4!null diff --git a/pkg/sql/opt/xform/testdata/rules/scan b/pkg/sql/opt/xform/testdata/rules/scan index 305e2b64c294..4f80852b7e55 100644 --- a/pkg/sql/opt/xform/testdata/rules/scan +++ b/pkg/sql/opt/xform/testdata/rules/scan @@ -348,9 +348,9 @@ CREATE TABLE abc ( memo SELECT d FROM abc ORDER BY lower(d) ---- -memo (optimized, ~3KB, required=[presentation: d:4] [ordering: +5]) +memo (optimized, ~3KB, required=[presentation: d:4] [ordering: +6]) ├── G1: (project G2 G3 d) - │ ├── [presentation: d:4] [ordering: +5] + │ ├── [presentation: d:4] [ordering: +6] │ │ ├── best: (sort G1) │ │ └── cost: 1289.36 │ └── [] @@ -583,10 +583,10 @@ opt SELECT sum(total) FROM "orders" WHERE seq_num >= 100 AND seq_num < 200 ---- scalar-group-by - ├── columns: sum:5 + ├── columns: sum:6 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(5) + ├── fd: ()-->(6) ├── scan orders@orders_by_seq_num │ ├── columns: total:3!null seq_num:4!null │ └── constraint: /1/4/2 @@ -594,7 +594,7 @@ scalar-group-by │ ├── [/'us-east1'/100 - /'us-east1'/199] │ └── [/'us-west1'/100 - /'us-west1'/199] └── aggregations - └── sum [as=sum:5, outer=(3)] + └── sum [as=sum:6, outer=(3)] └── total:3 exec-ddl diff --git a/pkg/sql/opt/xform/testdata/rules/select b/pkg/sql/opt/xform/testdata/rules/select index 132a942c682e..b39a4ab569ac 100644 --- a/pkg/sql/opt/xform/testdata/rules/select +++ b/pkg/sql/opt/xform/testdata/rules/select @@ -481,7 +481,7 @@ project memo SELECT k FROM a WHERE u = 1 AND v = 5 ---- -memo (optimized, ~6KB, required=[presentation: k:1]) +memo (optimized, ~9KB, required=[presentation: k:1]) ├── G1: (project G2 G3 k) │ └── [presentation: k:1] │ ├── best: (project G2 G3 k) @@ -491,19 +491,19 @@ memo (optimized, ~6KB, required=[presentation: k:1]) │ ├── best: (select G8 G9) │ └── cost: 1.10 ├── G3: (projections) - ├── G4: (scan a) (scan a@u) (scan a@v) + ├── G4: (scan a,cols=(1-3)) (scan a@u,cols=(1-3)) (scan a@v,cols=(1-3)) │ └── [] - │ ├── best: (scan a) + │ ├── best: (scan a,cols=(1-3)) │ └── cost: 1060.02 ├── G5: (filters G10 G11) - ├── G6: (scan a@u,constrained) + ├── G6: (scan a@u,cols=(1-3),constrained) │ └── [] - │ ├── best: (scan a@u,constrained) + │ ├── best: (scan a@u,cols=(1-3),constrained) │ └── cost: 10.61 ├── G7: (filters G11) - ├── G8: (scan a@v,constrained) + ├── G8: (scan a@v,cols=(1-3),constrained) │ └── [] - │ ├── best: (scan a@v,constrained) + │ ├── best: (scan a@v,cols=(1-3),constrained) │ └── cost: 1.07 ├── G9: (filters G10) ├── G10: (eq G12 G13) @@ -573,14 +573,14 @@ index-join b memo SELECT * FROM b WHERE v >= 1 AND v <= 10 ---- -memo (optimized, ~3KB, required=[presentation: k:1,u:2,v:3,j:4]) +memo (optimized, ~6KB, required=[presentation: k:1,u:2,v:3,j:4]) ├── G1: (select G2 G3) (index-join G4 b,cols=(1-4)) │ └── [presentation: k:1,u:2,v:3,j:4] │ ├── best: (index-join G4 b,cols=(1-4)) │ └── cost: 51.32 - ├── G2: (scan b) + ├── G2: (scan b,cols=(1-4)) │ └── [] - │ ├── best: (scan b) + │ ├── best: (scan b,cols=(1-4)) │ └── cost: 1080.02 ├── G3: (filters G5) ├── G4: (scan b@v,cols=(1,3),constrained) @@ -635,19 +635,19 @@ index-join b memo SELECT * FROM b WHERE v >= 1 AND v <= 10 AND k > 5 ---- -memo (optimized, ~6KB, required=[presentation: k:1,u:2,v:3,j:4]) +memo (optimized, ~8KB, required=[presentation: k:1,u:2,v:3,j:4]) ├── G1: (select G2 G3) (select G4 G5) (index-join G6 b,cols=(1-4)) │ └── [presentation: k:1,u:2,v:3,j:4] │ ├── best: (index-join G6 b,cols=(1-4)) │ └── cost: 24.17 - ├── G2: (scan b) + ├── G2: (scan b,cols=(1-4)) │ └── [] - │ ├── best: (scan b) + │ ├── best: (scan b,cols=(1-4)) │ └── cost: 1080.02 ├── G3: (filters G7 G8) - ├── G4: (scan b,constrained) + ├── G4: (scan b,cols=(1-4),constrained) │ └── [] - │ ├── best: (scan b,constrained) + │ ├── best: (scan b,cols=(1-4),constrained) │ └── cost: 360.01 ├── G5: (filters G7) ├── G6: (select G9 G10) @@ -733,14 +733,14 @@ select memo SELECT * FROM b WHERE v >= 1 AND v <= 10 AND k+u = 1 ---- -memo (optimized, ~5KB, required=[presentation: k:1,u:2,v:3,j:4]) +memo (optimized, ~7KB, required=[presentation: k:1,u:2,v:3,j:4]) ├── G1: (select G2 G3) (select G4 G5) │ └── [presentation: k:1,u:2,v:3,j:4] │ ├── best: (select G4 G5) │ └── cost: 51.44 - ├── G2: (scan b) + ├── G2: (scan b,cols=(1-4)) │ └── [] - │ ├── best: (scan b) + │ ├── best: (scan b,cols=(1-4)) │ └── cost: 1080.02 ├── G3: (filters G6 G7) ├── G4: (index-join G8 b,cols=(1-4)) @@ -797,19 +797,19 @@ select memo SELECT * FROM b WHERE v >= 1 AND v <= 10 AND k+u = 1 AND k > 5 ---- -memo (optimized, ~7KB, required=[presentation: k:1,u:2,v:3,j:4]) +memo (optimized, ~10KB, required=[presentation: k:1,u:2,v:3,j:4]) ├── G1: (select G2 G3) (select G4 G5) (select G6 G7) │ └── [presentation: k:1,u:2,v:3,j:4] │ ├── best: (select G6 G7) │ └── cost: 24.23 - ├── G2: (scan b) + ├── G2: (scan b,cols=(1-4)) │ └── [] - │ ├── best: (scan b) + │ ├── best: (scan b,cols=(1-4)) │ └── cost: 1080.02 ├── G3: (filters G8 G9 G10) - ├── G4: (scan b,constrained) + ├── G4: (scan b,cols=(1-4),constrained) │ └── [] - │ ├── best: (scan b,constrained) + │ ├── best: (scan b,cols=(1-4),constrained) │ └── cost: 360.01 ├── G5: (filters G8 G9) ├── G6: (index-join G11 b,cols=(1-4)) @@ -865,14 +865,14 @@ select memo SELECT * FROM b WHERE (u, k, v) > (1, 2, 3) AND (u, k, v) < (8, 9, 10) ---- -memo (optimized, ~5KB, required=[presentation: k:1,u:2,v:3,j:4]) +memo (optimized, ~7KB, required=[presentation: k:1,u:2,v:3,j:4]) ├── G1: (select G2 G3) (select G4 G3) │ └── [presentation: k:1,u:2,v:3,j:4] │ ├── best: (select G4 G3) │ └── cost: 411.25 - ├── G2: (scan b) + ├── G2: (scan b,cols=(1-4)) │ └── [] - │ ├── best: (scan b) + │ ├── best: (scan b,cols=(1-4)) │ └── cost: 1080.02 ├── G3: (filters G5 G6) ├── G4: (index-join G7 b,cols=(1-4)) @@ -1931,7 +1931,7 @@ project ├── union-all │ ├── columns: k:1!null u:2 v:3 │ ├── left columns: k:1!null u:2 v:3 - │ ├── right columns: k:5 u:6 v:7 + │ ├── right columns: k:6 u:7 v:8 │ ├── index-join d │ │ ├── columns: k:1!null u:2!null v:3 │ │ ├── key: (1) @@ -1942,14 +1942,14 @@ project │ │ ├── key: (1) │ │ └── fd: ()-->(2) │ └── index-join d - │ ├── columns: k:5!null u:6 v:7!null - │ ├── key: (5) - │ ├── fd: ()-->(7), (5)-->(6) + │ ├── columns: k:6!null u:7 v:8!null + │ ├── key: (6) + │ ├── fd: ()-->(8), (6)-->(7) │ └── scan d@v - │ ├── columns: k:5!null v:7!null - │ ├── constraint: /7/5: [/1 - /1] - │ ├── key: (5) - │ └── fd: ()-->(7) + │ ├── columns: k:6!null v:8!null + │ ├── constraint: /8/6: [/1 - /1] + │ ├── key: (6) + │ └── fd: ()-->(8) └── aggregations ├── const-agg [as=u:2, outer=(2)] │ └── u:2 @@ -1967,7 +1967,7 @@ distinct-on ├── union-all │ ├── columns: k:1!null u:2 v:3 w:4!null │ ├── left columns: k:1!null u:2 v:3 w:4!null - │ ├── right columns: k:5 u:6 v:7 w:8 + │ ├── right columns: k:6 u:7 v:8 w:9 │ ├── select │ │ ├── columns: k:1!null u:2!null v:3 w:4!null │ │ ├── key: (1) @@ -1984,20 +1984,20 @@ distinct-on │ │ └── filters │ │ └── w:4 = 1 [outer=(4), constraints=(/4: [/1 - /1]; tight), fd=()-->(4)] │ └── select - │ ├── columns: k:5!null u:6 v:7!null w:8!null - │ ├── key: (5) - │ ├── fd: ()-->(7,8), (5)-->(6) + │ ├── columns: k:6!null u:7 v:8!null w:9!null + │ ├── key: (6) + │ ├── fd: ()-->(8,9), (6)-->(7) │ ├── index-join d - │ │ ├── columns: k:5!null u:6 v:7 w:8 - │ │ ├── key: (5) - │ │ ├── fd: ()-->(7), (5)-->(6,8) + │ │ ├── columns: k:6!null u:7 v:8 w:9 + │ │ ├── key: (6) + │ │ ├── fd: ()-->(8), (6)-->(7,9) │ │ └── scan d@v - │ │ ├── columns: k:5!null v:7!null - │ │ ├── constraint: /7/5: [/1 - /1] - │ │ ├── key: (5) - │ │ └── fd: ()-->(7) + │ │ ├── columns: k:6!null v:8!null + │ │ ├── constraint: /8/6: [/1 - /1] + │ │ ├── key: (6) + │ │ └── fd: ()-->(8) │ └── filters - │ └── w:8 = 1 [outer=(8), constraints=(/8: [/1 - /1]; tight), fd=()-->(8)] + │ └── w:9 = 1 [outer=(9), constraints=(/9: [/1 - /1]; tight), fd=()-->(9)] └── aggregations ├── const-agg [as=u:2, outer=(2)] │ └── u:2 @@ -2020,7 +2020,7 @@ project ├── union-all │ ├── columns: k:1!null u:2!null v:3!null │ ├── left columns: k:1!null u:2!null v:3!null - │ ├── right columns: k:5 u:6 v:7 + │ ├── right columns: k:6 u:7 v:8 │ ├── inner-join (zigzag d@u d@v) │ │ ├── columns: k:1!null u:2!null v:3!null │ │ ├── eq columns: [1] = [1] @@ -2032,15 +2032,15 @@ project │ │ ├── u:2 = 1 [outer=(2), constraints=(/2: [/1 - /1]; tight), fd=()-->(2)] │ │ └── v:3 = 20 [outer=(3), constraints=(/3: [/20 - /20]; tight), fd=()-->(3)] │ └── inner-join (zigzag d@u d@v) - │ ├── columns: k:5!null u:6!null v:7!null - │ ├── eq columns: [5] = [5] - │ ├── left fixed columns: [6] = [10] - │ ├── right fixed columns: [7] = [2] - │ ├── key: (5) - │ ├── fd: ()-->(6,7) + │ ├── columns: k:6!null u:7!null v:8!null + │ ├── eq columns: [6] = [6] + │ ├── left fixed columns: [7] = [10] + │ ├── right fixed columns: [8] = [2] + │ ├── key: (6) + │ ├── fd: ()-->(7,8) │ └── filters - │ ├── v:7 = 2 [outer=(7), constraints=(/7: [/2 - /2]; tight), fd=()-->(7)] - │ └── u:6 = 10 [outer=(6), constraints=(/6: [/10 - /10]; tight), fd=()-->(6)] + │ ├── v:8 = 2 [outer=(8), constraints=(/8: [/2 - /2]; tight), fd=()-->(8)] + │ └── u:7 = 10 [outer=(7), constraints=(/7: [/10 - /10]; tight), fd=()-->(7)] └── aggregations ├── const-agg [as=u:2, outer=(2)] │ └── u:2 @@ -2051,10 +2051,10 @@ opt expect=SplitDisjunction SELECT sum(k) FROM d WHERE u = 1 OR v = 1 ---- scalar-group-by - ├── columns: sum:5 + ├── columns: sum:6 ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(5) + ├── fd: ()-->(6) ├── distinct-on │ ├── columns: k:1!null u:2 v:3 │ ├── grouping columns: k:1!null @@ -2063,7 +2063,7 @@ scalar-group-by │ ├── union-all │ │ ├── columns: k:1!null u:2 v:3 │ │ ├── left columns: k:1!null u:2 v:3 - │ │ ├── right columns: k:6 u:7 v:8 + │ │ ├── right columns: k:7 u:8 v:9 │ │ ├── index-join d │ │ │ ├── columns: k:1!null u:2!null v:3 │ │ │ ├── key: (1) @@ -2074,21 +2074,21 @@ scalar-group-by │ │ │ ├── key: (1) │ │ │ └── fd: ()-->(2) │ │ └── index-join d - │ │ ├── columns: k:6!null u:7 v:8!null - │ │ ├── key: (6) - │ │ ├── fd: ()-->(8), (6)-->(7) + │ │ ├── columns: k:7!null u:8 v:9!null + │ │ ├── key: (7) + │ │ ├── fd: ()-->(9), (7)-->(8) │ │ └── scan d@v - │ │ ├── columns: k:6!null v:8!null - │ │ ├── constraint: /8/6: [/1 - /1] - │ │ ├── key: (6) - │ │ └── fd: ()-->(8) + │ │ ├── columns: k:7!null v:9!null + │ │ ├── constraint: /9/7: [/1 - /1] + │ │ ├── key: (7) + │ │ └── fd: ()-->(9) │ └── aggregations │ ├── const-agg [as=u:2, outer=(2)] │ │ └── u:2 │ └── const-agg [as=v:3, outer=(3)] │ └── v:3 └── aggregations - └── sum [as=sum:5, outer=(1)] + └── sum [as=sum:6, outer=(1)] └── k:1 # Multi-column primary key. @@ -2106,7 +2106,7 @@ project ├── union-all │ ├── columns: k:1!null j:2!null u:3 v:4 │ ├── left columns: k:1!null j:2!null u:3 v:4 - │ ├── right columns: k:5 j:6 u:7 v:8 + │ ├── right columns: k:6 j:7 u:8 v:9 │ ├── index-join f │ │ ├── columns: k:1!null j:2!null u:3!null v:4 │ │ ├── key: (1,2) @@ -2117,14 +2117,14 @@ project │ │ ├── key: (1,2) │ │ └── fd: ()-->(3) │ └── index-join f - │ ├── columns: k:5!null j:6!null u:7 v:8!null - │ ├── key: (5,6) - │ ├── fd: ()-->(8), (5,6)-->(7) + │ ├── columns: k:6!null j:7!null u:8 v:9!null + │ ├── key: (6,7) + │ ├── fd: ()-->(9), (6,7)-->(8) │ └── scan f@v - │ ├── columns: k:5!null j:6!null v:8!null - │ ├── constraint: /8/5/6: [/2 - /2] - │ ├── key: (5,6) - │ └── fd: ()-->(8) + │ ├── columns: k:6!null j:7!null v:9!null + │ ├── constraint: /9/6/7: [/2 - /2] + │ ├── key: (6,7) + │ └── fd: ()-->(9) └── aggregations ├── const-agg [as=u:3, outer=(3)] │ └── u:3 @@ -2146,7 +2146,7 @@ project ├── union-all │ ├── columns: k:1!null u:2 v:3 │ ├── left columns: k:1!null u:2 v:3 - │ ├── right columns: k:5 u:6 v:7 + │ ├── right columns: k:6 u:7 v:8 │ ├── index-join d │ │ ├── columns: k:1!null u:2!null v:3 │ │ ├── key: (1) @@ -2157,14 +2157,14 @@ project │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ └── index-join d - │ ├── columns: k:5!null u:6 v:7!null - │ ├── key: (5) - │ ├── fd: (5)-->(6,7) + │ ├── columns: k:6!null u:7 v:8!null + │ ├── key: (6) + │ ├── fd: (6)-->(7,8) │ └── scan d@v - │ ├── columns: k:5!null v:7!null - │ ├── constraint: /7/5: [/5 - /8] - │ ├── key: (5) - │ └── fd: (5)-->(7) + │ ├── columns: k:6!null v:8!null + │ ├── constraint: /8/6: [/5 - /8] + │ ├── key: (6) + │ └── fd: (6)-->(8) └── aggregations ├── const-agg [as=u:2, outer=(2)] │ └── u:2 @@ -2188,7 +2188,7 @@ project ├── union-all │ ├── columns: k:1!null j:4 │ ├── left columns: k:1!null j:4 - │ ├── right columns: k:5 j:8 + │ ├── right columns: k:6 j:9 │ ├── immutable │ ├── scan b │ │ ├── columns: k:1!null j:4 @@ -2197,14 +2197,14 @@ project │ │ ├── key: () │ │ └── fd: ()-->(1,4) │ └── index-join b - │ ├── columns: k:5!null j:8 + │ ├── columns: k:6!null j:9 │ ├── immutable - │ ├── key: (5) - │ ├── fd: (5)-->(8) + │ ├── key: (6) + │ ├── fd: (6)-->(9) │ └── scan b@inv_idx - │ ├── columns: k:5!null - │ ├── constraint: /8/5: [/'{"foo": "bar"}' - /'{"foo": "bar"}'] - │ └── key: (5) + │ ├── columns: k:6!null + │ ├── constraint: /9/6: [/'{"foo": "bar"}' - /'{"foo": "bar"}'] + │ └── key: (6) └── aggregations └── const-agg [as=j:4, outer=(4)] └── j:4 @@ -2226,7 +2226,7 @@ project ├── union-all │ ├── columns: k:1!null a:2 │ ├── left columns: k:1!null a:2 - │ ├── right columns: k:4 a:5 + │ ├── right columns: k:5 a:6 │ ├── immutable │ ├── scan c │ │ ├── columns: k:1!null a:2 @@ -2235,14 +2235,14 @@ project │ │ ├── key: () │ │ └── fd: ()-->(1,2) │ └── index-join c - │ ├── columns: k:4!null a:5 + │ ├── columns: k:5!null a:6 │ ├── immutable - │ ├── key: (4) - │ ├── fd: (4)-->(5) + │ ├── key: (5) + │ ├── fd: (5)-->(6) │ └── scan c@inv_idx - │ ├── columns: k:4!null - │ ├── constraint: /5/4: [/ARRAY[2] - /ARRAY[2]] - │ └── key: (4) + │ ├── columns: k:5!null + │ ├── constraint: /6/5: [/ARRAY[2] - /ARRAY[2]] + │ └── key: (5) └── aggregations └── const-agg [as=a:2, outer=(2)] └── a:2 @@ -2262,7 +2262,7 @@ project ├── union-all │ ├── columns: d.k:1!null d.u:2 d.v:3 │ ├── left columns: d.k:1!null d.u:2 d.v:3 - │ ├── right columns: d.k:8 d.u:9 d.v:10 + │ ├── right columns: d.k:10 d.u:11 d.v:12 │ ├── index-join d │ │ ├── columns: d.k:1!null d.u:2!null d.v:3 │ │ ├── key: (1) @@ -2279,30 +2279,30 @@ project │ │ └── filters │ │ └── exists [subquery] │ │ └── scan a - │ │ ├── columns: a.u:6 a.v:7 + │ │ ├── columns: a.u:7 a.v:8 │ │ ├── limit: 1 │ │ ├── key: () - │ │ └── fd: ()-->(6,7) + │ │ └── fd: ()-->(7,8) │ └── index-join d - │ ├── columns: d.k:8!null d.u:9 d.v:10!null - │ ├── key: (8) - │ ├── fd: ()-->(10), (8)-->(9) + │ ├── columns: d.k:10!null d.u:11 d.v:12!null + │ ├── key: (10) + │ ├── fd: ()-->(12), (10)-->(11) │ └── select - │ ├── columns: d.k:8!null d.v:10!null - │ ├── key: (8) - │ ├── fd: ()-->(10) + │ ├── columns: d.k:10!null d.v:12!null + │ ├── key: (10) + │ ├── fd: ()-->(12) │ ├── scan d@v - │ │ ├── columns: d.k:8!null d.v:10!null - │ │ ├── constraint: /10/8: [/1 - /1] - │ │ ├── key: (8) - │ │ └── fd: ()-->(10) + │ │ ├── columns: d.k:10!null d.v:12!null + │ │ ├── constraint: /12/10: [/1 - /1] + │ │ ├── key: (10) + │ │ └── fd: ()-->(12) │ └── filters │ └── exists [subquery] │ └── scan a - │ ├── columns: a.u:6 a.v:7 + │ ├── columns: a.u:7 a.v:8 │ ├── limit: 1 │ ├── key: () - │ └── fd: ()-->(6,7) + │ └── fd: ()-->(7,8) └── aggregations ├── const-agg [as=d.u:2, outer=(2)] │ └── d.u:2 @@ -2321,10 +2321,10 @@ project ├── key: (1) ├── fd: (1)-->(2,3) └── inner-join (hash) - ├── columns: d.k:1!null d.u:2!null d.v:3 a.u:6!null + ├── columns: d.k:1!null d.u:2!null d.v:3 a.u:7!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) ├── key: (1) - ├── fd: (1)-->(2,3), (2)==(6), (6)==(2) + ├── fd: (1)-->(2,3), (2)==(7), (7)==(2) ├── distinct-on │ ├── columns: d.k:1!null d.u:2 d.v:3 │ ├── grouping columns: d.k:1!null @@ -2333,7 +2333,7 @@ project │ ├── union-all │ │ ├── columns: d.k:1!null d.u:2 d.v:3 │ │ ├── left columns: d.k:1!null d.u:2 d.v:3 - │ │ ├── right columns: d.k:8 d.u:9 d.v:10 + │ │ ├── right columns: d.k:10 d.u:11 d.v:12 │ │ ├── index-join d │ │ │ ├── columns: d.k:1!null d.u:2!null d.v:3 │ │ │ ├── key: (1) @@ -2344,29 +2344,29 @@ project │ │ │ ├── key: (1) │ │ │ └── fd: ()-->(2) │ │ └── index-join d - │ │ ├── columns: d.k:8!null d.u:9 d.v:10!null - │ │ ├── key: (8) - │ │ ├── fd: ()-->(10), (8)-->(9) + │ │ ├── columns: d.k:10!null d.u:11 d.v:12!null + │ │ ├── key: (10) + │ │ ├── fd: ()-->(12), (10)-->(11) │ │ └── scan d@v - │ │ ├── columns: d.k:8!null d.v:10!null - │ │ ├── constraint: /10/8: [/1 - /1] - │ │ ├── key: (8) - │ │ └── fd: ()-->(10) + │ │ ├── columns: d.k:10!null d.v:12!null + │ │ ├── constraint: /12/10: [/1 - /1] + │ │ ├── key: (10) + │ │ └── fd: ()-->(12) │ └── aggregations │ ├── const-agg [as=d.u:2, outer=(2)] │ │ └── d.u:2 │ └── const-agg [as=d.v:3, outer=(3)] │ └── d.v:3 ├── distinct-on - │ ├── columns: a.u:6 - │ ├── grouping columns: a.u:6 - │ ├── internal-ordering: +6 - │ ├── key: (6) + │ ├── columns: a.u:7 + │ ├── grouping columns: a.u:7 + │ ├── internal-ordering: +7 + │ ├── key: (7) │ └── scan a@u - │ ├── columns: a.u:6 - │ └── ordering: +6 + │ ├── columns: a.u:7 + │ └── ordering: +7 └── filters - └── a.u:6 = d.u:2 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + └── a.u:7 = d.u:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] # Correlated subquery with references to outer columns not in the scan columns. opt expect=SplitDisjunction @@ -2380,10 +2380,10 @@ project ├── key: (1) ├── fd: (1)-->(2-4) └── inner-join (hash) - ├── columns: d.k:1!null d.u:2 d.v:3 w:4!null a.u:6!null + ├── columns: d.k:1!null d.u:2 d.v:3 w:4!null a.u:7!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) ├── key: (1) - ├── fd: (1)-->(2-4), (4)==(6), (6)==(4) + ├── fd: (1)-->(2-4), (4)==(7), (7)==(4) ├── distinct-on │ ├── columns: d.k:1!null d.u:2 d.v:3 w:4 │ ├── grouping columns: d.k:1!null @@ -2392,7 +2392,7 @@ project │ ├── union-all │ │ ├── columns: d.k:1!null d.u:2 d.v:3 w:4 │ │ ├── left columns: d.k:1!null d.u:2 d.v:3 w:4 - │ │ ├── right columns: d.k:8 d.u:9 d.v:10 w:11 + │ │ ├── right columns: d.k:10 d.u:11 d.v:12 w:13 │ │ ├── index-join d │ │ │ ├── columns: d.k:1!null d.u:2!null d.v:3 w:4 │ │ │ ├── key: (1) @@ -2403,14 +2403,14 @@ project │ │ │ ├── key: (1) │ │ │ └── fd: ()-->(2) │ │ └── index-join d - │ │ ├── columns: d.k:8!null d.u:9 d.v:10!null w:11 - │ │ ├── key: (8) - │ │ ├── fd: ()-->(10), (8)-->(9,11) + │ │ ├── columns: d.k:10!null d.u:11 d.v:12!null w:13 + │ │ ├── key: (10) + │ │ ├── fd: ()-->(12), (10)-->(11,13) │ │ └── scan d@v - │ │ ├── columns: d.k:8!null d.v:10!null - │ │ ├── constraint: /10/8: [/1 - /1] - │ │ ├── key: (8) - │ │ └── fd: ()-->(10) + │ │ ├── columns: d.k:10!null d.v:12!null + │ │ ├── constraint: /12/10: [/1 - /1] + │ │ ├── key: (10) + │ │ └── fd: ()-->(12) │ └── aggregations │ ├── const-agg [as=d.u:2, outer=(2)] │ │ └── d.u:2 @@ -2419,15 +2419,15 @@ project │ └── const-agg [as=w:4, outer=(4)] │ └── w:4 ├── distinct-on - │ ├── columns: a.u:6 - │ ├── grouping columns: a.u:6 - │ ├── internal-ordering: +6 - │ ├── key: (6) + │ ├── columns: a.u:7 + │ ├── grouping columns: a.u:7 + │ ├── internal-ordering: +7 + │ ├── key: (7) │ └── scan a@u - │ ├── columns: a.u:6 - │ └── ordering: +6 + │ ├── columns: a.u:7 + │ └── ordering: +7 └── filters - └── a.u:6 = w:4 [outer=(4,6), constraints=(/4: (/NULL - ]; /6: (/NULL - ]), fd=(4)==(6), (6)==(4)] + └── a.u:7 = w:4 [outer=(4,7), constraints=(/4: (/NULL - ]; /7: (/NULL - ]), fd=(4)==(7), (7)==(4)] # Apply when outer columns of both sides of OR are a subset of index columns. opt expect=SplitDisjunction @@ -2441,7 +2441,7 @@ distinct-on ├── union-all │ ├── columns: k:1!null u:2 v:3 │ ├── left columns: k:1!null u:2 v:3 - │ ├── right columns: k:5 u:6 v:7 + │ ├── right columns: k:6 u:7 v:8 │ ├── index-join e │ │ ├── columns: k:1!null u:2!null v:3 │ │ ├── key: (1) @@ -2452,14 +2452,14 @@ distinct-on │ │ ├── key: (1) │ │ └── fd: ()-->(2) │ └── index-join e - │ ├── columns: k:5!null u:6 v:7!null - │ ├── key: (5) - │ ├── fd: ()-->(7), (5)-->(6) + │ ├── columns: k:6!null u:7 v:8!null + │ ├── key: (6) + │ ├── fd: ()-->(8), (6)-->(7) │ └── scan e@vw - │ ├── columns: k:5!null v:7!null - │ ├── constraint: /7/8/5: [/1 - /1] - │ ├── key: (5) - │ └── fd: ()-->(7) + │ ├── columns: k:6!null v:8!null + │ ├── constraint: /8/9/6: [/1 - /1] + │ ├── key: (6) + │ └── fd: ()-->(8) └── aggregations ├── const-agg [as=u:2, outer=(2)] │ └── u:2 @@ -2482,7 +2482,7 @@ project ├── union-all │ ├── columns: k:1!null u:2 v:3 w:4!null │ ├── left columns: k:1!null u:2 v:3 w:4!null - │ ├── right columns: k:5 u:6 v:7 w:8 + │ ├── right columns: k:6 u:7 v:8 w:9 │ ├── select │ │ ├── columns: k:1!null u:2!null v:3 w:4!null │ │ ├── key: (1) @@ -2499,20 +2499,20 @@ project │ │ └── filters │ │ └── w:4 = 2 [outer=(4), constraints=(/4: [/2 - /2]; tight), fd=()-->(4)] │ └── select - │ ├── columns: k:5!null u:6 v:7!null w:8!null - │ ├── key: (5) - │ ├── fd: ()-->(7,8), (5)-->(6) + │ ├── columns: k:6!null u:7 v:8!null w:9!null + │ ├── key: (6) + │ ├── fd: ()-->(8,9), (6)-->(7) │ ├── index-join d - │ │ ├── columns: k:5!null u:6 v:7 w:8 - │ │ ├── key: (5) - │ │ ├── fd: ()-->(7), (5)-->(6,8) + │ │ ├── columns: k:6!null u:7 v:8 w:9 + │ │ ├── key: (6) + │ │ ├── fd: ()-->(8), (6)-->(7,9) │ │ └── scan d@v - │ │ ├── columns: k:5!null v:7!null - │ │ ├── constraint: /7/5: [/1 - /1] - │ │ ├── key: (5) - │ │ └── fd: ()-->(7) + │ │ ├── columns: k:6!null v:8!null + │ │ ├── constraint: /8/6: [/1 - /1] + │ │ ├── key: (6) + │ │ └── fd: ()-->(8) │ └── filters - │ └── w:8 = 3 [outer=(8), constraints=(/8: [/3 - /3]; tight), fd=()-->(8)] + │ └── w:9 = 3 [outer=(9), constraints=(/9: [/3 - /3]; tight), fd=()-->(9)] └── aggregations ├── const-agg [as=u:2, outer=(2)] │ └── u:2 @@ -2533,7 +2533,7 @@ distinct-on ├── union-all │ ├── columns: k:1!null u:2 v:3 │ ├── left columns: k:1!null u:2 v:3 - │ ├── right columns: k:5 u:6 v:7 + │ ├── right columns: k:6 u:7 v:8 │ ├── index-join d │ │ ├── columns: k:1!null u:2!null v:3 │ │ ├── key: (1) @@ -2546,16 +2546,16 @@ distinct-on │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ └── index-join d - │ ├── columns: k:5!null u:6 v:7!null - │ ├── key: (5) - │ ├── fd: (5)-->(6,7) + │ ├── columns: k:6!null u:7 v:8!null + │ ├── key: (6) + │ ├── fd: (6)-->(7,8) │ └── scan d@v - │ ├── columns: k:5!null v:7!null - │ ├── constraint: /7/5 + │ ├── columns: k:6!null v:8!null + │ ├── constraint: /8/6 │ │ ├── [/2 - /2] │ │ └── [/4 - /4] - │ ├── key: (5) - │ └── fd: (5)-->(7) + │ ├── key: (6) + │ └── fd: (6)-->(8) └── aggregations ├── const-agg [as=u:2, outer=(2)] │ └── u:2 @@ -2574,7 +2574,7 @@ distinct-on ├── union-all │ ├── columns: k:1!null u:2 v:3 │ ├── left columns: k:1!null u:2 v:3 - │ ├── right columns: k:5 u:6 v:7 + │ ├── right columns: k:6 u:7 v:8 │ ├── index-join d │ │ ├── columns: k:1!null u:2!null v:3 │ │ ├── key: (1) @@ -2588,17 +2588,17 @@ distinct-on │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ └── index-join d - │ ├── columns: k:5!null u:6 v:7!null - │ ├── key: (5) - │ ├── fd: (5)-->(6,7) + │ ├── columns: k:6!null u:7 v:8!null + │ ├── key: (6) + │ ├── fd: (6)-->(7,8) │ └── scan d@v - │ ├── columns: k:5!null v:7!null - │ ├── constraint: /7/5 + │ ├── columns: k:6!null v:8!null + │ ├── constraint: /8/6 │ │ ├── [/2 - /2] │ │ ├── [/4 - /4] │ │ └── [/6 - /6] - │ ├── key: (5) - │ └── fd: (5)-->(7) + │ ├── key: (6) + │ └── fd: (6)-->(8) └── aggregations ├── const-agg [as=u:2, outer=(2)] │ └── u:2 @@ -2617,7 +2617,7 @@ distinct-on ├── union-all │ ├── columns: k:1!null u:2 v:3 │ ├── left columns: k:1!null u:2 v:3 - │ ├── right columns: k:5 u:6 v:7 + │ ├── right columns: k:6 u:7 v:8 │ ├── index-join d │ │ ├── columns: k:1!null u:2!null v:3 │ │ ├── key: (1) @@ -2630,17 +2630,17 @@ distinct-on │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ └── index-join d - │ ├── columns: k:5!null u:6 v:7!null - │ ├── key: (5) - │ ├── fd: (5)-->(6,7) + │ ├── columns: k:6!null u:7 v:8!null + │ ├── key: (6) + │ ├── fd: (6)-->(7,8) │ └── scan d@v - │ ├── columns: k:5!null v:7!null - │ ├── constraint: /7/5 + │ ├── columns: k:6!null v:8!null + │ ├── constraint: /8/6 │ │ ├── [/2 - /2] │ │ ├── [/4 - /4] │ │ └── [/6 - /6] - │ ├── key: (5) - │ └── fd: (5)-->(7) + │ ├── key: (6) + │ └── fd: (6)-->(8) └── aggregations ├── const-agg [as=u:2, outer=(2)] │ └── u:2 @@ -2663,7 +2663,7 @@ project ├── union-all │ ├── columns: k:1!null u:2 v:3 w:4!null │ ├── left columns: k:1!null u:2 v:3 w:4!null - │ ├── right columns: k:5 u:6 v:7 w:8 + │ ├── right columns: k:6 u:7 v:8 w:9 │ ├── select │ │ ├── columns: k:1!null u:2!null v:3 w:4!null │ │ ├── key: (1) @@ -2680,20 +2680,20 @@ project │ │ └── filters │ │ └── (w:4 = 1) OR (w:4 = 2) [outer=(4), constraints=(/4: [/1 - /1] [/2 - /2]; tight)] │ └── select - │ ├── columns: k:5!null u:6 v:7!null w:8!null - │ ├── key: (5) - │ ├── fd: ()-->(7), (5)-->(6,8) + │ ├── columns: k:6!null u:7 v:8!null w:9!null + │ ├── key: (6) + │ ├── fd: ()-->(8), (6)-->(7,9) │ ├── index-join d - │ │ ├── columns: k:5!null u:6 v:7 w:8 - │ │ ├── key: (5) - │ │ ├── fd: ()-->(7), (5)-->(6,8) + │ │ ├── columns: k:6!null u:7 v:8 w:9 + │ │ ├── key: (6) + │ │ ├── fd: ()-->(8), (6)-->(7,9) │ │ └── scan d@v - │ │ ├── columns: k:5!null v:7!null - │ │ ├── constraint: /7/5: [/4 - /4] - │ │ ├── key: (5) - │ │ └── fd: ()-->(7) + │ │ ├── columns: k:6!null v:8!null + │ │ ├── constraint: /8/6: [/4 - /4] + │ │ ├── key: (6) + │ │ └── fd: ()-->(8) │ └── filters - │ └── (w:8 = 1) OR (w:8 = 2) [outer=(8), constraints=(/8: [/1 - /1] [/2 - /2]; tight)] + │ └── (w:9 = 1) OR (w:9 = 2) [outer=(9), constraints=(/9: [/1 - /1] [/2 - /2]; tight)] └── aggregations ├── const-agg [as=u:2, outer=(2)] │ └── u:2 @@ -2718,7 +2718,7 @@ project ├── union-all │ ├── columns: k:1!null u:2 v:3 w:4 │ ├── left columns: k:1!null u:2 v:3 w:4 - │ ├── right columns: k:5 u:6 v:7 w:8 + │ ├── right columns: k:6 u:7 v:8 w:9 │ ├── select │ │ ├── columns: k:1!null u:2!null v:3 w:4!null │ │ ├── key: (1) @@ -2735,20 +2735,20 @@ project │ │ └── filters │ │ └── w:4 = 2 [outer=(4), constraints=(/4: [/2 - /2]; tight), fd=()-->(4)] │ └── select - │ ├── columns: k:5!null u:6 v:7!null w:8 - │ ├── key: (5) - │ ├── fd: ()-->(7), (5)-->(6,8) + │ ├── columns: k:6!null u:7 v:8!null w:9 + │ ├── key: (6) + │ ├── fd: ()-->(8), (6)-->(7,9) │ ├── index-join d - │ │ ├── columns: k:5!null u:6 v:7 w:8 - │ │ ├── key: (5) - │ │ ├── fd: ()-->(7), (5)-->(6,8) + │ │ ├── columns: k:6!null u:7 v:8 w:9 + │ │ ├── key: (6) + │ │ ├── fd: ()-->(8), (6)-->(7,9) │ │ └── scan d@v - │ │ ├── columns: k:5!null v:7!null - │ │ ├── constraint: /7/5: [/4 - /4] - │ │ ├── key: (5) - │ │ └── fd: ()-->(7) + │ │ ├── columns: k:6!null v:8!null + │ │ ├── constraint: /8/6: [/4 - /4] + │ │ ├── key: (6) + │ │ └── fd: ()-->(8) │ └── filters - │ └── (u:6 = 1) OR (w:8 = 2) [outer=(6,8)] + │ └── (u:7 = 1) OR (w:9 = 2) [outer=(7,9)] └── aggregations ├── const-agg [as=u:2, outer=(2)] │ └── u:2 @@ -2786,7 +2786,7 @@ project ├── union-all │ ├── columns: k:1!null u:2 v:3 │ ├── left columns: k:1!null u:2 v:3 - │ ├── right columns: k:5 u:6 v:7 + │ ├── right columns: k:6 u:7 v:8 │ ├── index-join d │ │ ├── columns: k:1!null u:2!null v:3 │ │ ├── key: (1) @@ -2797,14 +2797,14 @@ project │ │ ├── key: (1) │ │ └── fd: ()-->(2) │ └── index-join d - │ ├── columns: k:5!null u:6 v:7!null - │ ├── key: (5) - │ ├── fd: ()-->(7), (5)-->(6) + │ ├── columns: k:6!null u:7 v:8!null + │ ├── key: (6) + │ ├── fd: ()-->(8), (6)-->(7) │ └── scan d@v - │ ├── columns: k:5!null v:7!null - │ ├── constraint: /7/5: [/1 - /1] - │ ├── key: (5) - │ └── fd: ()-->(7) + │ ├── columns: k:6!null v:8!null + │ ├── constraint: /8/6: [/1 - /1] + │ ├── key: (6) + │ └── fd: ()-->(8) └── aggregations ├── const-agg [as=u:2, outer=(2)] │ └── u:2 @@ -2841,7 +2841,7 @@ project ├── union-all │ ├── columns: k:1!null u:2 v:3 │ ├── left columns: k:1!null u:2 v:3 - │ ├── right columns: k:4 u:5 v:6 + │ ├── right columns: k:5 u:6 v:7 │ ├── scan a@u │ │ ├── columns: k:1!null u:2!null v:3 │ │ ├── constraint: /2/1: [/1 - /1] @@ -2849,12 +2849,12 @@ project │ │ ├── key: (1) │ │ └── fd: ()-->(2), (1)-->(3), (3)~~>(1) │ └── scan a@v - │ ├── columns: k:4!null u:5 v:6!null - │ ├── constraint: /6: [/1 - /1] + │ ├── columns: k:5!null u:6 v:7!null + │ ├── constraint: /7: [/1 - /1] │ ├── flags: no-index-join │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(4-6) + │ └── fd: ()-->(5-7) └── aggregations ├── const-agg [as=u:2, outer=(2)] │ └── u:2 @@ -2908,7 +2908,7 @@ project ├── union-all │ ├── columns: k:1!null u:2 v:3 │ ├── left columns: k:1!null u:2 v:3 - │ ├── right columns: k:5 u:6 v:7 + │ ├── right columns: k:6 u:7 v:8 │ ├── index-join d │ │ ├── columns: k:1!null u:2!null v:3 │ │ ├── key: (1) @@ -2919,14 +2919,14 @@ project │ │ ├── key: (1) │ │ └── fd: ()-->(2) │ └── index-join d - │ ├── columns: k:5!null u:6 v:7!null - │ ├── key: (5) - │ ├── fd: ()-->(7), (5)-->(6) + │ ├── columns: k:6!null u:7 v:8!null + │ ├── key: (6) + │ ├── fd: ()-->(8), (6)-->(7) │ └── scan d@v - │ ├── columns: k:5!null v:7!null - │ ├── constraint: /7/5: [/1 - /1] - │ ├── key: (5) - │ └── fd: ()-->(7) + │ ├── columns: k:6!null v:8!null + │ ├── constraint: /8/6: [/1 - /1] + │ ├── key: (6) + │ └── fd: ()-->(8) └── aggregations ├── const-agg [as=u:2, outer=(2)] │ └── u:2 @@ -2947,7 +2947,7 @@ project ├── union-all │ ├── columns: k:1!null u:2 v:3 w:4!null │ ├── left columns: k:1!null u:2 v:3 w:4!null - │ ├── right columns: k:5 u:6 v:7 w:8 + │ ├── right columns: k:6 u:7 v:8 w:9 │ ├── select │ │ ├── columns: k:1!null u:2!null v:3 w:4!null │ │ ├── key: (1) @@ -2964,20 +2964,20 @@ project │ │ └── filters │ │ └── w:4 = 1 [outer=(4), constraints=(/4: [/1 - /1]; tight), fd=()-->(4)] │ └── select - │ ├── columns: k:5!null u:6 v:7!null w:8!null - │ ├── key: (5) - │ ├── fd: ()-->(7,8), (5)-->(6) + │ ├── columns: k:6!null u:7 v:8!null w:9!null + │ ├── key: (6) + │ ├── fd: ()-->(8,9), (6)-->(7) │ ├── index-join d - │ │ ├── columns: k:5!null u:6 v:7 w:8 - │ │ ├── key: (5) - │ │ ├── fd: ()-->(7), (5)-->(6,8) + │ │ ├── columns: k:6!null u:7 v:8 w:9 + │ │ ├── key: (6) + │ │ ├── fd: ()-->(8), (6)-->(7,9) │ │ └── scan d@v - │ │ ├── columns: k:5!null v:7!null - │ │ ├── constraint: /7/5: [/1 - /1] - │ │ ├── key: (5) - │ │ └── fd: ()-->(7) + │ │ ├── columns: k:6!null v:8!null + │ │ ├── constraint: /8/6: [/1 - /1] + │ │ ├── key: (6) + │ │ └── fd: ()-->(8) │ └── filters - │ └── w:8 = 1 [outer=(8), constraints=(/8: [/1 - /1]; tight), fd=()-->(8)] + │ └── w:9 = 1 [outer=(9), constraints=(/9: [/1 - /1]; tight), fd=()-->(9)] └── aggregations ├── const-agg [as=u:2, outer=(2)] │ └── u:2 @@ -2999,7 +2999,7 @@ project ├── union-all │ ├── columns: k:1!null u:2!null v:3!null │ ├── left columns: k:1!null u:2!null v:3!null - │ ├── right columns: k:5 u:6 v:7 + │ ├── right columns: k:6 u:7 v:8 │ ├── inner-join (zigzag d@u d@v) │ │ ├── columns: k:1!null u:2!null v:3!null │ │ ├── eq columns: [1] = [1] @@ -3011,15 +3011,15 @@ project │ │ ├── u:2 = 1 [outer=(2), constraints=(/2: [/1 - /1]; tight), fd=()-->(2)] │ │ └── v:3 = 20 [outer=(3), constraints=(/3: [/20 - /20]; tight), fd=()-->(3)] │ └── inner-join (zigzag d@u d@v) - │ ├── columns: k:5!null u:6!null v:7!null - │ ├── eq columns: [5] = [5] - │ ├── left fixed columns: [6] = [10] - │ ├── right fixed columns: [7] = [2] - │ ├── key: (5) - │ ├── fd: ()-->(6,7) + │ ├── columns: k:6!null u:7!null v:8!null + │ ├── eq columns: [6] = [6] + │ ├── left fixed columns: [7] = [10] + │ ├── right fixed columns: [8] = [2] + │ ├── key: (6) + │ ├── fd: ()-->(7,8) │ └── filters - │ ├── v:7 = 2 [outer=(7), constraints=(/7: [/2 - /2]; tight), fd=()-->(7)] - │ └── u:6 = 10 [outer=(6), constraints=(/6: [/10 - /10]; tight), fd=()-->(6)] + │ ├── v:8 = 2 [outer=(8), constraints=(/8: [/2 - /2]; tight), fd=()-->(8)] + │ └── u:7 = 10 [outer=(7), constraints=(/7: [/10 - /10]; tight), fd=()-->(7)] └── aggregations ├── const-agg [as=u:2, outer=(2)] │ └── u:2 @@ -3030,10 +3030,10 @@ opt expect=SplitDisjunctionAddKey SELECT count(*) FROM d WHERE u = 1 OR v = 1 ---- scalar-group-by - ├── columns: count:5!null + ├── columns: count:6!null ├── cardinality: [1 - 1] ├── key: () - ├── fd: ()-->(5) + ├── fd: ()-->(6) ├── project │ ├── columns: u:2 v:3 │ └── distinct-on @@ -3044,7 +3044,7 @@ scalar-group-by │ ├── union-all │ │ ├── columns: k:1!null u:2 v:3 │ │ ├── left columns: k:1!null u:2 v:3 - │ │ ├── right columns: k:6 u:7 v:8 + │ │ ├── right columns: k:7 u:8 v:9 │ │ ├── index-join d │ │ │ ├── columns: k:1!null u:2!null v:3 │ │ │ ├── key: (1) @@ -3055,21 +3055,21 @@ scalar-group-by │ │ │ ├── key: (1) │ │ │ └── fd: ()-->(2) │ │ └── index-join d - │ │ ├── columns: k:6!null u:7 v:8!null - │ │ ├── key: (6) - │ │ ├── fd: ()-->(8), (6)-->(7) + │ │ ├── columns: k:7!null u:8 v:9!null + │ │ ├── key: (7) + │ │ ├── fd: ()-->(9), (7)-->(8) │ │ └── scan d@v - │ │ ├── columns: k:6!null v:8!null - │ │ ├── constraint: /8/6: [/1 - /1] - │ │ ├── key: (6) - │ │ └── fd: ()-->(8) + │ │ ├── columns: k:7!null v:9!null + │ │ ├── constraint: /9/7: [/1 - /1] + │ │ ├── key: (7) + │ │ └── fd: ()-->(9) │ └── aggregations │ ├── const-agg [as=u:2, outer=(2)] │ │ └── u:2 │ └── const-agg [as=v:3, outer=(3)] │ └── v:3 └── aggregations - └── count-rows [as=count_rows:5] + └── count-rows [as=count_rows:6] # Multi-column primary key. @@ -3086,7 +3086,7 @@ project ├── union-all │ ├── columns: k:1!null j:2!null u:3 v:4 │ ├── left columns: k:1!null j:2!null u:3 v:4 - │ ├── right columns: k:5 j:6 u:7 v:8 + │ ├── right columns: k:6 j:7 u:8 v:9 │ ├── index-join f │ │ ├── columns: k:1!null j:2!null u:3!null v:4 │ │ ├── key: (1,2) @@ -3097,14 +3097,14 @@ project │ │ ├── key: (1,2) │ │ └── fd: ()-->(3) │ └── index-join f - │ ├── columns: k:5!null j:6!null u:7 v:8!null - │ ├── key: (5,6) - │ ├── fd: ()-->(8), (5,6)-->(7) + │ ├── columns: k:6!null j:7!null u:8 v:9!null + │ ├── key: (6,7) + │ ├── fd: ()-->(9), (6,7)-->(8) │ └── scan f@v - │ ├── columns: k:5!null j:6!null v:8!null - │ ├── constraint: /8/5/6: [/2 - /2] - │ ├── key: (5,6) - │ └── fd: ()-->(8) + │ ├── columns: k:6!null j:7!null v:9!null + │ ├── constraint: /9/6/7: [/2 - /2] + │ ├── key: (6,7) + │ └── fd: ()-->(9) └── aggregations ├── const-agg [as=u:3, outer=(3)] │ └── u:3 @@ -3125,7 +3125,7 @@ project ├── union-all │ ├── columns: k:1!null u:2 v:3 │ ├── left columns: k:1!null u:2 v:3 - │ ├── right columns: k:5 u:6 v:7 + │ ├── right columns: k:6 u:7 v:8 │ ├── index-join d │ │ ├── columns: k:1!null u:2!null v:3 │ │ ├── key: (1) @@ -3136,14 +3136,14 @@ project │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ └── index-join d - │ ├── columns: k:5!null u:6 v:7!null - │ ├── key: (5) - │ ├── fd: (5)-->(6,7) + │ ├── columns: k:6!null u:7 v:8!null + │ ├── key: (6) + │ ├── fd: (6)-->(7,8) │ └── scan d@v - │ ├── columns: k:5!null v:7!null - │ ├── constraint: /7/5: [/5 - /8] - │ ├── key: (5) - │ └── fd: (5)-->(7) + │ ├── columns: k:6!null v:8!null + │ ├── constraint: /8/6: [/5 - /8] + │ ├── key: (6) + │ └── fd: (6)-->(8) └── aggregations ├── const-agg [as=u:2, outer=(2)] │ └── u:2 @@ -3166,7 +3166,7 @@ project ├── union-all │ ├── columns: k:1!null u:2 j:4 │ ├── left columns: k:1!null u:2 j:4 - │ ├── right columns: k:5 u:6 j:8 + │ ├── right columns: k:6 u:7 j:9 │ ├── immutable │ ├── index-join b │ │ ├── columns: k:1!null u:2!null j:4 @@ -3178,14 +3178,14 @@ project │ │ ├── key: (1) │ │ └── fd: ()-->(2) │ └── index-join b - │ ├── columns: k:5!null u:6 j:8 + │ ├── columns: k:6!null u:7 j:9 │ ├── immutable - │ ├── key: (5) - │ ├── fd: (5)-->(6,8) + │ ├── key: (6) + │ ├── fd: (6)-->(7,9) │ └── scan b@inv_idx - │ ├── columns: k:5!null - │ ├── constraint: /8/5: [/'{"foo": "bar"}' - /'{"foo": "bar"}'] - │ └── key: (5) + │ ├── columns: k:6!null + │ ├── constraint: /9/6: [/'{"foo": "bar"}' - /'{"foo": "bar"}'] + │ └── key: (6) └── aggregations ├── const-agg [as=u:2, outer=(2)] │ └── u:2 @@ -3208,7 +3208,7 @@ project ├── union-all │ ├── columns: k:1!null a:2 u:3 │ ├── left columns: k:1!null a:2 u:3 - │ ├── right columns: k:4 a:5 u:6 + │ ├── right columns: k:5 a:6 u:7 │ ├── immutable │ ├── index-join c │ │ ├── columns: k:1!null a:2 u:3!null @@ -3220,14 +3220,14 @@ project │ │ ├── key: (1) │ │ └── fd: ()-->(3) │ └── index-join c - │ ├── columns: k:4!null a:5 u:6 + │ ├── columns: k:5!null a:6 u:7 │ ├── immutable - │ ├── key: (4) - │ ├── fd: (4)-->(5,6) + │ ├── key: (5) + │ ├── fd: (5)-->(6,7) │ └── scan c@inv_idx - │ ├── columns: k:4!null - │ ├── constraint: /5/4: [/ARRAY[2] - /ARRAY[2]] - │ └── key: (4) + │ ├── columns: k:5!null + │ ├── constraint: /6/5: [/ARRAY[2] - /ARRAY[2]] + │ └── key: (5) └── aggregations ├── const-agg [as=a:2, outer=(2)] │ └── a:2 @@ -3248,7 +3248,7 @@ project ├── union-all │ ├── columns: d.k:1!null d.u:2 d.v:3 │ ├── left columns: d.k:1!null d.u:2 d.v:3 - │ ├── right columns: d.k:8 d.u:9 d.v:10 + │ ├── right columns: d.k:10 d.u:11 d.v:12 │ ├── index-join d │ │ ├── columns: d.k:1!null d.u:2!null d.v:3 │ │ ├── key: (1) @@ -3265,30 +3265,30 @@ project │ │ └── filters │ │ └── exists [subquery] │ │ └── scan a - │ │ ├── columns: a.u:6 a.v:7 + │ │ ├── columns: a.u:7 a.v:8 │ │ ├── limit: 1 │ │ ├── key: () - │ │ └── fd: ()-->(6,7) + │ │ └── fd: ()-->(7,8) │ └── index-join d - │ ├── columns: d.k:8!null d.u:9 d.v:10!null - │ ├── key: (8) - │ ├── fd: ()-->(10), (8)-->(9) + │ ├── columns: d.k:10!null d.u:11 d.v:12!null + │ ├── key: (10) + │ ├── fd: ()-->(12), (10)-->(11) │ └── select - │ ├── columns: d.k:8!null d.v:10!null - │ ├── key: (8) - │ ├── fd: ()-->(10) + │ ├── columns: d.k:10!null d.v:12!null + │ ├── key: (10) + │ ├── fd: ()-->(12) │ ├── scan d@v - │ │ ├── columns: d.k:8!null d.v:10!null - │ │ ├── constraint: /10/8: [/1 - /1] - │ │ ├── key: (8) - │ │ └── fd: ()-->(10) + │ │ ├── columns: d.k:10!null d.v:12!null + │ │ ├── constraint: /12/10: [/1 - /1] + │ │ ├── key: (10) + │ │ └── fd: ()-->(12) │ └── filters │ └── exists [subquery] │ └── scan a - │ ├── columns: a.u:6 a.v:7 + │ ├── columns: a.u:7 a.v:8 │ ├── limit: 1 │ ├── key: () - │ └── fd: ()-->(6,7) + │ └── fd: ()-->(7,8) └── aggregations ├── const-agg [as=d.u:2, outer=(2)] │ └── d.u:2 @@ -3302,9 +3302,9 @@ SELECT u, v FROM d WHERE (u = 1 OR v = 1) AND EXISTS (SELECT * FROM a WHERE a.u project ├── columns: u:2 v:3 └── inner-join (hash) - ├── columns: d.u:2!null d.v:3 a.u:6!null + ├── columns: d.u:2!null d.v:3 a.u:7!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - ├── fd: (2)==(6), (6)==(2) + ├── fd: (2)==(7), (7)==(2) ├── project │ ├── columns: d.u:2 d.v:3 │ └── distinct-on @@ -3315,7 +3315,7 @@ project │ ├── union-all │ │ ├── columns: d.k:1!null d.u:2 d.v:3 │ │ ├── left columns: d.k:1!null d.u:2 d.v:3 - │ │ ├── right columns: d.k:8 d.u:9 d.v:10 + │ │ ├── right columns: d.k:10 d.u:11 d.v:12 │ │ ├── index-join d │ │ │ ├── columns: d.k:1!null d.u:2!null d.v:3 │ │ │ ├── key: (1) @@ -3326,29 +3326,29 @@ project │ │ │ ├── key: (1) │ │ │ └── fd: ()-->(2) │ │ └── index-join d - │ │ ├── columns: d.k:8!null d.u:9 d.v:10!null - │ │ ├── key: (8) - │ │ ├── fd: ()-->(10), (8)-->(9) + │ │ ├── columns: d.k:10!null d.u:11 d.v:12!null + │ │ ├── key: (10) + │ │ ├── fd: ()-->(12), (10)-->(11) │ │ └── scan d@v - │ │ ├── columns: d.k:8!null d.v:10!null - │ │ ├── constraint: /10/8: [/1 - /1] - │ │ ├── key: (8) - │ │ └── fd: ()-->(10) + │ │ ├── columns: d.k:10!null d.v:12!null + │ │ ├── constraint: /12/10: [/1 - /1] + │ │ ├── key: (10) + │ │ └── fd: ()-->(12) │ └── aggregations │ ├── const-agg [as=d.u:2, outer=(2)] │ │ └── d.u:2 │ └── const-agg [as=d.v:3, outer=(3)] │ └── d.v:3 ├── distinct-on - │ ├── columns: a.u:6 - │ ├── grouping columns: a.u:6 - │ ├── internal-ordering: +6 - │ ├── key: (6) + │ ├── columns: a.u:7 + │ ├── grouping columns: a.u:7 + │ ├── internal-ordering: +7 + │ ├── key: (7) │ └── scan a@u - │ ├── columns: a.u:6 - │ └── ordering: +6 + │ ├── columns: a.u:7 + │ └── ordering: +7 └── filters - └── a.u:6 = d.u:2 [outer=(2,6), constraints=(/2: (/NULL - ]; /6: (/NULL - ]), fd=(2)==(6), (6)==(2)] + └── a.u:7 = d.u:2 [outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] # Correlated subquery with references to outer columns not in the scan columns. opt expect=SplitDisjunctionAddKey @@ -3359,9 +3359,9 @@ project └── project ├── columns: d.u:2 d.v:3 w:4 └── inner-join (hash) - ├── columns: d.u:2 d.v:3 w:4!null a.u:6!null + ├── columns: d.u:2 d.v:3 w:4!null a.u:7!null ├── multiplicity: left-rows(zero-or-one), right-rows(zero-or-more) - ├── fd: (4)==(6), (6)==(4) + ├── fd: (4)==(7), (7)==(4) ├── project │ ├── columns: d.u:2 d.v:3 w:4 │ └── distinct-on @@ -3372,7 +3372,7 @@ project │ ├── union-all │ │ ├── columns: d.k:1!null d.u:2 d.v:3 w:4 │ │ ├── left columns: d.k:1!null d.u:2 d.v:3 w:4 - │ │ ├── right columns: d.k:8 d.u:9 d.v:10 w:11 + │ │ ├── right columns: d.k:10 d.u:11 d.v:12 w:13 │ │ ├── index-join d │ │ │ ├── columns: d.k:1!null d.u:2!null d.v:3 w:4 │ │ │ ├── key: (1) @@ -3383,14 +3383,14 @@ project │ │ │ ├── key: (1) │ │ │ └── fd: ()-->(2) │ │ └── index-join d - │ │ ├── columns: d.k:8!null d.u:9 d.v:10!null w:11 - │ │ ├── key: (8) - │ │ ├── fd: ()-->(10), (8)-->(9,11) + │ │ ├── columns: d.k:10!null d.u:11 d.v:12!null w:13 + │ │ ├── key: (10) + │ │ ├── fd: ()-->(12), (10)-->(11,13) │ │ └── scan d@v - │ │ ├── columns: d.k:8!null d.v:10!null - │ │ ├── constraint: /10/8: [/1 - /1] - │ │ ├── key: (8) - │ │ └── fd: ()-->(10) + │ │ ├── columns: d.k:10!null d.v:12!null + │ │ ├── constraint: /12/10: [/1 - /1] + │ │ ├── key: (10) + │ │ └── fd: ()-->(12) │ └── aggregations │ ├── const-agg [as=d.u:2, outer=(2)] │ │ └── d.u:2 @@ -3399,15 +3399,15 @@ project │ └── const-agg [as=w:4, outer=(4)] │ └── w:4 ├── distinct-on - │ ├── columns: a.u:6 - │ ├── grouping columns: a.u:6 - │ ├── internal-ordering: +6 - │ ├── key: (6) + │ ├── columns: a.u:7 + │ ├── grouping columns: a.u:7 + │ ├── internal-ordering: +7 + │ ├── key: (7) │ └── scan a@u - │ ├── columns: a.u:6 - │ └── ordering: +6 + │ ├── columns: a.u:7 + │ └── ordering: +7 └── filters - └── a.u:6 = w:4 [outer=(4,6), constraints=(/4: (/NULL - ]; /6: (/NULL - ]), fd=(4)==(6), (6)==(4)] + └── a.u:7 = w:4 [outer=(4,7), constraints=(/4: (/NULL - ]; /7: (/NULL - ]), fd=(4)==(7), (7)==(4)] # Use rowid when there is no explicit primary key. opt expect=SplitDisjunctionAddKey @@ -3423,7 +3423,7 @@ project ├── union-all │ ├── columns: u:2 v:3 rowid:4!null │ ├── left columns: u:2 v:3 rowid:4!null - │ ├── right columns: u:6 v:7 rowid:8 + │ ├── right columns: u:7 v:8 rowid:9 │ ├── index-join no_explicit_primary_key │ │ ├── columns: u:2!null v:3 rowid:4!null │ │ ├── key: (4) @@ -3434,14 +3434,14 @@ project │ │ ├── key: (4) │ │ └── fd: ()-->(2) │ └── index-join no_explicit_primary_key - │ ├── columns: u:6 v:7!null rowid:8!null - │ ├── key: (8) - │ ├── fd: ()-->(7), (8)-->(6) + │ ├── columns: u:7 v:8!null rowid:9!null + │ ├── key: (9) + │ ├── fd: ()-->(8), (9)-->(7) │ └── scan no_explicit_primary_key@v - │ ├── columns: v:7!null rowid:8!null - │ ├── constraint: /7/8: [/5 - /5] - │ ├── key: (8) - │ └── fd: ()-->(7) + │ ├── columns: v:8!null rowid:9!null + │ ├── constraint: /8/9: [/5 - /5] + │ ├── key: (9) + │ └── fd: ()-->(8) └── aggregations ├── const-agg [as=u:2, outer=(2)] │ └── u:2 @@ -3462,7 +3462,7 @@ project ├── union-all │ ├── columns: k:1!null u:2 v:3 │ ├── left columns: k:1!null u:2 v:3 - │ ├── right columns: k:5 u:6 v:7 + │ ├── right columns: k:6 u:7 v:8 │ ├── index-join e │ │ ├── columns: k:1!null u:2!null v:3 │ │ ├── key: (1) @@ -3473,14 +3473,14 @@ project │ │ ├── key: (1) │ │ └── fd: ()-->(2) │ └── index-join e - │ ├── columns: k:5!null u:6 v:7!null - │ ├── key: (5) - │ ├── fd: ()-->(7), (5)-->(6) + │ ├── columns: k:6!null u:7 v:8!null + │ ├── key: (6) + │ ├── fd: ()-->(8), (6)-->(7) │ └── scan e@vw - │ ├── columns: k:5!null v:7!null - │ ├── constraint: /7/8/5: [/1 - /1] - │ ├── key: (5) - │ └── fd: ()-->(7) + │ ├── columns: k:6!null v:8!null + │ ├── constraint: /8/9/6: [/1 - /1] + │ ├── key: (6) + │ └── fd: ()-->(8) └── aggregations ├── const-agg [as=u:2, outer=(2)] │ └── u:2 @@ -3503,7 +3503,7 @@ project ├── union-all │ ├── columns: k:1!null u:2 v:3 w:4!null │ ├── left columns: k:1!null u:2 v:3 w:4!null - │ ├── right columns: k:5 u:6 v:7 w:8 + │ ├── right columns: k:6 u:7 v:8 w:9 │ ├── select │ │ ├── columns: k:1!null u:2!null v:3 w:4!null │ │ ├── key: (1) @@ -3520,20 +3520,20 @@ project │ │ └── filters │ │ └── w:4 = 2 [outer=(4), constraints=(/4: [/2 - /2]; tight), fd=()-->(4)] │ └── select - │ ├── columns: k:5!null u:6 v:7!null w:8!null - │ ├── key: (5) - │ ├── fd: ()-->(7,8), (5)-->(6) + │ ├── columns: k:6!null u:7 v:8!null w:9!null + │ ├── key: (6) + │ ├── fd: ()-->(8,9), (6)-->(7) │ ├── index-join d - │ │ ├── columns: k:5!null u:6 v:7 w:8 - │ │ ├── key: (5) - │ │ ├── fd: ()-->(7), (5)-->(6,8) + │ │ ├── columns: k:6!null u:7 v:8 w:9 + │ │ ├── key: (6) + │ │ ├── fd: ()-->(8), (6)-->(7,9) │ │ └── scan d@v - │ │ ├── columns: k:5!null v:7!null - │ │ ├── constraint: /7/5: [/1 - /1] - │ │ ├── key: (5) - │ │ └── fd: ()-->(7) + │ │ ├── columns: k:6!null v:8!null + │ │ ├── constraint: /8/6: [/1 - /1] + │ │ ├── key: (6) + │ │ └── fd: ()-->(8) │ └── filters - │ └── w:8 = 3 [outer=(8), constraints=(/8: [/3 - /3]; tight), fd=()-->(8)] + │ └── w:9 = 3 [outer=(9), constraints=(/9: [/3 - /3]; tight), fd=()-->(9)] └── aggregations ├── const-agg [as=u:2, outer=(2)] │ └── u:2 @@ -3556,7 +3556,7 @@ project ├── union-all │ ├── columns: k:1!null u:2 v:3 │ ├── left columns: k:1!null u:2 v:3 - │ ├── right columns: k:5 u:6 v:7 + │ ├── right columns: k:6 u:7 v:8 │ ├── index-join d │ │ ├── columns: k:1!null u:2!null v:3 │ │ ├── key: (1) @@ -3569,16 +3569,16 @@ project │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ └── index-join d - │ ├── columns: k:5!null u:6 v:7!null - │ ├── key: (5) - │ ├── fd: (5)-->(6,7) + │ ├── columns: k:6!null u:7 v:8!null + │ ├── key: (6) + │ ├── fd: (6)-->(7,8) │ └── scan d@v - │ ├── columns: k:5!null v:7!null - │ ├── constraint: /7/5 + │ ├── columns: k:6!null v:8!null + │ ├── constraint: /8/6 │ │ ├── [/2 - /2] │ │ └── [/4 - /4] - │ ├── key: (5) - │ └── fd: (5)-->(7) + │ ├── key: (6) + │ └── fd: (6)-->(8) └── aggregations ├── const-agg [as=u:2, outer=(2)] │ └── u:2 @@ -3599,7 +3599,7 @@ project ├── union-all │ ├── columns: k:1!null u:2 v:3 │ ├── left columns: k:1!null u:2 v:3 - │ ├── right columns: k:5 u:6 v:7 + │ ├── right columns: k:6 u:7 v:8 │ ├── index-join d │ │ ├── columns: k:1!null u:2!null v:3 │ │ ├── key: (1) @@ -3613,17 +3613,17 @@ project │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ └── index-join d - │ ├── columns: k:5!null u:6 v:7!null - │ ├── key: (5) - │ ├── fd: (5)-->(6,7) + │ ├── columns: k:6!null u:7 v:8!null + │ ├── key: (6) + │ ├── fd: (6)-->(7,8) │ └── scan d@v - │ ├── columns: k:5!null v:7!null - │ ├── constraint: /7/5 + │ ├── columns: k:6!null v:8!null + │ ├── constraint: /8/6 │ │ ├── [/2 - /2] │ │ ├── [/4 - /4] │ │ └── [/6 - /6] - │ ├── key: (5) - │ └── fd: (5)-->(7) + │ ├── key: (6) + │ └── fd: (6)-->(8) └── aggregations ├── const-agg [as=u:2, outer=(2)] │ └── u:2 @@ -3644,7 +3644,7 @@ project ├── union-all │ ├── columns: k:1!null u:2 v:3 │ ├── left columns: k:1!null u:2 v:3 - │ ├── right columns: k:5 u:6 v:7 + │ ├── right columns: k:6 u:7 v:8 │ ├── index-join d │ │ ├── columns: k:1!null u:2!null v:3 │ │ ├── key: (1) @@ -3657,17 +3657,17 @@ project │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ └── index-join d - │ ├── columns: k:5!null u:6 v:7!null - │ ├── key: (5) - │ ├── fd: (5)-->(6,7) + │ ├── columns: k:6!null u:7 v:8!null + │ ├── key: (6) + │ ├── fd: (6)-->(7,8) │ └── scan d@v - │ ├── columns: k:5!null v:7!null - │ ├── constraint: /7/5 + │ ├── columns: k:6!null v:8!null + │ ├── constraint: /8/6 │ │ ├── [/2 - /2] │ │ ├── [/4 - /4] │ │ └── [/6 - /6] - │ ├── key: (5) - │ └── fd: (5)-->(7) + │ ├── key: (6) + │ └── fd: (6)-->(8) └── aggregations ├── const-agg [as=u:2, outer=(2)] │ └── u:2 @@ -3691,7 +3691,7 @@ project ├── union-all │ ├── columns: k:1!null u:2 v:3 w:4!null │ ├── left columns: k:1!null u:2 v:3 w:4!null - │ ├── right columns: k:5 u:6 v:7 w:8 + │ ├── right columns: k:6 u:7 v:8 w:9 │ ├── select │ │ ├── columns: k:1!null u:2!null v:3 w:4!null │ │ ├── key: (1) @@ -3708,20 +3708,20 @@ project │ │ └── filters │ │ └── (w:4 = 1) OR (w:4 = 2) [outer=(4), constraints=(/4: [/1 - /1] [/2 - /2]; tight)] │ └── select - │ ├── columns: k:5!null u:6 v:7!null w:8!null - │ ├── key: (5) - │ ├── fd: ()-->(7), (5)-->(6,8) + │ ├── columns: k:6!null u:7 v:8!null w:9!null + │ ├── key: (6) + │ ├── fd: ()-->(8), (6)-->(7,9) │ ├── index-join d - │ │ ├── columns: k:5!null u:6 v:7 w:8 - │ │ ├── key: (5) - │ │ ├── fd: ()-->(7), (5)-->(6,8) + │ │ ├── columns: k:6!null u:7 v:8 w:9 + │ │ ├── key: (6) + │ │ ├── fd: ()-->(8), (6)-->(7,9) │ │ └── scan d@v - │ │ ├── columns: k:5!null v:7!null - │ │ ├── constraint: /7/5: [/4 - /4] - │ │ ├── key: (5) - │ │ └── fd: ()-->(7) + │ │ ├── columns: k:6!null v:8!null + │ │ ├── constraint: /8/6: [/4 - /4] + │ │ ├── key: (6) + │ │ └── fd: ()-->(8) │ └── filters - │ └── (w:8 = 1) OR (w:8 = 2) [outer=(8), constraints=(/8: [/1 - /1] [/2 - /2]; tight)] + │ └── (w:9 = 1) OR (w:9 = 2) [outer=(9), constraints=(/9: [/1 - /1] [/2 - /2]; tight)] └── aggregations ├── const-agg [as=u:2, outer=(2)] │ └── u:2 @@ -3747,7 +3747,7 @@ project ├── union-all │ ├── columns: k:1!null u:2 v:3 w:4 │ ├── left columns: k:1!null u:2 v:3 w:4 - │ ├── right columns: k:5 u:6 v:7 w:8 + │ ├── right columns: k:6 u:7 v:8 w:9 │ ├── select │ │ ├── columns: k:1!null u:2!null v:3 w:4!null │ │ ├── key: (1) @@ -3764,20 +3764,20 @@ project │ │ └── filters │ │ └── w:4 = 2 [outer=(4), constraints=(/4: [/2 - /2]; tight), fd=()-->(4)] │ └── select - │ ├── columns: k:5!null u:6 v:7!null w:8 - │ ├── key: (5) - │ ├── fd: ()-->(7), (5)-->(6,8) + │ ├── columns: k:6!null u:7 v:8!null w:9 + │ ├── key: (6) + │ ├── fd: ()-->(8), (6)-->(7,9) │ ├── index-join d - │ │ ├── columns: k:5!null u:6 v:7 w:8 - │ │ ├── key: (5) - │ │ ├── fd: ()-->(7), (5)-->(6,8) + │ │ ├── columns: k:6!null u:7 v:8 w:9 + │ │ ├── key: (6) + │ │ ├── fd: ()-->(8), (6)-->(7,9) │ │ └── scan d@v - │ │ ├── columns: k:5!null v:7!null - │ │ ├── constraint: /7/5: [/4 - /4] - │ │ ├── key: (5) - │ │ └── fd: ()-->(7) + │ │ ├── columns: k:6!null v:8!null + │ │ ├── constraint: /8/6: [/4 - /4] + │ │ ├── key: (6) + │ │ └── fd: ()-->(8) │ └── filters - │ └── (u:6 = 1) OR (w:8 = 2) [outer=(6,8)] + │ └── (u:7 = 1) OR (w:9 = 2) [outer=(7,9)] └── aggregations ├── const-agg [as=u:2, outer=(2)] │ └── u:2 @@ -3809,7 +3809,7 @@ distinct-on ├── union-all │ ├── columns: k:1!null u:2 v:3 │ ├── left columns: k:1!null u:2 v:3 - │ ├── right columns: k:5 u:6 v:7 + │ ├── right columns: k:6 u:7 v:8 │ ├── index-join d │ │ ├── columns: k:1!null u:2!null v:3 │ │ ├── key: (1) @@ -3820,14 +3820,14 @@ distinct-on │ │ ├── key: (1) │ │ └── fd: ()-->(2) │ └── index-join d - │ ├── columns: k:5!null u:6 v:7!null - │ ├── key: (5) - │ ├── fd: ()-->(7), (5)-->(6) + │ ├── columns: k:6!null u:7 v:8!null + │ ├── key: (6) + │ ├── fd: ()-->(8), (6)-->(7) │ └── scan d@v - │ ├── columns: k:5!null v:7!null - │ ├── constraint: /7/5: [/1 - /1] - │ ├── key: (5) - │ └── fd: ()-->(7) + │ ├── columns: k:6!null v:8!null + │ ├── constraint: /8/6: [/1 - /1] + │ ├── key: (6) + │ └── fd: ()-->(8) └── aggregations ├── const-agg [as=u:2, outer=(2)] │ └── u:2 @@ -3860,7 +3860,7 @@ project ├── union-all │ ├── columns: k:1!null u:2 v:3 │ ├── left columns: k:1!null u:2 v:3 - │ ├── right columns: k:4 u:5 v:6 + │ ├── right columns: k:5 u:6 v:7 │ ├── scan a@u │ │ ├── columns: k:1!null u:2!null v:3 │ │ ├── constraint: /2/1: [/1 - /1] @@ -3868,12 +3868,12 @@ project │ │ ├── key: (1) │ │ └── fd: ()-->(2), (1)-->(3), (3)~~>(1) │ └── scan a@v - │ ├── columns: k:4!null u:5 v:6!null - │ ├── constraint: /6: [/1 - /1] + │ ├── columns: k:5!null u:6 v:7!null + │ ├── constraint: /7: [/1 - /1] │ ├── flags: no-index-join │ ├── cardinality: [0 - 1] │ ├── key: () - │ └── fd: ()-->(4-6) + │ └── fd: ()-->(5-7) └── aggregations ├── const-agg [as=u:2, outer=(2)] │ └── u:2 diff --git a/pkg/sql/opt/xform/testdata/rules/stats b/pkg/sql/opt/xform/testdata/rules/stats index 90d0e7830064..4a112c79c89a 100644 --- a/pkg/sql/opt/xform/testdata/rules/stats +++ b/pkg/sql/opt/xform/testdata/rules/stats @@ -9,7 +9,11 @@ CREATE TABLE xyz (x INT PRIMARY KEY, y INT, z INT, INDEX y(y), UNIQUE INDEX z(z) rulestats SELECT * FROM abc JOIN xyz ON a=x ---- -Normalization rules applied 0 times. +Normalization rules applied 3 times. +Top normalization rules: + EliminateProject applied 1 times. + PruneJoinLeftCols applied 1 times. + PruneJoinRightCols applied 1 times. Exploration rules applied 8 times, added 5 expressions. Top exploration rules: CommuteJoin applied 2 times, added 1 expressions. diff --git a/pkg/sql/opt_catalog.go b/pkg/sql/opt_catalog.go index 6eba6d59a774..0569aa3163a4 100644 --- a/pkg/sql/opt_catalog.go +++ b/pkg/sql/opt_catalog.go @@ -509,6 +509,12 @@ func (os *optSequence) SequenceMarker() {} type optTable struct { desc *sqlbase.ImmutableTableDescriptor + // systemColumnDescs is the set of implicit system columns for the table. + // It contains column definitions for system columns like the MVCC Timestamp + // column and others. System columns have ordinals larger than the ordinals + // of physical columns and columns in mutations. + systemColumnDescs []cat.Column + // indexes are the inlined wrappers for the table's primary and secondary // indexes. indexes []optIndex @@ -564,6 +570,15 @@ func newOptTable( zone: tblZone, } + // Set up the MVCC timestamp system column. However, we won't add it + // in case a column with the same name already exists in the table. + // Note that the column does not exist when err != nil. This check is done + // for migration purposes. We need to avoid adding the system column if the + // table has a column with this name for some reason. + if _, _, err := desc.FindColumnByName(sqlbase.MVCCTimestampColumnName); err != nil { + ot.systemColumnDescs = append(ot.systemColumnDescs, sqlbase.NewMVCCTimestampColumnDesc()) + } + // Create the table's column mapping from sqlbase.ColumnID to column ordinal. ot.colMap = make(map[sqlbase.ColumnID]int, ot.ColumnCount()) for i, n := 0, ot.ColumnCount(); i < n; i++ { @@ -786,11 +801,14 @@ func (ot *optTable) IsVirtualTable() bool { // ColumnCount is part of the cat.Table interface. func (ot *optTable) ColumnCount() int { - return len(ot.desc.DeletableColumns()) + return len(ot.desc.DeletableColumns()) + len(ot.systemColumnDescs) } // Column is part of the cat.Table interface. func (ot *optTable) Column(i int) cat.Column { + if i >= len(ot.desc.DeletableColumns()) { + return ot.systemColumnDescs[i-len(ot.desc.DeletableColumns())] + } return &ot.desc.DeletableColumns()[i] } @@ -801,8 +819,10 @@ func (ot *optTable) ColumnKind(i int) cat.ColumnKind { return cat.Ordinary case i < len(ot.desc.WritableColumns()): return cat.WriteOnly - default: + case i < len(ot.desc.DeletableColumns()): return cat.DeleteOnly + default: + return cat.System } } @@ -946,7 +966,7 @@ func (oi *optIndex) init( notNull := true for _, id := range desc.ColumnIDs { ord, _ := tab.lookupColumnOrdinal(id) - if tab.desc.DeletableColumns()[ord].Nullable { + if tab.Column(ord).IsNullable() { notNull = false break } diff --git a/pkg/sql/opt_exec_factory.go b/pkg/sql/opt_exec_factory.go index 7d00abf72053..79d10014179c 100644 --- a/pkg/sql/opt_exec_factory.go +++ b/pkg/sql/opt_exec_factory.go @@ -83,6 +83,9 @@ func (ef *execFactory) ConstructScan( scan := ef.planner.Scan() colCfg := makeScanColumnsConfig(table, params.NeededCols) + // Check if any system columns are requested, as they need special handling. + scan.systemColumns, scan.systemColumnOrdinals = collectSystemColumnsFromCfg(&colCfg, tabDesc.TableDesc()) + sb := span.MakeBuilder(ef.planner.ExecCfg().Codec, tabDesc.TableDesc(), indexDesc) // initTable checks that the current user has the correct privilege to access @@ -512,6 +515,9 @@ func (ef *execFactory) ConstructIndexJoin( tableScan := ef.planner.Scan() + // Check if any system columns are requested, as they need special handling. + tableScan.systemColumns, tableScan.systemColumnOrdinals = collectSystemColumnsFromCfg(&colCfg, tabDesc.TableDesc()) + if err := tableScan.initTable(context.TODO(), ef.planner, tabDesc, nil, colCfg); err != nil { return nil, err } diff --git a/pkg/sql/row/fetcher.go b/pkg/sql/row/fetcher.go index 0c9f990d7bdd..0517f25854cf 100644 --- a/pkg/sql/row/fetcher.go +++ b/pkg/sql/row/fetcher.go @@ -36,6 +36,10 @@ import ( // this to avoid using log.V in the hot path. const DebugRowFetch = false +// noTimestampColumn is a sentinel value to denote that the MVCC timestamp +// column is not part of the output. +const noTimestampColumn = -1 + type kvBatchFetcher interface { // nextBatch returns the next batch of rows. Returns false in the first // parameter if there are no more keys in the scan. May return either a slice @@ -99,12 +103,13 @@ type tableInfo struct { // The following fields contain MVCC metadata for each row and may be // returned to users of Fetcher immediately after NextRow returns. - // They're not important to ordinary consumers of Fetcher that only - // concern themselves with actual SQL row data. // // rowLastModified is the timestamp of the last time any family in the row // was modified in any way. rowLastModified hlc.Timestamp + // timestampOutputIdx controls at what row ordinal to write the timestamp. + timestampOutputIdx int + // rowIsDeleted is true when the row has been deleted. This is only // meaningful when kv deletion tombstones are returned by the kvBatchFetcher, // which the one used by `StartScan` (the common case) doesnt. Notably, @@ -199,6 +204,11 @@ type Fetcher struct { // when beginning a new scan. traceKV bool + // mvccDecodeStrategy controls whether or not MVCC timestamps should + // be decoded from KV's fetched. It is set if any of the requested tables + // are required to produce an MVCC timestamp system column. + mvccDecodeStrategy MVCCDecodingStrategy + // -- Fields updated during a scan -- kvFetcher *KVFetcher @@ -285,9 +295,10 @@ func (rf *Fetcher) Init( // These slice fields might get re-allocated below, so reslice them from // the old table here in case they've got enough capacity already. - indexColIdx: oldTable.indexColIdx[:0], - keyVals: oldTable.keyVals[:0], - extraVals: oldTable.extraVals[:0], + indexColIdx: oldTable.indexColIdx[:0], + keyVals: oldTable.keyVals[:0], + extraVals: oldTable.extraVals[:0], + timestampOutputIdx: noTimestampColumn, } var err error @@ -327,6 +338,12 @@ func (rf *Fetcher) Init( if tableArgs.ValNeededForCol.Contains(idx) { // The idx-th column is required. table.neededCols.Add(int(col)) + // If this column is the timestamp column, set up the output index. + sysColKind := sqlbase.GetSystemColumnKindFromColumnID(col) + if sysColKind == sqlbase.SystemColumnKind_MVCCTIMESTAMP { + table.timestampOutputIdx = idx + rf.mvccDecodeStrategy = MVCCDecodingRequired + } } } @@ -589,7 +606,7 @@ func (rf *Fetcher) NextKey(ctx context.Context) (rowDone bool, err error) { var ok bool for { - ok, rf.kv, _, err = rf.kvFetcher.NextKV(ctx) + ok, rf.kv, _, err = rf.kvFetcher.NextKV(ctx, rf.mvccDecodeStrategy) if err != nil { return false, err } @@ -1413,6 +1430,16 @@ func (rf *Fetcher) checkKeyOrdering(ctx context.Context) error { func (rf *Fetcher) finalizeRow() error { table := rf.rowReadyTable + + // If the MVCC timestamp system column was requested, write it to the row. + if table.timestampOutputIdx != noTimestampColumn { + // TODO (rohany): Datums are immutable, so we can't store a DDecimal on the + // fetcher and change its contents with each row. If that assumption gets + // lifted, then we can avoid an allocation of a new decimal datum here. + dec := rf.alloc.NewDDecimal(tree.DDecimal{Decimal: tree.TimestampToDecimal(rf.RowLastModified())}) + table.row[table.timestampOutputIdx] = sqlbase.EncDatum{Datum: dec} + } + // Fill in any missing values with NULLs for i := range table.cols { if rf.valueColsFound == table.neededValueCols { diff --git a/pkg/sql/row/fetcher_mvcc_test.go b/pkg/sql/row/fetcher_mvcc_test.go index 4d067ebd9cf5..194da7e34ca4 100644 --- a/pkg/sql/row/fetcher_mvcc_test.go +++ b/pkg/sql/row/fetcher_mvcc_test.go @@ -140,7 +140,7 @@ func TestRowFetcherMVCCMetadata(t *testing.T) { } row := rowWithMVCCMetadata{ RowIsDeleted: rf.RowIsDeleted(), - RowLastModified: tree.TimestampToDecimal(rf.RowLastModified()).String(), + RowLastModified: tree.TimestampToDecimalDatum(rf.RowLastModified()).String(), } for _, datum := range datums { if datum == tree.DNull { diff --git a/pkg/sql/row/kv_fetcher.go b/pkg/sql/row/kv_fetcher.go index 8ddab9919b21..bd43eb1c1f6b 100644 --- a/pkg/sql/row/kv_fetcher.go +++ b/pkg/sql/row/kv_fetcher.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" ) // KVFetcher wraps kvBatchFetcher, providing a NextKV interface that returns the @@ -58,11 +59,22 @@ func (f *KVFetcher) GetBytesRead() int64 { return f.bytesRead } +// MVCCDecodingStrategy controls if and how the fetcher should decode MVCC +// timestamps from returned KV's. +type MVCCDecodingStrategy int + +const ( + // MVCCDecodingNotRequired is used when timestamps aren't needed. + MVCCDecodingNotRequired MVCCDecodingStrategy = iota + // MVCCDecodingRequired is used when timestamps are needed. + MVCCDecodingRequired +) + // NextKV returns the next kv from this fetcher. Returns false if there are no // more kvs to fetch, the kv that was fetched, and any errors that may have // occurred. func (f *KVFetcher) NextKV( - ctx context.Context, + ctx context.Context, mvccDecodeStrategy MVCCDecodingStrategy, ) (ok bool, kv roachpb.KeyValue, newSpan bool, err error) { for { newSpan = f.newSpan @@ -76,14 +88,21 @@ func (f *KVFetcher) NextKV( var key []byte var rawBytes []byte var err error - key, rawBytes, f.batchResponse, err = enginepb.ScanDecodeKeyValueNoTS(f.batchResponse) + var ts hlc.Timestamp + switch mvccDecodeStrategy { + case MVCCDecodingRequired: + key, ts, rawBytes, f.batchResponse, err = enginepb.ScanDecodeKeyValue(f.batchResponse) + case MVCCDecodingNotRequired: + key, rawBytes, f.batchResponse, err = enginepb.ScanDecodeKeyValueNoTS(f.batchResponse) + } if err != nil { return false, kv, false, err } return true, roachpb.KeyValue{ Key: key, Value: roachpb.Value{ - RawBytes: rawBytes, + RawBytes: rawBytes, + Timestamp: ts, }, }, newSpan, nil } diff --git a/pkg/sql/rowexec/indexjoiner.go b/pkg/sql/rowexec/indexjoiner.go index ec2f6c8954a6..31cfcd84f4cf 100644 --- a/pkg/sql/rowexec/indexjoiner.go +++ b/pkg/sql/rowexec/indexjoiner.go @@ -79,10 +79,23 @@ func newIndexJoiner( batchSize: indexJoinerBatchSize, } needMutations := spec.Visibility == execinfra.ScanVisibilityPublicAndNotPublic + + colIdxMap := ij.desc.ColumnIdxMapWithMutations(needMutations) + resultTypes := ij.desc.ColumnTypesWithMutations(needMutations) + // Add all requested system columns to the output. + sysColTypes, sysColDescs, err := sqlbase.GetSystemColumnTypesAndDescriptors(&ij.desc, spec.SystemColumns) + if err != nil { + return nil, err + } + resultTypes = append(resultTypes, sysColTypes...) + for i := range sysColDescs { + colIdxMap[sysColDescs[i].ID] = len(colIdxMap) + } + if err := ij.Init( ij, post, - ij.desc.ColumnTypesWithMutations(needMutations), + resultTypes, flowCtx, processorID, output, @@ -102,14 +115,15 @@ func newIndexJoiner( flowCtx, &fetcher, &ij.desc, - 0, /* primary index */ - ij.desc.ColumnIdxMapWithMutations(needMutations), + 0, /* indexIdx */ + colIdxMap, false, /* reverse */ ij.Out.NeededColumns(), false, /* isCheck */ &ij.alloc, spec.Visibility, spec.LockingStrength, + sysColDescs, ); err != nil { return nil, err } diff --git a/pkg/sql/rowexec/inverted_joiner.go b/pkg/sql/rowexec/inverted_joiner.go index f6300143a76f..47caecda6da0 100644 --- a/pkg/sql/rowexec/inverted_joiner.go +++ b/pkg/sql/rowexec/inverted_joiner.go @@ -255,7 +255,7 @@ func newInvertedJoiner( _, _, err = initRowFetcher( flowCtx, &fetcher, &ij.desc, int(spec.IndexIdx), ij.colIdxMap, false, /* reverse */ allIndexCols, false /* isCheck */, &ij.alloc, execinfra.ScanVisibilityPublic, - sqlbase.ScanLockingStrength_FOR_NONE, + sqlbase.ScanLockingStrength_FOR_NONE, nil, /* systemColumns */ ) if err != nil { return nil, err diff --git a/pkg/sql/rowexec/joinreader.go b/pkg/sql/rowexec/joinreader.go index cf8028099c1e..ccd31255a979 100644 --- a/pkg/sql/rowexec/joinreader.go +++ b/pkg/sql/rowexec/joinreader.go @@ -164,7 +164,7 @@ func newJoinReader( var fetcher row.Fetcher _, _, err = initRowFetcher( flowCtx, &fetcher, &jr.desc, int(spec.IndexIdx), jr.colIdxMap, false, /* reverse */ - neededRightCols, false /* isCheck */, &jr.alloc, spec.Visibility, spec.LockingStrength, + neededRightCols, false /* isCheck */, &jr.alloc, spec.Visibility, spec.LockingStrength, nil, /* systemColumns */ ) if err != nil { return nil, err diff --git a/pkg/sql/rowexec/rowfetcher.go b/pkg/sql/rowexec/rowfetcher.go index 7365d2fd43a7..04e8f272655e 100644 --- a/pkg/sql/rowexec/rowfetcher.go +++ b/pkg/sql/rowexec/rowfetcher.go @@ -64,6 +64,7 @@ func initRowFetcher( alloc *sqlbase.DatumAlloc, scanVisibility execinfrapb.ScanVisibility, lockStr sqlbase.ScanLockingStrength, + systemColumns []sqlbase.ColumnDescriptor, ) (index *sqlbase.IndexDescriptor, isSecondaryIndex bool, err error) { immutDesc := sqlbase.NewImmutableTableDescriptor(*desc) index, isSecondaryIndex, err = immutDesc.FindIndexByIndexIdx(indexIdx) @@ -75,6 +76,9 @@ func initRowFetcher( if scanVisibility == execinfra.ScanVisibilityPublicAndNotPublic { cols = immutDesc.ReadableColumns } + // Add on any requested system columns. We slice cols to avoid modifying + // the underlying table descriptor. + cols = append(cols[:len(cols):len(cols)], systemColumns...) tableArgs := row.FetcherTableArgs{ Desc: immutDesc, Index: index, diff --git a/pkg/sql/rowexec/scrub_tablereader.go b/pkg/sql/rowexec/scrub_tablereader.go index a5e365b168a9..f05d48f7bbda 100644 --- a/pkg/sql/rowexec/scrub_tablereader.go +++ b/pkg/sql/rowexec/scrub_tablereader.go @@ -124,7 +124,7 @@ func newScrubTableReader( if _, _, err := initRowFetcher( flowCtx, &fetcher, &tr.tableDesc, int(spec.IndexIdx), tr.tableDesc.ColumnIdxMap(), spec.Reverse, neededColumns, true /* isCheck */, &tr.alloc, - execinfra.ScanVisibilityPublic, spec.LockingStrength, + execinfra.ScanVisibilityPublic, spec.LockingStrength, nil, /* systemColumns */ ); err != nil { return nil, err } diff --git a/pkg/sql/rowexec/tablereader.go b/pkg/sql/rowexec/tablereader.go index 7a78fc418fa8..206de7ebb15b 100644 --- a/pkg/sql/rowexec/tablereader.go +++ b/pkg/sql/rowexec/tablereader.go @@ -89,12 +89,23 @@ func newTableReader( tr.maxTimestampAge = time.Duration(spec.MaxTimestampAgeNanos) returnMutations := spec.Visibility == execinfra.ScanVisibilityPublicAndNotPublic - types := spec.Table.ColumnTypesWithMutations(returnMutations) + resultTypes := spec.Table.ColumnTypesWithMutations(returnMutations) + columnIdxMap := spec.Table.ColumnIdxMapWithMutations(returnMutations) + // Add all requested system columns to the output. + sysColTypes, sysColDescs, err := sqlbase.GetSystemColumnTypesAndDescriptors(&spec.Table, spec.SystemColumns) + if err != nil { + return nil, err + } + resultTypes = append(resultTypes, sysColTypes...) + for i := range sysColDescs { + columnIdxMap[sysColDescs[i].ID] = len(columnIdxMap) + } + tr.ignoreMisplannedRanges = flowCtx.Local if err := tr.Init( tr, post, - types, + resultTypes, flowCtx, processorID, output, @@ -114,10 +125,19 @@ func newTableReader( neededColumns := tr.Out.NeededColumns() var fetcher row.Fetcher - columnIdxMap := spec.Table.ColumnIdxMapWithMutations(returnMutations) if _, _, err := initRowFetcher( - flowCtx, &fetcher, &spec.Table, int(spec.IndexIdx), columnIdxMap, spec.Reverse, - neededColumns, spec.IsCheck, &tr.alloc, spec.Visibility, spec.LockingStrength, + flowCtx, + &fetcher, + &spec.Table, + int(spec.IndexIdx), + columnIdxMap, + spec.Reverse, + neededColumns, + spec.IsCheck, + &tr.alloc, + spec.Visibility, + spec.LockingStrength, + sysColDescs, ); err != nil { return nil, err } diff --git a/pkg/sql/rowexec/zigzagjoiner.go b/pkg/sql/rowexec/zigzagjoiner.go index 8c9508a49d9d..ccd12ac9861b 100644 --- a/pkg/sql/rowexec/zigzagjoiner.go +++ b/pkg/sql/rowexec/zigzagjoiner.go @@ -455,6 +455,7 @@ func (z *zigzagJoiner) setupInfo( // NB: zigzag joins are disabled when a row-level locking clause is // supplied, so there is no locking strength on *ZigzagJoinerSpec. sqlbase.ScanLockingStrength_FOR_NONE, + nil, /* systemColumns */ ) if err != nil { return err diff --git a/pkg/sql/scan.go b/pkg/sql/scan.go index 68073e0c0659..730776a02b14 100644 --- a/pkg/sql/scan.go +++ b/pkg/sql/scan.go @@ -98,6 +98,12 @@ type scanNode struct { // mode of the Scan. lockingStrength sqlbase.ScanLockingStrength lockingWaitPolicy sqlbase.ScanLockingWaitPolicy + + // systemColumns and systemColumnOrdinals contain information about what + // system columns the scan needs to produce, and what row ordinals to + // write those columns out into. + systemColumns []sqlbase.SystemColumnKind + systemColumnOrdinals []int } // scanColumnsConfig controls the "schema" of a scan node. @@ -256,13 +262,23 @@ func initColsForScan( for _, wc := range colCfg.wantedColumns { var c *sqlbase.ColumnDescriptor var err error - if id := sqlbase.ColumnID(wc); colCfg.visibility == execinfra.ScanVisibilityPublic { - c, err = desc.FindActiveColumnByID(id) + if sqlbase.IsColIDSystemColumn(sqlbase.ColumnID(wc)) { + // If the requested column is a system column, then retrieve the + // corresponding descriptor. + c, err = sqlbase.GetSystemColumnDescriptorFromID(sqlbase.ColumnID(wc)) + if err != nil { + return nil, err + } } else { - c, _, err = desc.FindReadableColumnByID(id) - } - if err != nil { - return cols, err + // Otherwise, collect the descriptors from the table's columns. + if id := sqlbase.ColumnID(wc); colCfg.visibility == execinfra.ScanVisibilityPublic { + c, err = desc.FindActiveColumnByID(id) + } else { + c, _, err = desc.FindReadableColumnByID(id) + } + if err != nil { + return cols, err + } } cols = append(cols, c) diff --git a/pkg/sql/sem/tree/eval.go b/pkg/sql/sem/tree/eval.go index dd1b7f6fd44c..019d313bf646 100644 --- a/pkg/sql/sem/tree/eval.go +++ b/pkg/sql/sem/tree/eval.go @@ -3224,7 +3224,7 @@ func (ctx *EvalContext) GetClusterTimestamp() *DDecimal { if ts == (hlc.Timestamp{}) { panic(errors.AssertionFailedf("zero cluster timestamp in txn")) } - return TimestampToDecimal(ts) + return TimestampToDecimalDatum(ts) } // HasPlaceholders returns true if this EvalContext's placeholders have been @@ -3236,11 +3236,11 @@ func (ctx *EvalContext) HasPlaceholders() bool { // TimestampToDecimal converts the logical timestamp into a decimal // value with the number of nanoseconds in the integer part and the // logical counter in the decimal part. -func TimestampToDecimal(ts hlc.Timestamp) *DDecimal { +func TimestampToDecimal(ts hlc.Timestamp) apd.Decimal { // Compute Walltime * 10^10 + Logical. // We need 10 decimals for the Logical field because its maximum // value is 4294967295 (2^32-1), a value with 10 decimal digits. - var res DDecimal + var res apd.Decimal val := &res.Coeff val.SetInt64(ts.WallTime) val.Mul(val, big10E10) @@ -3253,8 +3253,17 @@ func TimestampToDecimal(ts hlc.Timestamp) *DDecimal { // Shift 10 decimals to the right, so that the logical // field appears as fractional part. - res.Decimal.Exponent = -10 - return &res + res.Exponent = -10 + return res +} + +// TimestampToDecimalDatum is the same as TimestampToDecimal, but +// returns a datum. +func TimestampToDecimalDatum(ts hlc.Timestamp) *DDecimal { + res := TimestampToDecimal(ts) + return &DDecimal{ + Decimal: res, + } } // TimestampToInexactDTimestamp converts the logical timestamp into an diff --git a/pkg/sql/sqlbase/index_encoding.go b/pkg/sql/sqlbase/index_encoding.go index c39ade84c6f9..9d715a2bc463 100644 --- a/pkg/sql/sqlbase/index_encoding.go +++ b/pkg/sql/sqlbase/index_encoding.go @@ -240,7 +240,7 @@ func MakeSpanFromEncDatums( // retrieve neededCols for the specified table and index. The returned FamilyIDs // are in sorted order. func NeededColumnFamilyIDs( - neededCols util.FastIntSet, table *TableDescriptor, index *IndexDescriptor, + neededColOrdinals util.FastIntSet, table *TableDescriptor, index *IndexDescriptor, ) []FamilyID { if len(table.Families) == 1 { return []FamilyID{table.Families[0].ID} @@ -275,11 +275,14 @@ func NeededColumnFamilyIDs( hasSecondaryEncoding := index.GetEncodingType(table.PrimaryIndex.ID) == SecondaryIndexEncoding // First iterate over the needed columns and look for a few special cases: - // columns which can be decoded from the key and columns whose value is stored - // in family 0. + // * columns which can be decoded from the key and columns whose value is stored + // in family 0. + // * certain system columns, like the MVCC timestamp column require all of the + // column families to be scanned to produce a value. family0Needed := false - nc := neededCols.Copy() - neededCols.ForEach(func(columnOrdinal int) { + mvccColumnRequested := false + nc := neededColOrdinals.Copy() + neededColOrdinals.ForEach(func(columnOrdinal int) { if indexedCols.Contains(columnOrdinal) && !compositeCols.Contains(columnOrdinal) { // We can decode this column from the index key, so no particular family // is needed. @@ -292,8 +295,21 @@ func NeededColumnFamilyIDs( family0Needed = true nc.Remove(columnOrdinal) } + // System column ordinals are larger than the number of columns. + if columnOrdinal >= len(columns) { + mvccColumnRequested = true + } }) + // If the MVCC timestamp column was requested, then bail out. + if mvccColumnRequested { + var families []FamilyID + for i := range table.Families { + families = append(families, table.Families[i].ID) + } + return families + } + // Iterate over the column families to find which ones contain needed columns. // We also keep track of whether all of the needed families' columns are // nullable, since this means we need column family 0 as a sentinel, even if diff --git a/pkg/sql/sqlbase/structured.go b/pkg/sql/sqlbase/structured.go index bb92447381cf..8a75d96744d9 100644 --- a/pkg/sql/sqlbase/structured.go +++ b/pkg/sql/sqlbase/structured.go @@ -2002,6 +2002,10 @@ func (desc *TableDescriptor) ValidateTable() error { return pgerror.Newf(pgcode.DuplicateColumn, "duplicate: column %q in the middle of being added, not yet public", column.Name) } + if IsSystemColumnName(column.Name) { + return pgerror.Newf(pgcode.DuplicateColumn, + "column name %q conflicts with a system column name", column.Name) + } columnNames[column.Name] = column.ID if other, ok := columnIDs[column.ID]; ok { diff --git a/pkg/sql/sqlbase/structured.pb.go b/pkg/sql/sqlbase/structured.pb.go index 1df6b7bb5278..2530de5370c4 100644 --- a/pkg/sql/sqlbase/structured.pb.go +++ b/pkg/sql/sqlbase/structured.pb.go @@ -74,7 +74,48 @@ func (x *ConstraintValidity) UnmarshalJSON(data []byte) error { return nil } func (ConstraintValidity) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{0} + return fileDescriptor_structured_994b169dcbedf9d2, []int{0} +} + +// SystemColumnKind is an enum representing the different kind of system +// columns that can be synthesized by the execution engine. +type SystemColumnKind int32 + +const ( + // Default value, unused. + SystemColumnKind_NONE SystemColumnKind = 0 + // A system column containing the value of the MVCC timestamp associated + // with the kv's corresponding to the row. + SystemColumnKind_MVCCTIMESTAMP SystemColumnKind = 1 +) + +var SystemColumnKind_name = map[int32]string{ + 0: "NONE", + 1: "MVCCTIMESTAMP", +} +var SystemColumnKind_value = map[string]int32{ + "NONE": 0, + "MVCCTIMESTAMP": 1, +} + +func (x SystemColumnKind) Enum() *SystemColumnKind { + p := new(SystemColumnKind) + *p = x + return p +} +func (x SystemColumnKind) String() string { + return proto.EnumName(SystemColumnKind_name, int32(x)) +} +func (x *SystemColumnKind) UnmarshalJSON(data []byte) error { + value, err := proto.UnmarshalJSONEnum(SystemColumnKind_value, data, "SystemColumnKind") + if err != nil { + return err + } + *x = SystemColumnKind(value) + return nil +} +func (SystemColumnKind) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_structured_994b169dcbedf9d2, []int{1} } type ForeignKeyReference_Action int32 @@ -119,7 +160,7 @@ func (x *ForeignKeyReference_Action) UnmarshalJSON(data []byte) error { return nil } func (ForeignKeyReference_Action) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{0, 0} + return fileDescriptor_structured_994b169dcbedf9d2, []int{0, 0} } // Match is the algorithm used to compare composite keys. @@ -159,7 +200,7 @@ func (x *ForeignKeyReference_Match) UnmarshalJSON(data []byte) error { return nil } func (ForeignKeyReference_Match) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{0, 1} + return fileDescriptor_structured_994b169dcbedf9d2, []int{0, 1} } // The direction of a column in the index. @@ -196,7 +237,7 @@ func (x *IndexDescriptor_Direction) UnmarshalJSON(data []byte) error { return nil } func (IndexDescriptor_Direction) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{7, 0} + return fileDescriptor_structured_994b169dcbedf9d2, []int{7, 0} } // The type of the index. @@ -233,7 +274,7 @@ func (x *IndexDescriptor_Type) UnmarshalJSON(data []byte) error { return nil } func (IndexDescriptor_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{7, 1} + return fileDescriptor_structured_994b169dcbedf9d2, []int{7, 1} } type ConstraintToUpdate_ConstraintType int32 @@ -276,7 +317,7 @@ func (x *ConstraintToUpdate_ConstraintType) UnmarshalJSON(data []byte) error { return nil } func (ConstraintToUpdate_ConstraintType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{8, 0} + return fileDescriptor_structured_994b169dcbedf9d2, []int{8, 0} } // A descriptor within a mutation is unavailable for reads, writes @@ -341,7 +382,7 @@ func (x *DescriptorMutation_State) UnmarshalJSON(data []byte) error { return nil } func (DescriptorMutation_State) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{11, 0} + return fileDescriptor_structured_994b169dcbedf9d2, []int{11, 0} } // Direction of mutation. @@ -384,7 +425,7 @@ func (x *DescriptorMutation_Direction) UnmarshalJSON(data []byte) error { return nil } func (DescriptorMutation_Direction) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{11, 1} + return fileDescriptor_structured_994b169dcbedf9d2, []int{11, 1} } // State is set if this TableDescriptor is in the process of being added or deleted. @@ -435,7 +476,7 @@ func (x *TableDescriptor_State) UnmarshalJSON(data []byte) error { return nil } func (TableDescriptor_State) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{13, 0} + return fileDescriptor_structured_994b169dcbedf9d2, []int{13, 0} } // AuditMode indicates which auditing actions to take when this table is used. @@ -472,7 +513,7 @@ func (x *TableDescriptor_AuditMode) UnmarshalJSON(data []byte) error { return nil } func (TableDescriptor_AuditMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{13, 1} + return fileDescriptor_structured_994b169dcbedf9d2, []int{13, 1} } // Represents the kind of type that this type descriptor represents. @@ -512,7 +553,7 @@ func (x *TypeDescriptor_Kind) UnmarshalJSON(data []byte) error { return nil } func (TypeDescriptor_Kind) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{15, 0} + return fileDescriptor_structured_994b169dcbedf9d2, []int{15, 0} } // ForeignKeyReference is deprecated, replaced by ForeignKeyConstraint in v19.2 @@ -542,7 +583,7 @@ func (m *ForeignKeyReference) Reset() { *m = ForeignKeyReference{} } func (m *ForeignKeyReference) String() string { return proto.CompactTextString(m) } func (*ForeignKeyReference) ProtoMessage() {} func (*ForeignKeyReference) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{0} + return fileDescriptor_structured_994b169dcbedf9d2, []int{0} } func (m *ForeignKeyReference) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -590,7 +631,7 @@ func (m *ForeignKeyConstraint) Reset() { *m = ForeignKeyConstraint{} } func (m *ForeignKeyConstraint) String() string { return proto.CompactTextString(m) } func (*ForeignKeyConstraint) ProtoMessage() {} func (*ForeignKeyConstraint) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{1} + return fileDescriptor_structured_994b169dcbedf9d2, []int{1} } func (m *ForeignKeyConstraint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -646,13 +687,16 @@ type ColumnDescriptor struct { LogicalColumnID ColumnID `protobuf:"varint,13,opt,name=logical_id,json=logicalId,casttype=ColumnID" json:"logical_id"` // Used to indicate column is used and dropped for ALTER COLUMN TYPE mutation. AlterColumnTypeInProgress bool `protobuf:"varint,14,opt,name=alter_column_type_in_progress,json=alterColumnTypeInProgress" json:"alter_column_type_in_progress"` + // SystemColumnKind represents what kind of system column this column + // descriptor represents, if any. + SystemColumnKind SystemColumnKind `protobuf:"varint,15,opt,name=system_column_kind,json=systemColumnKind,enum=cockroach.sql.sqlbase.SystemColumnKind" json:"system_column_kind"` } func (m *ColumnDescriptor) Reset() { *m = ColumnDescriptor{} } func (m *ColumnDescriptor) String() string { return proto.CompactTextString(m) } func (*ColumnDescriptor) ProtoMessage() {} func (*ColumnDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{2} + return fileDescriptor_structured_994b169dcbedf9d2, []int{2} } func (m *ColumnDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -708,7 +752,7 @@ func (m *ColumnFamilyDescriptor) Reset() { *m = ColumnFamilyDescriptor{} func (m *ColumnFamilyDescriptor) String() string { return proto.CompactTextString(m) } func (*ColumnFamilyDescriptor) ProtoMessage() {} func (*ColumnFamilyDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{3} + return fileDescriptor_structured_994b169dcbedf9d2, []int{3} } func (m *ColumnFamilyDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -754,7 +798,7 @@ func (m *InterleaveDescriptor) Reset() { *m = InterleaveDescriptor{} } func (m *InterleaveDescriptor) String() string { return proto.CompactTextString(m) } func (*InterleaveDescriptor) ProtoMessage() {} func (*InterleaveDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{4} + return fileDescriptor_structured_994b169dcbedf9d2, []int{4} } func (m *InterleaveDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -798,7 +842,7 @@ func (m *InterleaveDescriptor_Ancestor) Reset() { *m = InterleaveDescrip func (m *InterleaveDescriptor_Ancestor) String() string { return proto.CompactTextString(m) } func (*InterleaveDescriptor_Ancestor) ProtoMessage() {} func (*InterleaveDescriptor_Ancestor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{4, 0} + return fileDescriptor_structured_994b169dcbedf9d2, []int{4, 0} } func (m *InterleaveDescriptor_Ancestor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -853,7 +897,7 @@ func (m *ShardedDescriptor) Reset() { *m = ShardedDescriptor{} } func (m *ShardedDescriptor) String() string { return proto.CompactTextString(m) } func (*ShardedDescriptor) ProtoMessage() {} func (*ShardedDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{5} + return fileDescriptor_structured_994b169dcbedf9d2, []int{5} } func (m *ShardedDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -898,7 +942,7 @@ func (m *PartitioningDescriptor) Reset() { *m = PartitioningDescriptor{} func (m *PartitioningDescriptor) String() string { return proto.CompactTextString(m) } func (*PartitioningDescriptor) ProtoMessage() {} func (*PartitioningDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{6} + return fileDescriptor_structured_994b169dcbedf9d2, []int{6} } func (m *PartitioningDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -941,7 +985,7 @@ func (m *PartitioningDescriptor_List) Reset() { *m = PartitioningDescrip func (m *PartitioningDescriptor_List) String() string { return proto.CompactTextString(m) } func (*PartitioningDescriptor_List) ProtoMessage() {} func (*PartitioningDescriptor_List) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{6, 0} + return fileDescriptor_structured_994b169dcbedf9d2, []int{6, 0} } func (m *PartitioningDescriptor_List) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -986,7 +1030,7 @@ func (m *PartitioningDescriptor_Range) Reset() { *m = PartitioningDescri func (m *PartitioningDescriptor_Range) String() string { return proto.CompactTextString(m) } func (*PartitioningDescriptor_Range) ProtoMessage() {} func (*PartitioningDescriptor_Range) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{6, 1} + return fileDescriptor_structured_994b169dcbedf9d2, []int{6, 1} } func (m *PartitioningDescriptor_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1150,7 +1194,7 @@ func (m *IndexDescriptor) Reset() { *m = IndexDescriptor{} } func (m *IndexDescriptor) String() string { return proto.CompactTextString(m) } func (*IndexDescriptor) ProtoMessage() {} func (*IndexDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{7} + return fileDescriptor_structured_994b169dcbedf9d2, []int{7} } func (m *IndexDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1201,7 +1245,7 @@ func (m *ConstraintToUpdate) Reset() { *m = ConstraintToUpdate{} } func (m *ConstraintToUpdate) String() string { return proto.CompactTextString(m) } func (*ConstraintToUpdate) ProtoMessage() {} func (*ConstraintToUpdate) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{8} + return fileDescriptor_structured_994b169dcbedf9d2, []int{8} } func (m *ConstraintToUpdate) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1244,7 +1288,7 @@ func (m *PrimaryKeySwap) Reset() { *m = PrimaryKeySwap{} } func (m *PrimaryKeySwap) String() string { return proto.CompactTextString(m) } func (*PrimaryKeySwap) ProtoMessage() {} func (*PrimaryKeySwap) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{9} + return fileDescriptor_structured_994b169dcbedf9d2, []int{9} } func (m *PrimaryKeySwap) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1284,7 +1328,7 @@ func (m *ComputedColumnSwap) Reset() { *m = ComputedColumnSwap{} } func (m *ComputedColumnSwap) String() string { return proto.CompactTextString(m) } func (*ComputedColumnSwap) ProtoMessage() {} func (*ComputedColumnSwap) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{10} + return fileDescriptor_structured_994b169dcbedf9d2, []int{10} } func (m *ComputedColumnSwap) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1340,7 +1384,7 @@ func (m *DescriptorMutation) Reset() { *m = DescriptorMutation{} } func (m *DescriptorMutation) String() string { return proto.CompactTextString(m) } func (*DescriptorMutation) ProtoMessage() {} func (*DescriptorMutation) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{11} + return fileDescriptor_structured_994b169dcbedf9d2, []int{11} } func (m *DescriptorMutation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1648,7 +1692,7 @@ func (m *NameInfo) Reset() { *m = NameInfo{} } func (m *NameInfo) String() string { return proto.CompactTextString(m) } func (*NameInfo) ProtoMessage() {} func (*NameInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{12} + return fileDescriptor_structured_994b169dcbedf9d2, []int{12} } func (m *NameInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1825,7 +1869,7 @@ func (m *TableDescriptor) Reset() { *m = TableDescriptor{} } func (m *TableDescriptor) String() string { return proto.CompactTextString(m) } func (*TableDescriptor) ProtoMessage() {} func (*TableDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{13} + return fileDescriptor_structured_994b169dcbedf9d2, []int{13} } func (m *TableDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2125,7 +2169,7 @@ func (m *TableDescriptor_SchemaChangeLease) Reset() { *m = TableDescript func (m *TableDescriptor_SchemaChangeLease) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_SchemaChangeLease) ProtoMessage() {} func (*TableDescriptor_SchemaChangeLease) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{13, 0} + return fileDescriptor_structured_994b169dcbedf9d2, []int{13, 0} } func (m *TableDescriptor_SchemaChangeLease) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2171,7 +2215,7 @@ func (m *TableDescriptor_CheckConstraint) Reset() { *m = TableDescriptor func (m *TableDescriptor_CheckConstraint) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_CheckConstraint) ProtoMessage() {} func (*TableDescriptor_CheckConstraint) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{13, 1} + return fileDescriptor_structured_994b169dcbedf9d2, []int{13, 1} } func (m *TableDescriptor_CheckConstraint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2211,7 +2255,7 @@ func (m *TableDescriptor_Reference) Reset() { *m = TableDescriptor_Refer func (m *TableDescriptor_Reference) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_Reference) ProtoMessage() {} func (*TableDescriptor_Reference) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{13, 2} + return fileDescriptor_structured_994b169dcbedf9d2, []int{13, 2} } func (m *TableDescriptor_Reference) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2248,7 +2292,7 @@ func (m *TableDescriptor_MutationJob) Reset() { *m = TableDescriptor_Mut func (m *TableDescriptor_MutationJob) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_MutationJob) ProtoMessage() {} func (*TableDescriptor_MutationJob) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{13, 3} + return fileDescriptor_structured_994b169dcbedf9d2, []int{13, 3} } func (m *TableDescriptor_MutationJob) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2291,7 +2335,7 @@ func (m *TableDescriptor_SequenceOpts) Reset() { *m = TableDescriptor_Se func (m *TableDescriptor_SequenceOpts) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_SequenceOpts) ProtoMessage() {} func (*TableDescriptor_SequenceOpts) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{13, 4} + return fileDescriptor_structured_994b169dcbedf9d2, []int{13, 4} } func (m *TableDescriptor_SequenceOpts) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2331,7 +2375,7 @@ func (m *TableDescriptor_SequenceOpts_SequenceOwner) String() string { } func (*TableDescriptor_SequenceOpts_SequenceOwner) ProtoMessage() {} func (*TableDescriptor_SequenceOpts_SequenceOwner) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{13, 4, 0} + return fileDescriptor_structured_994b169dcbedf9d2, []int{13, 4, 0} } func (m *TableDescriptor_SequenceOpts_SequenceOwner) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2371,7 +2415,7 @@ func (m *TableDescriptor_Replacement) Reset() { *m = TableDescriptor_Rep func (m *TableDescriptor_Replacement) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_Replacement) ProtoMessage() {} func (*TableDescriptor_Replacement) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{13, 5} + return fileDescriptor_structured_994b169dcbedf9d2, []int{13, 5} } func (m *TableDescriptor_Replacement) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2408,7 +2452,7 @@ func (m *TableDescriptor_GCDescriptorMutation) Reset() { *m = TableDescr func (m *TableDescriptor_GCDescriptorMutation) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_GCDescriptorMutation) ProtoMessage() {} func (*TableDescriptor_GCDescriptorMutation) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{13, 6} + return fileDescriptor_structured_994b169dcbedf9d2, []int{13, 6} } func (m *TableDescriptor_GCDescriptorMutation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2451,7 +2495,7 @@ func (m *DatabaseDescriptor) Reset() { *m = DatabaseDescriptor{} } func (m *DatabaseDescriptor) String() string { return proto.CompactTextString(m) } func (*DatabaseDescriptor) ProtoMessage() {} func (*DatabaseDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{14} + return fileDescriptor_structured_994b169dcbedf9d2, []int{14} } func (m *DatabaseDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2549,7 +2593,7 @@ func (m *TypeDescriptor) Reset() { *m = TypeDescriptor{} } func (m *TypeDescriptor) String() string { return proto.CompactTextString(m) } func (*TypeDescriptor) ProtoMessage() {} func (*TypeDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{15} + return fileDescriptor_structured_994b169dcbedf9d2, []int{15} } func (m *TypeDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2661,7 +2705,7 @@ func (m *TypeDescriptor_EnumMember) Reset() { *m = TypeDescriptor_EnumMe func (m *TypeDescriptor_EnumMember) String() string { return proto.CompactTextString(m) } func (*TypeDescriptor_EnumMember) ProtoMessage() {} func (*TypeDescriptor_EnumMember) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{15, 0} + return fileDescriptor_structured_994b169dcbedf9d2, []int{15, 0} } func (m *TypeDescriptor_EnumMember) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2707,7 +2751,7 @@ func (m *SchemaDescriptor) Reset() { *m = SchemaDescriptor{} } func (m *SchemaDescriptor) String() string { return proto.CompactTextString(m) } func (*SchemaDescriptor) ProtoMessage() {} func (*SchemaDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{16} + return fileDescriptor_structured_994b169dcbedf9d2, []int{16} } func (m *SchemaDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2796,7 +2840,7 @@ func (m *Descriptor) Reset() { *m = Descriptor{} } func (m *Descriptor) String() string { return proto.CompactTextString(m) } func (*Descriptor) ProtoMessage() {} func (*Descriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_455e90a118345fa6, []int{17} + return fileDescriptor_structured_994b169dcbedf9d2, []int{17} } func (m *Descriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3025,6 +3069,7 @@ func init() { proto.RegisterType((*SchemaDescriptor)(nil), "cockroach.sql.sqlbase.SchemaDescriptor") proto.RegisterType((*Descriptor)(nil), "cockroach.sql.sqlbase.Descriptor") proto.RegisterEnum("cockroach.sql.sqlbase.ConstraintValidity", ConstraintValidity_name, ConstraintValidity_value) + proto.RegisterEnum("cockroach.sql.sqlbase.SystemColumnKind", SystemColumnKind_name, SystemColumnKind_value) proto.RegisterEnum("cockroach.sql.sqlbase.ForeignKeyReference_Action", ForeignKeyReference_Action_name, ForeignKeyReference_Action_value) proto.RegisterEnum("cockroach.sql.sqlbase.ForeignKeyReference_Match", ForeignKeyReference_Match_name, ForeignKeyReference_Match_value) proto.RegisterEnum("cockroach.sql.sqlbase.IndexDescriptor_Direction", IndexDescriptor_Direction_name, IndexDescriptor_Direction_value) @@ -3213,6 +3258,9 @@ func (this *ColumnDescriptor) Equal(that interface{}) bool { if this.AlterColumnTypeInProgress != that1.AlterColumnTypeInProgress { return false } + if this.SystemColumnKind != that1.SystemColumnKind { + return false + } return true } func (this *ColumnFamilyDescriptor) Equal(that interface{}) bool { @@ -4837,6 +4885,9 @@ func (m *ColumnDescriptor) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0 } i++ + dAtA[i] = 0x78 + i++ + i = encodeVarintStructured(dAtA, i, uint64(m.SystemColumnKind)) return i, nil } @@ -6555,6 +6606,7 @@ func (m *ColumnDescriptor) Size() (n int) { } n += 1 + sovStructured(uint64(m.LogicalColumnID)) n += 2 + n += 1 + sovStructured(uint64(m.SystemColumnKind)) return n } @@ -8260,6 +8312,25 @@ func (m *ColumnDescriptor) Unmarshal(dAtA []byte) error { } } m.AlterColumnTypeInProgress = bool(v != 0) + case 15: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SystemColumnKind", wireType) + } + m.SystemColumnKind = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStructured + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.SystemColumnKind |= (SystemColumnKind(b) & 0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipStructured(dAtA[iNdEx:]) @@ -14324,273 +14395,276 @@ var ( ) func init() { - proto.RegisterFile("sql/sqlbase/structured.proto", fileDescriptor_structured_455e90a118345fa6) -} - -var fileDescriptor_structured_455e90a118345fa6 = []byte{ - // 4215 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x3b, 0x49, 0x73, 0x1b, 0x57, - 0x7a, 0x68, 0xec, 0xf8, 0xb0, 0xb0, 0xf9, 0x44, 0x51, 0x10, 0xad, 0x21, 0x29, 0xc8, 0xb2, 0x39, - 0x63, 0x9b, 0x94, 0xa9, 0x49, 0x46, 0x63, 0x27, 0x53, 0x06, 0x01, 0x50, 0x04, 0x49, 0x01, 0x74, - 0x93, 0xb2, 0x66, 0xb2, 0x75, 0x9a, 0xe8, 0x07, 0xb0, 0xad, 0x46, 0x37, 0xd4, 0xdd, 0x90, 0xc8, - 0xaa, 0x9c, 0x72, 0x9a, 0x53, 0x2a, 0x97, 0xe4, 0x96, 0x2a, 0x57, 0xe2, 0xaa, 0xcc, 0x29, 0xa9, - 0x5c, 0x92, 0x5b, 0xaa, 0x72, 0x73, 0x4e, 0x99, 0xdc, 0xe6, 0xc4, 0x4a, 0xe8, 0x4b, 0x7e, 0x41, - 0x52, 0xe5, 0x5c, 0x52, 0x6f, 0xeb, 0x05, 0x0b, 0x0d, 0x92, 0xca, 0x45, 0xc5, 0xfe, 0xb6, 0xf7, - 0xde, 0xf7, 0xbe, 0xfd, 0x41, 0x70, 0xcf, 0x7d, 0x65, 0x6e, 0xb8, 0xaf, 0xcc, 0x63, 0xcd, 0xc5, - 0x1b, 0xae, 0xe7, 0x0c, 0x3b, 0xde, 0xd0, 0xc1, 0xfa, 0xfa, 0xc0, 0xb1, 0x3d, 0x1b, 0xdd, 0xee, - 0xd8, 0x9d, 0x97, 0x8e, 0xad, 0x75, 0x4e, 0xd6, 0xdd, 0x57, 0xe6, 0x3a, 0xa7, 0x5b, 0x2a, 0x0f, - 0x3d, 0xc3, 0xdc, 0x38, 0x31, 0x3b, 0x1b, 0x9e, 0xd1, 0xc7, 0xae, 0xa7, 0xf5, 0x07, 0x8c, 0x61, - 0xe9, 0x9d, 0xb0, 0xb8, 0x81, 0x63, 0xbc, 0x36, 0x4c, 0xdc, 0xc3, 0x1c, 0x79, 0x9b, 0x20, 0xbd, - 0xb3, 0x01, 0x76, 0xd9, 0xbf, 0x1c, 0x7c, 0xb7, 0x87, 0xed, 0x8d, 0x1e, 0xb6, 0x0d, 0x4b, 0xc7, - 0xa7, 0x1b, 0x1d, 0xdb, 0xea, 0x1a, 0x3d, 0x8e, 0x5a, 0xe8, 0xd9, 0x3d, 0x9b, 0xfe, 0xb9, 0x41, - 0xfe, 0x62, 0xd0, 0xca, 0x9f, 0xa6, 0xe0, 0xd6, 0xb6, 0xed, 0x60, 0xa3, 0x67, 0xed, 0xe1, 0x33, - 0x05, 0x77, 0xb1, 0x83, 0xad, 0x0e, 0x46, 0xab, 0x90, 0xf2, 0xb4, 0x63, 0x13, 0x97, 0xa5, 0x55, - 0x69, 0xad, 0xb8, 0x05, 0xdf, 0x9c, 0xaf, 0xc4, 0xbe, 0x3b, 0x5f, 0x89, 0x37, 0xeb, 0x0a, 0x43, - 0xa0, 0x87, 0x90, 0xa2, 0xab, 0x94, 0xe3, 0x94, 0x62, 0x8e, 0x53, 0x64, 0x9a, 0x04, 0x48, 0xc8, - 0x28, 0x16, 0x95, 0x21, 0x69, 0x69, 0x7d, 0x5c, 0x4e, 0xac, 0x4a, 0x6b, 0xb9, 0xad, 0x24, 0xa1, - 0x52, 0x28, 0x04, 0xed, 0x41, 0xf6, 0xb5, 0x66, 0x1a, 0xba, 0xe1, 0x9d, 0x95, 0x93, 0xab, 0xd2, - 0x5a, 0x69, 0xf3, 0x87, 0xeb, 0x13, 0x75, 0xb4, 0x5e, 0xb3, 0x2d, 0xd7, 0x73, 0x34, 0xc3, 0xf2, - 0xbe, 0xe0, 0x0c, 0x5c, 0x90, 0x2f, 0x00, 0x3d, 0x82, 0x79, 0xf7, 0x44, 0x73, 0xb0, 0xae, 0x0e, - 0x1c, 0xdc, 0x35, 0x4e, 0x55, 0x13, 0x5b, 0xe5, 0xd4, 0xaa, 0xb4, 0x96, 0xe2, 0xa4, 0x73, 0x0c, - 0x7d, 0x40, 0xb1, 0xfb, 0xd8, 0x42, 0x47, 0x90, 0xb3, 0x2d, 0x55, 0xc7, 0x26, 0xf6, 0x70, 0x39, - 0x4d, 0xd7, 0xff, 0x78, 0xca, 0xfa, 0x13, 0x14, 0xb4, 0x5e, 0xed, 0x78, 0x86, 0x6d, 0x89, 0x7d, - 0xd8, 0x56, 0x9d, 0x0a, 0xe2, 0x52, 0x87, 0x03, 0x5d, 0xf3, 0x70, 0x39, 0x73, 0x63, 0xa9, 0xcf, - 0xa9, 0x20, 0xb4, 0x0f, 0xa9, 0xbe, 0xe6, 0x75, 0x4e, 0xca, 0x59, 0x2a, 0xf1, 0xd1, 0x15, 0x24, - 0x3e, 0x23, 0x7c, 0x5c, 0x20, 0x13, 0x52, 0x79, 0x01, 0x69, 0xb6, 0x0e, 0x2a, 0x42, 0xae, 0xd5, - 0x56, 0xab, 0xb5, 0xa3, 0x66, 0xbb, 0x25, 0xc7, 0x50, 0x01, 0xb2, 0x4a, 0xe3, 0xf0, 0x48, 0x69, - 0xd6, 0x8e, 0x64, 0x89, 0x7c, 0x1d, 0x36, 0x8e, 0xd4, 0xd6, 0xf3, 0xfd, 0x7d, 0x39, 0x8e, 0xe6, - 0x20, 0x4f, 0xbe, 0xea, 0x8d, 0xed, 0xea, 0xf3, 0xfd, 0x23, 0x39, 0x81, 0xf2, 0x90, 0xa9, 0x55, - 0x0f, 0x6b, 0xd5, 0x7a, 0x43, 0x4e, 0x2e, 0x25, 0x7f, 0xf5, 0xf5, 0x72, 0xac, 0xf2, 0x08, 0x52, - 0x74, 0x39, 0x04, 0x90, 0x3e, 0x6c, 0x3e, 0x3b, 0xd8, 0x6f, 0xc8, 0x31, 0x94, 0x85, 0xe4, 0x36, - 0x11, 0x21, 0x11, 0x8e, 0x83, 0xaa, 0x72, 0xd4, 0xac, 0xee, 0xcb, 0x71, 0xc6, 0xf1, 0x49, 0xf2, - 0xbf, 0xbe, 0x5a, 0x91, 0x2a, 0xff, 0x9e, 0x82, 0x85, 0x60, 0xef, 0xc1, 0x6d, 0xa3, 0x1a, 0xcc, - 0xd9, 0x8e, 0xd1, 0x33, 0x2c, 0x95, 0xda, 0x9c, 0x6a, 0xe8, 0xdc, 0x1e, 0xdf, 0x21, 0xe7, 0xb9, - 0x38, 0x5f, 0x29, 0xb6, 0x29, 0xfa, 0x88, 0x60, 0x9b, 0x75, 0x6e, 0xa0, 0x45, 0x3b, 0x04, 0xd4, - 0xd1, 0x1e, 0xcc, 0x73, 0x21, 0x1d, 0xdb, 0x1c, 0xf6, 0x2d, 0xd5, 0xd0, 0xdd, 0x72, 0x7c, 0x35, - 0xb1, 0x56, 0xdc, 0x5a, 0xb9, 0x38, 0x5f, 0x99, 0x63, 0x22, 0x6a, 0x14, 0xd7, 0xac, 0xbb, 0xdf, - 0x9d, 0xaf, 0x64, 0xc5, 0x87, 0xc2, 0x97, 0xe7, 0xdf, 0xba, 0x8b, 0x5e, 0xc0, 0x6d, 0x47, 0xe8, - 0x56, 0x0f, 0x0b, 0x4c, 0x50, 0x81, 0x0f, 0x2e, 0xce, 0x57, 0x6e, 0xf9, 0xca, 0xd7, 0x27, 0x0b, - 0xbd, 0xe5, 0x8c, 0x12, 0xe8, 0x2e, 0x6a, 0x43, 0x08, 0x1c, 0x1c, 0x37, 0x49, 0x8f, 0xbb, 0xc2, - 0x8f, 0x3b, 0x1f, 0x88, 0x8e, 0x1e, 0x79, 0xde, 0x19, 0x41, 0xe8, 0xbe, 0xe3, 0xa5, 0x2e, 0x75, - 0xbc, 0xf4, 0x4d, 0x1d, 0x2f, 0xe2, 0x46, 0x99, 0xff, 0x17, 0x37, 0xca, 0xbe, 0x75, 0x37, 0xca, - 0xbd, 0x05, 0x37, 0x62, 0xb6, 0xbb, 0x9b, 0xcc, 0x82, 0x9c, 0xdf, 0x4d, 0x66, 0xf3, 0x72, 0x61, - 0x37, 0x99, 0x2d, 0xc8, 0xc5, 0xdd, 0x64, 0xb6, 0x28, 0x97, 0x2a, 0x7f, 0x99, 0x04, 0x99, 0xdd, - 0x6e, 0x1d, 0xbb, 0x1d, 0xc7, 0x18, 0x78, 0xb6, 0xe3, 0xdf, 0x89, 0x34, 0x76, 0x27, 0xef, 0x41, - 0xdc, 0xd0, 0x79, 0x28, 0x5d, 0xe4, 0xb7, 0x1d, 0xa7, 0xd7, 0x1b, 0xd8, 0x4d, 0xdc, 0xd0, 0xd1, - 0x3a, 0x24, 0x49, 0xbc, 0xa7, 0xe1, 0x34, 0xbf, 0xb9, 0x34, 0x7a, 0x02, 0xdc, 0x5f, 0x67, 0xe9, - 0xe0, 0x48, 0xa1, 0x74, 0x68, 0x15, 0xb2, 0xd6, 0xd0, 0x34, 0x69, 0x28, 0x27, 0xb6, 0x94, 0x15, - 0x4a, 0x11, 0x50, 0x74, 0x1f, 0x0a, 0x3a, 0xee, 0x6a, 0x43, 0xd3, 0x53, 0xf1, 0xe9, 0xc0, 0x61, - 0xf6, 0xa2, 0xe4, 0x39, 0xac, 0x71, 0x3a, 0x70, 0xd0, 0x3d, 0x48, 0x9f, 0x18, 0xba, 0x8e, 0x2d, - 0x6a, 0x2e, 0x42, 0x04, 0x87, 0xa1, 0x4d, 0x98, 0x1f, 0xba, 0xd8, 0x55, 0x5d, 0xfc, 0x6a, 0x48, - 0x74, 0x45, 0xdd, 0x01, 0xa8, 0x3b, 0xa4, 0xb9, 0x79, 0xce, 0x11, 0x82, 0x43, 0x8e, 0x27, 0xd6, - 0x7e, 0x1f, 0x0a, 0x1d, 0xbb, 0x3f, 0x18, 0x7a, 0x98, 0x2d, 0x9a, 0x67, 0x8b, 0x72, 0x18, 0x5d, - 0x74, 0x13, 0xe6, 0xed, 0x37, 0xd6, 0x88, 0xd8, 0x42, 0x54, 0x2c, 0x21, 0x08, 0x8b, 0xdd, 0x02, - 0x30, 0xed, 0x9e, 0xd1, 0xd1, 0x4c, 0xe2, 0x3b, 0x45, 0xaa, 0xcd, 0x07, 0x5c, 0x9b, 0x73, 0xfb, - 0x0c, 0x23, 0xd4, 0x19, 0x51, 0x6d, 0x8e, 0xb3, 0x35, 0x75, 0xb4, 0x0d, 0x3f, 0xd0, 0x4c, 0x0f, - 0x3b, 0xc2, 0xb9, 0x89, 0x1a, 0x55, 0xc3, 0x52, 0x07, 0x8e, 0xdd, 0x73, 0xb0, 0xeb, 0x96, 0x4b, - 0x21, 0x1d, 0xdc, 0xa5, 0xa4, 0x4c, 0xcc, 0xd1, 0xd9, 0x00, 0x37, 0xad, 0x03, 0x4e, 0xe6, 0x9b, - 0x47, 0x56, 0xce, 0xed, 0x26, 0xb3, 0x39, 0x19, 0x76, 0x93, 0xd9, 0x8c, 0x9c, 0xad, 0xfc, 0x59, - 0x1c, 0x16, 0x19, 0xc3, 0xb6, 0xd6, 0x37, 0xcc, 0xb3, 0x9b, 0x9a, 0x07, 0x93, 0xc2, 0xcd, 0x83, - 0xea, 0x95, 0x6e, 0x9b, 0xb0, 0xb1, 0xa8, 0x44, 0xf5, 0x4a, 0x60, 0x2d, 0x02, 0x42, 0x4f, 0x00, - 0x42, 0x61, 0x2b, 0x49, 0x15, 0x7a, 0xf7, 0xe2, 0x7c, 0x25, 0x37, 0x39, 0x58, 0xe5, 0x3a, 0xa1, - 0x10, 0x35, 0x2f, 0x2c, 0xc5, 0x97, 0x40, 0xcd, 0x25, 0xa4, 0xe4, 0x3a, 0x23, 0x98, 0xa8, 0xe4, - 0x39, 0x3d, 0x82, 0xd4, 0x79, 0xf4, 0xff, 0xa7, 0x38, 0x2c, 0x34, 0x2d, 0x0f, 0x3b, 0x26, 0xd6, - 0x5e, 0xe3, 0x90, 0x3a, 0x7e, 0x0e, 0x39, 0xcd, 0xea, 0x60, 0xd7, 0xb3, 0x1d, 0xb7, 0x2c, 0xad, - 0x26, 0xd6, 0xf2, 0x9b, 0x3f, 0x9e, 0xe2, 0xb2, 0x93, 0xf8, 0xd7, 0xab, 0x9c, 0x99, 0x6b, 0x32, - 0x10, 0xb6, 0xf4, 0xcf, 0x12, 0x64, 0x05, 0x16, 0x3d, 0x82, 0xec, 0x48, 0x76, 0xb9, 0xcd, 0x4f, - 0x93, 0x89, 0x06, 0xd9, 0x8c, 0xc7, 0x43, 0xeb, 0x6f, 0x41, 0x96, 0x16, 0x37, 0xaa, 0x7f, 0x27, - 0x4b, 0x82, 0x83, 0x57, 0x3f, 0xe1, 0x42, 0x28, 0x43, 0x69, 0x9b, 0x3a, 0xaa, 0x4d, 0xaa, 0x51, - 0x12, 0x94, 0xff, 0x8e, 0xd0, 0xdf, 0x61, 0xb4, 0x4a, 0x19, 0x2b, 0x5b, 0x98, 0xce, 0xb8, 0xe6, - 0xfe, 0x51, 0x82, 0x79, 0xc2, 0xa0, 0x63, 0x3d, 0xa4, 0xb6, 0x07, 0x00, 0x86, 0xab, 0xba, 0x0c, - 0x4e, 0x4f, 0x24, 0xac, 0x35, 0x67, 0xb8, 0x9c, 0xdc, 0x37, 0xb5, 0xf8, 0x98, 0xa9, 0xfd, 0x14, - 0x8a, 0x94, 0x57, 0x3d, 0x1e, 0x76, 0x5e, 0x62, 0xcf, 0xa5, 0x3b, 0x4c, 0x6d, 0x2d, 0xf0, 0x1d, - 0x16, 0xa8, 0x84, 0x2d, 0x86, 0x53, 0x0a, 0x6e, 0xe8, 0x6b, 0xcc, 0xfa, 0x92, 0x63, 0xd6, 0xc7, - 0x37, 0xfe, 0x3f, 0x09, 0x58, 0x3c, 0xd0, 0x1c, 0xcf, 0x20, 0x61, 0xda, 0xb0, 0x7a, 0xa1, 0xdd, - 0x3f, 0x84, 0xbc, 0x35, 0xec, 0x73, 0x03, 0x73, 0xf9, 0x85, 0xb0, 0xfd, 0x81, 0x35, 0xec, 0x33, - 0xdb, 0x71, 0xd1, 0x3e, 0x24, 0x4d, 0xc3, 0xf5, 0x68, 0x1e, 0xcf, 0x6f, 0x6e, 0x4e, 0x31, 0x8b, - 0xc9, 0x6b, 0xac, 0xef, 0x1b, 0xae, 0x27, 0xce, 0x4c, 0xa4, 0xa0, 0x36, 0xa4, 0x1c, 0xcd, 0xea, - 0x61, 0xea, 0x2f, 0xf9, 0xcd, 0xc7, 0x57, 0x13, 0xa7, 0x10, 0x56, 0x91, 0x1b, 0xa8, 0x9c, 0xa5, - 0xbf, 0x92, 0x20, 0x49, 0x56, 0xb9, 0xc4, 0xa5, 0x17, 0x21, 0xfd, 0x5a, 0x33, 0x87, 0x98, 0xd5, - 0x22, 0x05, 0x85, 0x7f, 0xa1, 0x3f, 0x84, 0x39, 0x77, 0x78, 0x3c, 0x08, 0x2d, 0xc5, 0x83, 0xfd, - 0x47, 0x57, 0xda, 0x95, 0x5f, 0xf6, 0x46, 0x65, 0xb1, 0x0b, 0x58, 0x7a, 0x05, 0x29, 0xba, 0xeb, - 0x4b, 0xf6, 0x77, 0x1f, 0x0a, 0x9e, 0xad, 0xe2, 0xd3, 0x8e, 0x39, 0x74, 0x8d, 0xd7, 0xcc, 0x52, - 0x0a, 0x4a, 0xde, 0xb3, 0x1b, 0x02, 0x84, 0x1e, 0x42, 0xa9, 0xeb, 0xd8, 0x7d, 0xd5, 0xb0, 0x04, - 0x51, 0x82, 0x12, 0x15, 0x09, 0xb4, 0x29, 0x80, 0x11, 0x93, 0xfd, 0x8b, 0x02, 0xcc, 0x51, 0xc7, - 0x98, 0x29, 0xec, 0x3d, 0x0c, 0x85, 0xbd, 0xdb, 0x91, 0xb0, 0xe7, 0x7b, 0x17, 0x89, 0x7a, 0xf7, - 0x20, 0x3d, 0xb4, 0x8c, 0x57, 0x43, 0xb6, 0xbe, 0x9f, 0x9f, 0x18, 0x6c, 0x06, 0xab, 0x44, 0x1f, - 0x02, 0x22, 0xa1, 0x00, 0xab, 0x11, 0xc2, 0x14, 0x25, 0x94, 0x29, 0xa6, 0x36, 0x35, 0x82, 0xa6, - 0xaf, 0x10, 0x41, 0x77, 0x40, 0xc6, 0xa7, 0x9e, 0xa3, 0x85, 0x0b, 0xc7, 0x0c, 0xe5, 0x5f, 0xbe, - 0x38, 0x5f, 0x29, 0x35, 0x08, 0x6e, 0xb2, 0x90, 0x12, 0x0e, 0xe1, 0x74, 0x62, 0x25, 0xf3, 0x5c, - 0x86, 0x6e, 0x38, 0x98, 0x96, 0x3b, 0x6e, 0x39, 0xbb, 0x9a, 0xb8, 0xa4, 0xac, 0x19, 0x51, 0xfb, - 0x7a, 0x5d, 0x30, 0x2a, 0x32, 0x13, 0xe5, 0x03, 0x5c, 0x74, 0x08, 0xf9, 0x2e, 0xab, 0x82, 0xd4, - 0x97, 0xf8, 0x8c, 0xd6, 0x4b, 0xf9, 0xcd, 0x1f, 0xcd, 0x5e, 0x2f, 0x6d, 0xa5, 0xc9, 0x15, 0x94, - 0x25, 0x05, 0xba, 0x3e, 0x12, 0xbd, 0x80, 0x62, 0xa8, 0xc4, 0x3d, 0x3e, 0xa3, 0x45, 0xc2, 0xf5, - 0xc4, 0x16, 0x02, 0x41, 0x5b, 0x67, 0xe8, 0x73, 0x00, 0xc3, 0x4f, 0x00, 0xb4, 0x96, 0xc8, 0x6f, - 0x7e, 0x70, 0x85, 0x4c, 0x21, 0xe2, 0x4b, 0x20, 0x04, 0xbd, 0x80, 0x52, 0xf0, 0x45, 0x37, 0x5b, - 0xb8, 0xf2, 0x66, 0x99, 0xd4, 0x62, 0x48, 0xce, 0x16, 0xa9, 0x97, 0x17, 0x48, 0x95, 0x63, 0xbb, - 0x86, 0x87, 0xc3, 0x66, 0x50, 0xa4, 0x66, 0x50, 0xb9, 0x38, 0x5f, 0x41, 0x35, 0x81, 0x9f, 0x6c, - 0x0a, 0xa8, 0x33, 0x82, 0x67, 0x86, 0x15, 0x31, 0x60, 0x22, 0xb1, 0x14, 0x18, 0xd6, 0x61, 0x60, - 0xc2, 0x63, 0x86, 0x15, 0x32, 0x6f, 0xd6, 0xe0, 0x14, 0x22, 0xb1, 0x67, 0xee, 0xfa, 0xb1, 0x27, - 0x22, 0x08, 0x35, 0x78, 0xe5, 0x2a, 0xd3, 0xda, 0xfb, 0x83, 0x19, 0x8d, 0x94, 0x14, 0x55, 0x22, - 0x24, 0xd0, 0x82, 0xf6, 0x31, 0xa0, 0x8e, 0x83, 0x35, 0x0f, 0xeb, 0xa4, 0x72, 0x34, 0x8d, 0x8e, - 0xe1, 0x99, 0x67, 0xe5, 0xf9, 0x90, 0xdf, 0xcf, 0x73, 0x7c, 0xc3, 0x47, 0xa3, 0x27, 0x90, 0x79, - 0x8d, 0x1d, 0xd7, 0xb0, 0xad, 0x32, 0xa2, 0xc1, 0x64, 0x99, 0x4f, 0x2b, 0x16, 0x47, 0xd6, 0xfb, - 0x82, 0x51, 0x29, 0x82, 0x1c, 0xed, 0x40, 0x11, 0x5b, 0x1d, 0x5b, 0x37, 0xac, 0x1e, 0xad, 0x04, - 0xcb, 0xb7, 0x82, 0x7a, 0xe7, 0xbb, 0xf3, 0x95, 0x77, 0x46, 0xf8, 0x1b, 0x9c, 0x96, 0x6c, 0x5b, - 0x29, 0xe0, 0xd0, 0x17, 0xda, 0x81, 0x8c, 0xc8, 0xc9, 0x0b, 0x54, 0xa7, 0x6b, 0x53, 0x54, 0x30, - 0x96, 0xd1, 0xf9, 0xb9, 0x04, 0x3b, 0xa9, 0xe9, 0x75, 0xc3, 0x25, 0xb5, 0x88, 0x5e, 0xbe, 0x1d, - 0xae, 0xe9, 0x05, 0x14, 0xd5, 0x00, 0x7a, 0xd8, 0x56, 0xd9, 0xfc, 0xa7, 0xbc, 0x48, 0x97, 0x5b, - 0x0e, 0x2d, 0xd7, 0xc3, 0xf6, 0xba, 0x98, 0x12, 0x91, 0x26, 0xaf, 0x6b, 0xf4, 0x44, 0x89, 0xd0, - 0xc3, 0x36, 0x03, 0xa0, 0x0a, 0xe4, 0x06, 0x0e, 0xd6, 0x8d, 0x0e, 0xe9, 0xc1, 0xee, 0x84, 0x62, - 0x73, 0x00, 0xae, 0x2c, 0x43, 0xce, 0x8f, 0x1a, 0x28, 0x03, 0x89, 0xea, 0x61, 0x8d, 0xb5, 0xfc, - 0xf5, 0xc6, 0x61, 0x4d, 0x96, 0x2a, 0xf7, 0x21, 0x49, 0x0f, 0x9f, 0x87, 0xcc, 0x76, 0x5b, 0x79, - 0x51, 0x55, 0xea, 0x6c, 0xcc, 0xd0, 0x6c, 0x7d, 0xd1, 0x50, 0x8e, 0x1a, 0x75, 0x59, 0xe4, 0x85, - 0x7f, 0x49, 0x00, 0x0a, 0xba, 0xcd, 0x23, 0x9b, 0x77, 0x6c, 0x3d, 0x98, 0xeb, 0xf8, 0x50, 0x76, - 0x01, 0xd2, 0x6a, 0x7c, 0xad, 0xb4, 0xf9, 0xe4, 0x7b, 0x3b, 0x56, 0x21, 0x23, 0x0c, 0x0a, 0x8c, - 0xa9, 0xd4, 0x89, 0x40, 0x43, 0xf5, 0x50, 0x7c, 0x24, 0x07, 0x29, 0x90, 0xea, 0x9c, 0xe0, 0xce, - 0x4b, 0x9e, 0x85, 0x7f, 0x7b, 0xca, 0xc2, 0xb4, 0x54, 0x0c, 0x19, 0x6e, 0x8d, 0xf0, 0x04, 0x4b, - 0x8b, 0xf2, 0x80, 0x8a, 0x42, 0x4a, 0x34, 0xbc, 0x26, 0x2f, 0x8d, 0x58, 0x93, 0x26, 0x23, 0x22, - 0x62, 0x85, 0xa2, 0xeb, 0x13, 0x98, 0xb3, 0x6c, 0x4f, 0x25, 0x7d, 0x1d, 0x8f, 0x02, 0xb4, 0x5b, - 0x2b, 0x6e, 0xc9, 0xdc, 0x56, 0x03, 0x9f, 0x2f, 0x5a, 0xb6, 0xd7, 0x1a, 0x9a, 0xbc, 0x15, 0xaa, - 0x7c, 0x02, 0xa5, 0xa8, 0x8e, 0x50, 0x0e, 0x52, 0xb5, 0x9d, 0x46, 0x6d, 0x4f, 0x8e, 0xa1, 0x39, - 0xc8, 0x6f, 0xb7, 0x95, 0x46, 0xf3, 0x69, 0x4b, 0xdd, 0x6b, 0xfc, 0x82, 0x8d, 0x85, 0x5a, 0x6d, - 0x31, 0x16, 0xf2, 0xbb, 0x9c, 0x94, 0x9c, 0xae, 0xfc, 0xb7, 0x04, 0xa5, 0x03, 0xc7, 0xe8, 0x6b, - 0xce, 0xd9, 0x1e, 0x3e, 0x3b, 0x7c, 0xa3, 0x0d, 0xd0, 0x67, 0xb0, 0x60, 0xe1, 0x37, 0xea, 0x80, - 0x41, 0x55, 0xbf, 0x6a, 0x96, 0x26, 0xcf, 0x0c, 0xe7, 0x2d, 0xfc, 0x86, 0x4b, 0x68, 0xf2, 0xa2, - 0xf9, 0x43, 0xc8, 0xdb, 0xa6, 0xce, 0x38, 0xb1, 0x98, 0xdb, 0xe4, 0xc3, 0x4c, 0x60, 0x9b, 0x7a, - 0x93, 0xa1, 0x09, 0x35, 0x59, 0x4f, 0x50, 0x27, 0x26, 0x50, 0x5b, 0xf8, 0x8d, 0xa0, 0xfe, 0x0c, - 0x16, 0x88, 0xec, 0xb1, 0xdd, 0x25, 0xa7, 0xec, 0xce, 0x36, 0xf5, 0xe8, 0xee, 0xb8, 0xf1, 0xfe, - 0xbd, 0x04, 0x34, 0x68, 0x0f, 0x3d, 0x31, 0xd1, 0xa1, 0x87, 0xff, 0x31, 0x14, 0xc9, 0x66, 0x82, - 0x5e, 0x49, 0x9a, 0x72, 0x1f, 0x64, 0xcf, 0x22, 0x02, 0x13, 0x2e, 0xb2, 0xa9, 0x80, 0x2b, 0x3e, - 0x8d, 0xcb, 0x36, 0xfd, 0xf9, 0x11, 0x7a, 0x1f, 0x0a, 0x86, 0x45, 0x82, 0x16, 0x6f, 0xa8, 0xc3, - 0xe3, 0xd6, 0x3c, 0xc7, 0x90, 0xb6, 0x9a, 0xef, 0xf8, 0x6f, 0xd2, 0x80, 0x02, 0x63, 0x7d, 0x36, - 0xf4, 0x34, 0xea, 0xc1, 0x55, 0x48, 0x73, 0xd3, 0x91, 0xa8, 0x49, 0xbe, 0x3f, 0xd5, 0xcb, 0xa2, - 0x83, 0x8d, 0x9d, 0x98, 0xc2, 0x19, 0xd1, 0xcf, 0xc2, 0x63, 0xe1, 0xfc, 0xe6, 0x7b, 0xb3, 0xc5, - 0xf9, 0x9d, 0x98, 0x98, 0x17, 0xef, 0x41, 0xca, 0xf5, 0x48, 0xc4, 0x49, 0xd0, 0x3c, 0xb1, 0x31, - 0x85, 0x7f, 0x7c, 0xf3, 0xeb, 0x87, 0x84, 0x4d, 0xf8, 0x19, 0x95, 0x81, 0x5e, 0x40, 0xce, 0x2f, - 0x8f, 0xf8, 0x8c, 0xf9, 0xf1, 0xec, 0x02, 0xfd, 0xc8, 0x26, 0xe2, 0x9e, 0x2f, 0x0b, 0x55, 0x21, - 0xdf, 0xe7, 0x64, 0x41, 0x13, 0xbc, 0xca, 0x2b, 0x54, 0x10, 0x12, 0x68, 0xa5, 0x1a, 0xfa, 0x52, - 0x40, 0x30, 0x35, 0x69, 0x14, 0x77, 0x6c, 0xd3, 0x3c, 0xd6, 0x3a, 0x2f, 0xe9, 0xdc, 0xcc, 0x8f, - 0xe2, 0x02, 0x8a, 0xf6, 0x48, 0x9d, 0x29, 0xfc, 0x92, 0x4e, 0xc1, 0xf2, 0x33, 0x4c, 0xea, 0x44, - 0xdc, 0xdb, 0x89, 0x29, 0x21, 0x76, 0xd4, 0x86, 0xd2, 0x20, 0xe2, 0x9b, 0xbc, 0xa8, 0x7b, 0x38, - 0x2d, 0xb3, 0x47, 0x88, 0x77, 0x62, 0xca, 0x08, 0x3b, 0xfa, 0x7d, 0x40, 0x9d, 0x31, 0x9b, 0x2f, - 0xc3, 0xf7, 0xec, 0x72, 0x94, 0x61, 0x27, 0xa6, 0x4c, 0x10, 0x53, 0xf9, 0x0c, 0x52, 0xf4, 0x3a, - 0x49, 0xe2, 0x78, 0xde, 0xda, 0x6b, 0xb5, 0x5f, 0xb4, 0x58, 0x2c, 0xaa, 0x37, 0xf6, 0x1b, 0x47, - 0x0d, 0xb5, 0xdd, 0xda, 0x27, 0xb1, 0xe8, 0x2e, 0xdc, 0xe6, 0x80, 0x6a, 0xab, 0xae, 0xbe, 0x50, - 0x9a, 0x02, 0x15, 0xaf, 0xac, 0x85, 0x33, 0x53, 0x16, 0x92, 0xad, 0x76, 0xab, 0x21, 0xc7, 0x68, - 0x8e, 0xaa, 0xd7, 0x65, 0x89, 0xe6, 0x28, 0xa5, 0x7d, 0x20, 0x42, 0xd8, 0x56, 0x01, 0x40, 0xf7, - 0x4d, 0x60, 0x37, 0x99, 0x4d, 0xcb, 0x99, 0xca, 0xdf, 0x4a, 0x90, 0x25, 0x25, 0x7f, 0xd3, 0xea, - 0xda, 0xe8, 0x31, 0xe4, 0x06, 0x9a, 0x83, 0x2d, 0x2f, 0xf0, 0x64, 0x31, 0x89, 0xc9, 0x1e, 0x50, - 0x84, 0x3f, 0x28, 0xc8, 0x32, 0xc2, 0xe6, 0x65, 0x6d, 0xf6, 0x36, 0xc8, 0x5c, 0x9c, 0xdb, 0x39, - 0xc1, 0x7d, 0x8d, 0x48, 0x65, 0xb3, 0x80, 0x7b, 0x5c, 0x6a, 0x89, 0x49, 0x3d, 0xa4, 0x68, 0x5f, - 0x76, 0x69, 0x10, 0x86, 0x8a, 0x08, 0xf4, 0xaf, 0xf7, 0x61, 0x6e, 0x24, 0x03, 0x5d, 0xd2, 0x56, - 0xad, 0xd2, 0xb6, 0x2a, 0x11, 0xc4, 0x15, 0xbf, 0xad, 0x8a, 0xf3, 0x8e, 0x2a, 0x72, 0xd8, 0xe4, - 0x8c, 0x87, 0x7d, 0x1c, 0x54, 0x59, 0xcc, 0x21, 0xee, 0xf2, 0x98, 0x35, 0x7f, 0x49, 0x81, 0x75, - 0x00, 0xf3, 0x7d, 0x5b, 0x37, 0xba, 0xa4, 0x9e, 0x20, 0xde, 0xe4, 0x19, 0x7d, 0x36, 0x47, 0xce, - 0x6f, 0xfe, 0x20, 0x64, 0x45, 0x43, 0xcf, 0x30, 0xd7, 0x4f, 0xcc, 0xce, 0xfa, 0x91, 0x78, 0x25, - 0xe3, 0x27, 0x92, 0xc3, 0xdc, 0x04, 0x89, 0x9e, 0x42, 0x46, 0x4c, 0x0f, 0xb2, 0xb4, 0x66, 0x9f, - 0x35, 0x8a, 0x89, 0x3a, 0x8b, 0x73, 0xa3, 0x6d, 0x28, 0x59, 0xf8, 0x34, 0x3c, 0xec, 0xca, 0x45, - 0xfc, 0xbc, 0xd0, 0xc2, 0xa7, 0x93, 0x27, 0x5d, 0x05, 0x2b, 0xc0, 0xe8, 0xe8, 0x73, 0x28, 0x46, - 0x52, 0x0c, 0x77, 0x92, 0x19, 0x43, 0xa3, 0x5f, 0x4c, 0x87, 0x32, 0x0f, 0xda, 0x86, 0x8c, 0xc8, - 0x71, 0x79, 0x7a, 0xc6, 0xab, 0x09, 0x13, 0xcc, 0x68, 0x8b, 0xa4, 0xa8, 0x53, 0x2f, 0x48, 0x7d, - 0x85, 0xa0, 0x3c, 0xbe, 0x38, 0x5f, 0xc9, 0x93, 0x13, 0x4e, 0x18, 0x69, 0xe5, 0x2d, 0x1f, 0xae, - 0xa3, 0x5d, 0x00, 0xff, 0x75, 0xd2, 0xa5, 0x43, 0xd7, 0xe9, 0x6d, 0xd2, 0x81, 0x20, 0x0c, 0xb6, - 0xa4, 0x84, 0xb8, 0xd1, 0x33, 0xc8, 0x89, 0x10, 0xc9, 0x1a, 0x98, 0xe9, 0xb1, 0x64, 0x3c, 0x60, - 0x8b, 0x30, 0xed, 0x4b, 0x20, 0xb5, 0x9b, 0x89, 0x35, 0x17, 0xf3, 0x2e, 0xe6, 0xc9, 0x8c, 0xb5, - 0x1b, 0x73, 0xae, 0xda, 0x89, 0x66, 0xf5, 0xf0, 0x3e, 0xe1, 0xdf, 0x8a, 0x97, 0x25, 0x85, 0x89, - 0x42, 0x2d, 0x90, 0xa9, 0xca, 0xc2, 0xf1, 0x5f, 0xa6, 0x5a, 0x7b, 0x57, 0x38, 0x2e, 0xd1, 0xda, - 0xd4, 0x1c, 0x40, 0x6d, 0xea, 0x59, 0x90, 0x07, 0x7e, 0x07, 0x4a, 0x5d, 0xdb, 0xe9, 0x6b, 0x9e, - 0x2a, 0x9c, 0x67, 0x3e, 0x98, 0x77, 0x7c, 0x77, 0xbe, 0x52, 0xdc, 0xa6, 0x58, 0xe1, 0x38, 0xc5, - 0x6e, 0xf8, 0x13, 0xed, 0x88, 0x74, 0x79, 0x8b, 0x66, 0xb7, 0x0f, 0x67, 0x3d, 0xe1, 0x78, 0xae, - 0x6c, 0x41, 0x9a, 0x16, 0xa7, 0x6e, 0x79, 0x81, 0xea, 0xfd, 0x9a, 0x85, 0xae, 0xc2, 0xa5, 0xa0, - 0x7d, 0x28, 0xe9, 0x04, 0x42, 0x3a, 0x27, 0x36, 0x4f, 0xb9, 0x4d, 0xe5, 0xae, 0x4c, 0x91, 0x2b, - 0x02, 0xad, 0x68, 0x9b, 0x05, 0x33, 0x9b, 0xb9, 0xb4, 0x21, 0xdb, 0xd5, 0xfa, 0x86, 0x69, 0x60, - 0xb7, 0xbc, 0x48, 0xe5, 0x7c, 0x74, 0xa9, 0x57, 0x8f, 0xce, 0xd6, 0x45, 0x72, 0x15, 0x42, 0x7c, - 0xe7, 0xa6, 0x80, 0x33, 0x72, 0x89, 0x77, 0xc6, 0x9d, 0x5b, 0xcc, 0xd6, 0x23, 0x73, 0x76, 0xea, - 0xdc, 0xfc, 0x4b, 0x47, 0x0f, 0x00, 0x5e, 0x1b, 0xf8, 0x8d, 0xfa, 0x6a, 0x88, 0x9d, 0xb3, 0x72, - 0x39, 0xdc, 0x26, 0x11, 0xf8, 0xe7, 0x04, 0x8c, 0x3e, 0x86, 0x9c, 0x8e, 0x07, 0xd8, 0xd2, 0xdd, - 0xb6, 0x55, 0xbe, 0x4b, 0x8b, 0xd2, 0x5b, 0x17, 0xe7, 0x2b, 0xb9, 0xba, 0x00, 0xf2, 0x58, 0x1a, - 0x50, 0xa1, 0x2f, 0xa1, 0xc0, 0x3e, 0xb0, 0xde, 0xb6, 0xb6, 0xce, 0xca, 0x4b, 0xf4, 0xd0, 0x8f, - 0x66, 0xbc, 0x94, 0x60, 0x08, 0xe1, 0xcf, 0x6d, 0xeb, 0x21, 0x69, 0x4a, 0x44, 0x36, 0xfa, 0x03, - 0x28, 0x08, 0x6b, 0xde, 0xb5, 0x8f, 0xdd, 0xf2, 0x3b, 0x97, 0x0e, 0x55, 0x47, 0xd7, 0x7a, 0x16, - 0xb0, 0x8a, 0x58, 0x15, 0x96, 0x86, 0x7e, 0x0e, 0x45, 0xff, 0x0d, 0xc7, 0x1e, 0x78, 0x6e, 0xf9, - 0x1e, 0x75, 0xc6, 0xc7, 0xb3, 0x9a, 0x2a, 0xe7, 0x6d, 0x0f, 0xe8, 0xbc, 0x39, 0xf4, 0x85, 0xee, - 0x43, 0x4e, 0x77, 0xec, 0x01, 0xcb, 0x19, 0x3f, 0x58, 0x95, 0xd6, 0x12, 0x7e, 0x27, 0xec, 0xd8, - 0x03, 0x9a, 0x0c, 0x54, 0x28, 0x39, 0x78, 0x60, 0x6a, 0x1d, 0xdc, 0x27, 0xd9, 0xcc, 0xee, 0x96, - 0x97, 0xe9, 0xea, 0x9b, 0x33, 0x2b, 0xd2, 0x67, 0x16, 0x86, 0x19, 0x92, 0xd7, 0xee, 0xa2, 0xe7, - 0x00, 0xda, 0x50, 0x37, 0x3c, 0xb5, 0x6f, 0xeb, 0xb8, 0xbc, 0x72, 0xe9, 0xc3, 0xe2, 0xa8, 0xf0, - 0x2a, 0x61, 0x7c, 0x66, 0xeb, 0xd8, 0x7f, 0xa1, 0x10, 0x00, 0xf4, 0x31, 0xe4, 0xe9, 0xd1, 0xbe, - 0xb4, 0x8f, 0x89, 0x6d, 0xae, 0xd2, 0xc3, 0xcd, 0xf3, 0xbb, 0xcc, 0xd5, 0x1d, 0x7b, 0xb0, 0x6b, - 0x1f, 0x53, 0x8b, 0xe1, 0x7f, 0xea, 0xc8, 0x85, 0x42, 0xaf, 0xa3, 0x06, 0xe1, 0xf3, 0x3e, 0xbd, - 0xc5, 0x4f, 0x67, 0xdc, 0xcb, 0xd3, 0xda, 0x84, 0x80, 0x7a, 0x4b, 0xe4, 0x81, 0xa7, 0x35, 0x01, - 0x73, 0x95, 0x7c, 0xaf, 0xe3, 0x7f, 0x90, 0xbe, 0x83, 0x8d, 0x5b, 0xb8, 0x03, 0x54, 0xc2, 0x7d, - 0x07, 0xc3, 0x30, 0x17, 0x68, 0x01, 0x9f, 0xcb, 0xa8, 0x9a, 0xab, 0xda, 0x5d, 0x76, 0x67, 0x0f, - 0x66, 0xcf, 0xf3, 0x25, 0xc6, 0x5d, 0x75, 0xdb, 0x5d, 0x7a, 0xb1, 0x1d, 0x28, 0xd8, 0x43, 0xef, - 0xd8, 0x1e, 0x5a, 0xba, 0xda, 0x7d, 0xe9, 0x96, 0xdf, 0xa5, 0xa7, 0xbd, 0x52, 0x0f, 0xed, 0x9f, - 0xae, 0xcd, 0x05, 0x6d, 0xef, 0xb9, 0x4a, 0x5e, 0x48, 0xdd, 0x7e, 0xe9, 0xa2, 0x3f, 0x86, 0xbc, - 0x61, 0x05, 0x6b, 0x3c, 0xbc, 0xfa, 0x1a, 0x48, 0xf4, 0x04, 0x4d, 0xcb, 0x5f, 0x02, 0xb8, 0x4c, - 0xb2, 0xc2, 0x07, 0x50, 0xb2, 0xbb, 0x5d, 0xd3, 0xb0, 0xb0, 0xea, 0x60, 0xcd, 0xb5, 0xad, 0xf2, - 0x7b, 0x21, 0x0d, 0x16, 0x39, 0x4e, 0xa1, 0x28, 0x54, 0x81, 0x9c, 0x87, 0xfb, 0x03, 0xdb, 0xd1, - 0x9c, 0xb3, 0xf2, 0xfb, 0xe1, 0x87, 0x1d, 0x1f, 0x8c, 0x8e, 0x61, 0x69, 0x68, 0xe1, 0xd3, 0x81, - 0xed, 0x62, 0x5d, 0x1d, 0xab, 0x30, 0xd7, 0x68, 0x8c, 0x7b, 0xc8, 0x37, 0x75, 0xe7, 0xb9, 0xa0, - 0x9c, 0x58, 0x6a, 0xde, 0x19, 0x4e, 0x44, 0xeb, 0x4b, 0xbf, 0x92, 0x60, 0x7e, 0x2c, 0x47, 0xa2, - 0x3f, 0x82, 0x8c, 0x65, 0xeb, 0xa1, 0x67, 0xb4, 0x06, 0x5f, 0x26, 0xdd, 0xb2, 0x75, 0xf6, 0x8a, - 0xf6, 0xb8, 0x67, 0x78, 0x27, 0xc3, 0xe3, 0xf5, 0x8e, 0xdd, 0xdf, 0xf0, 0x95, 0xa8, 0x1f, 0x07, - 0x7f, 0x6f, 0x0c, 0x5e, 0xf6, 0x36, 0xe8, 0x5f, 0x83, 0xe3, 0x75, 0xc6, 0xa6, 0xa4, 0x89, 0xd4, - 0xa6, 0x8e, 0x3e, 0x82, 0x39, 0x7c, 0x3a, 0x30, 0x9c, 0x50, 0x9d, 0x18, 0x0f, 0xf9, 0x7c, 0x29, - 0x40, 0x12, 0x03, 0xe1, 0x0f, 0x1d, 0xff, 0x10, 0x87, 0xb9, 0x91, 0x0c, 0x45, 0x0a, 0x63, 0xda, - 0x23, 0x47, 0x0a, 0x63, 0x02, 0xb9, 0xa4, 0x5c, 0x0f, 0xff, 0x66, 0x22, 0x71, 0xd3, 0xdf, 0x4c, - 0x44, 0x1f, 0x10, 0x52, 0x57, 0x78, 0x40, 0xf8, 0x29, 0x2c, 0x1a, 0xae, 0x6a, 0xd9, 0x96, 0x98, - 0xf3, 0xf8, 0xed, 0x61, 0xf8, 0x65, 0xfe, 0x96, 0xe1, 0xb6, 0x6c, 0x8b, 0x4d, 0x78, 0xfc, 0x53, - 0x07, 0x8f, 0xf8, 0x99, 0xf1, 0x47, 0x7c, 0x7f, 0x8e, 0x93, 0x94, 0x53, 0x4b, 0x5f, 0x4b, 0x90, - 0x0b, 0xff, 0x12, 0x2c, 0x1e, 0x1d, 0x5d, 0x8c, 0x35, 0x0b, 0xd7, 0x7c, 0x0e, 0x8d, 0x6a, 0x21, - 0x31, 0xbb, 0x16, 0xf8, 0xd5, 0xfe, 0x09, 0xe4, 0x43, 0xa9, 0x67, 0xb4, 0x25, 0x97, 0xae, 0xd1, - 0x92, 0xbf, 0x0b, 0x69, 0x1e, 0x6f, 0x99, 0x61, 0x15, 0x39, 0x77, 0x8a, 0xc5, 0xda, 0xd4, 0x97, - 0x24, 0xce, 0xf2, 0xd5, 0xff, 0x2d, 0x01, 0x85, 0x70, 0x6a, 0x22, 0xce, 0x69, 0x58, 0x1d, 0x87, - 0xe6, 0x05, 0xba, 0x7a, 0xc2, 0x7f, 0x75, 0x15, 0x60, 0x92, 0xb0, 0xfa, 0x86, 0xa5, 0xd2, 0x97, - 0xbe, 0x88, 0xf1, 0x66, 0xfb, 0x86, 0xf5, 0x05, 0x81, 0x52, 0x12, 0xed, 0x94, 0x93, 0x24, 0x22, - 0x24, 0xda, 0x29, 0x23, 0x59, 0xa2, 0x35, 0x9f, 0xe3, 0xd1, 0xc6, 0x2c, 0x11, 0xaa, 0xe2, 0x1c, - 0x0f, 0x2d, 0x43, 0xe6, 0xb5, 0xe1, 0x78, 0x43, 0xcd, 0xa4, 0x3d, 0x98, 0xb8, 0x66, 0x01, 0x44, - 0x16, 0x94, 0x82, 0x64, 0xfc, 0xc6, 0xc2, 0x0e, 0x35, 0x9c, 0xfc, 0x66, 0xf5, 0x1a, 0xd9, 0x38, - 0xf8, 0x20, 0x82, 0x44, 0xc8, 0x72, 0xc3, 0xc0, 0xa5, 0xbf, 0x96, 0xa0, 0x18, 0x21, 0x43, 0x4d, - 0x98, 0xa3, 0x0b, 0x8f, 0xcd, 0xc5, 0xee, 0xfb, 0xbf, 0xe9, 0x22, 0xe8, 0x89, 0x7d, 0x55, 0xd1, - 0x0e, 0xa1, 0x74, 0xf4, 0x19, 0x94, 0x98, 0x28, 0xff, 0xfd, 0x3e, 0x6a, 0x7e, 0x05, 0x2a, 0x29, - 0xfa, 0x88, 0x5f, 0xb0, 0x03, 0x98, 0x1e, 0x7e, 0x9a, 0x5c, 0xb2, 0x20, 0x1f, 0xca, 0xf6, 0x33, - 0xd8, 0xfd, 0x4f, 0x20, 0xe9, 0x47, 0xa1, 0x19, 0xb3, 0x18, 0x65, 0xe0, 0xeb, 0x7d, 0x25, 0xc1, - 0xc2, 0xa4, 0xac, 0x1b, 0xf1, 0x27, 0x66, 0x48, 0x33, 0xf9, 0xd3, 0x83, 0x70, 0x35, 0xc4, 0x8c, - 0x4b, 0x3c, 0x97, 0x05, 0xf5, 0xd0, 0x7b, 0xbe, 0x89, 0x33, 0xdb, 0x9a, 0x8b, 0x98, 0x38, 0xe9, - 0x72, 0x42, 0x46, 0x5e, 0x79, 0x2c, 0xc6, 0x30, 0x00, 0xe9, 0x83, 0xe7, 0x5b, 0xfb, 0xcd, 0xda, - 0xc4, 0x11, 0x0a, 0xca, 0x43, 0xa6, 0xbd, 0xbd, 0xbd, 0xdf, 0x6c, 0x35, 0xe4, 0x44, 0x65, 0x0d, - 0x72, 0x7e, 0x61, 0x83, 0x0a, 0x90, 0xad, 0x37, 0x0f, 0xab, 0x5b, 0xfb, 0x8d, 0xba, 0x1c, 0x43, - 0x45, 0xc8, 0x29, 0x8d, 0x6a, 0x9d, 0x0e, 0x6a, 0x64, 0xe9, 0x93, 0xec, 0x2f, 0xbf, 0x5a, 0x91, - 0x78, 0xe0, 0x49, 0xcb, 0x99, 0xdd, 0x64, 0x16, 0xc9, 0xb7, 0x2a, 0xff, 0x1b, 0x07, 0x54, 0xd7, - 0x3c, 0x8d, 0xd8, 0xdf, 0x15, 0xc6, 0x19, 0xf1, 0x4b, 0x6e, 0x2a, 0xda, 0xa2, 0x26, 0x6e, 0xd4, - 0xa2, 0x4e, 0x1c, 0x58, 0x24, 0x6f, 0x32, 0xb0, 0xb8, 0xd6, 0xdc, 0x64, 0xbc, 0xbd, 0x4a, 0x5f, - 0xbf, 0xbd, 0x0a, 0x6e, 0xa2, 0x72, 0x91, 0x86, 0xd2, 0xd1, 0xd9, 0x20, 0xac, 0xf9, 0x6b, 0x4d, - 0xbe, 0x26, 0xcd, 0xb7, 0xe2, 0x57, 0x9f, 0x6f, 0x5d, 0xf2, 0xfb, 0x61, 0x76, 0xed, 0xc9, 0x4b, - 0xae, 0xbd, 0x0e, 0xc9, 0x97, 0x86, 0xc5, 0xc6, 0xb3, 0xa5, 0xa9, 0x17, 0x1e, 0x3d, 0xed, 0xfa, - 0x9e, 0x61, 0xe9, 0x62, 0x1d, 0xc2, 0x8d, 0x7e, 0x01, 0x05, 0x6c, 0x0d, 0xfb, 0x6a, 0x1f, 0xf7, - 0x8f, 0xb1, 0x23, 0xf4, 0xfc, 0x68, 0x36, 0x69, 0x0d, 0x6b, 0xd8, 0x7f, 0x46, 0x19, 0x45, 0x51, - 0x8c, 0x7d, 0x88, 0x8b, 0x1e, 0x41, 0x4a, 0x33, 0x0d, 0xcd, 0xe5, 0x03, 0xaf, 0xcb, 0x7e, 0xce, - 0xc7, 0x08, 0xd1, 0xef, 0x42, 0x51, 0x73, 0x1c, 0xed, 0x8c, 0xff, 0x2c, 0x4d, 0xa7, 0x63, 0x61, - 0x6e, 0x31, 0xa4, 0x94, 0xad, 0x12, 0x24, 0xfd, 0x25, 0x9a, 0x50, 0x44, 0x5e, 0xf3, 0x41, 0x91, - 0x11, 0x5d, 0xee, 0x66, 0x23, 0x3a, 0xb8, 0x89, 0xc5, 0x8f, 0x1b, 0x6f, 0xfe, 0xfa, 0xc6, 0xbb, - 0xf4, 0x4b, 0x09, 0x20, 0xd0, 0x33, 0xfa, 0x09, 0xdc, 0x19, 0x9c, 0x9c, 0xb9, 0xf4, 0x57, 0x80, - 0x0e, 0x1e, 0x38, 0xd8, 0xc5, 0x16, 0x8b, 0xac, 0xd4, 0x78, 0x0b, 0xca, 0xa2, 0x40, 0x2b, 0x11, - 0x2c, 0xfa, 0x14, 0x16, 0xc5, 0xaf, 0x07, 0x47, 0xf8, 0xc2, 0xf5, 0xe0, 0x6d, 0x4e, 0x13, 0x65, - 0xe6, 0x01, 0xf3, 0x1d, 0x48, 0x12, 0xfb, 0x21, 0xa1, 0xb1, 0xd1, 0x7a, 0xfe, 0x4c, 0x8e, 0xa1, - 0x1c, 0xa4, 0xaa, 0xfb, 0xcd, 0xea, 0x61, 0x38, 0xdc, 0x55, 0xfe, 0x2e, 0x01, 0x32, 0xb3, 0xf0, - 0x9b, 0xba, 0xd9, 0xf4, 0x8a, 0xf5, 0xfb, 0x87, 0xbc, 0xd1, 0xa8, 0x98, 0x7c, 0xfb, 0x51, 0x31, - 0xf5, 0x96, 0xa2, 0x62, 0xfa, 0x06, 0x51, 0x31, 0xf3, 0x56, 0xa2, 0xe2, 0xd7, 0x71, 0x80, 0xd0, - 0x55, 0xfd, 0x2c, 0xfc, 0xbf, 0x23, 0xa6, 0x0f, 0x5f, 0x47, 0x8a, 0xa7, 0x9d, 0x98, 0xf8, 0xbf, - 0x13, 0x4f, 0x21, 0xab, 0xf3, 0x0c, 0xc7, 0xab, 0x87, 0xa9, 0x53, 0xce, 0xb1, 0x44, 0xb8, 0x13, - 0x53, 0x7c, 0x66, 0xf4, 0x69, 0xe4, 0xe7, 0xc0, 0x0f, 0x67, 0x8a, 0x49, 0x3b, 0xe2, 0xa7, 0x14, - 0x55, 0x48, 0xb3, 0xd8, 0xcc, 0xef, 0x7e, 0xda, 0x9c, 0x7c, 0xd4, 0x52, 0x77, 0x62, 0x0a, 0x67, - 0xe4, 0x6f, 0x27, 0x19, 0x48, 0x0d, 0x2d, 0xc3, 0xb6, 0x7e, 0xa4, 0x84, 0x1f, 0xf1, 0x45, 0xfb, - 0x43, 0xb2, 0x3e, 0xfd, 0x5b, 0xf3, 0xb0, 0xce, 0x5e, 0x71, 0x9e, 0x5b, 0xaf, 0x7d, 0x80, 0x84, - 0x4a, 0x00, 0x1c, 0x6f, 0x58, 0x3d, 0x39, 0x4e, 0x6b, 0x06, 0xc7, 0x1e, 0x0c, 0xc8, 0x57, 0x62, - 0xeb, 0x87, 0xdf, 0xfc, 0xe7, 0x72, 0xec, 0x9b, 0x8b, 0x65, 0xe9, 0xd7, 0x17, 0xcb, 0xd2, 0x6f, - 0x2e, 0x96, 0xa5, 0xff, 0xb8, 0x58, 0x96, 0xfe, 0xfc, 0xdb, 0xe5, 0xd8, 0xaf, 0xbf, 0x5d, 0x8e, - 0xfd, 0xe6, 0xdb, 0xe5, 0xd8, 0xef, 0x65, 0xf8, 0x46, 0xff, 0x2f, 0x00, 0x00, 0xff, 0xff, 0x83, - 0xb6, 0xca, 0xed, 0x81, 0x33, 0x00, 0x00, + proto.RegisterFile("sql/sqlbase/structured.proto", fileDescriptor_structured_994b169dcbedf9d2) +} + +var fileDescriptor_structured_994b169dcbedf9d2 = []byte{ + // 4272 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x7b, 0x49, 0x6c, 0x1b, 0x59, + 0x7a, 0x30, 0xf7, 0xe5, 0xe3, 0x56, 0x7a, 0x96, 0x65, 0x5a, 0xed, 0x96, 0x64, 0xba, 0xdd, 0xad, + 0xe9, 0x45, 0x72, 0xcb, 0xf3, 0xff, 0xe3, 0xe9, 0x4e, 0x06, 0x4d, 0x91, 0x94, 0x45, 0x2d, 0xa4, + 0xba, 0x24, 0xdb, 0x33, 0x99, 0x24, 0x95, 0x12, 0xeb, 0x91, 0xaa, 0x76, 0xb1, 0x8a, 0xae, 0x2a, + 0xda, 0x12, 0x90, 0x53, 0x4e, 0x73, 0x0a, 0x72, 0xc9, 0x31, 0x40, 0x23, 0x69, 0x20, 0x73, 0x4a, + 0x90, 0x4b, 0x72, 0x0b, 0x90, 0x5b, 0xe7, 0x94, 0xc9, 0x6d, 0x4e, 0x42, 0xa2, 0xbe, 0xe4, 0x1e, + 0x20, 0x01, 0x3a, 0x97, 0xe0, 0x6d, 0xb5, 0x70, 0x51, 0x53, 0x92, 0x73, 0x11, 0x58, 0xdf, 0xf6, + 0xb6, 0x6f, 0x7f, 0x4f, 0x70, 0xcf, 0x79, 0x65, 0xac, 0x3b, 0xaf, 0x8c, 0x63, 0xd5, 0xc1, 0xeb, + 0x8e, 0x6b, 0x0f, 0x3b, 0xee, 0xd0, 0xc6, 0xda, 0xda, 0xc0, 0xb6, 0x5c, 0x0b, 0xdd, 0xee, 0x58, + 0x9d, 0x97, 0xb6, 0xa5, 0x76, 0x4e, 0xd6, 0x9c, 0x57, 0xc6, 0x1a, 0xa7, 0x5b, 0x2c, 0x0f, 0x5d, + 0xdd, 0x58, 0x3f, 0x31, 0x3a, 0xeb, 0xae, 0xde, 0xc7, 0x8e, 0xab, 0xf6, 0x07, 0x8c, 0x61, 0xf1, + 0x9d, 0xa0, 0xb8, 0x81, 0xad, 0xbf, 0xd6, 0x0d, 0xdc, 0xc3, 0x1c, 0x79, 0x9b, 0x20, 0xdd, 0xb3, + 0x01, 0x76, 0xd8, 0x5f, 0x0e, 0xbe, 0xdb, 0xc3, 0xd6, 0x7a, 0x0f, 0x5b, 0xba, 0xa9, 0xe1, 0xd3, + 0xf5, 0x8e, 0x65, 0x76, 0xf5, 0x1e, 0x47, 0xcd, 0xf7, 0xac, 0x9e, 0x45, 0x7f, 0xae, 0x93, 0x5f, + 0x0c, 0x5a, 0xf9, 0x93, 0x24, 0xdc, 0xda, 0xb2, 0x6c, 0xac, 0xf7, 0xcc, 0x5d, 0x7c, 0x26, 0xe3, + 0x2e, 0xb6, 0xb1, 0xd9, 0xc1, 0x68, 0x05, 0x92, 0xae, 0x7a, 0x6c, 0xe0, 0x72, 0x74, 0x25, 0xba, + 0x5a, 0xd8, 0x84, 0x6f, 0xcf, 0x97, 0x23, 0xdf, 0x9f, 0x2f, 0xc7, 0x9a, 0x75, 0x99, 0x21, 0xd0, + 0x43, 0x48, 0xd2, 0x51, 0xca, 0x31, 0x4a, 0x51, 0xe2, 0x14, 0xe9, 0x26, 0x01, 0x12, 0x32, 0x8a, + 0x45, 0x65, 0x48, 0x98, 0x6a, 0x1f, 0x97, 0xe3, 0x2b, 0xd1, 0xd5, 0xec, 0x66, 0x82, 0x50, 0xc9, + 0x14, 0x82, 0x76, 0x21, 0xf3, 0x5a, 0x35, 0x74, 0x4d, 0x77, 0xcf, 0xca, 0x89, 0x95, 0xe8, 0x6a, + 0x71, 0xe3, 0x47, 0x6b, 0x13, 0xf7, 0x68, 0xad, 0x66, 0x99, 0x8e, 0x6b, 0xab, 0xba, 0xe9, 0x3e, + 0xe7, 0x0c, 0x5c, 0x90, 0x27, 0x00, 0x3d, 0x82, 0x39, 0xe7, 0x44, 0xb5, 0xb1, 0xa6, 0x0c, 0x6c, + 0xdc, 0xd5, 0x4f, 0x15, 0x03, 0x9b, 0xe5, 0xe4, 0x4a, 0x74, 0x35, 0xc9, 0x49, 0x4b, 0x0c, 0x7d, + 0x40, 0xb1, 0x7b, 0xd8, 0x44, 0x47, 0x90, 0xb5, 0x4c, 0x45, 0xc3, 0x06, 0x76, 0x71, 0x39, 0x45, + 0xc7, 0xff, 0x74, 0xca, 0xf8, 0x13, 0x36, 0x68, 0xad, 0xda, 0x71, 0x75, 0xcb, 0x14, 0xf3, 0xb0, + 0xcc, 0x3a, 0x15, 0xc4, 0xa5, 0x0e, 0x07, 0x9a, 0xea, 0xe2, 0x72, 0xfa, 0xc6, 0x52, 0x9f, 0x51, + 0x41, 0x68, 0x0f, 0x92, 0x7d, 0xd5, 0xed, 0x9c, 0x94, 0x33, 0x54, 0xe2, 0xa3, 0x2b, 0x48, 0xdc, + 0x27, 0x7c, 0x5c, 0x20, 0x13, 0x52, 0x79, 0x01, 0x29, 0x36, 0x0e, 0x2a, 0x40, 0xb6, 0xd5, 0x56, + 0xaa, 0xb5, 0xa3, 0x66, 0xbb, 0x25, 0x45, 0x50, 0x1e, 0x32, 0x72, 0xe3, 0xf0, 0x48, 0x6e, 0xd6, + 0x8e, 0xa4, 0x28, 0xf9, 0x3a, 0x6c, 0x1c, 0x29, 0xad, 0x67, 0x7b, 0x7b, 0x52, 0x0c, 0x95, 0x20, + 0x47, 0xbe, 0xea, 0x8d, 0xad, 0xea, 0xb3, 0xbd, 0x23, 0x29, 0x8e, 0x72, 0x90, 0xae, 0x55, 0x0f, + 0x6b, 0xd5, 0x7a, 0x43, 0x4a, 0x2c, 0x26, 0x7e, 0xfd, 0xcd, 0x52, 0xa4, 0xf2, 0x08, 0x92, 0x74, + 0x38, 0x04, 0x90, 0x3a, 0x6c, 0xee, 0x1f, 0xec, 0x35, 0xa4, 0x08, 0xca, 0x40, 0x62, 0x8b, 0x88, + 0x88, 0x12, 0x8e, 0x83, 0xaa, 0x7c, 0xd4, 0xac, 0xee, 0x49, 0x31, 0xc6, 0xf1, 0x59, 0xe2, 0x3f, + 0xbe, 0x5e, 0x8e, 0x56, 0xfe, 0x35, 0x09, 0xf3, 0xfe, 0xdc, 0xfd, 0xd3, 0x46, 0x35, 0x28, 0x59, + 0xb6, 0xde, 0xd3, 0x4d, 0x85, 0xea, 0x9c, 0xa2, 0x6b, 0x5c, 0x1f, 0xdf, 0x21, 0xeb, 0xb9, 0x38, + 0x5f, 0x2e, 0xb4, 0x29, 0xfa, 0x88, 0x60, 0x9b, 0x75, 0xae, 0xa0, 0x05, 0x2b, 0x00, 0xd4, 0xd0, + 0x2e, 0xcc, 0x71, 0x21, 0x1d, 0xcb, 0x18, 0xf6, 0x4d, 0x45, 0xd7, 0x9c, 0x72, 0x6c, 0x25, 0xbe, + 0x5a, 0xd8, 0x5c, 0xbe, 0x38, 0x5f, 0x2e, 0x31, 0x11, 0x35, 0x8a, 0x6b, 0xd6, 0x9d, 0xef, 0xcf, + 0x97, 0x33, 0xe2, 0x43, 0xe6, 0xc3, 0xf3, 0x6f, 0xcd, 0x41, 0x2f, 0xe0, 0xb6, 0x2d, 0xf6, 0x56, + 0x0b, 0x0a, 0x8c, 0x53, 0x81, 0x0f, 0x2e, 0xce, 0x97, 0x6f, 0x79, 0x9b, 0xaf, 0x4d, 0x16, 0x7a, + 0xcb, 0x1e, 0x25, 0xd0, 0x1c, 0xd4, 0x86, 0x00, 0xd8, 0x5f, 0x6e, 0x82, 0x2e, 0x77, 0x99, 0x2f, + 0x77, 0xce, 0x17, 0x1d, 0x5e, 0xf2, 0x9c, 0x3d, 0x82, 0xd0, 0x3c, 0xc3, 0x4b, 0x5e, 0x6a, 0x78, + 0xa9, 0x9b, 0x1a, 0x5e, 0xc8, 0x8c, 0xd2, 0xff, 0x27, 0x66, 0x94, 0x79, 0xeb, 0x66, 0x94, 0x7d, + 0x0b, 0x66, 0xc4, 0x74, 0x77, 0x27, 0x91, 0x01, 0x29, 0xb7, 0x93, 0xc8, 0xe4, 0xa4, 0xfc, 0x4e, + 0x22, 0x93, 0x97, 0x0a, 0x3b, 0x89, 0x4c, 0x41, 0x2a, 0x56, 0xfe, 0x33, 0x01, 0x12, 0x3b, 0xdd, + 0x3a, 0x76, 0x3a, 0xb6, 0x3e, 0x70, 0x2d, 0xdb, 0x3b, 0x93, 0xe8, 0xd8, 0x99, 0xbc, 0x0f, 0x31, + 0x5d, 0xe3, 0xae, 0x74, 0x81, 0x9f, 0x76, 0x8c, 0x1e, 0xaf, 0xaf, 0x37, 0x31, 0x5d, 0x43, 0x6b, + 0x90, 0x20, 0xfe, 0x9e, 0xba, 0xd3, 0xdc, 0xc6, 0xe2, 0xe8, 0x0a, 0x70, 0x7f, 0x8d, 0x85, 0x83, + 0x23, 0x99, 0xd2, 0xa1, 0x15, 0xc8, 0x98, 0x43, 0xc3, 0xa0, 0xae, 0x9c, 0xe8, 0x52, 0x46, 0x6c, + 0x8a, 0x80, 0xa2, 0xfb, 0x90, 0xd7, 0x70, 0x57, 0x1d, 0x1a, 0xae, 0x82, 0x4f, 0x07, 0x36, 0xd3, + 0x17, 0x39, 0xc7, 0x61, 0x8d, 0xd3, 0x81, 0x8d, 0xee, 0x41, 0xea, 0x44, 0xd7, 0x34, 0x6c, 0x52, + 0x75, 0x11, 0x22, 0x38, 0x0c, 0x6d, 0xc0, 0xdc, 0xd0, 0xc1, 0x8e, 0xe2, 0xe0, 0x57, 0x43, 0xb2, + 0x57, 0xd4, 0x1c, 0x80, 0x9a, 0x43, 0x8a, 0xab, 0x67, 0x89, 0x10, 0x1c, 0x72, 0x3c, 0xd1, 0xf6, + 0xfb, 0x90, 0xef, 0x58, 0xfd, 0xc1, 0xd0, 0xc5, 0x6c, 0xd0, 0x1c, 0x1b, 0x94, 0xc3, 0xe8, 0xa0, + 0x1b, 0x30, 0x67, 0xbd, 0x31, 0x47, 0xc4, 0xe6, 0xc3, 0x62, 0x09, 0x41, 0x50, 0xec, 0x26, 0x80, + 0x61, 0xf5, 0xf4, 0x8e, 0x6a, 0x10, 0xdb, 0x29, 0xd0, 0xdd, 0x7c, 0xc0, 0x77, 0xb3, 0xb4, 0xc7, + 0x30, 0x62, 0x3b, 0x43, 0x5b, 0x9b, 0xe5, 0x6c, 0x4d, 0x0d, 0x6d, 0xc1, 0xbb, 0xaa, 0xe1, 0x62, + 0x5b, 0x18, 0x37, 0xd9, 0x46, 0x45, 0x37, 0x95, 0x81, 0x6d, 0xf5, 0x6c, 0xec, 0x38, 0xe5, 0x62, + 0x60, 0x0f, 0xee, 0x52, 0x52, 0x26, 0xe6, 0xe8, 0x6c, 0x80, 0x9b, 0xe6, 0x01, 0x27, 0x43, 0xbf, + 0x04, 0xe4, 0x9c, 0x39, 0x2e, 0xee, 0x0b, 0x41, 0x2f, 0x75, 0x53, 0x2b, 0x97, 0xa8, 0xe6, 0x7d, + 0x30, 0x45, 0xf3, 0x0e, 0x29, 0x03, 0x13, 0xb7, 0xab, 0x9b, 0x1a, 0x1f, 0x45, 0x72, 0x46, 0xe0, + 0x9e, 0xee, 0x65, 0xa4, 0xec, 0x4e, 0x22, 0x93, 0x95, 0x60, 0x27, 0x91, 0x49, 0x4b, 0x99, 0xca, + 0x9f, 0xc6, 0x60, 0x81, 0x91, 0x6d, 0xa9, 0x7d, 0xdd, 0x38, 0xbb, 0xa9, 0xee, 0x31, 0x29, 0x5c, + 0xf7, 0xe8, 0xa1, 0xd1, 0xa5, 0x10, 0x36, 0xe6, 0xf2, 0xe8, 0xa1, 0x11, 0x58, 0x8b, 0x80, 0xd0, + 0x13, 0x80, 0x80, 0x4f, 0x4c, 0xd0, 0xd3, 0xba, 0x7b, 0x71, 0xbe, 0x9c, 0x9d, 0xec, 0x09, 0xb3, + 0x9d, 0x80, 0xff, 0x9b, 0x13, 0x6a, 0xe8, 0x49, 0xa0, 0xba, 0x18, 0x38, 0xc1, 0x3a, 0x23, 0x98, + 0x78, 0x82, 0x25, 0x2d, 0x84, 0xe4, 0x5b, 0x54, 0xf9, 0x87, 0x18, 0xcc, 0x37, 0x4d, 0x17, 0xdb, + 0x06, 0x56, 0x5f, 0xe3, 0xc0, 0x76, 0xfc, 0x1c, 0xb2, 0xaa, 0xd9, 0xc1, 0x8e, 0x6b, 0xd9, 0x4e, + 0x39, 0xba, 0x12, 0x5f, 0xcd, 0x6d, 0xfc, 0x78, 0xca, 0xa9, 0x4c, 0xe2, 0x5f, 0xab, 0x72, 0x66, + 0xbe, 0x93, 0xbe, 0xb0, 0xc5, 0x7f, 0x8c, 0x42, 0x46, 0x60, 0xd1, 0x23, 0xc8, 0x8c, 0x84, 0xae, + 0xdb, 0x7c, 0x35, 0xe9, 0xb0, 0x07, 0x4f, 0xbb, 0xdc, 0x6f, 0xff, 0x3f, 0xc8, 0xd0, 0xcc, 0x49, + 0xf1, 0xce, 0x64, 0x51, 0x70, 0xf0, 0xd4, 0x2a, 0x98, 0x65, 0xa5, 0x29, 0x6d, 0x53, 0x43, 0xb5, + 0x49, 0x09, 0x50, 0x9c, 0xf2, 0xdf, 0x11, 0xfb, 0x77, 0x18, 0x4e, 0x81, 0xc6, 0x72, 0x22, 0xb6, + 0x67, 0x7c, 0xe7, 0xfe, 0x3e, 0x0a, 0x73, 0x84, 0x41, 0xc3, 0x5a, 0x60, 0xdb, 0x1e, 0x00, 0xe8, + 0x8e, 0xe2, 0x30, 0x38, 0x5d, 0x91, 0x30, 0x85, 0xac, 0xee, 0x70, 0x72, 0x4f, 0xd5, 0x62, 0x63, + 0xaa, 0xf6, 0x53, 0x28, 0x50, 0x5e, 0xe5, 0x78, 0xd8, 0x79, 0x89, 0x5d, 0x87, 0xce, 0x30, 0xb9, + 0x39, 0xcf, 0x67, 0x98, 0xa7, 0x12, 0x36, 0x19, 0x4e, 0xce, 0x3b, 0x81, 0xaf, 0x31, 0xed, 0x4b, + 0x8c, 0x69, 0x1f, 0x9f, 0xf8, 0x7f, 0xc7, 0x61, 0xe1, 0x40, 0xb5, 0x5d, 0x9d, 0xc4, 0x00, 0xdd, + 0xec, 0x05, 0x66, 0xff, 0x10, 0x72, 0xe6, 0x50, 0x18, 0xa4, 0xc3, 0x0f, 0x84, 0xcd, 0x0f, 0xcc, + 0x21, 0x37, 0x30, 0x07, 0xed, 0x41, 0xc2, 0xd0, 0x1d, 0x97, 0x26, 0x09, 0xb9, 0x8d, 0x8d, 0x29, + 0x6a, 0x31, 0x79, 0x8c, 0xb5, 0x3d, 0xdd, 0x71, 0xc5, 0x9a, 0x89, 0x14, 0xd4, 0x86, 0xa4, 0xad, + 0x9a, 0x3d, 0x4c, 0xed, 0x25, 0xb7, 0xf1, 0xf8, 0x6a, 0xe2, 0x64, 0xc2, 0x2a, 0x02, 0x0f, 0x95, + 0xb3, 0xf8, 0x17, 0x51, 0x48, 0x90, 0x51, 0x2e, 0x31, 0xe9, 0x05, 0x48, 0xbd, 0x56, 0x8d, 0x21, + 0x66, 0x89, 0x4e, 0x5e, 0xe6, 0x5f, 0xe8, 0x0f, 0xa0, 0xe4, 0x0c, 0x8f, 0x07, 0x81, 0xa1, 0x78, + 0x24, 0xf9, 0xe4, 0x4a, 0xb3, 0xf2, 0x72, 0xea, 0xb0, 0x2c, 0x76, 0x00, 0x8b, 0xaf, 0x20, 0x49, + 0x67, 0x7d, 0xc9, 0xfc, 0xee, 0x43, 0xde, 0xb5, 0x14, 0x7c, 0xda, 0x31, 0x86, 0x8e, 0xfe, 0x9a, + 0x69, 0x4a, 0x5e, 0xce, 0xb9, 0x56, 0x43, 0x80, 0xd0, 0x43, 0x28, 0x76, 0x6d, 0xab, 0xaf, 0xe8, + 0xa6, 0x20, 0x8a, 0x53, 0xa2, 0x02, 0x81, 0x36, 0x05, 0x30, 0xa4, 0xb2, 0x7f, 0x9e, 0x87, 0x12, + 0x35, 0x8c, 0x99, 0xdc, 0xde, 0xc3, 0x80, 0xdb, 0xbb, 0x1d, 0x72, 0x7b, 0x9e, 0x75, 0x11, 0xaf, + 0x77, 0x0f, 0x52, 0x43, 0x53, 0x7f, 0x35, 0x64, 0xe3, 0x7b, 0xc1, 0x8f, 0xc1, 0x66, 0xd0, 0x4a, + 0xf4, 0x31, 0x20, 0xe2, 0x0a, 0xb0, 0x12, 0x22, 0x4c, 0x52, 0x42, 0x89, 0x62, 0x6a, 0x53, 0x3d, + 0x68, 0xea, 0x0a, 0x1e, 0x74, 0x1b, 0x24, 0x7c, 0xea, 0xda, 0x6a, 0x30, 0x2b, 0x4d, 0x53, 0xfe, + 0xa5, 0x8b, 0xf3, 0xe5, 0x62, 0x83, 0xe0, 0x26, 0x0b, 0x29, 0xe2, 0x00, 0x4e, 0x23, 0x5a, 0x32, + 0xc7, 0x65, 0x68, 0xba, 0x8d, 0x69, 0x2e, 0xe5, 0x94, 0x33, 0x2b, 0xf1, 0x4b, 0x72, 0xa6, 0x91, + 0x6d, 0x5f, 0xab, 0x0b, 0x46, 0x59, 0x62, 0xa2, 0x3c, 0x80, 0x83, 0x0e, 0x21, 0xd7, 0x65, 0x29, + 0x96, 0xf2, 0x12, 0x9f, 0xd1, 0x64, 0x2c, 0xb7, 0xf1, 0xe1, 0xec, 0xc9, 0xd8, 0x66, 0x8a, 0x1c, + 0x41, 0x39, 0x2a, 0x43, 0xd7, 0x43, 0xa2, 0x17, 0x50, 0x08, 0xe4, 0xcf, 0xc7, 0x67, 0x34, 0x03, + 0xb9, 0x9e, 0xd8, 0xbc, 0x2f, 0x68, 0xf3, 0x0c, 0x7d, 0x09, 0xa0, 0x7b, 0x01, 0x80, 0x26, 0x2a, + 0xb9, 0x8d, 0x8f, 0xae, 0x10, 0x29, 0x84, 0x7f, 0xf1, 0x85, 0xa0, 0x17, 0x50, 0xf4, 0xbf, 0xe8, + 0x64, 0xf3, 0x57, 0x9e, 0x2c, 0x93, 0x5a, 0x08, 0xc8, 0xd9, 0x24, 0xc9, 0xf8, 0x3c, 0x49, 0xa1, + 0x2c, 0x47, 0x77, 0x71, 0x50, 0x0d, 0x0a, 0x54, 0x0d, 0x2a, 0x17, 0xe7, 0xcb, 0xa8, 0x26, 0xf0, + 0x93, 0x55, 0x01, 0x75, 0x46, 0xf0, 0x4c, 0xb1, 0x42, 0x0a, 0x4c, 0x24, 0x16, 0x7d, 0xc5, 0x3a, + 0xf4, 0x55, 0x78, 0x4c, 0xb1, 0x02, 0xea, 0xcd, 0xaa, 0xa7, 0x7c, 0xc8, 0xf7, 0x94, 0xae, 0xef, + 0x7b, 0x42, 0x82, 0x50, 0x83, 0xa7, 0xc5, 0x12, 0x4d, 0xaf, 0x3e, 0x9a, 0x51, 0x49, 0x49, 0xc6, + 0x26, 0x5c, 0x02, 0xcd, 0x96, 0x1f, 0x03, 0xea, 0xd8, 0x58, 0x75, 0xb1, 0x46, 0xd2, 0x52, 0x43, + 0xef, 0xe8, 0xae, 0x71, 0x56, 0x9e, 0x0b, 0xd8, 0xfd, 0x1c, 0xc7, 0x37, 0x3c, 0x34, 0x7a, 0x02, + 0xe9, 0xd7, 0xd8, 0x76, 0x74, 0xcb, 0x2c, 0x23, 0xea, 0x4c, 0x96, 0x78, 0x2b, 0x64, 0x61, 0x64, + 0xbc, 0xe7, 0x8c, 0x4a, 0x16, 0xe4, 0x68, 0x1b, 0x0a, 0xd8, 0xec, 0x58, 0x9a, 0x6e, 0xf6, 0x68, + 0x9a, 0x59, 0xbe, 0xe5, 0xe7, 0x3b, 0xdf, 0x9f, 0x2f, 0xbf, 0x33, 0xc2, 0xdf, 0xe0, 0xb4, 0x64, + 0xda, 0x72, 0x1e, 0x07, 0xbe, 0xd0, 0x36, 0xa4, 0x45, 0x4c, 0x9e, 0xa7, 0x7b, 0xba, 0x3a, 0x2d, + 0xc3, 0x1c, 0x8d, 0xe8, 0x7c, 0x5d, 0x82, 0x9d, 0x14, 0x0c, 0x9a, 0xee, 0x90, 0x5c, 0x44, 0x2b, + 0xdf, 0x0e, 0x16, 0x0c, 0x02, 0x8a, 0x6a, 0x00, 0x3d, 0x6c, 0x29, 0xac, 0xb9, 0x54, 0x5e, 0xa0, + 0xc3, 0x2d, 0x05, 0x86, 0xeb, 0x61, 0x6b, 0x4d, 0xb4, 0xa0, 0x48, 0x05, 0xd9, 0xd5, 0x7b, 0x22, + 0x45, 0xe8, 0x61, 0x8b, 0x01, 0x50, 0x05, 0xb2, 0x03, 0x1b, 0x6b, 0x7a, 0x87, 0x14, 0x78, 0x77, + 0x02, 0xbe, 0xd9, 0x07, 0x57, 0x96, 0x20, 0xeb, 0x79, 0x0d, 0x94, 0x86, 0x78, 0xf5, 0xb0, 0xc6, + 0xfa, 0x09, 0xf5, 0xc6, 0x61, 0x4d, 0x8a, 0x56, 0xee, 0x43, 0x82, 0x2e, 0x3e, 0x07, 0xe9, 0xad, + 0xb6, 0xfc, 0xa2, 0x2a, 0xd7, 0x59, 0x0f, 0xa3, 0xd9, 0x7a, 0xde, 0x90, 0x8f, 0x1a, 0x75, 0x49, + 0xc4, 0x85, 0x7f, 0x8a, 0x03, 0xf2, 0x4b, 0xd9, 0x23, 0x8b, 0x97, 0x83, 0x3d, 0x28, 0x75, 0x3c, + 0x28, 0x3b, 0x80, 0xe8, 0x4a, 0x6c, 0xb5, 0xb8, 0xf1, 0xe4, 0x07, 0xcb, 0x61, 0x21, 0x23, 0x08, + 0xf2, 0x95, 0xa9, 0xd8, 0x09, 0x41, 0x03, 0xf9, 0x50, 0x6c, 0x24, 0x06, 0xc9, 0x90, 0xec, 0x9c, + 0xe0, 0xce, 0x4b, 0x1e, 0x85, 0xff, 0xff, 0x94, 0x81, 0x69, 0xaa, 0x18, 0x50, 0xdc, 0x1a, 0xe1, + 0xf1, 0x87, 0x16, 0xe9, 0x01, 0x15, 0x85, 0xe4, 0xb0, 0x7b, 0x4d, 0x5c, 0xea, 0xb1, 0x26, 0xb5, + 0x5d, 0x84, 0xc7, 0x0a, 0x78, 0xd7, 0x27, 0x50, 0x32, 0x2d, 0x57, 0x21, 0x45, 0x23, 0xf7, 0x02, + 0xb4, 0x14, 0x2c, 0x6c, 0x4a, 0x5c, 0x57, 0x7d, 0x9b, 0x2f, 0x98, 0x96, 0xdb, 0x1a, 0x1a, 0xbc, + 0xce, 0xaa, 0x7c, 0x06, 0xc5, 0xf0, 0x1e, 0xa1, 0x2c, 0x24, 0x6b, 0xdb, 0x8d, 0xda, 0xae, 0x14, + 0x41, 0x25, 0xc8, 0x6d, 0xb5, 0xe5, 0x46, 0xf3, 0x69, 0x4b, 0xd9, 0x6d, 0xfc, 0x82, 0xf5, 0x9c, + 0x5a, 0x6d, 0xd1, 0x73, 0xf2, 0xaa, 0x9c, 0xa4, 0x94, 0xaa, 0xfc, 0x57, 0x14, 0x8a, 0x07, 0xb6, + 0xde, 0x57, 0xed, 0xb3, 0x5d, 0x7c, 0x76, 0xf8, 0x46, 0x1d, 0xa0, 0x2f, 0x60, 0xde, 0xc4, 0x6f, + 0x94, 0x01, 0x83, 0x2a, 0x5e, 0xd6, 0x1c, 0x9d, 0xdc, 0x90, 0x9c, 0x33, 0xf1, 0x1b, 0x2e, 0xa1, + 0xc9, 0x93, 0xe6, 0x8f, 0x21, 0x67, 0x19, 0x1a, 0xe3, 0xc4, 0xa2, 0x29, 0x94, 0x0b, 0x32, 0x81, + 0x65, 0x68, 0x4d, 0x86, 0x26, 0xd4, 0x64, 0x3c, 0x41, 0x1d, 0x9f, 0x40, 0x6d, 0xe2, 0x37, 0x82, + 0xfa, 0x0b, 0x98, 0x27, 0xb2, 0xc7, 0x66, 0x97, 0x98, 0x32, 0x3b, 0xcb, 0xd0, 0xc2, 0xb3, 0xe3, + 0xca, 0xfb, 0xb7, 0x51, 0xa0, 0x4e, 0x7b, 0xe8, 0x8a, 0x76, 0x11, 0x5d, 0xfc, 0x8f, 0xa1, 0x40, + 0x26, 0xe3, 0xd7, 0x4a, 0xd1, 0x29, 0xe7, 0x41, 0xe6, 0x2c, 0x3c, 0x30, 0xe1, 0x22, 0x93, 0xf2, + 0xb9, 0x62, 0xd3, 0xb8, 0x2c, 0xc3, 0x6b, 0x4e, 0xa1, 0x0f, 0x20, 0xaf, 0x9b, 0xc4, 0x69, 0xf1, + 0x6a, 0x3d, 0xd8, 0xcb, 0xcd, 0x71, 0x0c, 0xa9, 0xd9, 0xf9, 0x8c, 0xff, 0x2a, 0x05, 0xc8, 0x57, + 0xd6, 0xfd, 0xa1, 0xab, 0x52, 0x0b, 0xae, 0x42, 0x8a, 0xab, 0x4e, 0x94, 0xaa, 0xe4, 0x07, 0x53, + 0xad, 0x2c, 0xdc, 0x35, 0xd9, 0x8e, 0xc8, 0x9c, 0x11, 0xfd, 0x2c, 0xd8, 0x73, 0xce, 0x6d, 0xbc, + 0x3f, 0x9b, 0x9f, 0xdf, 0x8e, 0x88, 0x66, 0xf4, 0x2e, 0x24, 0x1d, 0x97, 0x78, 0x9c, 0x38, 0x8d, + 0x13, 0xeb, 0x53, 0xf8, 0xc7, 0x27, 0xbf, 0x76, 0x48, 0xd8, 0x84, 0x9d, 0x51, 0x19, 0xe8, 0x05, + 0x64, 0xbd, 0xf4, 0x88, 0x37, 0xb0, 0x1f, 0xcf, 0x2e, 0xd0, 0xf3, 0x6c, 0xc2, 0xef, 0x79, 0xb2, + 0x50, 0x15, 0x72, 0x7d, 0x4e, 0xe6, 0x17, 0xc1, 0x2b, 0x3c, 0x43, 0x05, 0x21, 0x81, 0x66, 0xaa, + 0x81, 0x2f, 0x19, 0x04, 0x53, 0x93, 0x7a, 0x71, 0xdb, 0x32, 0x8c, 0x63, 0xb5, 0xf3, 0x92, 0x36, + 0xe5, 0x3c, 0x2f, 0x2e, 0xa0, 0x68, 0x97, 0xe4, 0x99, 0xc2, 0x2e, 0x69, 0x8b, 0x2d, 0x37, 0x43, + 0x1b, 0x50, 0xf8, 0xbd, 0xed, 0x88, 0x1c, 0x60, 0x47, 0x6d, 0x28, 0x0e, 0x42, 0xb6, 0xc9, 0x93, + 0xba, 0x87, 0xd3, 0x22, 0x7b, 0x88, 0x78, 0x3b, 0x22, 0x8f, 0xb0, 0xa3, 0x5f, 0x02, 0xea, 0x8c, + 0xe9, 0x7c, 0x19, 0x7e, 0x60, 0x96, 0xa3, 0x0c, 0xdb, 0x11, 0x79, 0x82, 0x98, 0xca, 0x17, 0x90, + 0xa4, 0xc7, 0x49, 0x02, 0xc7, 0xb3, 0xd6, 0x6e, 0xab, 0xfd, 0xa2, 0xc5, 0x7c, 0x51, 0xbd, 0xb1, + 0xd7, 0x38, 0x6a, 0x28, 0xed, 0xd6, 0x1e, 0xf1, 0x45, 0x77, 0xe1, 0x36, 0x07, 0x54, 0x5b, 0x75, + 0xe5, 0x85, 0xdc, 0x14, 0xa8, 0x58, 0x65, 0x35, 0x18, 0x99, 0x32, 0x90, 0x68, 0xb5, 0x5b, 0x0d, + 0x29, 0x42, 0x63, 0x54, 0xbd, 0x2e, 0x45, 0x69, 0x8c, 0x92, 0xdb, 0x07, 0xc2, 0x85, 0x6d, 0xe6, + 0x01, 0x34, 0x4f, 0x05, 0x76, 0x12, 0x99, 0x94, 0x94, 0xae, 0xfc, 0x75, 0x14, 0x32, 0x24, 0xe5, + 0x6f, 0x9a, 0x5d, 0x0b, 0x3d, 0x86, 0xec, 0x40, 0xb5, 0xb1, 0xe9, 0xfa, 0x96, 0x2c, 0x3a, 0x31, + 0x99, 0x03, 0x8a, 0xf0, 0x1a, 0x05, 0x19, 0x46, 0xd8, 0xbc, 0xac, 0xcc, 0xde, 0x02, 0x89, 0x8b, + 0x73, 0x3a, 0x27, 0xb8, 0xaf, 0x12, 0xa9, 0xac, 0x17, 0x70, 0x8f, 0x4b, 0x2d, 0x32, 0xa9, 0x87, + 0x14, 0xed, 0xc9, 0x2e, 0x0e, 0x82, 0x50, 0xe1, 0x81, 0xfe, 0xf9, 0x3e, 0x94, 0x46, 0x22, 0xd0, + 0x25, 0x65, 0xd5, 0x0a, 0x2d, 0xab, 0xe2, 0xbe, 0x5f, 0xf1, 0xca, 0xaa, 0x18, 0xaf, 0xa8, 0x42, + 0x8b, 0x4d, 0xcc, 0xb8, 0xd8, 0xc7, 0x7e, 0x96, 0xc5, 0x0c, 0xe2, 0x2e, 0xf7, 0x59, 0x73, 0x97, + 0x24, 0x58, 0x07, 0x30, 0xd7, 0xb7, 0x34, 0xbd, 0x4b, 0xf2, 0x09, 0x62, 0x4d, 0xae, 0xde, 0x67, + 0x4d, 0xea, 0xdc, 0xc6, 0xbb, 0x01, 0x2d, 0x1a, 0xba, 0xba, 0xb1, 0x76, 0x62, 0x74, 0xd6, 0x8e, + 0xc4, 0x15, 0x9c, 0x68, 0xbc, 0x05, 0xb9, 0x09, 0x12, 0x3d, 0x85, 0xb4, 0xe8, 0x1e, 0x64, 0x68, + 0xce, 0x3e, 0xab, 0x17, 0x13, 0x79, 0x16, 0xe7, 0x46, 0x5b, 0x50, 0x34, 0xf1, 0x69, 0xb0, 0xd9, + 0x95, 0x0d, 0xd9, 0x79, 0xbe, 0x85, 0x4f, 0x27, 0x77, 0xba, 0xf2, 0xa6, 0x8f, 0xd1, 0xd0, 0x97, + 0x50, 0x08, 0x85, 0x18, 0x6e, 0x24, 0x33, 0xba, 0x46, 0x2f, 0x99, 0x0e, 0x44, 0x1e, 0xb4, 0x05, + 0x69, 0x11, 0xe3, 0x72, 0x74, 0x8d, 0x57, 0x13, 0x26, 0x98, 0xd1, 0x26, 0x09, 0x51, 0xa7, 0xae, + 0x1f, 0xfa, 0xf2, 0x7e, 0x7a, 0x7c, 0x71, 0xbe, 0x9c, 0x23, 0x2b, 0x9c, 0xd0, 0xd2, 0xca, 0x99, + 0x1e, 0x5c, 0x43, 0x3b, 0x00, 0xde, 0xd5, 0xa7, 0x43, 0x3b, 0xba, 0xd3, 0xcb, 0xa4, 0x03, 0x41, + 0xe8, 0x4f, 0x49, 0x0e, 0x70, 0xa3, 0x7d, 0xc8, 0x0a, 0x17, 0xc9, 0x0a, 0x98, 0xe9, 0xbe, 0x64, + 0xdc, 0x61, 0x0b, 0x37, 0xed, 0x49, 0x20, 0xb9, 0x9b, 0x81, 0x55, 0x07, 0xf3, 0x2a, 0xe6, 0xc9, + 0x8c, 0xb9, 0x1b, 0x33, 0xae, 0xda, 0x89, 0x6a, 0xf6, 0xf0, 0x1e, 0xe1, 0xdf, 0x8c, 0x95, 0xa3, + 0x32, 0x13, 0x85, 0x5a, 0x20, 0xd1, 0x2d, 0x0b, 0xfa, 0x7f, 0x89, 0xee, 0xda, 0x7b, 0xc2, 0x70, + 0xc9, 0xae, 0x4d, 0x8d, 0x01, 0x54, 0xa7, 0xf6, 0xfd, 0x38, 0xf0, 0x3b, 0x50, 0xec, 0x5a, 0x76, + 0x5f, 0x75, 0x15, 0x61, 0x3c, 0x73, 0x7e, 0xbf, 0xe3, 0xfb, 0xf3, 0xe5, 0xc2, 0x16, 0xc5, 0x0a, + 0xc3, 0x29, 0x74, 0x83, 0x9f, 0x68, 0x5b, 0x84, 0xcb, 0x5b, 0x34, 0xba, 0x7d, 0x3c, 0xeb, 0x0a, + 0xc7, 0x63, 0x65, 0x0b, 0x52, 0x34, 0x39, 0x75, 0xca, 0xf3, 0x74, 0xdf, 0xaf, 0x99, 0xe8, 0xca, + 0x5c, 0x0a, 0xda, 0x83, 0xa2, 0x46, 0x20, 0xa4, 0x72, 0x62, 0xfd, 0x94, 0xdb, 0x54, 0xee, 0xf2, + 0x14, 0xb9, 0xc2, 0xd1, 0x8a, 0xb2, 0x59, 0x30, 0xb3, 0x9e, 0x4b, 0x1b, 0x32, 0x5d, 0xb5, 0xaf, + 0x1b, 0x3a, 0x76, 0xca, 0x0b, 0x54, 0xce, 0x27, 0x97, 0x5a, 0xf5, 0x68, 0x6f, 0x5d, 0x04, 0x57, + 0x21, 0xc4, 0x33, 0x6e, 0x0a, 0x38, 0x23, 0x87, 0x78, 0x67, 0xdc, 0xb8, 0x45, 0x6f, 0x3d, 0xd4, + 0x67, 0xa7, 0xc6, 0xcd, 0xbf, 0x34, 0xf4, 0x00, 0xe0, 0xb5, 0x8e, 0xdf, 0x28, 0xaf, 0x86, 0xd8, + 0x3e, 0x2b, 0x97, 0x83, 0x65, 0x12, 0x81, 0x7f, 0x49, 0xc0, 0xe8, 0x53, 0xc8, 0x6a, 0x78, 0x80, + 0x4d, 0xcd, 0x69, 0x9b, 0xe5, 0xbb, 0x34, 0x29, 0xbd, 0x75, 0x71, 0xbe, 0x9c, 0xad, 0x0b, 0x20, + 0xf7, 0xa5, 0x3e, 0x15, 0xfa, 0x0a, 0xf2, 0xec, 0x03, 0x6b, 0x6d, 0x73, 0xf3, 0xac, 0xbc, 0x48, + 0x17, 0xfd, 0x68, 0xc6, 0x43, 0xf1, 0x9b, 0x10, 0x5e, 0xdf, 0xb6, 0x1e, 0x90, 0x26, 0x87, 0x64, + 0xa3, 0xdf, 0x87, 0xbc, 0xd0, 0xe6, 0x1d, 0xeb, 0xd8, 0x29, 0xbf, 0x73, 0x69, 0x53, 0x75, 0x74, + 0xac, 0x7d, 0x9f, 0x55, 0xf8, 0xaa, 0xa0, 0x34, 0xf4, 0x73, 0x28, 0x78, 0x17, 0x44, 0xd6, 0xc0, + 0x75, 0xca, 0xf7, 0xa8, 0x31, 0x3e, 0x9e, 0x55, 0x55, 0x39, 0x6f, 0x7b, 0x40, 0xfb, 0xcd, 0x81, + 0x2f, 0x74, 0x1f, 0xb2, 0x9a, 0x6d, 0x0d, 0x58, 0xcc, 0x78, 0x77, 0x25, 0xba, 0x1a, 0xf7, 0x2a, + 0x61, 0xdb, 0x1a, 0xd0, 0x60, 0xa0, 0x40, 0xd1, 0xc6, 0x03, 0x43, 0xed, 0xe0, 0x3e, 0x89, 0x66, + 0x56, 0xb7, 0xbc, 0x44, 0x47, 0xdf, 0x98, 0x79, 0x23, 0x3d, 0x66, 0xa1, 0x98, 0x01, 0x79, 0xed, + 0x2e, 0x7a, 0x06, 0xa0, 0x0e, 0x35, 0xdd, 0x55, 0xfa, 0x96, 0x86, 0xcb, 0xcb, 0x97, 0xde, 0x5a, + 0x8e, 0x0a, 0xaf, 0x12, 0xc6, 0x7d, 0x4b, 0xc3, 0xde, 0x0d, 0x85, 0x00, 0xa0, 0x4f, 0x21, 0x47, + 0x97, 0xf6, 0x95, 0x75, 0x4c, 0x74, 0x73, 0x85, 0x2e, 0x6e, 0x8e, 0x9f, 0x65, 0xb6, 0x6e, 0x5b, + 0x83, 0x1d, 0xeb, 0x98, 0x6a, 0x0c, 0xff, 0xa9, 0x21, 0x07, 0xf2, 0xbd, 0x8e, 0xe2, 0xbb, 0xcf, + 0xfb, 0xf4, 0x14, 0x3f, 0x9f, 0x71, 0x2e, 0x4f, 0x6b, 0x13, 0x1c, 0xea, 0x2d, 0x11, 0x07, 0x9e, + 0xd6, 0x04, 0xcc, 0x91, 0x73, 0xbd, 0x8e, 0xf7, 0x41, 0xea, 0x0e, 0xd6, 0x6e, 0xe1, 0x06, 0x50, + 0x09, 0xd6, 0x1d, 0x0c, 0xc3, 0x4c, 0xa0, 0x05, 0xbc, 0x2f, 0xa3, 0xa8, 0x8e, 0x62, 0x75, 0xd9, + 0x99, 0x3d, 0x98, 0x3d, 0xce, 0x17, 0x19, 0x77, 0xd5, 0x69, 0x77, 0xe9, 0xc1, 0x76, 0x20, 0x6f, + 0x0d, 0xdd, 0x63, 0x6b, 0x68, 0x6a, 0x4a, 0xf7, 0xa5, 0x53, 0x7e, 0x8f, 0xae, 0xf6, 0x4a, 0x35, + 0xb4, 0xb7, 0xba, 0x36, 0x17, 0xb4, 0xb5, 0xeb, 0xc8, 0x39, 0x21, 0x75, 0xeb, 0xa5, 0x83, 0xfe, + 0x08, 0x72, 0xba, 0xe9, 0x8f, 0xf1, 0xf0, 0xea, 0x63, 0x20, 0x51, 0x13, 0x34, 0x4d, 0x6f, 0x08, + 0xe0, 0x32, 0xc9, 0x08, 0x1f, 0x41, 0xd1, 0xea, 0x76, 0x0d, 0xdd, 0xc4, 0x8a, 0x8d, 0x55, 0xc7, + 0x32, 0xcb, 0xef, 0x07, 0x76, 0xb0, 0xc0, 0x71, 0x32, 0x45, 0xa1, 0x0a, 0x64, 0x5d, 0xdc, 0x1f, + 0x58, 0xb6, 0x6a, 0x9f, 0x95, 0x3f, 0x08, 0x5e, 0xec, 0x78, 0x60, 0x74, 0x0c, 0x8b, 0x43, 0x13, + 0x9f, 0x0e, 0x2c, 0x07, 0x6b, 0xca, 0x58, 0x86, 0xb9, 0x4a, 0x7d, 0xdc, 0x43, 0x3e, 0xa9, 0x3b, + 0xcf, 0x04, 0xe5, 0xc4, 0x54, 0xf3, 0xce, 0x70, 0x22, 0x5a, 0x5b, 0xfc, 0x75, 0x14, 0xe6, 0xc6, + 0x62, 0x24, 0xfa, 0x43, 0x48, 0x9b, 0x96, 0x16, 0xb8, 0x46, 0x6b, 0xf0, 0x61, 0x52, 0x2d, 0x4b, + 0x63, 0xb7, 0x68, 0x8f, 0x7b, 0xba, 0x7b, 0x32, 0x3c, 0x5e, 0xeb, 0x58, 0xfd, 0x75, 0x6f, 0x13, + 0xb5, 0x63, 0xff, 0xf7, 0xfa, 0xe0, 0x65, 0x6f, 0x9d, 0xfe, 0x1a, 0x1c, 0xaf, 0x31, 0x36, 0x39, + 0x45, 0xa4, 0x36, 0x35, 0xf4, 0x09, 0x94, 0xf0, 0xe9, 0x40, 0xb7, 0x03, 0x79, 0x62, 0x2c, 0x60, + 0xf3, 0x45, 0x1f, 0x49, 0x14, 0x84, 0x5f, 0x74, 0xfc, 0x5d, 0x0c, 0x4a, 0x23, 0x11, 0x8a, 0x24, + 0xc6, 0xb4, 0x46, 0x0e, 0x25, 0xc6, 0x04, 0x72, 0x49, 0xba, 0x1e, 0x7c, 0x90, 0x11, 0xbf, 0xe9, + 0x83, 0x8c, 0xf0, 0x05, 0x42, 0xf2, 0x0a, 0x17, 0x08, 0x3f, 0x85, 0x05, 0xdd, 0x51, 0x4c, 0xcb, + 0x14, 0x7d, 0x1e, 0xaf, 0x3c, 0x0c, 0x5e, 0xfb, 0xdf, 0xd2, 0x9d, 0x96, 0x65, 0xb2, 0x0e, 0x8f, + 0xb7, 0x6a, 0xff, 0x85, 0x40, 0x7a, 0xfc, 0x85, 0x80, 0xd7, 0xc7, 0x49, 0x48, 0xc9, 0xc5, 0x6f, + 0xa2, 0x90, 0x0d, 0x3e, 0x33, 0x8b, 0x85, 0x5b, 0x17, 0x63, 0xc5, 0xc2, 0x35, 0xaf, 0x43, 0xc3, + 0xbb, 0x10, 0x9f, 0x7d, 0x17, 0xf8, 0xd1, 0xfe, 0x31, 0xe4, 0x02, 0xa1, 0x67, 0xb4, 0x24, 0x8f, + 0x5e, 0xa3, 0x24, 0x7f, 0x0f, 0x52, 0xdc, 0xdf, 0x32, 0xc5, 0x2a, 0x70, 0xee, 0x24, 0xf3, 0xb5, + 0xc9, 0xaf, 0x88, 0x9f, 0xe5, 0xa3, 0xff, 0x4b, 0x1c, 0xf2, 0xc1, 0xd0, 0x44, 0x8c, 0x53, 0x37, + 0x3b, 0x36, 0x8d, 0x0b, 0x74, 0xf4, 0xb8, 0x77, 0xeb, 0x2a, 0xc0, 0x24, 0x60, 0xf5, 0x75, 0x53, + 0xa1, 0x37, 0x7d, 0x21, 0xe5, 0xcd, 0xf4, 0x75, 0xf3, 0x39, 0x81, 0x52, 0x12, 0xf5, 0x94, 0x93, + 0xc4, 0x43, 0x24, 0xea, 0x29, 0x23, 0x59, 0xa4, 0x39, 0x9f, 0xed, 0xd2, 0xc2, 0x2c, 0x1e, 0xc8, + 0xe2, 0x6c, 0x17, 0x2d, 0x41, 0xfa, 0xb5, 0x6e, 0xbb, 0x43, 0xd5, 0xa0, 0x35, 0x98, 0x38, 0x66, + 0x01, 0x44, 0x26, 0x14, 0xfd, 0x60, 0xfc, 0xc6, 0xc4, 0x36, 0x55, 0x9c, 0xdc, 0x46, 0xf5, 0x1a, + 0xd1, 0xd8, 0xff, 0x20, 0x82, 0x84, 0xcb, 0x72, 0x82, 0xc0, 0xc5, 0xbf, 0x8c, 0x42, 0x21, 0x44, + 0x86, 0x9a, 0x50, 0xa2, 0x03, 0x8f, 0xf5, 0xc5, 0xee, 0x7b, 0x0f, 0xc6, 0x08, 0x7a, 0x62, 0x5d, + 0x55, 0xb0, 0x02, 0x28, 0x0d, 0x7d, 0x01, 0x45, 0x26, 0xca, 0xbb, 0xbf, 0x0f, 0xab, 0x5f, 0x9e, + 0x4a, 0x0a, 0x5f, 0xe2, 0xe7, 0x2d, 0x1f, 0xa6, 0x05, 0xaf, 0x26, 0x17, 0x4d, 0xc8, 0x05, 0xa2, + 0xfd, 0x0c, 0x7a, 0xff, 0x13, 0x48, 0x78, 0x5e, 0x68, 0xc6, 0x28, 0x46, 0x19, 0xf8, 0x78, 0x5f, + 0x47, 0x61, 0x7e, 0x52, 0xd4, 0x0d, 0xd9, 0x13, 0x53, 0xa4, 0x99, 0xec, 0xe9, 0x41, 0x30, 0x1b, + 0x62, 0xca, 0x25, 0xae, 0xcb, 0xfc, 0x7c, 0xe8, 0x7d, 0x4f, 0xc5, 0x99, 0x6e, 0x95, 0x42, 0x2a, + 0x4e, 0xaa, 0x9c, 0x80, 0x92, 0x57, 0x1e, 0x8b, 0x36, 0x0c, 0x40, 0xea, 0xe0, 0xd9, 0xe6, 0x5e, + 0xb3, 0x36, 0xb1, 0x85, 0x82, 0x72, 0x90, 0x6e, 0x6f, 0x6d, 0xed, 0x35, 0x5b, 0x0d, 0x29, 0x5e, + 0x59, 0x85, 0xac, 0x97, 0xd8, 0xa0, 0x3c, 0x64, 0xea, 0xcd, 0xc3, 0xea, 0xe6, 0x5e, 0xa3, 0x2e, + 0x45, 0x50, 0x01, 0xb2, 0x72, 0xa3, 0x5a, 0xa7, 0x8d, 0x1a, 0x29, 0xfa, 0x59, 0xe6, 0x57, 0x5f, + 0x2f, 0x47, 0xb9, 0xe3, 0x49, 0x49, 0xe9, 0x9d, 0x44, 0x06, 0x49, 0xb7, 0x2a, 0xff, 0x13, 0x03, + 0x54, 0x57, 0x5d, 0x95, 0xe8, 0xdf, 0x15, 0xda, 0x19, 0xb1, 0x4b, 0x4e, 0x2a, 0x5c, 0xa2, 0xc6, + 0x6f, 0x54, 0xa2, 0x4e, 0x6c, 0x58, 0x24, 0x6e, 0xd2, 0xb0, 0xb8, 0x56, 0xdf, 0x64, 0xbc, 0xbc, + 0x4a, 0x5d, 0xbf, 0xbc, 0xf2, 0x4f, 0xa2, 0x72, 0x91, 0x82, 0xe2, 0xd1, 0xd9, 0x20, 0xb8, 0xf3, + 0xd7, 0xea, 0x7c, 0x4d, 0xea, 0x6f, 0xc5, 0xae, 0xde, 0xdf, 0xba, 0xe4, 0x71, 0x32, 0x3b, 0xf6, + 0xc4, 0x25, 0xc7, 0x5e, 0x87, 0x04, 0x7d, 0xd1, 0x95, 0xa4, 0x01, 0x7b, 0xda, 0x81, 0x87, 0x57, + 0xbb, 0x16, 0x78, 0xd4, 0x45, 0xb9, 0xd1, 0x2f, 0x20, 0x8f, 0xcd, 0x61, 0x5f, 0xe9, 0xe3, 0xfe, + 0x31, 0xb6, 0xc5, 0x3e, 0x3f, 0x9a, 0x4d, 0x5a, 0xc3, 0x1c, 0xf6, 0xf7, 0x29, 0xa3, 0x48, 0x8a, + 0xb1, 0x07, 0x71, 0xd0, 0x23, 0x48, 0xaa, 0x86, 0xae, 0x3a, 0xbc, 0xe1, 0x75, 0xd9, 0x5b, 0x41, + 0x46, 0x88, 0x7e, 0x17, 0x0a, 0xaa, 0x6d, 0xab, 0x67, 0xfc, 0xcd, 0x9b, 0x46, 0xdb, 0xc2, 0x5c, + 0x63, 0x48, 0x2a, 0x5b, 0x25, 0x48, 0xfa, 0xcc, 0x4d, 0x6c, 0x44, 0x4e, 0xf5, 0x40, 0xa1, 0x16, + 0x5d, 0xf6, 0x66, 0x2d, 0x3a, 0xb8, 0x89, 0xc6, 0x8f, 0x2b, 0x6f, 0xee, 0xfa, 0xca, 0xbb, 0xf8, + 0xab, 0x28, 0x80, 0xbf, 0xcf, 0xe8, 0x27, 0x70, 0x67, 0x70, 0x72, 0xe6, 0xd0, 0x27, 0x86, 0x36, + 0x1e, 0xd8, 0xd8, 0xc1, 0x26, 0xf3, 0xac, 0x54, 0x79, 0xf3, 0xf2, 0x82, 0x40, 0xcb, 0x21, 0x2c, + 0xfa, 0x1c, 0x16, 0xc4, 0xd3, 0xc4, 0x11, 0xbe, 0x60, 0x3e, 0x78, 0x9b, 0xd3, 0x84, 0x99, 0xb9, + 0xc3, 0x7c, 0x07, 0x12, 0x44, 0x7f, 0x88, 0x6b, 0x6c, 0xb4, 0x9e, 0xed, 0x4b, 0x11, 0x94, 0x85, + 0x64, 0x75, 0xaf, 0x59, 0x3d, 0x0c, 0xba, 0xbb, 0xca, 0xdf, 0xc4, 0x41, 0x62, 0x1a, 0x7e, 0x53, + 0x33, 0x9b, 0x9e, 0xb1, 0xfe, 0x70, 0x93, 0x37, 0xec, 0x15, 0x13, 0x6f, 0xdf, 0x2b, 0x26, 0xdf, + 0x92, 0x57, 0x4c, 0xdd, 0xc0, 0x2b, 0xa6, 0xdf, 0x8a, 0x57, 0xfc, 0x26, 0x06, 0x10, 0x38, 0xaa, + 0x9f, 0x05, 0xff, 0xf5, 0x62, 0x7a, 0xf3, 0x75, 0x24, 0x79, 0xda, 0x8e, 0x88, 0x7f, 0xcc, 0x78, + 0x0a, 0x19, 0x8d, 0x47, 0x38, 0x9e, 0x3d, 0x4c, 0xed, 0x72, 0x8e, 0x05, 0xc2, 0xed, 0x88, 0xec, + 0x31, 0xa3, 0xcf, 0x43, 0x6f, 0x8d, 0x1f, 0xce, 0xe4, 0x93, 0xb6, 0xc5, 0x53, 0x8a, 0x2a, 0xa4, + 0x98, 0x6f, 0xe6, 0x67, 0x3f, 0xf5, 0xc9, 0xeb, 0x88, 0xa6, 0x6e, 0x47, 0x64, 0xce, 0xc8, 0xef, + 0x4e, 0xd2, 0x90, 0x1c, 0x9a, 0xba, 0x65, 0x7e, 0x28, 0x07, 0x2f, 0xf1, 0x45, 0xf9, 0x43, 0xa2, + 0x3e, 0xfd, 0xad, 0xba, 0x58, 0x63, 0xb7, 0x38, 0xcf, 0xcc, 0xd7, 0x1e, 0x20, 0x8a, 0x8a, 0x00, + 0x1c, 0xaf, 0x9b, 0x3d, 0x29, 0x46, 0x73, 0x06, 0xdb, 0x1a, 0x0c, 0xc8, 0x57, 0xfc, 0xc3, 0x75, + 0x90, 0x46, 0xdf, 0xdc, 0x06, 0xee, 0x73, 0xe6, 0xa0, 0xb0, 0xff, 0xbc, 0x56, 0x3b, 0x6a, 0xee, + 0x37, 0x0e, 0x8f, 0xaa, 0xfb, 0x07, 0x52, 0x74, 0xf3, 0x47, 0xdf, 0xfe, 0xfb, 0x52, 0xe4, 0xdb, + 0x8b, 0xa5, 0xe8, 0x6f, 0x2e, 0x96, 0xa2, 0xbf, 0xbd, 0x58, 0x8a, 0xfe, 0xdb, 0xc5, 0x52, 0xf4, + 0xcf, 0xbe, 0x5b, 0x8a, 0xfc, 0xe6, 0xbb, 0xa5, 0xc8, 0x6f, 0xbf, 0x5b, 0x8a, 0xfc, 0x5e, 0x9a, + 0xaf, 0xec, 0x7f, 0x03, 0x00, 0x00, 0xff, 0xff, 0x8e, 0x1e, 0x60, 0x7f, 0x0f, 0x34, 0x00, 0x00, } diff --git a/pkg/sql/sqlbase/structured.proto b/pkg/sql/sqlbase/structured.proto index c4596ec442c6..c750f9b99827 100644 --- a/pkg/sql/sqlbase/structured.proto +++ b/pkg/sql/sqlbase/structured.proto @@ -139,8 +139,22 @@ message ColumnDescriptor { (gogoproto.customname) = "LogicalColumnID", (gogoproto.casttype) = "ColumnID"]; // Used to indicate column is used and dropped for ALTER COLUMN TYPE mutation. optional bool alter_column_type_in_progress = 14 [(gogoproto.nullable) = false]; + + // SystemColumnKind represents what kind of system column this column + // descriptor represents, if any. + optional SystemColumnKind system_column_kind = 15 [(gogoproto.nullable) = false]; } - + +// SystemColumnKind is an enum representing the different kind of system +// columns that can be synthesized by the execution engine. +enum SystemColumnKind { + // Default value, unused. + NONE = 0; + // A system column containing the value of the MVCC timestamp associated + // with the kv's corresponding to the row. + MVCCTIMESTAMP = 1; +} + // ColumnFamilyDescriptor is set of columns stored together in one kv entry. // For more information, look at `docs/tech-notes/encoding.md#value-encoding`. message ColumnFamilyDescriptor { diff --git a/pkg/sql/sqlbase/system_columns.go b/pkg/sql/sqlbase/system_columns.go new file mode 100644 index 000000000000..78cfda9b5ca4 --- /dev/null +++ b/pkg/sql/sqlbase/system_columns.go @@ -0,0 +1,137 @@ +// Copyright 2020 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 sqlbase + +import ( + "math" + + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/errors" +) + +// Similar to Postgres, we also expose hidden system columns on tables. +// These system columns are not explicitly part of the TableDescriptor, +// and instead are constructs that are manipulated purely during planning. +// As of now, these system columns are able to be generated from the set +// of KV's that make up a row, and are produced by the row and cfetcher. +// Each system column is given a kind, and associated with a particular +// column ID that is counting down from math.MaxUint32. This is done so that +// each system column ID won't conflict with existing column ID's and also +// will be stable across all changes to the table. +// * MVCC Timestamp: contains a representation of the row's HLC timestamp. + +// MVCCTimestampColumnName is the name of the MVCC timestamp system column. +const MVCCTimestampColumnName = "crdb_internal_mvcc_timestamp" + +// MVCCTimestampColumnType is the type of the MVCC timestamp system column. +var MVCCTimestampColumnType = types.Decimal + +// MVCCTimestampColumnID is the ColumnID of the MVCC timesatmp column. Future +// system columns will have ID's that decrement from this value. +const MVCCTimestampColumnID = math.MaxUint32 + +// NewMVCCTimestampColumnDesc creates a column descriptor for the MVCC timestamp +// system column. +func NewMVCCTimestampColumnDesc() *ColumnDescriptor { + return &ColumnDescriptor{ + Name: MVCCTimestampColumnName, + Type: MVCCTimestampColumnType, + Hidden: true, + Nullable: true, + SystemColumnKind: SystemColumnKind_MVCCTIMESTAMP, + ID: MVCCTimestampColumnID, + } +} + +// IsColIDSystemColumn returns whether a column ID refers to a system column. +func IsColIDSystemColumn(colID ColumnID) bool { + switch colID { + case MVCCTimestampColumnID: + return true + default: + return false + } +} + +// GetSystemColumnDescriptorFromID returns a column descriptor corresponding +// to the system column referred to by the input column ID. +func GetSystemColumnDescriptorFromID(colID ColumnID) (*ColumnDescriptor, error) { + switch colID { + case MVCCTimestampColumnID: + return NewMVCCTimestampColumnDesc(), nil + default: + return nil, errors.AssertionFailedf("unsupported system column ID %d", colID) + } +} + +// GetSystemColumnKindFromColumnID returns the kind of system column that colID +// refers to. +func GetSystemColumnKindFromColumnID(colID ColumnID) SystemColumnKind { + switch colID { + case MVCCTimestampColumnID: + return SystemColumnKind_MVCCTIMESTAMP + default: + return SystemColumnKind_NONE + } +} + +// GetSystemColumnIDByKind returns the column ID of the desired system column. +func GetSystemColumnIDByKind(kind SystemColumnKind) (ColumnID, error) { + switch kind { + case SystemColumnKind_MVCCTIMESTAMP: + return MVCCTimestampColumnID, nil + default: + return 0, errors.Newf("invalid system column kind %s", kind.String()) + } +} + +// GetSystemColumnTypeForKind returns the types.T of the input system column. +func GetSystemColumnTypeForKind(kind SystemColumnKind) *types.T { + switch kind { + case SystemColumnKind_MVCCTIMESTAMP: + return MVCCTimestampColumnType + default: + return nil + } +} + +// IsSystemColumnName returns whether or not a name is a reserved system +// column name. +func IsSystemColumnName(name string) bool { + switch name { + case MVCCTimestampColumnName: + return true + default: + return false + } +} + +// GetSystemColumnTypesAndDescriptors is a utility method to construct a set of +// types and column descriptors from an input list of system column kinds. +func GetSystemColumnTypesAndDescriptors( + desc *TableDescriptor, kinds []SystemColumnKind, +) ([]*types.T, []ColumnDescriptor, error) { + resTypes := make([]*types.T, len(kinds)) + resDescs := make([]ColumnDescriptor, len(kinds)) + for i, k := range kinds { + resTypes[i] = GetSystemColumnTypeForKind(k) + colID, err := GetSystemColumnIDByKind(k) + if err != nil { + return nil, nil, err + } + colDesc, err := GetSystemColumnDescriptorFromID(colID) + if err != nil { + return nil, nil, err + } + resDescs[i] = *colDesc + } + return resTypes, resDescs, nil +} diff --git a/pkg/sql/sqlbase/validate_test.go b/pkg/sql/sqlbase/validate_test.go index 043e5bebc3e9..a77dfe89b729 100644 --- a/pkg/sql/sqlbase/validate_test.go +++ b/pkg/sql/sqlbase/validate_test.go @@ -174,6 +174,7 @@ var validationMap = []struct { status: todoIAmKnowinglyAddingTechDebt, reason: "initial import: TODO(features): add validation"}, "AlterColumnTypeInProgress": {status: thisFieldReferencesNoObjects}, + "SystemColumnKind": {status: thisFieldReferencesNoObjects}, }, }, { diff --git a/pkg/sql/testdata/explain_tree b/pkg/sql/testdata/explain_tree index 6d728851c086..e2bb31a80e88 100644 --- a/pkg/sql/testdata/explain_tree +++ b/pkg/sql/testdata/explain_tree @@ -58,9 +58,9 @@ SELECT cid, sum(value) FROM t.orders WHERE date > '2015-01-01' GROUP BY cid ORDE render (cid int, sum decimal) │ render 0 (@2)[int] │ render 1 (@3)[decimal] - └── sort (column6 decimal, cid int, sum decimal) +column6 - │ order +column6 - └── render (column6 decimal, cid int, sum decimal) + └── sort (column7 decimal, cid int, sum decimal) +column7 + │ order +column7 + └── render (column7 decimal, cid int, sum decimal) │ render 0 ((1)[decimal] - (@2)[decimal])[decimal] │ render 1 (@1)[int] │ render 2 (@2)[decimal] @@ -90,7 +90,7 @@ children: - name: sort attrs: - key: order - value: +column6 + value: +column7 children: - name: render attrs: