Skip to content

Commit

Permalink
Merge #30707
Browse files Browse the repository at this point in the history
30707: sql: only fetch specific columns needed to validate check constraints for UPDATES r=nvanbenschoten a=nvanbenschoten

This change begins with some cleanup I stumbled upon when looking into addressing #30618. The tweaks are minor, although the second commit might be backport worthy. It then addresses a related issue that was slightly easier to fix.

The change addresses a longstanding TODO that's related to #30618 but was a little easier to start with. The goal is to only request the columns used in check constraints when performing an UPDATE instead of blindly requesting all columns when a check constraint is present. 8d37935 did all of the heavy lifting for this. This change just plugs it in.

The new logic tests fail if we don't correctly request the columns we need for the check constraint. Interestingly, this isn't because we don't fetch the other columns from KV. Instead, it's because we
don't decode them and pass them up the stack.

Co-authored-by: Nathan VanBenschoten <[email protected]>
  • Loading branch information
craig[bot] and nvanbenschoten committed Nov 12, 2018
2 parents 3bfc2b1 + 82a1d58 commit b2d892a
Show file tree
Hide file tree
Showing 4 changed files with 124 additions and 7 deletions.
74 changes: 74 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/check_constraints
Original file line number Diff line number Diff line change
Expand Up @@ -273,3 +273,77 @@ CREATE TABLE test2.t (
CHECK (t.a > 0),
CHECK (test2.t.a > 0)
)

# Use multiple column families.

statement ok
CREATE TABLE t9 (
a INT PRIMARY KEY,
b INT,
c INT,
d INT,
e INT,
FAMILY (a),
FAMILY (b),
FAMILY (c),
FAMILY (d, e),
CHECK (a > b),
CHECK (d IS NULL)
)

statement ok
INSERT INTO t9 VALUES (5, 3)

statement error pgcode 23514 failed to satisfy CHECK constraint \(a > b\)
INSERT INTO t9 VALUES (6, 7)

statement ok
UPDATE t9 SET b = 4 WHERE a = 5

statement error pgcode 23514 failed to satisfy CHECK constraint \(a > b\)
UPDATE t9 SET b = 6 WHERE a = 5

# Only column families that are needed to validate check constraints are fetched.
query TTTTT
EXPLAIN (VERBOSE) UPDATE t9 SET b = 2 WHERE a = 5
----
count · · () ·
└── update · · () ·
│ table t9 · ·
│ set b · ·
│ check 0 t9.a > t9.b · ·
│ check 1 t9.d IS NULL · ·
└── render · · (a, b, d, "?column?") a=CONST; "?column?"=CONST; key()
│ render 0 test.public.t9.a · ·
│ render 1 test.public.t9.b · ·
│ render 2 test.public.t9.d · ·
│ render 3 2 · ·
└── scan · · (a, b, c[omitted], d, e[omitted]) a=CONST; key()
· table t9@primary · ·
· spans /5/0-/5/1/2 /5/3/1-/5/3/2 · ·

statement ok
UPDATE t9 SET a = 7 WHERE a = 4

statement error pgcode 23514 failed to satisfy CHECK constraint \(a > b\)
UPDATE t9 SET a = 2 WHERE a = 5

query TTTTT
EXPLAIN (VERBOSE) UPDATE t9 SET a = 2 WHERE a = 5
----
count · · () ·
└── update · · () ·
│ table t9 · ·
│ set a · ·
│ check 0 t9.a > t9.b · ·
│ check 1 t9.d IS NULL · ·
└── render · · (a, b, c, d, e, "?column?") a=CONST; "?column?"=CONST; key()
│ render 0 test.public.t9.a · ·
│ render 1 test.public.t9.b · ·
│ render 2 test.public.t9.c · ·
│ render 3 test.public.t9.d · ·
│ render 4 test.public.t9.e · ·
│ render 5 2 · ·
└── scan · · (a, b, c, d, e) a=CONST; key()
· table t9@primary · ·
· spans /5-/5/# · ·
1 change: 0 additions & 1 deletion pkg/sql/returning.go
Original file line number Diff line number Diff line change
Expand Up @@ -73,7 +73,6 @@ func (p *planner) Returning(
// Ensure there are no special functions in the RETURNING clause.
p.semaCtx.Properties.Require("RETURNING", tree.RejectSpecial)

r.ivarHelper = tree.MakeIndexedVarHelper(r, len(r.source.info.SourceColumns))
err := p.initTargets(ctx, r, tree.SelectExprs(*t), desiredTypes)
if err != nil {
return nil, err
Expand Down
19 changes: 16 additions & 3 deletions pkg/sql/row/writer.go
Original file line number Diff line number Diff line change
Expand Up @@ -220,7 +220,8 @@ func (ri *Inserter) InsertRow(
// (must be adapted depending on whether 'overwrite' is set)
// - helper is the rowHelper that knows about the table being modified.
// - primaryIndexKey is the PK prefix for the current row.
// - updatedCols is the list of schema columns being updated.
// - fetchedCols is the list of schema columns that have been fetched
// in preparation for this update.
// - values is the SQL-level row values that are being written.
// - marshaledValues contains the pre-encoded KV-level row values.
// marshaledValues is only used when writing single column families.
Expand All @@ -241,7 +242,7 @@ func prepareInsertOrUpdateBatch(
batch putter,
helper *rowHelper,
primaryIndexKey []byte,
updatedCols []sqlbase.ColumnDescriptor,
fetchedCols []sqlbase.ColumnDescriptor,
values []tree.Datum,
valColIDMapping map[sqlbase.ColumnID]int,
marshaledValues []roachpb.Value,
Expand Down Expand Up @@ -317,7 +318,7 @@ func prepareInsertOrUpdateBatch(
continue
}

col := updatedCols[idx]
col := fetchedCols[idx]

if lastColID > col.ID {
return nil, pgerror.NewAssertionErrorf("cannot write column id %d after %d", col.ID, lastColID)
Expand Down Expand Up @@ -549,6 +550,8 @@ func makeUpdaterWithoutCascader(
ru.FetchCols = requestedCols[:len(requestedCols):len(requestedCols)]
ru.FetchColIDtoRowIndex = ColIDtoRowIndexFromCols(ru.FetchCols)

// maybeAddCol adds the provided column to ru.FetchCols and
// ru.FetchColIDtoRowIndex if it isn't already present.
maybeAddCol := func(colID sqlbase.ColumnID) error {
if _, ok := ru.FetchColIDtoRowIndex[colID]; !ok {
col, err := tableDesc.FindColumnByID(colID)
Expand All @@ -560,11 +563,18 @@ func makeUpdaterWithoutCascader(
}
return nil
}

// Fetch all columns in the primary key so that we can construct the
// keys when writing out the new kvs to the primary index.
for _, colID := range tableDesc.PrimaryIndex.ColumnIDs {
if err := maybeAddCol(colID); err != nil {
return Updater{}, err
}
}

// If any part of a column family is being updated, fetch all columns in
// that column family so that we can reconstruct the column family with
// the updated columns before writing it.
for _, fam := range tableDesc.Families {
familyBeingUpdated := false
for _, colID := range fam.ColumnIDs {
Expand All @@ -581,6 +591,9 @@ func makeUpdaterWithoutCascader(
}
}
}

// Fetch all columns from indices that are being update so that they can
// be used to create the new kv pairs for those indices.
for _, index := range indexes {
if err := index.RunOverAllColumns(maybeAddCol); err != nil {
return Updater{}, err
Expand Down
37 changes: 34 additions & 3 deletions pkg/sql/update.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sem/types"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/pkg/errors"
)
Expand Down Expand Up @@ -198,12 +199,42 @@ func (p *planner) Update(
rowsNeeded := resultsNeeded(n.Returning)

var requestedCols []sqlbase.ColumnDescriptor
if rowsNeeded || len(desc.Checks) > 0 {
if rowsNeeded {
// TODO(dan): This could be made tighter, just the rows needed for RETURNING
// exprs.
// TODO(nvanbenschoten): This could be made tighter, just the rows needed for
// the CHECK exprs.
requestedCols = desc.Columns
} else if len(desc.Checks) > 0 {
// Request any columns we'll need when validating check constraints. We
// could be smarter and only validate check constraints which depend on
// columns that are being modified in the UPDATE statement, in which
// case we'd only need to request the columns used by that subset of
// check constraints, but that doesn't seem worth the effort.
//
// TODO(nvanbenschoten): These conditions shouldn't be mutually
// exclusive, but since rowsNeeded implies that requestedCols =
// desc.Columns, there's no reason to enter this block if rowsNeeded is
// true. Remove this when the TODO above is addressed.
var requestedColSet util.FastIntSet
for _, col := range requestedCols {
requestedColSet.Add(int(col.ID))
}
for _, ck := range desc.Checks {
cols, err := ck.ColumnsUsed(desc)
if err != nil {
return nil, err
}
for _, colID := range cols {
if !requestedColSet.Contains(int(colID)) {
col, err := desc.FindColumnByID(colID)
if err != nil {
return nil, errors.Wrapf(err, "error finding column %d in table %s",
colID, desc.Name)
}
requestedCols = append(requestedCols, *col)
requestedColSet.Add(int(colID))
}
}
}
}

// Create the table updater, which does the bulk of the work.
Expand Down

0 comments on commit b2d892a

Please sign in to comment.