diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_frontier_processor_test.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_frontier_processor_test.go index 0d83b7fbf6cb..7a579fb83a6e 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_frontier_processor_test.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_frontier_processor_test.go @@ -152,7 +152,8 @@ func TestStreamIngestionFrontierProcessor(t *testing.T) { } { t.Run(tc.name, func(t *testing.T) { spec.PartitionAddresses = []streamingccl.PartitionAddress{pa1, pa2} - proc, err := newStreamIngestionDataProcessor(&flowCtx, 0 /* processorID */, spec, &post, out) + proc, err := newStreamIngestionDataProcessor(&flowCtx, 0 /* processorID */, spec, + nil, &post, out) require.NoError(t, err) sip, ok := proc.(*streamIngestionProcessor) if !ok { diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_job.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_job.go index a586a8a8e7ee..a058517b4ded 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_job.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_job.go @@ -63,20 +63,15 @@ func ingest( } // Construct stream ingestion processor specs. - streamIngestionSpecs, streamIngestionFrontierSpec, err := distStreamIngestionPlanSpecs( - streamAddress, topology, nodes, initialHighWater) + streamIngestionPollingSpec, streamIngestionSpecs, streamIngestionFrontierSpec, + err := distStreamIngestionPlanSpecs(streamAddress, topology, nodes, initialHighWater, jobID) if err != nil { return err } // Plan and run the DistSQL flow. - err = distStreamIngest(ctx, execCtx, nodes, jobID, planCtx, dsp, streamIngestionSpecs, - streamIngestionFrontierSpec) - if err != nil { - return err - } - - return nil + return distStreamIngest(ctx, execCtx, nodes, jobID, planCtx, dsp, streamIngestionSpecs, + streamIngestionFrontierSpec, streamIngestionPollingSpec) } // Resume is part of the jobs.Resumer interface. @@ -87,6 +82,16 @@ func (s *streamIngestionResumer) Resume(ctx context.Context, execCtx interface{} err := ingest(ctx, p, details.StreamAddress, s.job.Progress(), *s.job.ID()) if err != nil { + if errors.Is(err, errCutoverRequested) { + revertErr := s.revertToLatestResolvedTimestamp(ctx, execCtx) + if revertErr != nil { + return errors.Wrap(revertErr, "error while reverting stream ingestion job") + } + // The job should transition to a succeeded state once the cutover is + // complete. + + return nil + } return err } @@ -97,42 +102,66 @@ func (s *streamIngestionResumer) Resume(ctx context.Context, execCtx interface{} return nil } -// OnFailOrCancel is part of the jobs.Resumer interface. -func (s *streamIngestionResumer) OnFailOrCancel(ctx context.Context, execCtx interface{}) error { +// revertToLatestResolvedTimestamp reads the job progress for the high watermark +// and issues a RevertRangeRequest with the target time set to that high +// watermark. +func (s *streamIngestionResumer) revertToLatestResolvedTimestamp( + ctx context.Context, execCtx interface{}, +) error { p := execCtx.(sql.JobExecContext) db := p.ExecCfg().DB - details := s.job.Details().(jobspb.StreamIngestionDetails) + j, err := p.ExecCfg().JobRegistry.LoadJob(ctx, *s.job.ID()) + if err != nil { + return err + } + details := j.Details() + var sd jobspb.StreamIngestionDetails + var ok bool + if sd, ok = details.(jobspb.StreamIngestionDetails); !ok { + return errors.Newf("unknown details type %T in stream ingestion job %d", + details, *s.job.ID()) + } + progress := j.Progress() + streamIngestProgress := progress.GetStreamIngest() + + revertTargetTime := sd.StartTime + if streamIngestProgress != nil { + revertTargetTime = streamIngestProgress.CutoverTime + } - resolvedTime := details.StartTime - prog := s.job.Progress() - if highWatermark := prog.GetHighWater(); highWatermark != nil { - if highWatermark.Less(resolvedTime) { - return errors.Newf("progress timestamp %+v cannot be older than start time %+v", - highWatermark, resolvedTime) + if highWatermark := progress.GetHighWater(); highWatermark != nil { + if highWatermark.Less(revertTargetTime) { + return errors.Newf("progress timestamp %+v cannot be older than the requested "+ + "cutover time %+v", highWatermark, revertTargetTime) } - resolvedTime = *highWatermark } - // TODO(adityamaru): If the job progress was not set then we should - // probably ClearRange. Take this into account when writing the ClearRange - // OnFailOrCancel(). - if resolvedTime.IsEmpty() { - return nil + // Sanity check that the resolvedTime is not less than the time at which the + // ingestion job was started. + if revertTargetTime.Less(sd.StartTime) { + return errors.Newf("revert target time %+v cannot be older than the start time "+ + "cutover time %+v", revertTargetTime, sd.StartTime) } var b kv.Batch b.AddRawRequest(&roachpb.RevertRangeRequest{ RequestHeader: roachpb.RequestHeader{ - Key: details.Span.Key, - EndKey: details.Span.EndKey, + Key: sd.Span.Key, + EndKey: sd.Span.EndKey, }, - TargetTime: resolvedTime, + TargetTime: revertTargetTime, EnableTimeBoundIteratorOptimization: true, }) b.Header.MaxSpanRequestKeys = sql.RevertTableDefaultBatchSize return db.Run(ctx, &b) } +// OnFailOrCancel is part of the jobs.Resumer interface. +func (s *streamIngestionResumer) OnFailOrCancel(ctx context.Context, execCtx interface{}) error { + // TODO(adityamaru): Add ClearRange logic. + return nil +} + var _ jobs.Resumer = &streamIngestionResumer{} func init() { diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_planning.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_planning.go index 2aaed0cf2b88..9fe0b440fb39 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_planning.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_planning.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/errors" ) @@ -81,8 +82,9 @@ func ingestionPlanHook( streamIngestionDetails := jobspb.StreamIngestionDetails{ StreamAddress: streamingccl.StreamAddress(from[0]), Span: roachpb.Span{Key: prefix, EndKey: prefix.PrefixEnd()}, - // TODO: Figure out what the initial ts should be. - StartTime: hlc.Timestamp{}, + // TODO(adityamaru): This will change to the time the RESTORE was run at + // before starting the ingestion strea. + StartTime: hlc.Timestamp{WallTime: timeutil.Now().UnixNano()}, } jobDescription, err := streamIngestionJobDescription(p, ingestionStmt) diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_polling_processor.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_polling_processor.go new file mode 100644 index 000000000000..c284c62d98db --- /dev/null +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_polling_processor.go @@ -0,0 +1,165 @@ +// 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" + "time" + + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" + "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/rowenc" + "github.com/cockroachdb/cockroach/pkg/sql/rowexec" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/errors" +) + +const streamIngestionPollingProcName = `ingestpolling` + +type streamIngestionPollingProcessor struct { + execinfra.ProcessorBase + execinfra.StreamingProcessor + + flowCtx *execinfra.FlowCtx + spec execinfrapb.StreamIngestionPollingSpec + output execinfra.RowReceiver + + jobID int64 + registry *jobs.Registry + stopPoller chan struct{} +} + +var _ execinfra.Processor = &streamIngestionPollingProcessor{} +var _ execinfra.RowSource = &streamIngestionPollingProcessor{} + +func init() { + rowexec.NewStreamIngestionPollingProcessor = newStreamIngestionPollingProcessor +} + +func newStreamIngestionPollingProcessor( + flowCtx *execinfra.FlowCtx, + processorID int32, + spec execinfrapb.StreamIngestionPollingSpec, + post *execinfrapb.PostProcessSpec, + output execinfra.RowReceiver, +) (execinfra.Processor, error) { + sp := &streamIngestionPollingProcessor{ + flowCtx: flowCtx, + spec: spec, + output: output, + jobID: spec.JobID, + registry: flowCtx.Cfg.JobRegistry, + stopPoller: make(chan struct{}), + } + if err := sp.Init( + sp, + post, + streamIngestionResultTypes, + flowCtx, + processorID, + output, + nil, /* memMonitor */ + execinfra.ProcStateOpts{ + InputsToDrain: []execinfra.RowSource{}, + TrailingMetaCallback: func(ctx context.Context) []execinfrapb.ProducerMetadata { + sp.close() + return nil + }, + }, + ); err != nil { + return nil, err + } + return sp, nil +} + +func (sp *streamIngestionPollingProcessor) close() { + if sp.InternalClose() { + close(sp.stopPoller) + } +} + +// Start is part of the RowSource interface. +func (sp *streamIngestionPollingProcessor) Start(ctx context.Context) context.Context { + return sp.StartInternal(ctx, streamIngestionPollingProcName) +} + +// checkForCutoverSignal periodically loads the job progress to check for the +// sentinel value that signals the ingestion job to complete. +func (sp *streamIngestionPollingProcessor) checkForCutoverSignal( + ctx context.Context, +) (rowenc.EncDatumRow, error) { + tick := time.NewTicker(time.Second * 10) + defer tick.Stop() + for { + select { + case _, ok := <-sp.stopPoller: + if !ok { + return nil, nil + } + // Shouldn't come here. + return nil, nil + case <-tick.C: + j, err := sp.registry.LoadJob(ctx, sp.jobID) + if err != nil { + return nil, err + } + progress := j.Progress() + var prog *jobspb.Progress_StreamIngest + var ok bool + if prog, ok = progress.GetDetails().(*jobspb.Progress_StreamIngest); !ok { + return nil, errors.Newf("unknown progress type %T in stream ingestion job %d", + j.Progress().Progress, sp.jobID) + } + // Job has been signaled to complete. + if cutoverTime := prog.StreamIngest.CutoverTime; !cutoverTime.IsEmpty() { + streamIngestBytes, err := protoutil.Marshal(prog.StreamIngest) + if err != nil { + return nil, err + } + row := rowenc.EncDatumRow{ + rowenc.DatumToEncDatum(types.Bytes, tree.NewDBytes(tree.DBytes(streamIngestBytes))), + } + return row, nil + } + } + } +} + +// Next is part of the RowSource interface. +func (sp *streamIngestionPollingProcessor) Next() ( + rowenc.EncDatumRow, + *execinfrapb.ProducerMetadata, +) { + if sp.State != execinfra.StateRunning { + return nil, sp.DrainHelper() + } + + row, err := sp.checkForCutoverSignal(sp.flowCtx.EvalCtx.Context) + if err != nil { + sp.MoveToDraining(errors.Wrap(err, "error when polling for cutover signal")) + return nil, sp.DrainHelper() + } + + if row != nil { + return row, nil + } + + sp.MoveToDraining(nil /* error */) + return nil, sp.DrainHelper() +} + +// ConsumerClosed is part of the RowSource interface. +func (sp *streamIngestionPollingProcessor) ConsumerClosed() { + // The consumer is done, Next() will not be called again. + sp.close() +} diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor.go index a3b603734df8..d5d254b48925 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor.go @@ -10,6 +10,7 @@ package streamingest import ( "context" + "fmt" "sort" "time" @@ -46,6 +47,10 @@ var streamIngestionResultTypes = []*types.T{ types.Bytes, // jobspb.ResolvedSpans } +var ( + errCutoverRequested = errors.New("cutover requested") +) + type mvccKeyValues []storage.MVCCKeyValue func (s mvccKeyValues) Len() int { return len(s) } @@ -58,6 +63,7 @@ type streamIngestionProcessor struct { flowCtx *execinfra.FlowCtx spec execinfrapb.StreamIngestionDataSpec output execinfra.RowReceiver + alloc rowenc.DatumAlloc // curBatch temporarily batches MVCC Keys so they can be // sorted before ingestion. @@ -88,8 +94,17 @@ type streamIngestionProcessor struct { // that it can be forwarded through the DistSQL flow. ingestionErr error + ingestionMeta *execinfrapb.ProducerMetadata + // eventCh is the merged event channel of all of the partition event streams. eventCh chan partitionEvent + + // signalCh is the channel used to pass rows from input to the stream + // ingestion processor. + signalCh chan rowenc.EncDatum + + // input returns rows from the stream ingestion polling processor. + input execinfra.RowSource } // partitionEvent augments a normal event with the partition it came from. @@ -107,6 +122,7 @@ func newStreamIngestionDataProcessor( flowCtx *execinfra.FlowCtx, processorID int32, spec execinfrapb.StreamIngestionDataSpec, + input execinfra.RowSource, post *execinfrapb.PostProcessSpec, output execinfra.RowReceiver, ) (execinfra.Processor, error) { @@ -135,13 +151,15 @@ func newStreamIngestionDataProcessor( output: output, curBatch: make([]storage.MVCCKeyValue, 0), client: streamClient, + input: input, bufferedCheckpoints: make(map[streamingccl.PartitionAddress]hlc.Timestamp), timer: timeutil.NewTimer(), + signalCh: make(chan rowenc.EncDatum), } - if err := sip.Init(sip, post, streamIngestionResultTypes, flowCtx, processorID, output, nil, /* memMonitor */ + if err := sip.Init(sip, post, input.OutputTypes(), flowCtx, processorID, output, nil, /* memMonitor */ execinfra.ProcStateOpts{ - InputsToDrain: []execinfra.RowSource{}, + InputsToDrain: []execinfra.RowSource{sip.input}, TrailingMetaCallback: func(context.Context) []execinfrapb.ProducerMetadata { sip.close() return nil @@ -156,6 +174,7 @@ func newStreamIngestionDataProcessor( // Start is part of the RowSource interface. func (sip *streamIngestionProcessor) Start(ctx context.Context) context.Context { + sip.input.Start(ctx) ctx = sip.StartInternal(ctx, streamIngestionProcessorName) evalCtx := sip.FlowCtx.EvalCtx @@ -181,6 +200,23 @@ func (sip *streamIngestionProcessor) Start(ctx context.Context) context.Context } sip.eventCh = sip.merge(sip.Ctx, eventChs) + // Start listening for signals from the poller processor. + go func() { + row, meta := sip.input.Next() + if meta != nil { + if meta.Err != nil { + sip.MoveToDraining(nil /* err */) + } + sip.ingestionMeta = meta + return + } + if row == nil { + sip.MoveToDraining(nil /* err */) + return + } + sip.signalCh <- row[0] + }() + return ctx } @@ -189,7 +225,6 @@ func (sip *streamIngestionProcessor) Next() (rowenc.EncDatumRow, *execinfrapb.Pr if sip.State != execinfra.StateRunning { return nil, sip.DrainHelper() } - progressUpdate, err := sip.consumeEvents() if err != nil { sip.MoveToDraining(err) @@ -208,6 +243,10 @@ func (sip *streamIngestionProcessor) Next() (rowenc.EncDatumRow, *execinfrapb.Pr return row, nil } + if sip.ingestionMeta != nil { + return nil, sip.ingestionMeta + } + if sip.ingestionErr != nil { sip.MoveToDraining(sip.ingestionErr) return nil, sip.DrainHelper() @@ -230,6 +269,9 @@ func (sip *streamIngestionProcessor) close() { if sip.timer != nil { sip.timer.Stop() } + if sip.signalCh != nil { + close(sip.signalCh) + } } } @@ -298,6 +340,39 @@ func (sip *streamIngestionProcessor) consumeEvents() (*jobspb.ResolvedSpans, err for sip.State == execinfra.StateRunning { select { + case row, ok := <-sip.signalCh: + if !ok { + sip.internalDrained = true + return sip.flush() + } + if err := row.EnsureDecoded(streamIngestionResultTypes[0], &sip.alloc); err != nil { + return nil, err + } + raw, ok := row.Datum.(*tree.DBytes) + if !ok { + return nil, errors.AssertionFailedf(`unexpected datum type %T: %s`, row.Datum, + row.Datum) + } + var streamIngestionProg jobspb.StreamIngestionProgress + if err := protoutil.Unmarshal([]byte(*raw), &streamIngestionProg); err != nil { + return nil, errors.NewAssertionErrorWithWrappedErrf(err, + `unmarshalling stream ingestion progress: %x`, raw) + } + + // Check if the job has been signaled to cutover. + // TODO(adityamaru): The cutover ts could be in the future, in which case + // we would like to inform all partitions to ingest up until that ts and + // then stop. For the MVP we do not allow specifying a cutover ts past the + // resolved timestamp written to the job progress at the time the cutover + // was requested, so we can simply return. + // As of now, we don't flush the buffered KVs before returning as they + // will be reverted because of the invariant mentioned above. + if ts := streamIngestionProg.CutoverTime; !ts.IsEmpty() { + sip.ingestionErr = errors.Wrap(errCutoverRequested, fmt.Sprintf("cutover at ts %s", + ts.String())) + return nil, nil + } + return nil, errors.New("unsupported signal received from input") case event, ok := <-sip.eventCh: if !ok { sip.internalDrained = true diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor_planning.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor_planning.go index 77130f01a52f..1bc105a6e686 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor_planning.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor_planning.go @@ -31,7 +31,16 @@ func distStreamIngestionPlanSpecs( topology streamingccl.Topology, nodes []roachpb.NodeID, initialHighWater hlc.Timestamp, -) ([]*execinfrapb.StreamIngestionDataSpec, *execinfrapb.StreamIngestionFrontierSpec, error) { + jobID int64, +) ( + *execinfrapb.StreamIngestionPollingSpec, + []*execinfrapb.StreamIngestionDataSpec, + *execinfrapb.StreamIngestionFrontierSpec, + error, +) { + // Create a spec for the StreamIngestionPolling processor on the coordinator + // node. + streamIngestionPollingSpec := &execinfrapb.StreamIngestionPollingSpec{JobID: jobID} // For each stream partition in the topology, assign it to a node. streamIngestionSpecs := make([]*execinfrapb.StreamIngestionDataSpec, 0, len(nodes)) @@ -66,7 +75,7 @@ func distStreamIngestionPlanSpecs( streamIngestionFrontierSpec := &execinfrapb.StreamIngestionFrontierSpec{ HighWaterAtStart: initialHighWater, TrackedSpans: trackedSpans} - return streamIngestionSpecs, streamIngestionFrontierSpec, nil + return streamIngestionPollingSpec, streamIngestionSpecs, streamIngestionFrontierSpec, nil } func distStreamIngest( @@ -78,6 +87,7 @@ func distStreamIngest( dsp *sql.DistSQLPlanner, streamIngestionSpecs []*execinfrapb.StreamIngestionDataSpec, streamIngestionFrontierSpec *execinfrapb.StreamIngestionFrontierSpec, + streamIngestionPollingSpec *execinfrapb.StreamIngestionPollingSpec, ) error { ctx = logtags.AddTag(ctx, "stream-ingest-distsql", nil) evalCtx := execCtx.ExtendedEvalContext() @@ -87,29 +97,86 @@ func distStreamIngest( return nil } - // Setup a one-stage plan with one proc per input spec. - corePlacement := make([]physicalplan.ProcessorCorePlacement, len(streamIngestionSpecs)) - for i := range streamIngestionSpecs { - corePlacement[i].NodeID = nodes[i] - corePlacement[i].Core.StreamIngestionData = streamIngestionSpecs[i] - } - - p := planCtx.NewPhysicalPlan() - p.AddNoInputStage( - corePlacement, - execinfrapb.PostProcessSpec{}, - streamIngestionResultTypes, - execinfrapb.Ordering{}, - ) - execCfg := execCtx.ExecCfg() gatewayNodeID, err := execCfg.NodeID.OptionalNodeIDErr(48274) if err != nil { return err } - // The ResultRouters from the previous stage will feed in to the - // StreamIngestionFrontier processor. + p := planCtx.NewPhysicalPlan() + + // Plan the stream ingestion polling processor on the coordinator node. This + // processor does not take any input and uses a "mirror router" to replicate + // its output to all output streams. + var pollingProc physicalplan.Processor + if len(streamIngestionSpecs) == 1 { + pollingProc = physicalplan.Processor{ + Node: gatewayNodeID, + Spec: execinfrapb.ProcessorSpec{ + Core: execinfrapb.ProcessorCoreUnion{StreamIngestionPolling: streamIngestionPollingSpec}, + Post: execinfrapb.PostProcessSpec{}, + Output: []execinfrapb.OutputRouterSpec{{ + Type: execinfrapb.OutputRouterSpec_PASS_THROUGH, + }}, + StageID: p.NewStage(false /* containsRemoteProcessor */), + ResultTypes: streamIngestionResultTypes, + }, + } + } else { + pollingProc = physicalplan.Processor{ + Node: gatewayNodeID, + Spec: execinfrapb.ProcessorSpec{ + Core: execinfrapb.ProcessorCoreUnion{StreamIngestionPolling: streamIngestionPollingSpec}, + Post: execinfrapb.PostProcessSpec{}, + Output: []execinfrapb.OutputRouterSpec{{ + Type: execinfrapb.OutputRouterSpec_MIRROR, + }}, + StageID: p.NewStage(false /* containsRemoteProcessor */), + ResultTypes: streamIngestionResultTypes, + }, + } + } + pollingProcIdx := p.AddProcessor(pollingProc) + + // Plan the stream ingestion data processors. + streamIngestionStageID := p.NewStageOnNodes(nodes[:len(streamIngestionSpecs)]) + streamIngestionProcs := make(map[roachpb.NodeID]physicalplan.ProcessorIdx) + for i, n := range nodes { + // We can have more nodes than stream ingestion processors to setup. + if i >= len(streamIngestionSpecs) { + break + } + streamIngestionProc := physicalplan.Processor{ + Node: n, + Spec: execinfrapb.ProcessorSpec{ + Input: []execinfrapb.InputSyncSpec{{ColumnTypes: streamIngestionResultTypes}}, + Core: execinfrapb.ProcessorCoreUnion{StreamIngestionData: streamIngestionSpecs[i]}, + Post: execinfrapb.PostProcessSpec{}, + Output: []execinfrapb.OutputRouterSpec{{Type: execinfrapb.OutputRouterSpec_PASS_THROUGH}}, + StageID: streamIngestionStageID, + ResultTypes: streamIngestionResultTypes, + }, + } + pIdx := p.AddProcessor(streamIngestionProc) + streamIngestionProcs[n] = pIdx + p.ResultRouters = append(p.ResultRouters, pIdx) + } + + // Setup the streams from the polling processor to the stream ingestion + // processors. + slot := 0 + for _, destProc := range streamIngestionProcs { + p.Streams = append(p.Streams, physicalplan.Stream{ + SourceProcessor: pollingProcIdx, + SourceRouterSlot: slot, + DestProcessor: destProc, + DestInput: 0, + }) + slot++ + } + + // Plan the frontier processor. The ResultRouters from the stream ingestion + // processors will be merged and fed into this processor. p.AddSingleGroupStage(gatewayNodeID, execinfrapb.ProcessorCoreUnion{StreamIngestionFrontier: streamIngestionFrontierSpec}, execinfrapb.PostProcessSpec{}, streamIngestionResultTypes) diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor_test.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor_test.go index d96cb96b85c1..775974d7ce02 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor_test.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor_test.go @@ -407,7 +407,8 @@ func runStreamIngestionProcessor( spec.PartitionAddresses = partitionAddresses spec.StartTime = startTime processorID := int32(0) - proc, err := newStreamIngestionDataProcessor(&flowCtx, processorID, spec, &post, out) + proc, err := newStreamIngestionDataProcessor(&flowCtx, processorID, spec, nil /* input */, &post, + out) require.NoError(t, err) sip, ok := proc.(*streamIngestionProcessor) if !ok { diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_test.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_test.go index 90cca8516301..9771af04d85f 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_test.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_test.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" _ "github.com/cockroachdb/cockroach/pkg/ccl/kvccl/kvtenantccl" "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl" + _ "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/streamingutils" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" @@ -35,10 +36,30 @@ import ( "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/timeutil" "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" ) +func getHighWaterFromProgress( + t *testing.T, sqlDB *sqlutils.SQLRunner, streamJobID string, +) hlc.Timestamp { + progress := &jobspb.Progress{} + var streamProgress []byte + sqlDB.QueryRow( + t, `SELECT progress FROM system.jobs WHERE id=$1`, streamJobID, + ).Scan(&streamProgress) + + if err := protoutil.Unmarshal(streamProgress, progress); err != nil { + t.Fatal("cannot unmarshal job progress from system.jobs") + } + highWaterTimestamp := progress.GetHighWater() + if highWaterTimestamp == nil { + t.Fatal(errors.New("expected the highWaterTimestamp written to progress to be non-nil")) + } + return *highWaterTimestamp +} + // TestStreamIngestionJobWithRandomClient creates a stream ingestion job that is // fed KVs from the random stream client. After receiving a certain number of // resolved timestamp events the test cancels the job to tear down the flow, and @@ -121,15 +142,16 @@ func TestStreamIngestionJobWithRandomClient(t *testing.T) { <-cancelJobCh close(cancelJobCh) - // Canceling the job should shutdown the ingestion processors via a context - // cancellation, and subsequently rollback data above our frontier - // timestamp. - // TODO(adityamaru): Change this to cutover once we have cutover logic in - // place. - _, err = conn.Exec(`CANCEL JOB $1`, streamJobID) + curHighWater := getHighWaterFromProgress(t, sqlDB, streamJobID) + tz := timeutil.Unix(0, curHighWater.WallTime).UTC() + + // Signaling the job to cutover should shutdown the ingestion processors and + // trigger a revert range. + _, err = conn.Exec(`SELECT crdb_internal.complete_stream_ingestion_job($1,$2)`, streamJobID, + tz) require.NoError(t, err) // We expect the statement to fail. - if err := <-errCh; err == nil { + if err := <-errCh; err != nil { t.Fatal(err) } @@ -137,34 +159,19 @@ func TestStreamIngestionJobWithRandomClient(t *testing.T) { testutils.SucceedsSoon(t, func() error { var status string sqlDB.QueryRow(t, `SELECT status FROM system.jobs WHERE id = $1`, streamJobID).Scan(&status) - if jobs.Status(status) != jobs.StatusCanceled { - return errors.New("job not in canceled state") + if jobs.Status(status) != jobs.StatusSucceeded { + return errors.New("job not in a succeeded state") } return nil }) - progress := &jobspb.Progress{} - var streamProgress []byte - sqlDB.QueryRow( - t, `SELECT progress FROM system.jobs WHERE id=$1`, streamJobID, - ).Scan(&streamProgress) - - if err := protoutil.Unmarshal(streamProgress, progress); err != nil { - t.Fatal("cannot unmarshal job progress from system.jobs") - } - highWaterTimestamp := progress.GetHighWater() - if highWaterTimestamp == nil { - t.Fatal(errors.New("expected the highWaterTimestamp written to progress to be non-nil")) - } - ts := *highWaterTimestamp - require.True(t, !ts.IsEmpty()) - // Check the validator for any failures. for _, err := range streamValidator.failures() { t.Fatal(err) } tenantPrefix := keys.MakeTenantPrefix(roachpb.MakeTenantID(uint64(tenantID))) + ts := hlc.Timestamp{WallTime: tz.UnixNano()} maxIngestedTS := assertExactlyEqualKVs(t, tc, streamValidator, ts, tenantPrefix) //Sanity check that the max ts in the store is less than the ts stored in the diff --git a/pkg/ccl/streamingccl/streamingutils/utils.go b/pkg/ccl/streamingccl/streamingutils/utils.go index 2056bac36bdd..7b25197491be 100644 --- a/pkg/ccl/streamingccl/streamingutils/utils.go +++ b/pkg/ccl/streamingccl/streamingutils/utils.go @@ -50,6 +50,31 @@ func doCompleteIngestion( return errors.Newf("job %d: not of expected type StreamIngest", jobID) } + // Check that the supplied cutover time is a valid one. + // TODO(adityamaru): This will change once we allow a future cutover time to + // be specified. + hw := progress.GetHighWater() + if hw == nil { + return errors.Newf("cannot cutover to a timestamp %s that is after the latest resolved time"+ + " %s for job %d", cutoverTimestamp.String(), hlc.Timestamp{}.String(), jobID) + } + + highWaterTimestamp := *hw + if highWaterTimestamp.Less(cutoverTimestamp) { + return errors.Newf("cannot cutover to a timestamp %s that is after the latest resolved time"+ + " %s for job %d", cutoverTimestamp.String(), highWaterTimestamp.String(), jobID) + } + + // Reject setting a cutover time, if an earlier request to cutover has already + // been set. + // 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 !sp.StreamIngest.CutoverTime.IsEmpty() { + return errors.Newf("cutover timestamp already set to %s, "+ + "job %d is in the process of cutting over", sp.StreamIngest.CutoverTime.String(), jobID) + } + // Update the sentinel being polled by the stream ingestion job to // check if a complete has been signaled. sp.StreamIngest.CutoverTime = cutoverTimestamp diff --git a/pkg/ccl/streamingccl/streamingutils/utils_test.go b/pkg/ccl/streamingccl/streamingutils/utils_test.go index f4b89fd1100e..c8e07ba727c0 100644 --- a/pkg/ccl/streamingccl/streamingutils/utils_test.go +++ b/pkg/ccl/streamingccl/streamingutils/utils_test.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" @@ -59,14 +60,40 @@ func TestCutoverBuiltin(t *testing.T) { require.True(t, ok) require.True(t, sp.StreamIngest.CutoverTime.IsEmpty()) + // This should fail since no highwatermark is set on the progress. cutoverTime := timeutil.Now() - var jobID int + _, err = db.ExecContext( + ctx, + `SELECT crdb_internal.complete_stream_ingestion_job($1, $2)`, + *job.ID(), cutoverTime) + require.Error(t, err, "cannot cutover to a timestamp") + + var highWater time.Time + err = job.HighWaterProgressed(ctx, nil /* txn */, func(ctx context.Context, txn *kv.Txn, + details jobspb.ProgressDetails) (hlc.Timestamp, error) { + highWater = timeutil.Now() + hlcHighWater := hlc.Timestamp{WallTime: highWater.UnixNano()} + return hlcHighWater, nil + }) + require.NoError(t, err) + + // This should fail since the highwatermark is less than the cutover time + // passed to the builtin. + cutoverTime = timeutil.Now() + _, err = db.ExecContext( + ctx, + `SELECT crdb_internal.complete_stream_ingestion_job($1, $2)`, + *job.ID(), cutoverTime) + require.Error(t, err, "cannot cutover to a timestamp") + + // This should succeed since the highwatermark is equal to the cutover time. + var jobID int64 err = db.QueryRowContext( ctx, `SELECT crdb_internal.complete_stream_ingestion_job($1, $2)`, - *job.ID(), cutoverTime).Scan(&jobID) + *job.ID(), highWater).Scan(&jobID) require.NoError(t, err) - require.Equal(t, *job.ID(), int64(jobID)) + require.Equal(t, *job.ID(), jobID) // Check that sentinel is set on the job progress. sj, err := registry.LoadJob(ctx, *job.ID()) @@ -77,5 +104,5 @@ func TestCutoverBuiltin(t *testing.T) { // The builtin only offers microsecond precision and so we must account for // that when comparing against our chosen time. cutoverTime = cutoverTime.Round(time.Microsecond) - require.Equal(t, hlc.Timestamp{WallTime: cutoverTime.UnixNano()}, sp.StreamIngest.CutoverTime) + require.Equal(t, hlc.Timestamp{WallTime: highWater.UnixNano()}, sp.StreamIngest.CutoverTime) } diff --git a/pkg/sql/execinfrapb/processors.pb.go b/pkg/sql/execinfrapb/processors.pb.go index 0676136e38be..d2020547ce11 100644 --- a/pkg/sql/execinfrapb/processors.pb.go +++ b/pkg/sql/execinfrapb/processors.pb.go @@ -77,7 +77,7 @@ func (m *ProcessorSpec) Reset() { *m = ProcessorSpec{} } func (m *ProcessorSpec) String() string { return proto.CompactTextString(m) } func (*ProcessorSpec) ProtoMessage() {} func (*ProcessorSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_5e45e1d8c39180fc, []int{0} + return fileDescriptor_processors_8e17ec41849e88d7, []int{0} } func (m *ProcessorSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -135,13 +135,14 @@ type ProcessorCoreUnion struct { Filterer *FiltererSpec `protobuf:"bytes,34,opt,name=filterer" json:"filterer,omitempty"` StreamIngestionData *StreamIngestionDataSpec `protobuf:"bytes,35,opt,name=streamIngestionData" json:"streamIngestionData,omitempty"` StreamIngestionFrontier *StreamIngestionFrontierSpec `protobuf:"bytes,36,opt,name=streamIngestionFrontier" json:"streamIngestionFrontier,omitempty"` + StreamIngestionPolling *StreamIngestionPollingSpec `protobuf:"bytes,37,opt,name=streamIngestionPolling" json:"streamIngestionPolling,omitempty"` } func (m *ProcessorCoreUnion) Reset() { *m = ProcessorCoreUnion{} } func (m *ProcessorCoreUnion) String() string { return proto.CompactTextString(m) } func (*ProcessorCoreUnion) ProtoMessage() {} func (*ProcessorCoreUnion) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_5e45e1d8c39180fc, []int{1} + return fileDescriptor_processors_8e17ec41849e88d7, []int{1} } func (m *ProcessorCoreUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -176,7 +177,7 @@ func (m *NoopCoreSpec) Reset() { *m = NoopCoreSpec{} } func (m *NoopCoreSpec) String() string { return proto.CompactTextString(m) } func (*NoopCoreSpec) ProtoMessage() {} func (*NoopCoreSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_5e45e1d8c39180fc, []int{2} + return fileDescriptor_processors_8e17ec41849e88d7, []int{2} } func (m *NoopCoreSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -218,7 +219,7 @@ func (m *LocalPlanNodeSpec) Reset() { *m = LocalPlanNodeSpec{} } func (m *LocalPlanNodeSpec) String() string { return proto.CompactTextString(m) } func (*LocalPlanNodeSpec) ProtoMessage() {} func (*LocalPlanNodeSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_5e45e1d8c39180fc, []int{3} + return fileDescriptor_processors_8e17ec41849e88d7, []int{3} } func (m *LocalPlanNodeSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -251,7 +252,7 @@ func (m *MetadataTestSenderSpec) Reset() { *m = MetadataTestSenderSpec{} func (m *MetadataTestSenderSpec) String() string { return proto.CompactTextString(m) } func (*MetadataTestSenderSpec) ProtoMessage() {} func (*MetadataTestSenderSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_5e45e1d8c39180fc, []int{4} + return fileDescriptor_processors_8e17ec41849e88d7, []int{4} } func (m *MetadataTestSenderSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -284,7 +285,7 @@ func (m *MetadataTestReceiverSpec) Reset() { *m = MetadataTestReceiverSp func (m *MetadataTestReceiverSpec) String() string { return proto.CompactTextString(m) } func (*MetadataTestReceiverSpec) ProtoMessage() {} func (*MetadataTestReceiverSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_5e45e1d8c39180fc, []int{5} + return fileDescriptor_processors_8e17ec41849e88d7, []int{5} } func (m *MetadataTestReceiverSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -768,6 +769,18 @@ func (m *ProcessorCoreUnion) MarshalTo(dAtA []byte) (int, error) { } i += n34 } + if m.StreamIngestionPolling != nil { + dAtA[i] = 0xaa + i++ + dAtA[i] = 0x2 + i++ + i = encodeVarintProcessors(dAtA, i, uint64(m.StreamIngestionPolling.Size())) + n35, err := m.StreamIngestionPolling.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n35 + } return i, nil } @@ -1048,6 +1061,10 @@ func (m *ProcessorCoreUnion) Size() (n int) { l = m.StreamIngestionFrontier.Size() n += 2 + l + sovProcessors(uint64(l)) } + if m.StreamIngestionPolling != nil { + l = m.StreamIngestionPolling.Size() + n += 2 + l + sovProcessors(uint64(l)) + } return n } @@ -1209,6 +1226,9 @@ func (this *ProcessorCoreUnion) GetValue() interface{} { if this.StreamIngestionFrontier != nil { return this.StreamIngestionFrontier } + if this.StreamIngestionPolling != nil { + return this.StreamIngestionPolling + } return nil } @@ -1278,6 +1298,8 @@ func (this *ProcessorCoreUnion) SetValue(value interface{}) bool { this.StreamIngestionData = vt case *StreamIngestionFrontierSpec: this.StreamIngestionFrontier = vt + case *StreamIngestionPollingSpec: + this.StreamIngestionPolling = vt default: return false } @@ -2609,6 +2631,39 @@ func (m *ProcessorCoreUnion) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 37: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StreamIngestionPolling", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessors + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthProcessors + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.StreamIngestionPolling == nil { + m.StreamIngestionPolling = &StreamIngestionPollingSpec{} + } + if err := m.StreamIngestionPolling.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipProcessors(dAtA[iNdEx:]) @@ -3061,89 +3116,90 @@ var ( ) func init() { - proto.RegisterFile("sql/execinfrapb/processors.proto", fileDescriptor_processors_5e45e1d8c39180fc) + proto.RegisterFile("sql/execinfrapb/processors.proto", fileDescriptor_processors_8e17ec41849e88d7) } -var fileDescriptor_processors_5e45e1d8c39180fc = []byte{ - // 1265 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x97, 0xdf, 0x72, 0x13, 0xb7, - 0x17, 0xc7, 0xe3, 0xc4, 0x24, 0xb6, 0x9c, 0x10, 0x23, 0x02, 0xe8, 0xe7, 0x5f, 0xeb, 0xa4, 0x2e, - 0xb4, 0x2e, 0x05, 0xd3, 0x32, 0x6d, 0x2f, 0x98, 0x76, 0x5a, 0x6c, 0x97, 0xc9, 0xa6, 0x34, 0xd0, - 0x75, 0x0a, 0x33, 0xdc, 0xb8, 0xca, 0xae, 0xe2, 0x2c, 0xac, 0x57, 0x8b, 0x24, 0x13, 0xa0, 0x2f, - 0xd1, 0x47, 0xe8, 0x03, 0xf4, 0x41, 0x72, 0xd5, 0xe1, 0x92, 0x2b, 0xa6, 0x35, 0x2f, 0xd2, 0xd1, - 0xd1, 0x7a, 0x77, 0xed, 0xc4, 0xbb, 0xb9, 0xc9, 0x38, 0xf2, 0xf7, 0xfb, 0xd1, 0xd1, 0x39, 0x47, - 0x7f, 0x8c, 0xb6, 0xe4, 0x73, 0xff, 0x16, 0x7b, 0xc9, 0x1c, 0x2f, 0x38, 0x10, 0x34, 0xdc, 0xbf, - 0x15, 0x0a, 0xee, 0x30, 0x29, 0xb9, 0x90, 0xad, 0x50, 0x70, 0xc5, 0x31, 0x71, 0xb8, 0xf3, 0x4c, - 0x70, 0xea, 0x1c, 0xb6, 0xe4, 0x73, 0xbf, 0xe5, 0x7a, 0x52, 0xc9, 0xe7, 0xbe, 0x18, 0x05, 0xb5, - 0xda, 0xac, 0xd7, 0xa5, 0x8a, 0x1a, 0x57, 0xed, 0xda, 0x7c, 0x6e, 0x7f, 0x9f, 0x4a, 0x16, 0xc9, - 0xae, 0x66, 0xc8, 0xf4, 0x6c, 0x46, 0xd5, 0xcc, 0x82, 0x8d, 0xfc, 0x67, 0x7d, 0x8f, 0x47, 0xca, - 0x1b, 0x19, 0x4a, 0xe7, 0x90, 0x06, 0x03, 0x76, 0xc0, 0x98, 0x2b, 0xcf, 0xa0, 0x56, 0x74, 0xdf, - 0x67, 0x7d, 0xa9, 0xa8, 0x9a, 0xa8, 0x2f, 0x69, 0xb5, 0x7a, 0x15, 0x32, 0x69, 0xfe, 0x46, 0xc3, - 0x1b, 0x03, 0x3e, 0xe0, 0xf0, 0xf1, 0x96, 0xfe, 0x64, 0x46, 0x1b, 0x7f, 0x2f, 0xa1, 0xb5, 0x87, - 0x13, 0x5a, 0x2f, 0x64, 0x0e, 0xee, 0xa0, 0x73, 0x5e, 0x10, 0x8e, 0x14, 0x29, 0x6c, 0x2d, 0x35, - 0x2b, 0xb7, 0x3f, 0x6d, 0xcd, 0xcb, 0x6b, 0xcb, 0xd2, 0xb2, 0xde, 0xab, 0xc0, 0xd1, 0xbe, 0x76, - 0xf1, 0xf8, 0xdd, 0xe6, 0x82, 0x6d, 0xbc, 0xf8, 0x1e, 0x2a, 0x3a, 0x5c, 0x30, 0xb2, 0xb8, 0x55, - 0x68, 0x56, 0x6e, 0xdf, 0x98, 0xcf, 0x88, 0xe7, 0xee, 0x70, 0xc1, 0x7e, 0x0d, 0x3c, 0x1e, 0x44, - 0x20, 0xf0, 0xe3, 0x6d, 0xb4, 0xcc, 0x47, 0x4a, 0x47, 0xb3, 0x04, 0xd1, 0x5c, 0x9f, 0x4f, 0x7a, - 0x00, 0x3a, 0x9b, 0x8f, 0x14, 0x13, 0xa9, 0x80, 0x22, 0x3f, 0xee, 0xa0, 0x62, 0xc8, 0xa5, 0x22, - 0x45, 0x88, 0xe8, 0xb3, 0x8c, 0x88, 0xb8, 0x54, 0x51, 0x54, 0x29, 0x0c, 0x98, 0xf1, 0x75, 0x54, - 0x92, 0x8a, 0x0e, 0x58, 0xdf, 0x73, 0xc9, 0xb9, 0xad, 0x42, 0xf3, 0x5c, 0x7b, 0x5d, 0x7f, 0x3b, - 0x7e, 0xb7, 0xb9, 0xd2, 0xd3, 0xe3, 0x56, 0xd7, 0x5e, 0x01, 0x81, 0xe5, 0xe2, 0x6f, 0xd0, 0x6a, - 0x5c, 0x26, 0xad, 0x5f, 0x06, 0xfd, 0xc5, 0x48, 0x5f, 0x89, 0x17, 0x6e, 0x75, 0xed, 0x4a, 0x2c, - 0xb4, 0x5c, 0xfc, 0x1d, 0x5a, 0x15, 0x4c, 0x8e, 0x7c, 0xd5, 0x87, 0xea, 0x91, 0x15, 0x58, 0x78, - 0x6d, 0x26, 0x60, 0xc9, 0x86, 0x2d, 0x53, 0xdd, 0x3d, 0xbb, 0x62, 0xf4, 0x7b, 0xfa, 0xdf, 0xc6, - 0x5f, 0x1b, 0x08, 0x9f, 0x4c, 0x2a, 0xbe, 0x83, 0x8a, 0x01, 0xe7, 0x21, 0x29, 0xc0, 0xf2, 0x3f, - 0x99, 0xbf, 0xfc, 0x5d, 0xce, 0x43, 0x6d, 0xd3, 0x6b, 0xb7, 0xc1, 0x83, 0x7f, 0x42, 0x15, 0xe8, - 0x32, 0x9b, 0x51, 0x97, 0x89, 0xa8, 0xa6, 0x19, 0x19, 0xdc, 0x4b, 0xc4, 0x40, 0x49, 0xbb, 0xf1, - 0x36, 0x42, 0x4f, 0xb9, 0x17, 0x44, 0xac, 0x25, 0x60, 0x35, 0xe7, 0xb3, 0x76, 0x62, 0x2d, 0xa0, - 0x52, 0x5e, 0xfc, 0x2d, 0x5a, 0x96, 0x5c, 0x28, 0x26, 0xa2, 0x9a, 0x5e, 0x9d, 0x4f, 0xe9, 0x81, - 0x0e, 0x08, 0x91, 0x47, 0xc7, 0x41, 0x07, 0x03, 0xc1, 0x06, 0x54, 0x71, 0x01, 0xc5, 0xcc, 0x8c, - 0xe3, 0x6e, 0xac, 0x35, 0x71, 0x24, 0x5e, 0xdc, 0x46, 0x25, 0x2d, 0xf4, 0x02, 0x47, 0x91, 0x95, - 0xbc, 0xf4, 0x76, 0x23, 0x25, 0x50, 0x62, 0x9f, 0x4e, 0xf1, 0x90, 0x89, 0x01, 0xd3, 0xcb, 0x65, - 0x82, 0x94, 0xf2, 0x52, 0xfc, 0x73, 0x22, 0x36, 0x29, 0x4e, 0xb9, 0xf5, 0xd2, 0x0e, 0xa9, 0x3c, - 0x8c, 0x58, 0xe5, 0xbc, 0xa5, 0x6d, 0xc7, 0x5a, 0xb3, 0xb4, 0xc4, 0x8b, 0x7f, 0x40, 0xcb, 0x2f, - 0xa8, 0x3f, 0x62, 0x92, 0xa0, 0x3c, 0xca, 0x23, 0xd0, 0xc5, 0x9d, 0x13, 0xf9, 0x74, 0x2c, 0xfb, - 0xd4, 0x79, 0x76, 0xe0, 0xf9, 0x3e, 0x13, 0xa4, 0x92, 0x47, 0x69, 0xc7, 0x5a, 0x13, 0x4b, 0xe2, - 0xc5, 0xf7, 0x11, 0x12, 0x8c, 0xba, 0xd6, 0x30, 0xe4, 0x42, 0x91, 0xb5, 0xbc, 0x83, 0xc5, 0x8e, - 0xb5, 0x5d, 0xaa, 0xa8, 0xa1, 0x25, 0x7e, 0xfc, 0x3d, 0x5a, 0xe9, 0xd1, 0x61, 0xa8, 0x83, 0x5a, - 0x07, 0xd4, 0xb5, 0x8c, 0xee, 0x31, 0x42, 0x60, 0x4c, 0x5c, 0xf8, 0x09, 0xaa, 0x9a, 0x8f, 0x49, - 0x67, 0x90, 0x2a, 0x90, 0x5a, 0x79, 0xa4, 0x99, 0x5e, 0x3a, 0xc1, 0xc1, 0xbf, 0x21, 0x3c, 0x64, - 0x8a, 0xea, 0x6b, 0x6a, 0x8f, 0x49, 0xd5, 0x63, 0x81, 0xde, 0x2b, 0x18, 0xe8, 0x5f, 0x64, 0x35, - 0xc5, 0xac, 0x07, 0xf8, 0xa7, 0xb0, 0xf0, 0x01, 0xda, 0x48, 0x8f, 0xda, 0xcc, 0x61, 0xde, 0x0b, - 0x26, 0xc8, 0x45, 0x98, 0xe3, 0xf6, 0xd9, 0xe6, 0x98, 0xb8, 0x60, 0x96, 0x53, 0x79, 0xf8, 0x47, - 0x54, 0xee, 0xf4, 0x1e, 0x3d, 0x16, 0x9e, 0xde, 0xa6, 0x1b, 0x00, 0xcf, 0xb8, 0x50, 0x62, 0x29, - 0x10, 0x13, 0x27, 0xde, 0x45, 0xab, 0xaf, 0xbd, 0xc1, 0x6b, 0x3a, 0x88, 0x7a, 0xfa, 0x12, 0x90, - 0x32, 0x2e, 0x83, 0x27, 0x29, 0x35, 0xc0, 0xa6, 0xfc, 0xba, 0x2b, 0x43, 0xc1, 0x9f, 0x32, 0x47, - 0xf5, 0x98, 0x22, 0x97, 0xf3, 0xba, 0xf2, 0x61, 0xac, 0x35, 0x7d, 0x94, 0x78, 0xf5, 0xe6, 0x3f, - 0xf2, 0x02, 0x97, 0x1f, 0x31, 0x41, 0xae, 0xe4, 0x6d, 0xfe, 0xc7, 0x91, 0xd2, 0x6c, 0xfe, 0x89, - 0x0f, 0xff, 0x82, 0xd6, 0x7c, 0xee, 0x50, 0xff, 0xa1, 0x4f, 0x83, 0x5d, 0xee, 0x32, 0x42, 0x00, - 0xf4, 0xf9, 0x7c, 0xd0, 0xfd, 0xb4, 0x1c, 0x68, 0xd3, 0x04, 0xdd, 0x9d, 0xe6, 0x19, 0x91, 0xea, - 0xce, 0xff, 0xe5, 0x75, 0x67, 0x67, 0xc6, 0x61, 0xba, 0x73, 0x96, 0x83, 0xf7, 0xd0, 0x79, 0x33, - 0x76, 0x4f, 0xf0, 0x40, 0x79, 0x4c, 0x90, 0x5a, 0xde, 0x66, 0xec, 0x4c, 0xe9, 0x81, 0x3b, 0xc3, - 0xd0, 0x25, 0xe1, 0xc2, 0xf5, 0x02, 0xea, 0x7b, 0xea, 0x15, 0xf9, 0x7f, 0x5e, 0x49, 0x1e, 0xc4, - 0x5a, 0x53, 0x92, 0xc4, 0xab, 0xd3, 0xa9, 0x1f, 0x5b, 0x36, 0x3f, 0x8a, 0xfa, 0xee, 0x83, 0xbc, - 0x74, 0xb6, 0xd3, 0x72, 0x93, 0xce, 0x29, 0x82, 0x4e, 0xa7, 0x17, 0xbc, 0x60, 0x42, 0x31, 0xf7, - 0x9e, 0xe7, 0x2b, 0x26, 0x98, 0x20, 0x1f, 0xe6, 0xa5, 0xd3, 0x9a, 0x71, 0x98, 0x74, 0xce, 0x72, - 0x74, 0x3a, 0x27, 0x63, 0x51, 0x77, 0xd7, 0xf3, 0xd2, 0x69, 0x4d, 0xe9, 0x4d, 0x3a, 0xa7, 0x19, - 0x93, 0x73, 0x77, 0x14, 0xea, 0xd3, 0x8f, 0x6c, 0x9e, 0xe5, 0xdc, 0x35, 0xda, 0xe4, 0xdc, 0x35, - 0xff, 0xe3, 0xc7, 0x68, 0x5d, 0x86, 0xbe, 0xa7, 0xee, 0x06, 0x6e, 0xcf, 0xa1, 0x4a, 0x27, 0x74, - 0x0b, 0x70, 0x37, 0x33, 0xce, 0xb9, 0x69, 0x03, 0x30, 0x67, 0x29, 0xfa, 0xce, 0x13, 0x4c, 0x2a, - 0x2e, 0x18, 0xc4, 0xf8, 0x51, 0xde, 0x9d, 0x67, 0x27, 0x62, 0x73, 0xe7, 0xa5, 0xdc, 0x7a, 0x1f, - 0x1e, 0x4c, 0x2a, 0xd3, 0xc8, 0xdb, 0x87, 0x53, 0x15, 0x89, 0x7d, 0xd8, 0x41, 0x17, 0xa5, 0x12, - 0x8c, 0x0e, 0xad, 0x60, 0xc0, 0xa4, 0xf2, 0x78, 0x00, 0x81, 0x7d, 0x0c, 0xb8, 0x2f, 0x33, 0x56, - 0x7b, 0xd2, 0x04, 0xe4, 0xd3, 0x68, 0x98, 0xa3, 0x2b, 0x33, 0xc3, 0xf1, 0x36, 0xba, 0x0a, 0x13, - 0x7d, 0x7d, 0xe6, 0x89, 0xa6, 0xf6, 0xd3, 0x3c, 0xea, 0x9d, 0xe2, 0xf1, 0x9f, 0x9b, 0x85, 0x9d, - 0x62, 0xe9, 0x7c, 0x75, 0x7d, 0xa7, 0x58, 0xba, 0x50, 0xc5, 0x3b, 0xc5, 0xd2, 0x72, 0x75, 0x65, - 0xa7, 0x58, 0x5a, 0xad, 0xae, 0x35, 0xce, 0xa3, 0xd5, 0xf4, 0x8b, 0xaf, 0xf1, 0x3b, 0xba, 0x70, - 0xe2, 0x70, 0xc1, 0x4d, 0xb4, 0x6a, 0xf3, 0xa3, 0x1e, 0x1f, 0x09, 0x87, 0x59, 0xee, 0x4b, 0x78, - 0x44, 0xae, 0x45, 0x0f, 0xe3, 0xa9, 0x6f, 0x70, 0x03, 0x95, 0x77, 0x47, 0x43, 0xf8, 0x61, 0x20, - 0xe1, 0xa1, 0x38, 0x91, 0x25, 0xc3, 0x98, 0xa0, 0xe2, 0x2e, 0x1d, 0x32, 0x78, 0xfb, 0x95, 0x27, - 0xcf, 0x6b, 0x3d, 0xd2, 0xf8, 0x0a, 0x5d, 0x3e, 0xfd, 0x0e, 0xc3, 0x35, 0xb4, 0xe8, 0xb9, 0x30, - 0x6f, 0xb9, 0x8d, 0xa2, 0x27, 0xf4, 0xa2, 0xd5, 0xb5, 0x17, 0x3d, 0xb7, 0xb1, 0x8d, 0xc8, 0xbc, - 0x5b, 0x09, 0xdf, 0x40, 0x48, 0x02, 0xa5, 0xef, 0xb9, 0x12, 0x7e, 0xd1, 0x94, 0xdb, 0x6b, 0xe3, - 0x77, 0x9b, 0x65, 0xc3, 0xb6, 0xba, 0xd2, 0x2e, 0x1b, 0x81, 0xe5, 0xca, 0xf6, 0xcd, 0xe3, 0x7f, - 0xeb, 0x0b, 0xc7, 0xe3, 0x7a, 0xe1, 0xcd, 0xb8, 0x5e, 0x78, 0x3b, 0xae, 0x17, 0xfe, 0x19, 0xd7, - 0x0b, 0x7f, 0xbc, 0xaf, 0x2f, 0xbc, 0x79, 0x5f, 0x5f, 0x78, 0xfb, 0xbe, 0xbe, 0xf0, 0xa4, 0x92, - 0xfa, 0x15, 0xf6, 0x5f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x34, 0x76, 0x7e, 0x5f, 0x94, 0x0e, 0x00, - 0x00, +var fileDescriptor_processors_8e17ec41849e88d7 = []byte{ + // 1286 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x97, 0x4d, 0x73, 0xdb, 0x44, + 0x18, 0xc7, 0xe3, 0xc4, 0x4d, 0xec, 0x75, 0xd2, 0xa4, 0xdb, 0xb7, 0x25, 0x80, 0x13, 0x4c, 0x0b, + 0xa1, 0xb4, 0x2e, 0x74, 0x0a, 0x87, 0x0e, 0x0c, 0xd4, 0x09, 0x9d, 0x28, 0x94, 0x34, 0xc8, 0xa1, + 0x9d, 0xe9, 0x25, 0x6c, 0xa4, 0x8d, 0xb3, 0xad, 0xac, 0x55, 0x77, 0xd7, 0x4d, 0x5b, 0xbe, 0x04, + 0x67, 0x4e, 0x7c, 0x9c, 0x9c, 0x98, 0x1e, 0x7b, 0xea, 0x40, 0xf2, 0x45, 0x98, 0x7d, 0x56, 0x96, + 0x64, 0x25, 0x96, 0x72, 0xf1, 0xc8, 0xab, 0xff, 0xff, 0xb7, 0xcf, 0x3e, 0xfb, 0xec, 0x8b, 0xd0, + 0xb2, 0x7a, 0x11, 0xdc, 0x66, 0xaf, 0x98, 0xc7, 0xc3, 0x3d, 0x49, 0xa3, 0xdd, 0xdb, 0x91, 0x14, + 0x1e, 0x53, 0x4a, 0x48, 0xd5, 0x8e, 0xa4, 0xd0, 0x02, 0x13, 0x4f, 0x78, 0xcf, 0xa5, 0xa0, 0xde, + 0x7e, 0x5b, 0xbd, 0x08, 0xda, 0x3e, 0x57, 0x5a, 0xbd, 0x08, 0xe4, 0x20, 0x5c, 0x5c, 0xcc, 0x7b, + 0x7d, 0xaa, 0xa9, 0x75, 0x2d, 0x5e, 0x1f, 0xcf, 0xdd, 0xd9, 0xa5, 0x8a, 0xc5, 0xb2, 0x6b, 0x05, + 0x32, 0xd3, 0x9b, 0x55, 0xad, 0x14, 0xc1, 0x06, 0xc1, 0xf3, 0x1d, 0x2e, 0x62, 0xe5, 0xcd, 0x02, + 0xa5, 0xb7, 0x4f, 0xc3, 0x1e, 0xdb, 0x63, 0xcc, 0x57, 0x67, 0x50, 0x6b, 0xba, 0x1b, 0xb0, 0x1d, + 0xa5, 0xa9, 0x1e, 0xaa, 0x2f, 0x1b, 0xb5, 0x7e, 0x1d, 0x31, 0x65, 0x7f, 0xe3, 0xe6, 0x4b, 0x3d, + 0xd1, 0x13, 0xf0, 0x78, 0xdb, 0x3c, 0xd9, 0xd6, 0xd6, 0x3f, 0x53, 0x68, 0x6e, 0x6b, 0x48, 0xeb, + 0x46, 0xcc, 0xc3, 0xab, 0xe8, 0x1c, 0x0f, 0xa3, 0x81, 0x26, 0x95, 0xe5, 0xa9, 0x95, 0xc6, 0x9d, + 0xcf, 0xdb, 0xe3, 0xf2, 0xda, 0x76, 0x8c, 0xac, 0xfb, 0x3a, 0xf4, 0x8c, 0xaf, 0x53, 0x3d, 0x7c, + 0xbf, 0x34, 0xe1, 0x5a, 0x2f, 0x7e, 0x80, 0xaa, 0x9e, 0x90, 0x8c, 0x4c, 0x2e, 0x57, 0x56, 0x1a, + 0x77, 0x6e, 0x8e, 0x67, 0x24, 0x7d, 0xaf, 0x0a, 0xc9, 0x7e, 0x0b, 0xb9, 0x08, 0x63, 0x10, 0xf8, + 0xf1, 0x3a, 0x9a, 0x16, 0x03, 0x6d, 0xa2, 0x99, 0x82, 0x68, 0x6e, 0x8c, 0x27, 0x3d, 0x02, 0x9d, + 0x2b, 0x06, 0x9a, 0xc9, 0x4c, 0x40, 0xb1, 0x1f, 0xaf, 0xa2, 0x6a, 0x24, 0x94, 0x26, 0x55, 0x88, + 0xe8, 0x8b, 0x82, 0x88, 0x84, 0xd2, 0x71, 0x54, 0x19, 0x0c, 0x98, 0xf1, 0x0d, 0x54, 0x53, 0x9a, + 0xf6, 0xd8, 0x0e, 0xf7, 0xc9, 0xb9, 0xe5, 0xca, 0xca, 0xb9, 0xce, 0xbc, 0x79, 0x7b, 0xf4, 0x7e, + 0x69, 0xa6, 0x6b, 0xda, 0x9d, 0x35, 0x77, 0x06, 0x04, 0x8e, 0x8f, 0xbf, 0x45, 0xb3, 0xc9, 0x34, + 0x19, 0xfd, 0x34, 0xe8, 0x2f, 0xc6, 0xfa, 0x46, 0x32, 0x70, 0x67, 0xcd, 0x6d, 0x24, 0x42, 0xc7, + 0xc7, 0xdf, 0xa3, 0x59, 0xc9, 0xd4, 0x20, 0xd0, 0x3b, 0x30, 0x7b, 0x64, 0x06, 0x06, 0xbe, 0x98, + 0x0b, 0x58, 0xb1, 0x7e, 0xdb, 0xce, 0xee, 0xb6, 0xdb, 0xb0, 0xfa, 0x6d, 0xf3, 0xb7, 0xf5, 0xd7, + 0x65, 0x84, 0x4f, 0x26, 0x15, 0xdf, 0x43, 0xd5, 0x50, 0x88, 0x88, 0x54, 0x60, 0xf8, 0x9f, 0x8d, + 0x1f, 0xfe, 0xa6, 0x10, 0x91, 0xb1, 0x99, 0xb1, 0xbb, 0xe0, 0xc1, 0x3f, 0xa3, 0x06, 0x54, 0x99, + 0xcb, 0xa8, 0xcf, 0x64, 0x3c, 0xa7, 0x05, 0x19, 0xdc, 0x4e, 0xc5, 0x40, 0xc9, 0xba, 0xf1, 0x3a, + 0x42, 0xcf, 0x04, 0x0f, 0x63, 0xd6, 0x14, 0xb0, 0x56, 0xc6, 0xb3, 0x36, 0x12, 0x2d, 0xa0, 0x32, + 0x5e, 0xfc, 0x1d, 0x9a, 0x56, 0x42, 0x6a, 0x26, 0xe3, 0x39, 0xbd, 0x36, 0x9e, 0xd2, 0x05, 0x1d, + 0x10, 0x62, 0x8f, 0x89, 0x83, 0xf6, 0x7a, 0x92, 0xf5, 0xa8, 0x16, 0x12, 0x26, 0xb3, 0x30, 0x8e, + 0xfb, 0x89, 0xd6, 0xc6, 0x91, 0x7a, 0x71, 0x07, 0xd5, 0x8c, 0x90, 0x87, 0x9e, 0x26, 0x33, 0x65, + 0xe9, 0x5d, 0x8b, 0x95, 0x40, 0x49, 0x7c, 0x26, 0xc5, 0x7d, 0x26, 0x7b, 0xcc, 0x0c, 0x97, 0x49, + 0x52, 0x2b, 0x4b, 0xf1, 0x2f, 0xa9, 0xd8, 0xa6, 0x38, 0xe3, 0x36, 0x43, 0xdb, 0xa7, 0x6a, 0x3f, + 0x66, 0xd5, 0xcb, 0x86, 0xb6, 0x9e, 0x68, 0xed, 0xd0, 0x52, 0x2f, 0xfe, 0x11, 0x4d, 0xbf, 0xa4, + 0xc1, 0x80, 0x29, 0x82, 0xca, 0x28, 0x8f, 0x41, 0x97, 0x54, 0x4e, 0xec, 0x33, 0xb1, 0xec, 0x52, + 0xef, 0xf9, 0x1e, 0x0f, 0x02, 0x26, 0x49, 0xa3, 0x8c, 0xd2, 0x49, 0xb4, 0x36, 0x96, 0xd4, 0x8b, + 0x1f, 0x22, 0x24, 0x19, 0xf5, 0x9d, 0x7e, 0x24, 0xa4, 0x26, 0x73, 0x65, 0x1b, 0x8b, 0x9b, 0x68, + 0xd7, 0xa8, 0xa6, 0x96, 0x96, 0xfa, 0xf1, 0x0f, 0x68, 0xa6, 0x4b, 0xfb, 0x91, 0x09, 0x6a, 0x1e, + 0x50, 0xd7, 0x0b, 0xaa, 0xc7, 0x0a, 0x81, 0x31, 0x74, 0xe1, 0xa7, 0x68, 0xc1, 0x3e, 0xa6, 0x95, + 0x41, 0x16, 0x80, 0xd4, 0x2e, 0x23, 0xe5, 0x6a, 0xe9, 0x04, 0x07, 0xff, 0x8e, 0x70, 0x9f, 0x69, + 0x6a, 0x8e, 0xa9, 0x6d, 0xa6, 0x74, 0x97, 0x85, 0x66, 0xad, 0x60, 0xa0, 0x7f, 0x55, 0x54, 0x14, + 0x79, 0x0f, 0xf0, 0x4f, 0x61, 0xe1, 0x3d, 0x74, 0x29, 0xdb, 0xea, 0x32, 0x8f, 0xf1, 0x97, 0x4c, + 0x92, 0x8b, 0xd0, 0xc7, 0x9d, 0xb3, 0xf5, 0x31, 0x74, 0x41, 0x2f, 0xa7, 0xf2, 0xf0, 0x4f, 0xa8, + 0xbe, 0xda, 0x7d, 0xfc, 0x44, 0x72, 0xb3, 0x4c, 0x2f, 0x01, 0xbc, 0xe0, 0x40, 0x49, 0xa4, 0x40, + 0x4c, 0x9d, 0x78, 0x13, 0xcd, 0xbe, 0xe1, 0xbd, 0x37, 0xb4, 0x17, 0xd7, 0xf4, 0x65, 0x20, 0x15, + 0x1c, 0x06, 0x4f, 0x33, 0x6a, 0x80, 0x8d, 0xf8, 0x4d, 0x55, 0x46, 0x52, 0x3c, 0x63, 0x9e, 0xee, + 0x32, 0x4d, 0xae, 0x94, 0x55, 0xe5, 0x56, 0xa2, 0xb5, 0x75, 0x94, 0x7a, 0xcd, 0xe2, 0x3f, 0xe0, + 0xa1, 0x2f, 0x0e, 0x98, 0x24, 0x57, 0xcb, 0x16, 0xff, 0x93, 0x58, 0x69, 0x17, 0xff, 0xd0, 0x87, + 0x7f, 0x45, 0x73, 0x81, 0xf0, 0x68, 0xb0, 0x15, 0xd0, 0x70, 0x53, 0xf8, 0x8c, 0x10, 0x00, 0x7d, + 0x39, 0x1e, 0xf4, 0x30, 0x2b, 0x07, 0xda, 0x28, 0xc1, 0x54, 0xa7, 0xbd, 0x46, 0x64, 0xaa, 0xf3, + 0x83, 0xb2, 0xea, 0x5c, 0xcd, 0x39, 0x6c, 0x75, 0xe6, 0x39, 0x78, 0x1b, 0x9d, 0xb7, 0x6d, 0x0f, + 0xa4, 0x08, 0x35, 0x67, 0x92, 0x2c, 0x96, 0x2d, 0xc6, 0xd5, 0x11, 0x3d, 0x70, 0x73, 0x0c, 0x33, + 0x25, 0x42, 0xfa, 0x3c, 0xa4, 0x01, 0xd7, 0xaf, 0xc9, 0x87, 0x65, 0x53, 0xf2, 0x28, 0xd1, 0xda, + 0x29, 0x49, 0xbd, 0x26, 0x9d, 0xe6, 0xb2, 0xe5, 0x8a, 0x83, 0xb8, 0xee, 0x3e, 0x2a, 0x4b, 0x67, + 0x27, 0x2b, 0xb7, 0xe9, 0x1c, 0x21, 0x98, 0x74, 0xf2, 0xf0, 0x25, 0x93, 0x9a, 0xf9, 0x0f, 0x78, + 0xa0, 0x99, 0x64, 0x92, 0x7c, 0x5c, 0x96, 0x4e, 0x27, 0xe7, 0xb0, 0xe9, 0xcc, 0x73, 0x4c, 0x3a, + 0x87, 0x6d, 0x71, 0x75, 0x37, 0xcb, 0xd2, 0xe9, 0x8c, 0xe8, 0x6d, 0x3a, 0x47, 0x19, 0xc3, 0x7d, + 0x77, 0x10, 0x99, 0xdd, 0x8f, 0x2c, 0x9d, 0x65, 0xdf, 0xb5, 0xda, 0x74, 0xdf, 0xb5, 0xff, 0xf1, + 0x13, 0x34, 0xaf, 0xa2, 0x80, 0xeb, 0xfb, 0xa1, 0xdf, 0xf5, 0xa8, 0x36, 0x09, 0x5d, 0x06, 0xdc, + 0xad, 0x82, 0x7d, 0x6e, 0xd4, 0x00, 0xcc, 0x3c, 0xc5, 0x9c, 0x79, 0x92, 0x29, 0x2d, 0x24, 0x83, + 0x18, 0x3f, 0x29, 0x3b, 0xf3, 0xdc, 0x54, 0x6c, 0xcf, 0xbc, 0x8c, 0xdb, 0xac, 0xc3, 0xbd, 0xe1, + 0xcc, 0xb4, 0xca, 0xd6, 0xe1, 0xc8, 0x8c, 0x24, 0x3e, 0xec, 0xa1, 0x8b, 0x4a, 0x4b, 0x46, 0xfb, + 0x4e, 0xd8, 0x63, 0x4a, 0x73, 0x11, 0x42, 0x60, 0x9f, 0x02, 0xee, 0xeb, 0x82, 0xd1, 0x9e, 0x34, + 0x01, 0xf9, 0x34, 0x1a, 0x16, 0xe8, 0x6a, 0xae, 0x39, 0x59, 0x46, 0xd7, 0xa0, 0xa3, 0x6f, 0xce, + 0xdc, 0xd1, 0xc8, 0x7a, 0x1a, 0x47, 0xc5, 0x01, 0xba, 0x92, 0x7b, 0xb5, 0x25, 0x82, 0x80, 0x87, + 0x3d, 0x72, 0x1d, 0xfa, 0xbb, 0x7b, 0xe6, 0xfe, 0x62, 0x1f, 0x74, 0x37, 0x86, 0x79, 0xaf, 0x7a, + 0xf8, 0xf7, 0x52, 0x65, 0xa3, 0x5a, 0x3b, 0xbf, 0x30, 0xbf, 0x51, 0xad, 0x5d, 0x58, 0xc0, 0x1b, + 0xd5, 0xda, 0xf4, 0xc2, 0xcc, 0x46, 0xb5, 0x36, 0xbb, 0x30, 0xd7, 0x3a, 0x8f, 0x66, 0xb3, 0xf7, + 0xcb, 0xd6, 0x1f, 0xe8, 0xc2, 0x89, 0xad, 0x0c, 0xaf, 0xa0, 0x59, 0x57, 0x1c, 0x74, 0xc5, 0x40, + 0x7a, 0xcc, 0xf1, 0x5f, 0xc1, 0x95, 0x75, 0x2e, 0xbe, 0x86, 0x8f, 0xbc, 0xc1, 0x2d, 0x54, 0xdf, + 0x1c, 0xf4, 0xe1, 0x33, 0x44, 0xc1, 0xb5, 0x74, 0x28, 0x4b, 0x9b, 0x31, 0x41, 0xd5, 0x4d, 0xda, + 0x67, 0x70, 0xd3, 0xac, 0x0f, 0x2f, 0xf3, 0xa6, 0xa5, 0x75, 0x17, 0x5d, 0x39, 0xfd, 0xc4, 0xc4, + 0x8b, 0x68, 0x92, 0xfb, 0xd0, 0x6f, 0xbd, 0x83, 0xe2, 0x0b, 0xfb, 0xa4, 0xb3, 0xe6, 0x4e, 0x72, + 0xbf, 0xb5, 0x8e, 0xc8, 0xb8, 0x33, 0x10, 0xdf, 0x44, 0x48, 0x01, 0x65, 0x87, 0xfb, 0x0a, 0xbe, + 0x9f, 0xea, 0x9d, 0xb9, 0xa3, 0xf7, 0x4b, 0x75, 0xcb, 0x76, 0xd6, 0x94, 0x5b, 0xb7, 0x02, 0xc7, + 0x57, 0x9d, 0x5b, 0x87, 0xff, 0x35, 0x27, 0x0e, 0x8f, 0x9a, 0x95, 0xb7, 0x47, 0xcd, 0xca, 0xbb, + 0xa3, 0x66, 0xe5, 0xdf, 0xa3, 0x66, 0xe5, 0xcf, 0xe3, 0xe6, 0xc4, 0xdb, 0xe3, 0xe6, 0xc4, 0xbb, + 0xe3, 0xe6, 0xc4, 0xd3, 0x46, 0xe6, 0x9b, 0xef, 0xff, 0x00, 0x00, 0x00, 0xff, 0xff, 0x9f, 0x2f, + 0xcd, 0x5e, 0x02, 0x0f, 0x00, 0x00, } diff --git a/pkg/sql/execinfrapb/processors.proto b/pkg/sql/execinfrapb/processors.proto index f797f629fcf4..3b78322763f1 100644 --- a/pkg/sql/execinfrapb/processors.proto +++ b/pkg/sql/execinfrapb/processors.proto @@ -119,6 +119,7 @@ message ProcessorCoreUnion { optional FiltererSpec filterer = 34; optional StreamIngestionDataSpec streamIngestionData = 35; optional StreamIngestionFrontierSpec streamIngestionFrontier = 36; + optional StreamIngestionPollingSpec streamIngestionPolling = 37; reserved 6, 12; } diff --git a/pkg/sql/execinfrapb/processors_bulk_io.pb.go b/pkg/sql/execinfrapb/processors_bulk_io.pb.go index bbf197a9d798..3d3121354d88 100644 --- a/pkg/sql/execinfrapb/processors_bulk_io.pb.go +++ b/pkg/sql/execinfrapb/processors_bulk_io.pb.go @@ -73,7 +73,7 @@ func (x *FileCompression) UnmarshalJSON(data []byte) error { return nil } func (FileCompression) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_ac83a229416c6ca4, []int{0} + return fileDescriptor_processors_bulk_io_a1e65040804145d5, []int{0} } type BackfillerSpec_Type int32 @@ -112,7 +112,7 @@ func (x *BackfillerSpec_Type) UnmarshalJSON(data []byte) error { return nil } func (BackfillerSpec_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_ac83a229416c6ca4, []int{0, 0} + return fileDescriptor_processors_bulk_io_a1e65040804145d5, []int{0, 0} } // BackfillerSpec is the specification for a "schema change backfiller". @@ -148,7 +148,7 @@ func (m *BackfillerSpec) Reset() { *m = BackfillerSpec{} } func (m *BackfillerSpec) String() string { return proto.CompactTextString(m) } func (*BackfillerSpec) ProtoMessage() {} func (*BackfillerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_ac83a229416c6ca4, []int{0} + return fileDescriptor_processors_bulk_io_a1e65040804145d5, []int{0} } func (m *BackfillerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -188,7 +188,7 @@ func (m *JobProgress) Reset() { *m = JobProgress{} } func (m *JobProgress) String() string { return proto.CompactTextString(m) } func (*JobProgress) ProtoMessage() {} func (*JobProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_ac83a229416c6ca4, []int{1} + return fileDescriptor_processors_bulk_io_a1e65040804145d5, []int{1} } func (m *JobProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -251,7 +251,7 @@ func (m *ReadImportDataSpec) Reset() { *m = ReadImportDataSpec{} } func (m *ReadImportDataSpec) String() string { return proto.CompactTextString(m) } func (*ReadImportDataSpec) ProtoMessage() {} func (*ReadImportDataSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_ac83a229416c6ca4, []int{2} + return fileDescriptor_processors_bulk_io_a1e65040804145d5, []int{2} } func (m *ReadImportDataSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -289,7 +289,7 @@ func (m *ReadImportDataSpec_ImportTable) Reset() { *m = ReadImportDataSp func (m *ReadImportDataSpec_ImportTable) String() string { return proto.CompactTextString(m) } func (*ReadImportDataSpec_ImportTable) ProtoMessage() {} func (*ReadImportDataSpec_ImportTable) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_ac83a229416c6ca4, []int{2, 0} + return fileDescriptor_processors_bulk_io_a1e65040804145d5, []int{2, 0} } func (m *ReadImportDataSpec_ImportTable) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -328,7 +328,7 @@ func (m *StreamIngestionDataSpec) Reset() { *m = StreamIngestionDataSpec func (m *StreamIngestionDataSpec) String() string { return proto.CompactTextString(m) } func (*StreamIngestionDataSpec) ProtoMessage() {} func (*StreamIngestionDataSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_ac83a229416c6ca4, []int{3} + return fileDescriptor_processors_bulk_io_a1e65040804145d5, []int{3} } func (m *StreamIngestionDataSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -371,7 +371,7 @@ func (m *StreamIngestionFrontierSpec) Reset() { *m = StreamIngestionFron func (m *StreamIngestionFrontierSpec) String() string { return proto.CompactTextString(m) } func (*StreamIngestionFrontierSpec) ProtoMessage() {} func (*StreamIngestionFrontierSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_ac83a229416c6ca4, []int{4} + return fileDescriptor_processors_bulk_io_a1e65040804145d5, []int{4} } func (m *StreamIngestionFrontierSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -396,6 +396,39 @@ func (m *StreamIngestionFrontierSpec) XXX_DiscardUnknown() { var xxx_messageInfo_StreamIngestionFrontierSpec proto.InternalMessageInfo +type StreamIngestionPollingSpec struct { + JobID int64 `protobuf:"varint,1,opt,name=job_id,json=jobId" json:"job_id"` +} + +func (m *StreamIngestionPollingSpec) Reset() { *m = StreamIngestionPollingSpec{} } +func (m *StreamIngestionPollingSpec) String() string { return proto.CompactTextString(m) } +func (*StreamIngestionPollingSpec) ProtoMessage() {} +func (*StreamIngestionPollingSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bulk_io_a1e65040804145d5, []int{5} +} +func (m *StreamIngestionPollingSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StreamIngestionPollingSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *StreamIngestionPollingSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_StreamIngestionPollingSpec.Merge(dst, src) +} +func (m *StreamIngestionPollingSpec) XXX_Size() int { + return m.Size() +} +func (m *StreamIngestionPollingSpec) XXX_DiscardUnknown() { + xxx_messageInfo_StreamIngestionPollingSpec.DiscardUnknown(m) +} + +var xxx_messageInfo_StreamIngestionPollingSpec proto.InternalMessageInfo + type BackupDataSpec struct { Spans []roachpb.Span `protobuf:"bytes,1,rep,name=spans" json:"spans"` IntroducedSpans []roachpb.Span `protobuf:"bytes,2,rep,name=introduced_spans,json=introducedSpans" json:"introduced_spans"` @@ -417,7 +450,7 @@ func (m *BackupDataSpec) Reset() { *m = BackupDataSpec{} } func (m *BackupDataSpec) String() string { return proto.CompactTextString(m) } func (*BackupDataSpec) ProtoMessage() {} func (*BackupDataSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_ac83a229416c6ca4, []int{5} + return fileDescriptor_processors_bulk_io_a1e65040804145d5, []int{6} } func (m *BackupDataSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -456,7 +489,7 @@ func (m *RestoreSpanEntry) Reset() { *m = RestoreSpanEntry{} } func (m *RestoreSpanEntry) String() string { return proto.CompactTextString(m) } func (*RestoreSpanEntry) ProtoMessage() {} func (*RestoreSpanEntry) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_ac83a229416c6ca4, []int{6} + return fileDescriptor_processors_bulk_io_a1e65040804145d5, []int{7} } func (m *RestoreSpanEntry) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -494,7 +527,7 @@ func (m *RestoreDataSpec) Reset() { *m = RestoreDataSpec{} } func (m *RestoreDataSpec) String() string { return proto.CompactTextString(m) } func (*RestoreDataSpec) ProtoMessage() {} func (*RestoreDataSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_ac83a229416c6ca4, []int{7} + return fileDescriptor_processors_bulk_io_a1e65040804145d5, []int{8} } func (m *RestoreDataSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -528,7 +561,7 @@ func (m *SplitAndScatterSpec) Reset() { *m = SplitAndScatterSpec{} } func (m *SplitAndScatterSpec) String() string { return proto.CompactTextString(m) } func (*SplitAndScatterSpec) ProtoMessage() {} func (*SplitAndScatterSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_ac83a229416c6ca4, []int{8} + return fileDescriptor_processors_bulk_io_a1e65040804145d5, []int{9} } func (m *SplitAndScatterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -561,7 +594,7 @@ func (m *SplitAndScatterSpec_RestoreEntryChunk) Reset() { *m = SplitAndS func (m *SplitAndScatterSpec_RestoreEntryChunk) String() string { return proto.CompactTextString(m) } func (*SplitAndScatterSpec_RestoreEntryChunk) ProtoMessage() {} func (*SplitAndScatterSpec_RestoreEntryChunk) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_ac83a229416c6ca4, []int{8, 0} + return fileDescriptor_processors_bulk_io_a1e65040804145d5, []int{9, 0} } func (m *SplitAndScatterSpec_RestoreEntryChunk) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -608,7 +641,7 @@ func (m *CSVWriterSpec) Reset() { *m = CSVWriterSpec{} } func (m *CSVWriterSpec) String() string { return proto.CompactTextString(m) } func (*CSVWriterSpec) ProtoMessage() {} func (*CSVWriterSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_ac83a229416c6ca4, []int{9} + return fileDescriptor_processors_bulk_io_a1e65040804145d5, []int{10} } func (m *CSVWriterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -643,7 +676,7 @@ func (m *BulkRowWriterSpec) Reset() { *m = BulkRowWriterSpec{} } func (m *BulkRowWriterSpec) String() string { return proto.CompactTextString(m) } func (*BulkRowWriterSpec) ProtoMessage() {} func (*BulkRowWriterSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_ac83a229416c6ca4, []int{10} + return fileDescriptor_processors_bulk_io_a1e65040804145d5, []int{11} } func (m *BulkRowWriterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -678,6 +711,7 @@ func init() { proto.RegisterType((*ReadImportDataSpec_ImportTable)(nil), "cockroach.sql.distsqlrun.ReadImportDataSpec.ImportTable") proto.RegisterType((*StreamIngestionDataSpec)(nil), "cockroach.sql.distsqlrun.StreamIngestionDataSpec") proto.RegisterType((*StreamIngestionFrontierSpec)(nil), "cockroach.sql.distsqlrun.StreamIngestionFrontierSpec") + proto.RegisterType((*StreamIngestionPollingSpec)(nil), "cockroach.sql.distsqlrun.StreamIngestionPollingSpec") proto.RegisterType((*BackupDataSpec)(nil), "cockroach.sql.distsqlrun.BackupDataSpec") proto.RegisterMapType((map[uint64]bool)(nil), "cockroach.sql.distsqlrun.BackupDataSpec.PkIdsEntry") proto.RegisterMapType((map[string]string)(nil), "cockroach.sql.distsqlrun.BackupDataSpec.UrisByLocalityKvEntry") @@ -1036,6 +1070,27 @@ func (m *StreamIngestionFrontierSpec) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func (m *StreamIngestionPollingSpec) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *StreamIngestionPollingSpec) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0x8 + i++ + i = encodeVarintProcessorsBulkIo(dAtA, i, uint64(m.JobID)) + return i, nil +} + func (m *BackupDataSpec) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -1569,6 +1624,16 @@ func (m *StreamIngestionFrontierSpec) Size() (n int) { return n } +func (m *StreamIngestionPollingSpec) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 1 + sovProcessorsBulkIo(uint64(m.JobID)) + return n +} + func (m *BackupDataSpec) Size() (n int) { if m == nil { return 0 @@ -3018,6 +3083,75 @@ func (m *StreamIngestionFrontierSpec) Unmarshal(dAtA []byte) error { } return nil } +func (m *StreamIngestionPollingSpec) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsBulkIo + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: StreamIngestionPollingSpec: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: StreamIngestionPollingSpec: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field JobID", wireType) + } + m.JobID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsBulkIo + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.JobID |= (int64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipProcessorsBulkIo(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthProcessorsBulkIo + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *BackupDataSpec) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -4472,127 +4606,128 @@ var ( ) func init() { - proto.RegisterFile("sql/execinfrapb/processors_bulk_io.proto", fileDescriptor_processors_bulk_io_ac83a229416c6ca4) + proto.RegisterFile("sql/execinfrapb/processors_bulk_io.proto", fileDescriptor_processors_bulk_io_a1e65040804145d5) } -var fileDescriptor_processors_bulk_io_ac83a229416c6ca4 = []byte{ - // 1886 bytes of a gzipped FileDescriptorProto +var fileDescriptor_processors_bulk_io_a1e65040804145d5 = []byte{ + // 1902 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x58, 0x4d, 0x6f, 0x1b, 0xc7, 0xf9, 0xd7, 0xf2, 0x4d, 0xe4, 0x43, 0x53, 0xa6, 0xc6, 0x4e, 0xb2, 0x7f, 0xfd, 0x51, 0x49, 0x60, 0x62, 0x97, 0x75, 0x61, 0x12, 0xb1, 0xdb, 0xc2, 0x68, 0x9b, 0xb8, 0x22, 0x65, 0x39, 0x94, 0x12, 0x5b, 0x5d, 0x5a, 0x32, 0x10, 0xb4, 0x58, 0x0c, 0x77, 0x47, 0xd4, 0x98, 0xcb, 0x9d, 0xd5, 0xcc, 0xac, 0x64, 0xfa, 0xd2, 0x02, 0x3d, 0xf5, 0xd6, 0x8f, 0xd0, 0x43, 0x3f, 0x40, 0x2f, 0xfd, 0x04, 0xbd, 0xf8, 0x18, 0xf4, 0x14, 0xf4, 0x20, 0xb4, 0xf2, 0xb7, 0xc8, 0xa9, 0x98, 0xd9, 0x59, 0x6a, - 0x25, 0x4b, 0xb2, 0x14, 0x23, 0x17, 0x69, 0x39, 0x33, 0xbf, 0xdf, 0x3e, 0xef, 0xcf, 0x33, 0x0b, + 0x25, 0x4b, 0xb2, 0x14, 0x23, 0x17, 0x69, 0x39, 0x33, 0xbf, 0xdf, 0x3c, 0xef, 0xcf, 0xb3, 0x0b, 0x4d, 0xb1, 0x17, 0xb4, 0xc9, 0x4b, 0xe2, 0xd1, 0x70, 0x87, 0xe3, 0x68, 0xd0, 0x8e, 0x38, 0xf3, 0x88, 0x10, 0x8c, 0x0b, 0x77, 0x10, 0x07, 0x23, 0x97, 0xb2, 0x56, 0xc4, 0x99, 0x64, 0xc8, 0xf6, 0x98, 0x37, 0xe2, 0x0c, 0x7b, 0xbb, 0x2d, 0xb1, 0x17, 0xb4, 0x7c, 0x2a, 0xa4, 0xd8, 0x0b, 0x78, 0x1c, 0x2e, 0x7c, 0xf8, 0x82, 0x0d, 0x44, 0x5b, 0xfd, 0x89, 0x06, 0xfa, 0x5f, 0x82, 0x58, 0xb0, - 0xf5, 0xe9, 0x68, 0xd0, 0xa6, 0xec, 0xee, 0x0e, 0xe3, 0x63, 0x2c, 0xd3, 0x9d, 0x8f, 0xd5, 0x5b, - 0x3d, 0x2c, 0x71, 0xc0, 0x86, 0x6d, 0x9f, 0x08, 0x2f, 0x1a, 0xb4, 0x85, 0xe4, 0xb1, 0x27, 0x63, - 0x4e, 0x7c, 0x73, 0xe8, 0xd6, 0x45, 0xa2, 0x61, 0x41, 0xd2, 0xb7, 0xc4, 0x92, 0x06, 0xed, 0xdd, - 0xc0, 0x6b, 0x4b, 0x3a, 0x26, 0x42, 0xe2, 0x71, 0x64, 0x76, 0x6e, 0x0e, 0xd9, 0x90, 0xe9, 0xc7, - 0xb6, 0x7a, 0x32, 0xab, 0x28, 0x95, 0xca, 0xc7, 0x12, 0x9b, 0xb5, 0xf9, 0x74, 0x0d, 0x47, 0x34, - 0x59, 0x6a, 0xfc, 0xad, 0x00, 0x73, 0x1d, 0xec, 0x8d, 0x76, 0x68, 0x10, 0x10, 0xde, 0x8f, 0x88, - 0x87, 0x1e, 0x43, 0x41, 0x4e, 0x22, 0x62, 0x5b, 0xcb, 0x56, 0x73, 0xee, 0xde, 0xdd, 0xd6, 0x79, - 0x06, 0x69, 0x9d, 0xc4, 0xb5, 0x9e, 0x4d, 0x22, 0xd2, 0x29, 0xbc, 0x3e, 0x5c, 0x9a, 0x71, 0x34, - 0x01, 0xea, 0x40, 0x51, 0xe2, 0x41, 0x40, 0xec, 0xdc, 0xb2, 0xd5, 0xac, 0xde, 0xbb, 0x7d, 0x8a, - 0x49, 0xec, 0x05, 0x5a, 0xbf, 0x67, 0xea, 0xcc, 0x2a, 0x11, 0x1e, 0xa7, 0x91, 0x64, 0xdc, 0x50, - 0x24, 0x50, 0xf4, 0x08, 0x8a, 0x22, 0xc2, 0xa1, 0xb0, 0xf3, 0xcb, 0xf9, 0x66, 0xf5, 0xde, 0x4f, - 0xce, 0x97, 0x46, 0xd3, 0x38, 0x04, 0xfb, 0x4a, 0x1c, 0x1c, 0xa6, 0x34, 0x1a, 0x8d, 0x3e, 0x85, - 0xb2, 0x1f, 0x73, 0x2c, 0x29, 0x0b, 0xed, 0xc2, 0xb2, 0xd5, 0xcc, 0x77, 0x3e, 0x50, 0xdb, 0xdf, - 0x1d, 0x2e, 0xd5, 0x94, 0x39, 0x5b, 0xab, 0x66, 0xd3, 0x99, 0x1e, 0x43, 0x1f, 0x03, 0x78, 0xbb, - 0x71, 0x38, 0x72, 0x05, 0x7d, 0x45, 0xec, 0xa2, 0x06, 0x25, 0x9c, 0x15, 0xbd, 0xde, 0xa7, 0xaf, - 0x08, 0x7a, 0x08, 0x65, 0x4e, 0xb0, 0xbf, 0x22, 0x9e, 0xee, 0xd8, 0xb3, 0x5a, 0xcb, 0x1f, 0x65, - 0x24, 0x54, 0x2e, 0x6b, 0xed, 0x06, 0x5e, 0xeb, 0x59, 0xea, 0x32, 0xc3, 0x30, 0x05, 0x21, 0x01, - 0x37, 0x68, 0xe8, 0x93, 0x97, 0x44, 0xb8, 0x92, 0xb9, 0x03, 0x63, 0x51, 0xbb, 0xbc, 0x9c, 0x6f, - 0xd6, 0x3a, 0xdd, 0xef, 0x0e, 0x97, 0x1e, 0x0e, 0xa9, 0xdc, 0x8d, 0x07, 0x2d, 0x8f, 0x8d, 0xdb, - 0x53, 0x66, 0x7f, 0x70, 0xfc, 0xdc, 0x8e, 0x46, 0xc3, 0xf6, 0xdb, 0x81, 0xd6, 0xea, 0x29, 0xda, - 0xde, 0xaa, 0x33, 0x6f, 0xf8, 0x9f, 0xb1, 0xd4, 0x5f, 0x8d, 0x3b, 0x50, 0x50, 0xce, 0x42, 0x55, - 0x98, 0xed, 0x85, 0xfb, 0x38, 0xa0, 0x7e, 0x7d, 0x06, 0x01, 0x94, 0xba, 0x2c, 0x88, 0xc7, 0x61, - 0xdd, 0x42, 0x15, 0x28, 0x6a, 0x78, 0x3d, 0xb7, 0x5e, 0x28, 0x97, 0xea, 0xb3, 0x8d, 0x03, 0xa8, - 0xae, 0xb3, 0xc1, 0x26, 0x67, 0x43, 0x4e, 0x84, 0x40, 0x9f, 0x40, 0xe9, 0x05, 0x1b, 0xb8, 0xd4, - 0xd7, 0x41, 0x92, 0xef, 0xd4, 0x94, 0x56, 0x47, 0x87, 0x4b, 0xc5, 0x75, 0x36, 0xe8, 0xad, 0x3a, - 0xc5, 0x17, 0x6c, 0xd0, 0xf3, 0x51, 0x13, 0xae, 0x79, 0x2c, 0x94, 0x9c, 0x0e, 0x62, 0x6d, 0x78, - 0x15, 0x06, 0x39, 0x63, 0x81, 0x13, 0x3b, 0xc8, 0x86, 0x82, 0x08, 0x98, 0xb4, 0xf3, 0xcb, 0x56, - 0xb3, 0x98, 0xc6, 0x90, 0x5a, 0x69, 0xbc, 0x2e, 0x03, 0x52, 0x4e, 0xed, 0x8d, 0x23, 0xc6, 0xe5, - 0x2a, 0x96, 0x58, 0xc7, 0xe8, 0x2d, 0xa8, 0x0a, 0x3c, 0x8e, 0x02, 0x92, 0x78, 0x27, 0x97, 0xc1, - 0x41, 0xb2, 0xa1, 0xdd, 0xf3, 0x18, 0xca, 0x91, 0x91, 0xd9, 0x2e, 0x69, 0xf7, 0xdc, 0x3a, 0x3f, - 0x80, 0x32, 0x0a, 0xa6, 0x6e, 0x4a, 0xc1, 0xe8, 0x31, 0xe4, 0x63, 0x4e, 0xed, 0x59, 0x1d, 0x84, - 0x3f, 0x3f, 0x9f, 0xe3, 0x6d, 0x51, 0x5b, 0x5b, 0x9c, 0x3e, 0x0a, 0x25, 0x9f, 0x38, 0x8a, 0x01, - 0x7d, 0x06, 0xa5, 0xa4, 0x46, 0xd8, 0x65, 0x2d, 0xcf, 0x52, 0x86, 0xcb, 0x64, 0x67, 0xab, 0xf7, - 0x74, 0x8d, 0x06, 0x64, 0x4d, 0x1f, 0x33, 0x92, 0x18, 0x10, 0xda, 0x86, 0x92, 0xce, 0x0b, 0x61, - 0x57, 0xb4, 0x28, 0x0f, 0xae, 0x24, 0x8a, 0x4e, 0x11, 0xa1, 0xa5, 0xd1, 0xbc, 0x96, 0x63, 0xd8, - 0xd0, 0x43, 0xf8, 0x3f, 0x31, 0xa2, 0x91, 0x3b, 0xa6, 0x42, 0xd0, 0x70, 0xe8, 0xee, 0x30, 0x4e, - 0xe8, 0x30, 0x74, 0x47, 0x64, 0x22, 0x6c, 0x58, 0xb6, 0x9a, 0x65, 0x23, 0xc8, 0x87, 0xea, 0xd8, - 0x57, 0xc9, 0xa9, 0xb5, 0xe4, 0xd0, 0x06, 0x99, 0x08, 0x74, 0x07, 0x6a, 0x07, 0x38, 0x08, 0x54, - 0x32, 0x3d, 0xc1, 0x21, 0x13, 0x76, 0x35, 0x93, 0x30, 0x27, 0xb7, 0xd0, 0x3d, 0x98, 0xe7, 0x3a, - 0x4f, 0x37, 0x31, 0xc7, 0x41, 0x40, 0x02, 0x2a, 0xc6, 0x76, 0x2d, 0xe3, 0xc2, 0xb7, 0xb7, 0xd1, - 0xd7, 0x00, 0x9c, 0x88, 0x78, 0x4c, 0xdc, 0x88, 0x09, 0x7b, 0x4e, 0x2b, 0xff, 0xab, 0x2b, 0x29, - 0xef, 0x68, 0xf8, 0x26, 0x4b, 0xf4, 0x77, 0x2a, 0x3c, 0xfd, 0x8d, 0x08, 0x40, 0x2c, 0x08, 0x77, - 0x75, 0x45, 0xb4, 0xaf, 0x2f, 0x5b, 0xcd, 0x4a, 0x67, 0xcd, 0x94, 0x87, 0xcf, 0x2f, 0x97, 0x7e, - 0xc4, 0x8b, 0x39, 0x95, 0x93, 0x56, 0xff, 0xb7, 0x5f, 0x6e, 0x09, 0xc2, 0x43, 0x3c, 0x26, 0x9b, - 0x8a, 0xcd, 0xa9, 0x28, 0x66, 0xfd, 0xb8, 0x10, 0x43, 0x35, 0x11, 0x49, 0xbb, 0x01, 0xfd, 0x06, - 0x0a, 0x2a, 0x53, 0x75, 0x06, 0x5d, 0xad, 0x38, 0x5a, 0x8e, 0x46, 0xa2, 0x4f, 0x00, 0x24, 0xe6, - 0x43, 0x22, 0xbb, 0x2c, 0x10, 0x76, 0x6e, 0x39, 0xdf, 0xac, 0x98, 0xfd, 0xcc, 0xfa, 0x82, 0x80, - 0x6a, 0xc6, 0xef, 0xa8, 0x0e, 0xf9, 0x11, 0x99, 0xe8, 0xb7, 0x56, 0x1c, 0xf5, 0x88, 0x9e, 0x40, - 0x71, 0x1f, 0x07, 0x71, 0x5a, 0xa6, 0xaf, 0x16, 0x52, 0x19, 0x8d, 0x9c, 0x84, 0xe6, 0x97, 0xb9, - 0x07, 0xd6, 0xc2, 0x2f, 0xa0, 0x9c, 0xc6, 0x7d, 0xf6, 0x8d, 0xc5, 0xe4, 0x8d, 0x37, 0xb3, 0x6f, - 0xac, 0x64, 0x71, 0xbf, 0x86, 0xb9, 0x93, 0x7e, 0x7a, 0x17, 0x3a, 0x9f, 0x41, 0xaf, 0x17, 0xca, - 0x96, 0xae, 0x58, 0xf9, 0x7a, 0x61, 0xbd, 0x50, 0x2e, 0xd4, 0x8b, 0xeb, 0x85, 0x72, 0xb1, 0x5e, - 0x5a, 0x2f, 0x94, 0xaf, 0xd5, 0x6b, 0x8d, 0x7f, 0xe5, 0xe0, 0xa3, 0xbe, 0xe4, 0x04, 0x8f, 0x7b, - 0xe1, 0x90, 0x08, 0x55, 0x78, 0xa6, 0xf5, 0xe4, 0x0f, 0x70, 0x23, 0xc2, 0x5c, 0x52, 0xb5, 0xe8, - 0x62, 0xdf, 0x57, 0x49, 0x4f, 0x84, 0x6d, 0x69, 0x9b, 0x3e, 0x51, 0xb1, 0xf0, 0xef, 0xc3, 0xa5, - 0xb5, 0x4b, 0xc5, 0x82, 0xe7, 0x05, 0xaa, 0xc9, 0x13, 0x3c, 0xa6, 0xe1, 0xd0, 0xf3, 0x82, 0xd6, - 0x66, 0x4a, 0xbc, 0x92, 0xf0, 0x3a, 0x28, 0x3a, 0xb5, 0x42, 0x04, 0xea, 0x00, 0x08, 0x89, 0xb9, - 0x74, 0x55, 0x9a, 0x18, 0x4f, 0x5c, 0xaa, 0x95, 0x54, 0x34, 0x4c, 0xad, 0x22, 0x0e, 0x73, 0xc9, - 0x8b, 0x53, 0x0d, 0x74, 0x3d, 0xad, 0x74, 0x36, 0x8c, 0xfc, 0xdd, 0xef, 0x27, 0x7f, 0x62, 0xb3, - 0x54, 0xf8, 0x9a, 0xc8, 0xfe, 0x6c, 0xfc, 0xc3, 0x82, 0xff, 0x3f, 0x65, 0xd4, 0x35, 0xce, 0x42, - 0x49, 0xcd, 0x30, 0xe1, 0xc0, 0x8d, 0x5d, 0x3a, 0xdc, 0x75, 0x0f, 0xb0, 0x24, 0xdc, 0xc5, 0xd2, - 0xd5, 0xe2, 0x9a, 0xa0, 0xbf, 0x94, 0x82, 0x75, 0x85, 0x7f, 0xae, 0xe0, 0x2b, 0xb2, 0xaf, 0xc0, - 0xa8, 0x03, 0x35, 0xc9, 0xb1, 0x37, 0x22, 0xbe, 0x9b, 0xcc, 0x06, 0x39, 0x5d, 0x0e, 0x3e, 0x3a, - 0xa3, 0x94, 0x66, 0x26, 0x81, 0x6b, 0x06, 0xa3, 0x96, 0x44, 0xe3, 0x9f, 0xb3, 0xc9, 0xdc, 0x13, - 0x47, 0xd3, 0x18, 0xb8, 0x9f, 0x8e, 0x1a, 0xd6, 0x65, 0xe8, 0xcc, 0x60, 0xf1, 0x05, 0xd4, 0x69, - 0x28, 0x39, 0xf3, 0x63, 0xef, 0x6a, 0xe2, 0x5c, 0x3f, 0x86, 0x69, 0x89, 0xd0, 0x7d, 0xa8, 0xfa, - 0x64, 0x07, 0xc7, 0x81, 0x74, 0x55, 0xab, 0x49, 0x5c, 0x87, 0x4c, 0x63, 0x85, 0xd5, 0x64, 0x6b, - 0xcb, 0xe9, 0x39, 0x60, 0x8e, 0x6d, 0x71, 0x8a, 0xfe, 0x64, 0xc1, 0x8d, 0x98, 0x53, 0xe1, 0x0e, - 0x26, 0x6e, 0xc0, 0x3c, 0x1c, 0x50, 0x39, 0x71, 0x47, 0xfb, 0x76, 0x41, 0x8b, 0xf0, 0xf9, 0xc5, - 0xb3, 0xdb, 0xb1, 0xee, 0xaa, 0x49, 0x89, 0xce, 0xe4, 0x4b, 0xc3, 0xb0, 0xb1, 0x9f, 0xf4, 0x88, - 0x9b, 0x47, 0x87, 0x4b, 0xf5, 0x2d, 0xa7, 0x97, 0xdd, 0xda, 0x76, 0xea, 0xf1, 0xa9, 0xc3, 0xc8, - 0x81, 0xea, 0x78, 0xdf, 0xf3, 0xdc, 0x1d, 0x1a, 0x48, 0xc2, 0xf5, 0xac, 0x34, 0x77, 0xc2, 0xb9, - 0xa9, 0xfe, 0x5f, 0x6d, 0x77, 0xbb, 0x6b, 0xfa, 0xd0, 0xb1, 0x66, 0xc7, 0x6b, 0x0e, 0x28, 0x96, - 0xe4, 0x19, 0x7d, 0x01, 0x40, 0x42, 0x8f, 0x4f, 0x22, 0x3d, 0x3a, 0x24, 0xcd, 0xbb, 0x79, 0x06, - 0xa5, 0x6a, 0x95, 0x8f, 0xa6, 0x07, 0x9f, 0xea, 0xbf, 0xc2, 0xc9, 0x60, 0xd1, 0x53, 0x98, 0x1f, - 0x68, 0x6d, 0xdd, 0x4c, 0x86, 0x5d, 0x61, 0x58, 0xbb, 0x9e, 0xa0, 0xfb, 0xd3, 0x3c, 0xdb, 0x00, - 0xb3, 0xe4, 0x92, 0xd0, 0x4f, 0xe8, 0xca, 0x97, 0xa7, 0xab, 0x25, 0xd8, 0x47, 0xa1, 0xaf, 0xc9, - 0xb6, 0xa0, 0x14, 0x8d, 0x5c, 0xea, 0xa7, 0x1d, 0xfd, 0xfe, 0xa5, 0x7d, 0xb6, 0x39, 0xea, 0xf9, - 0xa6, 0x99, 0x57, 0xd4, 0xec, 0xb5, 0xb9, 0xd1, 0x5b, 0x15, 0x4e, 0x31, 0x52, 0xcb, 0xa7, 0x7a, - 0x1a, 0xfc, 0x50, 0x3d, 0xad, 0x0b, 0x1f, 0x9c, 0x19, 0x3a, 0x67, 0xb4, 0x99, 0xf3, 0x8b, 0xfe, - 0x03, 0x80, 0x63, 0x5d, 0xb2, 0xc8, 0xc2, 0x19, 0xc8, 0x72, 0x06, 0xd9, 0xf8, 0xbb, 0x05, 0x75, - 0x87, 0x08, 0xc9, 0x38, 0x51, 0x49, 0x94, 0x10, 0x7c, 0x0a, 0x05, 0x95, 0x87, 0xa6, 0xc6, 0xbc, - 0x23, 0x0d, 0xf5, 0x51, 0xb4, 0x02, 0xc5, 0x1d, 0xaa, 0xa6, 0xaa, 0x24, 0x75, 0x6f, 0x9d, 0x35, - 0x94, 0xe9, 0x46, 0xe7, 0x90, 0xbd, 0x98, 0x08, 0xa9, 0xa3, 0x2e, 0x2d, 0x04, 0x1a, 0x89, 0x6e, - 0x43, 0x35, 0x9d, 0x16, 0x7b, 0xfe, 0x4b, 0x9d, 0xbe, 0xe9, 0xf8, 0x93, 0xdd, 0x68, 0xfc, 0x31, - 0x0f, 0xd7, 0x8d, 0xc8, 0xd3, 0xca, 0xb3, 0x06, 0xd7, 0x78, 0xb2, 0x94, 0x44, 0xd3, 0x15, 0xaa, - 0x63, 0xd5, 0x00, 0x75, 0x2c, 0x9d, 0xcc, 0x99, 0xdc, 0x7b, 0xe4, 0x4c, 0x0f, 0x4a, 0x9c, 0xe8, - 0xe1, 0x2f, 0xb9, 0x77, 0xfd, 0xf4, 0x9d, 0x16, 0x31, 0xd7, 0xaf, 0x11, 0x99, 0xa4, 0x23, 0x6b, - 0x42, 0xa0, 0x46, 0x56, 0x13, 0xe0, 0x49, 0x51, 0xfa, 0xd9, 0x45, 0xf3, 0xc5, 0x09, 0xbb, 0x5c, - 0x18, 0xe1, 0xef, 0x11, 0x35, 0x7f, 0xcd, 0xc1, 0x8d, 0x7e, 0x14, 0x50, 0xb9, 0x12, 0xfa, 0x7d, - 0x0f, 0x4b, 0x69, 0x7a, 0xd5, 0xef, 0xa1, 0xa4, 0x6f, 0x76, 0x69, 0x07, 0x78, 0x78, 0xbe, 0xa4, - 0x67, 0xc0, 0x53, 0xe9, 0xb5, 0x3c, 0x5d, 0xc5, 0x93, 0x1a, 0x22, 0x21, 0xcd, 0xd8, 0x34, 0xf7, - 0x9e, 0x36, 0x5d, 0x70, 0x61, 0xfe, 0xad, 0xb7, 0xa1, 0x75, 0x98, 0x25, 0xea, 0x4e, 0x45, 0x52, - 0xf9, 0xef, 0xbc, 0xd3, 0xd2, 0xd3, 0xa4, 0x31, 0xfc, 0x29, 0x41, 0xe3, 0xcf, 0x79, 0xa8, 0x75, - 0xfb, 0xdb, 0xcf, 0x39, 0x4d, 0x8d, 0x73, 0x5b, 0xb5, 0x27, 0x21, 0x69, 0x98, 0x5c, 0xa2, 0x75, - 0x62, 0xa7, 0x31, 0x98, 0xd9, 0x40, 0x3f, 0x86, 0x6b, 0xaa, 0x52, 0xb8, 0x91, 0x36, 0x4c, 0x12, - 0x85, 0xd3, 0x83, 0xba, 0x86, 0x24, 0x1b, 0xe8, 0x33, 0x98, 0x65, 0x49, 0xe4, 0xe9, 0x64, 0xa9, - 0x9e, 0xd9, 0x30, 0xba, 0xfd, 0x6d, 0x13, 0x9e, 0xa9, 0x84, 0x06, 0x73, 0x7c, 0x3d, 0xe7, 0xec, - 0x40, 0x98, 0x3b, 0x7d, 0xf6, 0x7a, 0xee, 0xb0, 0x03, 0x81, 0x7e, 0x07, 0xf3, 0x1e, 0x1b, 0x47, - 0x2a, 0xf7, 0xd4, 0x60, 0xe7, 0x31, 0x9f, 0x78, 0xa6, 0x3d, 0x5d, 0xf0, 0x25, 0x41, 0xa5, 0x47, - 0xf7, 0x18, 0x96, 0xce, 0x21, 0x19, 0xa6, 0xae, 0x22, 0x3a, 0x55, 0x63, 0x4b, 0x3f, 0x50, 0x8d, - 0x6d, 0x3c, 0x87, 0xf9, 0x4e, 0x1c, 0x28, 0x85, 0x32, 0xee, 0x98, 0x7e, 0x5b, 0xb1, 0xbe, 0xf7, - 0xb7, 0x95, 0x3b, 0xb7, 0xe0, 0xfa, 0x29, 0x55, 0x51, 0x19, 0x0a, 0x4f, 0x58, 0x48, 0xea, 0x33, - 0xea, 0xe9, 0xf1, 0x2b, 0x1a, 0xd5, 0xad, 0xce, 0xdd, 0xd7, 0xff, 0x5d, 0x9c, 0x79, 0x7d, 0xb4, - 0x68, 0x7d, 0x73, 0xb4, 0x68, 0x7d, 0x7b, 0xb4, 0x68, 0xfd, 0xe7, 0x68, 0xd1, 0xfa, 0xcb, 0x9b, - 0xc5, 0x99, 0x6f, 0xde, 0x2c, 0xce, 0x7c, 0xfb, 0x66, 0x71, 0xe6, 0xeb, 0x6a, 0xe6, 0xf3, 0xd5, - 0xff, 0x02, 0x00, 0x00, 0xff, 0xff, 0x47, 0x83, 0x27, 0x38, 0x6b, 0x13, 0x00, 0x00, + 0xf5, 0xe9, 0x68, 0xd0, 0xa6, 0xec, 0xee, 0x0e, 0xe3, 0x63, 0x2c, 0xd3, 0x9d, 0x8f, 0xd5, 0xad, + 0x1e, 0x96, 0x38, 0x60, 0xc3, 0xb6, 0x4f, 0x84, 0x17, 0x0d, 0xda, 0x42, 0xf2, 0xd8, 0x93, 0x31, + 0x27, 0xbe, 0x39, 0x74, 0xeb, 0x22, 0xd1, 0xb0, 0x20, 0xe9, 0x2d, 0xb1, 0xa4, 0x41, 0x7b, 0x37, + 0xf0, 0xda, 0x92, 0x8e, 0x89, 0x90, 0x78, 0x1c, 0x99, 0x9d, 0x9b, 0x43, 0x36, 0x64, 0xfa, 0xb1, + 0xad, 0x9e, 0xcc, 0x2a, 0x4a, 0xa5, 0xf2, 0xb1, 0xc4, 0x66, 0x6d, 0x3e, 0x5d, 0xc3, 0x11, 0x4d, + 0x96, 0x1a, 0x7f, 0x2b, 0xc0, 0x5c, 0x07, 0x7b, 0xa3, 0x1d, 0x1a, 0x04, 0x84, 0xf7, 0x23, 0xe2, + 0xa1, 0xc7, 0x50, 0x90, 0x93, 0x88, 0xd8, 0xd6, 0xb2, 0xd5, 0x9c, 0xbb, 0x77, 0xb7, 0x75, 0x9e, + 0x41, 0x5a, 0x27, 0x71, 0xad, 0x67, 0x93, 0x88, 0x74, 0x0a, 0xaf, 0x0f, 0x97, 0x66, 0x1c, 0x4d, + 0x80, 0x3a, 0x50, 0x94, 0x78, 0x10, 0x10, 0x3b, 0xb7, 0x6c, 0x35, 0xab, 0xf7, 0x6e, 0x9f, 0x62, + 0x12, 0x7b, 0x81, 0xd6, 0xef, 0x99, 0x3a, 0xb3, 0x4a, 0x84, 0xc7, 0x69, 0x24, 0x19, 0x37, 0x14, + 0x09, 0x14, 0x3d, 0x82, 0xa2, 0x88, 0x70, 0x28, 0xec, 0xfc, 0x72, 0xbe, 0x59, 0xbd, 0xf7, 0x93, + 0xf3, 0xa5, 0xd1, 0x34, 0x0e, 0xc1, 0xbe, 0x12, 0x07, 0x87, 0x29, 0x8d, 0x46, 0xa3, 0x4f, 0xa1, + 0xec, 0xc7, 0x1c, 0x4b, 0xca, 0x42, 0xbb, 0xb0, 0x6c, 0x35, 0xf3, 0x9d, 0x0f, 0xd4, 0xf6, 0x77, + 0x87, 0x4b, 0x35, 0x65, 0xce, 0xd6, 0xaa, 0xd9, 0x74, 0xa6, 0xc7, 0xd0, 0xc7, 0x00, 0xde, 0x6e, + 0x1c, 0x8e, 0x5c, 0x41, 0x5f, 0x11, 0xbb, 0xa8, 0x41, 0x09, 0x67, 0x45, 0xaf, 0xf7, 0xe9, 0x2b, + 0x82, 0x1e, 0x42, 0x99, 0x13, 0xec, 0xaf, 0x88, 0xa7, 0x3b, 0xf6, 0xac, 0xd6, 0xf2, 0x47, 0x19, + 0x09, 0x95, 0xcb, 0x5a, 0xbb, 0x81, 0xd7, 0x7a, 0x96, 0xba, 0xcc, 0x30, 0x4c, 0x41, 0x48, 0xc0, + 0x0d, 0x1a, 0xfa, 0xe4, 0x25, 0x11, 0xae, 0x64, 0xee, 0xc0, 0x58, 0xd4, 0x2e, 0x2f, 0xe7, 0x9b, + 0xb5, 0x4e, 0xf7, 0xbb, 0xc3, 0xa5, 0x87, 0x43, 0x2a, 0x77, 0xe3, 0x41, 0xcb, 0x63, 0xe3, 0xf6, + 0x94, 0xd9, 0x1f, 0x1c, 0x3f, 0xb7, 0xa3, 0xd1, 0xb0, 0xfd, 0x76, 0xa0, 0xb5, 0x7a, 0x8a, 0xb6, + 0xb7, 0xea, 0xcc, 0x1b, 0xfe, 0x67, 0x2c, 0xf5, 0x57, 0xe3, 0x0e, 0x14, 0x94, 0xb3, 0x50, 0x15, + 0x66, 0x7b, 0xe1, 0x3e, 0x0e, 0xa8, 0x5f, 0x9f, 0x41, 0x00, 0xa5, 0x2e, 0x0b, 0xe2, 0x71, 0x58, + 0xb7, 0x50, 0x05, 0x8a, 0x1a, 0x5e, 0xcf, 0xad, 0x17, 0xca, 0xa5, 0xfa, 0x6c, 0xe3, 0x00, 0xaa, + 0xeb, 0x6c, 0xb0, 0xc9, 0xd9, 0x90, 0x13, 0x21, 0xd0, 0x27, 0x50, 0x7a, 0xc1, 0x06, 0x2e, 0xf5, + 0x75, 0x90, 0xe4, 0x3b, 0x35, 0xa5, 0xd5, 0xd1, 0xe1, 0x52, 0x71, 0x9d, 0x0d, 0x7a, 0xab, 0x4e, + 0xf1, 0x05, 0x1b, 0xf4, 0x7c, 0xd4, 0x84, 0x6b, 0x1e, 0x0b, 0x25, 0xa7, 0x83, 0x58, 0x1b, 0x5e, + 0x85, 0x41, 0xce, 0x58, 0xe0, 0xc4, 0x0e, 0xb2, 0xa1, 0x20, 0x02, 0x26, 0xed, 0xfc, 0xb2, 0xd5, + 0x2c, 0xa6, 0x31, 0xa4, 0x56, 0x1a, 0xaf, 0xcb, 0x80, 0x94, 0x53, 0x7b, 0xe3, 0x88, 0x71, 0xb9, + 0x8a, 0x25, 0xd6, 0x31, 0x7a, 0x0b, 0xaa, 0x02, 0x8f, 0xa3, 0x80, 0x24, 0xde, 0xc9, 0x65, 0x70, + 0x90, 0x6c, 0x68, 0xf7, 0x3c, 0x86, 0x72, 0x64, 0x64, 0xb6, 0x4b, 0xda, 0x3d, 0xb7, 0xce, 0x0f, + 0xa0, 0x8c, 0x82, 0xa9, 0x9b, 0x52, 0x30, 0x7a, 0x0c, 0xf9, 0x98, 0x53, 0x7b, 0x56, 0x07, 0xe1, + 0xcf, 0xcf, 0xe7, 0x78, 0x5b, 0xd4, 0xd6, 0x16, 0xa7, 0x8f, 0x42, 0xc9, 0x27, 0x8e, 0x62, 0x40, + 0x9f, 0x41, 0x29, 0xa9, 0x11, 0x76, 0x59, 0xcb, 0xb3, 0x94, 0xe1, 0x32, 0xd9, 0xd9, 0xea, 0x3d, + 0x5d, 0xa3, 0x01, 0x59, 0xd3, 0xc7, 0x8c, 0x24, 0x06, 0x84, 0xb6, 0xa1, 0xa4, 0xf3, 0x42, 0xd8, + 0x15, 0x2d, 0xca, 0x83, 0x2b, 0x89, 0xa2, 0x53, 0x44, 0x68, 0x69, 0x34, 0xaf, 0xe5, 0x18, 0x36, + 0xf4, 0x10, 0xfe, 0x4f, 0x8c, 0x68, 0xe4, 0x8e, 0xa9, 0x10, 0x34, 0x1c, 0xba, 0x3b, 0x8c, 0x13, + 0x3a, 0x0c, 0xdd, 0x11, 0x99, 0x08, 0x1b, 0x96, 0xad, 0x66, 0xd9, 0x08, 0xf2, 0xa1, 0x3a, 0xf6, + 0x55, 0x72, 0x6a, 0x2d, 0x39, 0xb4, 0x41, 0x26, 0x02, 0xdd, 0x81, 0xda, 0x01, 0x0e, 0x02, 0x95, + 0x4c, 0x4f, 0x70, 0xc8, 0x84, 0x5d, 0xcd, 0x24, 0xcc, 0xc9, 0x2d, 0x74, 0x0f, 0xe6, 0xb9, 0xce, + 0xd3, 0x4d, 0xcc, 0x71, 0x10, 0x90, 0x80, 0x8a, 0xb1, 0x5d, 0xcb, 0xb8, 0xf0, 0xed, 0x6d, 0xf4, + 0x35, 0x00, 0x27, 0x22, 0x1e, 0x13, 0x37, 0x62, 0xc2, 0x9e, 0xd3, 0xca, 0xff, 0xea, 0x4a, 0xca, + 0x3b, 0x1a, 0xbe, 0xc9, 0x12, 0xfd, 0x9d, 0x0a, 0x4f, 0x7f, 0x23, 0x02, 0x10, 0x0b, 0xc2, 0x5d, + 0x5d, 0x11, 0xed, 0xeb, 0xcb, 0x56, 0xb3, 0xd2, 0x59, 0x33, 0xe5, 0xe1, 0xf3, 0xcb, 0xa5, 0x1f, + 0xf1, 0x62, 0x4e, 0xe5, 0xa4, 0xd5, 0xff, 0xed, 0x97, 0x5b, 0x82, 0xf0, 0x10, 0x8f, 0xc9, 0xa6, + 0x62, 0x73, 0x2a, 0x8a, 0x59, 0x3f, 0x2e, 0xc4, 0x50, 0x4d, 0x44, 0xd2, 0x6e, 0x40, 0xbf, 0x81, + 0x82, 0xca, 0x54, 0x9d, 0x41, 0x57, 0x2b, 0x8e, 0x96, 0xa3, 0x91, 0xe8, 0x13, 0x00, 0x89, 0xf9, + 0x90, 0xc8, 0x2e, 0x0b, 0x84, 0x9d, 0x5b, 0xce, 0x37, 0x2b, 0x66, 0x3f, 0xb3, 0xbe, 0x20, 0xa0, + 0x9a, 0xf1, 0x3b, 0xaa, 0x43, 0x7e, 0x44, 0x26, 0xfa, 0xd6, 0x8a, 0xa3, 0x1e, 0xd1, 0x13, 0x28, + 0xee, 0xe3, 0x20, 0x4e, 0xcb, 0xf4, 0xd5, 0x42, 0x2a, 0xa3, 0x91, 0x93, 0xd0, 0xfc, 0x32, 0xf7, + 0xc0, 0x5a, 0xf8, 0x05, 0x94, 0xd3, 0xb8, 0xcf, 0xde, 0x58, 0x4c, 0x6e, 0xbc, 0x99, 0xbd, 0xb1, + 0x92, 0xc5, 0xfd, 0x1a, 0xe6, 0x4e, 0xfa, 0xe9, 0x5d, 0xe8, 0x7c, 0x06, 0xbd, 0x5e, 0x28, 0x5b, + 0xba, 0x62, 0xe5, 0xeb, 0x85, 0xf5, 0x42, 0xb9, 0x50, 0x2f, 0xae, 0x17, 0xca, 0xc5, 0x7a, 0x69, + 0xbd, 0x50, 0xbe, 0x56, 0xaf, 0x35, 0xfe, 0x95, 0x83, 0x8f, 0xfa, 0x92, 0x13, 0x3c, 0xee, 0x85, + 0x43, 0x22, 0x54, 0xe1, 0x99, 0xd6, 0x93, 0x3f, 0xc0, 0x8d, 0x08, 0x73, 0x49, 0xd5, 0xa2, 0x8b, + 0x7d, 0x5f, 0x25, 0x3d, 0x11, 0xb6, 0xa5, 0x6d, 0xfa, 0x44, 0xc5, 0xc2, 0xbf, 0x0f, 0x97, 0xd6, + 0x2e, 0x15, 0x0b, 0x9e, 0x17, 0xa8, 0x26, 0x4f, 0xf0, 0x98, 0x86, 0x43, 0xcf, 0x0b, 0x5a, 0x9b, + 0x29, 0xf1, 0x4a, 0xc2, 0xeb, 0xa0, 0xe8, 0xd4, 0x0a, 0x11, 0xa8, 0x03, 0x20, 0x24, 0xe6, 0xd2, + 0x55, 0x69, 0x62, 0x3c, 0x71, 0xa9, 0x56, 0x52, 0xd1, 0x30, 0xb5, 0x8a, 0x38, 0xcc, 0x25, 0x17, + 0xa7, 0x1a, 0xe8, 0x7a, 0x5a, 0xe9, 0x6c, 0x18, 0xf9, 0xbb, 0xdf, 0x4f, 0xfe, 0xc4, 0x66, 0xa9, + 0xf0, 0x35, 0x91, 0xfd, 0xd9, 0xf8, 0x87, 0x05, 0xff, 0x7f, 0xca, 0xa8, 0x6b, 0x9c, 0x85, 0x92, + 0x9a, 0x61, 0xc2, 0x81, 0x1b, 0xbb, 0x74, 0xb8, 0xeb, 0x1e, 0x60, 0x49, 0xb8, 0x8b, 0xa5, 0xab, + 0xc5, 0x35, 0x41, 0x7f, 0x29, 0x05, 0xeb, 0x0a, 0xff, 0x5c, 0xc1, 0x57, 0x64, 0x5f, 0x81, 0x51, + 0x07, 0x6a, 0x92, 0x63, 0x6f, 0x44, 0x7c, 0x37, 0x99, 0x0d, 0x72, 0xba, 0x1c, 0x7c, 0x74, 0x46, + 0x29, 0xcd, 0x4c, 0x02, 0xd7, 0x0c, 0x46, 0x2d, 0x89, 0x46, 0x07, 0x16, 0x4e, 0x89, 0xbd, 0xc9, + 0x82, 0x80, 0x86, 0x43, 0x2d, 0xf5, 0xa5, 0xfa, 0x5b, 0xe3, 0x9f, 0xb3, 0xc9, 0xec, 0x14, 0x47, + 0xd3, 0x38, 0xba, 0x9f, 0x8e, 0x2b, 0xd6, 0x65, 0x44, 0x32, 0xc3, 0xc9, 0x17, 0x50, 0xa7, 0xa1, + 0xe4, 0xcc, 0x8f, 0xbd, 0xab, 0xa9, 0x74, 0xfd, 0x18, 0xa6, 0xb5, 0x42, 0xf7, 0xa1, 0xea, 0x93, + 0x1d, 0x1c, 0x07, 0xd2, 0x55, 0xed, 0x2a, 0x71, 0x3f, 0x32, 0xc2, 0xc3, 0x6a, 0xb2, 0xb5, 0xe5, + 0xf4, 0x1c, 0x30, 0xc7, 0xb6, 0x38, 0x45, 0x7f, 0xb2, 0xe0, 0x46, 0xcc, 0xa9, 0x70, 0x07, 0x13, + 0x37, 0x60, 0x1e, 0x0e, 0xa8, 0x9c, 0xb8, 0xa3, 0x7d, 0xbb, 0xa0, 0x45, 0xf8, 0xfc, 0xe2, 0xf9, + 0xef, 0x58, 0x77, 0xd5, 0xe8, 0x44, 0x67, 0xf2, 0xa5, 0x61, 0xd8, 0xd8, 0x4f, 0xfa, 0xcc, 0xcd, + 0xa3, 0xc3, 0xa5, 0xfa, 0x96, 0xd3, 0xcb, 0x6e, 0x6d, 0x3b, 0xf5, 0xf8, 0xd4, 0x61, 0xe4, 0x40, + 0x75, 0xbc, 0xef, 0x79, 0xee, 0x0e, 0x0d, 0x24, 0xe1, 0x7a, 0xde, 0x9a, 0x3b, 0x11, 0x20, 0xa9, + 0xfe, 0x5f, 0x6d, 0x77, 0xbb, 0x6b, 0xfa, 0xd0, 0xb1, 0x66, 0xc7, 0x6b, 0x0e, 0x28, 0x96, 0xe4, + 0x19, 0x7d, 0x01, 0x40, 0x42, 0x8f, 0x4f, 0x22, 0x3d, 0x7e, 0x24, 0x03, 0x40, 0xf3, 0x0c, 0x4a, + 0xd5, 0x6e, 0x1f, 0x4d, 0x0f, 0x3e, 0xd5, 0x7f, 0x85, 0x93, 0xc1, 0xa2, 0xa7, 0x30, 0x3f, 0xd0, + 0xda, 0xba, 0x99, 0x2c, 0xbd, 0xc2, 0xc0, 0x77, 0x3d, 0x41, 0xf7, 0xa7, 0xb9, 0xba, 0x01, 0x66, + 0xc9, 0x25, 0xa1, 0x9f, 0xd0, 0x95, 0x2f, 0x4f, 0x57, 0x4b, 0xb0, 0x8f, 0x42, 0x5f, 0x93, 0x6d, + 0x41, 0x29, 0x1a, 0xb9, 0xd4, 0x4f, 0xa7, 0x82, 0xfb, 0x97, 0xf6, 0xd9, 0xe6, 0xa8, 0xe7, 0x9b, + 0x81, 0xa0, 0xa2, 0xe2, 0x7b, 0x73, 0xa3, 0xb7, 0x2a, 0x9c, 0x62, 0xa4, 0x96, 0x4f, 0xf5, 0x45, + 0xf8, 0xa1, 0xfa, 0x62, 0x17, 0x3e, 0x38, 0x33, 0x74, 0xce, 0x68, 0x55, 0xe7, 0x37, 0x8e, 0x07, + 0x00, 0xc7, 0xba, 0x64, 0x91, 0x85, 0x33, 0x90, 0xe5, 0x0c, 0xb2, 0xf1, 0x77, 0x0b, 0xea, 0x0e, + 0x11, 0x92, 0x71, 0xa2, 0x92, 0x28, 0x21, 0xf8, 0x14, 0x0a, 0x2a, 0x0f, 0x4d, 0x9d, 0x7a, 0x47, + 0x1a, 0xea, 0xa3, 0x68, 0x05, 0x8a, 0x3b, 0x54, 0x4d, 0x66, 0x49, 0xea, 0xde, 0x3a, 0x6b, 0xb0, + 0xd3, 0xcd, 0xd2, 0x21, 0x7b, 0x31, 0x11, 0x52, 0x47, 0x5d, 0x5a, 0x08, 0x34, 0x12, 0xdd, 0x86, + 0x6a, 0x3a, 0x71, 0xf6, 0xfc, 0x97, 0x3a, 0x7d, 0xd3, 0x11, 0x2a, 0xbb, 0xd1, 0xf8, 0x63, 0x1e, + 0xae, 0x1b, 0x91, 0xa7, 0x95, 0x67, 0x0d, 0xae, 0xf1, 0x64, 0x29, 0x89, 0xa6, 0x2b, 0x54, 0xd8, + 0xaa, 0x01, 0xea, 0x58, 0x3a, 0x99, 0x33, 0xb9, 0xf7, 0xc8, 0x99, 0x1e, 0x94, 0x38, 0xd1, 0x03, + 0x64, 0xf2, 0xee, 0xf6, 0xd3, 0x77, 0x5a, 0xc4, 0xbc, 0xc2, 0x8d, 0xc8, 0x24, 0x1d, 0x7b, 0x13, + 0x02, 0x35, 0xf6, 0x9a, 0x00, 0x4f, 0x8a, 0xd2, 0xcf, 0x2e, 0x9a, 0x51, 0x4e, 0xd8, 0xe5, 0xc2, + 0x08, 0x7f, 0x8f, 0xa8, 0xf9, 0x6b, 0x0e, 0x6e, 0xf4, 0xa3, 0x80, 0xca, 0x95, 0xd0, 0xef, 0x7b, + 0x58, 0x4a, 0xd3, 0xef, 0x7e, 0x0f, 0x25, 0xfd, 0x76, 0x98, 0x76, 0x80, 0x87, 0xe7, 0x4b, 0x7a, + 0x06, 0x3c, 0x95, 0x5e, 0xcb, 0xd3, 0x55, 0x3c, 0xa9, 0x21, 0x12, 0xd2, 0x8c, 0x4d, 0x73, 0xef, + 0x69, 0xd3, 0x05, 0x17, 0xe6, 0xdf, 0xba, 0x0d, 0xad, 0xc3, 0x2c, 0x51, 0xef, 0x65, 0x24, 0x95, + 0xff, 0xce, 0x3b, 0x2d, 0x3d, 0x4d, 0x1a, 0xc3, 0x9f, 0x12, 0x34, 0xfe, 0x9c, 0x87, 0x5a, 0xb7, + 0xbf, 0xfd, 0x9c, 0xd3, 0xd4, 0x38, 0xb7, 0x55, 0x7b, 0x12, 0x92, 0x86, 0xc9, 0x8b, 0xb8, 0x4e, + 0xec, 0x34, 0x06, 0x33, 0x1b, 0xe8, 0xc7, 0x70, 0x4d, 0x55, 0x0a, 0x37, 0xd2, 0x86, 0x49, 0xa2, + 0x70, 0x7a, 0x50, 0xd7, 0x90, 0x64, 0x03, 0x7d, 0x06, 0xb3, 0x2c, 0x89, 0x3c, 0x9d, 0x2c, 0xd5, + 0x33, 0x1b, 0x46, 0xb7, 0xbf, 0x6d, 0xc2, 0x33, 0x95, 0xd0, 0x60, 0x8e, 0x5f, 0xf1, 0x39, 0x3b, + 0x10, 0xe6, 0xbb, 0x40, 0xf6, 0x15, 0xdf, 0x61, 0x07, 0x02, 0xfd, 0x0e, 0xe6, 0x3d, 0x36, 0x8e, + 0x54, 0xee, 0xa9, 0xe1, 0xd0, 0x63, 0x3e, 0xf1, 0x4c, 0x7b, 0xba, 0xe0, 0x6b, 0x84, 0x4a, 0x8f, + 0xee, 0x31, 0x2c, 0x9d, 0x65, 0x32, 0x4c, 0x5d, 0x45, 0x74, 0xaa, 0xc6, 0x96, 0x7e, 0xa0, 0x1a, + 0xdb, 0x78, 0x0e, 0xf3, 0x9d, 0x38, 0x50, 0x0a, 0x65, 0xdc, 0x31, 0xfd, 0x3e, 0x63, 0x7d, 0xef, + 0xef, 0x33, 0x77, 0x6e, 0xc1, 0xf5, 0x53, 0xaa, 0xa2, 0x32, 0x14, 0x9e, 0xb0, 0x90, 0xd4, 0x67, + 0xd4, 0xd3, 0xe3, 0x57, 0x34, 0xaa, 0x5b, 0x9d, 0xbb, 0xaf, 0xff, 0xbb, 0x38, 0xf3, 0xfa, 0x68, + 0xd1, 0xfa, 0xe6, 0x68, 0xd1, 0xfa, 0xf6, 0x68, 0xd1, 0xfa, 0xcf, 0xd1, 0xa2, 0xf5, 0x97, 0x37, + 0x8b, 0x33, 0xdf, 0xbc, 0x59, 0x9c, 0xf9, 0xf6, 0xcd, 0xe2, 0xcc, 0xd7, 0xd5, 0xcc, 0x27, 0xb0, + 0xff, 0x05, 0x00, 0x00, 0xff, 0xff, 0x55, 0x37, 0xce, 0x3c, 0xaf, 0x13, 0x00, 0x00, } diff --git a/pkg/sql/execinfrapb/processors_bulk_io.proto b/pkg/sql/execinfrapb/processors_bulk_io.proto index 471ed54cb36d..8caaca0eb587 100644 --- a/pkg/sql/execinfrapb/processors_bulk_io.proto +++ b/pkg/sql/execinfrapb/processors_bulk_io.proto @@ -163,6 +163,10 @@ message StreamIngestionFrontierSpec { repeated roachpb.Span tracked_spans = 2 [(gogoproto.nullable) = false]; } +message StreamIngestionPollingSpec { + optional int64 job_id = 1 [(gogoproto.nullable) = false, (gogoproto.customname) = "JobID"]; +} + message BackupDataSpec { repeated roachpb.Span spans = 1 [(gogoproto.nullable) = false]; repeated roachpb.Span introduced_spans = 2 [(gogoproto.nullable) = false]; diff --git a/pkg/sql/rowexec/processors.go b/pkg/sql/rowexec/processors.go index 5b8f2ab32e14..5cf174a8378c 100644 --- a/pkg/sql/rowexec/processors.go +++ b/pkg/sql/rowexec/processors.go @@ -268,13 +268,14 @@ func NewProcessor( return NewRestoreDataProcessor(flowCtx, processorID, *core.RestoreData, post, inputs[0], outputs[0]) } if core.StreamIngestionData != nil { - if err := checkNumInOut(inputs, outputs, 0, 1); err != nil { + if err := checkNumInOut(inputs, outputs, 1, 1); err != nil { return nil, err } if NewStreamIngestionDataProcessor == nil { return nil, errors.New("StreamIngestionData processor unimplemented") } - return NewStreamIngestionDataProcessor(flowCtx, processorID, *core.StreamIngestionData, post, outputs[0]) + return NewStreamIngestionDataProcessor(flowCtx, processorID, *core.StreamIngestionData, + inputs[0], post, outputs[0]) } if core.CSVWriter != nil { if err := checkNumInOut(inputs, outputs, 1, 1); err != nil { @@ -365,6 +366,12 @@ func NewProcessor( } return NewStreamIngestionFrontierProcessor(flowCtx, processorID, *core.StreamIngestionFrontier, inputs[0], post, outputs[0]) } + if core.StreamIngestionPolling != nil { + if err := checkNumInOut(inputs, outputs, 0, 1); err != nil { + return nil, err + } + return NewStreamIngestionPollingProcessor(flowCtx, processorID, *core.StreamIngestionPolling, post, outputs[0]) + } return nil, errors.Errorf("unsupported processor core %q", core) } @@ -381,7 +388,7 @@ var NewSplitAndScatterProcessor func(*execinfra.FlowCtx, int32, execinfrapb.Spli var NewRestoreDataProcessor func(*execinfra.FlowCtx, int32, execinfrapb.RestoreDataSpec, *execinfrapb.PostProcessSpec, execinfra.RowSource, execinfra.RowReceiver) (execinfra.Processor, error) // NewStreamIngestionDataProcessor is implemented in the non-free (CCL) codebase and then injected here via runtime initialization. -var NewStreamIngestionDataProcessor func(*execinfra.FlowCtx, int32, execinfrapb.StreamIngestionDataSpec, *execinfrapb.PostProcessSpec, execinfra.RowReceiver) (execinfra.Processor, error) +var NewStreamIngestionDataProcessor func(*execinfra.FlowCtx, int32, execinfrapb.StreamIngestionDataSpec, execinfra.RowSource, *execinfrapb.PostProcessSpec, execinfra.RowReceiver) (execinfra.Processor, error) // NewCSVWriterProcessor is implemented in the non-free (CCL) codebase and then injected here via runtime initialization. var NewCSVWriterProcessor func(*execinfra.FlowCtx, int32, execinfrapb.CSVWriterSpec, execinfra.RowSource, execinfra.RowReceiver) (execinfra.Processor, error) @@ -394,3 +401,6 @@ var NewChangeFrontierProcessor func(*execinfra.FlowCtx, int32, execinfrapb.Chang // NewStreamIngestionFrontierProcessor is implemented in the non-free (CCL) codebase and then injected here via runtime initialization. var NewStreamIngestionFrontierProcessor func(*execinfra.FlowCtx, int32, execinfrapb.StreamIngestionFrontierSpec, execinfra.RowSource, *execinfrapb.PostProcessSpec, execinfra.RowReceiver) (execinfra.Processor, error) + +// NewStreamIngestionPollingProcessor is implemented in the non-free (CCL) codebase and then injected here via runtime initialization. +var NewStreamIngestionPollingProcessor func(*execinfra.FlowCtx, int32, execinfrapb.StreamIngestionPollingSpec, *execinfrapb.PostProcessSpec, execinfra.RowReceiver) (execinfra.Processor, error) diff --git a/pkg/util/mon/bytes_usage.go b/pkg/util/mon/bytes_usage.go index 74b3c24f9e82..e5677e4e4ef3 100644 --- a/pkg/util/mon/bytes_usage.go +++ b/pkg/util/mon/bytes_usage.go @@ -626,7 +626,6 @@ func (mm *BytesMonitor) reserveBytes(ctx context.Context, x int64) error { if mm.mu.maxAllocated < mm.mu.curAllocated { mm.mu.maxAllocated = mm.mu.curAllocated } - // Report "large" queries to the log for further investigation. if log.V(1) { if mm.mu.curAllocated > mm.noteworthyUsageBytes {