-
Notifications
You must be signed in to change notification settings - Fork 3.9k
/
Copy pathtablereader.go
431 lines (387 loc) · 13.5 KB
/
tablereader.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
// Copyright 2016 The Cockroach Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
// implied. See the License for the specific language governing
// permissions and limitations under the License.
package distsqlrun
import (
"bytes"
"context"
"sync"
opentracing "github.com/opentracing/opentracing-go"
"github.com/pkg/errors"
"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/util"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
)
// ScrubTypes is the schema for TableReaders that are doing a SCRUB
// check. This schema is what TableReader output streams are overrided
// to for check. The column types correspond to:
// - Error type.
// - Primary key as a string, if it was obtainable.
// - JSON of all decoded column values.
//
// TODO(joey): If we want a way find the key for the error, we will need
// additional data such as the key bytes and the table descriptor ID.
// Repair won't be possible without this.
var ScrubTypes = []sqlbase.ColumnType{
{SemanticType: sqlbase.ColumnType_STRING},
{SemanticType: sqlbase.ColumnType_STRING},
{SemanticType: sqlbase.ColumnType_JSON},
}
// tableReader is the start of a computation flow; it performs KV operations to
// retrieve rows for a table, runs a filter expression, and passes rows with the
// desired column values to an output RowReceiver.
// See docs/RFCS/distributed_sql.md
type tableReader struct {
processorBase
// ctx and span contain the tracing state while the processor is active
// (i.e. hasn't been closed). Initialized using flowCtx.ctx (which should not
// be otherwise used).
ctx context.Context
span opentracing.Span
flowCtx *FlowCtx
tableDesc sqlbase.TableDescriptor
spans roachpb.Spans
limitHint int64
fetcher sqlbase.MultiRowFetcher
alloc sqlbase.DatumAlloc
started bool
halfClosed bool
isCheck bool
// fetcherResultToColIdx maps RowFetcher results to the column index in
// the TableDescriptor. This is only initialized and used during scrub
// physical checks.
fetcherResultToColIdx []int
// indexIdx refers to the index being scanned. This is only used
// during scrub physical checks.
indexIdx int
// consumerStatus is used by the RowSource interface to signal that the
// consumer is done accepting rows or is no longer accepting data.
consumerStatus ConsumerStatus
}
var _ Processor = &tableReader{}
var _ RowSource = &tableReader{}
// newTableReader creates a tableReader.
func newTableReader(
flowCtx *FlowCtx, spec *TableReaderSpec, post *PostProcessSpec, output RowReceiver,
) (*tableReader, error) {
if flowCtx.nodeID == 0 {
return nil, errors.Errorf("attempting to create a tableReader with uninitialized NodeID")
}
tr := &tableReader{
flowCtx: flowCtx,
tableDesc: spec.Table,
isCheck: spec.IsCheck,
indexIdx: int(spec.IndexIdx),
}
tr.limitHint = limitHint(spec.LimitHint, post)
types := make([]sqlbase.ColumnType, len(spec.Table.Columns))
for i := range types {
types[i] = spec.Table.Columns[i].Type
}
// IsCheck is only enabled while running a scrub physical check on an
// index. When running the check, the output schema of the table
// reader is instead ScrubTypes.
if spec.IsCheck {
types = ScrubTypes
}
if err := tr.init(post, types, flowCtx, output); err != nil {
return nil, err
}
neededColumns := tr.out.neededColumns()
// If we are doing a scrub physical check, neededColumns needs to be
// changed to be all columns available in the index we are scanning.
// This is because the emitted schema is ScrubTypes so neededColumns
// does not correctly represent the data being scanned.
if tr.isCheck {
if spec.IndexIdx == 0 {
neededColumns = util.FastIntSet{}
neededColumns.AddRange(0, len(spec.Table.Columns)-1)
for i := range spec.Table.Columns {
tr.fetcherResultToColIdx = append(tr.fetcherResultToColIdx, i)
}
} else {
colIDToIdx := make(map[sqlbase.ColumnID]int, len(spec.Table.Columns))
for i := range spec.Table.Columns {
colIDToIdx[spec.Table.Columns[i].ID] = i
}
neededColumns = util.FastIntSet{}
for _, id := range spec.Table.Indexes[spec.IndexIdx-1].ColumnIDs {
neededColumns.Add(colIDToIdx[id])
}
for _, id := range spec.Table.Indexes[spec.IndexIdx-1].ExtraColumnIDs {
neededColumns.Add(colIDToIdx[id])
}
for _, id := range spec.Table.Indexes[spec.IndexIdx-1].StoreColumnIDs {
neededColumns.Add(colIDToIdx[id])
}
}
}
if _, _, err := initRowFetcher(
&tr.fetcher, &tr.tableDesc, int(spec.IndexIdx), spec.Reverse,
neededColumns, spec.IsCheck, &tr.alloc,
); err != nil {
return nil, err
}
tr.spans = make(roachpb.Spans, len(spec.Spans))
for i, s := range spec.Spans {
tr.spans[i] = s.Span
}
return tr, nil
}
func initRowFetcher(
fetcher *sqlbase.MultiRowFetcher,
desc *sqlbase.TableDescriptor,
indexIdx int,
reverseScan bool,
valNeededForCol util.FastIntSet,
isCheck bool,
alloc *sqlbase.DatumAlloc,
) (index *sqlbase.IndexDescriptor, isSecondaryIndex bool, err error) {
index, isSecondaryIndex, err = desc.FindIndexByIndexIdx(indexIdx)
if err != nil {
return nil, false, err
}
colIdxMap := make(map[sqlbase.ColumnID]int, len(desc.Columns))
for i, c := range desc.Columns {
colIdxMap[c.ID] = i
}
tableArgs := sqlbase.MultiRowFetcherTableArgs{
Desc: desc,
Index: index,
ColIdxMap: colIdxMap,
IsSecondaryIndex: isSecondaryIndex,
Cols: desc.Columns,
ValNeededForCol: valNeededForCol,
}
if err := fetcher.Init(
reverseScan, true /* returnRangeInfo */, isCheck, alloc, tableArgs,
); err != nil {
return nil, false, err
}
return index, isSecondaryIndex, nil
}
// Run is part of the processor interface.
func (tr *tableReader) Run(ctx context.Context, wg *sync.WaitGroup) {
if tr.out.output == nil {
panic("tableReader output not initialized for emitting rows")
}
if ctx != tr.flowCtx.ctx {
panic("unexpected context")
}
Run(ctx, tr, tr.out.output)
if wg != nil {
wg.Done()
}
}
// generateScrubErrorRow will create an EncDatumRow describing a
// physical check error encountered when scanning table data. The schema
// of the EncDatumRow is the ScrubTypes constant.
func (tr *tableReader) generateScrubErrorRow(
row sqlbase.EncDatumRow, scrubErr *scrub.Error,
) (sqlbase.EncDatumRow, error) {
details := make(map[string]interface{})
var index *sqlbase.IndexDescriptor
if tr.indexIdx == 0 {
index = &tr.tableDesc.PrimaryIndex
} else {
index = &tr.tableDesc.Indexes[tr.indexIdx-1]
}
// Collect all the row values into JSON
rowDetails := make(map[string]interface{})
for i, colIdx := range tr.fetcherResultToColIdx {
col := tr.tableDesc.Columns[colIdx]
// TODO(joey): We should maybe try to get the underlying type.
rowDetails[col.Name] = row[i].String(&col.Type)
}
details["row_data"] = rowDetails
details["index_name"] = index.Name
details["error_message"] = scrub.UnwrapScrubError(error(scrubErr)).Error()
detailsJSON, err := tree.MakeDJSON(details)
if err != nil {
return nil, err
}
primaryKeyValues := tr.prettyPrimaryKeyValues(row, &tr.tableDesc)
return sqlbase.EncDatumRow{
sqlbase.DatumToEncDatum(
ScrubTypes[0],
tree.NewDString(scrubErr.Code),
),
sqlbase.DatumToEncDatum(
ScrubTypes[1],
tree.NewDString(primaryKeyValues),
),
sqlbase.DatumToEncDatum(
ScrubTypes[2],
detailsJSON,
),
}, nil
}
func (tr *tableReader) prettyPrimaryKeyValues(
row sqlbase.EncDatumRow, table *sqlbase.TableDescriptor,
) string {
colIdxMap := make(map[sqlbase.ColumnID]int, len(table.Columns))
for i, c := range table.Columns {
colIdxMap[c.ID] = i
}
colIDToRowIdxMap := make(map[sqlbase.ColumnID]int, len(table.Columns))
for rowIdx, colIdx := range tr.fetcherResultToColIdx {
colIDToRowIdxMap[tr.tableDesc.Columns[colIdx].ID] = rowIdx
}
var primaryKeyValues bytes.Buffer
primaryKeyValues.WriteByte('(')
for i, id := range table.PrimaryIndex.ColumnIDs {
if i > 0 {
primaryKeyValues.WriteByte(',')
}
primaryKeyValues.WriteString(
row[colIDToRowIdxMap[id]].String(&table.Columns[colIdxMap[id]].Type))
}
primaryKeyValues.WriteByte(')')
return primaryKeyValues.String()
}
// halfClose closes the tableReader for returning rows but allows it to return
// producer metadata.
func (tr *tableReader) halfClose() {
tr.halfClosed = true
// This prevents Next() from returning more rows.
tr.out.rowIdx = tr.out.maxRowIdx
}
// close the tableReader and finish any tracing. Any subsequent calls to Next()
// will return empty data.
func (tr *tableReader) close() {
tr.halfClose()
if tr.ctx != nil {
log.VEventf(tr.ctx, 1, "exiting")
tracing.FinishSpan(tr.span)
tr.ctx, tr.span = nil, nil
}
}
// producerMeta constructs the ProducerMetadata after consumption of rows has
// terminated, either due to being indicated by the consumer, or because the
// tableReader ran out of rows or encountered an error. It is ok for err to be
// nil indicating that we're done producing rows even though no error occurred.
func (tr *tableReader) producerMeta(err error) ProducerMetadata {
var meta ProducerMetadata
if tr.ctx != nil {
meta = ProducerMetadata{Err: err}
if err != nil {
// Fully close as soon as an error is sent.
tr.close()
} else if !tr.halfClosed {
// TODO(peter): TestTrace is failing for some reason. If I send the trace
// data first and then the misplanned ranges the test passes. If I send
// both at the same time the test passes. Either the test is broken and
// needs to be fixed or something isn't draining all of the metadata.
// No error, go through the half-close dance where we return the
// misplanned ranges metadata first.
meta.Ranges = misplannedRanges(tr.ctx, tr.fetcher.GetRangeInfo(), tr.flowCtx.nodeID)
tr.halfClose()
} else {
// The second part of the no error close case: return the trace data.
meta.TraceData = getTraceData(tr.ctx)
tr.close()
}
}
return meta
}
// Types implements the RowSource interface.
func (tr *tableReader) Types() []sqlbase.ColumnType {
return tr.out.outputTypes
}
// Next implements the RowSource interface.
func (tr *tableReader) Next() (sqlbase.EncDatumRow, ProducerMetadata) {
if !tr.started {
tr.started = true
if tr.flowCtx.txn == nil {
log.Fatalf(tr.ctx, "tableReader outside of txn")
}
tr.ctx = log.WithLogTagInt(tr.flowCtx.ctx, "TableReader", int(tr.tableDesc.ID))
tr.ctx, tr.span = processorSpan(tr.ctx, "table reader")
log.VEventf(tr.ctx, 1, "starting")
// TODO(radu,andrei,knz): set the traceKV flag when requested by the session.
if err := tr.fetcher.StartScan(
tr.ctx, tr.flowCtx.txn, tr.spans,
true /* limit batches */, tr.limitHint, false, /* traceKV */
); err != nil {
log.Errorf(tr.ctx, "scan error: %s", err)
return nil, tr.producerMeta(err)
}
}
if tr.out.rowIdx == tr.out.maxRowIdx || tr.consumerStatus != NeedMoreRows {
return nil, tr.producerMeta(nil /* err */)
}
for {
var row sqlbase.EncDatumRow
var err error
if !tr.isCheck {
row, _, _, err = tr.fetcher.NextRow(tr.ctx)
} else {
// If we are running a scrub physical check, we use a specialized
// procedure that runs additional checks while fetching the row
// data.
row, err = tr.fetcher.NextRowWithErrors(tr.ctx)
// There are four cases that can happen after NextRowWithErrors:
// 1) We encounter a ScrubError. We do not propagate the error up,
// but instead generate and emit a row for the final results.
// 2) No errors were found. We simply continue scanning the data
// and discard the row values, as they are not needed for any
// results.
// 3) A non-scrub error was encountered. This was not considered a
// physical data error, and so we propagate this to the user
// immediately.
// 4) There was no error or row data. This signals that there is
// no more data to scan.
//
// NB: Cases 3 and 4 are handled further below, in the standard
// table scanning code path.
if v, ok := err.(*scrub.Error); ok {
row, err = tr.generateScrubErrorRow(row, v)
} else if err == nil && row != nil {
continue
}
}
if row == nil || err != nil {
// This was the last-row or an error was encountered, annotate the
// metadata with misplanned ranges and trace data.
return nil, tr.producerMeta(scrub.UnwrapScrubError(err))
}
outRow, status, err := tr.out.ProcessRow(tr.ctx, row)
if outRow == nil && err == nil && status == NeedMoreRows {
continue
}
if err != nil {
return nil, tr.producerMeta(err)
}
return outRow, ProducerMetadata{}
}
}
// ConsumerDone implements the RowSource interface.
func (tr *tableReader) ConsumerDone() {
if tr.consumerStatus != NeedMoreRows {
log.Fatalf(context.Background(), "tableReader already done or closed: %d", tr.consumerStatus)
}
tr.consumerStatus = DrainRequested
}
// ConsumerClosed implements the RowSource interface.
func (tr *tableReader) ConsumerClosed() {
if tr.consumerStatus == ConsumerClosed {
log.Fatalf(context.Background(), "tableReader already closed")
}
tr.consumerStatus = ConsumerClosed
// The consumer is done, Next() will not be called again.
tr.close()
}