Skip to content

Commit

Permalink
Merge #38938
Browse files Browse the repository at this point in the history
38938: exec: Add informative k/v tracing to cfetcher. r=jordanlewis a=rohany

This PR adds K/V tracing similar to that of the distsql fetcher to the cfetcher used by the vectorized engine.

For reviewing purposes, please look at the latest commit. This PR depends on a helper function added in the builtin functions support PR which has not landed yet.

Co-authored-by: Rohan Yadav <rohany@alumni.cmu.edu>
  • Loading branch information
craig[bot] and rohany committed Jul 17, 2019
2 parents e62629b + ba58397 commit 6092913
Showing 1 changed file with 33 additions and 17 deletions.
50 changes: 33 additions & 17 deletions pkg/sql/row/cfetcher.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/colencoding"
"github.com/cockroachdb/cockroach/pkg/sql/exec"
"github.com/cockroachdb/cockroach/pkg/sql/exec/coldata"
"github.com/cockroachdb/cockroach/pkg/sql/exec/types"
"github.com/cockroachdb/cockroach/pkg/sql/exec/types/conv"
Expand Down Expand Up @@ -90,6 +91,8 @@ type cTableInfo struct {

keyValTypes []semtypes.T
extraTypes []semtypes.T

da sqlbase.DatumAlloc
}

// colIdxMap is a "map" that contains the ordinal in cols for each ColumnID
Expand Down Expand Up @@ -505,7 +508,7 @@ const (
// state[1] must be set, and stateFinalizeRow will transition to that state
// once it finishes finalizing the row.
// 1. fill missing nulls
// 2. bump rowIDX
// 2. bump rowIdx
// -> nextState and optionally return if row-by-row or batch full
stateFinalizeRow

Expand Down Expand Up @@ -623,9 +626,13 @@ func (rf *CFetcher) NextBatch(ctx context.Context) (coldata.Batch, error) {
}
rf.machine.remainingValueColsByIdx.CopyFrom(rf.table.neededValueColsByIdx)
// Process the current KV's value component.
if _, _, err := rf.processValue(ctx, familyID); err != nil {
prettyKey, prettyVal, err := rf.processValue(ctx, familyID)
if err != nil {
return nil, err
}
if rf.traceKV {
log.VEventf(ctx, 2, "fetched: %s -> %s", prettyKey, prettyVal)
}
if rf.table.isSecondaryIndex || len(rf.table.desc.Families) == 1 {
rf.machine.state[0] = stateFinalizeRow
rf.machine.state[1] = stateInitFetch
Expand Down Expand Up @@ -701,9 +708,13 @@ func (rf *CFetcher) NextBatch(ctx context.Context) (coldata.Batch, error) {
}

// Process the current KV's value component.
if _, _, err := rf.processValue(ctx, familyID); err != nil {
prettyKey, prettyVal, err := rf.processValue(ctx, familyID)
if err != nil {
return nil, err
}
if rf.traceKV {
log.VEventf(ctx, 2, "fetched: %s -> %s", prettyKey, prettyVal)
}

if familyID == rf.table.maxColumnFamilyID {
// We know the row can't have any more keys, so finalize the row.
Expand Down Expand Up @@ -766,14 +777,21 @@ func (rf *CFetcher) processValue(
table := &rf.table

if rf.traceKV {
prettyKey = fmt.Sprintf(
"/%s/%s?",
table.desc.Name,
table.index.Name,
// TODO(jordan): handle this case. Can pull out values from the column
// slices.
//rf.prettyEncDatums(table.keyValTypes, table.keyVals),
)
var buf strings.Builder
buf.WriteByte('/')
buf.WriteString(rf.table.desc.Name)
buf.WriteByte('/')
buf.WriteString(rf.table.index.Name)
for _, idx := range rf.table.indexColOrdinals {
buf.WriteByte('/')
if idx != -1 {
dVal := exec.PhysicalTypeColElemToDatum(rf.machine.colvecs[idx], rf.machine.rowIdx, rf.table.da, rf.table.keyValTypes[idx])
buf.WriteString(fmt.Sprintf("%v", dVal.String()))
} else {
buf.WriteByte('?')
}
}
prettyKey = buf.String()
}

if len(table.neededColsList) == 0 {
Expand Down Expand Up @@ -874,6 +892,7 @@ func (rf *CFetcher) processValueSingle(
family *sqlbase.ColumnFamilyDescriptor,
prettyKeyPrefix string,
) (prettyKey string, prettyValue string, err error) {
prettyKey = prettyKeyPrefix
// If this is the row sentinel (in the legacy pre-family format),
// a value is not expected, so we're done.
if family.ID == 0 {
Expand Down Expand Up @@ -914,9 +933,7 @@ func (rf *CFetcher) processValueSingle(
rf.machine.remainingValueColsByIdx.Remove(idx)

if rf.traceKV {
// TODO(jordan): handle this case by reaching into the colvecs array and
// pulling out a pretty value.
prettyValue = "?"
prettyValue = exec.PhysicalTypeColElemToDatum(rf.machine.colvecs[idx], rf.machine.rowIdx, rf.table.da, *typ).String()
}
if debugRowFetch {
log.Infof(ctx, "Scan %s -> %v", rf.machine.nextKV.Key, "?")
Expand Down Expand Up @@ -1006,7 +1023,7 @@ func (rf *CFetcher) processValueBytes(
}
rf.machine.remainingValueColsByIdx.Remove(idx)
if rf.traceKV {
if _, err := fmt.Fprintf(rf.machine.prettyValueBuf, "/?"); err != nil {
if _, err := fmt.Fprintf(rf.machine.prettyValueBuf, "/%v", exec.PhysicalTypeColElemToDatum(vec, rf.machine.rowIdx, rf.table.da, *valTyp).String()); err != nil {
return "", "", err
}
}
Expand Down Expand Up @@ -1040,8 +1057,7 @@ func (rf *CFetcher) fillNulls() error {
var indexColValues []string
for _, idx := range table.indexColOrdinals {
if idx != -1 {
indexColValues = append(indexColValues,
rf.machine.colvecs[idx].PrettyValueAt(rf.machine.rowIdx, rf.table.typs[idx]))
indexColValues = append(indexColValues, exec.PhysicalTypeColElemToDatum(rf.machine.colvecs[idx], rf.machine.rowIdx, rf.table.da, rf.table.keyValTypes[idx]).String())
} else {
indexColValues = append(indexColValues, "?")
}
Expand Down

0 comments on commit 6092913

Please sign in to comment.