-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
import_processor_planning.go
313 lines (281 loc) · 9.82 KB
/
import_processor_planning.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
// Copyright 2017 The Cockroach Authors.
//
// Licensed as a CockroachDB Enterprise file under the Cockroach Community
// License (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt
package importccl
import (
"context"
"math"
"sync/atomic"
"time"
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/physicalplan"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/logtags"
)
// distImport is used by IMPORT to run a DistSQL flow to ingest data by starting
// reader processes on many nodes that each read and ingest their assigned files
// and then send back a summary of what they ingested. The combined summary is
// returned.
func distImport(
ctx context.Context,
execCtx sql.JobExecContext,
job *jobs.Job,
tables map[string]*execinfrapb.ReadImportDataSpec_ImportTable,
typeDescs []*descpb.TypeDescriptor,
from []string,
format roachpb.IOFileFormat,
walltime int64,
alwaysFlushProgress bool,
) (roachpb.BulkOpSummary, error) {
ctx = logtags.AddTag(ctx, "import-distsql-ingest", nil)
dsp := execCtx.DistSQLPlanner()
evalCtx := execCtx.ExtendedEvalContext()
planCtx, sqlInstanceIDs, err := dsp.SetupAllNodesPlanning(ctx, evalCtx, execCtx.ExecCfg())
if err != nil {
return roachpb.BulkOpSummary{}, err
}
// accumulatedBulkSummary accumulates the BulkOpSummary returned from each
// processor in their progress updates. It stores stats about the amount of
// data written since the last time we update the job progress.
accumulatedBulkSummary := struct {
syncutil.Mutex
roachpb.BulkOpSummary
}{}
accumulatedBulkSummary.Lock()
accumulatedBulkSummary.BulkOpSummary = getLastImportSummary(job)
accumulatedBulkSummary.Unlock()
inputSpecs := makeImportReaderSpecs(job, tables, typeDescs, from, format, sqlInstanceIDs, walltime,
execCtx.User())
p := planCtx.NewPhysicalPlan()
// Setup a one-stage plan with one proc per input spec.
corePlacement := make([]physicalplan.ProcessorCorePlacement, len(inputSpecs))
for i := range inputSpecs {
corePlacement[i].SQLInstanceID = sqlInstanceIDs[i]
corePlacement[i].Core.ReadImport = inputSpecs[i]
}
p.AddNoInputStage(
corePlacement,
execinfrapb.PostProcessSpec{},
// The direct-ingest readers will emit a binary encoded BulkOpSummary.
[]*types.T{types.Bytes, types.Bytes},
execinfrapb.Ordering{},
)
p.PlanToStreamColMap = []int{0, 1}
dsp.FinalizePlan(planCtx, p)
importDetails := job.Progress().Details.(*jobspb.Progress_Import).Import
if importDetails.ReadProgress == nil {
// Initialize the progress metrics on the first attempt.
if err := job.FractionProgressed(ctx, nil, /* txn */
func(ctx context.Context, details jobspb.ProgressDetails) float32 {
prog := details.(*jobspb.Progress_Import).Import
prog.ReadProgress = make([]float32, len(from))
prog.ResumePos = make([]int64, len(from))
if prog.SequenceDetails == nil {
prog.SequenceDetails = make([]*jobspb.SequenceDetails, len(from))
for i := range prog.SequenceDetails {
prog.SequenceDetails[i] = &jobspb.SequenceDetails{}
}
}
return 0.0
},
); err != nil {
return roachpb.BulkOpSummary{}, err
}
}
rowProgress := make([]int64, len(from))
fractionProgress := make([]uint32, len(from))
updateJobProgress := func() error {
return job.FractionProgressed(ctx, nil, /* txn */
func(ctx context.Context, details jobspb.ProgressDetails) float32 {
var overall float32
prog := details.(*jobspb.Progress_Import).Import
for i := range rowProgress {
prog.ResumePos[i] = atomic.LoadInt64(&rowProgress[i])
}
for i := range fractionProgress {
fileProgress := math.Float32frombits(atomic.LoadUint32(&fractionProgress[i]))
prog.ReadProgress[i] = fileProgress
overall += fileProgress
}
accumulatedBulkSummary.Lock()
prog.Summary.Add(accumulatedBulkSummary.BulkOpSummary)
accumulatedBulkSummary.Reset()
accumulatedBulkSummary.Unlock()
return overall / float32(len(from))
},
)
}
metaFn := func(_ context.Context, meta *execinfrapb.ProducerMetadata) error {
if meta.BulkProcessorProgress != nil {
for i, v := range meta.BulkProcessorProgress.ResumePos {
atomic.StoreInt64(&rowProgress[i], v)
}
for i, v := range meta.BulkProcessorProgress.CompletedFraction {
atomic.StoreUint32(&fractionProgress[i], math.Float32bits(v))
}
accumulatedBulkSummary.Lock()
accumulatedBulkSummary.Add(meta.BulkProcessorProgress.BulkSummary)
accumulatedBulkSummary.Unlock()
if alwaysFlushProgress {
return updateJobProgress()
}
}
return nil
}
var res roachpb.BulkOpSummary
rowResultWriter := sql.NewCallbackResultWriter(func(ctx context.Context, row tree.Datums) error {
var counts roachpb.BulkOpSummary
if err := protoutil.Unmarshal([]byte(*row[0].(*tree.DBytes)), &counts); err != nil {
return err
}
res.Add(counts)
return nil
})
if evalCtx.Codec.ForSystemTenant() {
if err := presplitTableBoundaries(ctx, execCtx.ExecCfg(), tables); err != nil {
return roachpb.BulkOpSummary{}, err
}
}
recv := sql.MakeDistSQLReceiver(
ctx,
sql.NewMetadataCallbackWriter(rowResultWriter, metaFn),
tree.Rows,
nil, /* rangeCache */
nil, /* txn - the flow does not read or write the database */
nil, /* clockUpdater */
evalCtx.Tracing,
evalCtx.ExecCfg.ContentionRegistry,
nil, /* testingPushCallback */
)
defer recv.Release()
stopProgress := make(chan struct{})
g := ctxgroup.WithContext(ctx)
g.GoCtx(func(ctx context.Context) error {
tick := time.NewTicker(time.Second * 10)
defer tick.Stop()
done := ctx.Done()
for {
select {
case <-stopProgress:
return nil
case <-done:
return ctx.Err()
case <-tick.C:
if err := updateJobProgress(); err != nil {
return err
}
}
}
})
g.GoCtx(func(ctx context.Context) error {
defer close(stopProgress)
// Copy the evalCtx, as dsp.Run() might change it.
evalCtxCopy := *evalCtx
dsp.Run(planCtx, nil, p, recv, &evalCtxCopy, nil /* finishedSetupFn */)()
return rowResultWriter.Err()
})
if err := g.Wait(); err != nil {
return roachpb.BulkOpSummary{}, err
}
return res, nil
}
func getLastImportSummary(job *jobs.Job) roachpb.BulkOpSummary {
progress := job.Progress()
importProgress := progress.GetImport()
return importProgress.Summary
}
func makeImportReaderSpecs(
job *jobs.Job,
tables map[string]*execinfrapb.ReadImportDataSpec_ImportTable,
typeDescs []*descpb.TypeDescriptor,
from []string,
format roachpb.IOFileFormat,
sqlInstanceIDs []base.SQLInstanceID,
walltime int64,
user security.SQLUsername,
) []*execinfrapb.ReadImportDataSpec {
details := job.Details().(jobspb.ImportDetails)
// For each input file, assign it to a node.
inputSpecs := make([]*execinfrapb.ReadImportDataSpec, 0, len(sqlInstanceIDs))
progress := job.Progress()
importProgress := progress.GetImport()
for i, input := range from {
// Round robin assign CSV files to sqlInstanceIDs. Files 0 through len(sqlInstanceIDs)-1
// creates the spec. Future files just add themselves to the Uris.
if i < len(sqlInstanceIDs) {
spec := &execinfrapb.ReadImportDataSpec{
Tables: tables,
Types: typeDescs,
Format: format,
Progress: execinfrapb.JobProgress{
JobID: job.ID(),
Slot: int32(i),
},
WalltimeNanos: walltime,
Uri: make(map[int32]string),
ResumePos: make(map[int32]int64),
UserProto: user.EncodeProto(),
DatabasePrimaryRegion: details.DatabasePrimaryRegion,
InitialSplits: int32(len(sqlInstanceIDs)),
}
inputSpecs = append(inputSpecs, spec)
}
n := i % len(sqlInstanceIDs)
inputSpecs[n].Uri[int32(i)] = input
if importProgress.ResumePos != nil {
inputSpecs[n].ResumePos[int32(i)] = importProgress.ResumePos[int32(i)]
}
}
for i := range inputSpecs {
// TODO(mjibson): using the actual file sizes here would improve progress
// accuracy.
inputSpecs[i].Progress.Contribution = float32(len(inputSpecs[i].Uri)) / float32(len(from))
}
return inputSpecs
}
func presplitTableBoundaries(
ctx context.Context,
cfg *sql.ExecutorConfig,
tables map[string]*execinfrapb.ReadImportDataSpec_ImportTable,
) error {
var span *tracing.Span
ctx, span = tracing.ChildSpan(ctx, "import-pre-splitting-table-boundaries")
defer span.Finish()
expirationTime := cfg.DB.Clock().Now().Add(time.Hour.Nanoseconds(), 0)
for _, tbl := range tables {
// TODO(ajwerner): Consider passing in the wrapped descriptors.
tblDesc := tabledesc.NewBuilder(tbl.Desc).BuildImmutableTable()
for _, span := range tblDesc.AllIndexSpans(cfg.Codec) {
if err := cfg.DB.AdminSplit(ctx, span.Key, expirationTime); err != nil {
return err
}
log.VEventf(ctx, 1, "scattering index range %s", span.Key)
scatterReq := &roachpb.AdminScatterRequest{
RequestHeader: roachpb.RequestHeaderFromSpan(span),
}
if _, pErr := kv.SendWrapped(ctx, cfg.DB.NonTransactionalSender(), scatterReq); pErr != nil {
log.Errorf(ctx, "failed to scatter span %s: %s", span.Key, pErr)
}
}
}
return nil
}