-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
fetcher.go
1572 lines (1420 loc) · 52.7 KB
/
fetcher.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// Copyright 2017 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
package row
import (
"bytes"
"context"
"fmt"
"strings"
"time"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/scrub"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
)
// DebugRowFetch can be used to turn on some low-level debugging logs. We use
// this to avoid using log.V in the hot path.
const DebugRowFetch = false
type kvBatchFetcher interface {
// nextBatch returns the next batch of rows. Returns false in the first
// parameter if there are no more keys in the scan. May return either a slice
// of KeyValues or a batchResponse, numKvs pair, depending on the server
// version - both must be handled by calling code.
nextBatch(ctx context.Context) (ok bool, kvs []roachpb.KeyValue,
batchResponse []byte, origSpan roachpb.Span, err error)
GetRangesInfo() []roachpb.RangeInfo
}
type tableInfo struct {
// -- Fields initialized once --
// Used to determine whether a key retrieved belongs to the span we
// want to scan.
spans roachpb.Spans
desc *sqlbase.ImmutableTableDescriptor
index *sqlbase.IndexDescriptor
isSecondaryIndex bool
indexColumnDirs []sqlbase.IndexDescriptor_Direction
// equivSignature is an equivalence class for each unique table-index
// pair. It allows us to check if an index key belongs to a given
// table-index.
equivSignature []byte
// The table columns to use for fetching, possibly including ones currently in
// schema changes.
cols []sqlbase.ColumnDescriptor
// The set of ColumnIDs that are required.
neededCols util.FastIntSet
// The set of indexes into the cols array that are required for columns
// in the value part.
neededValueColsByIdx util.FastIntSet
// The number of needed columns from the value part of the row. Once we've
// seen this number of value columns for a particular row, we can stop
// decoding values in that row.
neededValueCols int
// Map used to get the index for columns in cols.
colIdxMap map[sqlbase.ColumnID]int
// One value per column that is part of the key; each value is a column
// index (into cols); -1 if we don't need the value for that column.
indexColIdx []int
// knownPrefixLength is the number of bytes in the index key prefix this
// Fetcher is configured for. The index key prefix is the table id, index
// id pair at the start of the key.
knownPrefixLength int
// -- Fields updated during a scan --
keyValTypes []types.T
extraTypes []types.T
keyVals []sqlbase.EncDatum
extraVals []sqlbase.EncDatum
row sqlbase.EncDatumRow
decodedRow tree.Datums
// The following fields contain MVCC metadata for each row and may be
// returned to users of Fetcher immediately after NextRow returns.
// They're not important to ordinary consumers of Fetcher that only
// concern themselves with actual SQL row data.
//
// rowLastModified is the timestamp of the last time any family in the row
// was modified in any way.
rowLastModified hlc.Timestamp
// rowIsDeleted is true when the row has been deleted. This is only
// meaningful when kv deletion tombstones are returned by the kvBatchFetcher,
// which the one used by `StartScan` (the common case) doesnt. Notably,
// changefeeds use this by providing raw kvs with tombstones unfiltered via
// `StartScanFrom`.
rowIsDeleted bool
// hasLast indicates whether there was a previously scanned k/v.
hasLast bool
// lastDatums is a buffer for the current key. It is only present when
// doing a physical check in order to verify round-trip encoding.
// It is required because Fetcher.kv is overwritten before NextRow
// returns.
lastKV roachpb.KeyValue
// lastDatums is a buffer for the previously scanned k/v datums. It is
// only present when doing a physical check in order to verify
// ordering.
lastDatums tree.Datums
}
// FetcherTableArgs are the arguments passed to Fetcher.Init
// for a given table that includes descriptors and row information.
type FetcherTableArgs struct {
// The spans of keys to return for the given table. Fetcher
// ignores keys outside these spans.
// This is irrelevant if Fetcher is initialize with only one
// table.
Spans roachpb.Spans
Desc *sqlbase.ImmutableTableDescriptor
Index *sqlbase.IndexDescriptor
ColIdxMap map[sqlbase.ColumnID]int
IsSecondaryIndex bool
Cols []sqlbase.ColumnDescriptor
// The indexes (0 to # of columns - 1) of the columns to return.
ValNeededForCol util.FastIntSet
}
// Fetcher handles fetching kvs and forming table rows for an
// arbitrary number of tables.
// Usage:
// var rf Fetcher
// err := rf.Init(..)
// // Handle err
// err := rf.StartScan(..)
// // Handle err
// for {
// res, err := rf.NextRow()
// // Handle err
// if res.row == nil {
// // Done
// break
// }
// // Process res.row
// }
type Fetcher struct {
// tables is a slice of all the tables and their descriptors for which
// rows are returned.
tables []tableInfo
// allEquivSignatures is a map used for checking if an equivalence
// signature belongs to any table or table's ancestor. It also maps the
// string representation of every table's and every table's ancestors'
// signature to the table's index in 'tables' for lookup during decoding.
// If 2+ tables share the same ancestor signature, allEquivSignatures
// will map the signature to the largest 'tables' index.
// The full signature for a given table in 'tables' will always map to
// its own index in 'tables'.
allEquivSignatures map[string]int
// reverse denotes whether or not the spans should be read in reverse
// or not when StartScan is invoked.
reverse bool
// maxKeysPerRow memoizes the maximum number of keys per row
// out of all the tables. This is used to calculate the kvBatchFetcher's
// 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.
mustDecodeIndexKey bool
// lockStr represents the row-level locking mode to use when fetching rows.
lockStr sqlbase.ScanLockingStrength
// returnRangeInfo, if set, causes the underlying kvBatchFetcher to return
// information about the ranges descriptors/leases uses in servicing the
// requests. This has some cost, so it's only enabled by DistSQL when this
// info is actually useful for correcting the plan (e.g. not for the PK-side
// of an index-join).
// If set, GetRangesInfo() can be used to retrieve the accumulated info.
returnRangeInfo bool
// traceKV indicates whether or not session tracing is enabled. It is set
// when beginning a new scan.
traceKV bool
// -- Fields updated during a scan --
kvFetcher *KVFetcher
indexKey []byte // the index key of the current row
prettyValueBuf *bytes.Buffer
valueColsFound int // how many needed cols we've found so far in the value
rowReadyTable *tableInfo // the table for which a row was fully decoded and ready for output
currentTable *tableInfo // the most recent table for which a key was decoded
keySigBuf []byte // buffer for the index key's signature
keyRestBuf []byte // buffer for the rest of the index key that is not part of the signature
// The current key/value, unless kvEnd is true.
kv roachpb.KeyValue
keyRemainingBytes []byte
kvEnd bool
// isCheck indicates whether or not we are running checks for k/v
// correctness. It is set only during SCRUB commands.
isCheck bool
// Buffered allocation of decoded datums.
alloc *sqlbase.DatumAlloc
}
// Reset resets this Fetcher, preserving the memory capacity that was used
// for the tables slice, and the slices within each of the tableInfo objects
// within tables. This permits reuse of this objects without forcing total
// reallocation of all of those slice fields.
func (rf *Fetcher) Reset() {
*rf = Fetcher{
tables: rf.tables[:0],
}
}
// Init sets up a Fetcher for a given table and index. If we are using a
// non-primary index, tables.ValNeededForCol can only refer to columns in the
// index.
func (rf *Fetcher) Init(
reverse bool,
lockStr sqlbase.ScanLockingStrength,
returnRangeInfo bool,
isCheck bool,
alloc *sqlbase.DatumAlloc,
tables ...FetcherTableArgs,
) error {
if len(tables) == 0 {
return errors.AssertionFailedf("no tables to fetch from")
}
rf.reverse = reverse
rf.lockStr = lockStr
rf.returnRangeInfo = returnRangeInfo
rf.alloc = alloc
rf.isCheck = isCheck
// We must always decode the index key if we need to distinguish between
// rows from more than one table.
nTables := len(tables)
multipleTables := nTables >= 2
rf.mustDecodeIndexKey = multipleTables
if multipleTables {
rf.allEquivSignatures = make(map[string]int, len(tables))
}
if cap(rf.tables) >= nTables {
rf.tables = rf.tables[:nTables]
} else {
rf.tables = make([]tableInfo, nTables)
}
for tableIdx, tableArgs := range tables {
oldTable := rf.tables[tableIdx]
table := tableInfo{
spans: tableArgs.Spans,
desc: tableArgs.Desc,
colIdxMap: tableArgs.ColIdxMap,
index: tableArgs.Index,
isSecondaryIndex: tableArgs.IsSecondaryIndex,
cols: tableArgs.Cols,
row: make(sqlbase.EncDatumRow, len(tableArgs.Cols)),
decodedRow: make(tree.Datums, len(tableArgs.Cols)),
// These slice fields might get re-allocated below, so reslice them from
// the old table here in case they've got enough capacity already.
indexColIdx: oldTable.indexColIdx[:0],
keyVals: oldTable.keyVals[:0],
extraVals: oldTable.extraVals[:0],
}
var err error
if multipleTables {
// We produce references to every signature's reference.
equivSignatures, err := sqlbase.TableEquivSignatures(table.desc.TableDesc(), table.index)
if err != nil {
return err
}
for i, sig := range equivSignatures {
// We always map the table's equivalence signature (last
// 'sig' in 'equivSignatures') to its tableIdx.
// This allows us to overwrite previous "ancestor
// signatures" (see below).
if i == len(equivSignatures)-1 {
rf.allEquivSignatures[string(sig)] = tableIdx
break
}
// Map each table's ancestors' signatures to -1 so
// we know during ReadIndexKey if the parsed index
// key belongs to ancestor or one of our tables.
// We must check if the signature has already been set
// since it's possible for a later 'table' to have an
// ancestor that is a previous 'table', and we do not
// want to overwrite the previous table's tableIdx.
if _, exists := rf.allEquivSignatures[string(sig)]; !exists {
rf.allEquivSignatures[string(sig)] = -1
}
}
// The last signature is the given table's equivalence signature.
table.equivSignature = equivSignatures[len(equivSignatures)-1]
}
// Scan through the entire columns map to see which columns are
// required.
for col, idx := range table.colIdxMap {
if tableArgs.ValNeededForCol.Contains(idx) {
// The idx-th column is required.
table.neededCols.Add(int(col))
}
}
table.knownPrefixLength = len(sqlbase.MakeIndexKeyPrefix(table.desc.TableDesc(), table.index.ID))
var indexColumnIDs []sqlbase.ColumnID
indexColumnIDs, table.indexColumnDirs = table.index.FullColumnIDs()
table.neededValueColsByIdx = tableArgs.ValNeededForCol.Copy()
neededIndexCols := 0
nIndexCols := len(indexColumnIDs)
if cap(table.indexColIdx) >= nIndexCols {
table.indexColIdx = table.indexColIdx[:nIndexCols]
} else {
table.indexColIdx = make([]int, nIndexCols)
}
for i, id := range indexColumnIDs {
colIdx, ok := table.colIdxMap[id]
if ok {
table.indexColIdx[i] = colIdx
if table.neededCols.Contains(int(id)) {
neededIndexCols++
table.neededValueColsByIdx.Remove(colIdx)
}
} else {
table.indexColIdx[i] = -1
if table.neededCols.Contains(int(id)) {
return errors.AssertionFailedf("needed column %d not in colIdxMap", id)
}
}
}
// In order to track #40410 more effectively, check that the contents of
// table.neededValueColsByIdx are valid.
for idx, ok := table.neededValueColsByIdx.Next(0); ok; idx, ok = table.neededValueColsByIdx.Next(idx + 1) {
if idx >= len(table.row) || idx < 0 {
return errors.AssertionFailedf(
"neededValueColsByIdx contains an invalid index. column %d requested, but table has %d columns",
idx,
len(table.row),
)
}
}
// - 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 || len(table.index.InterleavedBy) > 0 || len(table.index.Interleave.Ancestors) > 0) {
rf.mustDecodeIndexKey = true
}
// 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
// index key, except for composite columns.
table.neededValueCols = table.neededCols.Len() - neededIndexCols + len(table.index.CompositeColumnIDs)
if table.isSecondaryIndex {
for i := range table.cols {
if table.neededCols.Contains(int(table.cols[i].ID)) && !table.index.ContainsColumnID(table.cols[i].ID) {
return errors.Errorf("requested column %s not in index", table.cols[i].Name)
}
}
}
// Prepare our index key vals slice.
table.keyValTypes, err = sqlbase.GetColumnTypes(table.desc.TableDesc(), indexColumnIDs)
if err != nil {
return err
}
if cap(table.keyVals) >= nIndexCols {
table.keyVals = table.keyVals[:nIndexCols]
} else {
table.keyVals = make([]sqlbase.EncDatum, nIndexCols)
}
if hasExtraCols(&table) {
// Unique secondary indexes have a value that is the
// primary index key.
// Primary indexes only contain ascendingly-encoded
// values. If this ever changes, we'll probably have to
// figure out the directions here too.
table.extraTypes, err = sqlbase.GetColumnTypes(table.desc.TableDesc(), table.index.ExtraColumnIDs)
nExtraColumns := len(table.index.ExtraColumnIDs)
if cap(table.extraVals) >= nExtraColumns {
table.extraVals = table.extraVals[:nExtraColumns]
} else {
table.extraVals = make([]sqlbase.EncDatum, nExtraColumns)
}
if err != nil {
return err
}
}
// Keep track of the maximum keys per row to accommodate a
// limitHint when StartScan is invoked.
keysPerRow, err := table.desc.KeysPerRow(table.index.ID)
if err != nil {
return err
}
if keysPerRow > rf.maxKeysPerRow {
rf.maxKeysPerRow = keysPerRow
}
rf.tables[tableIdx] = table
}
if len(tables) == 1 {
// If there is more than one table, currentTable will be
// updated every time NextKey is invoked and rowReadyTable
// will be updated when a row is fully decoded.
rf.currentTable = &(rf.tables[0])
rf.rowReadyTable = &(rf.tables[0])
}
return nil
}
// StartScan initializes and starts the key-value scan. Can be used multiple
// times.
func (rf *Fetcher) StartScan(
ctx context.Context,
txn *kv.Txn,
spans roachpb.Spans,
limitBatches bool,
limitHint int64,
traceKV bool,
) error {
if len(spans) == 0 {
return errors.AssertionFailedf("no spans")
}
rf.traceKV = traceKV
f, err := makeKVBatchFetcher(
txn,
spans,
rf.reverse,
limitBatches,
rf.firstBatchLimit(limitHint),
rf.lockStr,
rf.returnRangeInfo,
)
if err != nil {
return err
}
return rf.StartScanFrom(ctx, &f)
}
// StartInconsistentScan initializes and starts an inconsistent scan, where each
// KV batch can be read at a different historical timestamp.
//
// The scan uses the initial timestamp, until it becomes older than
// maxTimestampAge; at this time the timestamp is bumped by the amount of time
// that has passed. See the documentation for TableReaderSpec for more
// details.
//
// Can be used multiple times.
func (rf *Fetcher) StartInconsistentScan(
ctx context.Context,
db *kv.DB,
initialTimestamp hlc.Timestamp,
maxTimestampAge time.Duration,
spans roachpb.Spans,
limitBatches bool,
limitHint int64,
traceKV bool,
) error {
if len(spans) == 0 {
return errors.AssertionFailedf("no spans")
}
txnTimestamp := initialTimestamp
txnStartTime := timeutil.Now()
if txnStartTime.Sub(txnTimestamp.GoTime()) >= maxTimestampAge {
return errors.Errorf(
"AS OF SYSTEM TIME: cannot specify timestamp older than %s for this operation",
maxTimestampAge,
)
}
txn := kv.NewTxnWithSteppingEnabled(ctx, db, 0 /* gatewayNodeID */)
txn.SetFixedTimestamp(ctx, txnTimestamp)
if log.V(1) {
log.Infof(ctx, "starting inconsistent scan at timestamp %v", txnTimestamp)
}
sendFn := func(ctx context.Context, ba roachpb.BatchRequest) (*roachpb.BatchResponse, error) {
if now := timeutil.Now(); now.Sub(txnTimestamp.GoTime()) >= maxTimestampAge {
// Time to bump the transaction. First commit the old one (should be a no-op).
if err := txn.Commit(ctx); err != nil {
return nil, err
}
// Advance the timestamp by the time that passed.
txnTimestamp = txnTimestamp.Add(now.Sub(txnStartTime).Nanoseconds(), 0 /* logical */)
txnStartTime = now
txn = kv.NewTxnWithSteppingEnabled(ctx, db, 0 /* gatewayNodeID */)
txn.SetFixedTimestamp(ctx, txnTimestamp)
if log.V(1) {
log.Infof(ctx, "bumped inconsistent scan timestamp to %v", txnTimestamp)
}
}
res, err := txn.Send(ctx, ba)
if err != nil {
return nil, err.GoError()
}
return res, nil
}
// TODO(radu): we should commit the last txn. Right now the commit is a no-op
// on read transactions, but perhaps one day it will release some resources.
rf.traceKV = traceKV
f, err := makeKVBatchFetcherWithSendFunc(
sendFunc(sendFn),
spans,
rf.reverse,
limitBatches,
rf.firstBatchLimit(limitHint),
rf.lockStr,
rf.returnRangeInfo,
)
if err != nil {
return err
}
return rf.StartScanFrom(ctx, &f)
}
func (rf *Fetcher) firstBatchLimit(limitHint int64) int64 {
if limitHint == 0 {
return 0
}
// If we have a limit hint, we limit the first batch size. Subsequent
// batches get larger to avoid making things too slow (e.g. in case we have
// a very restrictive filter and actually have to retrieve a lot of rows).
// The limitHint is a row limit, but each row could be made up of more than
// one key. We take the maximum possible keys per row out of all the table
// rows we could potentially scan over.
//
// We add an extra key to make sure we form the last row.
return limitHint*int64(rf.maxKeysPerRow) + 1
}
// StartScanFrom initializes and starts a scan from the given kvBatchFetcher. Can be
// used multiple times.
func (rf *Fetcher) StartScanFrom(ctx context.Context, f kvBatchFetcher) error {
rf.indexKey = nil
rf.kvFetcher = newKVFetcher(f)
// Retrieve the first key.
_, err := rf.NextKey(ctx)
return err
}
// 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 ok bool
for {
ok, rf.kv, _, err = rf.kvFetcher.NextKV(ctx)
if err != nil {
return false, err
}
rf.kvEnd = !ok
if rf.kvEnd {
// No more keys in the scan. We need to transition
// rf.rowReadyTable to rf.currentTable for the last
// row.
//
// 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
rf.rowReadyTable = rf.currentTable
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.
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
}
}
// See Init() for a detailed description of when we can get away with not
// reading the index key.
if unchangedPrefix {
// Skip decoding!
// We must set the rowReadyTable to the currentTable like ReadIndexKey
// would do. This will happen when we see 2 rows in a row with the same
// prefix. If the previous prefix was from a different table, then we must
// update the ready table to the current table, updating the fetcher state
// machine to recognize that the next row that it outputs will be from
// rf.currentTable, which will be set to the table of the key that was
// last sent to ReadIndexKey.
//
// TODO(jordan): this is a major (but correct) mess. The fetcher is past
// due for a refactor, now that it's (more) clear what the state machine
// it's trying to model is.
rf.rowReadyTable = rf.currentTable
} else if rf.mustDecodeIndexKey || rf.traceKV {
rf.keyRemainingBytes, ok, foundNull, err = rf.ReadIndexKey(rf.kv.Key)
if err != nil {
return false, err
}
if !ok {
// 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:]
}
// 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.currentTable.isSecondaryIndex && rf.currentTable.index.Unique && len(rf.currentTable.desc.Families) != 1 {
for _, colID := range rf.currentTable.index.ExtraColumnIDs {
colIdx := rf.currentTable.colIdxMap[colID]
var err error
// Slice off an extra encoded column from rf.keyRemainingBytes.
rf.keyRemainingBytes, err = sqlbase.SkipTableKey(
&rf.currentTable.cols[colIdx].Type,
rf.keyRemainingBytes,
// Extra columns are always stored in ascending order.
sqlbase.IndexDescriptor_ASC,
)
if err != nil {
return false, err
}
}
}
switch {
case len(rf.currentTable.desc.Families) == 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
case rf.rowReadyTable != rf.currentTable:
// For rowFetchers with more than one table, if the table changes the row
// is done.
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
}
return false, nil
}
}
func (rf *Fetcher) prettyEncDatums(types []types.T, vals []sqlbase.EncDatum) string {
var buf strings.Builder
for i, v := range vals {
if err := v.EnsureDecoded(&types[i], rf.alloc); err != nil {
buf.WriteString("error decoding: ")
buf.WriteString(err.Error())
}
buf.WriteByte('/')
buf.WriteString(v.Datum.String())
}
return buf.String()
}
// ReadIndexKey decodes an index key for a given table.
// It returns whether or not the key is for any of the tables initialized
// in Fetcher, and the remaining part of the key if it is.
// ReadIndexKey additionally returns whether or not it encountered a null while decoding.
func (rf *Fetcher) ReadIndexKey(
key roachpb.Key,
) (remaining []byte, ok bool, foundNull bool, err error) {
// If there is only one table to check keys for, there is no need
// to go through the equivalence signature checks.
if len(rf.tables) == 1 {
return sqlbase.DecodeIndexKeyWithoutTableIDIndexIDPrefix(
rf.currentTable.desc.TableDesc(),
rf.currentTable.index,
rf.currentTable.keyValTypes,
rf.currentTable.keyVals,
rf.currentTable.indexColumnDirs,
key[rf.currentTable.knownPrefixLength:],
)
}
// Make a copy of the initial key for validating whether it's within
// the table's specified spans.
initialKey := key
// key now contains the bytes in the key (if match) that are not part
// of the signature in order.
tableIdx, key, match, err := sqlbase.IndexKeyEquivSignature(key, rf.allEquivSignatures, rf.keySigBuf, rf.keyRestBuf)
if err != nil {
return nil, false, false, err
}
// The index key does not belong to our table because either:
// !match: part of the index key's signature did not match any of
// rf.allEquivSignatures.
// tableIdx == -1: index key belongs to an ancestor.
if !match || tableIdx == -1 {
return nil, false, false, nil
}
// The index key is not within our specified span of keys for the
// particular table.
// TODO(richardwu): ContainsKey checks every span within spans. We
// can check that spans is ordered (or sort it) and memoize
// the last span we've checked for each table. We can pass in this
// information to ContainsKey as a hint for which span to start
// checking first.
if !rf.tables[tableIdx].spans.ContainsKey(initialKey) {
return nil, false, false, nil
}
// Either a new table is encountered or the rowReadyTable differs from
// the currentTable (the rowReadyTable was outputted in the previous
// read). We transition the references.
if &rf.tables[tableIdx] != rf.currentTable || rf.rowReadyTable != rf.currentTable {
rf.rowReadyTable = rf.currentTable
rf.currentTable = &rf.tables[tableIdx]
// rf.rowReadyTable is nil if this is the very first key.
// We want to ensure this does not differ from rf.currentTable
// to prevent another transition.
if rf.rowReadyTable == nil {
rf.rowReadyTable = rf.currentTable
}
}
// We can simply decode all the column values we retrieved
// when processing the ind
// ex key. The column values are at the
// front of the key.
if key, foundNull, err = sqlbase.DecodeKeyVals(
rf.currentTable.keyValTypes,
rf.currentTable.keyVals,
rf.currentTable.indexColumnDirs,
key,
); err != nil {
return nil, false, false, err
}
return key, true, foundNull, nil
}
// processKV processes the given key/value, setting values in the row
// accordingly. If debugStrings is true, returns pretty printed key and value
// information in prettyKey/prettyValue (otherwise they are empty strings).
func (rf *Fetcher) processKV(
ctx context.Context, kv roachpb.KeyValue,
) (prettyKey string, prettyValue string, err error) {
table := rf.currentTable
if rf.traceKV {
prettyKey = fmt.Sprintf(
"/%s/%s%s",
table.desc.Name,
table.index.Name,
rf.prettyEncDatums(table.keyValTypes, table.keyVals),
)
}
// Either this is the first key of the fetch or the first key of a new
// row.
if rf.indexKey == nil {
// This is the first key for the row.
rf.indexKey = []byte(kv.Key[:len(kv.Key)-len(rf.keyRemainingBytes)])
// Reset the row to nil; it will get filled in with the column
// values as we decode the key-value pairs for the row.
// We only need to reset the needed columns in the value component, because
// non-needed columns are never set and key columns are unconditionally set
// below.
for idx, ok := table.neededValueColsByIdx.Next(0); ok; idx, ok = table.neededValueColsByIdx.Next(idx + 1) {
table.row[idx].UnsetDatum()
}
// Fill in the column values that are part of the index key.
for i := range table.keyVals {
if idx := table.indexColIdx[i]; idx != -1 {
table.row[idx] = table.keyVals[i]
}
}
rf.valueColsFound = 0
// Reset the MVCC metadata for the next row.
// set rowLastModified to a sentinel that's before any real timestamp.
// As kvs are iterated for this row, it keeps track of the greatest
// timestamp seen.
table.rowLastModified = hlc.Timestamp{}
// All row encodings (both before and after column families) have a
// sentinel kv (column family 0) that is always present when a row is
// present, even if that row is all NULLs. Thus, a row is deleted if and
// only if the first kv in it a tombstone (RawBytes is empty).
table.rowIsDeleted = len(kv.Value.RawBytes) == 0
}
if table.rowLastModified.Less(kv.Value.Timestamp) {
table.rowLastModified = kv.Value.Timestamp
}
if table.neededCols.Empty() {
// We don't need to decode any values.
if rf.traceKV {
prettyValue = tree.DNull.String()
}
return prettyKey, prettyValue, nil
}
// For covering secondary indexes, allow for decoding as a primary key.
if table.index.GetEncodingType(table.desc.PrimaryIndex.ID) == sqlbase.PrimaryIndexEncoding &&
len(rf.keyRemainingBytes) > 0 {
// If familyID is 0, kv.Value contains values for composite key columns.
// These columns already have a table.row value assigned above, but that value
// (obtained from the key encoding) might not be correct (e.g. for decimals,
// it might not contain the right number of trailing 0s; for collated
// strings, it is one of potentially many strings with the same collation
// key).
//
// In these cases, the correct value will be present in family 0 and the
// table.row value gets overwritten.
switch kv.Value.GetTag() {
case roachpb.ValueType_TUPLE:
// In this case, we don't need to decode the column family ID, because
// the ValueType_TUPLE encoding includes the column id with every encoded
// column value.
prettyKey, prettyValue, err = rf.processValueTuple(ctx, table, kv, prettyKey)
default:
var familyID uint64
_, familyID, err = encoding.DecodeUvarintAscending(rf.keyRemainingBytes)
if err != nil {
return "", "", scrub.WrapError(scrub.IndexKeyDecodingError, err)
}
var family *sqlbase.ColumnFamilyDescriptor
family, err = table.desc.FindFamilyByID(sqlbase.FamilyID(familyID))
if err != nil {
return "", "", scrub.WrapError(scrub.IndexKeyDecodingError, err)
}
prettyKey, prettyValue, err = rf.processValueSingle(ctx, table, family, kv, prettyKey)
}
if err != nil {
return "", "", scrub.WrapError(scrub.IndexValueDecodingError, err)
}
} else {
tag := kv.Value.GetTag()
var valueBytes []byte
switch tag {
case roachpb.ValueType_BYTES:
// If we have the ValueType_BYTES on a secondary index, then we know we
// are looking at column family 0. Column family 0 stores the extra primary
// key columns if they are present, so we decode them here.
valueBytes, err = kv.Value.GetBytes()
if err != nil {
return "", "", scrub.WrapError(scrub.IndexValueDecodingError, err)
}
if hasExtraCols(table) {
// This is a unique secondary index; decode the extra
// column values from the value.
var err error
valueBytes, _, err = sqlbase.DecodeKeyVals(
table.extraTypes,
table.extraVals,
nil,
valueBytes,
)
if err != nil {
return "", "", scrub.WrapError(scrub.SecondaryIndexKeyExtraValueDecodingError, err)
}
for i, id := range table.index.ExtraColumnIDs {
if table.neededCols.Contains(int(id)) {
table.row[table.colIdxMap[id]] = table.extraVals[i]
}
}
if rf.traceKV {
prettyValue = rf.prettyEncDatums(table.extraTypes, table.extraVals)
}
}
case roachpb.ValueType_TUPLE:
valueBytes, err = kv.Value.GetTuple()
if err != nil {
return "", "", scrub.WrapError(scrub.IndexValueDecodingError, err)
}
}
if DebugRowFetch {
if hasExtraCols(table) && tag == roachpb.ValueType_BYTES {
log.Infof(ctx, "Scan %s -> %s", kv.Key, rf.prettyEncDatums(table.extraTypes, table.extraVals))
} else {
log.Infof(ctx, "Scan %s", kv.Key)
}
}
if len(valueBytes) > 0 {
prettyKey, prettyValue, err = rf.processValueBytes(
ctx, table, kv, valueBytes, prettyKey,
)
if err != nil {
return "", "", scrub.WrapError(scrub.IndexValueDecodingError, err)
}
}
}
if rf.traceKV && prettyValue == "" {
prettyValue = tree.DNull.String()
}
return prettyKey, prettyValue, nil
}
// processValueSingle processes the given value (of column
// family.DefaultColumnID), setting values in table.row accordingly. The key is
// only used for logging.