-
Notifications
You must be signed in to change notification settings - Fork 3.9k
/
Copy pathstream_ingestion_job.go
542 lines (487 loc) · 19.6 KB
/
stream_ingestion_job.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
// Copyright 2020 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 streamingest
import (
"context"
"fmt"
"time"
"github.com/cockroachdb/cockroach/pkg/ccl/streamingccl"
"github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/streamclient"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/repstream/streampb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/retry"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/errors"
)
// completeStreamIngestion terminates the stream as of specified time.
func completeStreamIngestion(
ctx context.Context,
evalCtx *eval.Context,
txn *kv.Txn,
ingestionJobID jobspb.JobID,
cutoverTimestamp hlc.Timestamp,
) error {
jobRegistry := evalCtx.Planner.ExecutorConfig().(*sql.ExecutorConfig).JobRegistry
return jobRegistry.UpdateJobWithTxn(ctx, ingestionJobID, txn, false, /* useReadLock */
func(txn *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error {
// TODO(adityamaru): This should change in the future, a user should be
// allowed to correct their cutover time if the process of reverting the job
// has not started.
if jobCutoverTime := md.Progress.GetStreamIngest().CutoverTime; !jobCutoverTime.IsEmpty() {
return errors.Newf("cutover timestamp already set to %s, "+
"job %d is in the process of cutting over", jobCutoverTime.String(), ingestionJobID)
}
// Update the sentinel being polled by the stream ingestion job to
// check if a complete has been signaled.
md.Progress.GetStreamIngest().CutoverTime = cutoverTimestamp
ju.UpdateProgress(md.Progress)
return nil
})
}
func getStreamIngestionStats(
ctx context.Context, evalCtx *eval.Context, txn *kv.Txn, ingestionJobID jobspb.JobID,
) (*streampb.StreamIngestionStats, error) {
registry := evalCtx.Planner.ExecutorConfig().(*sql.ExecutorConfig).JobRegistry
j, err := registry.LoadJobWithTxn(ctx, ingestionJobID, txn)
if err != nil {
return nil, err
}
details, ok := j.Details().(jobspb.StreamIngestionDetails)
if !ok {
return nil, errors.Errorf("job with id %d is not a stream ingestion job", ingestionJobID)
}
progress := j.Progress()
stats := &streampb.StreamIngestionStats{
IngestionDetails: &details,
IngestionProgress: progress.GetStreamIngest(),
}
if highwater := progress.GetHighWater(); highwater != nil && !highwater.IsEmpty() {
lagInfo := &streampb.StreamIngestionStats_ReplicationLagInfo{
MinIngestedTimestamp: *highwater,
}
lagInfo.EarliestCheckpointedTimestamp = hlc.MaxTimestamp
lagInfo.LatestCheckpointedTimestamp = hlc.MinTimestamp
// TODO(casper): track spans that the slowest partition is associated
for _, resolvedSpan := range progress.GetStreamIngest().Checkpoint.ResolvedSpans {
if resolvedSpan.Timestamp.Less(lagInfo.EarliestCheckpointedTimestamp) {
lagInfo.EarliestCheckpointedTimestamp = resolvedSpan.Timestamp
}
if lagInfo.LatestCheckpointedTimestamp.Less(resolvedSpan.Timestamp) {
lagInfo.LatestCheckpointedTimestamp = resolvedSpan.Timestamp
}
}
lagInfo.SlowestFastestIngestionLag = lagInfo.LatestCheckpointedTimestamp.GoTime().
Sub(lagInfo.EarliestCheckpointedTimestamp.GoTime())
lagInfo.ReplicationLag = timeutil.Since(highwater.GoTime())
stats.ReplicationLagInfo = lagInfo
}
client, err := streamclient.GetFirstActiveClient(ctx, progress.GetStreamIngest().StreamAddresses)
if err != nil {
return nil, err
}
streamStatus, err := client.Heartbeat(ctx, streampb.StreamID(details.StreamID), hlc.MaxTimestamp)
if err != nil {
stats.ProducerError = err.Error()
} else {
stats.ProducerStatus = &streamStatus
}
return stats, client.Close(ctx)
}
type streamIngestionResumer struct {
job *jobs.Job
}
func connectToActiveClient(
ctx context.Context, ingestionJob *jobs.Job,
) (streamclient.Client, error) {
details := ingestionJob.Details().(jobspb.StreamIngestionDetails)
progress := ingestionJob.Progress()
streamAddresses := progress.GetStreamIngest().StreamAddresses
if len(streamAddresses) > 0 {
log.Infof(ctx, "ingestion job %d attempting to connect to existing stream addresses", ingestionJob.ID())
client, err := streamclient.GetFirstActiveClient(ctx, streamAddresses)
if err == nil {
return client, err
}
// fall through to streamAddress, as even though it is likely part of the
// topology it may have been changed to a new valid address via an ALTER
// statement
}
// Without a list of addresses from existing progress we use the stream
// address from the creation statement
streamAddress := streamingccl.StreamAddress(details.StreamAddress)
client, err := streamclient.NewStreamClient(ctx, streamAddress)
return client, errors.Wrapf(err, "ingestion job %d failed to connect to stream address or existing topology for planning", ingestionJob.ID())
}
// Ping the producer job and waits until it is active/running, returns nil when
// the job is active.
func waitUntilProducerActive(
ctx context.Context,
client streamclient.Client,
streamID streampb.StreamID,
heartbeatTimestamp hlc.Timestamp,
ingestionJobID jobspb.JobID,
) error {
ro := retry.Options{
InitialBackoff: 1 * time.Second,
Multiplier: 2,
MaxBackoff: 5 * time.Second,
MaxRetries: 4,
}
// Make sure the producer job is active before start the stream replication.
var status streampb.StreamReplicationStatus
var err error
for r := retry.Start(ro); r.Next(); {
status, err = client.Heartbeat(ctx, streamID, heartbeatTimestamp)
if err != nil {
return errors.Wrapf(err, "failed to resume ingestion job %d due to producer job error",
ingestionJobID)
}
if status.StreamStatus != streampb.StreamReplicationStatus_UNKNOWN_STREAM_STATUS_RETRY {
break
}
log.Warningf(ctx, "producer job %d has status %s, retrying", streamID, status.StreamStatus)
}
if status.StreamStatus != streampb.StreamReplicationStatus_STREAM_ACTIVE {
return jobs.MarkAsPermanentJobError(errors.Errorf("failed to resume ingestion job %d "+
"as the producer job is not active and in status %s", ingestionJobID, status.StreamStatus))
}
return nil
}
func updateRunningStatus(ctx context.Context, ingestionJob *jobs.Job, status string) {
if err := ingestionJob.RunningStatus(ctx, nil,
func(ctx context.Context, details jobspb.Details) (jobs.RunningStatus, error) {
return jobs.RunningStatus(status), nil
}); err != nil {
log.Warningf(ctx, "error when updating job running status: %s", err)
}
}
func ingest(ctx context.Context, execCtx sql.JobExecContext, ingestionJob *jobs.Job) error {
// Cutover should be the *first* thing checked upon resumption as it is the
// most critical task in disaster recovery.
reverted, err := maybeRevertToCutoverTimestamp(ctx, execCtx, ingestionJob.ID())
if err != nil {
return err
}
if reverted {
log.Infof(ctx, "job completed cutover on resume")
return nil
}
if knobs := execCtx.ExecCfg().StreamingTestingKnobs; knobs != nil && knobs.BeforeIngestionStart != nil {
if err := knobs.BeforeIngestionStart(ctx); err != nil {
return err
}
}
details := ingestionJob.Details().(jobspb.StreamIngestionDetails)
progress := ingestionJob.Progress()
streamAddress := streamingccl.StreamAddress(details.StreamAddress)
startTime := progress.GetStreamIngest().StartTime
// Start from the last checkpoint if it exists.
if h := progress.GetHighWater(); h != nil && !h.IsEmpty() {
startTime = *h
}
// Initialize a stream client and resolve topology.
client, err := connectToActiveClient(ctx, ingestionJob)
if err != nil {
return err
}
ingestWithClient := func() error {
streamID := streampb.StreamID(details.StreamID)
updateRunningStatus(ctx, ingestionJob, fmt.Sprintf("connecting to the producer job %d "+
"and creating a stream replication plan", streamID))
if err := waitUntilProducerActive(ctx, client, streamID, startTime, ingestionJob.ID()); err != nil {
return err
}
log.Infof(ctx, "producer job %d is active, creating a stream replication plan", streamID)
topology, err := client.Plan(ctx, streamID)
if err != nil {
return err
}
// TODO(casper): update running status
err = ingestionJob.Update(ctx, nil, func(txn *kv.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error {
if md.Progress.GetStreamIngest().StartTime.Less(startTime) {
md.Progress.GetStreamIngest().StartTime = startTime
}
md.Progress.GetStreamIngest().StreamAddresses = topology.StreamAddresses()
ju.UpdateProgress(md.Progress)
return nil
})
if err != nil {
return errors.Wrap(err, "failed to update job progress")
}
log.Infof(ctx, "ingestion job %d resumes stream ingestion from start time %s",
ingestionJob.ID(), progress.GetStreamIngest().StartTime)
ingestProgress := progress.Details.(*jobspb.Progress_StreamIngest).StreamIngest
checkpoint := ingestProgress.Checkpoint
evalCtx := execCtx.ExtendedEvalContext()
dsp := execCtx.DistSQLPlanner()
planCtx, sqlInstanceIDs, err := dsp.SetupAllNodesPlanning(ctx, evalCtx, execCtx.ExecCfg())
if err != nil {
return err
}
// Construct stream ingestion processor specs.
streamIngestionSpecs, streamIngestionFrontierSpec, err := distStreamIngestionPlanSpecs(
streamAddress, topology, sqlInstanceIDs, progress.GetStreamIngest().StartTime, checkpoint,
ingestionJob.ID(), streamID, topology.SourceTenantID, details.DestinationTenantID)
if err != nil {
return err
}
// Plan and run the DistSQL flow.
log.Infof(ctx, "starting to run DistSQL flow for stream ingestion job %d",
ingestionJob.ID())
updateRunningStatus(ctx, ingestionJob, "running the SQL flow for the stream ingestion job")
if err = distStreamIngest(ctx, execCtx, sqlInstanceIDs, planCtx, dsp,
streamIngestionSpecs, streamIngestionFrontierSpec); err != nil {
return err
}
// A nil error is only possible if the job was signaled to cutover and the
// processors shut down gracefully, i.e stopped ingesting any additional
// events from the replication stream. At this point it is safe to revert to
// the cutoff time to leave the cluster in a consistent state.
log.Infof(ctx,
"starting to revert to the specified cutover timestamp for stream ingestion job %d",
ingestionJob.ID())
if err = revertToCutoverTimestamp(ctx, execCtx, ingestionJob.ID()); err != nil {
return err
}
log.Infof(ctx, "activating destination tenant %d", details.DestinationTenantID)
// Activate the tenant as it is now in a usable state.
if err = activateTenant(ctx, execCtx, details.DestinationTenantID); err != nil {
return err
}
log.Infof(ctx, "starting to complete the producer job %d", streamID)
updateRunningStatus(ctx, ingestionJob, "completing the producer job in the source cluster")
// Completes the producer job in the source cluster on best effort.
if err = client.Complete(ctx, streamID, true /* successfulIngestion */); err != nil {
log.Warningf(ctx, "encountered error when completing the source cluster producer job %d", streamID)
}
return nil
}
return errors.CombineErrors(ingestWithClient(), client.Close(ctx))
}
func ingestWithRetries(
ctx context.Context, execCtx sql.JobExecContext, ingestionJob *jobs.Job,
) error {
ro := retry.Options{
InitialBackoff: 3 * time.Second,
Multiplier: 2,
MaxBackoff: 1 * time.Minute,
MaxRetries: 60,
}
var err error
retryCount := 0
for r := retry.Start(ro); r.Next(); {
err = ingest(ctx, execCtx, ingestionJob)
if err == nil {
break
}
// By default, all errors are retryable unless it's marked as
// permanent job error in which case we pause the job.
// We also stop the job when this is a context cancellation error
// as requested pause or cancel will trigger a context cancellation.
if jobs.IsPermanentJobError(err) || errors.Is(err, context.Canceled) {
break
}
const msgFmt = "stream ingestion waits for retrying after error %s"
log.Warningf(ctx, msgFmt, err)
updateRunningStatus(ctx, ingestionJob, fmt.Sprintf(msgFmt, err))
retryCount++
}
status := "stream ingestion finished successfully"
if err != nil {
status = fmt.Sprintf("stream ingestion encountered error and is to be paused: %s", err)
}
updateRunningStatus(ctx, ingestionJob, status)
return err
}
// The ingestion job should never fail, only pause, as progress should never be lost.
func (s *streamIngestionResumer) handleResumeError(
resumeCtx context.Context, execCtx interface{}, err error,
) error {
const errorFmt = "ingestion job failed (%v) but is being paused"
errorMessage := fmt.Sprintf(errorFmt, err)
log.Warningf(resumeCtx, errorFmt, err)
// The ingestion job is paused but the producer job will keep
// running until it times out. Users can still resume ingestion before
// the producer job times out.
jobExecCtx := execCtx.(sql.JobExecContext)
return s.job.PauseRequested(resumeCtx, jobExecCtx.Txn(), func(ctx context.Context,
planHookState interface{}, txn *kv.Txn, progress *jobspb.Progress) error {
progress.RunningStatus = errorMessage
return nil
}, errorMessage)
}
// Resume is part of the jobs.Resumer interface. Ensure that any errors
// produced here are returned as s.handleResumeError.
func (s *streamIngestionResumer) Resume(resumeCtx context.Context, execCtx interface{}) error {
// Start ingesting KVs from the replication stream.
err := ingestWithRetries(resumeCtx, execCtx.(sql.JobExecContext), s.job)
if err != nil {
return s.handleResumeError(resumeCtx, execCtx, err)
}
return nil
}
// revertToCutoverTimestamp attempts a cutover and errors out if one was not
// executed.
func revertToCutoverTimestamp(
ctx context.Context, execCtx interface{}, ingestionJobID jobspb.JobID,
) error {
reverted, err := maybeRevertToCutoverTimestamp(ctx, execCtx, ingestionJobID)
if err != nil {
return err
}
if !reverted {
return errors.Errorf("required cutover was not completed")
}
return nil
}
// maybeRevertToCutoverTimestamp reads the job progress for the cutover time and
// if the job has progressed passed the cutover time issues a RevertRangeRequest
// with the target time set to that cutover time, to bring the ingesting cluster
// to a consistent state.
func maybeRevertToCutoverTimestamp(
ctx context.Context, execCtx interface{}, ingestionJobID jobspb.JobID,
) (bool, error) {
ctx, span := tracing.ChildSpan(ctx, "streamingest.revertToCutoverTimestamp")
defer span.Finish()
p := execCtx.(sql.JobExecContext)
db := p.ExecCfg().DB
j, err := p.ExecCfg().JobRegistry.LoadJob(ctx, ingestionJobID)
if err != nil {
return false, err
}
details := j.Details()
var sd jobspb.StreamIngestionDetails
var ok bool
if sd, ok = details.(jobspb.StreamIngestionDetails); !ok {
return false, errors.Newf("unknown details type %T in stream ingestion job %d",
details, ingestionJobID)
}
progress := j.Progress()
var sp *jobspb.Progress_StreamIngest
if sp, ok = progress.GetDetails().(*jobspb.Progress_StreamIngest); !ok {
return false, errors.Newf("unknown progress type %T in stream ingestion job %d",
j.Progress().Progress, ingestionJobID)
}
cutoverTime := sp.StreamIngest.CutoverTime
if cutoverTime.IsEmpty() {
log.Infof(ctx, "empty cutover time, no revert required")
return false, nil
}
if progress.GetHighWater() == nil || progress.GetHighWater().Less(cutoverTime) {
log.Infof(ctx, "job with highwater %s not yet ready to revert to cutover at %s", progress.GetHighWater(), cutoverTime.String())
return false, nil
}
updateRunningStatus(ctx, j, fmt.Sprintf("starting to cut over to the given timestamp %s", cutoverTime))
spans := []roachpb.Span{sd.Span}
for len(spans) != 0 {
var b kv.Batch
for _, span := range spans {
b.AddRawRequest(&roachpb.RevertRangeRequest{
RequestHeader: roachpb.RequestHeader{
Key: span.Key,
EndKey: span.EndKey,
},
TargetTime: sp.StreamIngest.CutoverTime,
})
}
b.Header.MaxSpanRequestKeys = sql.RevertTableDefaultBatchSize
if err := db.Run(ctx, &b); err != nil {
return false, err
}
spans = spans[:0]
for _, raw := range b.RawResponse().Responses {
r := raw.GetRevertRange()
if r.ResumeSpan != nil {
if !r.ResumeSpan.Valid() {
return false, errors.Errorf("invalid resume span: %s", r.ResumeSpan)
}
spans = append(spans, *r.ResumeSpan)
}
}
}
return true, j.SetProgress(ctx, nil /* txn */, *sp.StreamIngest)
}
func activateTenant(ctx context.Context, execCtx interface{}, newTenantID roachpb.TenantID) error {
p := execCtx.(sql.JobExecContext)
execCfg := p.ExecCfg()
return execCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error {
info, err := sql.GetTenantRecordByID(ctx, execCfg, txn, newTenantID)
if err != nil {
return err
}
info.State = descpb.TenantInfo_ACTIVE
info.TenantReplicationJobID = 0
return sql.UpdateTenantRecord(ctx, execCfg, txn, info)
})
}
func (s *streamIngestionResumer) cancelProducerJob(
ctx context.Context, details jobspb.StreamIngestionDetails,
) {
streamID := streampb.StreamID(details.StreamID)
addr := streamingccl.StreamAddress(details.StreamAddress)
client, err := streamclient.NewStreamClient(ctx, addr)
if err != nil {
log.Warningf(ctx, "encountered error when creating the stream client: %v", err)
return
}
log.Infof(ctx, "canceling the producer job %d as stream ingestion job %d is being canceled",
streamID, s.job.ID())
if err = client.Complete(ctx, streamID, false /* successfulIngestion */); err != nil {
log.Warningf(ctx, "encountered error when canceling the producer job: %v", err)
}
if err = client.Close(ctx); err != nil {
log.Warningf(ctx, "encountered error when closing the stream client: %v", err)
}
}
// OnFailOrCancel is part of the jobs.Resumer interface.
// There is a know race between the ingestion processors shutting down, and
// OnFailOrCancel being invoked. As a result of which we might see some keys
// leftover in the keyspace if a ClearRange were to be issued here. In general
// the tenant keyspace of a failed/canceled ingestion job should be treated as
// corrupted, and the tenant should be dropped before resuming the ingestion.
func (s *streamIngestionResumer) OnFailOrCancel(
ctx context.Context, execCtx interface{}, _ error,
) error {
// Cancel the producer job on best effort. The source job's protected timestamp is no
// longer needed as this ingestion job is in 'reverting' status and we won't resume
// ingestion anymore.
jobExecCtx := execCtx.(sql.JobExecContext)
details := s.job.Details().(jobspb.StreamIngestionDetails)
s.cancelProducerJob(ctx, details)
tenInfo, err := sql.GetTenantRecordByID(ctx, jobExecCtx.ExecCfg(), jobExecCtx.Txn(), details.DestinationTenantID)
if err != nil {
return errors.Wrap(err, "fetch tenant info")
}
tenInfo.TenantReplicationJobID = 0
if err := sql.UpdateTenantRecord(ctx, jobExecCtx.ExecCfg(), jobExecCtx.Txn(), tenInfo); err != nil {
return errors.Wrap(err, "update tenant record")
}
return nil
}
func (s *streamIngestionResumer) ForceRealSpan() bool { return true }
var _ jobs.Resumer = &streamIngestionResumer{}
func init() {
jobs.RegisterConstructor(
jobspb.TypeStreamIngestion,
func(job *jobs.Job,
settings *cluster.Settings) jobs.Resumer {
return &streamIngestionResumer{job: job}
},
jobs.UsesTenantCostControl,
)
}