Skip to content

Commit

Permalink
optbuilder: return scope from mutationBuilder.buildCheckInputScan
Browse files Browse the repository at this point in the history
This commit updates `mutationBuilder.buildCheckInputScan` to build a
scope containing the constructed WithScan expression and the scope
columns output by the expression.

Access to a scope for the WithScan expression will be required to
support partial `UNIQUE WITHOUT INDEX` constraints. For uniqueness
checks, a filter on the WithScan columns must be added to the semi-join
filters. The WithScan scope will be required in order to build the
filter expression.

Release note: None
  • Loading branch information
mgartner committed Feb 12, 2021
1 parent ea1ef0b commit 80b33c8
Show file tree
Hide file tree
Showing 4 changed files with 69 additions and 45 deletions.
39 changes: 24 additions & 15 deletions pkg/sql/opt/optbuilder/mutation_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -1281,25 +1281,28 @@ const (
checkInputScanFetchedVals
)

// makeCheckInputScan constructs a WithScan that iterates over the input to the
// buildCheckInputScan constructs a WithScan that iterates over the input to the
// mutation operator. Used in expressions that generate rows for checking for FK
// and uniqueness violations.
//
// The WithScan expression will scan either the new values or the fetched values
// for the given table ordinals (which correspond to FK or unique columns).
//
// Returns the output columns from the WithScan, which map 1-to-1 to
// tabOrdinals. Also returns the subset of these columns that can be assumed
// to be not null (either because they are not null in the mutation input or
// because they are non-nullable table columns).
// Returns a scope containing the WithScan expression and the output columns
// from the WithScan. The output columns map 1-to-1 to tabOrdinals. Also returns
// the subset of these columns that can be assumed to be not null (either
// because they are not null in the mutation input or because they are
// non-nullable table columns).
//
func (mb *mutationBuilder) makeCheckInputScan(
func (mb *mutationBuilder) buildCheckInputScan(
typ checkInputScanType, tabOrdinals []int,
) (scan memo.RelExpr, outCols opt.ColList, notNullOutCols opt.ColSet) {
) (withScanScope *scope, notNullOutCols opt.ColSet) {
// inputCols are the column IDs from the mutation input that we are scanning.
inputCols := make(opt.ColList, len(tabOrdinals))
// outCols will store the newly synthesized output columns for WithScan.
outCols = make(opt.ColList, len(inputCols))

withScanScope = mb.b.allocScope()
withScanScope.cols = make([]scopeColumn, len(inputCols))

for i, tabOrd := range tabOrdinals {
if typ == checkInputScanNewVals {
inputCols[i] = mb.mapToReturnColID(tabOrd)
Expand All @@ -1311,23 +1314,29 @@ func (mb *mutationBuilder) makeCheckInputScan(
}

// Synthesize new column.
c := mb.b.factory.Metadata().ColumnMeta(inputCols[i])
outCols[i] = mb.md.AddColumn(c.Alias, c.Type)
inputCol := mb.b.factory.Metadata().Table(mb.tabID).Column(tabOrd)
inputColMeta := mb.b.factory.Metadata().ColumnMeta(inputCols[i])
outCol := mb.md.AddColumn(inputColMeta.Alias, inputColMeta.Type)
withScanScope.cols[i] = scopeColumn{
id: outCol,
name: inputCol.ColName(),
typ: inputCol.DatumType(),
}

// If a table column is not nullable, NULLs cannot be inserted (the
// mutation will fail). So for the purposes of checks, we can treat
// these columns as not null.
if mb.outScope.expr.Relational().NotNullCols.Contains(inputCols[i]) ||
!mb.tab.Column(tabOrd).IsNullable() {
notNullOutCols.Add(outCols[i])
notNullOutCols.Add(outCol)
}
}

scan = mb.b.factory.ConstructWithScan(&memo.WithScanPrivate{
withScanScope.expr = mb.b.factory.ConstructWithScan(&memo.WithScanPrivate{
With: mb.withID,
InCols: inputCols,
OutCols: outCols,
OutCols: withScanScope.colList(),
ID: mb.b.factory.Metadata().NextUniqueID(),
})
return scan, outCols, notNullOutCols
return withScanScope, notNullOutCols
}
46 changes: 25 additions & 21 deletions pkg/sql/opt/optbuilder/mutation_builder_fk.go
Original file line number Diff line number Diff line change
Expand Up @@ -171,8 +171,8 @@ func (mb *mutationBuilder) buildFKChecksAndCascadesForDelete() {
}

mb.ensureWithID()
fkInput, withScanCols, _ := mb.makeCheckInputScan(checkInputScanFetchedVals, h.tabOrdinals)
mb.fkChecks = append(mb.fkChecks, h.buildDeletionCheck(fkInput, withScanCols))
withScanScope, _ := mb.buildCheckInputScan(checkInputScanFetchedVals, h.tabOrdinals)
mb.fkChecks = append(mb.fkChecks, h.buildDeletionCheck(withScanScope.expr, withScanScope.colList()))
}
telemetry.Inc(sqltelemetry.ForeignKeyChecksUseCounter)
}
Expand Down Expand Up @@ -336,15 +336,17 @@ func (mb *mutationBuilder) buildFKChecksForUpdate() {
// performance either: we would be incurring extra cost (more complicated
// expressions, scanning the input buffer twice) for a rare case.

oldRows, colsForOldRow, _ := mb.makeCheckInputScan(checkInputScanFetchedVals, h.tabOrdinals)
newRows, colsForNewRow, _ := mb.makeCheckInputScan(checkInputScanNewVals, h.tabOrdinals)
oldRowsScope, _ := mb.buildCheckInputScan(checkInputScanFetchedVals, h.tabOrdinals)
newRowsScope, _ := mb.buildCheckInputScan(checkInputScanNewVals, h.tabOrdinals)
colsForOldRow := oldRowsScope.colList()
colsForNewRow := newRowsScope.colList()

// The rows that no longer exist are the ones that were "deleted" by virtue
// of being updated _from_, minus the ones that were "added" by virtue of
// being updated _to_.
deletedRows := mb.b.factory.ConstructExcept(
oldRows,
newRows,
oldRowsScope.expr,
newRowsScope.expr,
&memo.SetPrivate{
LeftCols: colsForOldRow,
RightCols: colsForNewRow,
Expand Down Expand Up @@ -439,22 +441,24 @@ func (mb *mutationBuilder) buildFKChecksForUpsert() {
// insertions (using a "canaryCol IS NOT NULL" condition). But the rows we
// would filter out have all-null fetched values anyway and will never match
// in the semi join.
oldRows, colsForOldRow, _ := mb.makeCheckInputScan(checkInputScanFetchedVals, h.tabOrdinals)
newRows, colsForNewRow, _ := mb.makeCheckInputScan(checkInputScanNewVals, h.tabOrdinals)
oldRowsScope, _ := mb.buildCheckInputScan(checkInputScanFetchedVals, h.tabOrdinals)
newRowsScope, _ := mb.buildCheckInputScan(checkInputScanNewVals, h.tabOrdinals)
colsForOldRow := oldRowsScope.colList()
colsForNewRow := newRowsScope.colList()

// The rows that no longer exist are the ones that were "deleted" by virtue
// of being updated _from_, minus the ones that were "added" by virtue of
// being updated _to_.
deletedRows := mb.b.factory.ConstructExcept(
oldRows,
newRows,
oldRowsScope.expr,
newRowsScope.expr,
&memo.SetPrivate{
LeftCols: colsForOldRow,
RightCols: colsForNewRow,
OutCols: colsForOldRow,
},
)
mb.fkChecks = append(mb.fkChecks, h.buildDeletionCheck(deletedRows, colsForOldRow))
mb.fkChecks = append(mb.fkChecks, h.buildDeletionCheck(deletedRows, oldRowsScope.colList()))
}
telemetry.Inc(sqltelemetry.ForeignKeyChecksUseCounter)
}
Expand Down Expand Up @@ -638,11 +642,11 @@ func (h *fkCheckHelper) allocOrdinals(numCols int) {
// The input to the insertion check will be produced from the input to the
// mutation operator.
func (h *fkCheckHelper) buildInsertionCheck() memo.FKChecksItem {
fkInput, withScanCols, notNullWithScanCols := h.mb.makeCheckInputScan(
withScanScope, notNullWithScanCols := h.mb.buildCheckInputScan(
checkInputScanNewVals, h.tabOrdinals,
)

numCols := len(withScanCols)
numCols := len(withScanScope.cols)
f := h.mb.b.factory
if notNullWithScanCols.Len() < numCols {
// The columns we are inserting might have NULLs. These require special
Expand All @@ -667,7 +671,7 @@ func (h *fkCheckHelper) buildInsertionCheck() memo.FKChecksItem {
// Filter out any rows which have a NULL; build filters of the form
// (a IS NOT NULL) AND (b IS NOT NULL) ...
filters := make(memo.FiltersExpr, 0, numCols-notNullWithScanCols.Len())
for _, col := range withScanCols {
for _, col := range withScanScope.colList() {
if !notNullWithScanCols.Contains(col) {
filters = append(filters, f.ConstructFiltersItem(
f.ConstructIsNot(
Expand All @@ -677,7 +681,7 @@ func (h *fkCheckHelper) buildInsertionCheck() memo.FKChecksItem {
))
}
}
fkInput = f.ConstructSelect(fkInput, filters)
withScanScope.expr = f.ConstructSelect(withScanScope.expr, filters)

case tree.MatchFull:
// Filter out any rows which have NULLs on all referencing columns.
Expand All @@ -690,7 +694,7 @@ func (h *fkCheckHelper) buildInsertionCheck() memo.FKChecksItem {
// Build a filter of the form
// (a IS NOT NULL) OR (b IS NOT NULL) ...
var condition opt.ScalarExpr
for _, col := range withScanCols {
for _, col := range withScanScope.colList() {
is := f.ConstructIsNot(
f.ConstructVariable(col),
memo.NullSingleton,
Expand All @@ -701,8 +705,8 @@ func (h *fkCheckHelper) buildInsertionCheck() memo.FKChecksItem {
condition = f.ConstructOr(condition, is)
}
}
fkInput = f.ConstructSelect(
fkInput,
withScanScope.expr = f.ConstructSelect(
withScanScope.expr,
memo.FiltersExpr{f.ConstructFiltersItem(condition)},
)

Expand All @@ -722,7 +726,7 @@ func (h *fkCheckHelper) buildInsertionCheck() memo.FKChecksItem {
for j := 0; j < numCols; j++ {
antiJoinFilters[j] = f.ConstructFiltersItem(
f.ConstructEq(
f.ConstructVariable(withScanCols[j]),
f.ConstructVariable(withScanScope.cols[j].id),
f.ConstructVariable(scanScope.cols[j].id),
),
)
Expand All @@ -731,14 +735,14 @@ func (h *fkCheckHelper) buildInsertionCheck() memo.FKChecksItem {
if h.mb.b.evalCtx.SessionData.PreferLookupJoinsForFKs {
p.Flags = memo.PreferLookupJoinIntoRight
}
antiJoin := f.ConstructAntiJoin(fkInput, scanScope.expr, antiJoinFilters, &p)
antiJoin := f.ConstructAntiJoin(withScanScope.expr, scanScope.expr, antiJoinFilters, &p)

return f.ConstructFKChecksItem(antiJoin, &memo.FKChecksItemPrivate{
OriginTable: h.mb.tabID,
ReferencedTable: refTabMeta.MetaID,
FKOutbound: true,
FKOrdinal: h.fkOrdinal,
KeyCols: withScanCols,
KeyCols: withScanScope.colList(),
OpName: h.mb.opName,
})
}
Expand Down
19 changes: 10 additions & 9 deletions pkg/sql/opt/optbuilder/mutation_builder_unique.go
Original file line number Diff line number Diff line change
Expand Up @@ -189,11 +189,11 @@ func (h *uniqueCheckHelper) init(mb *mutationBuilder, uniqueOrdinal int) bool {
// table. The input to the insertion check will be produced from the input to
// the mutation operator.
func (h *uniqueCheckHelper) buildInsertionCheck() memo.UniqueChecksItem {
checkInput, withScanCols, _ := h.mb.makeCheckInputScan(
withScanScope, _ := h.mb.buildCheckInputScan(
checkInputScanNewVals, h.uniqueAndPrimaryKeyOrdinals,
)

numCols := len(withScanCols)
numCols := len(withScanScope.cols)
f := h.mb.b.factory

// Build a self semi-join, with the new values on the left and the
Expand All @@ -211,7 +211,7 @@ func (h *uniqueCheckHelper) buildInsertionCheck() memo.UniqueChecksItem {
for i := 0; i < len(h.uniqueOrdinals); i++ {
semiJoinFilters = append(semiJoinFilters, f.ConstructFiltersItem(
f.ConstructEq(
f.ConstructVariable(withScanCols[i]),
f.ConstructVariable(withScanScope.cols[i].id),
f.ConstructVariable(scanScope.cols[i].id),
),
))
Expand All @@ -224,7 +224,7 @@ func (h *uniqueCheckHelper) buildInsertionCheck() memo.UniqueChecksItem {
var pkFilter opt.ScalarExpr
for i := len(h.uniqueOrdinals); i < numCols; i++ {
pkFilterLocal := f.ConstructNe(
f.ConstructVariable(withScanCols[i]),
f.ConstructVariable(withScanScope.cols[i].id),
f.ConstructVariable(scanScope.cols[i].id),
)
if pkFilter == nil {
Expand All @@ -235,15 +235,16 @@ func (h *uniqueCheckHelper) buildInsertionCheck() memo.UniqueChecksItem {
}
semiJoinFilters = append(semiJoinFilters, f.ConstructFiltersItem(pkFilter))

semiJoin := f.ConstructSemiJoin(checkInput, scanScope.expr, semiJoinFilters, memo.EmptyJoinPrivate)
semiJoin := f.ConstructSemiJoin(withScanScope.expr, scanScope.expr, semiJoinFilters, memo.EmptyJoinPrivate)

return f.ConstructUniqueChecksItem(semiJoin, &memo.UniqueChecksItemPrivate{
Table: h.mb.tabID,
CheckOrdinal: h.uniqueOrdinal,
// uniqueOrdinals is always a prefix of uniqueAndPrimaryKeyOrdinals, which
// maps 1-to-1 to the columns in withScanCols. The remaining columns are
// primary key columns and should not be included in the KeyCols.
KeyCols: withScanCols[:len(h.uniqueOrdinals)],
// uniqueOrdinals is always a prefix of uniqueAndPrimaryKeyOrdinals,
// which maps 1-to-1 to the columns in withScanScope.cols. The remaining
// columns are primary key columns and should not be included in the
// KeyCols.
KeyCols: withScanScope.colList()[:len(h.uniqueOrdinals)],
OpName: h.mb.opName,
})
}
Expand Down
10 changes: 10 additions & 0 deletions pkg/sql/opt/optbuilder/scope.go
Original file line number Diff line number Diff line change
Expand Up @@ -526,6 +526,16 @@ func (s *scope) colSetWithExtraCols() opt.ColSet {
return colSet
}

// colList returns a ColList of all the columns in this scope,
// excluding orderByCols.
func (s *scope) colList() opt.ColList {
colList := make(opt.ColList, len(s.cols))
for i := range s.cols {
colList[i] = s.cols[i].id
}
return colList
}

// hasSameColumns returns true if this scope has the same columns
// as the other scope.
//
Expand Down

0 comments on commit 80b33c8

Please sign in to comment.