Skip to content

Commit c0aab48

Browse files
committed
sql: audit production callers of rowenc.DatumToEncDatum
This commit audits all callers of `rowenc.DatumToEncDatum` in order to replace possibly unsafe calls to `DatumToEncDatumEx` which requires explicit error handling (the former panics when the datum's resolved type is not equivalent to the passed-in type). I decided to not remove the former method altogether since it seems to be reasonable to use it when we're construcing the datum directly in the call site (and also in tests). Release note: None
1 parent 52833a4 commit c0aab48

File tree

15 files changed

+84
-32
lines changed

15 files changed

+84
-32
lines changed

pkg/ccl/changefeedccl/avro/avro.go

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -995,7 +995,10 @@ func (r *DataRecord) rowFromNative(native interface{}) (rowenc.EncDatumRow, erro
995995
if err != nil {
996996
return nil, err
997997
}
998-
row[r.colIdxByFieldIdx[fieldIdx]] = rowenc.DatumToEncDatum(field.typ, decoded)
998+
row[r.colIdxByFieldIdx[fieldIdx]], err = rowenc.DatumToEncDatumEx(field.typ, decoded)
999+
if err != nil {
1000+
return nil, err
1001+
}
9991002
}
10001003
return row, nil
10011004
}

pkg/sql/distsql_physical_planner.go

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -4478,7 +4478,10 @@ func (dsp *DistSQLPlanner) createValuesSpecFromTuples(
44784478
if err != nil {
44794479
return nil, err
44804480
}
4481-
encDatum := rowenc.DatumToEncDatum(resultTypes[colIdx], datum)
4481+
encDatum, err := rowenc.DatumToEncDatumEx(resultTypes[colIdx], datum)
4482+
if err != nil {
4483+
return nil, err
4484+
}
44824485
buf, err = encDatum.Encode(resultTypes[colIdx], &a, catenumpb.DatumEncoding_VALUE, buf)
44834486
if err != nil {
44844487
return nil, err

pkg/sql/distsql_plan_window.go

Lines changed: 8 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -180,7 +180,10 @@ func initBoundsFromAST(
180180
spec.Start.OffsetType = execinfrapb.DatumInfo{Encoding: catenumpb.DatumEncoding_VALUE, Type: typ}
181181
var buf []byte
182182
var a tree.DatumAlloc
183-
datum := rowenc.DatumToEncDatum(typ, dStartOffset)
183+
datum, err := rowenc.DatumToEncDatumEx(typ, dStartOffset)
184+
if err != nil {
185+
return err
186+
}
184187
buf, err = datum.Encode(typ, &a, catenumpb.DatumEncoding_VALUE, buf)
185188
if err != nil {
186189
return err
@@ -226,7 +229,10 @@ func initBoundsFromAST(
226229
spec.End.OffsetType = execinfrapb.DatumInfo{Encoding: catenumpb.DatumEncoding_VALUE, Type: typ}
227230
var buf []byte
228231
var a tree.DatumAlloc
229-
datum := rowenc.DatumToEncDatum(typ, dEndOffset)
232+
datum, err := rowenc.DatumToEncDatumEx(typ, dEndOffset)
233+
if err != nil {
234+
return err
235+
}
230236
buf, err = datum.Encode(typ, &a, catenumpb.DatumEncoding_VALUE, buf)
231237
if err != nil {
232238
return err

pkg/sql/execinfra/processorsbase.go

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -295,7 +295,10 @@ func (h *ProcOutputHelper) ProcessRow(
295295
if err != nil {
296296
return nil, false, err
297297
}
298-
h.outputRow[i] = rowenc.DatumToEncDatum(h.OutputTypes[i], datum)
298+
h.outputRow[i], err = rowenc.DatumToEncDatumEx(h.OutputTypes[i], datum)
299+
if err != nil {
300+
return nil, false, err
301+
}
299302
}
300303
} else if h.outputCols != nil {
301304
// Projection.

pkg/sql/plan_node_to_row_source.go

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -199,7 +199,11 @@ func (p *planNodeToRowSource) Next() (rowenc.EncDatumRow, *execinfrapb.ProducerM
199199

200200
for i, datum := range p.node.Values() {
201201
if datum != nil {
202-
p.row[i] = rowenc.DatumToEncDatum(p.outputTypes[i], datum)
202+
p.row[i], err = rowenc.DatumToEncDatumEx(p.outputTypes[i], datum)
203+
if err != nil {
204+
p.MoveToDraining(err)
205+
return nil, p.DrainHelper()
206+
}
203207
}
204208
}
205209
// ProcessRow here is required to deal with projections, which won't be

pkg/sql/row/fetcher.go

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1131,7 +1131,10 @@ func (rf *Fetcher) processValueSingle(
11311131
if rf.args.TraceKV {
11321132
prettyValue = value.String()
11331133
}
1134-
table.row[idx] = rowenc.DatumToEncDatum(typ, value)
1134+
table.row[idx], err = rowenc.DatumToEncDatumEx(typ, value)
1135+
if err != nil {
1136+
return "", "", err
1137+
}
11351138
return prettyKey, prettyValue, nil
11361139
}
11371140

pkg/sql/rowcontainer/disk_row_container_test.go

Lines changed: 7 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -281,17 +281,21 @@ func TestDiskRowContainer(t *testing.T) {
281281
}
282282

283283
// Check sorted order.
284-
sortedRows.getEncRow(sortedRows.scratchEncRow, numKeysRead)
284+
err = sortedRows.getEncRow(sortedRows.scratchEncRow, numKeysRead)
285+
if err != nil {
286+
t.Fatal(err)
287+
}
285288
if cmp, err := compareEncRows(
286289
ctx, types, sortedRows.scratchEncRow, row, &evalCtx, d.datumAlloc, ordering,
287290
); err != nil {
288291
t.Fatal(err)
289292
} else if cmp != 0 {
290-
sortedRows.getEncRow(sortedRows.scratchEncRow, numKeysRead)
293+
err = sortedRows.getEncRow(sortedRows.scratchEncRow, numKeysRead)
291294
t.Fatalf(
292-
"expected %s to be equal to %s",
295+
"expected %s to be equal to %s (err=%v)",
293296
row.String(types),
294297
sortedRows.scratchEncRow.String(types),
298+
err,
295299
)
296300
}
297301
numKeysRead++

pkg/sql/rowcontainer/hash_row_container.go

Lines changed: 7 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -362,8 +362,7 @@ func (i *hashMemRowBucketIterator) Next() {
362362

363363
// EncRow implements the RowIterator interface.
364364
func (i *hashMemRowBucketIterator) EncRow() (rowenc.EncDatumRow, error) {
365-
i.container.getEncRow(i.scratchEncRow, i.rowIdxs[i.curIdx])
366-
return i.scratchEncRow, nil
365+
return i.scratchEncRow, i.container.getEncRow(i.scratchEncRow, i.rowIdxs[i.curIdx])
367366
}
368367

369368
// Row implements the RowIterator interface.
@@ -452,7 +451,7 @@ func (i *hashMemRowIterator) computeKey() error {
452451
}
453452

454453
if valid {
455-
i.container.getEncRow(i.scratchEncRow, i.curIdx)
454+
err = i.container.getEncRow(i.scratchEncRow, i.curIdx)
456455
} else {
457456
if i.curIdx == 0 {
458457
// There are no rows in the container, so the key corresponding to the
@@ -464,7 +463,10 @@ func (i *hashMemRowIterator) computeKey() error {
464463
// will "simulate" the key corresponding to the non-existent row as the key
465464
// to the last existing row plus one (plus one part is done below where we
466465
// append the index of the row to curKey).
467-
i.container.getEncRow(i.scratchEncRow, i.curIdx-1)
466+
err = i.container.getEncRow(i.scratchEncRow, i.curIdx-1)
467+
}
468+
if err != nil {
469+
return err
468470
}
469471

470472
i.curKey = i.curKey[:0]
@@ -496,8 +498,7 @@ func (i *hashMemRowIterator) Next() {
496498

497499
// EncRow implements the RowIterator interface.
498500
func (i *hashMemRowIterator) EncRow() (rowenc.EncDatumRow, error) {
499-
i.container.getEncRow(i.scratchEncRow, i.curIdx)
500-
return i.scratchEncRow, nil
501+
return i.scratchEncRow, i.container.getEncRow(i.scratchEncRow, i.curIdx)
501502
}
502503

503504
// Row implements the RowIterator interface.

pkg/sql/rowcontainer/numbered_row_container.go

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -186,8 +186,7 @@ func (d *DiskBackedNumberedRowContainer) GetRow(
186186
return nil, nil
187187
}
188188
mrc := d.rc.mrc
189-
mrc.getEncRow(mrc.scratchEncRow, idx)
190-
return mrc.scratchEncRow, nil
189+
return mrc.scratchEncRow, mrc.getEncRow(mrc.scratchEncRow, idx)
191190
}
192191
return d.rowIter.getRow(ctx, idx, skip)
193192
}

pkg/sql/rowcontainer/row_container.go

Lines changed: 18 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -231,11 +231,16 @@ func (mc *MemRowContainer) Less(i, j int) bool {
231231
// getEncRow populates the given EncDatumRow with the values of the idx-th row.
232232
// The behavior is undefined if the given row is of a different width than the
233233
// rows stored in the container.
234-
func (mc *MemRowContainer) getEncRow(encRow rowenc.EncDatumRow, idx int) {
234+
func (mc *MemRowContainer) getEncRow(encRow rowenc.EncDatumRow, idx int) error {
235235
datums := mc.At(idx)
236236
for i, d := range datums {
237-
encRow[i] = rowenc.DatumToEncDatum(mc.types[i], d)
237+
var err error
238+
encRow[i], err = rowenc.DatumToEncDatumEx(mc.types[i], d)
239+
if err != nil {
240+
return err
241+
}
238242
}
243+
return nil
239244
}
240245

241246
// AddRow adds a row to the container.
@@ -344,7 +349,11 @@ func (i *memRowIterator) Next() {
344349
func (i *memRowIterator) EncRow() (rowenc.EncDatumRow, error) {
345350
datums := i.container.At(i.curIdx)
346351
for colidx, d := range datums {
347-
i.scratchEncRow[colidx] = rowenc.DatumToEncDatum(i.container.types[colidx], d)
352+
var err error
353+
i.scratchEncRow[colidx], err = rowenc.DatumToEncDatumEx(i.container.types[colidx], d)
354+
if err != nil {
355+
return nil, err
356+
}
348357
}
349358
return i.scratchEncRow, nil
350359
}
@@ -377,8 +386,7 @@ func (mc *MemRowContainer) NewFinalIterator(ctx context.Context) RowIterator {
377386

378387
// GetRow implements IndexedRowContainer.
379388
func (mc *MemRowContainer) GetRow(ctx context.Context, pos int) (eval.IndexedRow, error) {
380-
mc.getEncRow(mc.scratchEncRow, pos)
381-
return IndexedRow{Idx: pos, Row: mc.scratchEncRow}, nil
389+
return IndexedRow{Idx: pos, Row: mc.scratchEncRow}, mc.getEncRow(mc.scratchEncRow, pos)
382390
}
383391

384392
var _ RowIterator = &memRowFinalIterator{}
@@ -398,8 +406,7 @@ func (i *memRowFinalIterator) Next() {
398406

399407
// EncRow implements the RowIterator interface.
400408
func (i *memRowFinalIterator) EncRow() (rowenc.EncDatumRow, error) {
401-
i.container.getEncRow(i.scratchEncRow, 0)
402-
return i.scratchEncRow, nil
409+
return i.scratchEncRow, i.container.getEncRow(i.scratchEncRow, 0)
403410
}
404411

405412
// Row implements the RowIterator interface.
@@ -942,7 +949,10 @@ func (f *DiskBackedIndexedRowContainer) GetRow(
942949
}
943950
}
944951
mrc := f.DiskBackedRowContainer.mrc
945-
mrc.getEncRow(mrc.scratchEncRow, pos)
952+
err = mrc.getEncRow(mrc.scratchEncRow, pos)
953+
if err != nil {
954+
return nil, err
955+
}
946956
rowWithIdx = mrc.scratchEncRow
947957
row, rowIdx := rowWithIdx[:len(rowWithIdx)-1], rowWithIdx[len(rowWithIdx)-1].Datum
948958
if idx, ok := rowIdx.(*tree.DInt); ok {

0 commit comments

Comments
 (0)