diff --git a/pkg/ccl/streamingccl/streamingest/BUILD.bazel b/pkg/ccl/streamingccl/streamingest/BUILD.bazel index e10347b69c98..578e41793c7f 100644 --- a/pkg/ccl/streamingccl/streamingest/BUILD.bazel +++ b/pkg/ccl/streamingccl/streamingest/BUILD.bazel @@ -3,6 +3,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "streamingest", srcs = [ + "stream_ingestion_frontier_processor.go", "stream_ingestion_job.go", "stream_ingestion_processor.go", "stream_ingestion_processor_planning.go", @@ -28,10 +29,13 @@ go_library( "//pkg/sql/sem/tree", "//pkg/sql/types", "//pkg/storage", + "//pkg/util/hlc", "//pkg/util/protoutil", + "//pkg/util/span", "//pkg/util/timeutil", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_logtags//:logtags", + "@com_github_cockroachdb_redact//:redact", ], ) @@ -39,6 +43,7 @@ go_test( name = "streamingest_test", srcs = [ "main_test.go", + "stream_ingestion_frontier_processor_test.go", "stream_ingestion_job_test.go", "stream_ingestion_processor_test.go", ], @@ -72,6 +77,7 @@ go_test( "//pkg/util/protoutil", "//pkg/util/randutil", "//pkg/util/timeutil", + "@com_github_cockroachdb_errors//:errors", "@com_github_stretchr_testify//require", ], ) diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_frontier_processor.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_frontier_processor.go new file mode 100644 index 000000000000..076a1461568c --- /dev/null +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_frontier_processor.go @@ -0,0 +1,173 @@ +// 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" + + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "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/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/cockroach/pkg/util/span" + "github.com/cockroachdb/errors" + "github.com/cockroachdb/redact" +) + +const streamIngestionFrontierProcName = `ingestfntr` + +type streamIngestionFrontier struct { + execinfra.ProcessorBase + execinfra.StreamingProcessor + + flowCtx *execinfra.FlowCtx + spec execinfrapb.StreamIngestionFrontierSpec + a rowenc.DatumAlloc + + // input returns rows from one or more streamIngestion processors + input execinfra.RowSource + // highWaterAtStart is the job high-water. It's used in an assertion that we + // never regress the job high-water. + highWaterAtStart hlc.Timestamp + + // frontier contains the current resolved timestamp high-water for the tracked + // span set. + frontier *span.Frontier +} + +var _ execinfra.Processor = &streamIngestionFrontier{} +var _ execinfra.RowSource = &streamIngestionFrontier{} + +func newStreamIngestionFrontierProcessor( + flowCtx *execinfra.FlowCtx, + processorID int32, + spec execinfrapb.StreamIngestionFrontierSpec, + input execinfra.RowSource, + post *execinfrapb.PostProcessSpec, + output execinfra.RowReceiver, +) (execinfra.Processor, error) { + sf := &streamIngestionFrontier{ + flowCtx: flowCtx, + spec: spec, + input: input, + highWaterAtStart: spec.HighWaterAtStart, + frontier: span.MakeFrontier(spec.TrackedSpans...), + } + if err := sf.Init( + sf, + post, + input.OutputTypes(), + flowCtx, + processorID, + output, + nil, /* memMonitor */ + execinfra.ProcStateOpts{ + InputsToDrain: []execinfra.RowSource{sf.input}, + }, + ); err != nil { + return nil, err + } + return sf, nil +} + +// Start is part of the RowSource interface. +func (sf *streamIngestionFrontier) Start(ctx context.Context) context.Context { + sf.input.Start(ctx) + ctx = sf.StartInternal(ctx, streamIngestionFrontierProcName) + return ctx +} + +// Next is part of the RowSource interface. +func (sf *streamIngestionFrontier) Next() (rowenc.EncDatumRow, *execinfrapb.ProducerMetadata) { + for sf.State == execinfra.StateRunning { + row, meta := sf.input.Next() + if meta != nil { + if meta.Err != nil { + sf.MoveToDraining(nil /* err */) + } + return nil, meta + } + if row == nil { + sf.MoveToDraining(nil /* err */) + break + } + + var frontierChanged bool + var err error + if frontierChanged, err = sf.noteResolvedTimestamp(row[0]); err != nil { + sf.MoveToDraining(err) + break + } + if frontierChanged { + // Send back a row to the job so that it can update the progress. + newResolvedTS := sf.frontier.Frontier() + progressBytes, err := protoutil.Marshal(&newResolvedTS) + if err != nil { + sf.MoveToDraining(err) + return nil, sf.DrainHelper() + } + pushRow := rowenc.EncDatumRow{ + rowenc.DatumToEncDatum(types.Bytes, tree.NewDBytes(tree.DBytes(progressBytes))), + } + if outRow := sf.ProcessRowHelper(pushRow); outRow != nil { + return outRow, nil + } + } + } + return nil, sf.DrainHelper() +} + +func (sf *streamIngestionFrontier) noteResolvedTimestamp(d rowenc.EncDatum) (bool, error) { + var frontierChanged bool + if err := d.EnsureDecoded(streamIngestionResultTypes[0], &sf.a); err != nil { + return frontierChanged, err + } + raw, ok := d.Datum.(*tree.DBytes) + if !ok { + return frontierChanged, errors.AssertionFailedf(`unexpected datum type %T: %s`, d.Datum, + d.Datum) + } + var resolved jobspb.ResolvedSpan + if err := protoutil.Unmarshal([]byte(*raw), &resolved); err != nil { + return frontierChanged, errors.NewAssertionErrorWithWrappedErrf(err, + `unmarshalling resolved timestamp: %x`, raw) + } + + // Inserting a timestamp less than the one the ingestion flow started at could + // potentially regress the job progress. This is not expected and thus we + // assert to catch such unexpected behavior. + if !resolved.Timestamp.IsEmpty() && resolved.Timestamp.Less(sf.highWaterAtStart) { + return frontierChanged, errors.AssertionFailedf( + `got a resolved timestamp %s that is less than the frontier processor start time %s`, + redact.Safe(resolved.Timestamp), redact.Safe(sf.highWaterAtStart)) + } + + return sf.maybeMoveFrontier(resolved.Span, resolved.Timestamp), nil +} + +// maybeMoveFrontier updates the resolved ts for the provided span, and returns +// true if the update causes the frontier to move to higher resolved ts. +func (sf *streamIngestionFrontier) maybeMoveFrontier( + span roachpb.Span, resolved hlc.Timestamp, +) bool { + prevResolved := sf.frontier.Frontier() + sf.frontier.Forward(span, resolved) + return prevResolved.Less(sf.frontier.Frontier()) +} + +// ConsumerClosed is part of the RowSource interface. +func (sf *streamIngestionFrontier) ConsumerClosed() { + // The consumer is done, Next() will not be called again. + sf.InternalClose() +} diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_frontier_processor_test.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_frontier_processor_test.go new file mode 100644 index 000000000000..7eda9458777d --- /dev/null +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_frontier_processor_test.go @@ -0,0 +1,239 @@ +// Copyright 2020 The Cockroach Authors. +// +// Licensed as a CockroachDB Enterprise file under the Cockroach Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt + +package streamingest + +import ( + "context" + "fmt" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl" + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" + "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/distsqlutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/stretchr/testify/require" +) + +type partitionToEvent map[streamingccl.PartitionAddress][]streamingccl.Event + +func TestStreamIngestionFrontierProcessor(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx := context.Background() + + tc := testcluster.StartTestCluster(t, 3 /* nodes */, base.TestClusterArgs{}) + defer tc.Stopper().Stop(context.Background()) + kvDB := tc.Server(0).DB() + + st := cluster.MakeTestingClusterSettings() + evalCtx := tree.MakeTestingEvalContext(st) + + testDiskMonitor := execinfra.NewTestDiskMonitor(ctx, st) + defer testDiskMonitor.Stop(ctx) + + registry := tc.Server(0).JobRegistry().(*jobs.Registry) + flowCtx := execinfra.FlowCtx{ + Cfg: &execinfra.ServerConfig{ + Settings: st, + DB: kvDB, + DiskMonitor: testDiskMonitor, + JobRegistry: registry, + }, + EvalCtx: &evalCtx, + } + + out := &distsqlutils.RowBuffer{} + post := execinfrapb.PostProcessSpec{} + + var spec execinfrapb.StreamIngestionDataSpec + pa1 := streamingccl.PartitionAddress("s3://my_streams/stream/partition1") + pa2 := streamingccl.PartitionAddress("s3://my_streams/stream/partition2") + + v := roachpb.MakeValueFromString("value_1") + v.Timestamp = hlc.Timestamp{WallTime: 1} + sampleKV := roachpb.KeyValue{Key: roachpb.Key("key_1"), Value: v} + + for _, tc := range []struct { + name string + events partitionToEvent + expectedFrontierTimestamp hlc.Timestamp + frontierStartTime hlc.Timestamp + }{ + { + name: "same-resolved-ts-across-partitions", + events: partitionToEvent{pa1: []streamingccl.Event{ + streamingccl.MakeCheckpointEvent(hlc.Timestamp{WallTime: 1}), + streamingccl.MakeCheckpointEvent(hlc.Timestamp{WallTime: 4}), + }, pa2: []streamingccl.Event{ + streamingccl.MakeCheckpointEvent(hlc.Timestamp{WallTime: 1}), + streamingccl.MakeCheckpointEvent(hlc.Timestamp{WallTime: 4}), + }}, + expectedFrontierTimestamp: hlc.Timestamp{WallTime: 4}, + }, + { + // No progress should be reported to the job since partition 2 has not + // emitted a resolved ts. + name: "no-checkpoints", + events: partitionToEvent{pa1: []streamingccl.Event{ + streamingccl.MakeKVEvent(sampleKV), + }, pa2: []streamingccl.Event{ + streamingccl.MakeKVEvent(sampleKV), + }}, + }, + { + // No progress should be reported to the job since partition 2 has not + // emitted a resolved ts. + name: "no-checkpoint-from-one-partition", + events: partitionToEvent{pa1: []streamingccl.Event{ + streamingccl.MakeCheckpointEvent(hlc.Timestamp{WallTime: 1}), + streamingccl.MakeCheckpointEvent(hlc.Timestamp{WallTime: 4}), + }, pa2: []streamingccl.Event{}}, + }, + { + name: "one-partition-ahead-of-the-other", + events: partitionToEvent{pa1: []streamingccl.Event{ + streamingccl.MakeCheckpointEvent(hlc.Timestamp{WallTime: 1}), + streamingccl.MakeCheckpointEvent(hlc.Timestamp{WallTime: 4}), + }, pa2: []streamingccl.Event{ + streamingccl.MakeCheckpointEvent(hlc.Timestamp{WallTime: 1}), + }}, + expectedFrontierTimestamp: hlc.Timestamp{WallTime: 1}, + }, + { + name: "some-interleaved-timestamps", + events: partitionToEvent{pa1: []streamingccl.Event{ + streamingccl.MakeCheckpointEvent(hlc.Timestamp{WallTime: 2}), + streamingccl.MakeCheckpointEvent(hlc.Timestamp{WallTime: 4}), + }, pa2: []streamingccl.Event{ + streamingccl.MakeCheckpointEvent(hlc.Timestamp{WallTime: 3}), + streamingccl.MakeCheckpointEvent(hlc.Timestamp{WallTime: 5}), + }}, + expectedFrontierTimestamp: hlc.Timestamp{WallTime: 4}, + }, + { + name: "some-interleaved-logical-timestamps", + events: partitionToEvent{pa1: []streamingccl.Event{ + streamingccl.MakeCheckpointEvent(hlc.Timestamp{WallTime: 1, Logical: 2}), + streamingccl.MakeCheckpointEvent(hlc.Timestamp{WallTime: 1, Logical: 4}), + }, pa2: []streamingccl.Event{ + streamingccl.MakeCheckpointEvent(hlc.Timestamp{WallTime: 1, Logical: 1}), + streamingccl.MakeCheckpointEvent(hlc.Timestamp{WallTime: 2}), + }}, + expectedFrontierTimestamp: hlc.Timestamp{WallTime: 1, Logical: 4}, + }, + { + // The frontier should error out as it receives a checkpoint with a ts + // lower than its start time. + name: "checkpoint-lower-than-start-ts", + events: partitionToEvent{pa1: []streamingccl.Event{ + streamingccl.MakeCheckpointEvent(hlc.Timestamp{WallTime: 1, Logical: 4}), + }, pa2: []streamingccl.Event{ + streamingccl.MakeCheckpointEvent(hlc.Timestamp{WallTime: 1, Logical: 2}), + }}, + frontierStartTime: hlc.Timestamp{WallTime: 1, Logical: 3}, + }, + } { + t.Run(tc.name, func(t *testing.T) { + spec.PartitionAddresses = []streamingccl.PartitionAddress{pa1, pa2} + proc, err := newStreamIngestionDataProcessor(&flowCtx, 0 /* processorID */, spec, &post, out) + require.NoError(t, err) + sip, ok := proc.(*streamIngestionProcessor) + if !ok { + t.Fatal("expected the processor that's created to be a stream ingestion processor") + } + + // Inject a mock client with the events being tested against. + sip.client = &mockStreamClient{ + partitionEvents: tc.events, + } + + // Create a frontier processor. + var frontierSpec execinfrapb.StreamIngestionFrontierSpec + pa1Key := roachpb.Key(pa1) + pa2Key := roachpb.Key(pa2) + frontierSpec.TrackedSpans = []roachpb.Span{{Key: pa1Key, EndKey: pa1Key.Next()}, {Key: pa2Key, + EndKey: pa2Key.Next()}} + + if !tc.frontierStartTime.IsEmpty() { + frontierSpec.HighWaterAtStart = tc.frontierStartTime + } + + // Create a mock ingestion job. + record := jobs.Record{ + Description: "fake ingestion job", + Username: security.TestUserName(), + Details: jobspb.StreamIngestionDetails{StreamAddress: "foo"}, + // We don't use this so it does not matter what we set it too, as long + // as it is non-nil. + Progress: jobspb.ImportProgress{}, + } + record.CreatedBy = &jobs.CreatedByInfo{ + Name: "ingestion", + } + + frontierPost := execinfrapb.PostProcessSpec{} + frontierOut := distsqlutils.RowBuffer{} + frontierProc, err := newStreamIngestionFrontierProcessor(&flowCtx, 0, /* processorID*/ + frontierSpec, sip, &frontierPost, &frontierOut) + require.NoError(t, err) + fp, ok := frontierProc.(*streamIngestionFrontier) + if !ok { + t.Fatal("expected the processor that's created to be a stream ingestion frontier") + } + ctxWithCancel, cancel := context.WithCancel(ctx) + defer cancel() + fp.Run(ctxWithCancel) + + if !frontierOut.ProducerClosed() { + t.Fatal("producer for StreamFrontierProcessor not closed") + } + + var prevTimestamp hlc.Timestamp + for { + row, meta := frontierOut.Next() + if meta != nil { + if !tc.frontierStartTime.IsEmpty() { + require.True(t, testutils.IsError(meta.Err, fmt.Sprintf("got a resolved timestamp ."+ + "* that is less than the frontier processor start time %s", + tc.frontierStartTime.String()))) + return + } + t.Fatalf("unexpected meta record returned by frontier processor: %+v\n", *meta) + } + if row == nil { + break + } + datum := row[0].Datum + protoBytes, ok := datum.(*tree.DBytes) + require.True(t, ok) + + var ingestedTimestamp hlc.Timestamp + require.NoError(t, protoutil.Unmarshal([]byte(*protoBytes), &ingestedTimestamp)) + // Ensure that the rows emitted by the frontier never regress the ts. + if !prevTimestamp.IsEmpty() { + require.True(t, prevTimestamp.Less(ingestedTimestamp)) + } + prevTimestamp = ingestedTimestamp + } + // Check the final ts recorded by the frontier. + require.Equal(t, tc.expectedFrontierTimestamp, prevTimestamp) + }) + } +} diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_job.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_job.go index fdb0e088911d..0263c4367b2b 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_job.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_job.go @@ -30,7 +30,8 @@ func ingest( ctx context.Context, execCtx sql.JobExecContext, streamAddress streamingccl.StreamAddress, - job *jobs.Job, + progress jobspb.Progress, + jobID int64, ) error { // Initialize a stream client and resolve topology. client, err := streamclient.NewStreamClient(streamAddress) @@ -42,6 +43,12 @@ func ingest( return err } + // TODO(adityamaru): If the job is being resumed it is possible that it has + // check-pointed a resolved ts up to which all of its processors had ingested + // KVs. We can skip to ingesting after this resolved ts. Plumb the + // initialHighwatermark to the ingestion processor spec based on what we read + // from the job progress. + evalCtx := execCtx.ExtendedEvalContext() dsp := execCtx.DistSQLPlanner() @@ -51,7 +58,8 @@ func ingest( } // Construct stream ingestion processor specs. - streamIngestionSpecs, err := distStreamIngestionPlanSpecs(topology, nodes) + streamIngestionSpecs, streamIngestionFrontierSpec, err := distStreamIngestionPlanSpecs(topology, + nodes, jobID) if err != nil { return err } @@ -59,7 +67,8 @@ func ingest( // Plan and run the DistSQL flow. // TODO: Updates from this flow need to feed back into the job to update the // progress. - err = distStreamIngest(ctx, execCtx, nodes, planCtx, dsp, streamIngestionSpecs) + err = distStreamIngest(ctx, execCtx, nodes, jobID, planCtx, dsp, streamIngestionSpecs, + streamIngestionFrontierSpec) if err != nil { return err } @@ -72,7 +81,8 @@ func (s *streamIngestionResumer) Resume(ctx context.Context, execCtx interface{} details := s.job.Details().(jobspb.StreamIngestionDetails) p := execCtx.(sql.JobExecContext) - err := ingest(ctx, p, details.StreamAddress, s.job) + err := ingest(ctx, p, details.StreamAddress, s.job.Progress(), + *s.job.ID()) if err != nil { return err } diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor_planning.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor_planning.go index d0329e05f89f..7925f4bb2c55 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor_planning.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor_planning.go @@ -12,22 +12,33 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl" + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/physicalplan" + "github.com/cockroachdb/cockroach/pkg/sql/rowexec" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/errors" "github.com/cockroachdb/logtags" ) +func init() { + rowexec.NewStreamIngestionFrontierProcessor = newStreamIngestionFrontierProcessor +} + func distStreamIngestionPlanSpecs( - topology streamingccl.Topology, nodes []roachpb.NodeID, -) ([]*execinfrapb.StreamIngestionDataSpec, error) { + topology streamingccl.Topology, nodes []roachpb.NodeID, jobID int64, +) ([]*execinfrapb.StreamIngestionDataSpec, *execinfrapb.StreamIngestionFrontierSpec, error) { // For each stream partition in the topology, assign it to a node. streamIngestionSpecs := make([]*execinfrapb.StreamIngestionDataSpec, 0, len(nodes)) + trackedSpans := make([]roachpb.Span, 0) for i, partition := range topology.Partitions { // Round robin assign the stream partitions to nodes. Partitions 0 through // len(nodes) - 1 creates the spec. Future partitions just add themselves to @@ -39,19 +50,35 @@ func distStreamIngestionPlanSpecs( streamIngestionSpecs = append(streamIngestionSpecs, spec) } n := i % len(nodes) - streamIngestionSpecs[n].PartitionAddresses = append(streamIngestionSpecs[n].PartitionAddresses, partition) + streamIngestionSpecs[n].PartitionAddresses = append(streamIngestionSpecs[n].PartitionAddresses, + partition) + partitionKey := roachpb.Key(partition) + // We create "fake" spans to uniquely identify the partition. This is used + // to keep track of the resolved ts received for a particular partition in + // the frontier processor. + trackedSpans = append(trackedSpans, roachpb.Span{ + Key: partitionKey, + EndKey: partitionKey.Next(), + }) } - return streamIngestionSpecs, nil + // Create a spec for the StreamIngestionFrontier processor on the coordinator + // node. + // TODO: set HighWaterAtStart once the job progress logic has been hooked up. + streamIngestionFrontierSpec := &execinfrapb.StreamIngestionFrontierSpec{TrackedSpans: trackedSpans} + + return streamIngestionSpecs, streamIngestionFrontierSpec, nil } func distStreamIngest( ctx context.Context, execCtx sql.JobExecContext, nodes []roachpb.NodeID, + jobID int64, planCtx *sql.PlanningCtx, dsp *sql.DistSQLPlanner, streamIngestionSpecs []*execinfrapb.StreamIngestionDataSpec, + streamIngestionFrontierSpec *execinfrapb.StreamIngestionFrontierSpec, ) error { ctx = logtags.AddTag(ctx, "stream-ingest-distsql", nil) evalCtx := execCtx.ExtendedEvalContext() @@ -76,20 +103,26 @@ func distStreamIngest( execinfrapb.Ordering{}, ) - // TODO(adityamaru): It is likely that we will add a StreamIngestFrontier - // processor on the coordinator node. All the StreamIngestionProcessors will - // feed their results into this frontier. This is similar to the relationship - // between the ChangeAggregator and ChangeFrontier processors. The - // StreamIngestFrontier will be responsible for updating the job watermark - // with the min of the resolved ts outputted by all the processors. + 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.AddSingleGroupStage(gatewayNodeID, + execinfrapb.ProcessorCoreUnion{StreamIngestionFrontier: streamIngestionFrontierSpec}, + execinfrapb.PostProcessSpec{}, streamIngestionResultTypes) // TODO(adityamaru): Once result types are updated, add PlanToStreamColMap. dsp.FinalizePlan(planCtx, p) + rw := makeStreamIngestionResultWriter(ctx, jobID, execCfg.JobRegistry) + recv := sql.MakeDistSQLReceiver( ctx, - // TODO(adityamaru): Are there any results we want to surface to the user? - nil, /* resultWriter */ + rw, tree.Rows, nil, /* rangeCache */ noTxn, @@ -103,3 +136,62 @@ func distStreamIngest( dsp.Run(planCtx, noTxn, p, recv, &evalCtxCopy, nil /* finishedSetupFn */) return nil } + +type streamIngestionResultWriter struct { + ctx context.Context + registry *jobs.Registry + jobID int64 + rowsAffected int + err error +} + +func makeStreamIngestionResultWriter( + ctx context.Context, jobID int64, registry *jobs.Registry, +) *streamIngestionResultWriter { + return &streamIngestionResultWriter{ + ctx: ctx, + registry: registry, + jobID: jobID, + } +} + +// AddRow implements the sql.rowResultWriter interface. +func (s *streamIngestionResultWriter) AddRow(ctx context.Context, row tree.Datums) error { + if len(row) == 0 { + return errors.New("streamIngestionResultWriter received an empty row") + } + if row[0] == nil { + return errors.New("streamIngestionResultWriter expects non-nil row entry") + } + + job, err := s.registry.LoadJob(ctx, s.jobID) + if err != nil { + return err + } + return job.HighWaterProgressed(s.ctx, func(ctx context.Context, txn *kv.Txn, + details jobspb.ProgressDetails) (hlc.Timestamp, error) { + // Decode the row and write the ts. + var ingestedHighWatermark hlc.Timestamp + if err := protoutil.Unmarshal([]byte(*row[0].(*tree.DBytes)), + &ingestedHighWatermark); err != nil { + return ingestedHighWatermark, errors.NewAssertionErrorWithWrappedErrf(err, + `unmarshalling resolved timestamp`) + } + return ingestedHighWatermark, nil + }) +} + +// IncrementRowsAffected implements the sql.rowResultWriter interface. +func (s *streamIngestionResultWriter) IncrementRowsAffected(n int) { + s.rowsAffected += n +} + +// SetError implements the sql.rowResultWriter interface. +func (s *streamIngestionResultWriter) SetError(err error) { + s.err = err +} + +// Err implements the sql.rowResultWriter interface. +func (s *streamIngestionResultWriter) Err() error { + return s.err +} diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor_test.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor_test.go index 581ee81a7420..314d0aaa942d 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor_test.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_processor_test.go @@ -34,6 +34,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" ) @@ -43,10 +44,11 @@ type interceptableStreamClient interface { RegisterInterception(func(event streamingccl.Event)) } -// mockStreamClient will always return the given slice of events when consuming -// a stream partition. +// mockStreamClient will return the slice of events associated to the stream +// partition being consumed. Stream partitions are identified by unique +// partition addresses. type mockStreamClient struct { - partitionEvents []streamingccl.Event + partitionEvents map[streamingccl.PartitionAddress][]streamingccl.Event } var _ streamclient.Client = &mockStreamClient{} @@ -60,11 +62,17 @@ func (m *mockStreamClient) GetTopology( // ConsumePartition implements the StreamClient interface. func (m *mockStreamClient) ConsumePartition( - _ context.Context, _ streamingccl.PartitionAddress, _ time.Time, + _ context.Context, address streamingccl.PartitionAddress, _ time.Time, ) (chan streamingccl.Event, error) { - eventCh := make(chan streamingccl.Event, len(m.partitionEvents)) + var events []streamingccl.Event + var ok bool + if events, ok = m.partitionEvents[address]; !ok { + return nil, errors.Newf("no events found for paritition %s", address) + } + + eventCh := make(chan streamingccl.Event, len(events)) - for _, event := range m.partitionEvents { + for _, event := range events { eventCh <- event } close(eventCh) @@ -85,18 +93,22 @@ func TestStreamIngestionProcessor(t *testing.T) { defer tc.Stopper().Stop(ctx) kvDB := tc.Server(0).DB() + // Inject a mock client. v := roachpb.MakeValueFromString("value_1") v.Timestamp = hlc.Timestamp{WallTime: 1} sampleKV := roachpb.KeyValue{Key: roachpb.Key("key_1"), Value: v} + events := []streamingccl.Event{ + streamingccl.MakeKVEvent(sampleKV), + streamingccl.MakeKVEvent(sampleKV), + streamingccl.MakeCheckpointEvent(hlc.Timestamp{WallTime: 1}), + streamingccl.MakeKVEvent(sampleKV), + streamingccl.MakeKVEvent(sampleKV), + streamingccl.MakeCheckpointEvent(hlc.Timestamp{WallTime: 4}), + } + pa1 := streamingccl.PartitionAddress("partition1") + pa2 := streamingccl.PartitionAddress("partition2") mockClient := &mockStreamClient{ - partitionEvents: []streamingccl.Event{ - streamingccl.MakeKVEvent(sampleKV), - streamingccl.MakeKVEvent(sampleKV), - streamingccl.MakeCheckpointEvent(hlc.Timestamp{WallTime: 1}), - streamingccl.MakeKVEvent(sampleKV), - streamingccl.MakeKVEvent(sampleKV), - streamingccl.MakeCheckpointEvent(hlc.Timestamp{WallTime: 4}), - }, + partitionEvents: map[streamingccl.PartitionAddress][]streamingccl.Event{pa1: events, pa2: events}, } startTime := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} diff --git a/pkg/sql/execinfrapb/processors.pb.go b/pkg/sql/execinfrapb/processors.pb.go index 8cbd62d278a1..0676136e38be 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_f89a954727b0d81b, []int{0} + return fileDescriptor_processors_5e45e1d8c39180fc, []int{0} } func (m *ProcessorSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -103,44 +103,45 @@ func (m *ProcessorSpec) XXX_DiscardUnknown() { var xxx_messageInfo_ProcessorSpec proto.InternalMessageInfo type ProcessorCoreUnion struct { - Noop *NoopCoreSpec `protobuf:"bytes,1,opt,name=noop" json:"noop,omitempty"` - TableReader *TableReaderSpec `protobuf:"bytes,2,opt,name=tableReader" json:"tableReader,omitempty"` - JoinReader *JoinReaderSpec `protobuf:"bytes,3,opt,name=joinReader" json:"joinReader,omitempty"` - Sorter *SorterSpec `protobuf:"bytes,4,opt,name=sorter" json:"sorter,omitempty"` - Aggregator *AggregatorSpec `protobuf:"bytes,5,opt,name=aggregator" json:"aggregator,omitempty"` - Distinct *DistinctSpec `protobuf:"bytes,7,opt,name=distinct" json:"distinct,omitempty"` - MergeJoiner *MergeJoinerSpec `protobuf:"bytes,8,opt,name=mergeJoiner" json:"mergeJoiner,omitempty"` - HashJoiner *HashJoinerSpec `protobuf:"bytes,9,opt,name=hashJoiner" json:"hashJoiner,omitempty"` - Values *ValuesCoreSpec `protobuf:"bytes,10,opt,name=values" json:"values,omitempty"` - Backfiller *BackfillerSpec `protobuf:"bytes,11,opt,name=backfiller" json:"backfiller,omitempty"` - ReadImport *ReadImportDataSpec `protobuf:"bytes,13,opt,name=readImport" json:"readImport,omitempty"` - CSVWriter *CSVWriterSpec `protobuf:"bytes,20,opt,name=CSVWriter" json:"CSVWriter,omitempty"` - Sampler *SamplerSpec `protobuf:"bytes,15,opt,name=Sampler" json:"Sampler,omitempty"` - SampleAggregator *SampleAggregatorSpec `protobuf:"bytes,16,opt,name=SampleAggregator" json:"SampleAggregator,omitempty"` - MetadataTestSender *MetadataTestSenderSpec `protobuf:"bytes,18,opt,name=metadataTestSender" json:"metadataTestSender,omitempty"` - MetadataTestReceiver *MetadataTestReceiverSpec `protobuf:"bytes,19,opt,name=metadataTestReceiver" json:"metadataTestReceiver,omitempty"` - ZigzagJoiner *ZigzagJoinerSpec `protobuf:"bytes,21,opt,name=zigzagJoiner" json:"zigzagJoiner,omitempty"` - ProjectSet *ProjectSetSpec `protobuf:"bytes,22,opt,name=projectSet" json:"projectSet,omitempty"` - Windower *WindowerSpec `protobuf:"bytes,23,opt,name=windower" json:"windower,omitempty"` - LocalPlanNode *LocalPlanNodeSpec `protobuf:"bytes,24,opt,name=localPlanNode" json:"localPlanNode,omitempty"` - ChangeAggregator *ChangeAggregatorSpec `protobuf:"bytes,25,opt,name=changeAggregator" json:"changeAggregator,omitempty"` - ChangeFrontier *ChangeFrontierSpec `protobuf:"bytes,26,opt,name=changeFrontier" json:"changeFrontier,omitempty"` - Ordinality *OrdinalitySpec `protobuf:"bytes,27,opt,name=ordinality" json:"ordinality,omitempty"` - BulkRowWriter *BulkRowWriterSpec `protobuf:"bytes,28,opt,name=bulkRowWriter" json:"bulkRowWriter,omitempty"` - InvertedFilterer *InvertedFiltererSpec `protobuf:"bytes,29,opt,name=invertedFilterer" json:"invertedFilterer,omitempty"` - InvertedJoiner *InvertedJoinerSpec `protobuf:"bytes,30,opt,name=invertedJoiner" json:"invertedJoiner,omitempty"` - BackupData *BackupDataSpec `protobuf:"bytes,31,opt,name=backupData" json:"backupData,omitempty"` - SplitAndScatter *SplitAndScatterSpec `protobuf:"bytes,32,opt,name=splitAndScatter" json:"splitAndScatter,omitempty"` - RestoreData *RestoreDataSpec `protobuf:"bytes,33,opt,name=restoreData" json:"restoreData,omitempty"` - Filterer *FiltererSpec `protobuf:"bytes,34,opt,name=filterer" json:"filterer,omitempty"` - StreamIngestionData *StreamIngestionDataSpec `protobuf:"bytes,35,opt,name=streamIngestionData" json:"streamIngestionData,omitempty"` + Noop *NoopCoreSpec `protobuf:"bytes,1,opt,name=noop" json:"noop,omitempty"` + TableReader *TableReaderSpec `protobuf:"bytes,2,opt,name=tableReader" json:"tableReader,omitempty"` + JoinReader *JoinReaderSpec `protobuf:"bytes,3,opt,name=joinReader" json:"joinReader,omitempty"` + Sorter *SorterSpec `protobuf:"bytes,4,opt,name=sorter" json:"sorter,omitempty"` + Aggregator *AggregatorSpec `protobuf:"bytes,5,opt,name=aggregator" json:"aggregator,omitempty"` + Distinct *DistinctSpec `protobuf:"bytes,7,opt,name=distinct" json:"distinct,omitempty"` + MergeJoiner *MergeJoinerSpec `protobuf:"bytes,8,opt,name=mergeJoiner" json:"mergeJoiner,omitempty"` + HashJoiner *HashJoinerSpec `protobuf:"bytes,9,opt,name=hashJoiner" json:"hashJoiner,omitempty"` + Values *ValuesCoreSpec `protobuf:"bytes,10,opt,name=values" json:"values,omitempty"` + Backfiller *BackfillerSpec `protobuf:"bytes,11,opt,name=backfiller" json:"backfiller,omitempty"` + ReadImport *ReadImportDataSpec `protobuf:"bytes,13,opt,name=readImport" json:"readImport,omitempty"` + CSVWriter *CSVWriterSpec `protobuf:"bytes,20,opt,name=CSVWriter" json:"CSVWriter,omitempty"` + Sampler *SamplerSpec `protobuf:"bytes,15,opt,name=Sampler" json:"Sampler,omitempty"` + SampleAggregator *SampleAggregatorSpec `protobuf:"bytes,16,opt,name=SampleAggregator" json:"SampleAggregator,omitempty"` + MetadataTestSender *MetadataTestSenderSpec `protobuf:"bytes,18,opt,name=metadataTestSender" json:"metadataTestSender,omitempty"` + MetadataTestReceiver *MetadataTestReceiverSpec `protobuf:"bytes,19,opt,name=metadataTestReceiver" json:"metadataTestReceiver,omitempty"` + ZigzagJoiner *ZigzagJoinerSpec `protobuf:"bytes,21,opt,name=zigzagJoiner" json:"zigzagJoiner,omitempty"` + ProjectSet *ProjectSetSpec `protobuf:"bytes,22,opt,name=projectSet" json:"projectSet,omitempty"` + Windower *WindowerSpec `protobuf:"bytes,23,opt,name=windower" json:"windower,omitempty"` + LocalPlanNode *LocalPlanNodeSpec `protobuf:"bytes,24,opt,name=localPlanNode" json:"localPlanNode,omitempty"` + ChangeAggregator *ChangeAggregatorSpec `protobuf:"bytes,25,opt,name=changeAggregator" json:"changeAggregator,omitempty"` + ChangeFrontier *ChangeFrontierSpec `protobuf:"bytes,26,opt,name=changeFrontier" json:"changeFrontier,omitempty"` + Ordinality *OrdinalitySpec `protobuf:"bytes,27,opt,name=ordinality" json:"ordinality,omitempty"` + BulkRowWriter *BulkRowWriterSpec `protobuf:"bytes,28,opt,name=bulkRowWriter" json:"bulkRowWriter,omitempty"` + InvertedFilterer *InvertedFiltererSpec `protobuf:"bytes,29,opt,name=invertedFilterer" json:"invertedFilterer,omitempty"` + InvertedJoiner *InvertedJoinerSpec `protobuf:"bytes,30,opt,name=invertedJoiner" json:"invertedJoiner,omitempty"` + BackupData *BackupDataSpec `protobuf:"bytes,31,opt,name=backupData" json:"backupData,omitempty"` + SplitAndScatter *SplitAndScatterSpec `protobuf:"bytes,32,opt,name=splitAndScatter" json:"splitAndScatter,omitempty"` + RestoreData *RestoreDataSpec `protobuf:"bytes,33,opt,name=restoreData" json:"restoreData,omitempty"` + 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"` } 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_f89a954727b0d81b, []int{1} + return fileDescriptor_processors_5e45e1d8c39180fc, []int{1} } func (m *ProcessorCoreUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -175,7 +176,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_f89a954727b0d81b, []int{2} + return fileDescriptor_processors_5e45e1d8c39180fc, []int{2} } func (m *NoopCoreSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -217,7 +218,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_f89a954727b0d81b, []int{3} + return fileDescriptor_processors_5e45e1d8c39180fc, []int{3} } func (m *LocalPlanNodeSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -250,7 +251,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_f89a954727b0d81b, []int{4} + return fileDescriptor_processors_5e45e1d8c39180fc, []int{4} } func (m *MetadataTestSenderSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -283,7 +284,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_f89a954727b0d81b, []int{5} + return fileDescriptor_processors_5e45e1d8c39180fc, []int{5} } func (m *MetadataTestReceiverSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -755,6 +756,18 @@ func (m *ProcessorCoreUnion) MarshalTo(dAtA []byte) (int, error) { } i += n33 } + if m.StreamIngestionFrontier != nil { + dAtA[i] = 0xa2 + i++ + dAtA[i] = 0x2 + i++ + i = encodeVarintProcessors(dAtA, i, uint64(m.StreamIngestionFrontier.Size())) + n34, err := m.StreamIngestionFrontier.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n34 + } return i, nil } @@ -1031,6 +1044,10 @@ func (m *ProcessorCoreUnion) Size() (n int) { l = m.StreamIngestionData.Size() n += 2 + l + sovProcessors(uint64(l)) } + if m.StreamIngestionFrontier != nil { + l = m.StreamIngestionFrontier.Size() + n += 2 + l + sovProcessors(uint64(l)) + } return n } @@ -1189,6 +1206,9 @@ func (this *ProcessorCoreUnion) GetValue() interface{} { if this.StreamIngestionData != nil { return this.StreamIngestionData } + if this.StreamIngestionFrontier != nil { + return this.StreamIngestionFrontier + } return nil } @@ -1256,6 +1276,8 @@ func (this *ProcessorCoreUnion) SetValue(value interface{}) bool { this.Filterer = vt case *StreamIngestionDataSpec: this.StreamIngestionData = vt + case *StreamIngestionFrontierSpec: + this.StreamIngestionFrontier = vt default: return false } @@ -2554,6 +2576,39 @@ func (m *ProcessorCoreUnion) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 36: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StreamIngestionFrontier", 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.StreamIngestionFrontier == nil { + m.StreamIngestionFrontier = &StreamIngestionFrontierSpec{} + } + if err := m.StreamIngestionFrontier.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipProcessors(dAtA[iNdEx:]) @@ -3006,87 +3061,89 @@ var ( ) func init() { - proto.RegisterFile("sql/execinfrapb/processors.proto", fileDescriptor_processors_f89a954727b0d81b) + proto.RegisterFile("sql/execinfrapb/processors.proto", fileDescriptor_processors_5e45e1d8c39180fc) } -var fileDescriptor_processors_f89a954727b0d81b = []byte{ - // 1245 bytes of a gzipped FileDescriptorProto +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, 0x1c, 0x13, 0xa3, 0x00, 0x3f, 0xfd, 0xdc, 0xd6, 0x49, 0x5d, - 0x68, 0x5d, 0x0a, 0xa6, 0x65, 0x3a, 0xbd, 0x60, 0xda, 0x69, 0xb1, 0x5d, 0x26, 0x9b, 0xd2, 0x40, - 0xd7, 0x29, 0xcc, 0x70, 0xe3, 0x2a, 0xbb, 0x8a, 0xb3, 0xb0, 0x5e, 0x2d, 0x92, 0x4c, 0x80, 0xbe, - 0x44, 0x1f, 0xa1, 0xef, 0xd1, 0x17, 0xc8, 0x55, 0x87, 0x4b, 0xae, 0x98, 0xd6, 0xbc, 0x48, 0x47, - 0x47, 0xeb, 0xdd, 0xb5, 0x13, 0xef, 0x72, 0x93, 0x71, 0xe4, 0xef, 0xf7, 0xa3, 0xa3, 0x73, 0x8e, - 0xfe, 0x18, 0x6d, 0xcb, 0x67, 0xfe, 0x4d, 0xf6, 0x82, 0x39, 0x5e, 0x70, 0x28, 0x68, 0x78, 0x70, - 0x33, 0x14, 0xdc, 0x61, 0x52, 0x72, 0x21, 0xdb, 0xa1, 0xe0, 0x8a, 0x63, 0xe2, 0x70, 0xe7, 0xa9, - 0xe0, 0xd4, 0x39, 0x6a, 0xcb, 0x67, 0x7e, 0xdb, 0xf5, 0xa4, 0x92, 0xcf, 0x7c, 0x31, 0x0e, 0xea, - 0xf5, 0x79, 0xaf, 0x4b, 0x15, 0x35, 0xae, 0xfa, 0xd5, 0xc5, 0xdc, 0xc1, 0x01, 0x95, 0x2c, 0x92, - 0x5d, 0xc9, 0x90, 0xe9, 0xd9, 0x8c, 0xaa, 0x95, 0x05, 0x1b, 0xfb, 0x4f, 0x07, 0x1e, 0x8f, 0x94, - 0xd7, 0x33, 0x94, 0xce, 0x11, 0x0d, 0x86, 0xec, 0x90, 0x31, 0x57, 0xbe, 0x87, 0x5a, 0xd1, 0x03, - 0x9f, 0x0d, 0xa4, 0xa2, 0x6a, 0xaa, 0xbe, 0xa4, 0xd5, 0xea, 0x65, 0xc8, 0xa4, 0xf9, 0x1b, 0x0d, - 0x5f, 0x1c, 0xf2, 0x21, 0x87, 0x8f, 0x37, 0xf5, 0x27, 0x33, 0xda, 0xfc, 0x7b, 0x05, 0x55, 0x1f, - 0x4c, 0x69, 0xfd, 0x90, 0x39, 0xb8, 0x8b, 0xce, 0x79, 0x41, 0x38, 0x56, 0xa4, 0xb0, 0xbd, 0xd2, - 0xaa, 0xdc, 0xfa, 0xac, 0xbd, 0x28, 0xaf, 0x6d, 0x4b, 0xcb, 0xfa, 0x2f, 0x03, 0x47, 0xfb, 0x3a, - 0xc5, 0x93, 0xb7, 0x5b, 0x4b, 0xb6, 0xf1, 0xe2, 0xbb, 0xa8, 0xe8, 0x70, 0xc1, 0xc8, 0xf2, 0x76, - 0xa1, 0x55, 0xb9, 0x75, 0x7d, 0x31, 0x23, 0x9e, 0xbb, 0xcb, 0x05, 0xfb, 0x35, 0xf0, 0x78, 0x10, - 0x81, 0xc0, 0x8f, 0x77, 0xd0, 0x2a, 0x1f, 0x2b, 0x1d, 0xcd, 0x0a, 0x44, 0x73, 0x6d, 0x31, 0xe9, - 0x3e, 0xe8, 0x6c, 0x3e, 0x56, 0x4c, 0xa4, 0x02, 0x8a, 0xfc, 0xb8, 0x8b, 0x8a, 0x21, 0x97, 0x8a, - 0x14, 0x21, 0xa2, 0xcf, 0x33, 0x22, 0xe2, 0x52, 0x45, 0x51, 0xa5, 0x30, 0x60, 0xc6, 0xd7, 0x50, - 0x49, 0x2a, 0x3a, 0x64, 0x03, 0xcf, 0x25, 0xe7, 0xb6, 0x0b, 0xad, 0x73, 0x9d, 0x0d, 0xfd, 0xed, - 0xe4, 0xed, 0xd6, 0x5a, 0x5f, 0x8f, 0x5b, 0x3d, 0x7b, 0x0d, 0x04, 0x96, 0x8b, 0xbf, 0x41, 0xeb, - 0x71, 0x99, 0xb4, 0x7e, 0x15, 0xf4, 0x9b, 0x91, 0xbe, 0x12, 0x2f, 0xdc, 0xea, 0xd9, 0x95, 0x58, - 0x68, 0xb9, 0xf8, 0x3b, 0xb4, 0x2e, 0x98, 0x1c, 0xfb, 0x6a, 0x00, 0xd5, 0x23, 0x6b, 0xb0, 0xf0, - 0xfa, 0x5c, 0xc0, 0x92, 0x8d, 0xda, 0xa6, 0xba, 0xfb, 0x76, 0xc5, 0xe8, 0xf7, 0xf5, 0xbf, 0xcd, - 0xbf, 0x36, 0x11, 0x3e, 0x9d, 0x54, 0x7c, 0x1b, 0x15, 0x03, 0xce, 0x43, 0x52, 0x80, 0xe5, 0x7f, - 0xba, 0x78, 0xf9, 0x7b, 0x9c, 0x87, 0xda, 0xa6, 0xd7, 0x6e, 0x83, 0x07, 0xff, 0x84, 0x2a, 0xd0, - 0x65, 0x36, 0xa3, 0x2e, 0x13, 0x51, 0x4d, 0x33, 0x32, 0xb8, 0x9f, 0x88, 0x81, 0x92, 0x76, 0xe3, - 0x1d, 0x84, 0x9e, 0x70, 0x2f, 0x88, 0x58, 0x2b, 0xc0, 0x6a, 0x2d, 0x66, 0xed, 0xc6, 0x5a, 0x40, - 0xa5, 0xbc, 0xf8, 0x5b, 0xb4, 0x2a, 0xb9, 0x50, 0x4c, 0x44, 0x35, 0xbd, 0xb2, 0x98, 0xd2, 0x07, - 0x1d, 0x10, 0x22, 0x8f, 0x8e, 0x83, 0x0e, 0x87, 0x82, 0x0d, 0xa9, 0xe2, 0x02, 0x8a, 0x99, 0x19, - 0xc7, 0x9d, 0x58, 0x6b, 0xe2, 0x48, 0xbc, 0xb8, 0x83, 0x4a, 0x5a, 0xe8, 0x05, 0x8e, 0x22, 0x6b, - 0x79, 0xe9, 0xed, 0x45, 0x4a, 0xa0, 0xc4, 0x3e, 0x9d, 0xe2, 0x11, 0x13, 0x43, 0xa6, 0x97, 0xcb, - 0x04, 0x29, 0xe5, 0xa5, 0xf8, 0xe7, 0x44, 0x6c, 0x52, 0x9c, 0x72, 0xeb, 0xa5, 0x1d, 0x51, 0x79, - 0x14, 0xb1, 0xca, 0x79, 0x4b, 0xdb, 0x89, 0xb5, 0x66, 0x69, 0x89, 0x17, 0xff, 0x80, 0x56, 0x9f, - 0x53, 0x7f, 0xcc, 0x24, 0x41, 0x79, 0x94, 0x87, 0xa0, 0x8b, 0x3b, 0x27, 0xf2, 0xe9, 0x58, 0x0e, - 0xa8, 0xf3, 0xf4, 0xd0, 0xf3, 0x7d, 0x26, 0x48, 0x25, 0x8f, 0xd2, 0x89, 0xb5, 0x26, 0x96, 0xc4, - 0x8b, 0xef, 0x21, 0x24, 0x18, 0x75, 0xad, 0x51, 0xc8, 0x85, 0x22, 0xd5, 0xbc, 0x83, 0xc5, 0x8e, - 0xb5, 0x3d, 0xaa, 0xa8, 0xa1, 0x25, 0x7e, 0xfc, 0x3d, 0x5a, 0xeb, 0xd3, 0x51, 0xa8, 0x83, 0xda, - 0x00, 0xd4, 0xd5, 0x8c, 0xee, 0x31, 0x42, 0x60, 0x4c, 0x5d, 0xf8, 0x31, 0xaa, 0x99, 0x8f, 0x49, - 0x67, 0x90, 0x1a, 0x90, 0xda, 0x79, 0xa4, 0xb9, 0x5e, 0x3a, 0xc5, 0xc1, 0xbf, 0x21, 0x3c, 0x62, - 0x8a, 0xea, 0x6b, 0x6a, 0x9f, 0x49, 0xd5, 0x67, 0x81, 0xde, 0x2b, 0x18, 0xe8, 0x5f, 0x66, 0x35, - 0xc5, 0xbc, 0x07, 0xf8, 0x67, 0xb0, 0xf0, 0x21, 0xba, 0x98, 0x1e, 0xb5, 0x99, 0xc3, 0xbc, 0xe7, - 0x4c, 0x90, 0x4d, 0x98, 0xe3, 0xd6, 0xfb, 0xcd, 0x31, 0x75, 0xc1, 0x2c, 0x67, 0xf2, 0xf0, 0x8f, - 0xa8, 0xdc, 0xed, 0x3f, 0x7c, 0x24, 0x3c, 0xbd, 0x4d, 0x2f, 0x02, 0x3c, 0xe3, 0x42, 0x89, 0xa5, - 0x40, 0x4c, 0x9c, 0x78, 0x0f, 0xad, 0xbf, 0xf2, 0x86, 0xaf, 0xe8, 0x30, 0xea, 0xe9, 0x4b, 0x40, - 0xca, 0xb8, 0x0c, 0x1e, 0xa7, 0xd4, 0x00, 0x9b, 0xf1, 0xeb, 0xae, 0x0c, 0x05, 0x7f, 0xc2, 0x1c, - 0xd5, 0x67, 0x8a, 0x5c, 0xce, 0xeb, 0xca, 0x07, 0xb1, 0xd6, 0xf4, 0x51, 0xe2, 0xd5, 0x9b, 0xff, - 0xd8, 0x0b, 0x5c, 0x7e, 0xcc, 0x04, 0xf9, 0x5f, 0xde, 0xe6, 0x7f, 0x14, 0x29, 0xcd, 0xe6, 0x9f, - 0xfa, 0xf0, 0x2f, 0xa8, 0xea, 0x73, 0x87, 0xfa, 0x0f, 0x7c, 0x1a, 0xec, 0x71, 0x97, 0x11, 0x02, - 0xa0, 0x2f, 0x16, 0x83, 0xee, 0xa5, 0xe5, 0x40, 0x9b, 0x25, 0xe8, 0xee, 0x34, 0xcf, 0x88, 0x54, - 0x77, 0xfe, 0x3f, 0xaf, 0x3b, 0xbb, 0x73, 0x0e, 0xd3, 0x9d, 0xf3, 0x1c, 0xbc, 0x8f, 0xce, 0x9b, - 0xb1, 0xbb, 0x82, 0x07, 0xca, 0x63, 0x82, 0xd4, 0xf3, 0x36, 0x63, 0x77, 0x46, 0x0f, 0xdc, 0x39, - 0x86, 0x2e, 0x09, 0x17, 0xae, 0x17, 0x50, 0xdf, 0x53, 0x2f, 0xc9, 0x07, 0x79, 0x25, 0xb9, 0x1f, - 0x6b, 0x4d, 0x49, 0x12, 0xaf, 0x4e, 0xa7, 0x7e, 0x6c, 0xd9, 0xfc, 0x38, 0xea, 0xbb, 0x0f, 0xf3, - 0xd2, 0xd9, 0x49, 0xcb, 0x4d, 0x3a, 0x67, 0x08, 0x3a, 0x9d, 0x5e, 0xf0, 0x9c, 0x09, 0xc5, 0xdc, - 0xbb, 0x9e, 0xaf, 0x98, 0x60, 0x82, 0x7c, 0x94, 0x97, 0x4e, 0x6b, 0xce, 0x61, 0xd2, 0x39, 0xcf, - 0xd1, 0xe9, 0x9c, 0x8e, 0x45, 0xdd, 0xdd, 0xc8, 0x4b, 0xa7, 0x35, 0xa3, 0x37, 0xe9, 0x9c, 0x65, - 0x4c, 0xcf, 0xdd, 0x71, 0xa8, 0x4f, 0x3f, 0xb2, 0xf5, 0x3e, 0xe7, 0xae, 0xd1, 0x26, 0xe7, 0xae, - 0xf9, 0x1f, 0x3f, 0x42, 0x1b, 0x32, 0xf4, 0x3d, 0x75, 0x27, 0x70, 0xfb, 0x0e, 0x55, 0x3a, 0xa1, - 0xdb, 0x80, 0xbb, 0x91, 0x71, 0xce, 0xcd, 0x1a, 0x80, 0x39, 0x4f, 0xd1, 0x77, 0x9e, 0x60, 0x52, - 0x71, 0xc1, 0x20, 0xc6, 0x8f, 0xf3, 0xee, 0x3c, 0x3b, 0x11, 0x9b, 0x3b, 0x2f, 0xe5, 0xd6, 0xfb, - 0xf0, 0x70, 0x5a, 0x99, 0x66, 0xde, 0x3e, 0x9c, 0xa9, 0x48, 0xec, 0xc3, 0x0e, 0xda, 0x94, 0x4a, - 0x30, 0x3a, 0xb2, 0x82, 0x21, 0x93, 0xca, 0xe3, 0x01, 0x04, 0xf6, 0x09, 0xe0, 0xbe, 0xca, 0x58, - 0xed, 0x69, 0x13, 0x90, 0xcf, 0xa2, 0xdd, 0x2e, 0x9e, 0xfc, 0xb9, 0x55, 0xd8, 0x2d, 0x96, 0xce, - 0xd7, 0x36, 0x76, 0x8b, 0xa5, 0x0b, 0x35, 0xbc, 0x5b, 0x2c, 0xad, 0xd6, 0xd6, 0x76, 0x8b, 0xa5, - 0xf5, 0x5a, 0xb5, 0x79, 0x1e, 0xad, 0xa7, 0x1f, 0x60, 0xcd, 0xdf, 0xd1, 0x85, 0x53, 0x7b, 0x1d, - 0xb7, 0xd0, 0xba, 0xcd, 0x8f, 0xfb, 0x7c, 0x2c, 0x1c, 0x66, 0xb9, 0x2f, 0xe0, 0x4d, 0x57, 0x8d, - 0xde, 0xa9, 0x33, 0xdf, 0xe0, 0x26, 0x2a, 0xef, 0x8d, 0x47, 0xf0, 0x4e, 0x97, 0xf0, 0x6e, 0x9b, - 0xca, 0x92, 0x61, 0x4c, 0x50, 0x71, 0x8f, 0x8e, 0x18, 0x3c, 0xc5, 0xca, 0xd3, 0xd7, 0xae, 0x1e, - 0x69, 0x7e, 0x8d, 0x2e, 0x9f, 0x7d, 0xa5, 0xe0, 0x3a, 0x5a, 0xf6, 0x5c, 0x98, 0xb7, 0xdc, 0x41, - 0xd1, 0x8b, 0x76, 0xd9, 0xea, 0xd9, 0xcb, 0x9e, 0xdb, 0xdc, 0x41, 0x64, 0xd1, 0x25, 0x81, 0xaf, - 0x23, 0x24, 0x81, 0x32, 0xf0, 0x5c, 0x09, 0x3f, 0x30, 0xca, 0x9d, 0xea, 0xe4, 0xed, 0x56, 0xd9, - 0xb0, 0xad, 0x9e, 0xb4, 0xcb, 0x46, 0x60, 0xb9, 0xb2, 0x73, 0xe3, 0xe4, 0xdf, 0xc6, 0xd2, 0xc9, - 0xa4, 0x51, 0x78, 0x3d, 0x69, 0x14, 0xde, 0x4c, 0x1a, 0x85, 0x7f, 0x26, 0x8d, 0xc2, 0x1f, 0xef, - 0x1a, 0x4b, 0xaf, 0xdf, 0x35, 0x96, 0xde, 0xbc, 0x6b, 0x2c, 0x3d, 0xae, 0xa4, 0x7e, 0x14, 0xfd, - 0x17, 0x00, 0x00, 0xff, 0xff, 0x81, 0xe1, 0xe4, 0x3b, 0x23, 0x0e, 0x00, 0x00, + 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, } diff --git a/pkg/sql/execinfrapb/processors.proto b/pkg/sql/execinfrapb/processors.proto index 222ef2e97ea6..f797f629fcf4 100644 --- a/pkg/sql/execinfrapb/processors.proto +++ b/pkg/sql/execinfrapb/processors.proto @@ -118,6 +118,7 @@ message ProcessorCoreUnion { optional RestoreDataSpec restoreData = 33; optional FiltererSpec filterer = 34; optional StreamIngestionDataSpec streamIngestionData = 35; + optional StreamIngestionFrontierSpec streamIngestionFrontier = 36; reserved 6, 12; } diff --git a/pkg/sql/execinfrapb/processors_bulk_io.pb.go b/pkg/sql/execinfrapb/processors_bulk_io.pb.go index 4560f871184b..0ab475caabbb 100644 --- a/pkg/sql/execinfrapb/processors_bulk_io.pb.go +++ b/pkg/sql/execinfrapb/processors_bulk_io.pb.go @@ -72,7 +72,7 @@ func (x *FileCompression) UnmarshalJSON(data []byte) error { return nil } func (FileCompression) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_791817333cf840e0, []int{0} + return fileDescriptor_processors_bulk_io_4b564fcb5cc1c063, []int{0} } type BackfillerSpec_Type int32 @@ -111,7 +111,7 @@ func (x *BackfillerSpec_Type) UnmarshalJSON(data []byte) error { return nil } func (BackfillerSpec_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_791817333cf840e0, []int{0, 0} + return fileDescriptor_processors_bulk_io_4b564fcb5cc1c063, []int{0, 0} } // BackfillerSpec is the specification for a "schema change backfiller". @@ -143,7 +143,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_791817333cf840e0, []int{0} + return fileDescriptor_processors_bulk_io_4b564fcb5cc1c063, []int{0} } func (m *BackfillerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -183,7 +183,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_791817333cf840e0, []int{1} + return fileDescriptor_processors_bulk_io_4b564fcb5cc1c063, []int{1} } func (m *JobProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -246,7 +246,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_791817333cf840e0, []int{2} + return fileDescriptor_processors_bulk_io_4b564fcb5cc1c063, []int{2} } func (m *ReadImportDataSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -284,7 +284,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_791817333cf840e0, []int{2, 0} + return fileDescriptor_processors_bulk_io_4b564fcb5cc1c063, []int{2, 0} } func (m *ReadImportDataSpec_ImportTable) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -323,7 +323,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_791817333cf840e0, []int{3} + return fileDescriptor_processors_bulk_io_4b564fcb5cc1c063, []int{3} } func (m *StreamIngestionDataSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -348,6 +348,49 @@ func (m *StreamIngestionDataSpec) XXX_DiscardUnknown() { var xxx_messageInfo_StreamIngestionDataSpec proto.InternalMessageInfo +type StreamIngestionFrontierSpec struct { + // HighWaterAtStart is set by the ingestion job when initializing the frontier + // processor. It is used as sanity check by the frontier processor to ensure + // that it does not receive updates at a timestamp lower than this field. This + // consequently prevents the job progress from regressing during ingestion. + HighWaterAtStart hlc.Timestamp `protobuf:"bytes,1,opt,name=high_water_at_start,json=highWaterAtStart" json:"high_water_at_start"` + // TrackedSpans is the entire span set being watched. The spans do not really + // represent KV spans but uniquely identify the partitions in the ingestion + // stream. Once all the partitions in the ingestion stream have been resolved + // at a certain timestamp, then it's safe to resolve the ingestion at that + // timestamp. + TrackedSpans []roachpb.Span `protobuf:"bytes,2,rep,name=tracked_spans,json=trackedSpans" json:"tracked_spans"` +} + +func (m *StreamIngestionFrontierSpec) Reset() { *m = StreamIngestionFrontierSpec{} } +func (m *StreamIngestionFrontierSpec) String() string { return proto.CompactTextString(m) } +func (*StreamIngestionFrontierSpec) ProtoMessage() {} +func (*StreamIngestionFrontierSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bulk_io_4b564fcb5cc1c063, []int{4} +} +func (m *StreamIngestionFrontierSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StreamIngestionFrontierSpec) 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 *StreamIngestionFrontierSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_StreamIngestionFrontierSpec.Merge(dst, src) +} +func (m *StreamIngestionFrontierSpec) XXX_Size() int { + return m.Size() +} +func (m *StreamIngestionFrontierSpec) XXX_DiscardUnknown() { + xxx_messageInfo_StreamIngestionFrontierSpec.DiscardUnknown(m) +} + +var xxx_messageInfo_StreamIngestionFrontierSpec 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"` @@ -369,7 +412,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_791817333cf840e0, []int{4} + return fileDescriptor_processors_bulk_io_4b564fcb5cc1c063, []int{5} } func (m *BackupDataSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -408,7 +451,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_791817333cf840e0, []int{5} + return fileDescriptor_processors_bulk_io_4b564fcb5cc1c063, []int{6} } func (m *RestoreSpanEntry) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -446,7 +489,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_791817333cf840e0, []int{6} + return fileDescriptor_processors_bulk_io_4b564fcb5cc1c063, []int{7} } func (m *RestoreDataSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -480,7 +523,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_791817333cf840e0, []int{7} + return fileDescriptor_processors_bulk_io_4b564fcb5cc1c063, []int{8} } func (m *SplitAndScatterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -513,7 +556,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_791817333cf840e0, []int{7, 0} + return fileDescriptor_processors_bulk_io_4b564fcb5cc1c063, []int{8, 0} } func (m *SplitAndScatterSpec_RestoreEntryChunk) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -560,7 +603,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_791817333cf840e0, []int{8} + return fileDescriptor_processors_bulk_io_4b564fcb5cc1c063, []int{9} } func (m *CSVWriterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -595,7 +638,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_791817333cf840e0, []int{9} + return fileDescriptor_processors_bulk_io_4b564fcb5cc1c063, []int{10} } func (m *BulkRowWriterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -629,6 +672,7 @@ func init() { proto.RegisterMapType((map[int32]string)(nil), "cockroach.sql.distsqlrun.ReadImportDataSpec.UriEntry") 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((*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") @@ -942,6 +986,44 @@ func (m *StreamIngestionDataSpec) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func (m *StreamIngestionFrontierSpec) 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 *StreamIngestionFrontierSpec) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintProcessorsBulkIo(dAtA, i, uint64(m.HighWaterAtStart.Size())) + n8, err := m.HighWaterAtStart.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n8 + if len(m.TrackedSpans) > 0 { + for _, msg := range m.TrackedSpans { + dAtA[i] = 0x12 + i++ + i = encodeVarintProcessorsBulkIo(dAtA, i, uint64(msg.Size())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + func (m *BackupDataSpec) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -1014,28 +1096,28 @@ func (m *BackupDataSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x32 i++ i = encodeVarintProcessorsBulkIo(dAtA, i, uint64(m.Encryption.Size())) - n8, err := m.Encryption.MarshalTo(dAtA[i:]) + n9, err := m.Encryption.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n8 + i += n9 } dAtA[i] = 0x3a i++ i = encodeVarintProcessorsBulkIo(dAtA, i, uint64(m.BackupStartTime.Size())) - n9, err := m.BackupStartTime.MarshalTo(dAtA[i:]) + n10, err := m.BackupStartTime.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n9 + i += n10 dAtA[i] = 0x42 i++ i = encodeVarintProcessorsBulkIo(dAtA, i, uint64(m.BackupEndTime.Size())) - n10, err := m.BackupEndTime.MarshalTo(dAtA[i:]) + n11, err := m.BackupEndTime.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n10 + i += n11 if len(m.PKIDs) > 0 { keysForPKIDs := make([]uint64, 0, len(m.PKIDs)) for k := range m.PKIDs { @@ -1086,11 +1168,11 @@ func (m *RestoreSpanEntry) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintProcessorsBulkIo(dAtA, i, uint64(m.Span.Size())) - n11, err := m.Span.MarshalTo(dAtA[i:]) + n12, err := m.Span.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n11 + i += n12 if len(m.Files) > 0 { for _, msg := range m.Files { dAtA[i] = 0x12 @@ -1127,20 +1209,20 @@ func (m *RestoreDataSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintProcessorsBulkIo(dAtA, i, uint64(m.RestoreTime.Size())) - n12, err := m.RestoreTime.MarshalTo(dAtA[i:]) + n13, err := m.RestoreTime.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n12 + i += n13 if m.Encryption != nil { dAtA[i] = 0x12 i++ i = encodeVarintProcessorsBulkIo(dAtA, i, uint64(m.Encryption.Size())) - n13, err := m.Encryption.MarshalTo(dAtA[i:]) + n14, err := m.Encryption.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n13 + i += n14 } if len(m.Rekeys) > 0 { for _, msg := range m.Rekeys { @@ -1280,11 +1362,11 @@ func (m *CSVWriterSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintProcessorsBulkIo(dAtA, i, uint64(m.Options.Size())) - n14, err := m.Options.MarshalTo(dAtA[i:]) + n15, err := m.Options.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n14 + i += n15 dAtA[i] = 0x20 i++ i = encodeVarintProcessorsBulkIo(dAtA, i, uint64(m.ChunkRows)) @@ -1316,11 +1398,11 @@ func (m *BulkRowWriterSpec) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintProcessorsBulkIo(dAtA, i, uint64(m.Table.Size())) - n15, err := m.Table.MarshalTo(dAtA[i:]) + n16, err := m.Table.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n15 + i += n16 return i, nil } @@ -1453,6 +1535,23 @@ func (m *StreamIngestionDataSpec) Size() (n int) { return n } +func (m *StreamIngestionFrontierSpec) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = m.HighWaterAtStart.Size() + n += 1 + l + sovProcessorsBulkIo(uint64(l)) + if len(m.TrackedSpans) > 0 { + for _, e := range m.TrackedSpans { + l = e.Size() + n += 1 + l + sovProcessorsBulkIo(uint64(l)) + } + } + return n +} + func (m *BackupDataSpec) Size() (n int) { if m == nil { return 0 @@ -2718,6 +2817,117 @@ func (m *StreamIngestionDataSpec) Unmarshal(dAtA []byte) error { } return nil } +func (m *StreamIngestionFrontierSpec) 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: StreamIngestionFrontierSpec: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: StreamIngestionFrontierSpec: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field HighWaterAtStart", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsBulkIo + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthProcessorsBulkIo + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.HighWaterAtStart.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TrackedSpans", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsBulkIo + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthProcessorsBulkIo + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.TrackedSpans = append(m.TrackedSpans, roachpb.Span{}) + if err := m.TrackedSpans[len(m.TrackedSpans)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + 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 @@ -4172,121 +4382,125 @@ var ( ) func init() { - proto.RegisterFile("sql/execinfrapb/processors_bulk_io.proto", fileDescriptor_processors_bulk_io_791817333cf840e0) + proto.RegisterFile("sql/execinfrapb/processors_bulk_io.proto", fileDescriptor_processors_bulk_io_4b564fcb5cc1c063) } -var fileDescriptor_processors_bulk_io_791817333cf840e0 = []byte{ - // 1783 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x58, 0x4f, 0x6f, 0x1b, 0xb9, - 0x15, 0xf7, 0x48, 0x23, 0x59, 0x7a, 0x8a, 0x1d, 0x99, 0xc9, 0xee, 0x4e, 0x0d, 0xd4, 0x36, 0xb4, - 0xeb, 0x54, 0x4d, 0x11, 0x09, 0x9b, 0xb4, 0x45, 0xd0, 0x76, 0x37, 0x8d, 0xe4, 0x38, 0x2b, 0x7b, - 0x37, 0x71, 0x47, 0xb1, 0x17, 0x58, 0xb4, 0x18, 0x50, 0x33, 0xb4, 0xcc, 0x68, 0x34, 0x1c, 0x93, - 0x1c, 0x27, 0xca, 0xa5, 0x05, 0x7a, 0xea, 0xad, 0x1f, 0xa1, 0xdf, 0xa0, 0xfd, 0x0e, 0xbd, 0xe4, - 0xb8, 0xe8, 0x69, 0xd1, 0x83, 0xd1, 0x3a, 0xdf, 0xa2, 0xa7, 0x82, 0xe4, 0x8c, 0x3c, 0x76, 0x6c, - 0xc7, 0xde, 0x60, 0x2f, 0x36, 0x45, 0xbe, 0xdf, 0x8f, 0xef, 0x3d, 0xbe, 0x3f, 0xe4, 0x40, 0x53, - 0xec, 0x87, 0x6d, 0xf2, 0x92, 0xf8, 0x34, 0xda, 0xe5, 0x38, 0x1e, 0xb4, 0x63, 0xce, 0x7c, 0x22, - 0x04, 0xe3, 0xc2, 0x1b, 0x24, 0xe1, 0xc8, 0xa3, 0xac, 0x15, 0x73, 0x26, 0x19, 0x72, 0x7c, 0xe6, - 0x8f, 0x38, 0xc3, 0xfe, 0x5e, 0x4b, 0xec, 0x87, 0xad, 0x80, 0x0a, 0x29, 0xf6, 0x43, 0x9e, 0x44, - 0x8b, 0x1f, 0x3e, 0x67, 0x03, 0xd1, 0x56, 0x7f, 0xe2, 0x81, 0xfe, 0x67, 0x10, 0x8b, 0x8e, 0x96, - 0x8e, 0x07, 0x6d, 0xca, 0xee, 0xec, 0x32, 0x3e, 0xc6, 0x32, 0x5b, 0xf9, 0x58, 0xed, 0xea, 0x63, - 0x89, 0x43, 0x36, 0x6c, 0x07, 0x44, 0xf8, 0xf1, 0xa0, 0x2d, 0x24, 0x4f, 0x7c, 0x99, 0x70, 0x12, - 0xa4, 0x42, 0xab, 0x17, 0xa9, 0x86, 0x05, 0xc9, 0x76, 0x49, 0x24, 0x0d, 0xdb, 0x7b, 0xa1, 0xdf, - 0x96, 0x74, 0x4c, 0x84, 0xc4, 0xe3, 0x38, 0x5d, 0xb9, 0x39, 0x64, 0x43, 0xa6, 0x87, 0x6d, 0x35, - 0x4a, 0x67, 0x51, 0xa6, 0x55, 0x80, 0x25, 0x4e, 0xe7, 0x16, 0xb2, 0x39, 0x1c, 0x53, 0x33, 0xd5, - 0xf8, 0x7b, 0x11, 0xe6, 0x3b, 0xd8, 0x1f, 0xed, 0xd2, 0x30, 0x24, 0xbc, 0x1f, 0x13, 0x1f, 0x3d, - 0x06, 0x5b, 0x4e, 0x62, 0xe2, 0x58, 0x2b, 0x56, 0x73, 0xfe, 0xee, 0x9d, 0xd6, 0x79, 0x0e, 0x69, - 0x9d, 0xc4, 0xb5, 0x9e, 0x4d, 0x62, 0xd2, 0xb1, 0x5f, 0x1f, 0x2e, 0xcf, 0xb8, 0x9a, 0x00, 0x75, - 0xa0, 0x24, 0xf1, 0x20, 0x24, 0x4e, 0x61, 0xc5, 0x6a, 0xd6, 0xee, 0xde, 0x3a, 0xc5, 0x24, 0xf6, - 0x43, 0x6d, 0xdf, 0x33, 0x25, 0xb3, 0x46, 0x84, 0xcf, 0x69, 0x2c, 0x19, 0x4f, 0x29, 0x0c, 0x14, - 0x3d, 0x82, 0x92, 0x88, 0x71, 0x24, 0x9c, 0xe2, 0x4a, 0xb1, 0x59, 0xbb, 0xfb, 0xd3, 0xf3, 0xb5, - 0xd1, 0x34, 0x2e, 0xc1, 0x81, 0x52, 0x07, 0x47, 0x19, 0x8d, 0x46, 0xa3, 0x4f, 0xa1, 0x12, 0x24, - 0x1c, 0x4b, 0xca, 0x22, 0xc7, 0x5e, 0xb1, 0x9a, 0xc5, 0xce, 0x07, 0x6a, 0xf9, 0x7f, 0x87, 0xcb, - 0x73, 0xca, 0x9d, 0xad, 0xb5, 0x74, 0xd1, 0x9d, 0x8a, 0xa1, 0x8f, 0x01, 0xfc, 0xbd, 0x24, 0x1a, - 0x79, 0x82, 0xbe, 0x22, 0x4e, 0x49, 0x83, 0x0c, 0x67, 0x55, 0xcf, 0xf7, 0xe9, 0x2b, 0x82, 0x1e, - 0x40, 0x85, 0x13, 0x1c, 0x3c, 0x14, 0x4f, 0x77, 0x9d, 0x59, 0x6d, 0xe5, 0x8f, 0x73, 0x1a, 0xaa, - 0x23, 0x6b, 0xed, 0x85, 0x7e, 0xeb, 0x59, 0x76, 0x64, 0x29, 0xc3, 0x14, 0xd4, 0xb8, 0x0d, 0xb6, - 0xf2, 0x1b, 0xaa, 0xc1, 0x6c, 0x2f, 0x3a, 0xc0, 0x21, 0x0d, 0xea, 0x33, 0x08, 0xa0, 0xdc, 0x65, - 0x61, 0x32, 0x8e, 0xea, 0x16, 0xaa, 0x42, 0xa9, 0x17, 0x05, 0xe4, 0x65, 0xbd, 0xb0, 0x61, 0x57, - 0xca, 0xf5, 0xd9, 0xc6, 0x0b, 0xa8, 0x6d, 0xb0, 0xc1, 0x16, 0x67, 0x43, 0x4e, 0x84, 0x40, 0x9f, - 0x40, 0xf9, 0x39, 0x1b, 0x78, 0x34, 0xd0, 0xe7, 0x55, 0xec, 0xcc, 0xa9, 0x0d, 0x8e, 0x0e, 0x97, - 0x4b, 0x1b, 0x6c, 0xd0, 0x5b, 0x73, 0x4b, 0xcf, 0xd9, 0xa0, 0x17, 0xa0, 0x26, 0x5c, 0xf3, 0x59, - 0x24, 0x39, 0x1d, 0x24, 0xda, 0x07, 0xea, 0x44, 0x0a, 0xa9, 0x32, 0x27, 0x56, 0x90, 0x03, 0xb6, - 0x08, 0x99, 0x74, 0x8a, 0x2b, 0x56, 0xb3, 0x94, 0x1d, 0xa7, 0x9a, 0x69, 0xbc, 0xae, 0x00, 0x52, - 0xfe, 0xed, 0x8d, 0x63, 0xc6, 0xe5, 0x1a, 0x96, 0x58, 0x87, 0xcb, 0x2a, 0xd4, 0x04, 0x1e, 0xc7, - 0x21, 0x31, 0x8e, 0x2a, 0xe4, 0x70, 0x60, 0x16, 0xb4, 0xa7, 0x1e, 0x43, 0x25, 0x4e, 0x75, 0x76, - 0xca, 0xda, 0x53, 0xab, 0xe7, 0x9f, 0x65, 0xce, 0xc0, 0xcc, 0x63, 0x19, 0x18, 0x3d, 0x86, 0x62, - 0xc2, 0xa9, 0x33, 0xab, 0xe3, 0xe1, 0x17, 0xe7, 0x73, 0xbc, 0xad, 0x6a, 0x6b, 0x9b, 0xd3, 0x47, - 0x91, 0xe4, 0x13, 0x57, 0x31, 0xa0, 0xcf, 0xa0, 0x6c, 0xd2, 0xd5, 0xa9, 0x68, 0x7d, 0x96, 0x73, - 0x5c, 0x69, 0xa2, 0xb4, 0x7a, 0x4f, 0xd7, 0x69, 0x48, 0xd6, 0xb5, 0x58, 0xaa, 0x49, 0x0a, 0x42, - 0x3b, 0x50, 0xd6, 0x21, 0x2a, 0x9c, 0xaa, 0x56, 0xe5, 0xfe, 0x95, 0x54, 0xd1, 0xd1, 0x2a, 0xb4, - 0x36, 0x9a, 0xd7, 0x72, 0x53, 0x36, 0xf4, 0x00, 0x7e, 0x24, 0x46, 0x34, 0xf6, 0xc6, 0x54, 0x08, - 0x1a, 0x0d, 0xbd, 0x5d, 0xc6, 0x09, 0x1d, 0x46, 0xde, 0x88, 0x4c, 0x84, 0x03, 0x2b, 0x56, 0xb3, - 0x92, 0x2a, 0xf2, 0xa1, 0x12, 0xfb, 0xca, 0x48, 0xad, 0x1b, 0xa1, 0x4d, 0x32, 0x11, 0xe8, 0x36, - 0xcc, 0xbd, 0xc0, 0x61, 0xa8, 0xe2, 0xfa, 0x09, 0x8e, 0x98, 0x70, 0x6a, 0xb9, 0xd8, 0x3d, 0xb9, - 0x84, 0xee, 0xc2, 0x02, 0xd7, 0x29, 0xb3, 0x85, 0x39, 0x0e, 0x43, 0x12, 0x52, 0x31, 0x76, 0xe6, - 0x72, 0x47, 0xf8, 0xf6, 0x32, 0xfa, 0x06, 0x80, 0x13, 0x91, 0x8c, 0x89, 0x17, 0x33, 0xe1, 0xcc, - 0x6b, 0xe3, 0x7f, 0x7d, 0x25, 0xe3, 0x5d, 0x0d, 0xdf, 0x62, 0xc6, 0x7e, 0xb7, 0xca, 0xb3, 0xdf, - 0x88, 0x00, 0x24, 0x82, 0x70, 0x4f, 0x17, 0x27, 0xe7, 0xfa, 0x8a, 0xd5, 0xac, 0x76, 0xd6, 0xd3, - 0x4c, 0xfd, 0x7c, 0x48, 0xe5, 0x5e, 0x32, 0x68, 0xf9, 0x6c, 0xdc, 0x9e, 0xee, 0x16, 0x0c, 0x8e, - 0xc7, 0xed, 0x78, 0x34, 0x6c, 0x0b, 0xe2, 0x27, 0x9c, 0xca, 0x49, 0xab, 0xff, 0xbb, 0x2f, 0xb7, - 0x05, 0xe1, 0x11, 0x1e, 0x93, 0x2d, 0xc5, 0xe6, 0x56, 0x15, 0xb3, 0x1e, 0x2e, 0x26, 0x50, 0x33, - 0x2a, 0xe9, 0x63, 0x40, 0xbf, 0x05, 0x5b, 0x55, 0x67, 0x9d, 0x41, 0x57, 0xab, 0x53, 0x96, 0xab, - 0x91, 0xe8, 0x13, 0x00, 0x89, 0xf9, 0x90, 0xc8, 0x2e, 0x0b, 0x85, 0x53, 0x58, 0x29, 0x36, 0xab, - 0xe9, 0x7a, 0x6e, 0x7e, 0x51, 0x40, 0x2d, 0x77, 0xee, 0xa8, 0x0e, 0xc5, 0x11, 0x99, 0xe8, 0x5d, - 0xab, 0xae, 0x1a, 0xa2, 0x27, 0x50, 0x3a, 0xc0, 0x61, 0x92, 0x55, 0xcc, 0xab, 0x85, 0x54, 0xce, - 0x22, 0xd7, 0xd0, 0xfc, 0xaa, 0x70, 0xdf, 0x5a, 0xfc, 0x25, 0x54, 0xb2, 0xb8, 0xcf, 0xef, 0x58, - 0x32, 0x3b, 0xde, 0xcc, 0xef, 0x58, 0xcd, 0xe3, 0x7e, 0x03, 0xf3, 0x27, 0xcf, 0xe9, 0x5d, 0xe8, - 0x62, 0x0e, 0xbd, 0x61, 0x57, 0x2c, 0x5d, 0xb1, 0x8a, 0x75, 0x7b, 0xc3, 0xae, 0xd8, 0xf5, 0xd2, - 0x86, 0x5d, 0x29, 0xd5, 0xcb, 0x1b, 0x76, 0xe5, 0x5a, 0x7d, 0xae, 0xf1, 0xaf, 0x02, 0x7c, 0xd4, - 0x97, 0x9c, 0xe0, 0x71, 0x2f, 0x1a, 0x12, 0xa1, 0x0a, 0xcf, 0xb4, 0x9e, 0xfc, 0x11, 0x6e, 0xc4, - 0x98, 0x4b, 0xaa, 0x26, 0x3d, 0x1c, 0x04, 0x2a, 0xe9, 0x89, 0x70, 0x2c, 0xed, 0xd3, 0x27, 0x2a, - 0x16, 0xfe, 0x7d, 0xb8, 0xbc, 0x7e, 0xa9, 0x58, 0xf0, 0xfd, 0x50, 0xf5, 0x5b, 0x82, 0xc7, 0x34, - 0x1a, 0xfa, 0x7e, 0xd8, 0xda, 0xca, 0x88, 0x1f, 0x1a, 0x5e, 0x17, 0xc5, 0xa7, 0x66, 0x88, 0x40, - 0x1d, 0x00, 0x21, 0x31, 0x97, 0x9e, 0x4a, 0x93, 0xf4, 0x24, 0x2e, 0x55, 0xd5, 0xab, 0x1a, 0xa6, - 0x66, 0x11, 0x87, 0x79, 0xb3, 0x71, 0x66, 0x81, 0xae, 0xa7, 0xd5, 0xce, 0x66, 0xaa, 0x7f, 0xf7, - 0xfb, 0xe9, 0x6f, 0x7c, 0x96, 0x29, 0x3f, 0x27, 0xf2, 0x3f, 0x1b, 0xff, 0x9c, 0x35, 0xad, 0x3c, - 0x89, 0xa7, 0xbe, 0xbc, 0x97, 0x75, 0x4f, 0x4b, 0x67, 0xe9, 0x47, 0x67, 0x54, 0xb8, 0xb7, 0x7b, - 0xe5, 0x17, 0x50, 0xa7, 0x91, 0xe4, 0x2c, 0x48, 0x7c, 0x12, 0x78, 0x06, 0x5f, 0xb8, 0x0c, 0xfe, - 0xfa, 0x31, 0xac, 0xaf, 0x99, 0xee, 0x41, 0x2d, 0x20, 0xbb, 0x38, 0x09, 0xa5, 0xa7, 0x4a, 0xb6, - 0x71, 0x01, 0x4a, 0x1b, 0x14, 0xac, 0x99, 0xa5, 0x6d, 0xb7, 0xe7, 0x42, 0x2a, 0xb6, 0xcd, 0x29, - 0xfa, 0xb3, 0x05, 0x37, 0x12, 0x4e, 0x85, 0x37, 0x98, 0x78, 0x21, 0xf3, 0x71, 0x48, 0xe5, 0xc4, - 0x1b, 0x1d, 0x38, 0xb6, 0x56, 0xe1, 0xf3, 0x8b, 0xaf, 0x23, 0xc7, 0xb6, 0xab, 0x62, 0x2f, 0x3a, - 0x93, 0x2f, 0x53, 0x86, 0xcd, 0x03, 0x53, 0x6b, 0x6f, 0x1e, 0x1d, 0x2e, 0xd7, 0xb7, 0xdd, 0x5e, - 0x7e, 0x69, 0xc7, 0xad, 0x27, 0xa7, 0x84, 0x91, 0x0b, 0xb5, 0xf1, 0x81, 0xef, 0x7b, 0xbb, 0x34, - 0x94, 0x84, 0xeb, 0xf6, 0x3f, 0x7f, 0x22, 0x0a, 0x32, 0xfb, 0xbf, 0xda, 0xe9, 0x76, 0xd7, 0xb5, - 0xd0, 0xb1, 0x65, 0xc7, 0x73, 0x2e, 0x28, 0x16, 0x33, 0x46, 0x5f, 0x00, 0x90, 0xc8, 0xe7, 0x93, - 0x58, 0xb7, 0x60, 0xd3, 0x04, 0x9b, 0x67, 0x50, 0xaa, 0x96, 0xf3, 0x68, 0x2a, 0xf8, 0x54, 0xff, - 0x15, 0x6e, 0x0e, 0x8b, 0x9e, 0xc2, 0xc2, 0x40, 0x5b, 0xeb, 0xe5, 0x22, 0xf5, 0x0a, 0xf7, 0x8f, - 0xeb, 0x06, 0xdd, 0x9f, 0xc6, 0xeb, 0x26, 0xa4, 0x53, 0x1e, 0x89, 0x02, 0x43, 0x57, 0xb9, 0x3c, - 0xdd, 0x9c, 0xc1, 0x3e, 0x8a, 0x02, 0x4d, 0xb6, 0x0d, 0xe5, 0x78, 0xe4, 0xd1, 0x20, 0xeb, 0x8c, - 0xf7, 0x2e, 0x7d, 0x66, 0x5b, 0xa3, 0x5e, 0x90, 0x36, 0xc5, 0xaa, 0xba, 0xc3, 0x6c, 0x6d, 0xf6, - 0xd6, 0x84, 0x5b, 0x8a, 0xd5, 0xf4, 0xa9, 0xde, 0x00, 0x3f, 0x54, 0x6f, 0xe8, 0xc2, 0x07, 0x67, - 0x86, 0xce, 0x19, 0xe5, 0xfa, 0xfc, 0xe2, 0x79, 0x1f, 0xe0, 0xd8, 0x96, 0x3c, 0xd2, 0x3e, 0x03, - 0x59, 0xc9, 0x21, 0x1b, 0xff, 0xb0, 0xa0, 0xee, 0x12, 0x21, 0x19, 0x27, 0x2a, 0x89, 0x0c, 0xc1, - 0xa7, 0x60, 0xab, 0x3c, 0x4c, 0x1b, 0xd4, 0x3b, 0xd2, 0x50, 0x8b, 0xa2, 0x87, 0x50, 0xda, 0xa5, - 0xea, 0x76, 0x62, 0x52, 0x77, 0xf5, 0xac, 0xcb, 0x8d, 0x6e, 0x18, 0x2e, 0xd9, 0x4f, 0x88, 0x90, - 0x3a, 0xea, 0xb2, 0x42, 0xa0, 0x91, 0xe8, 0x16, 0xd4, 0xb2, 0x5b, 0x57, 0x2f, 0x78, 0xa9, 0xd3, - 0x37, 0xbb, 0x46, 0xe4, 0x17, 0x1a, 0x7f, 0x2a, 0xc2, 0xf5, 0x54, 0xe5, 0x69, 0xe5, 0x59, 0x87, - 0x6b, 0xdc, 0x4c, 0x99, 0x68, 0xb2, 0x2e, 0x1f, 0x4d, 0xb5, 0x14, 0xa8, 0x63, 0xe9, 0x64, 0xce, - 0x14, 0xde, 0x23, 0x67, 0x7a, 0x50, 0xe6, 0x44, 0x5f, 0xa2, 0xcc, 0x53, 0xe2, 0x67, 0xef, 0xf4, - 0x48, 0xfa, 0xa2, 0x18, 0x91, 0x49, 0x76, 0xf5, 0x33, 0x04, 0xea, 0xea, 0x97, 0x06, 0xb8, 0x29, - 0x4a, 0x3f, 0xbf, 0xa8, 0x4f, 0x9f, 0xf0, 0xcb, 0x85, 0x11, 0xfe, 0x1e, 0x51, 0xf3, 0xb7, 0x02, - 0xdc, 0xe8, 0xc7, 0x21, 0x95, 0x0f, 0xa3, 0xa0, 0xef, 0x63, 0x29, 0xd3, 0xb7, 0xdc, 0x1f, 0xa0, - 0xac, 0x1f, 0x2b, 0x59, 0x07, 0x78, 0x70, 0xbe, 0xa6, 0x67, 0xc0, 0x33, 0xed, 0xb5, 0x3e, 0x5d, - 0xc5, 0x93, 0x39, 0xc2, 0x90, 0xe6, 0x7c, 0x5a, 0x78, 0x4f, 0x9f, 0x2e, 0x7a, 0xb0, 0xf0, 0xd6, - 0x6e, 0x68, 0x03, 0x66, 0x89, 0x7a, 0x9b, 0x90, 0x4c, 0xff, 0xdb, 0xef, 0xf4, 0xf4, 0x34, 0x69, - 0x52, 0xfe, 0x8c, 0xa0, 0xf1, 0x97, 0x22, 0xcc, 0x75, 0xfb, 0x3b, 0x5f, 0x73, 0x9a, 0x39, 0xe7, - 0x96, 0x6a, 0x4f, 0x42, 0xd2, 0xc8, 0xbc, 0x0b, 0x75, 0x62, 0x67, 0x31, 0x98, 0x5b, 0x40, 0x3f, - 0x81, 0x6b, 0xaa, 0x52, 0x78, 0xb1, 0x76, 0x8c, 0x89, 0xc2, 0xa9, 0xa0, 0xae, 0x21, 0x66, 0x01, - 0x7d, 0x06, 0xb3, 0xcc, 0x44, 0x9e, 0x4e, 0x96, 0xda, 0x99, 0x0d, 0xa3, 0xdb, 0xdf, 0x49, 0xc3, - 0x33, 0xd3, 0x30, 0xc5, 0x1c, 0xbf, 0x38, 0x39, 0x7b, 0x21, 0xd2, 0x67, 0x6a, 0xfe, 0xc5, 0xe9, - 0xb2, 0x17, 0x02, 0xfd, 0x1e, 0x16, 0x7c, 0x36, 0x8e, 0x55, 0xee, 0xa9, 0x0b, 0x92, 0xcf, 0x02, - 0xe2, 0xa7, 0xed, 0xe9, 0x82, 0xc7, 0xb1, 0x4a, 0x8f, 0xee, 0x31, 0x2c, 0xa5, 0xad, 0xe7, 0x98, - 0xba, 0x8a, 0xe8, 0x54, 0x8d, 0x2d, 0xff, 0x40, 0x35, 0xb6, 0xf1, 0x35, 0x2c, 0x74, 0x92, 0x50, - 0x19, 0x94, 0x3b, 0x8e, 0xe9, 0xe7, 0x02, 0xeb, 0x7b, 0x7f, 0x2e, 0xb8, 0xbd, 0x0a, 0xd7, 0x4f, - 0x99, 0x8a, 0x2a, 0x60, 0x3f, 0x61, 0x11, 0xa9, 0xcf, 0xa8, 0xd1, 0xe3, 0x57, 0x34, 0xae, 0x5b, - 0x9d, 0x3b, 0xaf, 0xff, 0xbb, 0x34, 0xf3, 0xfa, 0x68, 0xc9, 0xfa, 0xf6, 0x68, 0xc9, 0xfa, 0xee, - 0x68, 0xc9, 0xfa, 0xcf, 0xd1, 0x92, 0xf5, 0xd7, 0x37, 0x4b, 0x33, 0xdf, 0xbe, 0x59, 0x9a, 0xf9, - 0xee, 0xcd, 0xd2, 0xcc, 0x37, 0xb5, 0xdc, 0x17, 0x99, 0xff, 0x07, 0x00, 0x00, 0xff, 0xff, 0xc0, - 0xb0, 0xb4, 0x69, 0x3e, 0x12, 0x00, 0x00, +var fileDescriptor_processors_bulk_io_4b564fcb5cc1c063 = []byte{ + // 1842 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x58, 0xcf, 0x52, 0x1b, 0xc9, + 0x19, 0x67, 0xf4, 0x0f, 0xe9, 0x93, 0x01, 0xd1, 0xf6, 0xee, 0x4e, 0x48, 0x05, 0x28, 0xed, 0xe2, + 0x28, 0x4e, 0x59, 0xaa, 0xb5, 0x93, 0x94, 0x2b, 0xc9, 0xae, 0x83, 0x84, 0xf1, 0x0a, 0x76, 0x6d, + 0x32, 0x32, 0xb8, 0x6a, 0x2b, 0xa9, 0xa9, 0xd6, 0x4c, 0x23, 0xda, 0x1a, 0x4d, 0x0f, 0xdd, 0x3d, + 0x60, 0xf9, 0x92, 0x54, 0xe5, 0x94, 0x5b, 0x1e, 0x21, 0x6f, 0x90, 0x5c, 0xf2, 0x04, 0xb9, 0xf8, + 0xb8, 0x95, 0xd3, 0x56, 0x0e, 0x54, 0x82, 0xdf, 0x22, 0xa7, 0x54, 0xf7, 0xf4, 0x88, 0x01, 0x03, + 0x86, 0x75, 0xed, 0xc5, 0x1e, 0xba, 0xfb, 0xf7, 0xeb, 0xef, 0xff, 0xf7, 0xb5, 0xa0, 0x21, 0xf6, + 0x83, 0x16, 0x79, 0x49, 0x3c, 0x1a, 0xee, 0x72, 0x1c, 0xf5, 0x5b, 0x11, 0x67, 0x1e, 0x11, 0x82, + 0x71, 0xe1, 0xf6, 0xe3, 0x60, 0xe8, 0x52, 0xd6, 0x8c, 0x38, 0x93, 0x0c, 0xd9, 0x1e, 0xf3, 0x86, + 0x9c, 0x61, 0x6f, 0xaf, 0x29, 0xf6, 0x83, 0xa6, 0x4f, 0x85, 0x14, 0xfb, 0x01, 0x8f, 0xc3, 0x85, + 0x0f, 0x5f, 0xb0, 0xbe, 0x68, 0xa9, 0x7f, 0xa2, 0xbe, 0xfe, 0x2f, 0x41, 0x2c, 0xd8, 0xfa, 0x74, + 0xd4, 0x6f, 0x51, 0x76, 0x77, 0x97, 0xf1, 0x11, 0x96, 0xe9, 0xce, 0xc7, 0xea, 0x56, 0x0f, 0x4b, + 0x1c, 0xb0, 0x41, 0xcb, 0x27, 0xc2, 0x8b, 0xfa, 0x2d, 0x21, 0x79, 0xec, 0xc9, 0x98, 0x13, 0xdf, + 0x1c, 0x5a, 0xb9, 0x4c, 0x34, 0x2c, 0x48, 0x7a, 0x4b, 0x2c, 0x69, 0xd0, 0xda, 0x0b, 0xbc, 0x96, + 0xa4, 0x23, 0x22, 0x24, 0x1e, 0x45, 0x66, 0xe7, 0xd6, 0x80, 0x0d, 0x98, 0xfe, 0x6c, 0xa9, 0x2f, + 0xb3, 0x8a, 0x52, 0xa9, 0x7c, 0x2c, 0xb1, 0x59, 0x9b, 0x4f, 0xd7, 0x70, 0x44, 0x93, 0xa5, 0xfa, + 0xdf, 0xf2, 0x30, 0xdb, 0xc6, 0xde, 0x70, 0x97, 0x06, 0x01, 0xe1, 0xbd, 0x88, 0x78, 0xe8, 0x31, + 0x14, 0xe4, 0x38, 0x22, 0xb6, 0xb5, 0x6c, 0x35, 0x66, 0xef, 0xdd, 0x6d, 0x5e, 0x64, 0x90, 0xe6, + 0x69, 0x5c, 0xf3, 0xd9, 0x38, 0x22, 0xed, 0xc2, 0xeb, 0xa3, 0xa5, 0x29, 0x47, 0x13, 0xa0, 0x36, + 0x14, 0x25, 0xee, 0x07, 0xc4, 0xce, 0x2d, 0x5b, 0x8d, 0xea, 0xbd, 0xdb, 0x67, 0x98, 0xc4, 0x7e, + 0xa0, 0xf5, 0x7b, 0xa6, 0xce, 0xac, 0x11, 0xe1, 0x71, 0x1a, 0x49, 0xc6, 0x0d, 0x45, 0x02, 0x45, + 0x8f, 0xa0, 0x28, 0x22, 0x1c, 0x0a, 0x3b, 0xbf, 0x9c, 0x6f, 0x54, 0xef, 0xfd, 0xe4, 0x62, 0x69, + 0x34, 0x8d, 0x43, 0xb0, 0xaf, 0xc4, 0xc1, 0x61, 0x4a, 0xa3, 0xd1, 0xe8, 0x53, 0x28, 0xfb, 0x31, + 0xc7, 0x92, 0xb2, 0xd0, 0x2e, 0x2c, 0x5b, 0x8d, 0x7c, 0xfb, 0x03, 0xb5, 0xfd, 0xbf, 0xa3, 0xa5, + 0x19, 0x65, 0xce, 0xe6, 0x9a, 0xd9, 0x74, 0x26, 0xc7, 0xd0, 0xc7, 0x00, 0xde, 0x5e, 0x1c, 0x0e, + 0x5d, 0x41, 0x5f, 0x11, 0xbb, 0xa8, 0x41, 0x09, 0x67, 0x45, 0xaf, 0xf7, 0xe8, 0x2b, 0x82, 0x1e, + 0x42, 0x99, 0x13, 0xec, 0xaf, 0x8a, 0xa7, 0xbb, 0xf6, 0xb4, 0xd6, 0xf2, 0x47, 0x19, 0x09, 0x95, + 0xcb, 0x9a, 0x7b, 0x81, 0xd7, 0x7c, 0x96, 0xba, 0xcc, 0x30, 0x4c, 0x40, 0xf5, 0x3b, 0x50, 0x50, + 0x76, 0x43, 0x55, 0x98, 0xee, 0x86, 0x07, 0x38, 0xa0, 0x7e, 0x6d, 0x0a, 0x01, 0x94, 0x3a, 0x2c, + 0x88, 0x47, 0x61, 0xcd, 0x42, 0x15, 0x28, 0x76, 0x43, 0x9f, 0xbc, 0xac, 0xe5, 0x36, 0x0a, 0xe5, + 0x52, 0x6d, 0xba, 0x7e, 0x08, 0xd5, 0x0d, 0xd6, 0xdf, 0xe2, 0x6c, 0xc0, 0x89, 0x10, 0xe8, 0x13, + 0x28, 0xbd, 0x60, 0x7d, 0x97, 0xfa, 0xda, 0x5f, 0xf9, 0xf6, 0x8c, 0xba, 0xe0, 0xf8, 0x68, 0xa9, + 0xb8, 0xc1, 0xfa, 0xdd, 0x35, 0xa7, 0xf8, 0x82, 0xf5, 0xbb, 0x3e, 0x6a, 0xc0, 0x0d, 0x8f, 0x85, + 0x92, 0xd3, 0x7e, 0xac, 0x6d, 0xa0, 0x3c, 0x92, 0x33, 0xc2, 0x9c, 0xda, 0x41, 0x36, 0x14, 0x44, + 0xc0, 0xa4, 0x9d, 0x5f, 0xb6, 0x1a, 0xc5, 0xd4, 0x9d, 0x6a, 0xa5, 0xfe, 0xba, 0x0c, 0x48, 0xd9, + 0xb7, 0x3b, 0x8a, 0x18, 0x97, 0x6b, 0x58, 0x62, 0x1d, 0x2e, 0x2b, 0x50, 0x15, 0x78, 0x14, 0x05, + 0x24, 0x31, 0x54, 0x2e, 0x83, 0x83, 0x64, 0x43, 0x5b, 0xea, 0x31, 0x94, 0x23, 0x23, 0xb3, 0x5d, + 0xd2, 0x96, 0x5a, 0xb9, 0xd8, 0x97, 0x19, 0x05, 0x53, 0x8b, 0xa5, 0x60, 0xf4, 0x18, 0xf2, 0x31, + 0xa7, 0xf6, 0xb4, 0x8e, 0x87, 0x9f, 0x5f, 0xcc, 0xf1, 0xb6, 0xa8, 0xcd, 0x6d, 0x4e, 0x1f, 0x85, + 0x92, 0x8f, 0x1d, 0xc5, 0x80, 0x3e, 0x83, 0x52, 0x92, 0xae, 0x76, 0x59, 0xcb, 0xb3, 0x94, 0xe1, + 0x32, 0x89, 0xd2, 0xec, 0x3e, 0x5d, 0xa7, 0x01, 0x59, 0xd7, 0xc7, 0x8c, 0x24, 0x06, 0x84, 0x76, + 0xa0, 0xa4, 0x43, 0x54, 0xd8, 0x15, 0x2d, 0xca, 0x83, 0x6b, 0x89, 0xa2, 0xa3, 0x55, 0x68, 0x69, + 0x34, 0xaf, 0xe5, 0x18, 0x36, 0xf4, 0x10, 0x7e, 0x20, 0x86, 0x34, 0x72, 0x47, 0x54, 0x08, 0x1a, + 0x0e, 0xdc, 0x5d, 0xc6, 0x09, 0x1d, 0x84, 0xee, 0x90, 0x8c, 0x85, 0x0d, 0xcb, 0x56, 0xa3, 0x6c, + 0x04, 0xf9, 0x50, 0x1d, 0xfb, 0x2a, 0x39, 0xb5, 0x9e, 0x1c, 0xda, 0x24, 0x63, 0x81, 0xee, 0xc0, + 0xcc, 0x21, 0x0e, 0x02, 0x15, 0xd7, 0x4f, 0x70, 0xc8, 0x84, 0x5d, 0xcd, 0xc4, 0xee, 0xe9, 0x2d, + 0x74, 0x0f, 0xe6, 0xb9, 0x4e, 0x99, 0x2d, 0xcc, 0x71, 0x10, 0x90, 0x80, 0x8a, 0x91, 0x3d, 0x93, + 0x71, 0xe1, 0xdb, 0xdb, 0xe8, 0x6b, 0x00, 0x4e, 0x44, 0x3c, 0x22, 0x6e, 0xc4, 0x84, 0x3d, 0xab, + 0x95, 0xff, 0xd5, 0xb5, 0x94, 0x77, 0x34, 0x7c, 0x8b, 0x25, 0xfa, 0x3b, 0x15, 0x9e, 0xfe, 0x8d, + 0x08, 0x40, 0x2c, 0x08, 0x77, 0x75, 0x71, 0xb2, 0xe7, 0x96, 0xad, 0x46, 0xa5, 0xbd, 0x6e, 0x32, + 0xf5, 0xf3, 0x01, 0x95, 0x7b, 0x71, 0xbf, 0xe9, 0xb1, 0x51, 0x6b, 0x72, 0x9b, 0xdf, 0x3f, 0xf9, + 0x6e, 0x45, 0xc3, 0x41, 0x4b, 0x10, 0x2f, 0xe6, 0x54, 0x8e, 0x9b, 0xbd, 0xdf, 0x7e, 0xb9, 0x2d, + 0x08, 0x0f, 0xf1, 0x88, 0x6c, 0x29, 0x36, 0xa7, 0xa2, 0x98, 0xf5, 0xe7, 0x42, 0x0c, 0xd5, 0x44, + 0x24, 0xed, 0x06, 0xf4, 0x1b, 0x28, 0xa8, 0xea, 0xac, 0x33, 0xe8, 0x7a, 0x75, 0xca, 0x72, 0x34, + 0x12, 0x7d, 0x02, 0x20, 0x31, 0x1f, 0x10, 0xd9, 0x61, 0x81, 0xb0, 0x73, 0xcb, 0xf9, 0x46, 0xc5, + 0xec, 0x67, 0xd6, 0x17, 0x04, 0x54, 0x33, 0x7e, 0x47, 0x35, 0xc8, 0x0f, 0xc9, 0x58, 0xdf, 0x5a, + 0x71, 0xd4, 0x27, 0x7a, 0x02, 0xc5, 0x03, 0x1c, 0xc4, 0x69, 0xc5, 0xbc, 0x5e, 0x48, 0x65, 0x34, + 0x72, 0x12, 0x9a, 0x5f, 0xe6, 0x1e, 0x58, 0x0b, 0xbf, 0x80, 0x72, 0x1a, 0xf7, 0xd9, 0x1b, 0x8b, + 0xc9, 0x8d, 0xb7, 0xb2, 0x37, 0x56, 0xb2, 0xb8, 0x5f, 0xc3, 0xec, 0x69, 0x3f, 0xbd, 0x0b, 0x9d, + 0xcf, 0xa0, 0x37, 0x0a, 0x65, 0x4b, 0x57, 0xac, 0x7c, 0xad, 0xb0, 0x51, 0x28, 0x17, 0x6a, 0xc5, + 0x8d, 0x42, 0xb9, 0x58, 0x2b, 0x6d, 0x14, 0xca, 0x37, 0x6a, 0x33, 0xf5, 0x7f, 0xe5, 0xe0, 0xa3, + 0x9e, 0xe4, 0x04, 0x8f, 0xba, 0xe1, 0x80, 0x08, 0x55, 0x78, 0x26, 0xf5, 0xe4, 0x0f, 0x70, 0x33, + 0xc2, 0x5c, 0x52, 0xb5, 0xe8, 0x62, 0xdf, 0x57, 0x49, 0x4f, 0x84, 0x6d, 0x69, 0x9b, 0x3e, 0x51, + 0xb1, 0xf0, 0xef, 0xa3, 0xa5, 0xf5, 0x2b, 0xc5, 0x82, 0xe7, 0x05, 0xaa, 0xdf, 0x12, 0x3c, 0xa2, + 0xe1, 0xc0, 0xf3, 0x82, 0xe6, 0x56, 0x4a, 0xbc, 0x9a, 0xf0, 0x3a, 0x28, 0x3a, 0xb3, 0x42, 0x04, + 0x6a, 0x03, 0x08, 0x89, 0xb9, 0x74, 0x55, 0x9a, 0x18, 0x4f, 0x5c, 0xa9, 0xaa, 0x57, 0x34, 0x4c, + 0xad, 0x22, 0x0e, 0xb3, 0xc9, 0xc5, 0xa9, 0x06, 0xba, 0x9e, 0x56, 0xda, 0x9b, 0x46, 0xfe, 0xce, + 0x77, 0x93, 0x3f, 0xb1, 0x59, 0x2a, 0xfc, 0x8c, 0xc8, 0xfe, 0x59, 0xff, 0x87, 0x05, 0x3f, 0x3c, + 0x63, 0xd4, 0x75, 0xce, 0x42, 0x49, 0x4d, 0x5f, 0x77, 0xe0, 0xe6, 0x1e, 0x1d, 0xec, 0xb9, 0x87, + 0x58, 0x12, 0xee, 0x62, 0xe9, 0x6a, 0x71, 0x4d, 0xd0, 0x5f, 0x49, 0xc1, 0x9a, 0xc2, 0x3f, 0x57, + 0xf0, 0x55, 0xd9, 0x53, 0x60, 0xd4, 0x86, 0x19, 0xc9, 0xb1, 0x37, 0x24, 0xbe, 0x9b, 0xb4, 0xe9, + 0x9c, 0x2e, 0x07, 0x1f, 0x9d, 0x53, 0x4a, 0x33, 0x4d, 0xf9, 0x86, 0xc1, 0xa8, 0x25, 0x51, 0xff, + 0xe7, 0x74, 0x32, 0x82, 0xc4, 0xd1, 0x24, 0x06, 0xee, 0xa7, 0x5d, 0xdf, 0xba, 0x0a, 0x9d, 0xe9, + 0xf1, 0x5f, 0x40, 0x8d, 0x86, 0x92, 0x33, 0x3f, 0xf6, 0xae, 0x27, 0xce, 0xdc, 0x09, 0x4c, 0x4b, + 0x84, 0xee, 0x43, 0xd5, 0x27, 0xbb, 0x38, 0x0e, 0xa4, 0xab, 0x5a, 0x4d, 0xe2, 0x3a, 0x64, 0x1a, + 0x2b, 0xac, 0x25, 0x5b, 0xdb, 0x4e, 0xd7, 0x01, 0x73, 0x6c, 0x9b, 0x53, 0xf4, 0x27, 0x0b, 0x6e, + 0xc6, 0x9c, 0x0a, 0xb7, 0x3f, 0x76, 0x03, 0xe6, 0xe1, 0x80, 0xca, 0xb1, 0x3b, 0x3c, 0xb0, 0x0b, + 0x5a, 0x84, 0xcf, 0x2f, 0x1f, 0xa3, 0x4e, 0x74, 0x57, 0x4d, 0x4a, 0xb4, 0xc7, 0x5f, 0x1a, 0x86, + 0xcd, 0x83, 0xa4, 0x47, 0xdc, 0x3a, 0x3e, 0x5a, 0xaa, 0x6d, 0x3b, 0xdd, 0xec, 0xd6, 0x8e, 0x53, + 0x8b, 0xcf, 0x1c, 0x46, 0x0e, 0x54, 0x47, 0x07, 0x9e, 0xe7, 0xee, 0xd2, 0x40, 0x12, 0xae, 0xc7, + 0x96, 0xd9, 0x53, 0xce, 0x4d, 0xf5, 0xff, 0x6a, 0xa7, 0xd3, 0x59, 0xd7, 0x87, 0x4e, 0x34, 0x3b, + 0x59, 0x73, 0x40, 0xb1, 0x24, 0xdf, 0xe8, 0x0b, 0x00, 0x12, 0x7a, 0x7c, 0x1c, 0xe9, 0xd1, 0x21, + 0x69, 0xde, 0x8d, 0x73, 0x28, 0x55, 0xab, 0x7c, 0x34, 0x39, 0xf8, 0x54, 0xff, 0x2b, 0x9c, 0x0c, + 0x16, 0x3d, 0x85, 0xf9, 0xbe, 0xd6, 0xd6, 0xcd, 0x64, 0xd8, 0x35, 0xe6, 0xa6, 0xb9, 0x04, 0xdd, + 0x9b, 0xe4, 0xd9, 0x26, 0x98, 0x25, 0x97, 0x84, 0x7e, 0x42, 0x57, 0xbe, 0x3a, 0xdd, 0x4c, 0x82, + 0x7d, 0x14, 0xfa, 0x9a, 0x6c, 0x1b, 0x4a, 0xd1, 0xd0, 0xa5, 0x7e, 0xda, 0xd1, 0xef, 0x5f, 0xd9, + 0x67, 0x5b, 0xc3, 0xae, 0x6f, 0x9a, 0x79, 0x45, 0xcd, 0x5e, 0x5b, 0x9b, 0xdd, 0x35, 0xe1, 0x14, + 0x23, 0xb5, 0x7c, 0xa6, 0xa7, 0xc1, 0xf7, 0xd5, 0xd3, 0x3a, 0xf0, 0xc1, 0xb9, 0xa1, 0x73, 0x4e, + 0x9b, 0xb9, 0xb8, 0xe8, 0x3f, 0x00, 0x38, 0xd1, 0x25, 0x8b, 0x2c, 0x9c, 0x83, 0x2c, 0x67, 0x90, + 0xf5, 0xbf, 0x5b, 0x50, 0x73, 0x88, 0x90, 0x8c, 0x13, 0x95, 0x44, 0x09, 0xc1, 0xa7, 0x50, 0x50, + 0x79, 0x68, 0x6a, 0xcc, 0x3b, 0xd2, 0x50, 0x1f, 0x45, 0xab, 0x50, 0xdc, 0xa5, 0x6a, 0xaa, 0x4a, + 0x52, 0x77, 0xe5, 0xbc, 0xa1, 0x4c, 0x37, 0x3a, 0x87, 0xec, 0xc7, 0x44, 0x48, 0x1d, 0x75, 0x69, + 0x21, 0xd0, 0x48, 0x74, 0x1b, 0xaa, 0xe9, 0xb4, 0xd8, 0xf5, 0x5f, 0xea, 0xf4, 0x4d, 0xc7, 0x9f, + 0xec, 0x46, 0xfd, 0x8f, 0x79, 0x98, 0x33, 0x22, 0x4f, 0x2a, 0xcf, 0x3a, 0xdc, 0xe0, 0xc9, 0x52, + 0x12, 0x4d, 0xd7, 0xa8, 0x8e, 0x55, 0x03, 0xd4, 0xb1, 0x74, 0x3a, 0x67, 0x72, 0xef, 0x91, 0x33, + 0x5d, 0x28, 0x71, 0xa2, 0x87, 0xbf, 0xe4, 0x09, 0xf4, 0xd3, 0x77, 0x5a, 0xc4, 0xbc, 0x84, 0x86, + 0x64, 0x9c, 0x8e, 0xac, 0x09, 0x81, 0x1a, 0x59, 0x4d, 0x80, 0x27, 0x45, 0xe9, 0x67, 0x97, 0xcd, + 0x17, 0xa7, 0xec, 0x72, 0x69, 0x84, 0xbf, 0x47, 0xd4, 0xfc, 0x35, 0x07, 0x37, 0x7b, 0x51, 0x40, + 0xe5, 0x6a, 0xe8, 0xf7, 0x3c, 0x2c, 0xa5, 0xe9, 0x55, 0xbf, 0x87, 0x92, 0x7e, 0x64, 0xa5, 0x1d, + 0xe0, 0xe1, 0xc5, 0x92, 0x9e, 0x03, 0x4f, 0xa5, 0xd7, 0xf2, 0x74, 0x14, 0x4f, 0x6a, 0x88, 0x84, + 0x34, 0x63, 0xd3, 0xdc, 0x7b, 0xda, 0x74, 0xc1, 0x85, 0xf9, 0xb7, 0x6e, 0x43, 0x1b, 0x30, 0x4d, + 0xd4, 0x9b, 0x8a, 0xa4, 0xf2, 0xdf, 0x79, 0xa7, 0xa5, 0x27, 0x49, 0x63, 0xf8, 0x53, 0x82, 0xfa, + 0x9f, 0xf3, 0x30, 0xd3, 0xe9, 0xed, 0x3c, 0xe7, 0x34, 0x35, 0xce, 0x6d, 0xd5, 0x9e, 0x84, 0xa4, + 0x61, 0xf2, 0x9e, 0xd5, 0x89, 0x9d, 0xc6, 0x60, 0x66, 0x03, 0xfd, 0x18, 0x6e, 0xa8, 0x4a, 0xe1, + 0x46, 0xda, 0x30, 0x49, 0x14, 0x4e, 0x0e, 0xea, 0x1a, 0x92, 0x6c, 0xa0, 0xcf, 0x60, 0x9a, 0x25, + 0x91, 0xa7, 0x93, 0xa5, 0x7a, 0x6e, 0xc3, 0xe8, 0xf4, 0x76, 0x4c, 0x78, 0xa6, 0x12, 0x1a, 0xcc, + 0xc9, 0x4b, 0x99, 0xb3, 0x43, 0x61, 0x9e, 0xd7, 0xd9, 0x97, 0xb2, 0xc3, 0x0e, 0x05, 0xfa, 0x1d, + 0xcc, 0x7b, 0x6c, 0x14, 0xa9, 0xdc, 0x53, 0x83, 0x9d, 0xc7, 0x7c, 0xe2, 0x99, 0xf6, 0x74, 0xc9, + 0xa3, 0x5e, 0xa5, 0x47, 0xe7, 0x04, 0x96, 0xce, 0x21, 0x19, 0xa6, 0x8e, 0x22, 0x3a, 0x53, 0x63, + 0x4b, 0xdf, 0x53, 0x8d, 0xad, 0x3f, 0x87, 0xf9, 0x76, 0x1c, 0x28, 0x85, 0x32, 0xee, 0x98, 0xfc, + 0xcc, 0x61, 0x7d, 0xe7, 0x9f, 0x39, 0xee, 0xac, 0xc0, 0xdc, 0x19, 0x55, 0x51, 0x19, 0x0a, 0x4f, + 0x58, 0x48, 0x6a, 0x53, 0xea, 0xeb, 0xf1, 0x2b, 0x1a, 0xd5, 0xac, 0xf6, 0xdd, 0xd7, 0xff, 0x5d, + 0x9c, 0x7a, 0x7d, 0xbc, 0x68, 0x7d, 0x73, 0xbc, 0x68, 0x7d, 0x7b, 0xbc, 0x68, 0xfd, 0xe7, 0x78, + 0xd1, 0xfa, 0xcb, 0x9b, 0xc5, 0xa9, 0x6f, 0xde, 0x2c, 0x4e, 0x7d, 0xfb, 0x66, 0x71, 0xea, 0xeb, + 0x6a, 0xe6, 0x97, 0xa4, 0xff, 0x07, 0x00, 0x00, 0xff, 0xff, 0x19, 0x65, 0xa0, 0x8f, 0xf6, 0x12, + 0x00, 0x00, } diff --git a/pkg/sql/execinfrapb/processors_bulk_io.proto b/pkg/sql/execinfrapb/processors_bulk_io.proto index 6027e9a4472d..365153f4213e 100644 --- a/pkg/sql/execinfrapb/processors_bulk_io.proto +++ b/pkg/sql/execinfrapb/processors_bulk_io.proto @@ -144,6 +144,20 @@ message StreamIngestionDataSpec { optional string stream_address = 3 [(gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl.StreamAddress",(gogoproto.nullable) = false]; } +message StreamIngestionFrontierSpec { + // HighWaterAtStart is set by the ingestion job when initializing the frontier + // processor. It is used as sanity check by the frontier processor to ensure + // that it does not receive updates at a timestamp lower than this field. This + // consequently prevents the job progress from regressing during ingestion. + optional util.hlc.Timestamp high_water_at_start = 1 [(gogoproto.nullable) = false]; + // TrackedSpans is the entire span set being watched. The spans do not really + // represent KV spans but uniquely identify the partitions in the ingestion + // stream. Once all the partitions in the ingestion stream have been resolved + // at a certain timestamp, then it's safe to resolve the ingestion at that + // timestamp. + repeated roachpb.Span tracked_spans = 2 [(gogoproto.nullable) = false]; +} + 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 703071e64f44..27f8ee87d9ec 100644 --- a/pkg/sql/rowexec/processors.go +++ b/pkg/sql/rowexec/processors.go @@ -359,6 +359,12 @@ func NewProcessor( } return newInvertedFilterer(flowCtx, processorID, core.InvertedFilterer, inputs[0], post, outputs[0]) } + if core.StreamIngestionFrontier != nil { + if err := checkNumInOut(inputs, outputs, 1, 1); err != nil { + return nil, err + } + return NewStreamIngestionFrontierProcessor(flowCtx, processorID, *core.StreamIngestionFrontier, inputs[0], post, outputs[0]) + } return nil, errors.Errorf("unsupported processor core %q", core) } @@ -385,3 +391,6 @@ var NewChangeAggregatorProcessor func(*execinfra.FlowCtx, int32, execinfrapb.Cha // NewChangeFrontierProcessor is implemented in the non-free (CCL) codebase and then injected here via runtime initialization. var NewChangeFrontierProcessor func(*execinfra.FlowCtx, int32, execinfrapb.ChangeFrontierSpec, execinfra.RowSource, *execinfrapb.PostProcessSpec, execinfra.RowReceiver) (execinfra.Processor, error) + +// 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)