From db3e8f47fe9096855e52afbd032cd51f1934926f Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Tue, 14 Sep 2021 20:34:51 -0400 Subject: [PATCH] sql: remove the interleaved logic from fetchers This commit removes the remaining bits of handling the interleaved tables from both row fetcher and cFetcher. Release note: None --- pkg/sql/colencoding/BUILD.bazel | 1 - pkg/sql/colencoding/key_encoding.go | 122 --------- pkg/sql/colfetcher/cfetcher.go | 109 +-------- pkg/sql/colfetcher/fetcherstate_string.go | 13 +- pkg/sql/row/fetcher.go | 286 ++++++++-------------- 5 files changed, 119 insertions(+), 412 deletions(-) diff --git a/pkg/sql/colencoding/BUILD.bazel b/pkg/sql/colencoding/BUILD.bazel index b7683b766d87..f5f0f4903499 100644 --- a/pkg/sql/colencoding/BUILD.bazel +++ b/pkg/sql/colencoding/BUILD.bazel @@ -11,7 +11,6 @@ go_library( deps = [ "//pkg/col/coldata", "//pkg/roachpb:with-mocks", - "//pkg/sql/catalog", "//pkg/sql/catalog/descpb", "//pkg/sql/rowenc", "//pkg/sql/sem/tree", diff --git a/pkg/sql/colencoding/key_encoding.go b/pkg/sql/colencoding/key_encoding.go index 0d8366bf4a4c..ef7179fb149c 100644 --- a/pkg/sql/colencoding/key_encoding.go +++ b/pkg/sql/colencoding/key_encoding.go @@ -16,7 +16,6 @@ import ( "github.com/cockroachdb/apd/v2" "github.com/cockroachdb/cockroach/pkg/col/coldata" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -28,123 +27,6 @@ import ( "github.com/cockroachdb/errors" ) -// DecodeIndexKeyToCols decodes an index key into the idx'th position of the -// provided slices of colexec.Vecs. The input index key must already have its -// tenant id and first table id / index id prefix removed. If matches is false, -// the key is from a different table, and the returned remainingKey indicates a -// "seek prefix": the next key that might be part of the table being searched -// for. See the analog in sqlbase/index_encoding.go. -// -// Sometimes it is necessary to determine if the value of a column is NULL even -// though the value itself is not needed. If checkAllColsForNull is true, then -// foundNull=true will be returned if any columns in the key are NULL, -// regardless of whether or not indexColIdx indicates that the column should be -// decoded. -func DecodeIndexKeyToCols( - da *rowenc.DatumAlloc, - vecs []coldata.Vec, - idx int, - desc catalog.TableDescriptor, - index catalog.Index, - indexColIdx []int, - checkAllColsForNull bool, - types []*types.T, - colDirs []descpb.IndexDescriptor_Direction, - key roachpb.Key, - invertedColIdx int, -) (remainingKey roachpb.Key, matches bool, foundNull bool, _ error) { - var decodedTableID descpb.ID - var decodedIndexID descpb.IndexID - var err error - - origKey := key - - if index.NumInterleaveAncestors() > 0 { - for i := 0; i < index.NumInterleaveAncestors(); i++ { - ancestor := index.GetInterleaveAncestor(i) - // Our input key had its first table id / index id chopped off, so - // don't try to decode those for the first ancestor. - if i != 0 { - lastKeyComponentLength := len(key) - key, decodedTableID, decodedIndexID, err = rowenc.DecodePartialTableIDIndexID(key) - if err != nil { - return nil, false, false, err - } - if decodedTableID != ancestor.TableID || decodedIndexID != ancestor.IndexID { - // We don't match. Return a key with the table ID / index ID we're - // searching for, so the caller knows what to seek to. - curPos := len(origKey) - lastKeyComponentLength - // Prevent unwanted aliasing on the origKey by setting the capacity. - key = rowenc.EncodePartialTableIDIndexID(origKey[:curPos:curPos], ancestor.TableID, ancestor.IndexID) - return key, false, false, nil - } - } - - length := int(ancestor.SharedPrefixLen) - // We don't care about whether this call to DecodeKeyVals found a null or not, because - // it is a interleaving ancestor. - var isNull bool - key, isNull, err = DecodeKeyValsToCols( - da, vecs, idx, indexColIdx[:length], checkAllColsForNull, types[:length], - colDirs[:length], nil /* unseen */, key, invertedColIdx, - ) - if err != nil { - return nil, false, false, err - } - indexColIdx, types, colDirs = indexColIdx[length:], types[length:], colDirs[length:] - foundNull = foundNull || isNull - - // Consume the interleaved sentinel. - var ok bool - key, ok = encoding.DecodeIfInterleavedSentinel(key) - if !ok { - // We're expecting an interleaved sentinel but didn't find one. Append - // one so the caller can seek to it. - curPos := len(origKey) - len(key) - // Prevent unwanted aliasing on the origKey by setting the capacity. - key = encoding.EncodeInterleavedSentinel(origKey[:curPos:curPos]) - return key, false, false, nil - } - } - - lastKeyComponentLength := len(key) - key, decodedTableID, decodedIndexID, err = rowenc.DecodePartialTableIDIndexID(key) - if err != nil { - return nil, false, false, err - } - if decodedTableID != desc.GetID() || decodedIndexID != index.GetID() { - // We don't match. Return a key with the table ID / index ID we're - // searching for, so the caller knows what to seek to. - curPos := len(origKey) - lastKeyComponentLength - // Prevent unwanted aliasing on the origKey by setting the capacity. - key = rowenc.EncodePartialTableIDIndexID(origKey[:curPos:curPos], desc.GetID(), index.GetID()) - return key, false, false, nil - } - } - - var isNull bool - key, isNull, err = DecodeKeyValsToCols( - da, vecs, idx, indexColIdx, checkAllColsForNull, types, colDirs, nil /* unseen */, key, invertedColIdx, - ) - if err != nil { - return nil, false, false, err - } - foundNull = foundNull || isNull - - // We're expecting a column family id next (a varint). If - // interleavedSentinel is actually next, then this key is for a child - // table. - lastKeyComponentLength := len(key) - if _, ok := encoding.DecodeIfInterleavedSentinel(key); ok { - curPos := len(origKey) - lastKeyComponentLength - // Prevent unwanted aliasing on the origKey by setting the capacity. - key = encoding.EncodeNullDescending(origKey[:curPos:curPos]) - return key, false, false, nil - } - - return key, true, foundNull, nil -} - // DecodeKeyValsToCols decodes the values that are part of the key, writing the // result to the idx'th slot of the input slice of colexec.Vecs. If the // directions slice is nil, the direction used will default to @@ -223,10 +105,6 @@ func decodeTableKeyToCol( vec.Nulls().SetNull(idx) return key, true, nil } - // We might have read a NULL value in the interleaved child table which - // would update the nulls vector, so we need to explicitly unset the null - // value here. - vec.Nulls().UnsetNull(idx) // Inverted columns should not be decoded, but should instead be // passed on as a DBytes datum. diff --git a/pkg/sql/colfetcher/cfetcher.go b/pkg/sql/colfetcher/cfetcher.go index b44a742dd3b0..64b5fac54560 100644 --- a/pkg/sql/colfetcher/cfetcher.go +++ b/pkg/sql/colfetcher/cfetcher.go @@ -237,10 +237,8 @@ type cFetcher struct { // firstBatchLimit. maxKeysPerRow int - // True if the index key must be decoded. - // If there is more than one table, the index key must always be decoded. - // This is only false if there are no needed columns and the (single) - // table has no interleave children. + // True if the index key must be decoded. This is only false if there are no + // needed columns. mustDecodeIndexKey bool // lockStrength represents the row-level locking mode to use when fetching rows. @@ -282,8 +280,6 @@ type cFetcher struct { rowIdx int // nextKV is the kv to process next. nextKV roachpb.KeyValue - // seekPrefix is the prefix to seek to in stateSeekPrefix. - seekPrefix roachpb.Key // limitHint is a hint as to the number of rows that the caller expects // to be returned from this fetch. It will be decremented whenever a @@ -567,14 +563,9 @@ func (rf *cFetcher) Init( } } - // - If there are interleaves, we need to read the index key in order to - // determine whether this row is actually part of the index we're scanning. - // - If there are needed columns from the index key, we need to read it. - // - // Otherwise, we can completely avoid decoding the index key. - if neededIndexCols > 0 || table.index.NumInterleavedBy() > 0 || table.index.NumInterleaveAncestors() > 0 { - rf.mustDecodeIndexKey = true - } + // If there are needed columns from the index key, we need to read it; + // otherwise, we can completely avoid decoding the index key. + rf.mustDecodeIndexKey = neededIndexCols > 0 if table.isSecondaryIndex { colIDs := table.index.CollectKeyColumnIDs() @@ -744,27 +735,13 @@ const ( // set. // 1. skip common prefix // 2. parse key (past common prefix) into row buffer, setting last row prefix buffer - // 3. interleave detected? - // - set skip prefix - // -> seekPrefix(decodeFirstKVOfRow) - // 4. parse value into row buffer. - // 5. 1-cf or secondary index? + // 3. parse value into row buffer. + // 4. 1-cf or secondary index? // -> doneRow(initFetch) // else: // -> fetchNextKVWithUnfinishedRow stateDecodeFirstKVOfRow - // stateSeekPrefix is the state of skipping all keys that sort before - // (or after, in the case of a reverse scan) a prefix. s.machine.seekPrefix - // must be set to the prefix to seek to. state[1] must be set, and seekPrefix - // will transition to that state once it finds the first key with that prefix. - // 1. fetch next kv into nextKV buffer - // 2. kv doesn't match seek prefix? - // -> seekPrefix - // else: - // -> nextState - stateSeekPrefix - // stateFetchNextKVWithUnfinishedRow is the state of getting a new key for // the current row. The machine will read a new key from the underlying // fetcher, process it, and either add the results to the current row, or @@ -775,9 +752,6 @@ const ( // 4. no? // -> finalizeRow(decodeFirstKVOfRow) // 5. skip to end of last row prefix buffer - // 6. interleave detected? - // - set skip prefix - // -> finalizeRow(seekPrefix(decodeFirstKVOfRow)) // 6. parse value into row buffer // 7. -> fetchNextKVWithUnfinishedRow stateFetchNextKVWithUnfinishedRow @@ -895,9 +869,8 @@ func (rf *cFetcher) NextBatch(ctx context.Context) (coldata.Batch, error) { log.Infof(ctx, "decoding first key %s", rf.machine.nextKV.Key) } var ( - key []byte - matches bool - err error + key []byte + err error ) indexOrds := rf.table.indexColOrdinals if rf.traceKV { @@ -908,39 +881,24 @@ func (rf *cFetcher) NextBatch(ctx context.Context) (coldata.Batch, error) { // to determine whether a KV belongs to the same row as the // previous KV or a different row. checkAllColsForNull := rf.table.isSecondaryIndex && rf.table.index.IsUnique() && rf.table.desc.NumFamilies() != 1 - key, matches, foundNull, err = colencoding.DecodeIndexKeyToCols( + key, foundNull, err = colencoding.DecodeKeyValsToCols( &rf.table.da, rf.machine.colvecs, rf.machine.rowIdx, - rf.table.desc, - rf.table.index, indexOrds, checkAllColsForNull, rf.table.keyValTypes, rf.table.indexColumnDirs, + nil, /* unseen */ rf.machine.nextKV.Key[rf.table.knownPrefixLength:], rf.table.invertedColOrdinal, ) if err != nil { return nil, err } - if !matches { - // We found an interleave. Set our skip prefix. - seekPrefix := rf.machine.nextKV.Key[:len(key)+rf.table.knownPrefixLength] - if debugState { - log.Infof(ctx, "setting seek prefix to %s", seekPrefix) - } - rf.machine.seekPrefix = seekPrefix - rf.machine.state[0] = stateSeekPrefix - rf.machine.state[1] = stateDecodeFirstKVOfRow - continue - } prefix := rf.machine.nextKV.Key[:len(rf.machine.nextKV.Key)-len(key)] rf.machine.lastRowPrefix = prefix } else { - // If mustDecodeIndexKey was false, we can't possibly have an - // interleaved row on our hands, so we can figure out our row prefix - // without parsing any keys by using GetRowPrefixLength. prefixLen, err := keys.GetRowPrefixLength(rf.machine.nextKV.Key) if err != nil { return nil, err @@ -1019,39 +977,6 @@ func (rf *cFetcher) NextBatch(ctx context.Context) (coldata.Batch, error) { // If the table has more than one column family, then the next KV // may belong to the same row as the current KV. rf.machine.state[0] = stateFetchNextKVWithUnfinishedRow - case stateSeekPrefix: - // Note: seekPrefix is only used for interleaved tables. - for { - moreKVs, kv, finalReferenceToBatch, err := rf.fetcher.NextKV(ctx, rf.mvccDecodeStrategy) - if err != nil { - return nil, rf.convertFetchError(ctx, err) - } - if debugState { - log.Infof(ctx, "found kv %s, seeking to prefix %s", kv.Key, rf.machine.seekPrefix) - } - if !moreKVs { - // We ran out of data, so ignore whatever our next state was going to - // be and emit the final batch. - rf.machine.state[1] = stateEmitLastBatch - break - } - // The order we perform the comparison in depends on whether we are - // performing a reverse scan or not. If we are performing a reverse - // scan, then we want to seek until we find a key less than seekPrefix. - var comparison int - if rf.reverse { - comparison = bytes.Compare(rf.machine.seekPrefix, kv.Key) - } else { - comparison = bytes.Compare(kv.Key, rf.machine.seekPrefix) - } - // TODO(jordan): if nextKV returns newSpan = true, set the new span - // prefix and indicate that it needs decoding. - if comparison >= 0 { - rf.setNextKV(kv, finalReferenceToBatch) - break - } - } - rf.shiftState() case stateFetchNextKVWithUnfinishedRow: moreKVs, kv, finalReferenceToBatch, err := rf.fetcher.NextKV(ctx, rf.mvccDecodeStrategy) @@ -1080,18 +1005,6 @@ func (rf *cFetcher) NextBatch(ctx context.Context) (coldata.Batch, error) { continue } - key := kv.Key[len(rf.machine.lastRowPrefix):] - _, foundInterleave := encoding.DecodeIfInterleavedSentinel(key) - - if foundInterleave { - // The key we just found isn't relevant to the current row, so finalize - // the current row, then skip all KVs with the current interleave prefix. - rf.machine.state[0] = stateFinalizeRow - rf.machine.state[1] = stateSeekPrefix - rf.machine.state[2] = stateDecodeFirstKVOfRow - continue - } - familyID, err := rf.getCurrentColumnFamilyID() if err != nil { return nil, err diff --git a/pkg/sql/colfetcher/fetcherstate_string.go b/pkg/sql/colfetcher/fetcherstate_string.go index 7ed041ee4496..63c72b198f7b 100644 --- a/pkg/sql/colfetcher/fetcherstate_string.go +++ b/pkg/sql/colfetcher/fetcherstate_string.go @@ -12,16 +12,15 @@ func _() { _ = x[stateInitFetch-1] _ = x[stateResetBatch-2] _ = x[stateDecodeFirstKVOfRow-3] - _ = x[stateSeekPrefix-4] - _ = x[stateFetchNextKVWithUnfinishedRow-5] - _ = x[stateFinalizeRow-6] - _ = x[stateEmitLastBatch-7] - _ = x[stateFinished-8] + _ = x[stateFetchNextKVWithUnfinishedRow-4] + _ = x[stateFinalizeRow-5] + _ = x[stateEmitLastBatch-6] + _ = x[stateFinished-7] } -const _fetcherState_name = "stateInvalidstateInitFetchstateResetBatchstateDecodeFirstKVOfRowstateSeekPrefixstateFetchNextKVWithUnfinishedRowstateFinalizeRowstateEmitLastBatchstateFinished" +const _fetcherState_name = "stateInvalidstateInitFetchstateResetBatchstateDecodeFirstKVOfRowstateFetchNextKVWithUnfinishedRowstateFinalizeRowstateEmitLastBatchstateFinished" -var _fetcherState_index = [...]uint8{0, 12, 26, 41, 64, 79, 112, 128, 146, 159} +var _fetcherState_index = [...]uint8{0, 12, 26, 41, 64, 97, 113, 131, 144} func (i fetcherState) String() string { if i < 0 || i >= fetcherState(len(_fetcherState_index)-1) { diff --git a/pkg/sql/row/fetcher.go b/pkg/sql/row/fetcher.go index 9e891796cb55..ce1cb38bfd15 100644 --- a/pkg/sql/row/fetcher.go +++ b/pkg/sql/row/fetcher.go @@ -208,10 +208,8 @@ type Fetcher struct { // calculate the kvBatchFetcher's firstBatchLimit. numKeysPerRow int - // True if the index key must be decoded. - // If there is more than one table, the index key must always be decoded. - // This is only false if there are no needed columns and the (single) - // table has no interleave children. + // True if the index key must be decoded. This is only false if there are no + // needed columns. mustDecodeIndexKey bool // lockStrength represents the row-level locking mode to use when fetching @@ -395,16 +393,9 @@ func (rf *Fetcher) Init( } } - // - If there is more than one table, we have to decode the index key to - // figure out which table the row belongs to. - // - If there are interleaves, we need to read the index key in order to - // determine whether this row is actually part of the index we're scanning. - // - If there are needed columns from the index key, we need to read it. - // - // Otherwise, we can completely avoid decoding the index key. - if !rf.mustDecodeIndexKey && (neededIndexCols > 0 || table.index.NumInterleavedBy() > 0 || table.index.NumInterleaveAncestors() > 0) { - rf.mustDecodeIndexKey = true - } + // If there are needed columns from the index key, we need to read it; + // otherwise, we can completely avoid decoding the index key. + rf.mustDecodeIndexKey = neededIndexCols > 0 // The number of columns we need to read from the value part of the key. // It's the total number of needed columns minus the ones we read from the @@ -664,125 +655,107 @@ func (rf *Fetcher) setNextKV(kv roachpb.KeyValue, needsCopy bool) { // NextKey retrieves the next key/value and sets kv/kvEnd. Returns whether a row // has been completed. -func (rf *Fetcher) NextKey(ctx context.Context) (rowDone bool, err error) { - var moreKVs bool +func (rf *Fetcher) NextKey(ctx context.Context) (rowDone bool, _ error) { + moreKVs, kv, finalReferenceToBatch, err := rf.kvFetcher.NextKV(ctx, rf.mvccDecodeStrategy) + if err != nil { + return false, ConvertFetchError(ctx, rf, err) + } + rf.setNextKV(kv, finalReferenceToBatch) - for { - var finalReferenceToBatch bool - var kv roachpb.KeyValue - moreKVs, kv, finalReferenceToBatch, err = rf.kvFetcher.NextKV(ctx, rf.mvccDecodeStrategy) + rf.kvEnd = !moreKVs + if rf.kvEnd { + // No more keys in the scan. + // + // NB: this assumes that the KV layer will never split a range + // between column families, which is a brittle assumption. + // See: + // https://github.com/cockroachdb/cockroach/pull/42056 + return true, nil + } + + // foundNull is set when decoding a new index key for a row finds a NULL value + // in the index key. This is used when decoding unique secondary indexes in order + // to tell whether they have extra columns appended to the key. + var foundNull bool + + // unchangedPrefix will be set to true if we can skip decoding the index key + // completely, because the last key we saw has identical prefix to the + // current key. + // + // See Init() for a detailed description of when we can get away with not + // reading the index key. + unchangedPrefix := rf.indexKey != nil && bytes.HasPrefix(rf.kv.Key, rf.indexKey) + if unchangedPrefix { + // Skip decoding! + rf.keyRemainingBytes = rf.kv.Key[len(rf.indexKey):] + } else if rf.mustDecodeIndexKey || rf.traceKV { + rf.keyRemainingBytes, moreKVs, foundNull, err = rf.ReadIndexKey(rf.kv.Key) if err != nil { - return false, ConvertFetchError(ctx, rf, err) + return false, err } - rf.setNextKV(kv, finalReferenceToBatch) - - rf.kvEnd = !moreKVs - if rf.kvEnd { - // No more keys in the scan. - // - // NB: this assumes that the KV layer will never split a range - // between column families, which is a brittle assumption. - // See: - // https://github.com/cockroachdb/cockroach/pull/42056 - return true, nil + if !moreKVs { + return false, errors.AssertionFailedf("key did not match any of the table descriptors") } - - // foundNull is set when decoding a new index key for a row finds a NULL value - // in the index key. This is used when decoding unique secondary indexes in order - // to tell whether they have extra columns appended to the key. - var foundNull bool - - // unchangedPrefix will be set to true if we can skip decoding the index key - // completely, because the last key we saw has identical prefix to the - // current key. - unchangedPrefix := rf.indexKey != nil && bytes.HasPrefix(rf.kv.Key, rf.indexKey) - if unchangedPrefix { - keySuffix := rf.kv.Key[len(rf.indexKey):] - if _, foundSentinel := encoding.DecodeIfInterleavedSentinel(keySuffix); foundSentinel { - // We found an interleaved sentinel, which means that the key we just - // found belongs to a different interleave. That means we have to go - // through with index key decoding. - unchangedPrefix = false - } else { - rf.keyRemainingBytes = keySuffix - } + } else { + // We still need to consume the key until the family + // id, so processKV can know whether we've finished a + // row or not. + prefixLen, err := keys.GetRowPrefixLength(rf.kv.Key) + if err != nil { + return false, err } - // See Init() for a detailed description of when we can get away with not - // reading the index key. - if unchangedPrefix { - // Skip decoding! - } else if rf.mustDecodeIndexKey || rf.traceKV { - rf.keyRemainingBytes, moreKVs, foundNull, err = rf.ReadIndexKey(rf.kv.Key) - if err != nil { - return false, err - } - if !moreKVs { - // The key did not match any of the table - // descriptors, which means it's interleaved - // data from some other table or index. - continue - } - } else { - // We still need to consume the key until the family - // id, so processKV can know whether we've finished a - // row or not. - prefixLen, err := keys.GetRowPrefixLength(rf.kv.Key) - if err != nil { - return false, err - } - rf.keyRemainingBytes = rf.kv.Key[prefixLen:] - } + rf.keyRemainingBytes = rf.kv.Key[prefixLen:] + } - // For unique secondary indexes, the index-key does not distinguish one row - // from the next if both rows contain identical values along with a NULL. - // Consider the keys: - // - // /test/unique_idx/NULL/0 - // /test/unique_idx/NULL/1 - // - // The index-key extracted from the above keys is /test/unique_idx/NULL. The - // trailing /0 and /1 are the primary key used to unique-ify the keys when a - // NULL is present. When a null is present in the index key, we cut off more - // of the index key so that the prefix includes the primary key columns. - // - // Note that we do not need to do this for non-unique secondary indexes because - // the extra columns in the primary key will _always_ be there, so we can decode - // them when processing the index. The difference with unique secondary indexes - // is that the extra columns are not always there, and are used to unique-ify - // the index key, rather than provide the primary key column values. - if foundNull && rf.table.isSecondaryIndex && rf.table.index.IsUnique() && len(rf.table.desc.GetFamilies()) != 1 { - for i := 0; i < rf.table.index.NumKeySuffixColumns(); i++ { - var err error - // Slice off an extra encoded column from rf.keyRemainingBytes. - rf.keyRemainingBytes, err = rowenc.SkipTableKey(rf.keyRemainingBytes) - if err != nil { - return false, err - } + // For unique secondary indexes, the index-key does not distinguish one row + // from the next if both rows contain identical values along with a NULL. + // Consider the keys: + // + // /test/unique_idx/NULL/0 + // /test/unique_idx/NULL/1 + // + // The index-key extracted from the above keys is /test/unique_idx/NULL. The + // trailing /0 and /1 are the primary key used to unique-ify the keys when a + // NULL is present. When a null is present in the index key, we cut off more + // of the index key so that the prefix includes the primary key columns. + // + // Note that we do not need to do this for non-unique secondary indexes because + // the extra columns in the primary key will _always_ be there, so we can decode + // them when processing the index. The difference with unique secondary indexes + // is that the extra columns are not always there, and are used to unique-ify + // the index key, rather than provide the primary key column values. + if foundNull && rf.table.isSecondaryIndex && rf.table.index.IsUnique() && len(rf.table.desc.GetFamilies()) != 1 { + for i := 0; i < rf.table.index.NumKeySuffixColumns(); i++ { + var err error + // Slice off an extra encoded column from rf.keyRemainingBytes. + rf.keyRemainingBytes, err = rowenc.SkipTableKey(rf.keyRemainingBytes) + if err != nil { + return false, err } } + } - switch { - case len(rf.table.desc.GetFamilies()) == 1: - // If we only have one family, we know that there is only 1 k/v pair per row. - rowDone = true - case !unchangedPrefix: - // If the prefix of the key has changed, current key is from a different - // row than the previous one. - rowDone = true - default: - rowDone = false - } - - if rf.indexKey != nil && rowDone { - // The current key belongs to a new row. Output the - // current row. - rf.indexKey = nil - return true, nil - } + switch { + case len(rf.table.desc.GetFamilies()) == 1: + // If we only have one family, we know that there is only 1 k/v pair per row. + rowDone = true + case !unchangedPrefix: + // If the prefix of the key has changed, current key is from a different + // row than the previous one. + rowDone = true + default: + rowDone = false + } - return false, nil + if rf.indexKey != nil && rowDone { + // The current key belongs to a new row. Output the + // current row. + rf.indexKey = nil + return true, nil } + + return false, nil } func (rf *Fetcher) prettyEncDatums(types []*types.T, vals []rowenc.EncDatum) string { @@ -1486,12 +1459,15 @@ func (rf *Fetcher) PartialKey(nCols int) (roachpb.Key, error) { if rf.kv.Key == nil { return nil, nil } - n, err := consumeIndexKeyWithoutTableIDIndexIDPrefix( - rf.table.index, nCols, rf.kv.Key[rf.table.knownPrefixLength:]) - if err != nil { - return nil, err + partialKeyLength := rf.table.knownPrefixLength + for consumedCols := 0; consumedCols < nCols; consumedCols++ { + l, err := encoding.PeekLength(rf.kv.Key[partialKeyLength:]) + if err != nil { + return nil, err + } + partialKeyLength += l } - return rf.kv.Key[:n+rf.table.knownPrefixLength], nil + return rf.kv.Key[:partialKeyLength], nil } // GetBytesRead returns total number of bytes read by the underlying KVFetcher. @@ -1504,61 +1480,3 @@ func (rf *Fetcher) GetBytesRead() int64 { func hasExtraCols(table *tableInfo) bool { return table.isSecondaryIndex && table.index.IsUnique() } - -// consumeIndexKeyWithoutTableIDIndexIDPrefix consumes an index key that's -// already pre-stripped of its table ID index ID prefix, up to nCols columns, -// returning the number of bytes consumed. For example, given an input key -// with values (6,7,8,9) such as /Table/60/1/6/7/#/61/1/8/9, stripping 3 columns -// from this key would eat all but the final, 4th column 9 in this example, -// producing /Table/60/1/6/7/#/61/1/8. If nCols was 2, instead, the result -// would include the trailing table ID index ID pair, since that's a more -// precise key: /Table/60/1/6/7/#/61/1. -func consumeIndexKeyWithoutTableIDIndexIDPrefix( - index catalog.Index, nCols int, key []byte, -) (int, error) { - origKeyLen := len(key) - consumedCols := 0 - for i := 0; i < index.NumInterleaveAncestors(); i++ { - ancestor := index.GetInterleaveAncestor(i) - length := int(ancestor.SharedPrefixLen) - // Skip up to length values. - for j := 0; j < length; j++ { - if consumedCols == nCols { - // We're done early, in the middle of an interleave. - return origKeyLen - len(key), nil - } - l, err := encoding.PeekLength(key) - if err != nil { - return 0, err - } - key = key[l:] - consumedCols++ - } - var ok bool - key, ok = encoding.DecodeIfInterleavedSentinel(key) - if !ok { - return 0, errors.New("unexpected lack of sentinel key") - } - - // Skip the TableID/IndexID pair for each ancestor except for the - // first, which has already been skipped in our input. - for j := 0; j < 2; j++ { - idLen, err := encoding.PeekLength(key) - if err != nil { - return 0, err - } - key = key[idLen:] - } - } - - // Decode the remaining values in the key, in the final interleave. - for ; consumedCols < nCols; consumedCols++ { - l, err := encoding.PeekLength(key) - if err != nil { - return 0, err - } - key = key[l:] - } - - return origKeyLen - len(key), nil -}