diff --git a/pkg/sql/opt/exec/execbuilder/testdata/lookup_join b/pkg/sql/opt/exec/execbuilder/testdata/lookup_join index 7222951d027f..e1ca97cec27a 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/lookup_join +++ b/pkg/sql/opt/exec/execbuilder/testdata/lookup_join @@ -217,25 +217,25 @@ ALTER TABLE authors INJECT STATISTICS '[ query TTTTT colnames EXPLAIN (VERBOSE) SELECT DISTINCT authors.name FROM books AS b1, books2 AS b2, authors WHERE b1.title = b2.title AND authors.book = b1.title AND b1.shelf <> b2.shelf ---- -tree field description columns ordering -distinct · · (name) weak-key(name) - │ distinct on name · · - └── render · · (name) · - │ render 0 name · · - └── hash-join · · (title, shelf, title, shelf, name, book) · - │ type inner · · - │ equality (title) = (book) · · - ├── lookup-join · · (title, shelf, title, shelf) · - │ │ table books2@primary · · - │ │ type inner · · - │ │ equality (title) = (title) · · - │ │ pred @2 != @4 · · - │ └── scan · · (title, shelf) · - │ table books@primary · · - │ spans ALL · · - └── scan · · (name, book) · -· table authors@primary · · -· spans ALL · · +tree field description columns ordering +distinct · · (name) weak-key(name) + │ distinct on name · · + └── render · · (name) · + │ render 0 name · · + └── lookup-join · · (name, book, title, shelf, title, shelf) · + │ table books2@primary · · + │ type inner · · + │ equality (title) = (title) · · + │ pred @4 != @6 · · + └── hash-join · · (name, book, title, shelf) · + │ type inner · · + │ equality (book) = (title) · · + ├── scan · · (name, book) · + │ table authors@primary · · + │ spans ALL · · + └── scan · · (title, shelf) · +· table books@primary · · +· spans ALL · · # Verify data placement. query TTTI colnames @@ -244,12 +244,10 @@ SELECT start_key, end_key, replicas, lease_holder from [SHOW RANGES FROM TABLE b start_key end_key replicas lease_holder NULL NULL {5} 5 -# TODO(radu): this doesn't seem to be a lookup join, but it should be. - query T SELECT url FROM [EXPLAIN (DISTSQL) SELECT DISTINCT authors.name FROM books AS b1, books2 AS b2, authors WHERE b1.title = b2.title AND authors.book = b1.title AND b1.shelf <> b2.shelf] ---- -https://cockroachdb.github.io/distsqlplan/decode.html#eJyck89q3DAQh-99CndOCSjYku0cDAEdemhK2ZS0t7IHxZruqnE0RpKhJey7F9mlWZu10s1Rf76Zb362nsGSxo16Qg_Nd-DAoIYtg95Ri96Ti9vTpVv9C5qCgbH9EOL2lkFLDqF5hmBCh9DAhq6oz2tgoDEo043XDgxoCC-QD2qH0Fwf2FFhni78TT10eI9Ko8uLWXnonXlS7rd8IHr0wOBrr6xvsitgcDeEJpOcyRLWLPg5Fp_I2L8SPCEhgMFnosehz36SsRnZaBF9NpkU2fubTNbHdoLJksl61VG8NSlxWlINYR-_7OmsxKpHuerx0n6w5DQ61Mt_4PUrJ4b5qPw-ho4uL-ezdPgjXEh-eePMbh8upLj8N8N6kNU5QX4wPhjbhryad5Z8tX49q__KW7lH35P1-F-PpYjpoN7hlLanwbX4xVE7tpmWdyM3bmj0YTq9nha3djqKgscwT8IiDYskXM1gvoTLM2CxhKskXKe16yRcLODt4d2fAAAA__8CwK1z +https://cockroachdb.github.io/distsqlplan/decode.html#eJyck89q3DAQh-99CnVOCSjsyn9SEAR06KEpZbekvZU9KNZ0V42jMZIMLWHfvcgubGxipZujpPlG3_xsPYEjgxv9iAHkDxDAoYYdh85TgyGQT9tj0a35DXLNwbquj2l7x6EhjyCfINrYIkjY0BV1qxo4GIzatkPZkQP18QSFqPcI8vrInzUW-cbf9X2Ld6gN-tV60h46bx-1_6N0Hw_Jl8O3Trsg2RVw2PZRMiW4KmDJQ7zVQ7zscU_0sGRRLloUixany3tH3qBHM8_39ZIXRvmkw-EzWYd-VUwnafFnvFDF5Y23-0O8UOJykiRXJVfV4iTlOXkmgX9xlpk4k-AXooe-Y7_IOkZOMpWA7Yapir2_YerDyXFRrTpH7aMN0bomrqqpWKZ_Pen_ylu5w9CRC_hfj2WdviCaPY5_RKDeN_jVUzNcMy63AzdsGAxxPL0eF7duPEqCz2GRhcsJLOZwcQZczOEyC1d57SoL13m4zsLrGbw7vvsbAAD__5P3rXQ= query TTTTT colnames EXPLAIN (VERBOSE) SELECT a.name FROM authors AS a JOIN books2 AS b2 ON a.book = b2.title ORDER BY a.name @@ -675,18 +673,18 @@ render · · (a, b, c) · query TTTTT EXPLAIN (VERBOSE) SELECT * from abc WHERE EXISTS (SELECT * FROM def WHERE a=f AND c=e) ---- -render · · (a, b, c) · - │ render 0 a · · - │ render 1 b · · - │ render 2 c · · - └── lookup-join · · (a, b, c, e, f) · - │ table def@primary · · - │ type inner · · - │ equality (a, c) = (f, e) · · - │ equality cols are key · · · - └── scan · · (a, b, c) · -· table abc@primary · · -· spans ALL · · +render · · (a, b, c) · + │ render 0 a · · + │ render 1 b · · + │ render 2 c · · + └── lookup-join · · (a, b, c) · + │ table def@primary · · + │ type semi · · + │ equality (a, c) = (f, e) · · + │ equality cols are key · · · + └── scan · · (a, b, c) · +· table abc@primary · · +· spans ALL · · query TTTTT EXPLAIN (VERBOSE) SELECT * from abc WHERE NOT EXISTS (SELECT * FROM def WHERE a=f AND c=e) @@ -741,7 +739,7 @@ SELECT url FROM [ EXPLAIN (DISTSQL) SELECT a,b from small WHERE EXISTS (SELECT a FROM data WHERE small.a=data.a) ORDER BY a ] ---- -https://cockroachdb.github.io/distsqlplan/decode.html#eJzMlEFr2zAUx-_7FOJdJ-NIdtLUp1w6SOmS0eY2ctCiR-fN0TOSDCsl3304HqtdWlngQ3q0lD__X35PvGcwpHGjjuig-A4COEjgkAGHHDjMYc-htnRA58i2P-kCa_0HihmH0tSNb4_3HA5kEYpn8KWvEArYqR8V3qPSaNMZcNDoVVmda2pbHpV9Wrmjqirg8FAr4wqWpIIpo5lg5H-iBQ7bxhdsJfhKwv7EgRr_0ue8ekQoxInHM91Saf4hzYdIu6caC3Z382XHHm6-rtntdr0B_p9UK6-Awx3R76Zmv6g0jEyLFgcp34V8YSOr0aIeYq3EZ9if3vgnG0qoTsVQ7Hv12aBexM9NRM0tFUkqp05uhKo3ucXlJifj1ck4dTJJs6nqRqh66q4upy6LV5fFqcuSNJ-qboSqp255OXV5vLo8Tl2epPOp6kaoeuquP8aqfQPyHl1NxmHUFp21exj1I3Z721FjD_jN0uFc031uz7nzgUbnu1vRfaxNd9UC9sMiGJaDsHgdluHmkeosmM7D4XwK9zwYXoSbF1Oar4LhZbh5OaX5Ojyr2cgzCT-y193706e_AQAA__8Shyy4 +https://cockroachdb.github.io/distsqlplan/decode.html#eJzEll9v2jAUxd_3KdB92lSjxE7Cn0iTeNhLJ62dur1NPLjEg0g0RraRVlV89wkyKSKBe-0F6Fsp-XGPfc65yhtUulAP8kVZyH8BBwYCGCTAIAUGGcwZbIxeKGu12T9SA_fFH8hjBmW12br9v-cMFtooyN_AlW6tIIef8nmtnpQslIliYFAoJ8v1YczGlC_SvM7si1yvgcGPjaxsPhhGfCCrYsAH2q2UAQaPW5cPZpzNBMx3DPTW_ZvXjHl-HaykXR0PmHGY7-YMrJNLBTnfsf_TnZ3WXUgnu7JFLds2uluiG0EiRNCX0rqyWriIx51TMng0hTKqQMclZ8c1U3T9O-0Jd_VFnteUBWkK8y3tp5s1T22rU8-dPNo3ZZbqqy4rZSI-Ov7dtfrtPs743afPplyu6j-xmDZHyS5uwYMe6k0keOtEp8ePjsZz_wZwr-ZGfBiJW3Q3QPnIp7uIcM_yEoqaovCLlJe_QwnG1ymB8PdS-KVQDKPkFikMUD72SuF54Z4pJBQ1KRQXSaF4hxROrpPCxN_LxC-FyTBKb5HCAOUTrxSeF-6ZQkJRk8LkIilM3iGF0-ukMPX3MvVLYTqMslukMED51CuFe-HBySNUNMlLL5K89PbJE_H1X0VPSHhSdqMrq7zeMuP9IVSxVPW9WL01C_Xd6MVhTP3x8cAdXmQKZV39bVp_uK_qr_YC_eFpH5gnvehxH1pwnOZtOj6ij-C4DQsUJk6dBLglwuBpH7jlVig97kO33OrQKXrhGX7hGW414fWoT7lwmCgXDlPlImiiXDhNlWuM3vgEv_BJn3JN-wSF91qkBE2YTdCU2xRO2E3g5DLFtyknKsZ77VPeWS1BnndWS5DnOE15jtOk5wROeY7jpOf4WuUjwvPOhgnyHN8wlOedFRPkOU5TnuM06TmBU57jOOk5vl5FTLw2dbZMiOcCXzJtz-e7D38DAAD__xe8Uow= query T SELECT url FROM [ EXPLAIN (DISTSQL) diff --git a/pkg/sql/opt/memo/extract.go b/pkg/sql/opt/memo/extract.go index 6d84071a34d6..5e58664ebea6 100644 --- a/pkg/sql/opt/memo/extract.go +++ b/pkg/sql/opt/memo/extract.go @@ -178,6 +178,32 @@ func ExtractJoinEqualityColumns( return leftEq, rightEq } +// ExtractJoinEqualityFilters returns the filters containing pairs of columns +// (one from the left side, one from the right side) which are constrained to +// be equal in a join (and have equivalent types). +func ExtractJoinEqualityFilters(leftCols, rightCols opt.ColSet, on FiltersExpr) FiltersExpr { + // We want to avoid allocating a new slice unless strictly necessary. + var newFilters FiltersExpr + for i := range on { + condition := on[i].Condition + ok, _, _ := isJoinEquality(leftCols, rightCols, condition) + if ok { + if newFilters != nil { + newFilters = append(newFilters, on[i]) + } + } else { + if newFilters == nil { + newFilters = make(FiltersExpr, i, len(on)-1) + copy(newFilters, on[:i]) + } + } + } + if newFilters != nil { + return newFilters + } + return on +} + func isVarEquality(condition opt.ScalarExpr) (leftVar, rightVar *VariableExpr, ok bool) { if eq, ok := condition.(*EqExpr); ok { if leftVar, ok := eq.Left.(*VariableExpr); ok { diff --git a/pkg/sql/opt/memo/logical_props_builder.go b/pkg/sql/opt/memo/logical_props_builder.go index 46f5b1d6c083..c5e136e94b2f 100644 --- a/pkg/sql/opt/memo/logical_props_builder.go +++ b/pkg/sql/opt/memo/logical_props_builder.go @@ -1586,6 +1586,14 @@ func ensureLookupJoinInputProps(join *LookupJoinExpr, sb *statisticsBuilder) *pr if relational.OutputCols.Empty() { md := join.Memo().Metadata() relational.OutputCols = join.Cols.Difference(join.Input.Relational().OutputCols) + + // Include the key columns in the output columns. + index := md.Table(join.Table).Index(join.Index) + for i := range join.KeyCols { + indexColID := join.Table.ColumnID(index.Column(i).Ordinal) + relational.OutputCols.Add(indexColID) + } + relational.NotNullCols = tableNotNullCols(md, join.Table) relational.NotNullCols.IntersectionWith(relational.OutputCols) relational.Cardinality = props.AnyCardinality @@ -1679,6 +1687,8 @@ type joinPropsHelper struct { filterNotNullCols opt.ColSet filterIsTrue bool filterIsFalse bool + + selfJoinCols opt.ColSet } func (h *joinPropsHelper) init(b *logicalPropsBuilder, joinExpr RelExpr) { @@ -1702,6 +1712,10 @@ func (h *joinPropsHelper) init(b *logicalPropsBuilder, joinExpr RelExpr) { h.filterNotNullCols.Add(colID) h.filterNotNullCols.Add(indexColID) h.filtersFD.AddEquivalency(colID, indexColID) + if colID == indexColID { + // This can happen if an index join was converted into a lookup join. + h.selfJoinCols.Add(colID) + } } // Lookup join has implicit equality conditions on KeyCols. diff --git a/pkg/sql/opt/memo/memo.go b/pkg/sql/opt/memo/memo.go index 634091c028f2..b7593e72b5ba 100644 --- a/pkg/sql/opt/memo/memo.go +++ b/pkg/sql/opt/memo/memo.go @@ -358,6 +358,17 @@ func (m *Memo) RequestColStat( return nil, false } +// RowsProcessed calculates and returns the number of rows processed by the +// relational expression. It is currently only supported for joins. +func (m *Memo) RowsProcessed(expr RelExpr) (_ float64, ok bool) { + // When SetRoot is called, the statistics builder may have been cleared. + // If this happens, we can't serve the request anymore. + if m.logPropsBuilder.sb.md != nil { + return m.logPropsBuilder.sb.rowsProcessed(expr), true + } + return 0, false +} + // NextWithID returns a not-yet-assigned identifier for a WITH expression. func (m *Memo) NextWithID() opt.WithID { m.curWithID++ diff --git a/pkg/sql/opt/memo/statistics_builder.go b/pkg/sql/opt/memo/statistics_builder.go index 821fb295cddb..8699648a4a01 100644 --- a/pkg/sql/opt/memo/statistics_builder.go +++ b/pkg/sql/opt/memo/statistics_builder.go @@ -259,6 +259,7 @@ func (sb *statisticsBuilder) colStatFromInput(colSet opt.ColSet, e RelExpr) *pro } else { leftProps = e.Child(0).(RelExpr).Relational() } + intersectsLeft := leftProps.OutputCols.Intersects(colSet) var intersectsRight bool if lookupJoin != nil { @@ -268,13 +269,11 @@ func (sb *statisticsBuilder) colStatFromInput(colSet opt.ColSet, e RelExpr) *pro } else { intersectsRight = e.Child(1).(RelExpr).Relational().OutputCols.Intersects(colSet) } + + // It's possible that colSet intersects both left and right if we have a + // lookup join that was converted from an index join, so check the left + // side first. if intersectsLeft { - if intersectsRight { - // TODO(radu): what if both sides have columns in colSet? - panic(errors.AssertionFailedf( - "colSet %v contains both left and right columns", log.Safe(colSet), - )) - } if zigzagJoin != nil { return sb.colStatTable(zigzagJoin.LeftTable, colSet) } @@ -829,19 +828,11 @@ func (sb *statisticsBuilder) buildJoin( rightCols := h.rightProps.OutputCols.Copy() equivReps := h.filtersFD.EquivReps() - // Estimating selectivity for semi-join and anti-join is error-prone. - // For now, just propagate stats from the left side. - switch h.joinType { - case opt.SemiJoinOp, opt.SemiJoinApplyOp, opt.AntiJoinOp, opt.AntiJoinApplyOp: - s.RowCount = leftStats.RowCount - s.Selectivity = 1 - return - } - // Shortcut if there are no ON conditions. Note that for lookup join, there // are implicit equality conditions on KeyCols. if h.filterIsTrue { s.RowCount = leftStats.RowCount * rightStats.RowCount + s.Selectivity = 1 switch h.joinType { case opt.InnerJoinOp, opt.InnerJoinApplyOp: case opt.LeftJoinOp, opt.LeftJoinApplyOp: @@ -856,13 +847,20 @@ func (sb *statisticsBuilder) buildJoin( // All rows from both sides should be in the result. s.RowCount = max(s.RowCount, leftStats.RowCount) s.RowCount = max(s.RowCount, rightStats.RowCount) + + case opt.SemiJoinOp, opt.SemiJoinApplyOp: + s.RowCount = leftStats.RowCount + + case opt.AntiJoinOp, opt.AntiJoinApplyOp: + s.RowCount = 0 + s.Selectivity = 0 } - s.Selectivity = 1 return } // Shortcut if the ON condition is false or there is a contradiction. if h.filters.IsFalse() { + s.Selectivity = 0 switch h.joinType { case opt.InnerJoinOp, opt.InnerJoinApplyOp: s.RowCount = 0 @@ -878,8 +876,14 @@ func (sb *statisticsBuilder) buildJoin( case opt.FullJoinOp: // All rows from both sides should be in the result. s.RowCount = leftStats.RowCount + rightStats.RowCount + + case opt.SemiJoinOp, opt.SemiJoinApplyOp: + s.RowCount = 0 + + case opt.AntiJoinOp, opt.AntiJoinApplyOp: + s.RowCount = leftStats.RowCount + s.Selectivity = 1 } - s.Selectivity = 0 return } @@ -902,9 +906,35 @@ func (sb *statisticsBuilder) buildJoin( // Calculate selectivity and row count // ----------------------------------- s.RowCount = leftStats.RowCount * rightStats.RowCount + + // Save the initial row count before ON conditions are applied. inputRowCount := s.RowCount + + switch h.joinType { + case opt.SemiJoinOp, opt.SemiJoinApplyOp, opt.AntiJoinOp, opt.AntiJoinApplyOp: + // Treat anti join as if it were a semi join for the selectivity + // calculations. It will be fixed below. + s.RowCount = leftStats.RowCount + inputRowCount = leftStats.RowCount + s.ApplySelectivity(sb.selectivityFromEquivalenciesSemiJoin( + equivReps, h.leftProps.OutputCols, h.rightProps.OutputCols, &h.filtersFD, join, s, + )) + + default: + if h.rightProps.FuncDeps.ColsAreStrictKey(h.selfJoinCols) { + // This is like an index join, so apply a selectivity that will result + // in leftStats.RowCount rows. + s.ApplySelectivity(1 / rightStats.RowCount) + } else { + // Add the self join columns to equivReps so they are included in the + // calculation for selectivityFromEquivalencies below. + equivReps.UnionWith(h.selfJoinCols) + } + + s.ApplySelectivity(sb.selectivityFromEquivalencies(equivReps, &h.filtersFD, join, s)) + } + s.ApplySelectivity(sb.selectivityFromDistinctCounts(constrainedCols, join, s)) - s.ApplySelectivity(sb.selectivityFromEquivalencies(equivReps, &h.filtersFD, join, s)) s.ApplySelectivity(sb.selectivityFromUnappliedConjuncts(numUnappliedConjuncts)) // Update distinct counts based on equivalencies; this should happen after @@ -914,16 +944,24 @@ func (sb *statisticsBuilder) buildJoin( // Update null counts for non-nullable columns. sb.updateNullCountsFromProps(join, relProps, inputRowCount) - s.ApplySelectivity(sb.joinSelectivityFromNullCounts( - constrainedCols, - join, - s, - inputRowCount, - leftCols, - leftStats.RowCount, - rightCols, - rightStats.RowCount, - )) + switch h.joinType { + case opt.SemiJoinOp, opt.SemiJoinApplyOp, opt.AntiJoinOp, opt.AntiJoinApplyOp: + // Keep only column stats from the left side. + s.ColStats.RemoveIntersecting(h.rightProps.OutputCols) + s.ApplySelectivity(sb.selectivityFromNullCounts(constrainedCols, join, s, inputRowCount)) + + default: + s.ApplySelectivity(sb.joinSelectivityFromNullCounts( + constrainedCols, + join, + s, + inputRowCount, + leftCols, + leftStats.RowCount, + rightCols, + rightStats.RowCount, + )) + } // The above calculation is for inner joins. Other joins need to remove stats // that involve outer columns. @@ -962,6 +1000,18 @@ func (sb *statisticsBuilder) buildJoin( s.RowCount = leftJoinRowCount + rightJoinRowCount - innerJoinRowCount } + // Fix the stats for anti join. + switch h.joinType { + case opt.AntiJoinOp, opt.AntiJoinApplyOp: + s.RowCount = max(leftStats.RowCount-s.RowCount, epsilon) + s.Selectivity = max(1-s.Selectivity, epsilon) + + // Converting column stats is error-prone. If any column stats are needed, + // colStatJoin will use the selectivity calculated above to estimate the + // column stats from the input. + s.ColStats.Clear() + } + // Loop through all colSets added in this step, and adjust null counts and // distinct counts. for i := 0; i < s.ColStats.Count(); i++ { @@ -2271,6 +2321,120 @@ func (sb *statisticsBuilder) shouldUseHistogram(relProps *props.Relational) bool return relProps.Cardinality.Max >= minCardinalityForHistogram } +// rowsProcessed calculates and returns the number of rows processed by the +// relational expression. It is currently only supported for joins. +func (sb *statisticsBuilder) rowsProcessed(e RelExpr) float64 { + semiAntiJoinToInnerJoin := func(joinType opt.Operator) opt.Operator { + switch joinType { + case opt.SemiJoinOp, opt.AntiJoinOp: + return opt.InnerJoinOp + case opt.SemiJoinApplyOp, opt.AntiJoinApplyOp: + return opt.InnerJoinApplyOp + default: + return joinType + } + } + + switch t := e.(type) { + case *LookupJoinExpr: + var lookupJoinPrivate *LookupJoinPrivate + switch t.JoinType { + case opt.SemiJoinOp, opt.SemiJoinApplyOp, opt.AntiJoinOp, opt.AntiJoinApplyOp: + // The number of rows processed for semi and anti joins is closer to the + // number of output rows for an equivalent inner join. + copy := t.LookupJoinPrivate + copy.JoinType = semiAntiJoinToInnerJoin(t.JoinType) + lookupJoinPrivate = © + + default: + if t.On.IsTrue() { + // If there are no additional ON filters, the number of rows processed + // equals the number of output rows. + return e.Relational().Stats.RowCount + } + lookupJoinPrivate = &t.LookupJoinPrivate + } + + // We need to determine the row count of the join before the + // ON conditions are applied. + withoutOn := e.Memo().MemoizeLookupJoin(t.Input, nil /* on */, lookupJoinPrivate) + return withoutOn.Relational().Stats.RowCount + + case *MergeJoinExpr: + var mergeJoinPrivate *MergeJoinPrivate + switch t.JoinType { + case opt.SemiJoinOp, opt.SemiJoinApplyOp, opt.AntiJoinOp, opt.AntiJoinApplyOp: + // The number of rows processed for semi and anti joins is closer to the + // number of output rows for an equivalent inner join. + copy := t.MergeJoinPrivate + copy.JoinType = semiAntiJoinToInnerJoin(t.JoinType) + mergeJoinPrivate = © + + default: + if t.On.IsTrue() { + // If there are no additional ON filters, the number of rows processed + // equals the number of output rows. + return e.Relational().Stats.RowCount + } + mergeJoinPrivate = &t.MergeJoinPrivate + } + + // We need to determine the row count of the join before the + // ON conditions are applied. + withoutOn := e.Memo().MemoizeMergeJoin(t.Left, t.Right, nil /* on */, mergeJoinPrivate) + return withoutOn.Relational().Stats.RowCount + + default: + if !opt.IsJoinOp(e) { + panic(errors.AssertionFailedf("rowsProcessed not supported for operator type %v", log.Safe(e.Op()))) + } + + leftCols := e.Child(0).(RelExpr).Relational().OutputCols + rightCols := e.Child(1).(RelExpr).Relational().OutputCols + filters := e.Child(2).(*FiltersExpr) + + // Remove ON conditions that are not equality conditions, + on := ExtractJoinEqualityFilters(leftCols, rightCols, *filters) + + switch t := e.(type) { + // The number of rows processed for semi and anti joins is closer to the + // number of output rows for an equivalent inner join. + case *SemiJoinExpr: + e = e.Memo().MemoizeInnerJoin(t.Left, t.Right, on, &t.JoinPrivate) + case *SemiJoinApplyExpr: + e = e.Memo().MemoizeInnerJoinApply(t.Left, t.Right, on, &t.JoinPrivate) + case *AntiJoinExpr: + e = e.Memo().MemoizeInnerJoin(t.Left, t.Right, on, &t.JoinPrivate) + case *AntiJoinApplyExpr: + e = e.Memo().MemoizeInnerJoinApply(t.Left, t.Right, on, &t.JoinPrivate) + + default: + if len(on) == len(*filters) { + // No filters were removed. + return e.Relational().Stats.RowCount + } + + switch t := e.(type) { + case *InnerJoinExpr: + e = e.Memo().MemoizeInnerJoin(t.Left, t.Right, on, &t.JoinPrivate) + case *InnerJoinApplyExpr: + e = e.Memo().MemoizeInnerJoinApply(t.Left, t.Right, on, &t.JoinPrivate) + case *LeftJoinExpr: + e = e.Memo().MemoizeLeftJoin(t.Left, t.Right, on, &t.JoinPrivate) + case *LeftJoinApplyExpr: + e = e.Memo().MemoizeLeftJoinApply(t.Left, t.Right, on, &t.JoinPrivate) + case *RightJoinExpr: + e = e.Memo().MemoizeRightJoin(t.Left, t.Right, on, &t.JoinPrivate) + case *FullJoinExpr: + e = e.Memo().MemoizeFullJoin(t.Left, t.Right, on, &t.JoinPrivate) + default: + panic(errors.AssertionFailedf("join type %v not handled", log.Safe(e.Op()))) + } + } + return e.Relational().Stats.RowCount + } +} + func min(a float64, b float64) float64 { if a < b { return a @@ -3002,6 +3166,60 @@ func (sb *statisticsBuilder) selectivityFromEquivalency( return selectivity } +// selectivityFromEquivalenciesSemiJoin determines the selectivity of equality +// constraints on a semi join. It must be called before applyEquivalencies. +func (sb *statisticsBuilder) selectivityFromEquivalenciesSemiJoin( + equivReps, leftOutputCols, rightOutputCols opt.ColSet, + filterFD *props.FuncDepSet, + e RelExpr, + s *props.Statistics, +) (selectivity float64) { + selectivity = 1.0 + equivReps.ForEach(func(i opt.ColumnID) { + equivGroup := filterFD.ComputeEquivGroup(i) + selectivity *= sb.selectivityFromEquivalencySemiJoin( + equivGroup, leftOutputCols, rightOutputCols, e, s, + ) + }) + return selectivity +} + +func (sb *statisticsBuilder) selectivityFromEquivalencySemiJoin( + equivGroup, leftOutputCols, rightOutputCols opt.ColSet, e RelExpr, s *props.Statistics, +) (selectivity float64) { + // Find the minimum (maximum) input distinct count for all columns in this + // equivalency group from the right (left). + minDistinctCountRight := math.MaxFloat64 + maxDistinctCountLeft := float64(0) + equivGroup.ForEach(func(i opt.ColumnID) { + // If any of the distinct counts were updated by the filter, we want to use + // the updated value. + colSet := opt.MakeColSet(i) + colStat, ok := s.ColStats.Lookup(colSet) + if !ok { + colStat = sb.colStatFromInput(colSet, e) + } + if leftOutputCols.Contains(i) { + if maxDistinctCountLeft < colStat.DistinctCount { + maxDistinctCountLeft = colStat.DistinctCount + } + } else if rightOutputCols.Contains(i) { + if minDistinctCountRight > colStat.DistinctCount { + minDistinctCountRight = colStat.DistinctCount + } + } + }) + if maxDistinctCountLeft > s.RowCount { + maxDistinctCountLeft = s.RowCount + } + + selectivity = 1.0 + if maxDistinctCountLeft > minDistinctCountRight { + selectivity = minDistinctCountRight / maxDistinctCountLeft + } + return selectivity +} + func (sb *statisticsBuilder) selectivityFromUnappliedConjuncts( numUnappliedConjuncts float64, ) (selectivity float64) { diff --git a/pkg/sql/opt/memo/testdata/stats/join b/pkg/sql/opt/memo/testdata/stats/join index 6a7752ded123..18a942c47b1d 100644 --- a/pkg/sql/opt/memo/testdata/stats/join +++ b/pkg/sql/opt/memo/testdata/stats/join @@ -316,17 +316,17 @@ SELECT * FROM xysd WHERE EXISTS (SELECT * FROM uv WHERE x=u) ---- semi-join (hash) ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) - ├── stats: [rows=5000] + ├── stats: [rows=500, distinct(1)=500, null(1)=0, distinct(4)=325.66078, null(4)=0] ├── key: (1) ├── 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) - │ ├── stats: [rows=5000] + │ ├── 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) - │ └── stats: [rows=10000] + │ └── stats: [rows=10000, distinct(5)=500, null(5)=0] └── filters └── x = u [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] @@ -336,17 +336,17 @@ SELECT * FROM xysd WHERE NOT EXISTS (SELECT * FROM uv WHERE x=u) ---- anti-join (hash) ├── columns: x:1(int!null) y:2(int) s:3(string) d:4(decimal!null) - ├── stats: [rows=5000] + ├── stats: [rows=4500] ├── key: (1) ├── 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) - │ ├── stats: [rows=5000] + │ ├── 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) - │ └── stats: [rows=10000] + │ └── stats: [rows=10000, distinct(5)=500, null(5)=0] └── filters └── x = u [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] @@ -405,21 +405,21 @@ GROUP BY y ---- project ├── columns: count:8(int) - ├── stats: [rows=400] + ├── stats: [rows=138.170075] └── group-by ├── columns: y:2(int) count_rows:8(int) ├── grouping columns: y:2(int) - ├── stats: [rows=400, distinct(2)=400, null(2)=0] + ├── stats: [rows=138.170075, distinct(2)=138.170075, null(2)=0] ├── key: (2) ├── fd: (2)-->(8) ├── semi-join (hash) │ ├── columns: x:1(int!null) y:2(int) - │ ├── stats: [rows=5000, distinct(2)=400, null(2)=0] + │ ├── stats: [rows=166.666667, distinct(1)=166.666667, null(1)=0, distinct(2)=138.170075, null(2)=0] │ ├── key: (1) │ ├── fd: (1)-->(2) │ ├── scan xysd │ │ ├── columns: x:1(int!null) y:2(int) - │ │ ├── stats: [rows=5000, distinct(2)=400, null(2)=0] + │ │ ├── stats: [rows=5000, distinct(1)=5000, null(1)=0, distinct(2)=400, null(2)=0] │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ ├── scan uv @@ -448,12 +448,12 @@ project ├── fd: (2)-->(8) ├── anti-join (hash) │ ├── columns: x:1(int!null) y:2(int) - │ ├── stats: [rows=5000, distinct(2)=400, null(2)=0] + │ ├── stats: [rows=4833.33333, distinct(2)=400, null(2)=0] │ ├── key: (1) │ ├── fd: (1)-->(2) │ ├── scan xysd │ │ ├── columns: x:1(int!null) y:2(int) - │ │ ├── stats: [rows=5000, distinct(2)=400, null(2)=0] + │ │ ├── stats: [rows=5000, distinct(1)=5000, null(1)=0, distinct(2)=400, null(2)=0] │ │ ├── key: (1) │ │ └── fd: (1)-->(2) │ ├── scan uv @@ -1244,7 +1244,32 @@ 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)=1] - ├── cost: 712.03 + ├── cost: 506.0506 + ├── key: (1,2) + ├── fd: (1,2)-->(3) + ├── 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) + │ ├── stats: [rows=100, distinct(1)=100, null(1)=0, distinct(2)=10, null(2)=0, distinct(3)=10, null(3)=1, distinct(1-3)=100, null(1-3)=1] + │ ├── cost: 106.02 + │ ├── key: (1,2) + │ ├── fd: (1,2)-->(3) + │ ├── prune: (1-3) + │ └── 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) +(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") ] + [ ] +) +---- +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)=1e-10] + ├── cost: 506.0506 ├── key: (1,2) ├── fd: (1,2)-->(3) ├── interesting orderings: (+1,+2) @@ -1257,3 +1282,55 @@ semi-join (lookup def) │ ├── prune: (1-3) │ └── 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) +(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") ] + [ (False) ] +) +---- +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] + ├── cost: 506.0606 + ├── key: (1,2) + ├── fd: (1,2)-->(3) + ├── 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) + │ ├── stats: [rows=100, distinct(1)=100, null(1)=0, distinct(2)=10, null(2)=0, distinct(3)=10, null(3)=1, distinct(1-3)=100, null(1-3)=1] + │ ├── cost: 106.02 + │ ├── key: (1,2) + │ ├── fd: (1,2)-->(3) + │ ├── prune: (1-3) + │ └── interesting orderings: (+1,+2) + └── 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) +(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") ] + [ (False) ] +) +---- +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)=1] + ├── cost: 506.0606 + ├── key: (1,2) + ├── fd: (1,2)-->(3) + ├── 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) + │ ├── stats: [rows=100, distinct(1)=100, null(1)=0, distinct(2)=10, null(2)=0, distinct(3)=10, null(3)=1, distinct(1-3)=100, null(1-3)=1] + │ ├── cost: 106.02 + │ ├── key: (1,2) + │ ├── fd: (1,2)-->(3) + │ ├── prune: (1-3) + │ └── interesting orderings: (+1,+2) + └── filters + └── false [type=bool] diff --git a/pkg/sql/opt/memo/testdata/stats/lookup-join b/pkg/sql/opt/memo/testdata/stats/lookup-join index e09947c73bbd..a714e8d0fa53 100644 --- a/pkg/sql/opt/memo/testdata/stats/lookup-join +++ b/pkg/sql/opt/memo/testdata/stats/lookup-join @@ -135,7 +135,7 @@ CREATE TABLE abc (a INT, b INT, c INT, PRIMARY KEY (a, c)) ---- exec-ddl -CREATE TABLE def (d INT, e INT, f INT, g FLOAT, PRIMARY KEY (f, e), INDEX e_idx (e) STORING (d)) +CREATE TABLE def (d INT, e INT, f INT, g FLOAT, PRIMARY KEY (f, e), INDEX e_idx (e) STORING (d), INDEX d_idx (d)) ---- # Set up the statistics as if the first table is much smaller than the second. @@ -152,6 +152,12 @@ ALTER TABLE abc INJECT STATISTICS '[ exec-ddl ALTER TABLE def INJECT STATISTICS '[ + { + "columns": ["d"], + "created_at": "2018-01-01 1:00:00.00000+00:00", + "row_count": 10000, + "distinct_count": 1000 + }, { "columns": ["e"], "created_at": "2018-01-01 1:00:00.00000+00:00", @@ -216,7 +222,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)=1, distinct(5)=63.2138954, null(5)=0, distinct(6)=100, null(6)=0, distinct(2,5,6)=100, null(2,5,6)=1] + ├── 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)=1] ├── key: (3,5,6) ├── fd: (1,3)-->(2), (5,6)-->(4), (1)==(6), (6)==(1) ├── scan abc @@ -228,18 +234,18 @@ inner-join (lookup def) # Check column statistics for double lookup join. opt colstat=7 -SELECT * FROM abc LEFT JOIN DEF ON a = e AND b = 3 +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] - ├── stats: [rows=1000, distinct(5)=100, null(5)=0, distinct(7)=632.304575, null(7)=10] + ├── stats: [rows=100, distinct(4)=100, null(4)=0, distinct(7)=95.1671064, null(7)=1] ├── key: (1,3,5,6) ├── fd: (1,3)-->(2), (5,6)-->(4,7) - ├── left-join (lookup def@e_idx) + ├── 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] = [5] - │ ├── stats: [rows=1000, distinct(5)=100, null(5)=0] + │ ├── key columns: [1] = [4] + │ ├── stats: [rows=100, distinct(4)=100, null(4)=0] │ ├── key: (1,3,5,6) │ ├── fd: (1,3)-->(2), (5,6)-->(4) │ ├── scan abc @@ -250,3 +256,94 @@ left-join (lookup def) │ └── filters │ └── b = 3 [type=bool, outer=(2), constraints=(/2: [/3 - /3]; tight), fd=()-->(2)] └── filters (true) + +# 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 +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=1000, distinct(5)=100, null(5)=0, distinct(7)=632.304575, null(7)=10] + ├── key: (1,3,5,6) + ├── fd: (1,3)-->(2), (5,6)-->(4,7) + ├── 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) + ├── 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 = e [type=bool, outer=(1,5), constraints=(/1: (/NULL - ]; /5: (/NULL - ]), fd=(1)==(5), (5)==(1)] + └── b = 3 [type=bool, outer=(2), constraints=(/2: [/3 - /3]; tight), fd=()-->(2)] + +exec-ddl +CREATE TABLE t (x INT, y INT, INDEX x_idx (x) STORING (y), INDEX y_idx (y) STORING (x), INDEX xy_idx (x, y)) +---- + +exec-ddl +CREATE TABLE u (x INT, y INT, INDEX x_idx (x) STORING (y), INDEX y_idx (y) STORING (x), INDEX xy_idx (x, y)) +---- + +exec-ddl +ALTER TABLE t INJECT STATISTICS '[ + { + "columns": ["x"], + "created_at": "2018-01-01 1:00:00.00000+00:00", + "row_count": 100, + "distinct_count": 10 + }, + { + "columns": ["y"], + "created_at": "2018-01-01 1:00:00.00000+00:00", + "row_count": 100, + "distinct_count": 10 + } +]' +---- + +exec-ddl +ALTER TABLE u INJECT STATISTICS '[ + { + "columns": ["x"], + "created_at": "2018-01-01 1:00:00.00000+00:00", + "row_count": 10, + "distinct_count": 2 + }, + { + "columns": ["y"], + "created_at": "2018-01-01 1:00:00.00000+00:00", + "row_count": 10, + "distinct_count": 2 + } +]' +---- + +# Test that the correct index is used for the lookup join. +opt +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] + ├── 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) + │ └── stats: [rows=10, distinct(1)=2, null(1)=0, distinct(2)=2, null(2)=0] + └── filters (true) + +opt +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] + ├── stats: [rows=1e-10] + ├── scan u + │ ├── columns: u.x:1(int) u.y:2(int) + │ └── stats: [rows=10, distinct(1)=2, null(1)=0, distinct(2)=2, null(2)=0] + └── filters (true) diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpcc b/pkg/sql/opt/memo/testdata/stats_quality/tpcc index df66e06826e7..da6920c26040 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpcc +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpcc @@ -614,7 +614,7 @@ scalar-group-by ├── columns: count:28(int) ├── cardinality: [1 - 1] ├── stats: [rows=1, distinct(28)=1, null(28)=0] - ├── cost: 1436.73853 + ├── cost: 1366.93732 ├── key: () ├── fd: ()-->(28) ├── prune: (28) @@ -623,7 +623,7 @@ scalar-group-by │ ├── 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] │ ├── stats: [rows=216.137889, distinct(1)=19.9995949, null(1)=0, distinct(2)=1, null(2)=0, distinct(3)=1, null(3)=0, distinct(5)=185.570315, null(5)=0, distinct(11)=185.570315, null(11)=0, distinct(12)=1, null(12)=0, distinct(13)=30.3089364, null(13)=0] - │ ├── cost: 1434.55715 + │ ├── cost: 1364.75594 │ ├── fd: ()-->(2,3,12), (11)-->(13), (5)==(11), (11)==(5), (3)==(12), (12)==(3) │ ├── interesting orderings: (+3,+2,-1) │ ├── scan order_line @@ -714,7 +714,7 @@ scalar-group-by ├── columns: count:22(int) ├── cardinality: [1 - 1] ├── stats: [rows=1, distinct(22)=1, null(22)=0] - ├── cost: 126.546667 + ├── cost: 126.623333 ├── key: () ├── fd: ()-->(22) ├── prune: (22) @@ -724,7 +724,7 @@ scalar-group-by │ ├── left ordering: +1 │ ├── right ordering: +11 │ ├── 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] - │ ├── cost: 126.493333 + │ ├── cost: 126.57 │ ├── key: (11) │ ├── fd: (1)-->(9), (11)-->(21), (1)==(11), (11)==(1) │ ├── scan warehouse diff --git a/pkg/sql/opt/memo/testdata/stats_quality/tpch b/pkg/sql/opt/memo/testdata/stats_quality/tpch index e650f1f6452d..b98fe0b5b1d2 100644 --- a/pkg/sql/opt/memo/testdata/stats_quality/tpch +++ b/pkg/sql/opt/memo/testdata/stats_quality/tpch @@ -4904,7 +4904,7 @@ 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) - ├── stats: [rows=3489.49147, distinct(9)=8.33333333, null(9)=0, distinct(10)=150, null(10)=0, distinct(11)=8, null(11)=0, distinct(22)=3489.49147, null(22)=0, distinct(9-11)=3489.49147, null(9-11)=0] + ├── 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 @@ -4912,13 +4912,13 @@ sort ├── 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) ├── grouping columns: p_brand:9(char!null) p_type:10(varchar!null) p_size:11(int!null) - ├── stats: [rows=3489.49147, distinct(9)=8.33333333, null(9)=0, distinct(10)=150, null(10)=0, distinct(11)=8, null(11)=0, distinct(22)=3489.49147, null(22)=0, distinct(9-11)=3489.49147, null(9-11)=0] + ├── 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) ├── inner-join (hash) │ ├── save-table-name: q16_inner_join_3 │ ├── 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) - │ ├── stats: [rows=14276.4012, distinct(1)=3555.43444, null(1)=0, distinct(2)=7567.69437, 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)=3489.49147, null(9-11)=0] + │ ├── 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] │ ├── key: (2,6) │ ├── fd: (6)-->(9-11), (1)==(6), (6)==(1) │ ├── anti-join (merge) @@ -4926,7 +4926,7 @@ sort │ │ ├── columns: ps_partkey:1(int!null) ps_suppkey:2(int!null) │ │ ├── left ordering: +2 │ │ ├── right ordering: +15 - │ │ ├── stats: [rows=800000, distinct(1)=199241, null(1)=0, distinct(2)=9920, null(2)=0] + │ │ ├── stats: [rows=531592.246, distinct(1)=196758.028, null(1)=0, distinct(2)=9920, null(2)=0] │ │ ├── key: (1,2) │ │ ├── scan partsupp@ps_sk │ │ │ ├── save-table-name: q16_scan_5 @@ -4991,10 +4991,10 @@ column_names row_count distinct_count null_count {supplier_cnt} 18314 15 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{p_brand} 3489.00 5.25 <== 8.00 3.00 <== 0.00 1.00 -{p_size} 3489.00 5.25 <== 8.00 1.00 0.00 1.00 -{p_type} 3489.00 5.25 <== 150.00 1.03 0.00 1.00 -{supplier_cnt} 3489.00 5.25 <== 3489.00 232.60 <== 0.00 1.00 +{p_brand} 3315.00 5.52 <== 8.00 3.00 <== 0.00 1.00 +{p_size} 3315.00 5.52 <== 8.00 1.00 0.00 1.00 +{p_type} 3315.00 5.52 <== 150.00 1.03 0.00 1.00 +{supplier_cnt} 3315.00 5.52 <== 3315.00 221.00 <== 0.00 1.00 stats table=q16_group_by_2 ---- @@ -5005,10 +5005,10 @@ column_names row_count distinct_count null_count {count} 18314 15 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{count} 3489.00 5.25 <== 3489.00 232.60 <== 0.00 1.00 -{p_brand} 3489.00 5.25 <== 8.00 3.00 <== 0.00 1.00 -{p_size} 3489.00 5.25 <== 8.00 1.00 0.00 1.00 -{p_type} 3489.00 5.25 <== 150.00 1.03 0.00 1.00 +{count} 3315.00 5.52 <== 3315.00 221.00 <== 0.00 1.00 +{p_brand} 3315.00 5.52 <== 8.00 3.00 <== 0.00 1.00 +{p_size} 3315.00 5.52 <== 8.00 1.00 0.00 1.00 +{p_type} 3315.00 5.52 <== 150.00 1.03 0.00 1.00 stats table=q16_inner_join_3 ---- @@ -5021,12 +5021,12 @@ column_names row_count distinct_count null_count {ps_suppkey} 118274 9916 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{p_brand} 14276.00 8.28 <== 8.00 3.00 <== 0.00 1.00 -{p_partkey} 14276.00 8.28 <== 3555.00 8.28 <== 0.00 1.00 -{p_size} 14276.00 8.28 <== 8.00 1.00 0.00 1.00 -{p_type} 14276.00 8.28 <== 150.00 1.03 0.00 1.00 -{ps_partkey} 14276.00 8.28 <== 3555.00 8.28 <== 0.00 1.00 -{ps_suppkey} 14276.00 8.28 <== 7568.00 1.31 0.00 1.00 +{p_brand} 9606.00 12.31 <== 8.00 3.00 <== 0.00 1.00 +{p_partkey} 9606.00 12.31 <== 3555.00 8.28 <== 0.00 1.00 +{p_size} 9606.00 12.31 <== 8.00 1.00 0.00 1.00 +{p_type} 9606.00 12.31 <== 150.00 1.03 0.00 1.00 +{ps_partkey} 9606.00 12.31 <== 3555.00 8.28 <== 0.00 1.00 +{ps_suppkey} 9606.00 12.31 <== 6153.00 1.61 0.00 1.00 stats table=q16_merge_join_4 ---- @@ -5035,8 +5035,8 @@ column_names row_count distinct_count null_count {ps_suppkey} 799680 9916 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{ps_partkey} 800000.00 1.00 199241.00 1.00 0.00 1.00 -{ps_suppkey} 800000.00 1.00 9920.00 1.00 0.00 1.00 +{ps_partkey} 531592.00 1.50 196758.00 1.01 0.00 1.00 +{ps_suppkey} 531592.00 1.50 9920.00 1.00 0.00 1.00 stats table=q16_scan_5 ---- @@ -5407,7 +5407,7 @@ limit ├── sort │ ├── save-table-name: q18_sort_2 │ ├── columns: c_custkey:1(int) c_name:2(varchar) o_orderkey:9(int!null) o_totalprice:12(float) o_orderdate:13(date) sum:51(float) - │ ├── stats: [rows=1471426.19, distinct(1)=1471426.19, null(1)=0, distinct(2)=1471426.19, null(2)=0, distinct(9)=1471426.19, null(9)=0, distinct(12)=1471426.19, null(12)=0, distinct(13)=1471426.19, null(13)=0, distinct(51)=1471426.19, null(51)=0] + │ ├── 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 @@ -5415,36 +5415,30 @@ limit │ ├── save-table-name: q18_group_by_3 │ ├── columns: c_custkey:1(int) c_name:2(varchar) o_orderkey:9(int!null) o_totalprice:12(float) o_orderdate:13(date) sum:51(float) │ ├── grouping columns: o_orderkey:9(int!null) - │ ├── stats: [rows=1471426.19, distinct(1)=1471426.19, null(1)=0, distinct(2)=1471426.19, null(2)=0, distinct(9)=1471426.19, null(9)=0, distinct(12)=1471426.19, null(12)=0, distinct(13)=1471426.19, null(13)=0, distinct(51)=1471426.19, null(51)=0] + │ ├── 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) │ ├── 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) - │ │ ├── stats: [rows=5941074.68, distinct(1)=99846, null(1)=0, distinct(2)=150000, null(2)=0, distinct(9)=1471426.19, null(9)=0, distinct(10)=99846, null(10)=0, distinct(12)=1410750.85, null(12)=0, distinct(13)=2406, null(13)=0, distinct(18)=1471426.19, null(18)=0, distinct(22)=50, null(22)=0] + │ │ ├── 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) - │ │ ├── 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 - │ │ │ <--- 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) - │ │ │ ├── stats: [rows=1511964.68, distinct(1)=99846, null(1)=0, distinct(2)=149993.712, null(2)=0, distinct(9)=952566.744, null(9)=0, distinct(10)=99846, null(10)=0, distinct(12)=941447.245, null(12)=0, distinct(13)=2406, null(13)=0] - │ │ │ ├── key: (9) - │ │ │ ├── fd: (9)-->(10,12,13), (1)-->(2), (1)==(10), (10)==(1) + │ │ ├── inner-join (lookup lineitem) + │ │ │ ├── save-table-name: q18_lookup_join_5 + │ │ │ ├── columns: 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) + │ │ │ ├── key columns: [9] = [18] + │ │ │ ├── stats: [rows=2000405, distinct(9)=509090, null(9)=0, distinct(10)=99649.071, null(10)=0, distinct(12)=496607.042, null(12)=0, distinct(13)=2406, null(13)=0, distinct(18)=509090, null(18)=0, distinct(22)=50, null(22)=0] + │ │ │ ├── fd: (9)-->(10,12,13), (9)==(18), (18)==(9) │ │ │ ├── semi-join (merge) - │ │ │ │ ├── save-table-name: q18_merge_join_7 + │ │ │ │ ├── save-table-name: q18_merge_join_6 │ │ │ │ ├── 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=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] + │ │ │ │ ├── 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) │ │ │ │ ├── scan orders - │ │ │ │ │ ├── save-table-name: q18_scan_8 + │ │ │ │ │ ├── save-table-name: q18_scan_7 │ │ │ │ │ ├── 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 @@ -5457,14 +5451,14 @@ limit │ │ │ │ │ ├── fd: (9)-->(10,12,13) │ │ │ │ │ └── ordering: +9 │ │ │ │ ├── select - │ │ │ │ │ ├── save-table-name: q18_select_9 + │ │ │ │ │ ├── save-table-name: q18_select_8 │ │ │ │ │ ├── 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 │ │ │ │ │ ├── group-by - │ │ │ │ │ │ ├── save-table-name: q18_group_by_10 + │ │ │ │ │ │ ├── save-table-name: q18_group_by_9 │ │ │ │ │ │ ├── columns: l_orderkey:34(int!null) sum:50(float) │ │ │ │ │ │ ├── grouping columns: l_orderkey:34(int!null) │ │ │ │ │ │ ├── stats: [rows=1527270, distinct(34)=1527270, null(34)=0, distinct(50)=1527270, null(50)=0] @@ -5472,7 +5466,7 @@ limit │ │ │ │ │ │ ├── fd: (34)-->(50) │ │ │ │ │ │ ├── ordering: +34 │ │ │ │ │ │ ├── scan lineitem - │ │ │ │ │ │ │ ├── save-table-name: q18_scan_11 + │ │ │ │ │ │ │ ├── save-table-name: q18_scan_10 │ │ │ │ │ │ │ ├── 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 @@ -5484,18 +5478,17 @@ limit │ │ │ │ │ └── filters │ │ │ │ │ └── sum > 300.0 [type=bool, outer=(50), constraints=(/50: [/300.00000000000006 - ]; tight)] │ │ │ │ └── filters (true) - │ │ │ ├── scan customer - │ │ │ │ ├── save-table-name: q18_scan_12 - │ │ │ │ ├── columns: c_custkey:1(int!null) c_name:2(varchar!null) - │ │ │ │ ├── stats: [rows=150000, distinct(1)=148813, null(1)=0, distinct(2)=150000, null(2)=0] - │ │ │ │ │ histogram(1)= 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 - │ │ │ │ ├── key: (1) - │ │ │ │ └── fd: (1)-->(2) - │ │ │ └── filters - │ │ │ └── c_custkey = o_custkey [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] + │ │ │ └── filters (true) + │ │ ├── scan customer + │ │ │ ├── save-table-name: q18_scan_11 + │ │ │ ├── columns: c_custkey:1(int!null) c_name:2(varchar!null) + │ │ │ ├── stats: [rows=150000, distinct(1)=148813, null(1)=0, distinct(2)=150000, null(2)=0] + │ │ │ │ histogram(1)= 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 + │ │ │ ├── key: (1) + │ │ │ └── fd: (1)-->(2) │ │ └── filters - │ │ └── o_orderkey = l_orderkey [type=bool, outer=(9,18), constraints=(/9: (/NULL - ]; /18: (/NULL - ]), fd=(9)==(18), (18)==(9)] + │ │ └── c_custkey = o_custkey [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] │ └── aggregations │ ├── sum [type=float, outer=(22)] │ │ └── variable: l_quantity [type=float] @@ -5538,12 +5531,12 @@ column_names row_count distinct_count null_count {sum} 57 18 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{c_custkey} 1471426.00 25814.49 <== 1471426.00 25814.49 <== 0.00 1.00 -{c_name} 1471426.00 25814.49 <== 1471426.00 25814.49 <== 0.00 1.00 -{o_orderdate} 1471426.00 25814.49 <== 1471426.00 25814.49 <== 0.00 1.00 -{o_orderkey} 1471426.00 25814.49 <== 1471426.00 25814.49 <== 0.00 1.00 -{o_totalprice} 1471426.00 25814.49 <== 1471426.00 25814.49 <== 0.00 1.00 -{sum} 1471426.00 25814.49 <== 1471426.00 81745.89 <== 0.00 1.00 +{c_custkey} 499392.00 8761.26 <== 499392.00 8761.26 <== 0.00 1.00 +{c_name} 499392.00 8761.26 <== 499392.00 8761.26 <== 0.00 1.00 +{o_orderdate} 499392.00 8761.26 <== 499392.00 8761.26 <== 0.00 1.00 +{o_orderkey} 499392.00 8761.26 <== 499392.00 8761.26 <== 0.00 1.00 +{o_totalprice} 499392.00 8761.26 <== 499392.00 8761.26 <== 0.00 1.00 +{sum} 499392.00 8761.26 <== 499392.00 27744.00 <== 0.00 1.00 stats table=q18_group_by_3 ---- @@ -5556,12 +5549,12 @@ column_names row_count distinct_count null_count {sum} 57 18 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{c_custkey} 1471426.00 25814.49 <== 1471426.00 25814.49 <== 0.00 1.00 -{c_name} 1471426.00 25814.49 <== 1471426.00 25814.49 <== 0.00 1.00 -{o_orderdate} 1471426.00 25814.49 <== 1471426.00 25814.49 <== 0.00 1.00 -{o_orderkey} 1471426.00 25814.49 <== 1471426.00 25814.49 <== 0.00 1.00 -{o_totalprice} 1471426.00 25814.49 <== 1471426.00 25814.49 <== 0.00 1.00 -{sum} 1471426.00 25814.49 <== 1471426.00 81745.89 <== 0.00 1.00 +{c_custkey} 499392.00 8761.26 <== 499392.00 8761.26 <== 0.00 1.00 +{c_name} 499392.00 8761.26 <== 499392.00 8761.26 <== 0.00 1.00 +{o_orderdate} 499392.00 8761.26 <== 499392.00 8761.26 <== 0.00 1.00 +{o_orderkey} 499392.00 8761.26 <== 499392.00 8761.26 <== 0.00 1.00 +{o_totalprice} 499392.00 8761.26 <== 499392.00 8761.26 <== 0.00 1.00 +{sum} 499392.00 8761.26 <== 499392.00 27744.00 <== 0.00 1.00 stats table=q18_inner_join_4 ---- @@ -5576,44 +5569,34 @@ column_names row_count distinct_count null_count {o_totalprice} 399 57 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{c_custkey} 5941075.00 14889.91 <== 99846.00 1751.68 <== 0.00 1.00 -{c_name} 5941075.00 14889.91 <== 150000.00 2631.58 <== 0.00 1.00 -{l_orderkey} 5941075.00 14889.91 <== 1471426.00 25814.49 <== 0.00 1.00 -{l_quantity} 5941075.00 14889.91 <== 50.00 1.85 0.00 1.00 -{o_custkey} 5941075.00 14889.91 <== 99846.00 1751.68 <== 0.00 1.00 -{o_orderdate} 5941075.00 14889.91 <== 2406.00 42.21 <== 0.00 1.00 -{o_orderkey} 5941075.00 14889.91 <== 1471426.00 25814.49 <== 0.00 1.00 -{o_totalprice} 5941075.00 14889.91 <== 1410751.00 24750.02 <== 0.00 1.00 +{c_custkey} 2016361.00 5053.54 <== 99649.00 1748.23 <== 0.00 1.00 +{c_name} 2016361.00 5053.54 <== 150000.00 2631.58 <== 0.00 1.00 +{l_orderkey} 2016361.00 5053.54 <== 499392.00 8761.26 <== 0.00 1.00 +{l_quantity} 2016361.00 5053.54 <== 50.00 1.85 0.00 1.00 +{o_custkey} 2016361.00 5053.54 <== 99649.00 1748.23 <== 0.00 1.00 +{o_orderdate} 2016361.00 5053.54 <== 2406.00 42.21 <== 0.00 1.00 +{o_orderkey} 2016361.00 5053.54 <== 499392.00 8761.26 <== 0.00 1.00 +{o_totalprice} 2016361.00 5053.54 <== 488044.00 8562.18 <== 0.00 1.00 -stats table=q18_scan_5 ----- -column_names row_count distinct_count null_count -{l_orderkey} 6001215 1527270 0 -{l_quantity} 6001215 50 0 -~~~~ -column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{l_orderkey} 6001215.00 1.00 1527270.00 1.00 0.00 1.00 -{l_quantity} 6001215.00 1.00 50.00 1.00 0.00 1.00 - -stats table=q18_inner_join_6 +stats table=q18_lookup_join_5 ---- column_names row_count distinct_count null_count -{c_custkey} 57 57 0 -{c_name} 57 57 0 -{o_custkey} 57 57 0 -{o_orderdate} 57 57 0 -{o_orderkey} 57 57 0 -{o_totalprice} 57 57 0 +{l_orderkey} 399 57 0 +{l_quantity} 399 27 0 +{o_custkey} 399 57 0 +{o_orderdate} 399 57 0 +{o_orderkey} 399 57 0 +{o_totalprice} 399 57 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{c_custkey} 1511965.00 26525.70 <== 99846.00 1751.68 <== 0.00 1.00 -{c_name} 1511965.00 26525.70 <== 149994.00 2631.47 <== 0.00 1.00 -{o_custkey} 1511965.00 26525.70 <== 99846.00 1751.68 <== 0.00 1.00 -{o_orderdate} 1511965.00 26525.70 <== 2406.00 42.21 <== 0.00 1.00 -{o_orderkey} 1511965.00 26525.70 <== 952567.00 16711.70 <== 0.00 1.00 -{o_totalprice} 1511965.00 26525.70 <== 941447.00 16516.61 <== 0.00 1.00 +{l_orderkey} 2000405.00 5013.55 <== 509090.00 8931.40 <== 0.00 1.00 +{l_quantity} 2000405.00 5013.55 <== 50.00 1.85 0.00 1.00 +{o_custkey} 2000405.00 5013.55 <== 99649.00 1748.23 <== 0.00 1.00 +{o_orderdate} 2000405.00 5013.55 <== 2406.00 42.21 <== 0.00 1.00 +{o_orderkey} 2000405.00 5013.55 <== 509090.00 8931.40 <== 0.00 1.00 +{o_totalprice} 2000405.00 5013.55 <== 496607.00 8712.40 <== 0.00 1.00 -stats table=q18_merge_join_7 +stats table=q18_merge_join_6 ---- column_names row_count distinct_count null_count {o_custkey} 57 57 0 @@ -5622,12 +5605,12 @@ column_names row_count distinct_count null_count {o_totalprice} 57 57 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{o_custkey} 1500000.00 26315.79 <== 99846.00 1751.68 <== 0.00 1.00 -{o_orderdate} 1500000.00 26315.79 <== 2406.00 42.21 <== 0.00 1.00 -{o_orderkey} 1500000.00 26315.79 <== 1500000.00 26315.79 <== 0.00 1.00 -{o_totalprice} 1500000.00 26315.79 <== 1459167.00 25599.42 <== 0.00 1.00 +{o_custkey} 509090.00 8931.40 <== 99649.00 1748.23 <== 0.00 1.00 +{o_orderdate} 509090.00 8931.40 <== 2406.00 42.21 <== 0.00 1.00 +{o_orderkey} 509090.00 8931.40 <== 509090.00 8931.40 <== 0.00 1.00 +{o_totalprice} 509090.00 8931.40 <== 506350.00 8883.33 <== 0.00 1.00 -stats table=q18_scan_8 +stats table=q18_scan_7 ---- column_names row_count distinct_count null_count {o_custkey} 1500000 99846 0 @@ -5641,7 +5624,7 @@ column_names row_count_est row_count_err distinct_count_est distinct_count {o_orderkey} 1500000.00 1.00 1500000.00 1.02 0.00 1.00 {o_totalprice} 1500000.00 1.00 1459167.00 1.00 0.00 1.00 -stats table=q18_select_9 +stats table=q18_select_8 ---- column_names row_count distinct_count null_count {l_orderkey} 57 57 0 @@ -5651,7 +5634,7 @@ column_names row_count_est row_count_err distinct_count_est distinct_count_e {l_orderkey} 509090.00 8931.40 <== 509090.00 8931.40 <== 0.00 1.00 {sum} 509090.00 8931.40 <== 509090.00 28282.78 <== 0.00 1.00 -stats table=q18_group_by_10 +stats table=q18_group_by_9 ---- column_names row_count distinct_count null_count {l_orderkey} 1500000 1527270 0 @@ -5661,7 +5644,7 @@ column_names row_count_est row_count_err distinct_count_est distinct_count_e {l_orderkey} 1527270.00 1.02 1527270.00 1.00 0.00 1.00 {sum} 1527270.00 1.02 1527270.00 4802.74 <== 0.00 1.00 -stats table=q18_scan_11 +stats table=q18_scan_10 ---- column_names row_count distinct_count null_count {l_orderkey} 6001215 1527270 0 @@ -5671,7 +5654,7 @@ column_names row_count_est row_count_err distinct_count_est distinct_count_e {l_orderkey} 6001215.00 1.00 1527270.00 1.00 0.00 1.00 {l_quantity} 6001215.00 1.00 50.00 1.00 0.00 1.00 -stats table=q18_scan_12 +stats table=q18_scan_11 ---- column_names row_count distinct_count null_count {c_custkey} 150000 148813 0 @@ -5947,22 +5930,22 @@ ORDER BY sort ├── save-table-name: q20_sort_1 ├── columns: s_name:2(char!null) s_address:3(varchar!null) - ├── stats: [rows=400, distinct(2)=399.991883, null(2)=0, distinct(3)=400, null(3)=0] + ├── stats: [rows=392.784801, distinct(2)=392.777418, null(2)=0, distinct(3)=392.784801, null(3)=0] ├── ordering: +2 └── project ├── save-table-name: q20_project_2 ├── columns: s_name:2(char!null) s_address:3(varchar!null) - ├── stats: [rows=400, distinct(2)=399.991883, null(2)=0, distinct(3)=400, null(3)=0] + ├── stats: [rows=392.784801, distinct(2)=392.777418, null(2)=0, distinct(3)=392.784801, 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) - ├── stats: [rows=400, distinct(1)=399.934613, null(1)=0, distinct(2)=399.991883, null(2)=0, distinct(3)=400, null(3)=0, distinct(4)=1, null(4)=0, distinct(8)=1, null(8)=0, distinct(9)=1, null(9)=0] + ├── stats: [rows=392.784801, distinct(1)=392.720593, null(1)=0, distinct(2)=392.777418, null(2)=0, distinct(3)=392.784801, null(3)=0, distinct(4)=1, null(4)=0, distinct(8)=1, null(8)=0, distinct(9)=1, null(9)=0] ├── key: (1) ├── fd: ()-->(9), (1)-->(2-4), (4)==(8), (8)==(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) - │ ├── stats: [rows=10000, distinct(1)=9920, null(1)=0, distinct(2)=9990, null(2)=0, distinct(3)=10000, null(3)=0, distinct(4)=25, null(4)=0] + │ ├── stats: [rows=9819.62002, distinct(1)=9741.06306, null(1)=0, distinct(2)=9810.52322, null(2)=0, distinct(3)=9819.62002, null(3)=0, distinct(4)=25, null(4)=0] │ ├── key: (1) │ ├── fd: (1)-->(2-4) │ ├── scan supplier @@ -5978,7 +5961,7 @@ sort │ ├── project │ │ ├── save-table-name: q20_project_6 │ │ ├── columns: ps_partkey:12(int!null) ps_suppkey:13(int!null) - │ │ ├── stats: [rows=266666.667, distinct(12)=160127.162, null(12)=0, distinct(13)=9920, null(13)=0] + │ │ ├── stats: [rows=36999.4864, distinct(12)=22217.3354, null(12)=0, distinct(13)=9741.06306, null(13)=0] │ │ ├── key: (12,13) │ │ └── project │ │ ├── save-table-name: q20_project_7 @@ -6091,8 +6074,8 @@ column_names row_count distinct_count null_count {s_name} 186 186 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{s_address} 400.00 2.15 <== 400.00 2.15 <== 0.00 1.00 -{s_name} 400.00 2.15 <== 400.00 2.15 <== 0.00 1.00 +{s_address} 393.00 2.11 <== 393.00 2.11 <== 0.00 1.00 +{s_name} 393.00 2.11 <== 393.00 2.11 <== 0.00 1.00 stats table=q20_project_2 ---- @@ -6101,8 +6084,8 @@ column_names row_count distinct_count null_count {s_name} 186 186 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{s_address} 400.00 2.15 <== 400.00 2.15 <== 0.00 1.00 -{s_name} 400.00 2.15 <== 400.00 2.15 <== 0.00 1.00 +{s_address} 393.00 2.11 <== 393.00 2.11 <== 0.00 1.00 +{s_name} 393.00 2.11 <== 393.00 2.11 <== 0.00 1.00 stats table=q20_inner_join_3 ---- @@ -6115,12 +6098,12 @@ column_names row_count distinct_count null_count {s_suppkey} 186 186 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{n_name} 400.00 2.15 <== 1.00 1.00 0.00 1.00 -{n_nationkey} 400.00 2.15 <== 1.00 1.00 0.00 1.00 -{s_address} 400.00 2.15 <== 400.00 2.15 <== 0.00 1.00 -{s_name} 400.00 2.15 <== 400.00 2.15 <== 0.00 1.00 -{s_nationkey} 400.00 2.15 <== 1.00 1.00 0.00 1.00 -{s_suppkey} 400.00 2.15 <== 400.00 2.15 <== 0.00 1.00 +{n_name} 393.00 2.11 <== 1.00 1.00 0.00 1.00 +{n_nationkey} 393.00 2.11 <== 1.00 1.00 0.00 1.00 +{s_address} 393.00 2.11 <== 393.00 2.11 <== 0.00 1.00 +{s_name} 393.00 2.11 <== 393.00 2.11 <== 0.00 1.00 +{s_nationkey} 393.00 2.11 <== 1.00 1.00 0.00 1.00 +{s_suppkey} 393.00 2.11 <== 393.00 2.11 <== 0.00 1.00 stats table=q20_semi_join_4 ---- @@ -6131,10 +6114,10 @@ column_names row_count distinct_count null_count {s_suppkey} 4397 4434 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{s_address} 10000.00 2.27 <== 10000.00 2.29 <== 0.00 1.00 -{s_name} 10000.00 2.27 <== 9990.00 2.28 <== 0.00 1.00 -{s_nationkey} 10000.00 2.27 <== 25.00 1.00 0.00 1.00 -{s_suppkey} 10000.00 2.27 <== 9920.00 2.24 <== 0.00 1.00 +{s_address} 9820.00 2.23 <== 9820.00 2.25 <== 0.00 1.00 +{s_name} 9820.00 2.23 <== 9811.00 2.24 <== 0.00 1.00 +{s_nationkey} 9820.00 2.23 <== 25.00 1.00 0.00 1.00 +{s_suppkey} 9820.00 2.23 <== 9741.00 2.20 <== 0.00 1.00 stats table=q20_scan_5 ---- @@ -6157,8 +6140,8 @@ column_names row_count distinct_count null_count {ps_suppkey} 5833 4434 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{ps_partkey} 266667.00 45.72 <== 160127.00 76.03 <== 0.00 1.00 -{ps_suppkey} 266667.00 45.72 <== 9920.00 2.24 <== 0.00 1.00 +{ps_partkey} 36999.00 6.34 <== 22217.00 10.55 <== 0.00 1.00 +{ps_suppkey} 36999.00 6.34 <== 9741.00 2.20 <== 0.00 1.00 stats table=q20_project_7 ---- @@ -6370,7 +6353,7 @@ limit ├── sort │ ├── save-table-name: q21_sort_2 │ ├── columns: s_name:2(char!null) count_rows:69(int) - │ ├── stats: [rows=9628.02122, distinct(2)=9628.02122, null(2)=0, distinct(69)=9628.02122, null(69)=0] + │ ├── stats: [rows=8329.18391, distinct(2)=8329.18391, null(2)=0, distinct(69)=8329.18391, null(69)=0] │ ├── key: (2) │ ├── fd: (2)-->(69) │ ├── ordering: -69,+2 @@ -6378,30 +6361,30 @@ limit │ ├── save-table-name: q21_group_by_3 │ ├── columns: s_name:2(char!null) count_rows:69(int) │ ├── grouping columns: s_name:2(char!null) - │ ├── stats: [rows=9628.02122, distinct(2)=9628.02122, null(2)=0, distinct(69)=9628.02122, null(69)=0] + │ ├── stats: [rows=8329.18391, distinct(2)=8329.18391, null(2)=0, distinct(69)=8329.18391, null(69)=0] │ ├── key: (2) │ ├── fd: (2)-->(69) │ ├── 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] - │ │ ├── stats: [rows=33144.2984, distinct(1)=9920, null(1)=0, distinct(2)=9628.02122, null(2)=0, distinct(4)=1, null(4)=0, distinct(8)=32069.6931, null(8)=0, distinct(10)=9920, null(10)=0, distinct(19)=2465.99641, null(19)=0, distinct(20)=2553.9941, null(20)=0, distinct(24)=32069.6931, null(24)=0, distinct(26)=1, null(26)=0, distinct(33)=1, null(33)=0, distinct(34)=1, null(34)=0] + │ │ ├── stats: [rows=17924.776, distinct(1)=9920, null(1)=0, distinct(2)=8329.18391, null(2)=0, distinct(4)=1, null(4)=0, distinct(8)=17568.2329, null(8)=0, distinct(10)=9920, null(10)=0, distinct(19)=2464.28129, null(19)=0, distinct(20)=2551.71335, null(20)=0, distinct(24)=17568.2329, 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) │ │ ├── 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) - │ │ │ ├── stats: [rows=80661.4919, distinct(1)=9920, null(1)=0, distinct(2)=9986.88851, null(2)=0, distinct(4)=1, null(4)=0, distinct(8)=78046.2829, null(8)=0, distinct(10)=9920, null(10)=0, distinct(19)=2466, null(19)=0, distinct(20)=2554, null(20)=0, distinct(33)=1, null(33)=0, distinct(34)=1, null(34)=0] + │ │ │ ├── stats: [rows=17924.776, distinct(1)=9920, null(1)=0, distinct(2)=8329.18391, null(2)=0, distinct(4)=1, null(4)=0, distinct(8)=17568.2329, null(8)=0, distinct(10)=9920, null(10)=0, distinct(19)=2464.28129, null(19)=0, distinct(20)=2551.71335, null(20)=0, distinct(33)=1, null(33)=0, distinct(34)=1, null(34)=0] │ │ │ ├── fd: ()-->(34), (1)-->(2,4), (1)==(10), (10)==(1), (4)==(33), (33)==(4) │ │ │ ├── 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=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] + │ │ │ │ ├── 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] │ │ │ │ ├── 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=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] + │ │ │ │ │ ├── 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] │ │ │ │ │ ├── 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) @@ -6512,8 +6495,8 @@ column_names row_count distinct_count null_count {s_name} 100 100 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{count_rows} 9628.00 96.28 <== 9628.00 1203.50 <== 0.00 1.00 -{s_name} 9628.00 96.28 <== 9628.00 96.28 <== 0.00 1.00 +{count_rows} 8329.00 83.29 <== 8329.00 1041.12 <== 0.00 1.00 +{s_name} 8329.00 83.29 <== 8329.00 83.29 <== 0.00 1.00 stats table=q21_group_by_3 ---- @@ -6522,8 +6505,8 @@ column_names row_count distinct_count null_count {s_name} 411 411 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{count_rows} 9628.00 23.43 <== 9628.00 566.35 <== 0.00 1.00 -{s_name} 9628.00 23.43 <== 9628.00 23.43 <== 0.00 1.00 +{count_rows} 8329.00 20.27 <== 8329.00 489.94 <== 0.00 1.00 +{s_name} 8329.00 20.27 <== 8329.00 20.27 <== 0.00 1.00 stats table=q21_lookup_join_4 ---- @@ -6541,17 +6524,17 @@ column_names row_count distinct_count null_count {s_suppkey} 4141 411 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{l_commitdate} 33144.00 8.00 <== 2466.00 2.08 <== 0.00 1.00 -{l_orderkey} 33144.00 8.00 <== 32070.00 7.77 <== 0.00 1.00 -{l_receiptdate} 33144.00 8.00 <== 2554.00 2.18 <== 0.00 1.00 -{l_suppkey} 33144.00 8.00 <== 9920.00 24.14 <== 0.00 1.00 -{n_name} 33144.00 8.00 <== 1.00 1.00 0.00 1.00 -{n_nationkey} 33144.00 8.00 <== 1.00 1.00 0.00 1.00 -{o_orderkey} 33144.00 8.00 <== 32070.00 7.77 <== 0.00 1.00 -{o_orderstatus} 33144.00 8.00 <== 1.00 1.00 0.00 1.00 -{s_name} 33144.00 8.00 <== 9628.00 23.43 <== 0.00 1.00 -{s_nationkey} 33144.00 8.00 <== 1.00 1.00 0.00 1.00 -{s_suppkey} 33144.00 8.00 <== 9920.00 24.14 <== 0.00 1.00 +{l_commitdate} 17925.00 4.33 <== 2464.00 2.07 <== 0.00 1.00 +{l_orderkey} 17925.00 4.33 <== 17568.00 4.26 <== 0.00 1.00 +{l_receiptdate} 17925.00 4.33 <== 2552.00 2.17 <== 0.00 1.00 +{l_suppkey} 17925.00 4.33 <== 9920.00 24.14 <== 0.00 1.00 +{n_name} 17925.00 4.33 <== 1.00 1.00 0.00 1.00 +{n_nationkey} 17925.00 4.33 <== 1.00 1.00 0.00 1.00 +{o_orderkey} 17925.00 4.33 <== 17568.00 4.26 <== 0.00 1.00 +{o_orderstatus} 17925.00 4.33 <== 1.00 1.00 0.00 1.00 +{s_name} 17925.00 4.33 <== 8329.00 20.27 <== 0.00 1.00 +{s_nationkey} 17925.00 4.33 <== 1.00 1.00 0.00 1.00 +{s_suppkey} 17925.00 4.33 <== 9920.00 24.14 <== 0.00 1.00 stats table=q21_inner_join_5 ---- @@ -6567,15 +6550,15 @@ column_names row_count distinct_count null_count {s_suppkey} 8357 411 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{l_commitdate} 80661.00 9.65 <== 2466.00 1.05 0.00 1.00 -{l_orderkey} 80661.00 9.65 <== 78046.00 9.35 <== 0.00 1.00 -{l_receiptdate} 80661.00 9.65 <== 2554.00 1.07 0.00 1.00 -{l_suppkey} 80661.00 9.65 <== 9920.00 24.14 <== 0.00 1.00 -{n_name} 80661.00 9.65 <== 1.00 1.00 0.00 1.00 -{n_nationkey} 80661.00 9.65 <== 1.00 1.00 0.00 1.00 -{s_name} 80661.00 9.65 <== 9987.00 24.30 <== 0.00 1.00 -{s_nationkey} 80661.00 9.65 <== 1.00 1.00 0.00 1.00 -{s_suppkey} 80661.00 9.65 <== 9920.00 24.14 <== 0.00 1.00 +{l_commitdate} 17925.00 2.14 <== 2464.00 1.05 0.00 1.00 +{l_orderkey} 17925.00 2.14 <== 17568.00 2.11 <== 0.00 1.00 +{l_receiptdate} 17925.00 2.14 <== 2552.00 1.07 0.00 1.00 +{l_suppkey} 17925.00 2.14 <== 9920.00 24.14 <== 0.00 1.00 +{n_name} 17925.00 2.14 <== 1.00 1.00 0.00 1.00 +{n_nationkey} 17925.00 2.14 <== 1.00 1.00 0.00 1.00 +{s_name} 17925.00 2.14 <== 8329.00 20.27 <== 0.00 1.00 +{s_nationkey} 17925.00 2.14 <== 1.00 1.00 0.00 1.00 +{s_suppkey} 17925.00 2.14 <== 9920.00 24.14 <== 0.00 1.00 stats table=q21_semi_join_6 ---- @@ -6586,10 +6569,10 @@ column_names row_count distinct_count null_count {l_suppkey} 202092 9920 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{l_commitdate} 2000405.00 9.90 <== 2466.00 1.00 0.00 1.00 -{l_orderkey} 2000405.00 9.90 <== 1216823.00 6.01 <== 0.00 1.00 -{l_receiptdate} 2000405.00 9.90 <== 2554.00 1.02 0.00 1.00 -{l_suppkey} 2000405.00 9.90 <== 9920.00 1.00 0.00 1.00 +{l_commitdate} 444534.00 2.20 <== 2466.00 1.00 0.00 1.00 +{l_orderkey} 444534.00 2.20 <== 444534.00 2.20 <== 0.00 1.00 +{l_receiptdate} 444534.00 2.20 <== 2554.00 1.02 0.00 1.00 +{l_suppkey} 444534.00 2.20 <== 9920.00 1.00 0.00 1.00 stats table=q21_merge_join_7 ---- @@ -6600,10 +6583,10 @@ column_names row_count distinct_count null_count {l_suppkey} 337680 9920 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{l_commitdate} 2000405.00 5.92 <== 2466.00 1.00 0.00 1.00 -{l_orderkey} 2000405.00 5.92 <== 1216823.00 3.60 <== 0.00 1.00 -{l_receiptdate} 2000405.00 5.92 <== 2554.00 1.02 0.00 1.00 -{l_suppkey} 2000405.00 5.92 <== 9920.00 1.00 0.00 1.00 +{l_commitdate} 1333603.00 3.95 <== 2466.00 1.00 0.00 1.00 +{l_orderkey} 1333603.00 3.95 <== 1016901.00 3.00 <== 0.00 1.00 +{l_receiptdate} 1333603.00 3.95 <== 2554.00 1.02 0.00 1.00 +{l_suppkey} 1333603.00 3.95 <== 9920.00 1.00 0.00 1.00 stats table=q21_select_8 ---- @@ -6770,79 +6753,80 @@ GROUP BY ORDER BY cntrycode; ---- -group-by - ├── save-table-name: q22_group_by_1 +sort + ├── save-table-name: q22_sort_1 ├── columns: cntrycode:27(string) numcust:28(int) totacctbal:29(float) - ├── grouping columns: cntrycode:27(string) - ├── stats: [rows=16666.6667, distinct(27)=16666.6667, null(27)=0, distinct(28)=16666.6667, null(28)=0, distinct(29)=16666.6667, null(29)=0] + ├── 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 - ├── sort - │ ├── save-table-name: q22_sort_2 - │ ├── columns: c_acctbal:6(float!null) cntrycode:27(string) - │ ├── stats: [rows=16666.6667, distinct(6)=16666.6667, null(6)=0, distinct(27)=16666.6667, null(27)=0] - │ ├── ordering: +27 - │ └── project - │ ├── save-table-name: q22_project_3 - │ ├── columns: cntrycode:27(string) c_acctbal:6(float!null) - │ ├── stats: [rows=16666.6667, distinct(6)=16666.6667, null(6)=0, distinct(27)=16666.6667, null(27)=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] - │ │ ├── stats: [rows=16666.6667, distinct(1)=16658.9936, null(1)=0, distinct(5)=16666.6667, null(5)=0, distinct(6)=16666.6667, null(6)=0] - │ │ ├── key: (1) - │ │ ├── fd: (1)-->(5,6) - │ │ ├── select - │ │ │ ├── save-table-name: q22_select_5 - │ │ │ ├── columns: c_custkey:1(int!null) c_phone:5(char!null) c_acctbal:6(float!null) - │ │ │ ├── stats: [rows=16666.6667, distinct(1)=16658.9936, null(1)=0, distinct(5)=16666.6667, null(5)=0, distinct(6)=16666.6667, null(6)=0] - │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(5,6) - │ │ │ ├── scan customer - │ │ │ │ ├── save-table-name: q22_scan_6 - │ │ │ │ ├── columns: c_custkey:1(int!null) c_phone:5(char!null) c_acctbal:6(float!null) - │ │ │ │ ├── stats: [rows=150000, distinct(1)=148813, null(1)=0, distinct(5)=150000, null(5)=0, distinct(6)=140628, null(6)=0] - │ │ │ │ │ histogram(1)= 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 - │ │ │ │ ├── key: (1) - │ │ │ │ └── fd: (1)-->(5,6) - │ │ │ └── filters - │ │ │ ├── substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31') [type=bool, outer=(5)] - │ │ │ └── gt [type=bool, outer=(6), subquery, constraints=(/6: (/NULL - ])] - │ │ │ ├── variable: c_acctbal [type=float] - │ │ │ └── subquery [type=float] - │ │ │ └── scalar-group-by - │ │ │ ├── save-table-name: q22_scalar_group_by_7 - │ │ │ ├── columns: avg:17(float) - │ │ │ ├── cardinality: [1 - 1] - │ │ │ ├── stats: [rows=1, distinct(17)=1, null(17)=0] - │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(17) - │ │ │ ├── select - │ │ │ │ ├── save-table-name: q22_select_8 - │ │ │ │ ├── columns: c_phone:13(char!null) c_acctbal:14(float!null) - │ │ │ │ ├── stats: [rows=16666.6667, distinct(13)=16666.6667, null(13)=0, distinct(14)=16666.6667, null(14)=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] - │ │ │ │ └── filters - │ │ │ │ ├── c_acctbal > 0.0 [type=bool, outer=(14), constraints=(/14: [/5e-324 - ]; tight)] - │ │ │ │ └── substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31') [type=bool, outer=(13)] - │ │ │ └── aggregations - │ │ │ └── avg [type=float, outer=(14)] - │ │ │ └── variable: c_acctbal [type=float] - │ │ └── filters (true) - │ └── projections - │ └── substring(c_phone, 1, 2) [type=string, outer=(5)] - └── aggregations - ├── count-rows [type=int] - └── sum [type=float, outer=(6)] - └── variable: c_acctbal [type=float] + └── group-by + ├── save-table-name: q22_group_by_2 + ├── columns: cntrycode:27(string) count_rows:28(int) sum:29(float) + ├── grouping columns: cntrycode:27(string) + ├── 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) + ├── project + │ ├── save-table-name: q22_project_3 + │ ├── columns: cntrycode:27(string) c_acctbal:6(float!null) + │ ├── stats: [rows=1e-10, distinct(6)=1e-10, null(6)=0, distinct(27)=1e-10, null(27)=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] + │ │ ├── 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) + │ │ ├── fd: (1)-->(5,6) + │ │ ├── select + │ │ │ ├── save-table-name: q22_select_5 + │ │ │ ├── columns: c_custkey:1(int!null) c_phone:5(char!null) c_acctbal:6(float!null) + │ │ │ ├── stats: [rows=16666.6667, distinct(1)=16658.9936, null(1)=0, distinct(5)=16666.6667, null(5)=0, distinct(6)=16666.6667, null(6)=0] + │ │ │ ├── key: (1) + │ │ │ ├── fd: (1)-->(5,6) + │ │ │ ├── scan customer + │ │ │ │ ├── save-table-name: q22_scan_6 + │ │ │ │ ├── columns: c_custkey:1(int!null) c_phone:5(char!null) c_acctbal:6(float!null) + │ │ │ │ ├── stats: [rows=150000, distinct(1)=148813, null(1)=0, distinct(5)=150000, null(5)=0, distinct(6)=140628, null(6)=0] + │ │ │ │ │ histogram(1)= 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 + │ │ │ │ ├── key: (1) + │ │ │ │ └── fd: (1)-->(5,6) + │ │ │ └── filters + │ │ │ ├── substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31') [type=bool, outer=(5)] + │ │ │ └── gt [type=bool, outer=(6), subquery, constraints=(/6: (/NULL - ])] + │ │ │ ├── variable: c_acctbal [type=float] + │ │ │ └── subquery [type=float] + │ │ │ └── scalar-group-by + │ │ │ ├── save-table-name: q22_scalar_group_by_7 + │ │ │ ├── columns: avg:17(float) + │ │ │ ├── cardinality: [1 - 1] + │ │ │ ├── stats: [rows=1, distinct(17)=1, null(17)=0] + │ │ │ ├── key: () + │ │ │ ├── fd: ()-->(17) + │ │ │ ├── select + │ │ │ │ ├── save-table-name: q22_select_8 + │ │ │ │ ├── columns: c_phone:13(char!null) c_acctbal:14(float!null) + │ │ │ │ ├── stats: [rows=16666.6667, distinct(13)=16666.6667, null(13)=0, distinct(14)=16666.6667, null(14)=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] + │ │ │ │ └── filters + │ │ │ │ ├── c_acctbal > 0.0 [type=bool, outer=(14), constraints=(/14: [/5e-324 - ]; tight)] + │ │ │ │ └── substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31') [type=bool, outer=(13)] + │ │ │ └── aggregations + │ │ │ └── avg [type=float, outer=(14)] + │ │ │ └── variable: c_acctbal [type=float] + │ │ └── filters (true) + │ └── projections + │ └── substring(c_phone, 1, 2) [type=string, outer=(5)] + └── aggregations + ├── count-rows [type=int] + └── sum [type=float, outer=(6)] + └── variable: c_acctbal [type=float] -stats table=q22_group_by_1 +stats table=q22_sort_1 ---- column_names row_count distinct_count null_count {cntrycode} 7 7 0 @@ -6850,19 +6834,21 @@ column_names row_count distinct_count null_count {totacctbal} 7 7 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{cntrycode} 16667.00 2381.00 <== 16667.00 2381.00 <== 0.00 1.00 -{numcust} 16667.00 2381.00 <== 16667.00 2381.00 <== 0.00 1.00 -{totacctbal} 16667.00 2381.00 <== 16667.00 2381.00 <== 0.00 1.00 +{cntrycode} 0.00 +Inf <== 0.00 +Inf <== 0.00 1.00 +{numcust} 0.00 +Inf <== 0.00 +Inf <== 0.00 1.00 +{totacctbal} 0.00 +Inf <== 0.00 +Inf <== 0.00 1.00 -stats table=q22_sort_2 +stats table=q22_group_by_2 ---- column_names row_count distinct_count null_count -{c_acctbal} 6384 6304 0 -{cntrycode} 6384 7 0 +{cntrycode} 7 7 0 +{count_rows} 7 7 0 +{sum} 7 7 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{c_acctbal} 16667.00 2.61 <== 16667.00 2.64 <== 0.00 1.00 -{cntrycode} 16667.00 2.61 <== 16667.00 2381.00 <== 0.00 1.00 +{cntrycode} 0.00 +Inf <== 0.00 +Inf <== 0.00 1.00 +{count_rows} 0.00 +Inf <== 0.00 +Inf <== 0.00 1.00 +{sum} 0.00 +Inf <== 0.00 +Inf <== 0.00 1.00 stats table=q22_project_3 ---- @@ -6871,8 +6857,8 @@ column_names row_count distinct_count null_count {cntrycode} 6384 7 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{c_acctbal} 16667.00 2.61 <== 16667.00 2.64 <== 0.00 1.00 -{cntrycode} 16667.00 2.61 <== 16667.00 2381.00 <== 0.00 1.00 +{c_acctbal} 0.00 +Inf <== 0.00 +Inf <== 0.00 1.00 +{cntrycode} 0.00 +Inf <== 0.00 +Inf <== 0.00 1.00 stats table=q22_lookup_join_4 ---- @@ -6882,9 +6868,9 @@ column_names row_count distinct_count null_count {c_phone} 6384 6428 0 ~~~~ column_names row_count_est row_count_err distinct_count_est distinct_count_err null_count_est null_count_err -{c_acctbal} 16667.00 2.61 <== 16667.00 2.64 <== 0.00 1.00 -{c_custkey} 16667.00 2.61 <== 16659.00 2.62 <== 0.00 1.00 -{c_phone} 16667.00 2.61 <== 16667.00 2.59 <== 0.00 1.00 +{c_acctbal} 0.00 +Inf <== 0.00 +Inf <== 0.00 1.00 +{c_custkey} 0.00 +Inf <== 0.00 +Inf <== 0.00 1.00 +{c_phone} 0.00 +Inf <== 0.00 +Inf <== 0.00 1.00 stats table=q22_select_5 ---- diff --git a/pkg/sql/opt/norm/testdata/rules/combo b/pkg/sql/opt/norm/testdata/rules/combo index 9ae0e1f031d7..92fc5fc544a7 100644 --- a/pkg/sql/opt/norm/testdata/rules/combo +++ b/pkg/sql/opt/norm/testdata/rules/combo @@ -21,7 +21,7 @@ SELECT s FROM a INNER JOIN xy ON a.k=xy.x AND i+1=10 ---- ================================================================================ Initial expression - Cost: 8836.73 + Cost: 15503.39 ================================================================================ project ├── columns: s:4(string) @@ -41,7 +41,7 @@ Initial expression └── (k = x) AND ((i + 1) = 10) [type=bool, outer=(1,2,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ])] ================================================================================ NormalizeCmpPlusConst - Cost: 8770.06 + Cost: 15470.06 ================================================================================ project ├── columns: s:4(string) @@ -63,7 +63,7 @@ NormalizeCmpPlusConst + └── (k = x) AND (i = (10 - 1)) [type=bool, outer=(1,2,6), constraints=(/1: (/NULL - ]; /2: (/NULL - ]; /6: (/NULL - ])] ================================================================================ FoldBinary - Cost: 2236.06 + Cost: 12203.06 ================================================================================ project ├── columns: s:4(string) @@ -85,7 +85,7 @@ FoldBinary + └── (k = x) AND (i = 9) [type=bool, outer=(1,2,6), constraints=(/1: (/NULL - ]; /2: [/9 - /9]; /6: (/NULL - ]), fd=()-->(2)] ================================================================================ SimplifyJoinFilters - Cost: 2170.26 + Cost: 2180.16 ================================================================================ project ├── columns: s:4(string) @@ -575,7 +575,7 @@ Initial expression └── y = i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] ================================================================================ HoistSelectExists - Cost: 2200.07 + Cost: 12190.07 ================================================================================ select ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) @@ -619,7 +619,7 @@ HoistSelectExists + └── filters (true) ================================================================================ TryDecorrelateSelect - Cost: 2190.06 + Cost: 2280.06 ================================================================================ select - ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) @@ -655,7 +655,7 @@ TryDecorrelateSelect └── filters (true) ================================================================================ DecorrelateJoin - Cost: 2190.06 + Cost: 2280.06 ================================================================================ select - ├── columns: k:1(int!null) i:2(int!null) f:3(float) s:4(string) j:5(jsonb) @@ -681,7 +681,7 @@ DecorrelateJoin └── filters (true) ================================================================================ PruneSemiAntiJoinRightCols - Cost: 2180.06 + Cost: 2270.06 ================================================================================ select ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) @@ -705,7 +705,7 @@ PruneSemiAntiJoinRightCols └── filters (true) ================================================================================ EliminateSelect - Cost: 2170.05 + Cost: 2260.05 ================================================================================ -select +semi-join (hash) @@ -737,18 +737,10 @@ GenerateIndexScans (no changes) -------------------------------------------------------------------------------- GenerateIndexScans (no changes) -------------------------------------------------------------------------------- --------------------------------------------------------------------------------- -CommuteSemiJoin (no changes) --------------------------------------------------------------------------------- --------------------------------------------------------------------------------- -GenerateMergeJoins (no changes) --------------------------------------------------------------------------------- --------------------------------------------------------------------------------- -GenerateLookupJoins (no changes) --------------------------------------------------------------------------------- --------------------------------------------------------------------------------- -GenerateStreamingGroupBy (higher cost) --------------------------------------------------------------------------------- +================================================================================ +CommuteSemiJoin + Cost: 2185.32 +================================================================================ -semi-join (hash) +project ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) @@ -779,6 +771,15 @@ GenerateStreamingGroupBy (higher cost) + └── filters + └── y = i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] -------------------------------------------------------------------------------- +GenerateMergeJoins (no changes) +-------------------------------------------------------------------------------- +-------------------------------------------------------------------------------- +GenerateLookupJoins (no changes) +-------------------------------------------------------------------------------- +-------------------------------------------------------------------------------- +GenerateStreamingGroupBy (no changes) +-------------------------------------------------------------------------------- +-------------------------------------------------------------------------------- CommuteJoin (no changes) -------------------------------------------------------------------------------- -------------------------------------------------------------------------------- @@ -795,20 +796,28 @@ GenerateLookupJoins (no changes) -------------------------------------------------------------------------------- ================================================================================ Final best expression - Cost: 2170.05 + Cost: 2185.32 ================================================================================ - semi-join (hash) + project ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) ├── key: (1) ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5) - ├── scan a - │ ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) - │ ├── key: (1) - │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) - ├── scan xy - │ └── columns: y:7(int) - └── filters - └── y = i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] + └── inner-join (hash) + ├── columns: k:1(int!null) i:2(int!null) f:3(float) s:4(string) j:5(jsonb) y:7(int!null) + ├── key: (1) + ├── fd: (1)-->(2-5), (3,4)~~>(1,2,5), (2)==(7), (7)==(2) + ├── scan a + │ ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) + │ ├── key: (1) + │ └── fd: (1)-->(2-5), (3,4)~~>(1,2,5) + ├── distinct-on + │ ├── columns: y:7(int) + │ ├── grouping columns: y:7(int) + │ ├── key: (7) + │ └── scan xy + │ └── columns: y:7(int) + └── filters + └── y = i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] # Decorrelation pattern using ANY function. optsteps diff --git a/pkg/sql/opt/norm/testdata/rules/decorrelate b/pkg/sql/opt/norm/testdata/rules/decorrelate index 76a464e1b0b4..515930be7189 100644 --- a/pkg/sql/opt/norm/testdata/rules/decorrelate +++ b/pkg/sql/opt/norm/testdata/rules/decorrelate @@ -1587,18 +1587,26 @@ semi-join-apply ├── scan a │ ├── columns: k:1(int!null) │ └── key: (1) - ├── semi-join (hash) + ├── project │ ├── outer: (1) - │ ├── scan xy - │ ├── inner-join (hash) - │ │ ├── columns: uv2.u:10(int!null) - │ │ ├── scan uv - │ │ ├── scan uv2 - │ │ │ ├── columns: uv2.u:10(int!null) - │ │ │ └── key: (10) - │ │ └── filters (true) - │ └── filters - │ └── uv2.u = k [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] + │ └── inner-join (hash) + │ ├── columns: uv2.u:10(int!null) + │ ├── outer: (1) + │ ├── fd: ()-->(10) + │ ├── scan xy + │ ├── distinct-on + │ │ ├── columns: uv2.u:10(int!null) + │ │ ├── grouping columns: uv2.u:10(int!null) + │ │ ├── key: (10) + │ │ └── inner-join (hash) + │ │ ├── columns: uv2.u:10(int!null) + │ │ ├── scan uv + │ │ ├── scan uv2 + │ │ │ ├── columns: uv2.u:10(int!null) + │ │ │ └── key: (10) + │ │ └── filters (true) + │ └── filters + │ └── uv2.u = k [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] └── filters (true) # -------------------------------------------------- @@ -2732,32 +2740,40 @@ WHERE EXISTS ON x=u ) ---- -semi-join (hash) +project ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) ├── key: (1) ├── fd: (1)-->(2-5) - ├── scan a - │ ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) - │ ├── key: (1) - │ └── fd: (1)-->(2-5) - ├── inner-join (merge) - │ ├── columns: x:6(int!null) u:8(int!null) v:9(int) - │ ├── left ordering: +6 - │ ├── right ordering: +8 - │ ├── key: (8) - │ ├── fd: (8)-->(9), (6)==(8), (8)==(6) - │ ├── scan xy - │ │ ├── columns: x:6(int!null) - │ │ ├── key: (6) - │ │ └── ordering: +6 - │ ├── scan uv - │ │ ├── columns: u:8(int!null) v:9(int) - │ │ ├── key: (8) - │ │ ├── fd: (8)-->(9) - │ │ └── ordering: +8 - │ └── filters (true) - └── filters - └── v = i [type=bool, outer=(2,9), constraints=(/2: (/NULL - ]; /9: (/NULL - ]), fd=(2)==(9), (9)==(2)] + └── inner-join (hash) + ├── columns: k:1(int!null) i:2(int!null) f:3(float) s:4(string) j:5(jsonb) v:9(int!null) + ├── key: (1) + ├── fd: (1)-->(2-5), (2)==(9), (9)==(2) + ├── scan a + │ ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) + │ ├── key: (1) + │ └── fd: (1)-->(2-5) + ├── distinct-on + │ ├── columns: v:9(int) + │ ├── grouping columns: v:9(int) + │ ├── key: (9) + │ └── inner-join (merge) + │ ├── columns: x:6(int!null) u:8(int!null) v:9(int) + │ ├── left ordering: +6 + │ ├── right ordering: +8 + │ ├── key: (8) + │ ├── fd: (8)-->(9), (6)==(8), (8)==(6) + │ ├── scan xy + │ │ ├── columns: x:6(int!null) + │ │ ├── key: (6) + │ │ └── ordering: +6 + │ ├── scan uv + │ │ ├── columns: u:8(int!null) v:9(int) + │ │ ├── key: (8) + │ │ ├── fd: (8)-->(9) + │ │ └── ordering: +8 + │ └── filters (true) + └── filters + └── v = i [type=bool, outer=(2,9), constraints=(/2: (/NULL - ]; /9: (/NULL - ]), fd=(2)==(9), (9)==(2)] # With left join. opt expect=TryDecorrelateLimitOne @@ -3222,8 +3238,9 @@ SELECT * FROM a WHERE NOT EXISTS(SELECT * FROM xy WHERE x=k) AND NOT EXISTS(SELECT * FROM xy WHERE x=i) ---- -anti-join (hash) +anti-join (lookup xy) ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) + ├── key columns: [1] = [6] ├── key: (1) ├── fd: (1)-->(2-5) ├── anti-join (hash) @@ -3239,11 +3256,7 @@ anti-join (hash) │ │ └── key: (8) │ └── filters │ └── x = i [type=bool, outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] - ├── scan xy - │ ├── columns: x:6(int!null) - │ └── key: (6) - └── filters - └── x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── filters (true) # Don't hoist uncorrelated subqueries. opt expect-not=HoistSelectNotExists @@ -3272,8 +3285,9 @@ select opt expect=(HoistSelectExists,HoistSelectNotExists) SELECT * FROM a WHERE EXISTS(SELECT * FROM xy WHERE x=k) AND NOT EXISTS(SELECT * FROM xy WHERE x=i) ---- -semi-join (hash) +semi-join (lookup xy) ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) + ├── key columns: [1] = [6] ├── key: (1) ├── fd: (1)-->(2-5) ├── anti-join (hash) @@ -3289,11 +3303,7 @@ semi-join (hash) │ │ └── key: (8) │ └── filters │ └── x = i [type=bool, outer=(2,8), constraints=(/2: (/NULL - ]; /8: (/NULL - ]), fd=(2)==(8), (8)==(2)] - ├── scan xy - │ ├── columns: x:6(int!null) - │ └── key: (6) - └── filters - └── x = k [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + └── filters (true) # -------------------------------------------------- # HoistSelectSubquery @@ -4904,18 +4914,26 @@ distinct-on opt expect=NormalizeSelectAnyFilter SELECT * FROM a WHERE i IN (SELECT y FROM xy) ---- -semi-join (hash) +project ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) ├── key: (1) ├── fd: (1)-->(2-5) - ├── scan a - │ ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) - │ ├── key: (1) - │ └── fd: (1)-->(2-5) - ├── scan xy - │ └── columns: y:7(int) - └── filters - └── i = y [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] + └── inner-join (hash) + ├── columns: k:1(int!null) i:2(int!null) f:3(float) s:4(string) j:5(jsonb) y:7(int!null) + ├── key: (1) + ├── fd: (1)-->(2-5), (2)==(7), (7)==(2) + ├── scan a + │ ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) + │ ├── key: (1) + │ └── fd: (1)-->(2-5) + ├── distinct-on + │ ├── columns: y:7(int) + │ ├── grouping columns: y:7(int) + │ ├── key: (7) + │ └── scan xy + │ └── columns: y:7(int) + └── filters + └── i = y [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] # Any is one of several conjuncts. opt expect=NormalizeSelectAnyFilter @@ -4952,22 +4970,30 @@ semi-join (hash) ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) ├── key: (1) ├── fd: (1)-->(2-5) - ├── semi-join (hash) + ├── project │ ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) │ ├── key: (1) │ ├── fd: (1)-->(2-5) - │ ├── scan a - │ │ ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) - │ │ ├── key: (1) - │ │ └── fd: (1)-->(2-5) - │ ├── project - │ │ ├── columns: y:10(string) - │ │ ├── scan xy - │ │ │ └── columns: xy.y:9(int) - │ │ └── projections - │ │ └── xy.y::STRING [type=string, outer=(9)] - │ └── filters - │ └── s = y [type=bool, outer=(4,10), constraints=(/4: (/NULL - ]; /10: (/NULL - ]), fd=(4)==(10), (10)==(4)] + │ └── inner-join (hash) + │ ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string!null) j:5(jsonb) y:10(string!null) + │ ├── key: (1) + │ ├── fd: (1)-->(2-5), (4)==(10), (10)==(4) + │ ├── scan a + │ │ ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) + │ │ ├── key: (1) + │ │ └── fd: (1)-->(2-5) + │ ├── distinct-on + │ │ ├── columns: y:10(string) + │ │ ├── grouping columns: y:10(string) + │ │ ├── key: (10) + │ │ └── project + │ │ ├── columns: y:10(string) + │ │ ├── scan xy + │ │ │ └── columns: xy.y:9(int) + │ │ └── projections + │ │ └── xy.y::STRING [type=string, outer=(9)] + │ └── filters + │ └── s = y [type=bool, outer=(4,10), constraints=(/4: (/NULL - ]; /10: (/NULL - ]), fd=(4)==(10), (10)==(4)] ├── scan xy │ └── columns: xy.y:7(int) └── filters @@ -5009,18 +5035,26 @@ inner-join (hash) ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) x:6(int!null) y:7(int) ├── key: (6) ├── fd: (1)-->(2-5), (6)-->(7), (1)==(6), (6)==(1) - ├── semi-join (hash) + ├── project │ ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) │ ├── key: (1) │ ├── fd: (1)-->(2-5) - │ ├── scan a - │ │ ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) - │ │ ├── key: (1) - │ │ └── fd: (1)-->(2-5) - │ ├── scan uv - │ │ └── columns: v:9(int) - │ └── filters - │ └── i = v [type=bool, outer=(2,9), constraints=(/2: (/NULL - ]; /9: (/NULL - ]), fd=(2)==(9), (9)==(2)] + │ └── inner-join (hash) + │ ├── columns: k:1(int!null) i:2(int!null) f:3(float) s:4(string) j:5(jsonb) v:9(int!null) + │ ├── key: (1) + │ ├── fd: (1)-->(2-5), (2)==(9), (9)==(2) + │ ├── scan a + │ │ ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) + │ │ ├── key: (1) + │ │ └── fd: (1)-->(2-5) + │ ├── distinct-on + │ │ ├── columns: v:9(int) + │ │ ├── grouping columns: v:9(int) + │ │ ├── key: (9) + │ │ └── scan uv + │ │ └── columns: v:9(int) + │ └── filters + │ └── i = v [type=bool, outer=(2,9), constraints=(/2: (/NULL - ]; /9: (/NULL - ]), fd=(2)==(9), (9)==(2)] ├── scan xy │ ├── columns: x:6(int!null) y:7(int) │ ├── key: (6) @@ -5133,6 +5167,10 @@ inner-join (hash) ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) x:6(int!null) y:7(int) ├── key: (6) ├── fd: (1)-->(2-5), (6)-->(7), (1)==(6), (6)==(1) + ├── scan xy + │ ├── columns: x:6(int!null) y:7(int) + │ ├── key: (6) + │ └── fd: (6)-->(7) ├── anti-join (hash) │ ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) │ ├── key: (1) @@ -5145,10 +5183,6 @@ inner-join (hash) │ │ └── columns: v:9(int) │ └── filters │ └── (i = v) IS NOT false [type=bool, outer=(2,9)] - ├── scan xy - │ ├── columns: x:6(int!null) y:7(int) - │ ├── key: (6) - │ └── fd: (6)-->(7) └── filters └── k = x [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] @@ -5158,27 +5192,35 @@ inner-join (hash) opt expect=(NormalizeSelectAnyFilter,NormalizeSelectNotAnyFilter) SELECT * FROM a WHERE i = ANY(SELECT y FROM xy) AND s <> ALL(SELECT y::string FROM xy) ---- -semi-join (hash) +project ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) ├── key: (1) ├── fd: (1)-->(2-5) - ├── anti-join (hash) - │ ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) - │ ├── key: (1) - │ ├── fd: (1)-->(2-5) - │ ├── scan a - │ │ ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) - │ │ ├── key: (1) - │ │ └── fd: (1)-->(2-5) - │ ├── project - │ │ ├── columns: y:10(string) - │ │ ├── scan xy - │ │ │ └── columns: xy.y:9(int) - │ │ └── projections - │ │ └── xy.y::STRING [type=string, outer=(9)] - │ └── filters - │ └── (s = y) IS NOT false [type=bool, outer=(4,10)] - ├── scan xy - │ └── columns: xy.y:7(int) - └── filters - └── i = xy.y [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] + └── inner-join (hash) + ├── columns: k:1(int!null) i:2(int!null) f:3(float) s:4(string) j:5(jsonb) xy.y:7(int!null) + ├── key: (1) + ├── fd: (1)-->(2-5), (2)==(7), (7)==(2) + ├── anti-join (hash) + │ ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) + │ ├── key: (1) + │ ├── fd: (1)-->(2-5) + │ ├── scan a + │ │ ├── columns: k:1(int!null) i:2(int) f:3(float) s:4(string) j:5(jsonb) + │ │ ├── key: (1) + │ │ └── fd: (1)-->(2-5) + │ ├── project + │ │ ├── columns: y:10(string) + │ │ ├── scan xy + │ │ │ └── columns: xy.y:9(int) + │ │ └── projections + │ │ └── xy.y::STRING [type=string, outer=(9)] + │ └── filters + │ └── (s = y) IS NOT false [type=bool, outer=(4,10)] + ├── distinct-on + │ ├── columns: xy.y:7(int) + │ ├── grouping columns: xy.y:7(int) + │ ├── key: (7) + │ └── scan xy + │ └── columns: xy.y:7(int) + └── filters + └── i = xy.y [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)] diff --git a/pkg/sql/opt/norm/testdata/rules/prune_cols b/pkg/sql/opt/norm/testdata/rules/prune_cols index 0737d39ada40..6ff5c862ee63 100644 --- a/pkg/sql/opt/norm/testdata/rules/prune_cols +++ b/pkg/sql/opt/norm/testdata/rules/prune_cols @@ -937,10 +937,9 @@ WHERE ---- project ├── columns: i:2(int) - └── anti-join (merge) + └── anti-join (lookup xy) ├── columns: k:1(int!null) i:2(int) - ├── left ordering: +1 - ├── right ordering: +5 + ├── key columns: [1] = [5] ├── key: (1) ├── fd: (1)-->(2) ├── anti-join (merge) @@ -949,7 +948,6 @@ project │ ├── right ordering: +7 │ ├── key: (1) │ ├── fd: (1)-->(2) - │ ├── ordering: +1 │ ├── scan a │ │ ├── columns: k:1(int!null) i:2(int) │ │ ├── key: (1) @@ -960,10 +958,6 @@ project │ │ ├── key: (7) │ │ └── ordering: +7 │ └── filters (true) - ├── scan xy - │ ├── columns: x:5(int!null) - │ ├── key: (5) - │ └── ordering: +5 └── filters (true) # -------------------------------------------------- diff --git a/pkg/sql/opt/optgen/exprgen/testdata/join b/pkg/sql/opt/optgen/exprgen/testdata/join index 0b7907a790d2..006ac4553ee9 100644 --- a/pkg/sql/opt/optgen/exprgen/testdata/join +++ b/pkg/sql/opt/optgen/exprgen/testdata/join @@ -47,8 +47,8 @@ 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=333333.333, distinct(5)=100, null(5)=3333.33333] - ├── cost: 691726.707 + ├── stats: [rows=3333.33333, distinct(5)=100, null(5)=33.3333333] + ├── cost: 25560.04 ├── scan t.public.def │ ├── columns: t.public.def.d:1(int) t.public.def.e:2(int) │ ├── stats: [rows=1000, distinct(2)=100, null(2)=10] diff --git a/pkg/sql/opt/xform/coster.go b/pkg/sql/opt/xform/coster.go index ea5448887ed4..197f18ca8812 100644 --- a/pkg/sql/opt/xform/coster.go +++ b/pkg/sql/opt/xform/coster.go @@ -347,9 +347,16 @@ func (c *coster) computeHashJoinCost(join memo.RelExpr) memo.Cost { cost := memo.Cost(1.25*leftRowCount+1.75*rightRowCount) * cpuCostFactor // Add the CPU cost of emitting the rows. - // TODO(radu): ideally we would have an estimate of how many rows we actually - // have to run the ON condition on. - cost += memo.Cost(join.Relational().Stats.RowCount) * cpuCostFactor + rowsProcessed, ok := c.mem.RowsProcessed(join) + if !ok { + // This can happen as part of testing. In this case just return the number + // of rows. + rowsProcessed = join.Relational().Stats.RowCount + } + cost += memo.Cost(rowsProcessed) * cpuCostFactor + + // TODO(rytaft): Add a constant "setup" cost per extra ON condition similar + // to merge join and lookup join. return cost } @@ -360,9 +367,19 @@ func (c *coster) computeMergeJoinCost(join *memo.MergeJoinExpr) memo.Cost { cost := memo.Cost(leftRowCount+rightRowCount) * cpuCostFactor // Add the CPU cost of emitting the rows. - // TODO(radu): ideally we would have an estimate of how many rows we actually - // have to run the ON condition on. - cost += memo.Cost(join.Relational().Stats.RowCount) * cpuCostFactor + rowsProcessed, ok := c.mem.RowsProcessed(join) + if !ok { + // We shouldn't ever get here. Since we don't allow the memo + // to be optimized twice, the coster should never be used after + // logPropsBuilder.clear() is called. + panic(errors.AssertionFailedf("could not get rows processed for merge join")) + } + cost += memo.Cost(rowsProcessed) * cpuCostFactor + + // Add a constant "setup" cost per ON condition to account for the fact that + // the rowsProcessed estimate alone cannot effectively discriminate between + // plans when RowCount is too small. + cost += cpuCostFactor * memo.Cost(len(join.On)) return cost } @@ -393,36 +410,28 @@ func (c *coster) computeLookupJoinCost(join *memo.LookupJoinExpr) memo.Cost { perRowCost := lookupJoinRetrieveRowCost + c.rowScanCost(join.Table, join.Index, numLookupCols) - // Add a cost if we have to evaluate an ON condition on every row. The more - // leftover conditions, the more expensive it should be. We want to - // differentiate between two lookup joins where one uses only a subset of the - // columns. For example: - // abc JOIN xyz ON a=x AND b=y - // We could have a lookup join using an index on y (and left-over condition - // a=x), and another lookup join on an index on x,y. The latter is definitely - // preferable (the former could generate a lot of internal results that are - // then discarded). - // - // TODO(radu): we should take into account that the "internal" row count is - // higher, according to the selectivities of the conditions. Unfortunately - // this is very tricky, in particular because of left-over conditions that are - // not selective. + // Take into account that the "internal" row count is higher, according to + // the selectivities of the conditions. In particular, we need to ignore + // left-over conditions that are not selective. // For example: // ab JOIN xy ON a=x AND x=10 // becomes (during normalization): // ab JOIN xy ON a=x AND a=10 AND x=10 // which can become a lookup join with left-over condition x=10 which doesn't // actually filter anything. - // - // TODO(radu): this should be extended to all join types. It's tricky for hash - // joins where we don't have the equality and leftover filters readily - // available. - perRowCost += cpuCostFactor * memo.Cost(len(join.On)) - // We also add a constant "setup" cost per ON condition. Without this, the - // adjustment above can be inconsequential when the RowCount is too small. - cost += cpuCostFactor * memo.Cost(len(join.On)) + rowsProcessed, ok := c.mem.RowsProcessed(join) + if !ok { + // We shouldn't ever get here. Since we don't allow the memo + // to be optimized twice, the coster should never be used after + // logPropsBuilder.clear() is called. + panic(errors.AssertionFailedf("could not get rows processed for lookup join")) + } + cost += memo.Cost(rowsProcessed) * perRowCost - cost += memo.Cost(join.Relational().Stats.RowCount) * perRowCost + // Add a constant "setup" cost per ON condition to account for the fact that + // the rowsProcessed estimate alone cannot effectively discriminate between + // plans when RowCount is too small. + cost += cpuCostFactor * memo.Cost(len(join.On)) return cost } diff --git a/pkg/sql/opt/xform/testdata/coster/join b/pkg/sql/opt/xform/testdata/coster/join index 584fd4d8482c..2b73972af120 100644 --- a/pkg/sql/opt/xform/testdata/coster/join +++ b/pkg/sql/opt/xform/testdata/coster/join @@ -225,13 +225,13 @@ WHERE w = 'foo' AND x = '2AB23800-06B1-4E19-A3BB-DF3768B808D2' project ├── columns: w:1(string!null) x:2(uuid!null) y:3(uuid!null) z:4(string!null) ├── stats: [rows=500.488759] - ├── cost: 1566.10128 + ├── cost: 1555.07489 ├── fd: ()-->(1,2) └── inner-join (lookup abcde@idx_abcd) ├── columns: w:1(string!null) x:2(uuid!null) y:3(uuid!null) z:4(string!null) a:6(string!null) b:7(uuid!null) c:8(uuid!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(4)=10, null(4)=0, distinct(6)=1, null(6)=0, distinct(7)=1, null(7)=0, distinct(8)=25, null(8)=0] - ├── cost: 1561.08639 + ├── cost: 1550.06 ├── fd: ()-->(1,2,6,7), (1)==(6), (6)==(1), (2)==(7), (7)==(2), (3)==(8), (8)==(3) ├── select │ ├── columns: w:1(string!null) x:2(uuid!null) y:3(uuid!null) z:4(string!null) diff --git a/pkg/sql/opt/xform/testdata/coster/zone b/pkg/sql/opt/xform/testdata/coster/zone index eee9c3da3caf..104732b984c6 100644 --- a/pkg/sql/opt/xform/testdata/coster/zone +++ b/pkg/sql/opt/xform/testdata/coster/zone @@ -367,7 +367,7 @@ inner-join (lookup xy@y2) ├── flags: no-merge-join;no-hash-join ├── key columns: [2] = [5] ├── stats: [rows=98.01, distinct(1)=9.9, null(1)=0, distinct(2)=1, null(2)=0, distinct(4)=9.9, null(4)=0, distinct(5)=1, null(5)=0] - ├── cost: 251.0445 + ├── cost: 252.084 ├── key: (1,4) ├── fd: ()-->(2,5), (1)-->(3), (2,3)~~>(1), (2)==(5), (5)==(2) ├── prune: (1,3,4) @@ -405,7 +405,7 @@ inner-join (lookup xy@y1) ├── flags: no-merge-join;no-hash-join ├── key columns: [2] = [5] ├── stats: [rows=98.01, distinct(1)=9.9, null(1)=0, distinct(2)=1, null(2)=0, distinct(4)=9.9, null(4)=0, distinct(5)=1, null(5)=0] - ├── cost: 251.0445 + ├── cost: 252.084 ├── key: (1,4) ├── fd: ()-->(2,5), (1)-->(3), (2,3)~~>(1), (2)==(5), (5)==(2) ├── prune: (1,3,4) diff --git a/pkg/sql/opt/xform/testdata/external/hibernate b/pkg/sql/opt/xform/testdata/external/hibernate index 52c9ab81044a..338b890f9213 100644 --- a/pkg/sql/opt/xform/testdata/external/hibernate +++ b/pkg/sql/opt/xform/testdata/external/hibernate @@ -180,41 +180,34 @@ inner-join (hash) ├── columns: id1_6_0_:1(int!null) id1_4_1_:6(int!null) phone_nu2_6_0_:2(varchar) person_i4_6_0_:4(int!null) phone_ty3_6_0_:3(varchar) person_i1_5_0__:12(int!null) addresse2_5_0__:13(varchar) addresse3_0__:14(varchar!null) address2_4_1_:7(varchar) createdo3_4_1_:8(timestamp) name4_4_1_:9(varchar) nickname5_4_1_:10(varchar) version6_4_1_:11(int4!null) person_i1_5_0__:12(int!null) addresse2_5_0__:13(varchar) addresse3_0__:14(varchar!null) ├── key: (1,14) ├── fd: (1)-->(2-4), (6)-->(7-11), (4)==(6,12), (6)==(4,12), (12,14)-->(13), (12)==(4,6) - ├── project - │ ├── columns: phone0_.id:1(int!null) phone_number:2(varchar) phone_type:3(varchar) phone0_.person_id:4(int) - │ ├── key: (1) - │ ├── fd: (1)-->(2-4) - │ └── inner-join (lookup phone) - │ ├── columns: phone0_.id:1(int!null) phone_number:2(varchar) phone_type:3(varchar) phone0_.person_id:4(int) phone_id:18(int!null) - │ ├── key columns: [18] = [1] - │ ├── key: (18) - │ ├── fd: (1)-->(2-4), (1)==(18), (18)==(1) - │ ├── distinct-on - │ │ ├── columns: phone_id:18(int) - │ │ ├── grouping columns: phone_id:18(int) - │ │ ├── key: (18) - │ │ └── scan calls3_ - │ │ └── columns: phone_id:18(int) - │ └── filters (true) - ├── inner-join (merge) - │ ├── columns: person1_.id:6(int!null) address:7(varchar) createdon:8(timestamp) name:9(varchar) nickname:10(varchar) version:11(int4!null) addresses2_.person_id:12(int!null) addresses:13(varchar) addresses_key:14(varchar!null) - │ ├── left ordering: +6 - │ ├── right ordering: +12 + ├── scan addresses2_ + │ ├── columns: addresses2_.person_id:12(int!null) addresses:13(varchar) addresses_key:14(varchar!null) │ ├── key: (12,14) - │ ├── fd: (6)-->(7-11), (12,14)-->(13), (6)==(12), (12)==(6) - │ ├── scan person1_ - │ │ ├── columns: person1_.id:6(int!null) address:7(varchar) createdon:8(timestamp) name:9(varchar) nickname:10(varchar) version:11(int4!null) - │ │ ├── key: (6) - │ │ ├── fd: (6)-->(7-11) - │ │ └── ordering: +6 - │ ├── scan addresses2_ - │ │ ├── columns: addresses2_.person_id:12(int!null) addresses:13(varchar) addresses_key:14(varchar!null) - │ │ ├── key: (12,14) - │ │ ├── fd: (12,14)-->(13) - │ │ └── ordering: +12 + │ └── fd: (12,14)-->(13) + ├── inner-join (lookup person) + │ ├── columns: phone0_.id:1(int!null) phone_number:2(varchar) phone_type:3(varchar) phone0_.person_id:4(int!null) person1_.id:6(int!null) address:7(varchar) createdon:8(timestamp) name:9(varchar) nickname:10(varchar) version:11(int4!null) + │ ├── key columns: [4] = [6] + │ ├── key: (1) + │ ├── fd: (1)-->(2-4), (6)-->(7-11), (4)==(6), (6)==(4) + │ ├── project + │ │ ├── columns: phone0_.id:1(int!null) phone_number:2(varchar) phone_type:3(varchar) phone0_.person_id:4(int) + │ │ ├── key: (1) + │ │ ├── fd: (1)-->(2-4) + │ │ └── inner-join (lookup phone) + │ │ ├── columns: phone0_.id:1(int!null) phone_number:2(varchar) phone_type:3(varchar) phone0_.person_id:4(int) phone_id:18(int!null) + │ │ ├── key columns: [18] = [1] + │ │ ├── key: (18) + │ │ ├── fd: (1)-->(2-4), (1)==(18), (18)==(1) + │ │ ├── distinct-on + │ │ │ ├── columns: phone_id:18(int) + │ │ │ ├── grouping columns: phone_id:18(int) + │ │ │ ├── key: (18) + │ │ │ └── scan calls3_ + │ │ │ └── columns: phone_id:18(int) + │ │ └── filters (true) │ └── filters (true) └── filters - └── phone0_.person_id = person1_.id [type=bool, outer=(4,6), constraints=(/4: (/NULL - ]; /6: (/NULL - ]), fd=(4)==(6), (6)==(4)] + └── person1_.id = addresses2_.person_id [type=bool, outer=(6,12), constraints=(/6: (/NULL - ]; /12: (/NULL - ]), fd=(6)==(12), (12)==(6)] opt select @@ -246,37 +239,32 @@ project └── inner-join (hash) ├── columns: phone0_.id:1(int!null) phone_number:2(varchar) phone_type:3(varchar) phone0_.person_id:4(int!null) person1_.id:6(int!null) addresses2_.person_id:12(int!null) ├── fd: (1)-->(2-4), (4)==(6,12), (6)==(4,12), (12)==(4,6) - ├── project - │ ├── columns: phone0_.id:1(int!null) phone_number:2(varchar) phone_type:3(varchar) phone0_.person_id:4(int) + ├── scan addresses2_ + │ └── columns: addresses2_.person_id:12(int!null) + ├── inner-join (lookup person) + │ ├── columns: phone0_.id:1(int!null) phone_number:2(varchar) phone_type:3(varchar) phone0_.person_id:4(int!null) person1_.id:6(int!null) + │ ├── key columns: [4] = [6] │ ├── key: (1) - │ ├── fd: (1)-->(2-4) - │ └── inner-join (lookup phone) - │ ├── columns: phone0_.id:1(int!null) phone_number:2(varchar) phone_type:3(varchar) phone0_.person_id:4(int) phone_id:18(int!null) - │ ├── key columns: [18] = [1] - │ ├── key: (18) - │ ├── fd: (1)-->(2-4), (1)==(18), (18)==(1) - │ ├── distinct-on - │ │ ├── columns: phone_id:18(int) - │ │ ├── grouping columns: phone_id:18(int) - │ │ ├── key: (18) - │ │ └── scan calls3_ - │ │ └── columns: phone_id:18(int) - │ └── filters (true) - ├── inner-join (merge) - │ ├── columns: person1_.id:6(int!null) addresses2_.person_id:12(int!null) - │ ├── left ordering: +6 - │ ├── right ordering: +12 - │ ├── fd: (6)==(12), (12)==(6) - │ ├── scan person1_ - │ │ ├── columns: person1_.id:6(int!null) - │ │ ├── key: (6) - │ │ └── ordering: +6 - │ ├── scan addresses2_ - │ │ ├── columns: addresses2_.person_id:12(int!null) - │ │ └── ordering: +12 + │ ├── fd: (1)-->(2-4), (4)==(6), (6)==(4) + │ ├── project + │ │ ├── columns: phone0_.id:1(int!null) phone_number:2(varchar) phone_type:3(varchar) phone0_.person_id:4(int) + │ │ ├── key: (1) + │ │ ├── fd: (1)-->(2-4) + │ │ └── inner-join (lookup phone) + │ │ ├── columns: phone0_.id:1(int!null) phone_number:2(varchar) phone_type:3(varchar) phone0_.person_id:4(int) phone_id:18(int!null) + │ │ ├── key columns: [18] = [1] + │ │ ├── key: (18) + │ │ ├── fd: (1)-->(2-4), (1)==(18), (18)==(1) + │ │ ├── distinct-on + │ │ │ ├── columns: phone_id:18(int) + │ │ │ ├── grouping columns: phone_id:18(int) + │ │ │ ├── key: (18) + │ │ │ └── scan calls3_ + │ │ │ └── columns: phone_id:18(int) + │ │ └── filters (true) │ └── filters (true) └── filters - └── phone0_.person_id = person1_.id [type=bool, outer=(4,6), constraints=(/4: (/NULL - ]; /6: (/NULL - ]), fd=(4)==(6), (6)==(4)] + └── person1_.id = addresses2_.person_id [type=bool, outer=(6,12), constraints=(/6: (/NULL - ]; /12: (/NULL - ]), fd=(6)==(12), (12)==(6)] opt select @@ -673,9 +661,16 @@ distinct-on ├── inner-join (hash) │ ├── columns: phone0_.id:1(int!null) person_id:4(int!null) calls1_.phone_id:9(int!null) person2_.id:10(int!null) address:11(varchar) createdon:12(timestamp) name:13(varchar) nickname:14(varchar) version:15(int4!null) │ ├── fd: (1)-->(4), (1)==(9), (9)==(1), (10)-->(11-15), (4)==(10), (10)==(4) + │ ├── scan calls1_ + │ │ └── columns: calls1_.phone_id:9(int) │ ├── inner-join (hash) - │ │ ├── columns: phone0_.id:1(int!null) person_id:4(int) calls1_.phone_id:9(int!null) - │ │ ├── fd: (1)-->(4), (1)==(9), (9)==(1) + │ │ ├── columns: phone0_.id:1(int!null) person_id:4(int!null) person2_.id:10(int!null) address:11(varchar) createdon:12(timestamp) name:13(varchar) nickname:14(varchar) version:15(int4!null) + │ │ ├── key: (1) + │ │ ├── fd: (1)-->(4), (10)-->(11-15), (4)==(10), (10)==(4) + │ │ ├── scan person2_ + │ │ │ ├── columns: person2_.id:10(int!null) address:11(varchar) createdon:12(timestamp) name:13(varchar) nickname:14(varchar) version:15(int4!null) + │ │ │ ├── key: (10) + │ │ │ └── fd: (10)-->(11-15) │ │ ├── anti-join (hash) │ │ │ ├── columns: phone0_.id:1(int!null) person_id:4(int) │ │ │ ├── key: (1) @@ -692,16 +687,10 @@ distinct-on │ │ │ │ └── (call3_.duration >= 50) IS NOT false [type=bool, outer=(17)] │ │ │ └── filters │ │ │ └── call3_.phone_id = phone0_.id [type=bool, outer=(1,19), constraints=(/1: (/NULL - ]; /19: (/NULL - ]), fd=(1)==(19), (19)==(1)] - │ │ ├── scan calls1_ - │ │ │ └── columns: calls1_.phone_id:9(int) │ │ └── filters - │ │ └── phone0_.id = calls1_.phone_id [type=bool, outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ]), fd=(1)==(9), (9)==(1)] - │ ├── scan person2_ - │ │ ├── columns: person2_.id:10(int!null) address:11(varchar) createdon:12(timestamp) name:13(varchar) nickname:14(varchar) version:15(int4!null) - │ │ ├── key: (10) - │ │ └── fd: (10)-->(11-15) + │ │ └── person_id = person2_.id [type=bool, outer=(4,10), constraints=(/4: (/NULL - ]; /10: (/NULL - ]), fd=(4)==(10), (10)==(4)] │ └── filters - │ └── person_id = person2_.id [type=bool, outer=(4,10), constraints=(/4: (/NULL - ]; /10: (/NULL - ]), fd=(4)==(10), (10)==(4)] + │ └── phone0_.id = calls1_.phone_id [type=bool, outer=(1,9), constraints=(/1: (/NULL - ]; /9: (/NULL - ]), fd=(1)==(9), (9)==(1)] └── aggregations ├── const-agg [type=varchar, outer=(11)] │ └── variable: address [type=varchar] @@ -1588,10 +1577,14 @@ where company0_.id=employees1_.Company_id and employees1_.employees_id=employee2_.id)) ---- -left-join (hash) +right-join (hash) ├── columns: id1_0_0_:1(int!null) id1_8_1_:3(int) location2_0_0_:2(int) address2_8_1_:4(varchar) zip3_8_1_:5(int4) ├── key: (1,3) ├── fd: (1)-->(2), (3)-->(4,5) + ├── scan location3_ + │ ├── columns: location3_.id:3(int!null) address:4(varchar) zip:5(int4!null) + │ ├── key: (3) + │ └── fd: (3)-->(4,5) ├── anti-join (hash) │ ├── columns: company0_.id:1(int!null) location_id:2(int) │ ├── key: (1) @@ -1632,10 +1625,6 @@ left-join (hash) │ │ └── employees_id = id [type=bool, outer=(7,12), constraints=(/7: (/NULL - ]; /12: (/NULL - ]), fd=(7)==(12), (12)==(7)] │ └── filters │ └── company0_.id = company_id [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] - ├── scan location3_ - │ ├── columns: location3_.id:3(int!null) address:4(varchar) zip:5(int4!null) - │ ├── key: (3) - │ └── fd: (3)-->(4,5) └── filters └── location_id = location3_.id [type=bool, outer=(2,3), constraints=(/2: (/NULL - ]; /3: (/NULL - ]), fd=(2)==(3), (3)==(2)] diff --git a/pkg/sql/opt/xform/testdata/external/nova b/pkg/sql/opt/xform/testdata/external/nova index 551e1605921e..e20b93fe81bb 100644 --- a/pkg/sql/opt/xform/testdata/external/nova +++ b/pkg/sql/opt/xform/testdata/external/nova @@ -947,121 +947,124 @@ project ├── side-effects, 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) - └── left-join (lookup instance_type_extra_specs) + └── right-join (hash) ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) instance_type_extra_specs_1.id:28(int) key:29(string) value:30(string) instance_type_extra_specs_1.instance_type_id:31(int) instance_type_extra_specs_1.deleted:32(bool) instance_type_extra_specs_1.deleted_at:33(timestamp) instance_type_extra_specs_1.created_at:34(timestamp) instance_type_extra_specs_1.updated_at:35(timestamp) - ├── key columns: [28] = [28] ├── side-effects, 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) - ├── left-join (lookup instance_type_extra_specs@secondary) - │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) instance_type_extra_specs_1.id:28(int) key:29(string) instance_type_extra_specs_1.instance_type_id:31(int) instance_type_extra_specs_1.deleted:32(bool) - │ ├── key columns: [1] = [31] + ├── select + │ ├── columns: instance_type_extra_specs_1.id:28(int!null) key:29(string) value:30(string) instance_type_extra_specs_1.instance_type_id:31(int!null) instance_type_extra_specs_1.deleted:32(bool!null) instance_type_extra_specs_1.deleted_at:33(timestamp) instance_type_extra_specs_1.created_at:34(timestamp) instance_type_extra_specs_1.updated_at:35(timestamp) + │ ├── has-placeholder + │ ├── key: (28) + │ ├── fd: (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + │ ├── scan instance_type_extra_specs_1 + │ │ ├── columns: instance_type_extra_specs_1.id:28(int!null) key:29(string) value:30(string) instance_type_extra_specs_1.instance_type_id:31(int!null) instance_type_extra_specs_1.deleted:32(bool) instance_type_extra_specs_1.deleted_at:33(timestamp) instance_type_extra_specs_1.created_at:34(timestamp) instance_type_extra_specs_1.updated_at:35(timestamp) + │ │ ├── key: (28) + │ │ └── fd: (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + │ └── filters + │ └── instance_type_extra_specs_1.deleted = $7 [type=bool, outer=(32), constraints=(/32: (/NULL - ])] + ├── limit + │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) │ ├── side-effects, 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,31,32), (29,31,32)~~>(28) - │ ├── limit + │ ├── key: (1) + │ ├── fd: (1)-->(2-16,26), (7,13)~~>(1-6,8-12,14-16), (2,13)-->(1,3-12,14-16) + │ ├── offset │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) - │ │ ├── side-effects, has-placeholder + │ │ ├── has-placeholder │ │ ├── key: (1) │ │ ├── fd: (1)-->(2-16,26), (7,13)~~>(1-6,8-12,14-16), (2,13)-->(1,3-12,14-16) - │ │ ├── offset + │ │ ├── select │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) │ │ │ ├── has-placeholder │ │ │ ├── key: (1) │ │ │ ├── fd: (1)-->(2-16,26), (7,13)~~>(1-6,8-12,14-16), (2,13)-->(1,3-12,14-16) - │ │ │ ├── select + │ │ │ ├── group-by │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) + │ │ │ │ ├── grouping columns: instance_types.id:1(int!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) - │ │ │ │ ├── group-by - │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) - │ │ │ │ │ ├── grouping columns: instance_types.id:1(int!null) - │ │ │ │ │ ├── internal-ordering: +1 + │ │ │ │ ├── left-join (merge) + │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string!null) memory_mb:3(int!null) vcpus:4(int!null) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int!null) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool!null) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) instance_type_projects.instance_type_id:18(int) true:25(bool) + │ │ │ │ │ ├── left ordering: +1 + │ │ │ │ │ ├── right ordering: +18 │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ ├── key: (1) - │ │ │ │ │ ├── fd: (1)-->(2-16,26), (7,13)~~>(1-6,8-12,14-16), (2,13)-->(1,3-12,14-16) - │ │ │ │ │ ├── left-join (merge) - │ │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string!null) memory_mb:3(int!null) vcpus:4(int!null) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int!null) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool!null) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) instance_type_projects.instance_type_id:18(int) true:25(bool) - │ │ │ │ │ │ ├── left ordering: +1 - │ │ │ │ │ │ ├── right ordering: +18 + │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)-->(1,3-12,14-16), ()~~>(25) + │ │ │ │ │ ├── ordering: +1 + │ │ │ │ │ ├── select + │ │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string!null) memory_mb:3(int!null) vcpus:4(int!null) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int!null) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool!null) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)-->(1,3-12,14-16), ()~~>(25) + │ │ │ │ │ │ ├── key: (1) + │ │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)-->(1,3-12,14-16) │ │ │ │ │ │ ├── ordering: +1 + │ │ │ │ │ │ ├── scan instance_types + │ │ │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int!null) vcpus:4(int!null) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int!null) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) + │ │ │ │ │ │ │ ├── key: (1) + │ │ │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ │ │ │ │ │ └── ordering: +1 + │ │ │ │ │ │ └── filters + │ │ │ │ │ │ ├── instance_types.deleted = $1 [type=bool, outer=(13), constraints=(/13: (/NULL - ])] + │ │ │ │ │ │ └── name = $4 [type=bool, outer=(2), constraints=(/2: (/NULL - ])] + │ │ │ │ │ ├── project + │ │ │ │ │ │ ├── columns: true:25(bool!null) instance_type_projects.instance_type_id:18(int!null) + │ │ │ │ │ │ ├── has-placeholder + │ │ │ │ │ │ ├── fd: ()-->(25) + │ │ │ │ │ │ ├── ordering: +18 opt(25) [actual: +18] │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string!null) memory_mb:3(int!null) vcpus:4(int!null) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int!null) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool!null) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) + │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18(int!null) project_id:19(string!null) instance_type_projects.deleted:20(bool!null) │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ ├── key: (1) - │ │ │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)-->(1,3-12,14-16) - │ │ │ │ │ │ │ ├── ordering: +1 - │ │ │ │ │ │ │ ├── scan instance_types - │ │ │ │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int!null) vcpus:4(int!null) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int!null) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) - │ │ │ │ │ │ │ │ ├── key: (1) - │ │ │ │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) - │ │ │ │ │ │ │ │ └── ordering: +1 + │ │ │ │ │ │ │ ├── key: (18-20) + │ │ │ │ │ │ │ ├── ordering: +18 + │ │ │ │ │ │ │ ├── scan instance_type_projects@secondary + │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18(int!null) project_id:19(string) instance_type_projects.deleted:20(bool) + │ │ │ │ │ │ │ │ ├── lax-key: (18-20) + │ │ │ │ │ │ │ │ └── ordering: +18 │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ ├── instance_types.deleted = $1 [type=bool, outer=(13), constraints=(/13: (/NULL - ])] - │ │ │ │ │ │ │ └── name = $4 [type=bool, outer=(2), constraints=(/2: (/NULL - ])] - │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ ├── columns: true:25(bool!null) instance_type_projects.instance_type_id:18(int!null) - │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ ├── fd: ()-->(25) - │ │ │ │ │ │ │ ├── ordering: +18 opt(25) [actual: +18] - │ │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18(int!null) project_id:19(string!null) instance_type_projects.deleted:20(bool!null) - │ │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ │ ├── key: (18-20) - │ │ │ │ │ │ │ │ ├── ordering: +18 - │ │ │ │ │ │ │ │ ├── scan instance_type_projects@secondary - │ │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18(int!null) project_id:19(string) instance_type_projects.deleted:20(bool) - │ │ │ │ │ │ │ │ │ ├── lax-key: (18-20) - │ │ │ │ │ │ │ │ │ └── ordering: +18 - │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ ├── instance_type_projects.deleted = $2 [type=bool, outer=(20), constraints=(/20: (/NULL - ])] - │ │ │ │ │ │ │ │ └── project_id = $3 [type=bool, outer=(19), constraints=(/19: (/NULL - ])] - │ │ │ │ │ │ │ └── projections - │ │ │ │ │ │ │ └── true [type=bool] - │ │ │ │ │ │ └── filters (true) - │ │ │ │ │ └── aggregations - │ │ │ │ │ ├── const-not-null-agg [type=bool, outer=(25)] - │ │ │ │ │ │ └── variable: true [type=bool] - │ │ │ │ │ ├── const-agg [type=string, outer=(2)] - │ │ │ │ │ │ └── variable: name [type=string] - │ │ │ │ │ ├── const-agg [type=int, outer=(3)] - │ │ │ │ │ │ └── variable: memory_mb [type=int] - │ │ │ │ │ ├── const-agg [type=int, outer=(4)] - │ │ │ │ │ │ └── variable: vcpus [type=int] - │ │ │ │ │ ├── const-agg [type=int, outer=(5)] - │ │ │ │ │ │ └── variable: root_gb [type=int] - │ │ │ │ │ ├── const-agg [type=int, outer=(6)] - │ │ │ │ │ │ └── variable: ephemeral_gb [type=int] - │ │ │ │ │ ├── const-agg [type=string, outer=(7)] - │ │ │ │ │ │ └── variable: flavorid [type=string] - │ │ │ │ │ ├── const-agg [type=int, outer=(8)] - │ │ │ │ │ │ └── variable: swap [type=int] - │ │ │ │ │ ├── const-agg [type=float, outer=(9)] - │ │ │ │ │ │ └── variable: rxtx_factor [type=float] - │ │ │ │ │ ├── const-agg [type=int, outer=(10)] - │ │ │ │ │ │ └── variable: vcpu_weight [type=int] - │ │ │ │ │ ├── const-agg [type=bool, outer=(11)] - │ │ │ │ │ │ └── variable: disabled [type=bool] - │ │ │ │ │ ├── const-agg [type=bool, outer=(12)] - │ │ │ │ │ │ └── variable: is_public [type=bool] - │ │ │ │ │ ├── const-agg [type=bool, outer=(13)] - │ │ │ │ │ │ └── variable: instance_types.deleted [type=bool] - │ │ │ │ │ ├── const-agg [type=timestamp, outer=(14)] - │ │ │ │ │ │ └── variable: instance_types.deleted_at [type=timestamp] - │ │ │ │ │ ├── const-agg [type=timestamp, outer=(15)] - │ │ │ │ │ │ └── variable: instance_types.created_at [type=timestamp] - │ │ │ │ │ └── const-agg [type=timestamp, outer=(16)] - │ │ │ │ │ └── variable: instance_types.updated_at [type=timestamp] - │ │ │ │ └── filters - │ │ │ │ └── (is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,26)] - │ │ │ └── placeholder: $5 [type=int] - │ │ └── placeholder: $6 [type=int] - │ └── filters - │ └── instance_type_extra_specs_1.deleted = $7 [type=bool, outer=(32), constraints=(/32: (/NULL - ])] - └── filters (true) + │ │ │ │ │ │ │ ├── instance_type_projects.deleted = $2 [type=bool, outer=(20), constraints=(/20: (/NULL - ])] + │ │ │ │ │ │ │ └── project_id = $3 [type=bool, outer=(19), constraints=(/19: (/NULL - ])] + │ │ │ │ │ │ └── projections + │ │ │ │ │ │ └── true [type=bool] + │ │ │ │ │ └── filters (true) + │ │ │ │ └── aggregations + │ │ │ │ ├── const-not-null-agg [type=bool, outer=(25)] + │ │ │ │ │ └── variable: true [type=bool] + │ │ │ │ ├── const-agg [type=string, outer=(2)] + │ │ │ │ │ └── variable: name [type=string] + │ │ │ │ ├── const-agg [type=int, outer=(3)] + │ │ │ │ │ └── variable: memory_mb [type=int] + │ │ │ │ ├── const-agg [type=int, outer=(4)] + │ │ │ │ │ └── variable: vcpus [type=int] + │ │ │ │ ├── const-agg [type=int, outer=(5)] + │ │ │ │ │ └── variable: root_gb [type=int] + │ │ │ │ ├── const-agg [type=int, outer=(6)] + │ │ │ │ │ └── variable: ephemeral_gb [type=int] + │ │ │ │ ├── const-agg [type=string, outer=(7)] + │ │ │ │ │ └── variable: flavorid [type=string] + │ │ │ │ ├── const-agg [type=int, outer=(8)] + │ │ │ │ │ └── variable: swap [type=int] + │ │ │ │ ├── const-agg [type=float, outer=(9)] + │ │ │ │ │ └── variable: rxtx_factor [type=float] + │ │ │ │ ├── const-agg [type=int, outer=(10)] + │ │ │ │ │ └── variable: vcpu_weight [type=int] + │ │ │ │ ├── const-agg [type=bool, outer=(11)] + │ │ │ │ │ └── variable: disabled [type=bool] + │ │ │ │ ├── const-agg [type=bool, outer=(12)] + │ │ │ │ │ └── variable: is_public [type=bool] + │ │ │ │ ├── const-agg [type=bool, outer=(13)] + │ │ │ │ │ └── variable: instance_types.deleted [type=bool] + │ │ │ │ ├── const-agg [type=timestamp, outer=(14)] + │ │ │ │ │ └── variable: instance_types.deleted_at [type=timestamp] + │ │ │ │ ├── const-agg [type=timestamp, outer=(15)] + │ │ │ │ │ └── variable: instance_types.created_at [type=timestamp] + │ │ │ │ └── const-agg [type=timestamp, outer=(16)] + │ │ │ │ └── variable: instance_types.updated_at [type=timestamp] + │ │ │ └── filters + │ │ │ └── (is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,26)] + │ │ └── placeholder: $5 [type=int] + │ └── placeholder: $6 [type=int] + └── filters + └── instance_type_extra_specs_1.instance_type_id = instance_types.id [type=bool, outer=(1,31), constraints=(/1: (/NULL - ]; /31: (/NULL - ]), fd=(1)==(31), (31)==(1)] opt select anon_1.instance_types_created_at as anon_1_instance_types_created_at, @@ -1125,122 +1128,125 @@ project ├── side-effects, 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) - └── left-join (lookup instance_type_extra_specs) + └── right-join (hash) ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) instance_type_extra_specs_1.id:28(int) key:29(string) value:30(string) instance_type_extra_specs_1.instance_type_id:31(int) instance_type_extra_specs_1.deleted:32(bool) instance_type_extra_specs_1.deleted_at:33(timestamp) instance_type_extra_specs_1.created_at:34(timestamp) instance_type_extra_specs_1.updated_at:35(timestamp) - ├── key columns: [28] = [28] ├── side-effects, 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) - ├── left-join (lookup instance_type_extra_specs@secondary) - │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) instance_type_extra_specs_1.id:28(int) key:29(string) instance_type_extra_specs_1.instance_type_id:31(int) instance_type_extra_specs_1.deleted:32(bool) - │ ├── key columns: [1] = [31] + ├── select + │ ├── columns: instance_type_extra_specs_1.id:28(int!null) key:29(string) value:30(string) instance_type_extra_specs_1.instance_type_id:31(int!null) instance_type_extra_specs_1.deleted:32(bool!null) instance_type_extra_specs_1.deleted_at:33(timestamp) instance_type_extra_specs_1.created_at:34(timestamp) instance_type_extra_specs_1.updated_at:35(timestamp) + │ ├── has-placeholder + │ ├── key: (28) + │ ├── fd: (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + │ ├── scan instance_type_extra_specs_1 + │ │ ├── columns: instance_type_extra_specs_1.id:28(int!null) key:29(string) value:30(string) instance_type_extra_specs_1.instance_type_id:31(int!null) instance_type_extra_specs_1.deleted:32(bool) instance_type_extra_specs_1.deleted_at:33(timestamp) instance_type_extra_specs_1.created_at:34(timestamp) instance_type_extra_specs_1.updated_at:35(timestamp) + │ │ ├── key: (28) + │ │ └── fd: (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + │ └── filters + │ └── instance_type_extra_specs_1.deleted = $7 [type=bool, outer=(32), constraints=(/32: (/NULL - ])] + ├── limit + │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) │ ├── side-effects, 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,31,32), (29,31,32)~~>(28) - │ ├── limit + │ ├── key: (1) + │ ├── fd: (1)-->(2-16,26), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ ├── offset │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) - │ │ ├── side-effects, has-placeholder + │ │ ├── has-placeholder │ │ ├── key: (1) │ │ ├── fd: (1)-->(2-16,26), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) - │ │ ├── offset + │ │ ├── select │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) │ │ │ ├── has-placeholder │ │ │ ├── key: (1) │ │ │ ├── fd: (1)-->(2-16,26), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) - │ │ │ ├── select + │ │ │ ├── group-by │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) + │ │ │ │ ├── grouping columns: instance_types.id:1(int!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) - │ │ │ │ ├── group-by - │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) - │ │ │ │ │ ├── grouping columns: instance_types.id:1(int!null) - │ │ │ │ │ ├── internal-ordering: +1 + │ │ │ │ ├── left-join (merge) + │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int!null) vcpus:4(int!null) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int!null) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool!null) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) instance_type_projects.instance_type_id:18(int) true:25(bool) + │ │ │ │ │ ├── left ordering: +1 + │ │ │ │ │ ├── right ordering: +18 │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ ├── key: (1) - │ │ │ │ │ ├── fd: (1)-->(2-16,26), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) - │ │ │ │ │ ├── left-join (merge) - │ │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int!null) vcpus:4(int!null) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int!null) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool!null) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) instance_type_projects.instance_type_id:18(int) true:25(bool) - │ │ │ │ │ │ ├── left ordering: +1 - │ │ │ │ │ │ ├── right ordering: +18 + │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), ()~~>(25) + │ │ │ │ │ ├── ordering: +1 + │ │ │ │ │ ├── select + │ │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int!null) vcpus:4(int!null) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int!null) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool!null) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), ()~~>(25) + │ │ │ │ │ │ ├── key: (1) + │ │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ │ │ │ │ ├── ordering: +1 - │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int!null) vcpus:4(int!null) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int!null) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool!null) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) - │ │ │ │ │ │ │ ├── has-placeholder + │ │ │ │ │ │ ├── scan instance_types + │ │ │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int!null) vcpus:4(int!null) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int!null) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) │ │ │ │ │ │ │ ├── key: (1) │ │ │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) - │ │ │ │ │ │ │ ├── ordering: +1 - │ │ │ │ │ │ │ ├── scan instance_types - │ │ │ │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int!null) vcpus:4(int!null) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int!null) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) - │ │ │ │ │ │ │ │ ├── key: (1) - │ │ │ │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) - │ │ │ │ │ │ │ │ └── ordering: +1 - │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ ├── instance_types.id = $4 [type=bool, outer=(1), constraints=(/1: (/NULL - ])] - │ │ │ │ │ │ │ └── instance_types.deleted = $1 [type=bool, outer=(13), constraints=(/13: (/NULL - ])] - │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ ├── columns: true:25(bool!null) instance_type_projects.instance_type_id:18(int!null) + │ │ │ │ │ │ │ └── ordering: +1 + │ │ │ │ │ │ └── filters + │ │ │ │ │ │ ├── instance_types.id = $4 [type=bool, outer=(1), constraints=(/1: (/NULL - ])] + │ │ │ │ │ │ └── instance_types.deleted = $1 [type=bool, outer=(13), constraints=(/13: (/NULL - ])] + │ │ │ │ │ ├── project + │ │ │ │ │ │ ├── columns: true:25(bool!null) instance_type_projects.instance_type_id:18(int!null) + │ │ │ │ │ │ ├── has-placeholder + │ │ │ │ │ │ ├── fd: ()-->(25) + │ │ │ │ │ │ ├── ordering: +18 opt(25) [actual: +18] + │ │ │ │ │ │ ├── select + │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18(int!null) project_id:19(string!null) instance_type_projects.deleted:20(bool!null) │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ ├── fd: ()-->(25) - │ │ │ │ │ │ │ ├── ordering: +18 opt(25) [actual: +18] - │ │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18(int!null) project_id:19(string!null) instance_type_projects.deleted:20(bool!null) - │ │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ │ ├── key: (18-20) - │ │ │ │ │ │ │ │ ├── ordering: +18 - │ │ │ │ │ │ │ │ ├── scan instance_type_projects@secondary - │ │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18(int!null) project_id:19(string) instance_type_projects.deleted:20(bool) - │ │ │ │ │ │ │ │ │ ├── lax-key: (18-20) - │ │ │ │ │ │ │ │ │ └── ordering: +18 - │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ ├── instance_type_projects.deleted = $2 [type=bool, outer=(20), constraints=(/20: (/NULL - ])] - │ │ │ │ │ │ │ │ ├── project_id = $3 [type=bool, outer=(19), constraints=(/19: (/NULL - ])] - │ │ │ │ │ │ │ │ └── instance_type_projects.instance_type_id = $4 [type=bool, outer=(18), constraints=(/18: (/NULL - ])] - │ │ │ │ │ │ │ └── projections - │ │ │ │ │ │ │ └── true [type=bool] - │ │ │ │ │ │ └── filters (true) - │ │ │ │ │ └── aggregations - │ │ │ │ │ ├── const-not-null-agg [type=bool, outer=(25)] - │ │ │ │ │ │ └── variable: true [type=bool] - │ │ │ │ │ ├── const-agg [type=string, outer=(2)] - │ │ │ │ │ │ └── variable: name [type=string] - │ │ │ │ │ ├── const-agg [type=int, outer=(3)] - │ │ │ │ │ │ └── variable: memory_mb [type=int] - │ │ │ │ │ ├── const-agg [type=int, outer=(4)] - │ │ │ │ │ │ └── variable: vcpus [type=int] - │ │ │ │ │ ├── const-agg [type=int, outer=(5)] - │ │ │ │ │ │ └── variable: root_gb [type=int] - │ │ │ │ │ ├── const-agg [type=int, outer=(6)] - │ │ │ │ │ │ └── variable: ephemeral_gb [type=int] - │ │ │ │ │ ├── const-agg [type=string, outer=(7)] - │ │ │ │ │ │ └── variable: flavorid [type=string] - │ │ │ │ │ ├── const-agg [type=int, outer=(8)] - │ │ │ │ │ │ └── variable: swap [type=int] - │ │ │ │ │ ├── const-agg [type=float, outer=(9)] - │ │ │ │ │ │ └── variable: rxtx_factor [type=float] - │ │ │ │ │ ├── const-agg [type=int, outer=(10)] - │ │ │ │ │ │ └── variable: vcpu_weight [type=int] - │ │ │ │ │ ├── const-agg [type=bool, outer=(11)] - │ │ │ │ │ │ └── variable: disabled [type=bool] - │ │ │ │ │ ├── const-agg [type=bool, outer=(12)] - │ │ │ │ │ │ └── variable: is_public [type=bool] - │ │ │ │ │ ├── const-agg [type=bool, outer=(13)] - │ │ │ │ │ │ └── variable: instance_types.deleted [type=bool] - │ │ │ │ │ ├── const-agg [type=timestamp, outer=(14)] - │ │ │ │ │ │ └── variable: instance_types.deleted_at [type=timestamp] - │ │ │ │ │ ├── const-agg [type=timestamp, outer=(15)] - │ │ │ │ │ │ └── variable: instance_types.created_at [type=timestamp] - │ │ │ │ │ └── const-agg [type=timestamp, outer=(16)] - │ │ │ │ │ └── variable: instance_types.updated_at [type=timestamp] - │ │ │ │ └── filters - │ │ │ │ └── (is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,26)] - │ │ │ └── placeholder: $5 [type=int] - │ │ └── placeholder: $6 [type=int] - │ └── filters - │ └── instance_type_extra_specs_1.deleted = $7 [type=bool, outer=(32), constraints=(/32: (/NULL - ])] - └── filters (true) + │ │ │ │ │ │ │ ├── key: (18-20) + │ │ │ │ │ │ │ ├── ordering: +18 + │ │ │ │ │ │ │ ├── scan instance_type_projects@secondary + │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18(int!null) project_id:19(string) instance_type_projects.deleted:20(bool) + │ │ │ │ │ │ │ │ ├── lax-key: (18-20) + │ │ │ │ │ │ │ │ └── ordering: +18 + │ │ │ │ │ │ │ └── filters + │ │ │ │ │ │ │ ├── instance_type_projects.deleted = $2 [type=bool, outer=(20), constraints=(/20: (/NULL - ])] + │ │ │ │ │ │ │ ├── project_id = $3 [type=bool, outer=(19), constraints=(/19: (/NULL - ])] + │ │ │ │ │ │ │ └── instance_type_projects.instance_type_id = $4 [type=bool, outer=(18), constraints=(/18: (/NULL - ])] + │ │ │ │ │ │ └── projections + │ │ │ │ │ │ └── true [type=bool] + │ │ │ │ │ └── filters (true) + │ │ │ │ └── aggregations + │ │ │ │ ├── const-not-null-agg [type=bool, outer=(25)] + │ │ │ │ │ └── variable: true [type=bool] + │ │ │ │ ├── const-agg [type=string, outer=(2)] + │ │ │ │ │ └── variable: name [type=string] + │ │ │ │ ├── const-agg [type=int, outer=(3)] + │ │ │ │ │ └── variable: memory_mb [type=int] + │ │ │ │ ├── const-agg [type=int, outer=(4)] + │ │ │ │ │ └── variable: vcpus [type=int] + │ │ │ │ ├── const-agg [type=int, outer=(5)] + │ │ │ │ │ └── variable: root_gb [type=int] + │ │ │ │ ├── const-agg [type=int, outer=(6)] + │ │ │ │ │ └── variable: ephemeral_gb [type=int] + │ │ │ │ ├── const-agg [type=string, outer=(7)] + │ │ │ │ │ └── variable: flavorid [type=string] + │ │ │ │ ├── const-agg [type=int, outer=(8)] + │ │ │ │ │ └── variable: swap [type=int] + │ │ │ │ ├── const-agg [type=float, outer=(9)] + │ │ │ │ │ └── variable: rxtx_factor [type=float] + │ │ │ │ ├── const-agg [type=int, outer=(10)] + │ │ │ │ │ └── variable: vcpu_weight [type=int] + │ │ │ │ ├── const-agg [type=bool, outer=(11)] + │ │ │ │ │ └── variable: disabled [type=bool] + │ │ │ │ ├── const-agg [type=bool, outer=(12)] + │ │ │ │ │ └── variable: is_public [type=bool] + │ │ │ │ ├── const-agg [type=bool, outer=(13)] + │ │ │ │ │ └── variable: instance_types.deleted [type=bool] + │ │ │ │ ├── const-agg [type=timestamp, outer=(14)] + │ │ │ │ │ └── variable: instance_types.deleted_at [type=timestamp] + │ │ │ │ ├── const-agg [type=timestamp, outer=(15)] + │ │ │ │ │ └── variable: instance_types.created_at [type=timestamp] + │ │ │ │ └── const-agg [type=timestamp, outer=(16)] + │ │ │ │ └── variable: instance_types.updated_at [type=timestamp] + │ │ │ └── filters + │ │ │ └── (is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,26)] + │ │ └── placeholder: $5 [type=int] + │ └── placeholder: $6 [type=int] + └── filters + └── instance_type_extra_specs_1.instance_type_id = instance_types.id [type=bool, outer=(1,31), constraints=(/1: (/NULL - ]; /31: (/NULL - ]), fd=(1)==(31), (31)==(1)] opt select anon_1.flavors_created_at as anon_1_flavors_created_at, @@ -1997,121 +2003,124 @@ project ├── side-effects, 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) - └── left-join (lookup instance_type_extra_specs) + └── right-join (hash) ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) instance_type_extra_specs_1.id:28(int) key:29(string) value:30(string) instance_type_extra_specs_1.instance_type_id:31(int) instance_type_extra_specs_1.deleted:32(bool) instance_type_extra_specs_1.deleted_at:33(timestamp) instance_type_extra_specs_1.created_at:34(timestamp) instance_type_extra_specs_1.updated_at:35(timestamp) - ├── key columns: [28] = [28] ├── side-effects, 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) - ├── left-join (lookup instance_type_extra_specs@secondary) - │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) instance_type_extra_specs_1.id:28(int) key:29(string) instance_type_extra_specs_1.instance_type_id:31(int) instance_type_extra_specs_1.deleted:32(bool) - │ ├── key columns: [1] = [31] + ├── select + │ ├── columns: instance_type_extra_specs_1.id:28(int!null) key:29(string) value:30(string) instance_type_extra_specs_1.instance_type_id:31(int!null) instance_type_extra_specs_1.deleted:32(bool!null) instance_type_extra_specs_1.deleted_at:33(timestamp) instance_type_extra_specs_1.created_at:34(timestamp) instance_type_extra_specs_1.updated_at:35(timestamp) + │ ├── has-placeholder + │ ├── key: (28) + │ ├── fd: (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + │ ├── scan instance_type_extra_specs_1 + │ │ ├── columns: instance_type_extra_specs_1.id:28(int!null) key:29(string) value:30(string) instance_type_extra_specs_1.instance_type_id:31(int!null) instance_type_extra_specs_1.deleted:32(bool) instance_type_extra_specs_1.deleted_at:33(timestamp) instance_type_extra_specs_1.created_at:34(timestamp) instance_type_extra_specs_1.updated_at:35(timestamp) + │ │ ├── key: (28) + │ │ └── fd: (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + │ └── filters + │ └── instance_type_extra_specs_1.deleted = $7 [type=bool, outer=(32), constraints=(/32: (/NULL - ])] + ├── limit + │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) │ ├── side-effects, 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,31,32), (29,31,32)~~>(28) - │ ├── limit + │ ├── key: (1) + │ ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ ├── offset │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) - │ │ ├── side-effects, has-placeholder + │ │ ├── has-placeholder │ │ ├── key: (1) │ │ ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) - │ │ ├── offset + │ │ ├── select │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) │ │ │ ├── has-placeholder │ │ │ ├── key: (1) │ │ │ ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) - │ │ │ ├── select + │ │ │ ├── group-by │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) + │ │ │ │ ├── grouping columns: instance_types.id:1(int!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) - │ │ │ │ ├── group-by - │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) - │ │ │ │ │ ├── grouping columns: instance_types.id:1(int!null) - │ │ │ │ │ ├── internal-ordering: +1 + │ │ │ │ ├── left-join (merge) + │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int!null) vcpus:4(int!null) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string!null) swap:8(int!null) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool!null) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) instance_type_projects.instance_type_id:18(int) true:25(bool) + │ │ │ │ │ ├── left ordering: +1 + │ │ │ │ │ ├── right ordering: +18 │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ ├── key: (1) - │ │ │ │ │ ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) - │ │ │ │ │ ├── left-join (merge) - │ │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int!null) vcpus:4(int!null) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string!null) swap:8(int!null) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool!null) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) instance_type_projects.instance_type_id:18(int) true:25(bool) - │ │ │ │ │ │ ├── left ordering: +1 - │ │ │ │ │ │ ├── right ordering: +18 + │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), ()~~>(25) + │ │ │ │ │ ├── ordering: +1 + │ │ │ │ │ ├── select + │ │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int!null) vcpus:4(int!null) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string!null) swap:8(int!null) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool!null) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), ()~~>(25) + │ │ │ │ │ │ ├── key: (1) + │ │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) │ │ │ │ │ │ ├── ordering: +1 + │ │ │ │ │ │ ├── scan instance_types + │ │ │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int!null) vcpus:4(int!null) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int!null) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) + │ │ │ │ │ │ │ ├── key: (1) + │ │ │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ │ │ │ │ │ └── ordering: +1 + │ │ │ │ │ │ └── filters + │ │ │ │ │ │ ├── instance_types.deleted = $1 [type=bool, outer=(13), constraints=(/13: (/NULL - ])] + │ │ │ │ │ │ └── flavorid = $4 [type=bool, outer=(7), constraints=(/7: (/NULL - ])] + │ │ │ │ │ ├── project + │ │ │ │ │ │ ├── columns: true:25(bool!null) instance_type_projects.instance_type_id:18(int!null) + │ │ │ │ │ │ ├── has-placeholder + │ │ │ │ │ │ ├── fd: ()-->(25) + │ │ │ │ │ │ ├── ordering: +18 opt(25) [actual: +18] │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int!null) vcpus:4(int!null) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string!null) swap:8(int!null) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool!null) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) + │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18(int!null) project_id:19(string!null) instance_type_projects.deleted:20(bool!null) │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ ├── key: (1) - │ │ │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) - │ │ │ │ │ │ │ ├── ordering: +1 - │ │ │ │ │ │ │ ├── scan instance_types - │ │ │ │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int!null) vcpus:4(int!null) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int!null) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) - │ │ │ │ │ │ │ │ ├── key: (1) - │ │ │ │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) - │ │ │ │ │ │ │ │ └── ordering: +1 + │ │ │ │ │ │ │ ├── key: (18-20) + │ │ │ │ │ │ │ ├── ordering: +18 + │ │ │ │ │ │ │ ├── scan instance_type_projects@secondary + │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18(int!null) project_id:19(string) instance_type_projects.deleted:20(bool) + │ │ │ │ │ │ │ │ ├── lax-key: (18-20) + │ │ │ │ │ │ │ │ └── ordering: +18 │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ ├── instance_types.deleted = $1 [type=bool, outer=(13), constraints=(/13: (/NULL - ])] - │ │ │ │ │ │ │ └── flavorid = $4 [type=bool, outer=(7), constraints=(/7: (/NULL - ])] - │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ ├── columns: true:25(bool!null) instance_type_projects.instance_type_id:18(int!null) - │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ ├── fd: ()-->(25) - │ │ │ │ │ │ │ ├── ordering: +18 opt(25) [actual: +18] - │ │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18(int!null) project_id:19(string!null) instance_type_projects.deleted:20(bool!null) - │ │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ │ ├── key: (18-20) - │ │ │ │ │ │ │ │ ├── ordering: +18 - │ │ │ │ │ │ │ │ ├── scan instance_type_projects@secondary - │ │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18(int!null) project_id:19(string) instance_type_projects.deleted:20(bool) - │ │ │ │ │ │ │ │ │ ├── lax-key: (18-20) - │ │ │ │ │ │ │ │ │ └── ordering: +18 - │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ ├── instance_type_projects.deleted = $2 [type=bool, outer=(20), constraints=(/20: (/NULL - ])] - │ │ │ │ │ │ │ │ └── project_id = $3 [type=bool, outer=(19), constraints=(/19: (/NULL - ])] - │ │ │ │ │ │ │ └── projections - │ │ │ │ │ │ │ └── true [type=bool] - │ │ │ │ │ │ └── filters (true) - │ │ │ │ │ └── aggregations - │ │ │ │ │ ├── const-not-null-agg [type=bool, outer=(25)] - │ │ │ │ │ │ └── variable: true [type=bool] - │ │ │ │ │ ├── const-agg [type=string, outer=(2)] - │ │ │ │ │ │ └── variable: name [type=string] - │ │ │ │ │ ├── const-agg [type=int, outer=(3)] - │ │ │ │ │ │ └── variable: memory_mb [type=int] - │ │ │ │ │ ├── const-agg [type=int, outer=(4)] - │ │ │ │ │ │ └── variable: vcpus [type=int] - │ │ │ │ │ ├── const-agg [type=int, outer=(5)] - │ │ │ │ │ │ └── variable: root_gb [type=int] - │ │ │ │ │ ├── const-agg [type=int, outer=(6)] - │ │ │ │ │ │ └── variable: ephemeral_gb [type=int] - │ │ │ │ │ ├── const-agg [type=string, outer=(7)] - │ │ │ │ │ │ └── variable: flavorid [type=string] - │ │ │ │ │ ├── const-agg [type=int, outer=(8)] - │ │ │ │ │ │ └── variable: swap [type=int] - │ │ │ │ │ ├── const-agg [type=float, outer=(9)] - │ │ │ │ │ │ └── variable: rxtx_factor [type=float] - │ │ │ │ │ ├── const-agg [type=int, outer=(10)] - │ │ │ │ │ │ └── variable: vcpu_weight [type=int] - │ │ │ │ │ ├── const-agg [type=bool, outer=(11)] - │ │ │ │ │ │ └── variable: disabled [type=bool] - │ │ │ │ │ ├── const-agg [type=bool, outer=(12)] - │ │ │ │ │ │ └── variable: is_public [type=bool] - │ │ │ │ │ ├── const-agg [type=bool, outer=(13)] - │ │ │ │ │ │ └── variable: instance_types.deleted [type=bool] - │ │ │ │ │ ├── const-agg [type=timestamp, outer=(14)] - │ │ │ │ │ │ └── variable: instance_types.deleted_at [type=timestamp] - │ │ │ │ │ ├── const-agg [type=timestamp, outer=(15)] - │ │ │ │ │ │ └── variable: instance_types.created_at [type=timestamp] - │ │ │ │ │ └── const-agg [type=timestamp, outer=(16)] - │ │ │ │ │ └── variable: instance_types.updated_at [type=timestamp] - │ │ │ │ └── filters - │ │ │ │ └── (is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,26)] - │ │ │ └── placeholder: $5 [type=int] - │ │ └── placeholder: $6 [type=int] - │ └── filters - │ └── instance_type_extra_specs_1.deleted = $7 [type=bool, outer=(32), constraints=(/32: (/NULL - ])] - └── filters (true) + │ │ │ │ │ │ │ ├── instance_type_projects.deleted = $2 [type=bool, outer=(20), constraints=(/20: (/NULL - ])] + │ │ │ │ │ │ │ └── project_id = $3 [type=bool, outer=(19), constraints=(/19: (/NULL - ])] + │ │ │ │ │ │ └── projections + │ │ │ │ │ │ └── true [type=bool] + │ │ │ │ │ └── filters (true) + │ │ │ │ └── aggregations + │ │ │ │ ├── const-not-null-agg [type=bool, outer=(25)] + │ │ │ │ │ └── variable: true [type=bool] + │ │ │ │ ├── const-agg [type=string, outer=(2)] + │ │ │ │ │ └── variable: name [type=string] + │ │ │ │ ├── const-agg [type=int, outer=(3)] + │ │ │ │ │ └── variable: memory_mb [type=int] + │ │ │ │ ├── const-agg [type=int, outer=(4)] + │ │ │ │ │ └── variable: vcpus [type=int] + │ │ │ │ ├── const-agg [type=int, outer=(5)] + │ │ │ │ │ └── variable: root_gb [type=int] + │ │ │ │ ├── const-agg [type=int, outer=(6)] + │ │ │ │ │ └── variable: ephemeral_gb [type=int] + │ │ │ │ ├── const-agg [type=string, outer=(7)] + │ │ │ │ │ └── variable: flavorid [type=string] + │ │ │ │ ├── const-agg [type=int, outer=(8)] + │ │ │ │ │ └── variable: swap [type=int] + │ │ │ │ ├── const-agg [type=float, outer=(9)] + │ │ │ │ │ └── variable: rxtx_factor [type=float] + │ │ │ │ ├── const-agg [type=int, outer=(10)] + │ │ │ │ │ └── variable: vcpu_weight [type=int] + │ │ │ │ ├── const-agg [type=bool, outer=(11)] + │ │ │ │ │ └── variable: disabled [type=bool] + │ │ │ │ ├── const-agg [type=bool, outer=(12)] + │ │ │ │ │ └── variable: is_public [type=bool] + │ │ │ │ ├── const-agg [type=bool, outer=(13)] + │ │ │ │ │ └── variable: instance_types.deleted [type=bool] + │ │ │ │ ├── const-agg [type=timestamp, outer=(14)] + │ │ │ │ │ └── variable: instance_types.deleted_at [type=timestamp] + │ │ │ │ ├── const-agg [type=timestamp, outer=(15)] + │ │ │ │ │ └── variable: instance_types.created_at [type=timestamp] + │ │ │ │ └── const-agg [type=timestamp, outer=(16)] + │ │ │ │ └── variable: instance_types.updated_at [type=timestamp] + │ │ │ └── filters + │ │ │ └── (is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,26)] + │ │ └── placeholder: $5 [type=int] + │ └── placeholder: $6 [type=int] + └── filters + └── instance_type_extra_specs_1.instance_type_id = instance_types.id [type=bool, outer=(1,31), constraints=(/1: (/NULL - ]; /31: (/NULL - ]), fd=(1)==(31), (31)==(1)] opt select flavors.created_at as flavors_created_at, @@ -2303,139 +2312,144 @@ from (select instance_types.created_at as instance_types_created_at, order by anon_1.instance_types_flavorid asc, anon_1.instance_types_id asc ---- -project +sort ├── columns: anon_1_instance_types_created_at:15(timestamp) anon_1_instance_types_updated_at:16(timestamp) anon_1_instance_types_deleted_at:14(timestamp) anon_1_instance_types_deleted:13(bool) anon_1_instance_types_id:1(int!null) anon_1_instance_types_name:2(string) anon_1_instance_types_memory_mb:3(int) anon_1_instance_types_vcpus:4(int) anon_1_instance_types_root_gb:5(int) anon_1_instance_types_ephemeral_gb:6(int) anon_1_instance_types_flavorid:7(string) anon_1_instance_types_swap:8(int) anon_1_instance_types_rxtx_factor:9(float) anon_1_instance_types_vcpu_weight:10(int) anon_1_instance_types_disabled:11(bool) anon_1_instance_types_is_public:12(bool) instance_type_extra_specs_1_created_at:34(timestamp) instance_type_extra_specs_1_updated_at:35(timestamp) instance_type_extra_specs_1_deleted_at:33(timestamp) instance_type_extra_specs_1_deleted:32(bool) instance_type_extra_specs_1_id:28(int) instance_type_extra_specs_1_key:29(string) instance_type_extra_specs_1_value:30(string) instance_type_extra_specs_1_instance_type_id:31(int) ├── side-effects, 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) ├── ordering: +7,+1 - └── left-join (lookup instance_type_extra_specs) - ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) instance_type_extra_specs_1.id:28(int) key:29(string) value:30(string) instance_type_extra_specs_1.instance_type_id:31(int) instance_type_extra_specs_1.deleted:32(bool) instance_type_extra_specs_1.deleted_at:33(timestamp) instance_type_extra_specs_1.created_at:34(timestamp) instance_type_extra_specs_1.updated_at:35(timestamp) - ├── key columns: [28] = [28] + └── project + ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) instance_type_extra_specs_1.id:28(int) key:29(string) value:30(string) instance_type_extra_specs_1.instance_type_id:31(int) instance_type_extra_specs_1.deleted:32(bool) instance_type_extra_specs_1.deleted_at:33(timestamp) instance_type_extra_specs_1.created_at:34(timestamp) instance_type_extra_specs_1.updated_at:35(timestamp) ├── side-effects, 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) - ├── ordering: +7,+1 - ├── left-join (lookup instance_type_extra_specs@secondary) - │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) instance_type_extra_specs_1.id:28(int) key:29(string) instance_type_extra_specs_1.instance_type_id:31(int) instance_type_extra_specs_1.deleted:32(bool) - │ ├── key columns: [1] = [31] - │ ├── side-effects, 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,31,32), (29,31,32)~~>(28) - │ ├── ordering: +7,+1 - │ ├── limit - │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) - │ │ ├── internal-ordering: +7,+1 - │ │ ├── side-effects, has-placeholder - │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) - │ │ ├── ordering: +7,+1 - │ │ ├── offset - │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) - │ │ │ ├── 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) - │ │ │ ├── ordering: +7,+1 - │ │ │ ├── sort - │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) - │ │ │ │ ├── has-placeholder - │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) - │ │ │ │ ├── ordering: +7,+1 - │ │ │ │ └── select - │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) - │ │ │ │ ├── has-placeholder - │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) - │ │ │ │ ├── group-by - │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) - │ │ │ │ │ ├── grouping columns: instance_types.id:1(int!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) - │ │ │ │ │ ├── left-join (merge) - │ │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int!null) vcpus:4(int!null) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string!null) swap:8(int!null) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool!null) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) instance_type_projects.instance_type_id:18(int) true:25(bool) - │ │ │ │ │ │ ├── left ordering: +1 - │ │ │ │ │ │ ├── right ordering: +18 - │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), ()~~>(25) - │ │ │ │ │ │ ├── ordering: +1 - │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int!null) vcpus:4(int!null) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string!null) swap:8(int!null) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool!null) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) - │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ ├── key: (1) - │ │ │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) - │ │ │ │ │ │ │ ├── ordering: +1 - │ │ │ │ │ │ │ ├── scan instance_types - │ │ │ │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int!null) vcpus:4(int!null) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int!null) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) - │ │ │ │ │ │ │ │ ├── key: (1) - │ │ │ │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) - │ │ │ │ │ │ │ │ └── ordering: +1 - │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ ├── instance_types.deleted = $1 [type=bool, outer=(13), constraints=(/13: (/NULL - ])] - │ │ │ │ │ │ │ └── (flavorid > $4) OR ((flavorid = $5) AND (instance_types.id > $6)) [type=bool, outer=(1,7), constraints=(/7: (/NULL - ])] - │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ ├── columns: true:25(bool!null) instance_type_projects.instance_type_id:18(int!null) - │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ ├── fd: ()-->(25) - │ │ │ │ │ │ │ ├── ordering: +18 opt(25) [actual: +18] - │ │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18(int!null) project_id:19(string!null) instance_type_projects.deleted:20(bool!null) - │ │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ │ ├── key: (18-20) - │ │ │ │ │ │ │ │ ├── ordering: +18 - │ │ │ │ │ │ │ │ ├── scan instance_type_projects@secondary - │ │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18(int!null) project_id:19(string) instance_type_projects.deleted:20(bool) - │ │ │ │ │ │ │ │ │ ├── lax-key: (18-20) - │ │ │ │ │ │ │ │ │ └── ordering: +18 - │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ ├── instance_type_projects.deleted = $2 [type=bool, outer=(20), constraints=(/20: (/NULL - ])] - │ │ │ │ │ │ │ │ └── project_id = $3 [type=bool, outer=(19), constraints=(/19: (/NULL - ])] - │ │ │ │ │ │ │ └── projections - │ │ │ │ │ │ │ └── true [type=bool] - │ │ │ │ │ │ └── filters (true) - │ │ │ │ │ └── aggregations - │ │ │ │ │ ├── const-not-null-agg [type=bool, outer=(25)] - │ │ │ │ │ │ └── variable: true [type=bool] - │ │ │ │ │ ├── const-agg [type=string, outer=(2)] - │ │ │ │ │ │ └── variable: name [type=string] - │ │ │ │ │ ├── const-agg [type=int, outer=(3)] - │ │ │ │ │ │ └── variable: memory_mb [type=int] - │ │ │ │ │ ├── const-agg [type=int, outer=(4)] - │ │ │ │ │ │ └── variable: vcpus [type=int] - │ │ │ │ │ ├── const-agg [type=int, outer=(5)] - │ │ │ │ │ │ └── variable: root_gb [type=int] - │ │ │ │ │ ├── const-agg [type=int, outer=(6)] - │ │ │ │ │ │ └── variable: ephemeral_gb [type=int] - │ │ │ │ │ ├── const-agg [type=string, outer=(7)] - │ │ │ │ │ │ └── variable: flavorid [type=string] - │ │ │ │ │ ├── const-agg [type=int, outer=(8)] - │ │ │ │ │ │ └── variable: swap [type=int] - │ │ │ │ │ ├── const-agg [type=float, outer=(9)] - │ │ │ │ │ │ └── variable: rxtx_factor [type=float] - │ │ │ │ │ ├── const-agg [type=int, outer=(10)] - │ │ │ │ │ │ └── variable: vcpu_weight [type=int] - │ │ │ │ │ ├── const-agg [type=bool, outer=(11)] - │ │ │ │ │ │ └── variable: disabled [type=bool] - │ │ │ │ │ ├── const-agg [type=bool, outer=(12)] - │ │ │ │ │ │ └── variable: is_public [type=bool] - │ │ │ │ │ ├── const-agg [type=bool, outer=(13)] - │ │ │ │ │ │ └── variable: instance_types.deleted [type=bool] - │ │ │ │ │ ├── const-agg [type=timestamp, outer=(14)] - │ │ │ │ │ │ └── variable: instance_types.deleted_at [type=timestamp] - │ │ │ │ │ ├── const-agg [type=timestamp, outer=(15)] - │ │ │ │ │ │ └── variable: instance_types.created_at [type=timestamp] - │ │ │ │ │ └── const-agg [type=timestamp, outer=(16)] - │ │ │ │ │ └── variable: instance_types.updated_at [type=timestamp] - │ │ │ │ └── filters - │ │ │ │ └── (is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,26)] - │ │ │ └── placeholder: $7 [type=int] - │ │ └── placeholder: $8 [type=int] - │ └── filters - │ └── instance_type_extra_specs_1.deleted = $9 [type=bool, outer=(32), constraints=(/32: (/NULL - ])] - └── filters (true) + ├── 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) + └── right-join (hash) + ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) instance_type_extra_specs_1.id:28(int) key:29(string) value:30(string) instance_type_extra_specs_1.instance_type_id:31(int) instance_type_extra_specs_1.deleted:32(bool) instance_type_extra_specs_1.deleted_at:33(timestamp) instance_type_extra_specs_1.created_at:34(timestamp) instance_type_extra_specs_1.updated_at:35(timestamp) + ├── side-effects, 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) + ├── select + │ ├── columns: instance_type_extra_specs_1.id:28(int!null) key:29(string) value:30(string) instance_type_extra_specs_1.instance_type_id:31(int!null) instance_type_extra_specs_1.deleted:32(bool!null) instance_type_extra_specs_1.deleted_at:33(timestamp) instance_type_extra_specs_1.created_at:34(timestamp) instance_type_extra_specs_1.updated_at:35(timestamp) + │ ├── has-placeholder + │ ├── key: (28) + │ ├── fd: (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + │ ├── scan instance_type_extra_specs_1 + │ │ ├── columns: instance_type_extra_specs_1.id:28(int!null) key:29(string) value:30(string) instance_type_extra_specs_1.instance_type_id:31(int!null) instance_type_extra_specs_1.deleted:32(bool) instance_type_extra_specs_1.deleted_at:33(timestamp) instance_type_extra_specs_1.created_at:34(timestamp) instance_type_extra_specs_1.updated_at:35(timestamp) + │ │ ├── key: (28) + │ │ └── fd: (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + │ └── filters + │ └── instance_type_extra_specs_1.deleted = $9 [type=bool, outer=(32), constraints=(/32: (/NULL - ])] + ├── limit + │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) + │ ├── internal-ordering: +7,+1 + │ ├── side-effects, has-placeholder + │ ├── key: (1) + │ ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ ├── offset + │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) + │ │ ├── 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) + │ │ ├── ordering: +7,+1 + │ │ ├── sort + │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) + │ │ │ ├── has-placeholder + │ │ │ ├── key: (1) + │ │ │ ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ │ ├── ordering: +7,+1 + │ │ │ └── select + │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) + │ │ │ ├── has-placeholder + │ │ │ ├── key: (1) + │ │ │ ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ │ ├── group-by + │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) + │ │ │ │ ├── grouping columns: instance_types.id:1(int!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) + │ │ │ │ ├── left-join (merge) + │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int!null) vcpus:4(int!null) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string!null) swap:8(int!null) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool!null) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) instance_type_projects.instance_type_id:18(int) true:25(bool) + │ │ │ │ │ ├── left ordering: +1 + │ │ │ │ │ ├── right ordering: +18 + │ │ │ │ │ ├── has-placeholder + │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), ()~~>(25) + │ │ │ │ │ ├── ordering: +1 + │ │ │ │ │ ├── select + │ │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int!null) vcpus:4(int!null) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string!null) swap:8(int!null) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool!null) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) + │ │ │ │ │ │ ├── has-placeholder + │ │ │ │ │ │ ├── key: (1) + │ │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ │ │ │ │ ├── ordering: +1 + │ │ │ │ │ │ ├── scan instance_types + │ │ │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int!null) vcpus:4(int!null) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int!null) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) + │ │ │ │ │ │ │ ├── key: (1) + │ │ │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ │ │ │ │ │ └── ordering: +1 + │ │ │ │ │ │ └── filters + │ │ │ │ │ │ ├── instance_types.deleted = $1 [type=bool, outer=(13), constraints=(/13: (/NULL - ])] + │ │ │ │ │ │ └── (flavorid > $4) OR ((flavorid = $5) AND (instance_types.id > $6)) [type=bool, outer=(1,7), constraints=(/7: (/NULL - ])] + │ │ │ │ │ ├── project + │ │ │ │ │ │ ├── columns: true:25(bool!null) instance_type_projects.instance_type_id:18(int!null) + │ │ │ │ │ │ ├── has-placeholder + │ │ │ │ │ │ ├── fd: ()-->(25) + │ │ │ │ │ │ ├── ordering: +18 opt(25) [actual: +18] + │ │ │ │ │ │ ├── select + │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18(int!null) project_id:19(string!null) instance_type_projects.deleted:20(bool!null) + │ │ │ │ │ │ │ ├── has-placeholder + │ │ │ │ │ │ │ ├── key: (18-20) + │ │ │ │ │ │ │ ├── ordering: +18 + │ │ │ │ │ │ │ ├── scan instance_type_projects@secondary + │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18(int!null) project_id:19(string) instance_type_projects.deleted:20(bool) + │ │ │ │ │ │ │ │ ├── lax-key: (18-20) + │ │ │ │ │ │ │ │ └── ordering: +18 + │ │ │ │ │ │ │ └── filters + │ │ │ │ │ │ │ ├── instance_type_projects.deleted = $2 [type=bool, outer=(20), constraints=(/20: (/NULL - ])] + │ │ │ │ │ │ │ └── project_id = $3 [type=bool, outer=(19), constraints=(/19: (/NULL - ])] + │ │ │ │ │ │ └── projections + │ │ │ │ │ │ └── true [type=bool] + │ │ │ │ │ └── filters (true) + │ │ │ │ └── aggregations + │ │ │ │ ├── const-not-null-agg [type=bool, outer=(25)] + │ │ │ │ │ └── variable: true [type=bool] + │ │ │ │ ├── const-agg [type=string, outer=(2)] + │ │ │ │ │ └── variable: name [type=string] + │ │ │ │ ├── const-agg [type=int, outer=(3)] + │ │ │ │ │ └── variable: memory_mb [type=int] + │ │ │ │ ├── const-agg [type=int, outer=(4)] + │ │ │ │ │ └── variable: vcpus [type=int] + │ │ │ │ ├── const-agg [type=int, outer=(5)] + │ │ │ │ │ └── variable: root_gb [type=int] + │ │ │ │ ├── const-agg [type=int, outer=(6)] + │ │ │ │ │ └── variable: ephemeral_gb [type=int] + │ │ │ │ ├── const-agg [type=string, outer=(7)] + │ │ │ │ │ └── variable: flavorid [type=string] + │ │ │ │ ├── const-agg [type=int, outer=(8)] + │ │ │ │ │ └── variable: swap [type=int] + │ │ │ │ ├── const-agg [type=float, outer=(9)] + │ │ │ │ │ └── variable: rxtx_factor [type=float] + │ │ │ │ ├── const-agg [type=int, outer=(10)] + │ │ │ │ │ └── variable: vcpu_weight [type=int] + │ │ │ │ ├── const-agg [type=bool, outer=(11)] + │ │ │ │ │ └── variable: disabled [type=bool] + │ │ │ │ ├── const-agg [type=bool, outer=(12)] + │ │ │ │ │ └── variable: is_public [type=bool] + │ │ │ │ ├── const-agg [type=bool, outer=(13)] + │ │ │ │ │ └── variable: instance_types.deleted [type=bool] + │ │ │ │ ├── const-agg [type=timestamp, outer=(14)] + │ │ │ │ │ └── variable: instance_types.deleted_at [type=timestamp] + │ │ │ │ ├── const-agg [type=timestamp, outer=(15)] + │ │ │ │ │ └── variable: instance_types.created_at [type=timestamp] + │ │ │ │ └── const-agg [type=timestamp, outer=(16)] + │ │ │ │ └── variable: instance_types.updated_at [type=timestamp] + │ │ │ └── filters + │ │ │ └── (is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,26)] + │ │ └── placeholder: $7 [type=int] + │ └── placeholder: $8 [type=int] + └── filters + └── instance_type_extra_specs_1.instance_type_id = instance_types.id [type=bool, outer=(1,31), constraints=(/1: (/NULL - ]; /31: (/NULL - ]), fd=(1)==(31), (31)==(1)] opt select anon_1.flavors_created_at as anon_1_flavors_created_at, @@ -2941,139 +2955,144 @@ from (select instance_types.created_at as instance_types_created_at, order by anon_1.instance_types_deleted asc, anon_1.instance_types_id asc ---- -project +sort ├── columns: anon_1_instance_types_created_at:15(timestamp) anon_1_instance_types_updated_at:16(timestamp) anon_1_instance_types_deleted_at:14(timestamp) anon_1_instance_types_deleted:13(bool) anon_1_instance_types_id:1(int!null) anon_1_instance_types_name:2(string) anon_1_instance_types_memory_mb:3(int) anon_1_instance_types_vcpus:4(int) anon_1_instance_types_root_gb:5(int) anon_1_instance_types_ephemeral_gb:6(int) anon_1_instance_types_flavorid:7(string) anon_1_instance_types_swap:8(int) anon_1_instance_types_rxtx_factor:9(float) anon_1_instance_types_vcpu_weight:10(int) anon_1_instance_types_disabled:11(bool) anon_1_instance_types_is_public:12(bool) instance_type_extra_specs_1_created_at:34(timestamp) instance_type_extra_specs_1_updated_at:35(timestamp) instance_type_extra_specs_1_deleted_at:33(timestamp) instance_type_extra_specs_1_deleted:32(bool) instance_type_extra_specs_1_id:28(int) instance_type_extra_specs_1_key:29(string) instance_type_extra_specs_1_value:30(string) instance_type_extra_specs_1_instance_type_id:31(int) ├── side-effects, 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) ├── ordering: +13,+1 - └── left-join (lookup instance_type_extra_specs) - ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) instance_type_extra_specs_1.id:28(int) key:29(string) value:30(string) instance_type_extra_specs_1.instance_type_id:31(int) instance_type_extra_specs_1.deleted:32(bool) instance_type_extra_specs_1.deleted_at:33(timestamp) instance_type_extra_specs_1.created_at:34(timestamp) instance_type_extra_specs_1.updated_at:35(timestamp) - ├── key columns: [28] = [28] + └── project + ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) instance_type_extra_specs_1.id:28(int) key:29(string) value:30(string) instance_type_extra_specs_1.instance_type_id:31(int) instance_type_extra_specs_1.deleted:32(bool) instance_type_extra_specs_1.deleted_at:33(timestamp) instance_type_extra_specs_1.created_at:34(timestamp) instance_type_extra_specs_1.updated_at:35(timestamp) ├── side-effects, 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) - ├── ordering: +13,+1 - ├── left-join (lookup instance_type_extra_specs@secondary) - │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) instance_type_extra_specs_1.id:28(int) key:29(string) instance_type_extra_specs_1.instance_type_id:31(int) instance_type_extra_specs_1.deleted:32(bool) - │ ├── key columns: [1] = [31] - │ ├── side-effects, 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,31,32), (29,31,32)~~>(28) - │ ├── ordering: +13,+1 - │ ├── limit - │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) - │ │ ├── internal-ordering: +13,+1 - │ │ ├── side-effects, has-placeholder - │ │ ├── key: (1) - │ │ ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) - │ │ ├── ordering: +13,+1 - │ │ ├── offset - │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) - │ │ │ ├── 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) - │ │ │ ├── ordering: +13,+1 - │ │ │ ├── sort - │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) - │ │ │ │ ├── has-placeholder - │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) - │ │ │ │ ├── ordering: +13,+1 - │ │ │ │ └── select - │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) - │ │ │ │ ├── has-placeholder - │ │ │ │ ├── key: (1) - │ │ │ │ ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) - │ │ │ │ ├── group-by - │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) - │ │ │ │ │ ├── grouping columns: instance_types.id:1(int!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) - │ │ │ │ │ ├── left-join (merge) - │ │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int!null) vcpus:4(int!null) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string!null) swap:8(int!null) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool!null) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) instance_type_projects.instance_type_id:18(int) true:25(bool) - │ │ │ │ │ │ ├── left ordering: +1 - │ │ │ │ │ │ ├── right ordering: +18 - │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), ()~~>(25) - │ │ │ │ │ │ ├── ordering: +1 - │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int!null) vcpus:4(int!null) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string!null) swap:8(int!null) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool!null) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) - │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ ├── key: (1) - │ │ │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) - │ │ │ │ │ │ │ ├── ordering: +1 - │ │ │ │ │ │ │ ├── scan instance_types - │ │ │ │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int!null) vcpus:4(int!null) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int!null) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) - │ │ │ │ │ │ │ │ ├── key: (1) - │ │ │ │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) - │ │ │ │ │ │ │ │ └── ordering: +1 - │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ ├── instance_types.deleted = $1 [type=bool, outer=(13), constraints=(/13: (/NULL - ])] - │ │ │ │ │ │ │ └── flavorid = $4 [type=bool, outer=(7), constraints=(/7: (/NULL - ])] - │ │ │ │ │ │ ├── project - │ │ │ │ │ │ │ ├── columns: true:25(bool!null) instance_type_projects.instance_type_id:18(int!null) - │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ ├── fd: ()-->(25) - │ │ │ │ │ │ │ ├── ordering: +18 opt(25) [actual: +18] - │ │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18(int!null) project_id:19(string!null) instance_type_projects.deleted:20(bool!null) - │ │ │ │ │ │ │ │ ├── has-placeholder - │ │ │ │ │ │ │ │ ├── key: (18-20) - │ │ │ │ │ │ │ │ ├── ordering: +18 - │ │ │ │ │ │ │ │ ├── scan instance_type_projects@secondary - │ │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18(int!null) project_id:19(string) instance_type_projects.deleted:20(bool) - │ │ │ │ │ │ │ │ │ ├── lax-key: (18-20) - │ │ │ │ │ │ │ │ │ └── ordering: +18 - │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ ├── instance_type_projects.deleted = $2 [type=bool, outer=(20), constraints=(/20: (/NULL - ])] - │ │ │ │ │ │ │ │ └── project_id = $3 [type=bool, outer=(19), constraints=(/19: (/NULL - ])] - │ │ │ │ │ │ │ └── projections - │ │ │ │ │ │ │ └── true [type=bool] - │ │ │ │ │ │ └── filters (true) - │ │ │ │ │ └── aggregations - │ │ │ │ │ ├── const-not-null-agg [type=bool, outer=(25)] - │ │ │ │ │ │ └── variable: true [type=bool] - │ │ │ │ │ ├── const-agg [type=string, outer=(2)] - │ │ │ │ │ │ └── variable: name [type=string] - │ │ │ │ │ ├── const-agg [type=int, outer=(3)] - │ │ │ │ │ │ └── variable: memory_mb [type=int] - │ │ │ │ │ ├── const-agg [type=int, outer=(4)] - │ │ │ │ │ │ └── variable: vcpus [type=int] - │ │ │ │ │ ├── const-agg [type=int, outer=(5)] - │ │ │ │ │ │ └── variable: root_gb [type=int] - │ │ │ │ │ ├── const-agg [type=int, outer=(6)] - │ │ │ │ │ │ └── variable: ephemeral_gb [type=int] - │ │ │ │ │ ├── const-agg [type=string, outer=(7)] - │ │ │ │ │ │ └── variable: flavorid [type=string] - │ │ │ │ │ ├── const-agg [type=int, outer=(8)] - │ │ │ │ │ │ └── variable: swap [type=int] - │ │ │ │ │ ├── const-agg [type=float, outer=(9)] - │ │ │ │ │ │ └── variable: rxtx_factor [type=float] - │ │ │ │ │ ├── const-agg [type=int, outer=(10)] - │ │ │ │ │ │ └── variable: vcpu_weight [type=int] - │ │ │ │ │ ├── const-agg [type=bool, outer=(11)] - │ │ │ │ │ │ └── variable: disabled [type=bool] - │ │ │ │ │ ├── const-agg [type=bool, outer=(12)] - │ │ │ │ │ │ └── variable: is_public [type=bool] - │ │ │ │ │ ├── const-agg [type=bool, outer=(13)] - │ │ │ │ │ │ └── variable: instance_types.deleted [type=bool] - │ │ │ │ │ ├── const-agg [type=timestamp, outer=(14)] - │ │ │ │ │ │ └── variable: instance_types.deleted_at [type=timestamp] - │ │ │ │ │ ├── const-agg [type=timestamp, outer=(15)] - │ │ │ │ │ │ └── variable: instance_types.created_at [type=timestamp] - │ │ │ │ │ └── const-agg [type=timestamp, outer=(16)] - │ │ │ │ │ └── variable: instance_types.updated_at [type=timestamp] - │ │ │ │ └── filters - │ │ │ │ └── (is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,26)] - │ │ │ └── placeholder: $5 [type=int] - │ │ └── placeholder: $6 [type=int] - │ └── filters - │ └── instance_type_extra_specs_1.deleted = $7 [type=bool, outer=(32), constraints=(/32: (/NULL - ])] - └── filters (true) + ├── 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) + └── right-join (hash) + ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) instance_type_extra_specs_1.id:28(int) key:29(string) value:30(string) instance_type_extra_specs_1.instance_type_id:31(int) instance_type_extra_specs_1.deleted:32(bool) instance_type_extra_specs_1.deleted_at:33(timestamp) instance_type_extra_specs_1.created_at:34(timestamp) instance_type_extra_specs_1.updated_at:35(timestamp) + ├── side-effects, 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) + ├── select + │ ├── columns: instance_type_extra_specs_1.id:28(int!null) key:29(string) value:30(string) instance_type_extra_specs_1.instance_type_id:31(int!null) instance_type_extra_specs_1.deleted:32(bool!null) instance_type_extra_specs_1.deleted_at:33(timestamp) instance_type_extra_specs_1.created_at:34(timestamp) instance_type_extra_specs_1.updated_at:35(timestamp) + │ ├── has-placeholder + │ ├── key: (28) + │ ├── fd: (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + │ ├── scan instance_type_extra_specs_1 + │ │ ├── columns: instance_type_extra_specs_1.id:28(int!null) key:29(string) value:30(string) instance_type_extra_specs_1.instance_type_id:31(int!null) instance_type_extra_specs_1.deleted:32(bool) instance_type_extra_specs_1.deleted_at:33(timestamp) instance_type_extra_specs_1.created_at:34(timestamp) instance_type_extra_specs_1.updated_at:35(timestamp) + │ │ ├── key: (28) + │ │ └── fd: (28)-->(29-35), (29,31,32)~~>(28,30,33-35) + │ └── filters + │ └── instance_type_extra_specs_1.deleted = $7 [type=bool, outer=(32), constraints=(/32: (/NULL - ])] + ├── limit + │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) + │ ├── internal-ordering: +13,+1 + │ ├── side-effects, has-placeholder + │ ├── key: (1) + │ ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ ├── offset + │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) + │ │ ├── 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) + │ │ ├── ordering: +13,+1 + │ │ ├── sort + │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) + │ │ │ ├── has-placeholder + │ │ │ ├── key: (1) + │ │ │ ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ │ ├── ordering: +13,+1 + │ │ │ └── select + │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) + │ │ │ ├── has-placeholder + │ │ │ ├── key: (1) + │ │ │ ├── fd: (1)-->(2-16,26), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ │ ├── group-by + │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int) vcpus:4(int) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) true_agg:26(bool) + │ │ │ │ ├── grouping columns: instance_types.id:1(int!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) + │ │ │ │ ├── left-join (merge) + │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int!null) vcpus:4(int!null) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string!null) swap:8(int!null) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool!null) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) instance_type_projects.instance_type_id:18(int) true:25(bool) + │ │ │ │ │ ├── left ordering: +1 + │ │ │ │ │ ├── right ordering: +18 + │ │ │ │ │ ├── has-placeholder + │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16), ()~~>(25) + │ │ │ │ │ ├── ordering: +1 + │ │ │ │ │ ├── select + │ │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int!null) vcpus:4(int!null) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string!null) swap:8(int!null) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool!null) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) + │ │ │ │ │ │ ├── has-placeholder + │ │ │ │ │ │ ├── key: (1) + │ │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)-->(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ │ │ │ │ ├── ordering: +1 + │ │ │ │ │ │ ├── scan instance_types + │ │ │ │ │ │ │ ├── columns: instance_types.id:1(int!null) name:2(string) memory_mb:3(int!null) vcpus:4(int!null) root_gb:5(int) ephemeral_gb:6(int) flavorid:7(string) swap:8(int!null) rxtx_factor:9(float) vcpu_weight:10(int) disabled:11(bool) is_public:12(bool) instance_types.deleted:13(bool) instance_types.deleted_at:14(timestamp) instance_types.created_at:15(timestamp) instance_types.updated_at:16(timestamp) + │ │ │ │ │ │ │ ├── key: (1) + │ │ │ │ │ │ │ ├── fd: (1)-->(2-16), (7,13)~~>(1-6,8-12,14-16), (2,13)~~>(1,3-12,14-16) + │ │ │ │ │ │ │ └── ordering: +1 + │ │ │ │ │ │ └── filters + │ │ │ │ │ │ ├── instance_types.deleted = $1 [type=bool, outer=(13), constraints=(/13: (/NULL - ])] + │ │ │ │ │ │ └── flavorid = $4 [type=bool, outer=(7), constraints=(/7: (/NULL - ])] + │ │ │ │ │ ├── project + │ │ │ │ │ │ ├── columns: true:25(bool!null) instance_type_projects.instance_type_id:18(int!null) + │ │ │ │ │ │ ├── has-placeholder + │ │ │ │ │ │ ├── fd: ()-->(25) + │ │ │ │ │ │ ├── ordering: +18 opt(25) [actual: +18] + │ │ │ │ │ │ ├── select + │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18(int!null) project_id:19(string!null) instance_type_projects.deleted:20(bool!null) + │ │ │ │ │ │ │ ├── has-placeholder + │ │ │ │ │ │ │ ├── key: (18-20) + │ │ │ │ │ │ │ ├── ordering: +18 + │ │ │ │ │ │ │ ├── scan instance_type_projects@secondary + │ │ │ │ │ │ │ │ ├── columns: instance_type_projects.instance_type_id:18(int!null) project_id:19(string) instance_type_projects.deleted:20(bool) + │ │ │ │ │ │ │ │ ├── lax-key: (18-20) + │ │ │ │ │ │ │ │ └── ordering: +18 + │ │ │ │ │ │ │ └── filters + │ │ │ │ │ │ │ ├── instance_type_projects.deleted = $2 [type=bool, outer=(20), constraints=(/20: (/NULL - ])] + │ │ │ │ │ │ │ └── project_id = $3 [type=bool, outer=(19), constraints=(/19: (/NULL - ])] + │ │ │ │ │ │ └── projections + │ │ │ │ │ │ └── true [type=bool] + │ │ │ │ │ └── filters (true) + │ │ │ │ └── aggregations + │ │ │ │ ├── const-not-null-agg [type=bool, outer=(25)] + │ │ │ │ │ └── variable: true [type=bool] + │ │ │ │ ├── const-agg [type=string, outer=(2)] + │ │ │ │ │ └── variable: name [type=string] + │ │ │ │ ├── const-agg [type=int, outer=(3)] + │ │ │ │ │ └── variable: memory_mb [type=int] + │ │ │ │ ├── const-agg [type=int, outer=(4)] + │ │ │ │ │ └── variable: vcpus [type=int] + │ │ │ │ ├── const-agg [type=int, outer=(5)] + │ │ │ │ │ └── variable: root_gb [type=int] + │ │ │ │ ├── const-agg [type=int, outer=(6)] + │ │ │ │ │ └── variable: ephemeral_gb [type=int] + │ │ │ │ ├── const-agg [type=string, outer=(7)] + │ │ │ │ │ └── variable: flavorid [type=string] + │ │ │ │ ├── const-agg [type=int, outer=(8)] + │ │ │ │ │ └── variable: swap [type=int] + │ │ │ │ ├── const-agg [type=float, outer=(9)] + │ │ │ │ │ └── variable: rxtx_factor [type=float] + │ │ │ │ ├── const-agg [type=int, outer=(10)] + │ │ │ │ │ └── variable: vcpu_weight [type=int] + │ │ │ │ ├── const-agg [type=bool, outer=(11)] + │ │ │ │ │ └── variable: disabled [type=bool] + │ │ │ │ ├── const-agg [type=bool, outer=(12)] + │ │ │ │ │ └── variable: is_public [type=bool] + │ │ │ │ ├── const-agg [type=bool, outer=(13)] + │ │ │ │ │ └── variable: instance_types.deleted [type=bool] + │ │ │ │ ├── const-agg [type=timestamp, outer=(14)] + │ │ │ │ │ └── variable: instance_types.deleted_at [type=timestamp] + │ │ │ │ ├── const-agg [type=timestamp, outer=(15)] + │ │ │ │ │ └── variable: instance_types.created_at [type=timestamp] + │ │ │ │ └── const-agg [type=timestamp, outer=(16)] + │ │ │ │ └── variable: instance_types.updated_at [type=timestamp] + │ │ │ └── filters + │ │ │ └── (is_public = true) OR (true_agg IS NOT NULL) [type=bool, outer=(12,26)] + │ │ └── placeholder: $5 [type=int] + │ └── placeholder: $6 [type=int] + └── filters + └── instance_type_extra_specs_1.instance_type_id = instance_types.id [type=bool, outer=(1,31), constraints=(/1: (/NULL - ]; /31: (/NULL - ]), fd=(1)==(31), (31)==(1)] opt select anon_1.flavors_created_at as anon_1_flavors_created_at, diff --git a/pkg/sql/opt/xform/testdata/external/tpcc b/pkg/sql/opt/xform/testdata/external/tpcc index 513b26c5b5b4..fc4b20b2a77a 100644 --- a/pkg/sql/opt/xform/testdata/external/tpcc +++ b/pkg/sql/opt/xform/testdata/external/tpcc @@ -176,7 +176,7 @@ insert "order" ├── cardinality: [0 - 0] ├── side-effects, mutations ├── stats: [rows=0] - ├── cost: 6.15 + ├── cost: 6.29 ├── values │ ├── columns: column1:9(int!null) column2:10(int!null) column3:11(int!null) column4:12(int!null) column5:13(timestamp!null) column6:14(int!null) column7:15(int!null) column16:16(int) │ ├── cardinality: [1 - 1] @@ -196,12 +196,12 @@ insert "order" │ └── null [type=int] └── 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@customer_idx) + └── anti-join (lookup customer) ├── columns: column3:38(int!null) column2:39(int!null) column4:40(int!null) - ├── key columns: [38 39] = [19 18] + ├── key columns: [38 39 40] = [19 18 17] ├── cardinality: [0 - 1] - ├── stats: [rows=1] - ├── cost: 6.12 + ├── stats: [rows=1e-10] + ├── cost: 6.26 ├── key: () ├── fd: ()-->(9-16), (11)-->(38), (10)-->(39), (12)-->(40) ├── with-scan &1 @@ -215,10 +215,7 @@ insert "order" │ ├── cost: 0.01 │ ├── key: () │ └── fd: ()-->(9-16), (11)-->(38), (10)-->(39), (12)-->(40) - └── filters - └── eq [type=bool, outer=(17,40), constraints=(/17: (/NULL - ]; /40: (/NULL - ]), fd=(17)==(40), (40)==(17)] - ├── variable: column4 [type=int] - └── variable: c_id [type=int] + └── filters (true) opt format=hide-qual INSERT INTO new_order (no_o_id, no_d_id, no_w_id) VALUES (2000, 100, 10) @@ -233,7 +230,7 @@ insert new_order ├── cardinality: [0 - 0] ├── side-effects, mutations ├── stats: [rows=0] - ├── cost: 6.14 + ├── cost: 6.16 ├── values │ ├── columns: column1:4(int!null) column2:5(int!null) column3:6(int!null) │ ├── cardinality: [1 - 1] @@ -248,12 +245,12 @@ insert new_order │ └── const: 10 [type=int] └── 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@order_idx) + └── anti-join (lookup order) ├── columns: column3:15(int!null) column2:16(int!null) column1:17(int!null) - ├── key columns: [15 16] = [9 8] + ├── key columns: [15 16 17] = [9 8 7] ├── cardinality: [0 - 1] - ├── stats: [rows=1] - ├── cost: 6.11 + ├── stats: [rows=1e-10] + ├── cost: 6.13 ├── key: () ├── fd: ()-->(4-6), (6)-->(15), (5)-->(16), (4)-->(17) ├── with-scan &1 @@ -267,10 +264,7 @@ insert new_order │ ├── cost: 0.01 │ ├── key: () │ └── fd: ()-->(4-6), (6)-->(15), (5)-->(16), (4)-->(17) - └── filters - └── eq [type=bool, outer=(7,17), constraints=(/7: (/NULL - ]; /17: (/NULL - ]), fd=(7)==(17), (17)==(7)] - ├── variable: column1 [type=int] - └── variable: o_id [type=int] + └── filters (true) opt format=hide-qual UPDATE @@ -703,7 +697,7 @@ insert order_line ├── cardinality: [0 - 0] ├── side-effects, mutations ├── stats: [rows=0] - ├── cost: 73.04 + ├── cost: 73.0561434 ├── project │ ├── columns: ol_amount:21(decimal) column20:20(timestamp) column1:11(int!null) column2:12(int!null) column3:13(int!null) column4:14(int!null) column5:15(int!null) column6:16(int!null) column7:17(int!null) column9:19(string!null) │ ├── cardinality: [6 - 6] @@ -784,12 +778,12 @@ insert order_line │ └── null [type=timestamp] └── 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@order_idx) + │ └── anti-join (lookup order) │ ├── columns: column3:30(int!null) column2:31(int!null) column1:32(int!null) - │ ├── key columns: [30 31] = [24 23] + │ ├── key columns: [30 31 32] = [24 23 22] │ ├── cardinality: [0 - 6] - │ ├── stats: [rows=6] - │ ├── cost: 36.51 + │ ├── stats: [rows=1e-10] + │ ├── cost: 36.68 │ ├── fd: ()-->(20), (13)-->(30), (12)-->(31), (11)-->(32) │ ├── with-scan &1 │ │ ├── columns: column3:30(int!null) column2:31(int!null) column1:32(int!null) @@ -801,17 +795,14 @@ insert order_line │ │ ├── stats: [rows=6] │ │ ├── cost: 0.01 │ │ └── fd: ()-->(20), (13)-->(30), (12)-->(31), (11)-->(32) - │ └── filters - │ └── eq [type=bool, outer=(22,32), constraints=(/22: (/NULL - ]; /32: (/NULL - ]), fd=(22)==(32), (32)==(22)] - │ ├── variable: column1 [type=int] - │ └── variable: o_id [type=int] + │ └── 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:50(int!null) column5:51(int!null) ├── key columns: [51 50] = [33 34] ├── cardinality: [0 - 6] - ├── stats: [rows=6] - ├── cost: 36.26 + ├── stats: [rows=1e-10] + ├── cost: 36.1061434 ├── fd: ()-->(20), (16)-->(50), (15)-->(51) ├── with-scan &1 │ ├── columns: column6:50(int!null) column5:51(int!null) @@ -1110,7 +1101,7 @@ insert history ├── cardinality: [0 - 0] ├── side-effects, mutations ├── stats: [rows=0] - ├── cost: 12.3 + ├── cost: 12.44 ├── values │ ├── columns: column1:10(int!null) column2:11(int!null) column3:12(int!null) column4:13(int!null) column5:14(int!null) column7:16(timestamp!null) column8:17(string!null) column18:18(uuid) h_amount:19(decimal) │ ├── cardinality: [1 - 1] @@ -1134,12 +1125,12 @@ insert history │ └── const: 2 [type=int] └── 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@customer_idx) + │ └── anti-join (lookup customer) │ ├── columns: column3:41(int!null) column2:42(int!null) column1:43(int!null) - │ ├── key columns: [41 42] = [22 21] + │ ├── key columns: [41 42 43] = [22 21 20] │ ├── cardinality: [0 - 1] - │ ├── stats: [rows=1] - │ ├── cost: 6.12 + │ ├── stats: [rows=1e-10] + │ ├── cost: 6.26 │ ├── key: () │ ├── fd: ()-->(10-14,16-19), (12)-->(41), (11)-->(42), (10)-->(43) │ ├── with-scan &1 @@ -1153,16 +1144,13 @@ insert history │ │ ├── cost: 0.01 │ │ ├── key: () │ │ └── fd: ()-->(10-14,16-19), (12)-->(41), (11)-->(42), (10)-->(43) - │ └── filters - │ └── eq [type=bool, outer=(20,43), constraints=(/20: (/NULL - ]; /43: (/NULL - ]), fd=(20)==(43), (43)==(20)] - │ ├── variable: column1 [type=int] - │ └── variable: c_id [type=int] + │ └── 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:55(int!null) column4:56(int!null) ├── key columns: [55 56] = [45 44] ├── cardinality: [0 - 1] - ├── stats: [rows=1] + ├── stats: [rows=1e-10] ├── cost: 6.15 ├── key: () ├── fd: ()-->(10-14,16-19), (14)-->(55), (13)-->(56) @@ -1649,7 +1637,7 @@ scalar-group-by ├── columns: count:28(int) ├── cardinality: [1 - 1] ├── stats: [rows=1] - ├── cost: 1551.05048 + ├── cost: 1465.1653 ├── key: () ├── fd: ()-->(28) ├── prune: (28) @@ -1657,7 +1645,7 @@ scalar-group-by │ ├── 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] │ ├── stats: [rows=234.432912, distinct(1)=19.9998377, null(1)=0, distinct(2)=1, null(2)=0, distinct(3)=1, null(3)=0, distinct(5)=199.843131, null(5)=0, distinct(11)=199.843131, null(11)=0, distinct(12)=1, null(12)=0, distinct(13)=30.3199861, null(13)=0] - │ ├── cost: 1548.68615 + │ ├── cost: 1462.80097 │ ├── fd: ()-->(2,3,12), (11)-->(13), (5)==(11), (11)==(5), (3)==(12), (12)==(3) │ ├── interesting orderings: (+3,+2,-1) │ ├── scan order_line @@ -1702,7 +1690,7 @@ scalar-group-by ├── columns: count:22(int) ├── cardinality: [1 - 1] ├── stats: [rows=1] - ├── cost: 1264.74667 + ├── cost: 1265.42333 ├── key: () ├── fd: ()-->(22) ├── prune: (22) @@ -1711,7 +1699,7 @@ scalar-group-by │ ├── left ordering: +1 │ ├── right ordering: +11 │ ├── stats: [rows=33.3333333, distinct(1)=33.3333333, null(1)=0, distinct(9)=1, null(9)=0, distinct(11)=33.3333333, null(11)=0, distinct(21)=33.3333333, null(21)=0] - │ ├── cost: 1264.39333 + │ ├── cost: 1265.07 │ ├── key: (11) │ ├── fd: (1)-->(9), (11)-->(21), (1)==(11), (11)==(1) │ ├── scan warehouse diff --git a/pkg/sql/opt/xform/testdata/external/tpcc-later-stats b/pkg/sql/opt/xform/testdata/external/tpcc-later-stats index c0b5bdde86ac..7390a6a754b5 100644 --- a/pkg/sql/opt/xform/testdata/external/tpcc-later-stats +++ b/pkg/sql/opt/xform/testdata/external/tpcc-later-stats @@ -179,7 +179,7 @@ insert "order" ├── cardinality: [0 - 0] ├── side-effects, mutations ├── stats: [rows=0] - ├── cost: 6.15 + ├── cost: 6.29 ├── values │ ├── columns: column1:9(int!null) column2:10(int!null) column3:11(int!null) column4:12(int!null) column5:13(timestamp!null) column6:14(int!null) column7:15(int!null) column16:16(int) │ ├── cardinality: [1 - 1] @@ -199,12 +199,12 @@ insert "order" │ └── null [type=int] └── 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@customer_idx) + └── anti-join (lookup customer) ├── columns: column3:38(int!null) column2:39(int!null) column4:40(int!null) - ├── key columns: [38 39] = [19 18] + ├── key columns: [38 39 40] = [19 18 17] ├── cardinality: [0 - 1] - ├── stats: [rows=1] - ├── cost: 6.12 + ├── stats: [rows=1e-10] + ├── cost: 6.26 ├── key: () ├── fd: ()-->(9-16), (11)-->(38), (10)-->(39), (12)-->(40) ├── with-scan &1 @@ -218,10 +218,7 @@ insert "order" │ ├── cost: 0.01 │ ├── key: () │ └── fd: ()-->(9-16), (11)-->(38), (10)-->(39), (12)-->(40) - └── filters - └── eq [type=bool, outer=(17,40), constraints=(/17: (/NULL - ]; /40: (/NULL - ]), fd=(17)==(40), (40)==(17)] - ├── variable: column4 [type=int] - └── variable: c_id [type=int] + └── filters (true) opt format=hide-qual INSERT INTO new_order (no_o_id, no_d_id, no_w_id) VALUES (2000, 100, 10) @@ -236,7 +233,7 @@ insert new_order ├── cardinality: [0 - 0] ├── side-effects, mutations ├── stats: [rows=0] - ├── cost: 6.14 + ├── cost: 6.14975196 ├── values │ ├── columns: column1:4(int!null) column2:5(int!null) column3:6(int!null) │ ├── cardinality: [1 - 1] @@ -251,12 +248,12 @@ insert new_order │ └── const: 10 [type=int] └── 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@order_idx) + └── anti-join (lookup order) ├── columns: column3:15(int!null) column2:16(int!null) column1:17(int!null) - ├── key columns: [15 16] = [9 8] + ├── key columns: [15 16 17] = [9 8 7] ├── cardinality: [0 - 1] - ├── stats: [rows=1] - ├── cost: 6.11 + ├── stats: [rows=1e-10] + ├── cost: 6.11975196 ├── key: () ├── fd: ()-->(4-6), (6)-->(15), (5)-->(16), (4)-->(17) ├── with-scan &1 @@ -270,10 +267,7 @@ insert new_order │ ├── cost: 0.01 │ ├── key: () │ └── fd: ()-->(4-6), (6)-->(15), (5)-->(16), (4)-->(17) - └── filters - └── eq [type=bool, outer=(7,17), constraints=(/7: (/NULL - ]; /17: (/NULL - ]), fd=(7)==(17), (17)==(7)] - ├── variable: column1 [type=int] - └── variable: o_id [type=int] + └── filters (true) opt format=hide-qual UPDATE @@ -706,7 +700,7 @@ insert order_line ├── cardinality: [0 - 0] ├── side-effects, mutations ├── stats: [rows=0] - ├── cost: 73.04 + ├── cost: 72.9946552 ├── project │ ├── columns: ol_amount:21(decimal) column20:20(timestamp) column1:11(int!null) column2:12(int!null) column3:13(int!null) column4:14(int!null) column5:15(int!null) column6:16(int!null) column7:17(int!null) column9:19(string!null) │ ├── cardinality: [6 - 6] @@ -787,12 +781,12 @@ insert order_line │ └── null [type=timestamp] └── 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@order_idx) + │ └── anti-join (lookup order) │ ├── columns: column3:30(int!null) column2:31(int!null) column1:32(int!null) - │ ├── key columns: [30 31] = [24 23] + │ ├── key columns: [30 31 32] = [24 23 22] │ ├── cardinality: [0 - 6] - │ ├── stats: [rows=6] - │ ├── cost: 36.51 + │ ├── stats: [rows=1e-10] + │ ├── cost: 36.6185118 │ ├── fd: ()-->(20), (13)-->(30), (12)-->(31), (11)-->(32) │ ├── with-scan &1 │ │ ├── columns: column3:30(int!null) column2:31(int!null) column1:32(int!null) @@ -804,17 +798,14 @@ insert order_line │ │ ├── stats: [rows=6] │ │ ├── cost: 0.01 │ │ └── fd: ()-->(20), (13)-->(30), (12)-->(31), (11)-->(32) - │ └── filters - │ └── eq [type=bool, outer=(22,32), constraints=(/22: (/NULL - ]; /32: (/NULL - ]), fd=(22)==(32), (32)==(22)] - │ ├── variable: column1 [type=int] - │ └── variable: o_id [type=int] + │ └── 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:50(int!null) column5:51(int!null) ├── key columns: [51 50] = [33 34] ├── cardinality: [0 - 6] - ├── stats: [rows=6] - ├── cost: 36.26 + ├── stats: [rows=1e-10] + ├── cost: 36.1061434 ├── fd: ()-->(20), (16)-->(50), (15)-->(51) ├── with-scan &1 │ ├── columns: column6:50(int!null) column5:51(int!null) @@ -1113,7 +1104,7 @@ insert history ├── cardinality: [0 - 0] ├── side-effects, mutations ├── stats: [rows=0] - ├── cost: 12.3 + ├── cost: 12.44 ├── values │ ├── columns: column1:10(int!null) column2:11(int!null) column3:12(int!null) column4:13(int!null) column5:14(int!null) column7:16(timestamp!null) column8:17(string!null) column18:18(uuid) h_amount:19(decimal) │ ├── cardinality: [1 - 1] @@ -1137,12 +1128,12 @@ insert history │ └── const: 2 [type=int] └── 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@customer_idx) + │ └── anti-join (lookup customer) │ ├── columns: column3:41(int!null) column2:42(int!null) column1:43(int!null) - │ ├── key columns: [41 42] = [22 21] + │ ├── key columns: [41 42 43] = [22 21 20] │ ├── cardinality: [0 - 1] - │ ├── stats: [rows=1] - │ ├── cost: 6.12 + │ ├── stats: [rows=1e-10] + │ ├── cost: 6.26 │ ├── key: () │ ├── fd: ()-->(10-14,16-19), (12)-->(41), (11)-->(42), (10)-->(43) │ ├── with-scan &1 @@ -1156,16 +1147,13 @@ insert history │ │ ├── cost: 0.01 │ │ ├── key: () │ │ └── fd: ()-->(10-14,16-19), (12)-->(41), (11)-->(42), (10)-->(43) - │ └── filters - │ └── eq [type=bool, outer=(20,43), constraints=(/20: (/NULL - ]; /43: (/NULL - ]), fd=(20)==(43), (43)==(20)] - │ ├── variable: column1 [type=int] - │ └── variable: c_id [type=int] + │ └── 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:55(int!null) column4:56(int!null) ├── key columns: [55 56] = [45 44] ├── cardinality: [0 - 1] - ├── stats: [rows=1] + ├── stats: [rows=1e-10] ├── cost: 6.15 ├── key: () ├── fd: ()-->(10-14,16-19), (14)-->(55), (13)-->(56) @@ -1651,7 +1639,7 @@ scalar-group-by ├── columns: count:28(int) ├── cardinality: [1 - 1] ├── stats: [rows=1] - ├── cost: 1534.09466 + ├── cost: 1455.37864 ├── key: () ├── fd: ()-->(28) ├── prune: (28) @@ -1659,7 +1647,7 @@ scalar-group-by │ ├── 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] │ ├── stats: [rows=229.899982, distinct(1)=19.9997964, null(1)=0, distinct(2)=1, null(2)=0, distinct(3)=1, null(3)=0, distinct(5)=198.51294, null(5)=0, distinct(11)=198.51294, null(11)=0, distinct(12)=1, null(12)=0, distinct(13)=30.3178347, null(13)=0] - │ ├── cost: 1531.77566 + │ ├── cost: 1453.05964 │ ├── fd: ()-->(2,3,12), (11)-->(13), (5)==(11), (11)==(5), (3)==(12), (12)==(3) │ ├── interesting orderings: (+3,+2,-1) │ ├── scan order_line @@ -1704,7 +1692,7 @@ scalar-group-by ├── columns: count:22(int) ├── cardinality: [1 - 1] ├── stats: [rows=1] - ├── cost: 126.546667 + ├── cost: 126.623333 ├── key: () ├── fd: ()-->(22) ├── prune: (22) @@ -1713,7 +1701,7 @@ scalar-group-by │ ├── left ordering: +1 │ ├── right ordering: +11 │ ├── stats: [rows=3.33333333, distinct(1)=3.33333333, null(1)=0, distinct(9)=3.33333333, null(9)=0, distinct(11)=3.33333333, null(11)=0, distinct(21)=3.33333333, null(21)=0] - │ ├── cost: 126.493333 + │ ├── cost: 126.57 │ ├── key: (11) │ ├── fd: (1)-->(9), (11)-->(21), (1)==(11), (11)==(1) │ ├── scan warehouse diff --git a/pkg/sql/opt/xform/testdata/external/tpcc-no-stats b/pkg/sql/opt/xform/testdata/external/tpcc-no-stats index 0dcc6ddc10f3..5ff7e315db7a 100644 --- a/pkg/sql/opt/xform/testdata/external/tpcc-no-stats +++ b/pkg/sql/opt/xform/testdata/external/tpcc-no-stats @@ -173,7 +173,7 @@ insert "order" ├── cardinality: [0 - 0] ├── side-effects, mutations ├── stats: [rows=0] - ├── cost: 6.15 + ├── cost: 4.05224 ├── values │ ├── columns: column1:9(int!null) column2:10(int!null) column3:11(int!null) column4:12(int!null) column5:13(timestamp!null) column6:14(int!null) column7:15(int!null) column16:16(int) │ ├── cardinality: [1 - 1] @@ -193,12 +193,12 @@ insert "order" │ └── null [type=int] └── 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@customer_idx) + └── anti-join (lookup customer) ├── columns: column3:38(int!null) column2:39(int!null) column4:40(int!null) - ├── key columns: [38 39] = [19 18] + ├── key columns: [38 39 40] = [19 18 17] ├── cardinality: [0 - 1] - ├── stats: [rows=1] - ├── cost: 6.12 + ├── stats: [rows=1e-10] + ├── cost: 4.02224 ├── key: () ├── fd: ()-->(9-16), (11)-->(38), (10)-->(39), (12)-->(40) ├── with-scan &1 @@ -212,10 +212,7 @@ insert "order" │ ├── cost: 0.01 │ ├── key: () │ └── fd: ()-->(9-16), (11)-->(38), (10)-->(39), (12)-->(40) - └── filters - └── eq [type=bool, outer=(17,40), constraints=(/17: (/NULL - ]; /40: (/NULL - ]), fd=(17)==(40), (40)==(17)] - ├── variable: column4 [type=int] - └── variable: c_id [type=int] + └── filters (true) opt format=hide-qual INSERT INTO new_order (no_o_id, no_d_id, no_w_id) VALUES (2000, 100, 10) @@ -230,7 +227,7 @@ insert new_order ├── cardinality: [0 - 0] ├── side-effects, mutations ├── stats: [rows=0] - ├── cost: 6.14 + ├── cost: 4.05211 ├── values │ ├── columns: column1:4(int!null) column2:5(int!null) column3:6(int!null) │ ├── cardinality: [1 - 1] @@ -245,12 +242,12 @@ insert new_order │ └── const: 10 [type=int] └── 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@order_idx) + └── anti-join (lookup order) ├── columns: column3:15(int!null) column2:16(int!null) column1:17(int!null) - ├── key columns: [15 16] = [9 8] + ├── key columns: [15 16 17] = [9 8 7] ├── cardinality: [0 - 1] - ├── stats: [rows=1] - ├── cost: 6.11 + ├── stats: [rows=1e-10] + ├── cost: 4.02211 ├── key: () ├── fd: ()-->(4-6), (6)-->(15), (5)-->(16), (4)-->(17) ├── with-scan &1 @@ -264,10 +261,7 @@ insert new_order │ ├── cost: 0.01 │ ├── key: () │ └── fd: ()-->(4-6), (6)-->(15), (5)-->(16), (4)-->(17) - └── filters - └── eq [type=bool, outer=(7,17), constraints=(/7: (/NULL - ]; /17: (/NULL - ]), fd=(7)==(17), (17)==(7)] - ├── variable: column1 [type=int] - └── variable: o_id [type=int] + └── filters (true) opt format=hide-qual UPDATE @@ -700,7 +694,7 @@ insert order_line ├── cardinality: [0 - 0] ├── side-effects, mutations ├── stats: [rows=0] - ├── cost: 73.04 + ├── cost: 49.54666 ├── project │ ├── columns: ol_amount:21(decimal) column20:20(timestamp) column1:11(int!null) column2:12(int!null) column3:13(int!null) column4:14(int!null) column5:15(int!null) column6:16(int!null) column7:17(int!null) column9:19(string!null) │ ├── cardinality: [6 - 6] @@ -781,12 +775,12 @@ insert order_line │ └── null [type=timestamp] └── 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@order_idx) + │ └── anti-join (lookup order) │ ├── columns: column3:30(int!null) column2:31(int!null) column1:32(int!null) - │ ├── key columns: [30 31] = [24 23] + │ ├── key columns: [30 31 32] = [24 23 22] │ ├── cardinality: [0 - 6] - │ ├── stats: [rows=6] - │ ├── cost: 36.51 + │ ├── stats: [rows=1e-10] + │ ├── cost: 24.03266 │ ├── fd: ()-->(20), (13)-->(30), (12)-->(31), (11)-->(32) │ ├── with-scan &1 │ │ ├── columns: column3:30(int!null) column2:31(int!null) column1:32(int!null) @@ -798,17 +792,14 @@ insert order_line │ │ ├── stats: [rows=6] │ │ ├── cost: 0.01 │ │ └── fd: ()-->(20), (13)-->(30), (12)-->(31), (11)-->(32) - │ └── filters - │ └── eq [type=bool, outer=(22,32), constraints=(/22: (/NULL - ]; /32: (/NULL - ]), fd=(22)==(32), (32)==(22)] - │ ├── variable: column1 [type=int] - │ └── variable: o_id [type=int] + │ └── 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:50(int!null) column5:51(int!null) ├── key columns: [51 50] = [33 34] ├── cardinality: [0 - 6] - ├── stats: [rows=6] - ├── cost: 36.26 + ├── stats: [rows=1e-10] + ├── cost: 25.244 ├── fd: ()-->(20), (16)-->(50), (15)-->(51) ├── with-scan &1 │ ├── columns: column6:50(int!null) column5:51(int!null) @@ -1107,7 +1098,7 @@ insert history ├── cardinality: [0 - 0] ├── side-effects, mutations ├── stats: [rows=0] - ├── cost: 12.3 + ├── cost: 8.28524 ├── values │ ├── columns: column1:10(int!null) column2:11(int!null) column3:12(int!null) column4:13(int!null) column5:14(int!null) column7:16(timestamp!null) column8:17(string!null) column18:18(uuid) h_amount:19(decimal) │ ├── cardinality: [1 - 1] @@ -1131,12 +1122,12 @@ insert history │ └── const: 2 [type=int] └── 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@customer_idx) + │ └── anti-join (lookup customer) │ ├── columns: column3:41(int!null) column2:42(int!null) column1:43(int!null) - │ ├── key columns: [41 42] = [22 21] + │ ├── key columns: [41 42 43] = [22 21 20] │ ├── cardinality: [0 - 1] - │ ├── stats: [rows=1] - │ ├── cost: 6.12 + │ ├── stats: [rows=1e-10] + │ ├── cost: 4.02224 │ ├── key: () │ ├── fd: ()-->(10-14,16-19), (12)-->(41), (11)-->(42), (10)-->(43) │ ├── with-scan &1 @@ -1150,17 +1141,14 @@ insert history │ │ ├── cost: 0.01 │ │ ├── key: () │ │ └── fd: ()-->(10-14,16-19), (12)-->(41), (11)-->(42), (10)-->(43) - │ └── filters - │ └── eq [type=bool, outer=(20,43), constraints=(/20: (/NULL - ]; /43: (/NULL - ]), fd=(20)==(43), (43)==(20)] - │ ├── variable: column1 [type=int] - │ └── variable: c_id [type=int] + │ └── 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:55(int!null) column4:56(int!null) ├── key columns: [55 56] = [45 44] ├── cardinality: [0 - 1] - ├── stats: [rows=1] - ├── cost: 6.15 + ├── stats: [rows=1e-10] + ├── cost: 4.233 ├── key: () ├── fd: ()-->(10-14,16-19), (14)-->(55), (13)-->(56) ├── with-scan &1 @@ -1645,7 +1633,7 @@ scalar-group-by ├── columns: count:28(int) ├── cardinality: [1 - 1] ├── stats: [rows=1] - ├── cost: 0.30998 + ├── cost: 0.27346 ├── key: () ├── fd: ()-->(28) ├── prune: (28) @@ -1653,7 +1641,7 @@ scalar-group-by │ ├── 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] │ ├── stats: [rows=0.066, distinct(1)=0.02, null(1)=0, distinct(2)=0.02, null(2)=0, distinct(3)=0.02, null(3)=0, distinct(5)=0.0199982001, null(5)=0, distinct(11)=0.0199982001, null(11)=0, distinct(12)=0.02, null(12)=0, distinct(13)=0.066, null(13)=0] - │ ├── cost: 0.28932 + │ ├── cost: 0.2528 │ ├── fd: ()-->(2,3,12), (11)-->(13), (5)==(11), (11)==(5), (3)==(12), (12)==(3) │ ├── interesting orderings: (+3,+2,-1) │ ├── scan order_line @@ -1698,7 +1686,7 @@ scalar-group-by ├── columns: count:22(int) ├── cardinality: [1 - 1] ├── stats: [rows=1] - ├── cost: 1621.36 + ├── cost: 1762.4 ├── key: () ├── fd: ()-->(22) ├── prune: (22) @@ -1706,7 +1694,7 @@ scalar-group-by │ ├── columns: w_id:1(int!null) w_ytd:9(decimal!null) d_w_id:11(int!null) sum:21(decimal!null) │ ├── key columns: [11] = [1] │ ├── stats: [rows=33, distinct(1)=33, null(1)=0, distinct(9)=33, null(9)=0, distinct(11)=33, null(11)=0, distinct(21)=33, null(21)=0] - │ ├── cost: 1621.01 + │ ├── cost: 1762.05 │ ├── key: (11) │ ├── fd: (1)-->(9), (11)-->(21), (1)==(11), (11)==(1) │ ├── interesting orderings: (+11) diff --git a/pkg/sql/opt/xform/testdata/external/tpch b/pkg/sql/opt/xform/testdata/external/tpch index a4e70eed8457..5b924baef171 100644 --- a/pkg/sql/opt/xform/testdata/external/tpch +++ b/pkg/sql/opt/xform/testdata/external/tpch @@ -1901,12 +1901,10 @@ limit │ ├── inner-join (hash) │ │ ├── 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) │ │ ├── fd: (1)-->(2), (9)-->(10,12,13), (9)==(18), (18)==(9), (1)==(10), (10)==(1) - │ │ ├── scan lineitem - │ │ │ └── columns: l_orderkey:18(int!null) l_quantity:22(float!null) - │ │ ├── inner-join (hash) - │ │ │ ├── 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) - │ │ │ ├── key: (9) - │ │ │ ├── fd: (9)-->(10,12,13), (1)-->(2), (1)==(10), (10)==(1) + │ │ ├── inner-join (lookup lineitem) + │ │ │ ├── columns: 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) + │ │ │ ├── key columns: [9] = [18] + │ │ │ ├── fd: (9)-->(10,12,13), (9)==(18), (18)==(9) │ │ │ ├── semi-join (merge) │ │ │ │ ├── columns: o_orderkey:9(int!null) o_custkey:10(int!null) o_totalprice:12(float!null) o_orderdate:13(date!null) │ │ │ │ ├── left ordering: +9 @@ -1938,14 +1936,13 @@ limit │ │ │ │ │ └── filters │ │ │ │ │ └── sum > 300.0 [type=bool, outer=(50), constraints=(/50: [/300.00000000000006 - ]; tight)] │ │ │ │ └── filters (true) - │ │ │ ├── scan customer - │ │ │ │ ├── columns: c_custkey:1(int!null) c_name:2(varchar!null) - │ │ │ │ ├── key: (1) - │ │ │ │ └── fd: (1)-->(2) - │ │ │ └── filters - │ │ │ └── c_custkey = o_custkey [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] + │ │ │ └── filters (true) + │ │ ├── scan customer + │ │ │ ├── columns: c_custkey:1(int!null) c_name:2(varchar!null) + │ │ │ ├── key: (1) + │ │ │ └── fd: (1)-->(2) │ │ └── filters - │ │ └── o_orderkey = l_orderkey [type=bool, outer=(9,18), constraints=(/9: (/NULL - ]; /18: (/NULL - ]), fd=(9)==(18), (18)==(9)] + │ │ └── c_custkey = o_custkey [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] │ └── aggregations │ ├── sum [type=float, outer=(22)] │ │ └── variable: l_quantity [type=float] @@ -2370,54 +2367,55 @@ GROUP BY ORDER BY cntrycode; ---- -group-by +sort ├── columns: cntrycode:27(string) numcust:28(int) totacctbal:29(float) - ├── grouping columns: cntrycode:27(string) ├── key: (27) ├── fd: (27)-->(28,29) ├── ordering: +27 - ├── sort - │ ├── columns: c_acctbal:6(float!null) cntrycode:27(string) - │ ├── ordering: +27 - │ └── project - │ ├── columns: cntrycode:27(string) c_acctbal:6(float!null) - │ ├── anti-join (lookup orders@o_ck) - │ │ ├── columns: c_custkey:1(int!null) c_phone:5(char!null) c_acctbal:6(float!null) - │ │ ├── key columns: [1] = [19] - │ │ ├── key: (1) - │ │ ├── fd: (1)-->(5,6) - │ │ ├── select - │ │ │ ├── columns: c_custkey:1(int!null) c_phone:5(char!null) c_acctbal:6(float!null) - │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(5,6) - │ │ │ ├── scan customer - │ │ │ │ ├── columns: c_custkey:1(int!null) c_phone:5(char!null) c_acctbal:6(float!null) - │ │ │ │ ├── key: (1) - │ │ │ │ └── fd: (1)-->(5,6) - │ │ │ └── filters - │ │ │ ├── substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31') [type=bool, outer=(5)] - │ │ │ └── gt [type=bool, outer=(6), subquery, constraints=(/6: (/NULL - ])] - │ │ │ ├── variable: c_acctbal [type=float] - │ │ │ └── subquery [type=float] - │ │ │ └── scalar-group-by - │ │ │ ├── columns: avg:17(float) - │ │ │ ├── cardinality: [1 - 1] - │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(17) - │ │ │ ├── select - │ │ │ │ ├── columns: c_phone:13(char!null) c_acctbal:14(float!null) - │ │ │ │ ├── scan customer - │ │ │ │ │ └── columns: c_phone:13(char!null) c_acctbal:14(float!null) - │ │ │ │ └── filters - │ │ │ │ ├── c_acctbal > 0.0 [type=bool, outer=(14), constraints=(/14: [/5e-324 - ]; tight)] - │ │ │ │ └── substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31') [type=bool, outer=(13)] - │ │ │ └── aggregations - │ │ │ └── avg [type=float, outer=(14)] - │ │ │ └── variable: c_acctbal [type=float] - │ │ └── filters (true) - │ └── projections - │ └── substring(c_phone, 1, 2) [type=string, outer=(5)] - └── aggregations - ├── count-rows [type=int] - └── sum [type=float, outer=(6)] - └── variable: c_acctbal [type=float] + └── group-by + ├── columns: cntrycode:27(string) count_rows:28(int) sum:29(float) + ├── grouping columns: cntrycode:27(string) + ├── key: (27) + ├── fd: (27)-->(28,29) + ├── project + │ ├── columns: cntrycode:27(string) c_acctbal:6(float!null) + │ ├── anti-join (lookup orders@o_ck) + │ │ ├── columns: c_custkey:1(int!null) c_phone:5(char!null) c_acctbal:6(float!null) + │ │ ├── key columns: [1] = [19] + │ │ ├── key: (1) + │ │ ├── fd: (1)-->(5,6) + │ │ ├── select + │ │ │ ├── columns: c_custkey:1(int!null) c_phone:5(char!null) c_acctbal:6(float!null) + │ │ │ ├── key: (1) + │ │ │ ├── fd: (1)-->(5,6) + │ │ │ ├── scan customer + │ │ │ │ ├── columns: c_custkey:1(int!null) c_phone:5(char!null) c_acctbal:6(float!null) + │ │ │ │ ├── key: (1) + │ │ │ │ └── fd: (1)-->(5,6) + │ │ │ └── filters + │ │ │ ├── substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31') [type=bool, outer=(5)] + │ │ │ └── gt [type=bool, outer=(6), subquery, constraints=(/6: (/NULL - ])] + │ │ │ ├── variable: c_acctbal [type=float] + │ │ │ └── subquery [type=float] + │ │ │ └── scalar-group-by + │ │ │ ├── columns: avg:17(float) + │ │ │ ├── cardinality: [1 - 1] + │ │ │ ├── key: () + │ │ │ ├── fd: ()-->(17) + │ │ │ ├── select + │ │ │ │ ├── columns: c_phone:13(char!null) c_acctbal:14(float!null) + │ │ │ │ ├── scan customer + │ │ │ │ │ └── columns: c_phone:13(char!null) c_acctbal:14(float!null) + │ │ │ │ └── filters + │ │ │ │ ├── c_acctbal > 0.0 [type=bool, outer=(14), constraints=(/14: [/5e-324 - ]; tight)] + │ │ │ │ └── substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31') [type=bool, outer=(13)] + │ │ │ └── aggregations + │ │ │ └── avg [type=float, outer=(14)] + │ │ │ └── variable: c_acctbal [type=float] + │ │ └── filters (true) + │ └── projections + │ └── substring(c_phone, 1, 2) [type=string, outer=(5)] + └── aggregations + ├── count-rows [type=int] + └── sum [type=float, outer=(6)] + └── variable: c_acctbal [type=float] diff --git a/pkg/sql/opt/xform/testdata/external/tpch-no-stats b/pkg/sql/opt/xform/testdata/external/tpch-no-stats index bcd13f23ef7a..cd595c2ed47b 100644 --- a/pkg/sql/opt/xform/testdata/external/tpch-no-stats +++ b/pkg/sql/opt/xform/testdata/external/tpch-no-stats @@ -422,44 +422,42 @@ GROUP BY ORDER BY o_orderpriority; ---- -group-by +sort ├── columns: o_orderpriority:6(char!null) order_count:26(int) - ├── grouping columns: o_orderpriority:6(char!null) ├── key: (6) ├── fd: (6)-->(26) ├── ordering: +6 - ├── project - │ ├── columns: o_orderkey:1(int!null) o_orderdate:5(date!null) o_orderpriority:6(char!null) - │ ├── key: (1) - │ ├── fd: (1)-->(5,6) - │ ├── ordering: +6 - │ └── sort - │ ├── columns: o_orderkey:1(int!null) o_orderdate:5(date!null) o_orderpriority:6(char!null) l_orderkey:10(int!null) - │ ├── key: (10) - │ ├── fd: (1)-->(5,6), (1)==(10), (10)==(1) - │ ├── ordering: +6 - │ └── inner-join (lookup orders) - │ ├── columns: o_orderkey:1(int!null) o_orderdate:5(date!null) o_orderpriority:6(char!null) l_orderkey:10(int!null) - │ ├── key columns: [10] = [1] - │ ├── key: (10) - │ ├── fd: (1)-->(5,6), (1)==(10), (10)==(1) - │ ├── distinct-on - │ │ ├── columns: l_orderkey:10(int!null) - │ │ ├── grouping columns: l_orderkey:10(int!null) - │ │ ├── internal-ordering: +10 - │ │ ├── key: (10) - │ │ └── select - │ │ ├── columns: l_orderkey:10(int!null) l_commitdate:21(date!null) l_receiptdate:22(date!null) - │ │ ├── ordering: +10 - │ │ ├── scan lineitem - │ │ │ ├── columns: l_orderkey:10(int!null) l_commitdate:21(date!null) l_receiptdate:22(date!null) - │ │ │ └── ordering: +10 - │ │ └── filters - │ │ └── l_commitdate < l_receiptdate [type=bool, outer=(21,22), constraints=(/21: (/NULL - ]; /22: (/NULL - ])] - │ └── filters - │ └── (o_orderdate >= '1993-07-01') AND (o_orderdate < '1993-10-01') [type=bool, outer=(5), constraints=(/5: [/'1993-07-01' - /'1993-09-30']; tight)] - └── aggregations - └── count-rows [type=int] + └── group-by + ├── columns: o_orderpriority:6(char!null) count_rows:26(int) + ├── grouping columns: o_orderpriority:6(char!null) + ├── key: (6) + ├── fd: (6)-->(26) + ├── project + │ ├── columns: o_orderkey:1(int!null) o_orderdate:5(date!null) o_orderpriority:6(char!null) + │ ├── key: (1) + │ ├── fd: (1)-->(5,6) + │ └── inner-join (lookup orders) + │ ├── columns: o_orderkey:1(int!null) o_orderdate:5(date!null) o_orderpriority:6(char!null) l_orderkey:10(int!null) + │ ├── key columns: [10] = [1] + │ ├── key: (10) + │ ├── fd: (1)-->(5,6), (1)==(10), (10)==(1) + │ ├── distinct-on + │ │ ├── columns: l_orderkey:10(int!null) + │ │ ├── grouping columns: l_orderkey:10(int!null) + │ │ ├── internal-ordering: +10 + │ │ ├── key: (10) + │ │ └── select + │ │ ├── columns: l_orderkey:10(int!null) l_commitdate:21(date!null) l_receiptdate:22(date!null) + │ │ ├── ordering: +10 + │ │ ├── scan lineitem + │ │ │ ├── columns: l_orderkey:10(int!null) l_commitdate:21(date!null) l_receiptdate:22(date!null) + │ │ │ └── ordering: +10 + │ │ └── filters + │ │ └── l_commitdate < l_receiptdate [type=bool, outer=(21,22), constraints=(/21: (/NULL - ]; /22: (/NULL - ])] + │ └── filters + │ └── (o_orderdate >= '1993-07-01') AND (o_orderdate < '1993-10-01') [type=bool, outer=(5), constraints=(/5: [/'1993-07-01' - /'1993-09-30']; tight)] + └── aggregations + └── count-rows [type=int] # -------------------------------------------------- # Q5 @@ -1665,8 +1663,9 @@ sort ├── grouping columns: p_brand:9(char!null) p_type:10(varchar!null) p_size:11(int!null) ├── key: (9-11) ├── fd: (9-11)-->(22) - ├── inner-join (hash) + ├── inner-join (lookup part) │ ├── 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) + │ ├── key columns: [1] = [6] │ ├── key: (2,6) │ ├── fd: (6)-->(9-11), (1)==(6), (6)==(1) │ ├── anti-join (merge) @@ -1691,20 +1690,10 @@ sort │ │ │ └── filters │ │ │ └── s_comment LIKE '%Customer%Complaints%' [type=bool, outer=(21), constraints=(/21: (/NULL - ])] │ │ └── filters (true) - │ ├── select - │ │ ├── columns: p_partkey:6(int!null) p_brand:9(char!null) p_type:10(varchar!null) p_size:11(int!null) - │ │ ├── key: (6) - │ │ ├── fd: (6)-->(9-11) - │ │ ├── scan part - │ │ │ ├── columns: p_partkey:6(int!null) p_brand:9(char!null) p_type:10(varchar!null) p_size:11(int!null) - │ │ │ ├── key: (6) - │ │ │ └── fd: (6)-->(9-11) - │ │ └── filters - │ │ ├── p_brand != 'Brand#45' [type=bool, outer=(9), constraints=(/9: (/NULL - /'Brand#45') [/e'Brand#45\x00' - ]; tight)] - │ │ ├── p_type NOT LIKE 'MEDIUM POLISHED %' [type=bool, outer=(10), constraints=(/10: (/NULL - ])] - │ │ └── p_size 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)] │ └── filters - │ └── p_partkey = ps_partkey [type=bool, outer=(1,6), constraints=(/1: (/NULL - ]; /6: (/NULL - ]), fd=(1)==(6), (6)==(1)] + │ ├── p_brand != 'Brand#45' [type=bool, outer=(9), constraints=(/9: (/NULL - /'Brand#45') [/e'Brand#45\x00' - ]; tight)] + │ ├── p_type NOT LIKE 'MEDIUM POLISHED %' [type=bool, outer=(10), constraints=(/10: (/NULL - ])] + │ └── p_size 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)] └── aggregations └── count [type=int, outer=(2)] └── agg-distinct [type=int] @@ -1876,20 +1865,20 @@ limit │ └── group-by │ ├── columns: c_custkey:1(int) c_name:2(varchar) o_orderkey:9(int!null) o_totalprice:12(float) o_orderdate:13(date) sum:51(float) │ ├── grouping columns: o_orderkey:9(int!null) + │ ├── internal-ordering: +(9|18) │ ├── key: (9) │ ├── fd: (1)-->(2), (9)-->(1,2,12,13,51) - │ ├── inner-join (hash) + │ ├── inner-join (lookup lineitem) │ │ ├── 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) + │ │ ├── key columns: [9] = [18] │ │ ├── fd: (1)-->(2), (9)-->(10,12,13), (9)==(18), (18)==(9), (1)==(10), (10)==(1) - │ │ ├── scan customer - │ │ │ ├── columns: c_custkey:1(int!null) c_name:2(varchar!null) - │ │ │ ├── key: (1) - │ │ │ └── fd: (1)-->(2) - │ │ ├── inner-join (merge) - │ │ │ ├── columns: 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) - │ │ │ ├── left ordering: +9 - │ │ │ ├── right ordering: +18 - │ │ │ ├── fd: (9)-->(10,12,13), (9)==(18), (18)==(9) + │ │ ├── ordering: +(9|18) [actual: +9] + │ │ ├── inner-join (lookup customer) + │ │ │ ├── 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) + │ │ │ ├── key columns: [10] = [1] + │ │ │ ├── key: (9) + │ │ │ ├── fd: (9)-->(10,12,13), (1)-->(2), (1)==(10), (10)==(1) + │ │ │ ├── ordering: +9 │ │ │ ├── project │ │ │ │ ├── columns: o_orderkey:9(int!null) o_custkey:10(int!null) o_totalprice:12(float!null) o_orderdate:13(date!null) │ │ │ │ ├── key: (9) @@ -1921,12 +1910,8 @@ limit │ │ │ │ │ └── filters │ │ │ │ │ └── sum > 300.0 [type=bool, outer=(50), constraints=(/50: [/300.00000000000006 - ]; tight)] │ │ │ │ └── filters (true) - │ │ │ ├── scan lineitem - │ │ │ │ ├── columns: l_orderkey:18(int!null) l_quantity:22(float!null) - │ │ │ │ └── ordering: +18 │ │ │ └── filters (true) - │ │ └── filters - │ │ └── c_custkey = o_custkey [type=bool, outer=(1,10), constraints=(/1: (/NULL - ]; /10: (/NULL - ]), fd=(1)==(10), (10)==(1)] + │ │ └── filters (true) │ └── aggregations │ ├── sum [type=float, outer=(22)] │ │ └── variable: l_quantity [type=float] @@ -2075,8 +2060,9 @@ sort ├── ordering: +2 └── project ├── columns: s_name:2(char!null) s_address:3(varchar!null) - └── inner-join (hash) + └── inner-join (lookup nation) ├── 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) + ├── key columns: [4] = [8] ├── key: (1) ├── fd: ()-->(9), (1)-->(2-4), (4)==(8), (8)==(4) ├── project @@ -2143,18 +2129,8 @@ sort │ │ └── filters │ │ └── ps_partkey = p_partkey [type=bool, outer=(12,17), constraints=(/12: (/NULL - ]; /17: (/NULL - ]), fd=(12)==(17), (17)==(12)] │ └── filters (true) - ├── select - │ ├── columns: n_nationkey:8(int!null) n_name:9(char!null) - │ ├── key: (8) - │ ├── fd: ()-->(9) - │ ├── scan nation - │ │ ├── columns: n_nationkey:8(int!null) n_name:9(char!null) - │ │ ├── key: (8) - │ │ └── fd: (8)-->(9) - │ └── filters - │ └── n_name = 'CANADA' [type=bool, outer=(9), constraints=(/9: [/'CANADA' - /'CANADA']; tight), fd=()-->(9)] └── filters - └── s_nationkey = n_nationkey [type=bool, outer=(4,8), constraints=(/4: (/NULL - ]; /8: (/NULL - ]), fd=(4)==(8), (8)==(4)] + └── n_name = 'CANADA' [type=bool, outer=(9), constraints=(/9: [/'CANADA' - /'CANADA']; tight), fd=()-->(9)] # -------------------------------------------------- # Q21 @@ -2233,8 +2209,9 @@ limit │ │ │ ├── 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) │ │ │ ├── key columns: [10] = [1] │ │ │ ├── fd: ()-->(26), (8)==(24), (24)==(8), (1)-->(2,4), (1)==(10), (10)==(1) - │ │ │ ├── inner-join (hash) + │ │ │ ├── inner-join (lookup orders) │ │ │ │ ├── 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) o_orderkey:24(int!null) o_orderstatus:26(char!null) + │ │ │ │ ├── key columns: [8] = [24] │ │ │ │ ├── fd: ()-->(26), (8)==(24), (24)==(8) │ │ │ │ ├── semi-join (hash) │ │ │ │ │ ├── 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) @@ -2265,18 +2242,8 @@ limit │ │ │ │ │ └── filters │ │ │ │ │ ├── l2.l_orderkey = l1.l_orderkey [type=bool, outer=(8,37), constraints=(/8: (/NULL - ]; /37: (/NULL - ]), fd=(8)==(37), (37)==(8)] │ │ │ │ │ └── l2.l_suppkey != l1.l_suppkey [type=bool, outer=(10,39), constraints=(/10: (/NULL - ]; /39: (/NULL - ])] - │ │ │ │ ├── select - │ │ │ │ │ ├── columns: o_orderkey:24(int!null) o_orderstatus:26(char!null) - │ │ │ │ │ ├── key: (24) - │ │ │ │ │ ├── fd: ()-->(26) - │ │ │ │ │ ├── scan orders - │ │ │ │ │ │ ├── columns: o_orderkey:24(int!null) o_orderstatus:26(char!null) - │ │ │ │ │ │ ├── key: (24) - │ │ │ │ │ │ └── fd: (24)-->(26) - │ │ │ │ │ └── filters - │ │ │ │ │ └── o_orderstatus = 'F' [type=bool, outer=(26), constraints=(/26: [/'F' - /'F']; tight), fd=()-->(26)] │ │ │ │ └── filters - │ │ │ │ └── o_orderkey = l1.l_orderkey [type=bool, outer=(8,24), constraints=(/8: (/NULL - ]; /24: (/NULL - ]), fd=(8)==(24), (24)==(8)] + │ │ │ │ └── o_orderstatus = 'F' [type=bool, outer=(26), constraints=(/26: [/'F' - /'F']; tight), fd=()-->(26)] │ │ │ └── filters (true) │ │ └── filters │ │ └── n_name = 'SAUDI ARABIA' [type=bool, outer=(34), constraints=(/34: [/'SAUDI ARABIA' - /'SAUDI ARABIA']; tight), fd=()-->(34)] @@ -2333,55 +2300,60 @@ GROUP BY ORDER BY cntrycode; ---- -sort +group-by ├── columns: cntrycode:27(string) numcust:28(int) totacctbal:29(float) + ├── grouping columns: cntrycode:27(string) ├── key: (27) ├── fd: (27)-->(28,29) ├── ordering: +27 - └── group-by - ├── columns: cntrycode:27(string) count_rows:28(int) sum:29(float) - ├── grouping columns: cntrycode:27(string) - ├── key: (27) - ├── fd: (27)-->(28,29) - ├── project - │ ├── columns: cntrycode:27(string) c_acctbal:6(float!null) - │ ├── anti-join (lookup orders@o_ck) - │ │ ├── columns: c_custkey:1(int!null) c_phone:5(char!null) c_acctbal:6(float!null) - │ │ ├── key columns: [1] = [19] - │ │ ├── key: (1) - │ │ ├── fd: (1)-->(5,6) - │ │ ├── select - │ │ │ ├── columns: c_custkey:1(int!null) c_phone:5(char!null) c_acctbal:6(float!null) - │ │ │ ├── key: (1) - │ │ │ ├── fd: (1)-->(5,6) - │ │ │ ├── scan customer - │ │ │ │ ├── columns: c_custkey:1(int!null) c_phone:5(char!null) c_acctbal:6(float!null) - │ │ │ │ ├── key: (1) - │ │ │ │ └── fd: (1)-->(5,6) - │ │ │ └── filters - │ │ │ ├── substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31') [type=bool, outer=(5)] - │ │ │ └── gt [type=bool, outer=(6), subquery, constraints=(/6: (/NULL - ])] - │ │ │ ├── variable: c_acctbal [type=float] - │ │ │ └── subquery [type=float] - │ │ │ └── scalar-group-by - │ │ │ ├── columns: avg:17(float) - │ │ │ ├── cardinality: [1 - 1] - │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(17) - │ │ │ ├── select - │ │ │ │ ├── columns: c_phone:13(char!null) c_acctbal:14(float!null) - │ │ │ │ ├── scan customer - │ │ │ │ │ └── columns: c_phone:13(char!null) c_acctbal:14(float!null) - │ │ │ │ └── filters - │ │ │ │ ├── c_acctbal > 0.0 [type=bool, outer=(14), constraints=(/14: [/5e-324 - ]; tight)] - │ │ │ │ └── substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31') [type=bool, outer=(13)] - │ │ │ └── aggregations - │ │ │ └── avg [type=float, outer=(14)] - │ │ │ └── variable: c_acctbal [type=float] - │ │ └── filters (true) - │ └── projections - │ └── substring(c_phone, 1, 2) [type=string, outer=(5)] - └── aggregations - ├── count-rows [type=int] - └── sum [type=float, outer=(6)] - └── variable: c_acctbal [type=float] + ├── sort + │ ├── columns: c_acctbal:6(float!null) cntrycode:27(string) + │ ├── ordering: +27 + │ └── project + │ ├── columns: cntrycode:27(string) c_acctbal:6(float!null) + │ ├── anti-join (merge) + │ │ ├── columns: c_custkey:1(int!null) c_phone:5(char!null) c_acctbal:6(float!null) + │ │ ├── left ordering: +1 + │ │ ├── right ordering: +19 + │ │ ├── key: (1) + │ │ ├── fd: (1)-->(5,6) + │ │ ├── select + │ │ │ ├── columns: c_custkey:1(int!null) c_phone:5(char!null) c_acctbal:6(float!null) + │ │ │ ├── key: (1) + │ │ │ ├── fd: (1)-->(5,6) + │ │ │ ├── ordering: +1 + │ │ │ ├── scan customer + │ │ │ │ ├── columns: c_custkey:1(int!null) c_phone:5(char!null) c_acctbal:6(float!null) + │ │ │ │ ├── key: (1) + │ │ │ │ ├── fd: (1)-->(5,6) + │ │ │ │ └── ordering: +1 + │ │ │ └── filters + │ │ │ ├── substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31') [type=bool, outer=(5)] + │ │ │ └── gt [type=bool, outer=(6), subquery, constraints=(/6: (/NULL - ])] + │ │ │ ├── variable: c_acctbal [type=float] + │ │ │ └── subquery [type=float] + │ │ │ └── scalar-group-by + │ │ │ ├── columns: avg:17(float) + │ │ │ ├── cardinality: [1 - 1] + │ │ │ ├── key: () + │ │ │ ├── fd: ()-->(17) + │ │ │ ├── select + │ │ │ │ ├── columns: c_phone:13(char!null) c_acctbal:14(float!null) + │ │ │ │ ├── scan customer + │ │ │ │ │ └── columns: c_phone:13(char!null) c_acctbal:14(float!null) + │ │ │ │ └── filters + │ │ │ │ ├── c_acctbal > 0.0 [type=bool, outer=(14), constraints=(/14: [/5e-324 - ]; tight)] + │ │ │ │ └── substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31') [type=bool, outer=(13)] + │ │ │ └── aggregations + │ │ │ └── avg [type=float, outer=(14)] + │ │ │ └── variable: c_acctbal [type=float] + │ │ ├── scan orders@o_ck + │ │ │ ├── columns: o_custkey:19(int!null) + │ │ │ └── ordering: +19 + │ │ └── filters (true) + │ └── projections + │ └── substring(c_phone, 1, 2) [type=string, outer=(5)] + └── aggregations + ├── count-rows [type=int] + └── sum [type=float, outer=(6)] + └── variable: c_acctbal [type=float] diff --git a/pkg/sql/opt/xform/testdata/rules/join b/pkg/sql/opt/xform/testdata/rules/join index 8f6630a0d031..bcf96de010e4 100644 --- a/pkg/sql/opt/xform/testdata/rules/join +++ b/pkg/sql/opt/xform/testdata/rules/join @@ -691,11 +691,11 @@ CREATE TABLE kfloat (k FLOAT PRIMARY KEY) memo SELECT * FROM abc JOIN kfloat ON a=k ---- -memo (optimized, ~5KB, required=[presentation: a:1,b:2,c:3,k:5]) +memo (optimized, ~8KB, required=[presentation: a:1,b:2,c:3,k:5]) ├── G1: (inner-join G2 G3 G4) (inner-join G3 G2 G4) │ └── [presentation: a:1,b:2,c:3,k:5] │ ├── best: (inner-join G2 G3 G4) - │ └── cost: 2130.05 + │ └── cost: 12120.05 ├── G2: (scan abc,cols=(1-3)) (scan abc@ab,cols=(1-3)) (scan abc@bc,cols=(1-3)) │ └── [] │ ├── best: (scan abc,cols=(1-3)) @@ -1789,7 +1789,7 @@ select memo SELECT p,q,r,s FROM pqr WHERE q = 1 AND r = 1 AND s = 'foo' ---- -memo (optimized, ~34KB, required=[presentation: p:1,q:2,r:3,s:4]) +memo (optimized, ~41KB, required=[presentation: p:1,q:2,r:3,s:4]) ├── G1: (select G2 G3) (lookup-join G4 G5 pqr,keyCols=[1],outCols=(1-4)) (zigzag-join G3 pqr@q pqr@s) (zigzag-join G3 pqr@q pqr@rs) (lookup-join G6 G7 pqr,keyCols=[1],outCols=(1-4)) (lookup-join G8 G7 pqr,keyCols=[1],outCols=(1-4)) (lookup-join G9 G7 pqr,keyCols=[1],outCols=(1-4)) (select G10 G11) (select G12 G13) (select G14 G7) (select G15 G7) │ └── [presentation: p:1,q:2,r:3,s:4] │ ├── best: (zigzag-join G3 pqr@q pqr@s) @@ -1983,15 +1983,15 @@ inner-join (lookup t5) memo SELECT a FROM t5 WHERE b @> '{"a":1, "c":2}' ---- -memo (optimized, ~11KB, required=[presentation: a:1]) +memo (optimized, ~13KB, required=[presentation: a:1]) ├── G1: (project G2 G3 a) │ └── [presentation: a:1] │ ├── best: (project G2 G3 a) - │ └── cost: 100.41 + │ └── cost: 100.29 ├── G2: (select G4 G5) (lookup-join G6 G5 t5,keyCols=[1],outCols=(1,2)) (select G7 G5) │ └── [] │ ├── best: (lookup-join G6 G5 t5,keyCols=[1],outCols=(1,2)) - │ └── cost: 100.28 + │ └── cost: 100.15 ├── G3: (projections) ├── G4: (scan t5,cols=(1,2)) │ └── [] @@ -2390,13 +2390,13 @@ 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] - ├── stats: [rows=100] - ├── cost: 712.03 + ├── stats: [rows=100, distinct(1)=100, null(1)=0, distinct(3)=10, null(3)=0] + ├── cost: 507.0505 ├── prune: (2) ├── interesting orderings: (+1,+2) (+2,+3) ├── scan t.public.abc │ ├── columns: t.public.abc.a:1(int) t.public.abc.b:2(int) t.public.abc.c:3(int) - │ ├── stats: [rows=100] + │ ├── stats: [rows=100, distinct(1)=100, null(1)=0, distinct(3)=10, null(3)=1] │ ├── cost: 107.02 │ ├── prune: (1-3) │ └── interesting orderings: (+1,+2) (+2,+3) @@ -2408,26 +2408,20 @@ semi-join (lookup def) opt format=show-all SELECT * from abc WHERE EXISTS (SELECT * FROM def WHERE a=d AND c=e) ---- -project +semi-join (lookup def) ├── columns: a:1(int) b:2(int) c:3(int) - ├── stats: [rows=100] - ├── cost: 508.0605 + ├── key columns: [1 3] = [5 6] + ├── stats: [rows=100, distinct(1)=100, null(1)=0, distinct(3)=10, null(3)=0] + ├── cost: 507.0505 ├── prune: (2) ├── interesting orderings: (+1,+2) (+2,+3) - └── inner-join (lookup def) - ├── columns: t.public.abc.a:1(int!null) t.public.abc.b:2(int) t.public.abc.c:3(int!null) t.public.def.d:5(int!null) t.public.def.e:6(int!null) - ├── key columns: [1 3] = [5 6] - ├── stats: [rows=0.01, distinct(1)=0.01, null(1)=0, distinct(3)=0.01, null(3)=0, distinct(5)=0.01, null(5)=0, distinct(6)=0.01, null(6)=0] - ├── cost: 507.0505 - ├── fd: (1)==(5), (5)==(1), (3)==(6), (6)==(3) - ├── interesting orderings: (+1,+2) (+2,+3) - ├── scan t.public.abc - │ ├── columns: t.public.abc.a:1(int) t.public.abc.b:2(int) t.public.abc.c:3(int) - │ ├── stats: [rows=100, distinct(1)=100, null(1)=0, distinct(3)=10, null(3)=1] - │ ├── cost: 107.02 - │ ├── prune: (1-3) - │ └── interesting orderings: (+1,+2) (+2,+3) - └── filters (true) + ├── scan t.public.abc + │ ├── columns: t.public.abc.a:1(int) t.public.abc.b:2(int) t.public.abc.c:3(int) + │ ├── stats: [rows=100, distinct(1)=100, null(1)=0, distinct(3)=10, null(3)=1] + │ ├── cost: 107.02 + │ ├── prune: (1-3) + │ └── interesting orderings: (+1,+2) (+2,+3) + └── filters (true) exec-ddl CREATE TABLE customers (id INT PRIMARY KEY, name STRING) diff --git a/pkg/sql/opt/xform/testdata/rules/join_order b/pkg/sql/opt/xform/testdata/rules/join_order index 926433587226..7944f748a9be 100644 --- a/pkg/sql/opt/xform/testdata/rules/join_order +++ b/pkg/sql/opt/xform/testdata/rules/join_order @@ -127,7 +127,7 @@ inner-join (lookup bx) memo join-limit=3 SELECT * FROM bx, cy, abc WHERE a = 1 AND abc.b = bx.b AND abc.c = cy.c ---- -memo (optimized, ~25KB, required=[presentation: b:1,x:2,c:3,y:4,a:5,b:6,c:7,d:8]) +memo (optimized, ~30KB, 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)) (inner-join G6 G7 G8) (inner-join G9 G10 G11) (inner-join G7 G6 G8) (merge-join G6 G7 G5 inner-join,+3,+7) (inner-join G10 G9 G11) (lookup-join G7 G5 cy,keyCols=[7],outCols=(1-8)) (lookup-join G12 G11 abc,keyCols=[11],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)) diff --git a/pkg/sql/testdata/explain_tree b/pkg/sql/testdata/explain_tree index fc24b3465671..f92e99b7a711 100644 --- a/pkg/sql/testdata/explain_tree +++ b/pkg/sql/testdata/explain_tree @@ -152,14 +152,17 @@ SELECT cid, date, value FROM t.orders WHERE date IN (SELECT date FROM t.orders) 0 .render 1 (@3)[date] (cid int, date date, value decimal) 0 .render 2 (@2)[decimal] (cid int, date date, value decimal) 1 hash-join (cid int, date date, value decimal) -1 .type semi (cid int, date date, value decimal) +1 .type inner (cid int, date date, value decimal) 1 .equality (date) = (date) (cid int, date date, value decimal) +1 .right cols are key (cid int, date date, value decimal) 2 scan (cid int, date date, value decimal) 2 .table orders@primary (cid int, date date, value decimal) 2 .spans ALL (cid int, date date, value decimal) -2 scan (cid int, date date, value decimal) -2 .table orders@primary (cid int, date date, value decimal) -2 .spans ALL (cid int, date date, value decimal) +2 distinct (cid int, date date, value decimal) +2 .distinct on date (cid int, date date, value decimal) +3 scan (cid int, date date, value decimal) +3 .table orders@primary (cid int, date date, value decimal) +3 .spans ALL (cid int, date date, value decimal) plan-tree SELECT cid, date, value FROM t.orders WHERE date IN (SELECT date FROM t.orders) @@ -176,9 +179,11 @@ children: - name: hash-join attrs: - key: type - value: semi + value: inner - key: equality value: (date) = (date) + - key: right cols are key + value: "" children: - name: scan attrs: @@ -187,13 +192,18 @@ children: - key: spans value: ALL children: [] - - name: scan + - name: distinct attrs: - - key: table - value: orders@primary - - key: spans - value: ALL - children: [] + - key: distinct on + value: date + children: + - name: scan + attrs: + - key: table + value: orders@primary + - key: spans + value: ALL + children: [] exec CREATE TABLE t.movies (