Skip to content

Commit

Permalink
sql/row: fix updates of single-composite-column families
Browse files Browse the repository at this point in the history
When updating a single-column family which contains what could be a
composite value from the primary key, we still need to issue a Del even
if the new value for the column is not composite, because the previous
value might have been composite.

Fixes: #131860
Informs: #131645

Release note (bug fix): Fix a rare bug in which an update of a primary
key column which is also the only column in a separate column family can
sometimes fail to update the primary index. This bug has existed since
v22.2.
  • Loading branch information
michae2 committed Oct 4, 2024
1 parent 74af8a5 commit 1672db9
Show file tree
Hide file tree
Showing 5 changed files with 71 additions and 27 deletions.
2 changes: 1 addition & 1 deletion pkg/sql/colenc/encode.go
Original file line number Diff line number Diff line change
Expand Up @@ -701,7 +701,7 @@ func (b *BatchEncoder) skipColumnNotInPrimaryIndexValue(
colID catid.ColumnID, vec *coldata.Vec, row int,
) bool {
// Reuse this function but fake out the value and handle composites here.
if skip := b.rh.SkipColumnNotInPrimaryIndexValue(colID, tree.DNull); skip {
if skip, _ := b.rh.SkipColumnNotInPrimaryIndexValue(colID, tree.DNull); skip {
if !b.compositeColumnIDs.Contains(int(colID)) {
return true
}
Expand Down
27 changes: 27 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/column_families
Original file line number Diff line number Diff line change
Expand Up @@ -79,3 +79,30 @@ ORDER BY message
fetched: /t/t_pkey/1/2.00/x -> /1.00
fetched: /t/t_pkey/1/2/y -> /2.00
fetched: /t/t_pkey/1/2/z -> /1

# Regression test for #131860.

statement ok
CREATE TABLE abc (a INT NOT NULL, b FLOAT NOT NULL, c INT, FAMILY (a), FAMILY (b), FAMILY (c))

statement ok
INSERT INTO abc VALUES (4, -0, 6)

statement ok
ALTER TABLE abc ADD PRIMARY KEY (a, b)

statement ok
UPDATE abc SET c = NULL WHERE a = 4 AND b = -0

query IFI
SELECT * FROM abc
----
4 -0 NULL

statement ok
UPDATE abc SET b = 0 WHERE a = 4 AND b = -0;

query IFI
SELECT * FROM abc
----
4 0 NULL
4 changes: 2 additions & 2 deletions pkg/sql/row/deleter.go
Original file line number Diff line number Diff line change
Expand Up @@ -194,7 +194,7 @@ func (rd *Deleter) encodeValueForPrimaryIndexFamily(
if !ok {
return roachpb.Value{}, nil
}
if rd.Helper.SkipColumnNotInPrimaryIndexValue(family.DefaultColumnID, values[idx]) {
if skip, _ := rd.Helper.SkipColumnNotInPrimaryIndexValue(family.DefaultColumnID, values[idx]); skip {
return roachpb.Value{}, nil
}
typ := rd.FetchCols[idx].GetType()
Expand All @@ -218,7 +218,7 @@ func (rd *Deleter) encodeValueForPrimaryIndexFamily(
continue
}

if skip := rd.Helper.SkipColumnNotInPrimaryIndexValue(colID, values[idx]); skip {
if skip, _ := rd.Helper.SkipColumnNotInPrimaryIndexValue(colID, values[idx]); skip {
continue
}

Expand Down
11 changes: 6 additions & 5 deletions pkg/sql/row/helper.go
Original file line number Diff line number Diff line change
Expand Up @@ -210,25 +210,26 @@ func (rh *RowHelper) encodeSecondaryIndexes(
// SkipColumnNotInPrimaryIndexValue returns true if the value at column colID
// does not need to be encoded, either because it is already part of the primary
// key, or because it is not part of the primary index altogether. Composite
// datums are considered too, so a composite datum in a PK will return false.
// datums are considered too, so a composite datum in a PK will return false
// (but will return true for couldBeComposite).
func (rh *RowHelper) SkipColumnNotInPrimaryIndexValue(
colID descpb.ColumnID, value tree.Datum,
) bool {
) (skip, couldBeComposite bool) {
if rh.primaryIndexKeyCols.Empty() {
rh.primaryIndexKeyCols = rh.TableDesc.GetPrimaryIndex().CollectKeyColumnIDs()
rh.primaryIndexValueCols = rh.TableDesc.GetPrimaryIndex().CollectPrimaryStoredColumnIDs()
}
if !rh.primaryIndexKeyCols.Contains(colID) {
return !rh.primaryIndexValueCols.Contains(colID)
return !rh.primaryIndexValueCols.Contains(colID), false
}
if cdatum, ok := value.(tree.CompositeDatum); ok {
// Composite columns are encoded in both the key and the value.
return !cdatum.IsComposite()
return !cdatum.IsComposite(), true
}
// Skip primary key columns as their values are encoded in the key of
// each family. Family 0 is guaranteed to exist and acts as a
// sentinel.
return true
return true, false
}

func (rh *RowHelper) SortedColumnFamily(famID descpb.FamilyID) ([]descpb.ColumnID, bool) {
Expand Down
54 changes: 35 additions & 19 deletions pkg/sql/row/writer.go
Original file line number Diff line number Diff line change
Expand Up @@ -136,38 +136,54 @@ func prepareInsertOrUpdateBatch(
if !ok {
continue
}

var marshaled roachpb.Value
var err error
typ := fetchedCols[idx].GetType()

// Skip any values with a default ID not stored in the primary index,
// which can happen if we are adding new columns.
if skip := helper.SkipColumnNotInPrimaryIndexValue(family.DefaultColumnID, values[idx]); skip {
continue
}
typ := fetchedCols[idx].GetType()
marshaled, err := valueside.MarshalLegacy(typ, values[idx])
if err != nil {
return nil, err
skip, couldBeComposite := helper.SkipColumnNotInPrimaryIndexValue(family.DefaultColumnID, values[idx])
if skip {
// If the column could be composite, there could be a previous KV, so we
// still need to issue a Delete.
if !couldBeComposite {
continue
}
} else {
marshaled, err = valueside.MarshalLegacy(typ, values[idx])
if err != nil {
return nil, err
}
}

// TODO(ssd): Here and below investigate reducing the
// number of allocations required to marshal the old
// value.
var oldVal []byte
if oth.IsSet() && len(oldValues) > 0 {
old, err := valueside.MarshalLegacy(typ, oldValues[idx])
if err != nil {
return nil, err
// If the column could be composite, we only encode the old value if it
// was a composite value.
if !couldBeComposite || oldValues[idx].(tree.CompositeDatum).IsComposite() {
old, err := valueside.MarshalLegacy(typ, oldValues[idx])
if err != nil {
return nil, err
}
if old.IsPresent() {
oldVal = old.TagAndDataBytes()
}
}
oldVal = old.TagAndDataBytes()
}

if marshaled.RawBytes == nil {
if overwrite {
if !marshaled.IsPresent() {
if oth.IsSet() {
// If using OriginTimestamp'd CPuts, we _always_ want to issue a Delete
// so that we can confirm our expected bytes were correct.
oth.DelWithCPut(ctx, batch, kvKey, oldVal, traceKV)
} else if overwrite {
// If the new family contains a NULL value, then we must
// delete any pre-existing row.
if oth.IsSet() {
oth.DelWithCPut(ctx, batch, kvKey, oldVal, traceKV)
} else {
insertDelFn(ctx, batch, kvKey, traceKV)
}
insertDelFn(ctx, batch, kvKey, traceKV)
}
} else {
// We only output non-NULL values. Non-existent column keys are
Expand Down Expand Up @@ -203,7 +219,7 @@ func prepareInsertOrUpdateBatch(
continue
}

if skip := helper.SkipColumnNotInPrimaryIndexValue(colID, values[idx]); skip {
if skip, _ := helper.SkipColumnNotInPrimaryIndexValue(colID, values[idx]); skip {
continue
}

Expand Down

0 comments on commit 1672db9

Please sign in to comment.