Skip to content

Commit

Permalink
colfetcher,row: decode only needed columns when tracing is enabled
Browse files Browse the repository at this point in the history
In a recent change we made it so that the cFetcher would treat all
columns present in the index as "needed" when the KV tracing is enabled.
This led to some bugs (like reading of a virtual column that doesn't
exist in the index) but is also not very meaningful since the goal of
the tracing is to show what the cFetcher is doing in without tracing.
This commit starts treating the KV tracing as the request to log all
needed operations without doing anything extra (like decoding
unnecessary columns).

A similar change is applied to the `row.Fetcher` as well.

Release note: None
  • Loading branch information
yuzefovich committed Dec 23, 2021
1 parent 4df3ad3 commit 387d847
Show file tree
Hide file tree
Showing 10 changed files with 90 additions and 74 deletions.
52 changes: 29 additions & 23 deletions pkg/sql/colfetcher/cfetcher.go
Original file line number Diff line number Diff line change
Expand Up @@ -1046,6 +1046,24 @@ func (rf *cFetcher) getDatumAt(colIdx int, rowIdx int) tree.Datum {
return res[0]
}

// writeDecodedCols writes the stringified representation of the decoded columns
// specified by colOrdinals. -1 in colOrdinals indicates that a column wasn't
// actually decoded (this is represented as "?" in the result). separator is
// inserted between each two consequent decoded column values (but not before
// the first one).
func (rf *cFetcher) writeDecodedCols(buf *strings.Builder, colOrdinals []int, separator byte) {
for i, idx := range colOrdinals {
if i > 0 {
buf.WriteByte(separator)
}
if idx != -1 {
buf.WriteString(rf.getDatumAt(idx, rf.machine.rowIdx).String())
} else {
buf.WriteByte('?')
}
}
}

// processValue processes the state machine's current value component, setting
// columns in the rowIdx'th tuple in the current batch depending on what data
// is found in the current value component.
Expand All @@ -1065,12 +1083,8 @@ func (rf *cFetcher) processValue(ctx context.Context, familyID descpb.FamilyID)
buf.WriteString(rf.table.desc.GetName())
buf.WriteByte('/')
buf.WriteString(rf.table.index.GetName())
// Note that because rf.traceKV is true, rf.table.indexColOrdinals will
// not include any -1, so idx values will all be valid.
for _, idx := range rf.table.indexColOrdinals {
buf.WriteByte('/')
buf.WriteString(rf.getDatumAt(idx, rf.machine.rowIdx).String())
}
buf.WriteByte('/')
rf.writeDecodedCols(&buf, rf.table.indexColOrdinals, '/')
prettyKey = buf.String()
}

Expand Down Expand Up @@ -1147,12 +1161,10 @@ func (rf *cFetcher) processValue(ctx context.Context, familyID descpb.FamilyID)
if err != nil {
return scrub.WrapError(scrub.SecondaryIndexKeyExtraValueDecodingError, err)
}
if rf.traceKV {
if rf.traceKV && len(table.extraValColOrdinals) > 0 {
var buf strings.Builder
for _, idx := range table.extraValColOrdinals {
buf.WriteByte('/')
buf.WriteString(rf.getDatumAt(idx, rf.machine.rowIdx).String())
}
buf.WriteByte('/')
rf.writeDecodedCols(&buf, table.extraValColOrdinals, '/')
prettyValue = buf.String()
}
}
Expand Down Expand Up @@ -1181,8 +1193,8 @@ func (rf *cFetcher) processValue(ctx context.Context, familyID descpb.FamilyID)
}

// processValueSingle processes the given value (of column
// family.DefaultColumnID), setting values in table.row accordingly. The key is
// only used for logging.
// family.DefaultColumnID), setting values in rf.machine.colvecs accordingly.
// The key is only used for logging.
func (rf *cFetcher) processValueSingle(
ctx context.Context,
table *cTableInfo,
Expand Down Expand Up @@ -1232,7 +1244,7 @@ func (rf *cFetcher) processValueSingle(
if row.DebugRowFetch {
log.Infof(ctx, "Scan %s -> [%d] (skipped)", rf.machine.nextKV.Key, colID)
}
return "", "", nil
return prettyKey, prettyValue, nil
}

func (rf *cFetcher) processValueBytes(
Expand Down Expand Up @@ -1333,18 +1345,12 @@ func (rf *cFetcher) fillNulls() error {
continue
}
if !table.cols[i].IsNullable() {
var indexColValues []string
for _, idx := range table.indexColOrdinals {
if idx != -1 {
indexColValues = append(indexColValues, rf.getDatumAt(idx, rf.machine.rowIdx).String())
} else {
indexColValues = append(indexColValues, "?")
}
}
var indexColValues strings.Builder
rf.writeDecodedCols(&indexColValues, table.indexColOrdinals, ',')
return scrub.WrapError(scrub.UnexpectedNullValueError, errors.Errorf(
"non-nullable column \"%s:%s\" with no value! Index scanned was %q with the index key columns (%s) and the values (%s)",
table.desc.GetName(), table.cols[i].GetName(), table.index.GetName(),
strings.Join(table.index.IndexDesc().KeyColumnNames, ","), strings.Join(indexColValues, ",")))
strings.Join(table.index.IndexDesc().KeyColumnNames, ","), indexColValues.String()))
}
rf.machine.colvecs.Nulls[i].SetNull(rf.machine.rowIdx)
}
Expand Down
9 changes: 3 additions & 6 deletions pkg/sql/colfetcher/cfetcher_setup.go
Original file line number Diff line number Diff line change
Expand Up @@ -180,9 +180,6 @@ func populateTableArgs(
// remapping that needs to be used for column ordinals from neededColumns.
// post is updated accordingly to refer to new ordinals of columns. The method
// also populates tableArgs.ColIdxMap.
//
// If traceKV is true, then all columns are considered as needed, and
// neededColumns is ignored.
func keepOnlyNeededColumns(
flowCtx *execinfra.FlowCtx,
tableArgs *cFetcherTableArgs,
Expand All @@ -191,9 +188,9 @@ func keepOnlyNeededColumns(
post *execinfrapb.PostProcessSpec,
helper *colexecargs.ExprHelper,
) error {
if !flowCtx.TraceKV && len(neededColumns) < len(tableArgs.cols) {
// If the tracing is not enabled and we don't need all of the available
// columns, we will prune all of the not needed columns away.
if len(neededColumns) < len(tableArgs.cols) {
// If we don't need all of the available columns, we will prune all of
// the not needed columns away.

// First, populate a set of needed columns.
var neededColumnsSet util.FastIntSet
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/exec/execbuilder/testdata/ddl
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ SELECT message FROM [SHOW KV TRACE FOR SESSION] WITH ORDINALITY
WHERE message LIKE 'fetched:%' OR message LIKE 'output row%'
ORDER BY message LIKE 'fetched:%' DESC, ordinality ASC
----
fetched: /t/foo/NULL -> /1
fetched: /t/foo/NULL -> /?
output row: [NULL]

statement ok
Expand Down
8 changes: 4 additions & 4 deletions pkg/sql/opt/exec/execbuilder/testdata/orderby
Original file line number Diff line number Diff line change
Expand Up @@ -317,8 +317,8 @@ SELECT message FROM [SHOW KV TRACE FOR SESSION] WITH ORDINALITY
WHERE message LIKE 'fetched:%' OR message LIKE 'output row%'
ORDER BY message LIKE 'fetched:%' DESC, ordinality ASC
----
fetched: /abc/ba/2/1/3 -> NULL
fetched: /abc/ba/5/4/6 -> NULL
fetched: /abc/ba/2/1/? -> NULL
fetched: /abc/ba/5/4/? -> NULL
output row: [1 2]
output row: [4 5]

Expand Down Expand Up @@ -464,8 +464,8 @@ SELECT message FROM [SHOW KV TRACE FOR SESSION] WITH ORDINALITY
WHERE message LIKE 'fetched:%' OR message LIKE 'output row%'
ORDER BY message LIKE 'fetched:%' DESC, ordinality ASC
----
fetched: /abc/bc/2/3 -> /1
fetched: /abc/bc/5/6 -> /4
fetched: /abc/bc/2/3 -> /?
fetched: /abc/bc/5/6 -> /?
output row: [2 3]
output row: [5 6]

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,7 @@ query T
SELECT message FROM [SHOW KV TRACE FOR SESSION] WHERE
message LIKE 'fetched: /t1/nonuniqueidx/%'
----
fetched: /t1/nonuniqueidx/1/1 -> NULL
fetched: /t1/nonuniqueidx/1/? -> NULL

query I
SET TRACING=on,kv,results;
Expand All @@ -72,7 +72,7 @@ query T
SELECT message FROM [SHOW KV TRACE FOR SESSION] WHERE
message LIKE 'fetched: /t1/uniqueidx/%'
----
fetched: /t1/uniqueidx/1 -> /1
fetched: /t1/uniqueidx/1 -> /?

query IIIII
SET TRACING=on,kv,results;
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/opt/exec/execbuilder/testdata/select
Original file line number Diff line number Diff line change
Expand Up @@ -839,7 +839,7 @@ SELECT message FROM [SHOW KV TRACE FOR SESSION] WITH ORDINALITY
WHERE message LIKE 'fetched:%' OR message LIKE 'output row%'
ORDER BY message LIKE 'fetched:%' DESC, ordinality ASC
----
fetched: /dt/dt_b_key/'2015-08-25' -> /'2015-08-25 04:45:45.53453'
fetched: /dt/dt_b_key/'2015-08-25' -> /?
output row: ['2015-08-25']

statement ok
Expand All @@ -850,8 +850,8 @@ SELECT message FROM [SHOW KV TRACE FOR SESSION] WITH ORDINALITY
WHERE message LIKE 'fetched:%' OR message LIKE 'output row%'
ORDER BY message LIKE 'fetched:%' DESC, ordinality ASC
----
fetched: /dt/dt_c_key/'02:45:02.234' -> /'2015-08-25 04:45:45.53453'
fetched: /dt/dt_c_key/'34:00:02' -> /'2015-08-30 03:34:45.34567'
fetched: /dt/dt_c_key/'02:45:02.234' -> /?
fetched: /dt/dt_c_key/'34:00:02' -> /?
output row: ['02:45:02.234']
output row: ['34:00:02']

Expand Down
14 changes: 7 additions & 7 deletions pkg/sql/opt/exec/execbuilder/testdata/select_index
Original file line number Diff line number Diff line change
Expand Up @@ -126,8 +126,8 @@ SELECT message FROM [SHOW KV TRACE FOR SESSION] WITH ORDINALITY
WHERE message LIKE 'fetched:%' OR message LIKE 'output row%'
ORDER BY message LIKE 'fetched:%' DESC, ordinality ASC
----
fetched: /t/bc/'three'/33/3 -> NULL
fetched: /t/bc/'two'/22/2 -> NULL
fetched: /t/bc/'three'/?/3 -> NULL
fetched: /t/bc/'two'/?/2 -> NULL
output row: [3 'three']
output row: [2 'two']

Expand Down Expand Up @@ -170,9 +170,9 @@ SELECT message FROM [SHOW KV TRACE FOR SESSION] WITH ORDINALITY
WHERE message LIKE 'fetched:%' OR message LIKE 'output row%'
ORDER BY message LIKE 'fetched:%' DESC, ordinality ASC
----
fetched: /t/a_desc/3/'three' -> NULL
fetched: /t/a_desc/2/'two' -> NULL
fetched: /t/a_desc/1/'one' -> NULL
fetched: /t/a_desc/3/? -> NULL
fetched: /t/a_desc/2/? -> NULL
fetched: /t/a_desc/1/? -> NULL
output row: [3]
output row: [2]
output row: [1]
Expand All @@ -186,8 +186,8 @@ SELECT message FROM [SHOW KV TRACE FOR SESSION] WITH ORDINALITY
WHERE message LIKE 'fetched:%' OR message LIKE 'output row%'
ORDER BY message LIKE 'fetched:%' DESC, ordinality ASC
----
fetched: /t/a_desc/3/'three' -> NULL
fetched: /t/a_desc/2/'two' -> NULL
fetched: /t/a_desc/3/? -> NULL
fetched: /t/a_desc/2/? -> NULL
output row: [3]
output row: [2]

Expand Down
40 changes: 20 additions & 20 deletions pkg/sql/opt/exec/execbuilder/testdata/select_index_vectorize_off
Original file line number Diff line number Diff line change
Expand Up @@ -35,26 +35,26 @@ SELECT regexp_replace(message, '\d\d\d\d\d+', '...PK...')
FROM [SHOW KV TRACE FOR SESSION]
WHERE message LIKE 'fetched:%'
----
fetched: /test2/test2_k_key/'030' -> /...PK...
fetched: /test2/test2_k_key/'029' -> /...PK...
fetched: /test2/test2_k_key/'028' -> /...PK...
fetched: /test2/test2_k_key/'027' -> /...PK...
fetched: /test2/test2_k_key/'026' -> /...PK...
fetched: /test2/test2_k_key/'025' -> /...PK...
fetched: /test2/test2_k_key/'024' -> /...PK...
fetched: /test2/test2_k_key/'023' -> /...PK...
fetched: /test2/test2_k_key/'022' -> /...PK...
fetched: /test2/test2_k_key/'021' -> /...PK...
fetched: /test2/test2_k_key/'020' -> /...PK...
fetched: /test2/test2_k_key/'019' -> /...PK...
fetched: /test2/test2_k_key/'018' -> /...PK...
fetched: /test2/test2_k_key/'017' -> /...PK...
fetched: /test2/test2_k_key/'016' -> /...PK...
fetched: /test2/test2_k_key/'015' -> /...PK...
fetched: /test2/test2_k_key/'014' -> /...PK...
fetched: /test2/test2_k_key/'013' -> /...PK...
fetched: /test2/test2_k_key/'012' -> /...PK...
fetched: /test2/test2_k_key/'011' -> /...PK...
fetched: /test2/test2_k_key/? -> /...PK...
fetched: /test2/test2_k_key/? -> /...PK...
fetched: /test2/test2_k_key/? -> /...PK...
fetched: /test2/test2_k_key/? -> /...PK...
fetched: /test2/test2_k_key/? -> /...PK...
fetched: /test2/test2_k_key/? -> /...PK...
fetched: /test2/test2_k_key/? -> /...PK...
fetched: /test2/test2_k_key/? -> /...PK...
fetched: /test2/test2_k_key/? -> /...PK...
fetched: /test2/test2_k_key/? -> /...PK...
fetched: /test2/test2_k_key/? -> /...PK...
fetched: /test2/test2_k_key/? -> /...PK...
fetched: /test2/test2_k_key/? -> /...PK...
fetched: /test2/test2_k_key/? -> /...PK...
fetched: /test2/test2_k_key/? -> /...PK...
fetched: /test2/test2_k_key/? -> /...PK...
fetched: /test2/test2_k_key/? -> /...PK...
fetched: /test2/test2_k_key/? -> /...PK...
fetched: /test2/test2_k_key/? -> /...PK...
fetched: /test2/test2_k_key/? -> /...PK...
fetched: /test2/test2_pkey/...PK.../k/v -> /'030'/42
fetched: /test2/test2_pkey/...PK.../k/v -> /'029'/42
fetched: /test2/test2_pkey/...PK.../k/v -> /'028'/42
Expand Down
15 changes: 15 additions & 0 deletions pkg/sql/opt/exec/execbuilder/testdata/virtual_columns
Original file line number Diff line number Diff line change
Expand Up @@ -1566,3 +1566,18 @@ vectorized: true
missing stats
table: t65343@t65343_c_idx
spans: [/'foo' COLLATE en_US - /'foo' COLLATE en_US]

# Regression test for the cFetcher trying to fetch virtual computed columns from
# a primary index (#73745).
statement ok
CREATE TABLE t73745 (
i INT PRIMARY KEY,
v INT NOT NULL AS ((i * 2)) VIRTUAL
);
INSERT INTO t73745 VALUES (1), (2), (3);

# Use kvtrace to simulate \set auto_trace=on,kv.
query T kvtrace
SELECT * FROM t73745
----
Scan /Table/61/{1-2}
14 changes: 6 additions & 8 deletions pkg/sql/row/fetcher.go
Original file line number Diff line number Diff line change
Expand Up @@ -694,7 +694,7 @@ func (rf *Fetcher) NextKey(ctx context.Context) (rowDone bool, _ error) {
if unchangedPrefix {
// Skip decoding!
rf.keyRemainingBytes = rf.kv.Key[len(rf.indexKey):]
} else if rf.mustDecodeIndexKey || rf.traceKV {
} else if rf.mustDecodeIndexKey {
rf.keyRemainingBytes, moreKVs, foundNull, err = rf.ReadIndexKey(rf.kv.Key)
if err != nil {
return false, err
Expand Down Expand Up @@ -767,14 +767,12 @@ func (rf *Fetcher) NextKey(ctx context.Context) (rowDone bool, _ error) {
func (rf *Fetcher) prettyEncDatums(types []*types.T, vals []rowenc.EncDatum) string {
var buf strings.Builder
for i, v := range vals {
buf.WriteByte('/')
if err := v.EnsureDecoded(types[i], rf.alloc); err != nil {
buf.WriteString("/{error decoding: ")
buf.WriteString(err.Error())
buf.WriteByte('}')
continue
buf.WriteByte('?')
} else {
buf.WriteString(v.Datum.String())
}
buf.WriteByte('/')
buf.WriteString(v.Datum.String())
}
return buf.String()
}
Expand Down Expand Up @@ -1008,7 +1006,7 @@ func (rf *Fetcher) processValueSingle(
return "", "", errors.Errorf("single entry value with no default column id")
}

if rf.traceKV || table.neededCols.Contains(int(colID)) {
if table.neededCols.Contains(int(colID)) {
if idx, ok := table.colIdxMap.Get(colID); ok {
if rf.traceKV {
prettyKey = fmt.Sprintf("%s/%s", prettyKey, table.desc.DeletableColumns()[idx].GetName())
Expand Down

0 comments on commit 387d847

Please sign in to comment.