From 5797332b012da21d287efa143b77d22f76d8fb1b Mon Sep 17 00:00:00 2001 From: Aditya Maru Date: Wed, 30 Dec 2020 10:26:19 -0500 Subject: [PATCH] streamingccl: add ingestion job framework This change introduces a new StreamIngestionJob. It does not do much more than laying out the general outline of the job, which is very similar to other bulk jobs such as changefeed, backup etc. More precisely: - Introduces StreamIngestionDetails job details proto - Hooks up the dependancy to a mock stream client - Introduces a StreamIngestionProcessorSpec - Sets up a simple DistSQL flow which round robin assigns the partitions to the processors. Most notable TODOs in job land which will be addressed in follow up PRs: - StreamIngestionPlanHook to create this job. Will involve figuring out SQL syntax. - Introducing a ts watermark in both the job and processors. This watermark will represent the lowest resolved ts which all processors have ingested till. Iron out semantics on job start and resumption. - Introducing a StreamIngestionFrontier processor which will slurp the results from the StreamIngestionProcessors, and use them to keep track of the minimum resolved ts across all processors. Release note: None --- pkg/ccl/streamingccl/BUILD.bazel | 19 +- pkg/ccl/streamingccl/stream_ingestion_job.go | 85 +- .../stream_ingestion_processor_planning.go | 109 +++ pkg/ccl/streamingccl/streamclient/BUILD.bazel | 1 + .../streamingccl/streamclient/client_test.go | 10 +- .../streamclient/stream_client.go | 33 + pkg/jobs/jobspb/BUILD.bazel | 1 + pkg/jobs/jobspb/jobs.pb.go | 823 +++++++++++------- pkg/jobs/jobspb/jobs.proto | 6 + pkg/jobs/jobspb/wrap.go | 2 +- pkg/sql/execinfrapb/BUILD.bazel | 1 + pkg/sql/execinfrapb/processors.pb.go | 227 +++-- pkg/sql/execinfrapb/processors.proto | 1 + pkg/sql/execinfrapb/processors_bulk_io.pb.go | 413 ++++++--- pkg/sql/execinfrapb/processors_bulk_io.proto | 4 + pkg/ts/catalog/chart_catalog.go | 12 + 16 files changed, 1200 insertions(+), 547 deletions(-) create mode 100644 pkg/ccl/streamingccl/stream_ingestion_processor_planning.go create mode 100644 pkg/ccl/streamingccl/streamclient/stream_client.go diff --git a/pkg/ccl/streamingccl/BUILD.bazel b/pkg/ccl/streamingccl/BUILD.bazel index 655fb40643ab..a47c426b7088 100644 --- a/pkg/ccl/streamingccl/BUILD.bazel +++ b/pkg/ccl/streamingccl/BUILD.bazel @@ -2,7 +2,24 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "streamingccl", - srcs = ["stream_ingestion_job.go"], + srcs = [ + "stream_ingestion_job.go", + "stream_ingestion_processor_planning.go", + ], importpath = "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl", visibility = ["//visibility:public"], + deps = [ + "//pkg/ccl/streamingccl/streamclient", + "//pkg/jobs", + "//pkg/jobs/jobspb", + "//pkg/kv", + "//pkg/roachpb", + "//pkg/settings/cluster", + "//pkg/sql", + "//pkg/sql/execinfrapb", + "//pkg/sql/physicalplan", + "//pkg/sql/sem/tree", + "//pkg/sql/types", + "@com_github_cockroachdb_logtags//:logtags", + ], ) diff --git a/pkg/ccl/streamingccl/stream_ingestion_job.go b/pkg/ccl/streamingccl/stream_ingestion_job.go index c1ee4d2eb72e..e930aad3d806 100644 --- a/pkg/ccl/streamingccl/stream_ingestion_job.go +++ b/pkg/ccl/streamingccl/stream_ingestion_job.go @@ -8,6 +8,89 @@ package streamingccl +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/streamclient" + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" +) + +type streamIngestionResumer struct { + job *jobs.Job +} + +func ingest( + ctx context.Context, + execCtx sql.JobExecContext, + streamAddress streamclient.PartitionAddress, + job *jobs.Job, +) error { + // Initialize a stream client and resolve topology. + client := streamclient.NewStreamClient() + sa := streamclient.StreamAddress(streamAddress) + topology, err := client.GetTopology(sa) + if err != nil { + return err + } + + evalCtx := execCtx.ExtendedEvalContext() + dsp := execCtx.DistSQLPlanner() + + planCtx, nodes, err := dsp.SetupAllNodesPlanning(ctx, evalCtx, execCtx.ExecCfg()) + if err != nil { + return err + } + + // Construct stream ingestion processor specs. + streamIngestionSpecs, err := distStreamIngestionPlanSpecs(topology, nodes) + if err != nil { + return err + } + + // Plan and run the DistSQL flow. + err = distStreamIngest(ctx, execCtx, nodes, planCtx, dsp, streamIngestionSpecs) + if err != nil { + return err + } + + return nil +} + +// Resume is part of the jobs.Resumer interface. +func (s *streamIngestionResumer) Resume( + ctx context.Context, execCtx interface{}, resultsCh chan<- tree.Datums, +) error { + details := s.job.Details().(jobspb.StreamIngestionDetails) + p := execCtx.(sql.JobExecContext) + + err := ingest(ctx, p, streamclient.PartitionAddress(details.StreamAddress), s.job) + if err != nil { + return err + } + + // TODO(adityamaru): We probably want to use the resultsCh to indicate that + // the processors have completed setup. We can then return the job ID in the + // plan hook similar to how changefeeds do it. + + return nil +} + +// OnFailOrCancel is part of the jobs.Resumer interface. +func (s *streamIngestionResumer) OnFailOrCancel(ctx context.Context, execCtx interface{}) error { + return nil +} + +var _ jobs.Resumer = &streamIngestionResumer{} + func init() { - // TODO: Implement me. + jobs.RegisterConstructor( + jobspb.TypeStreamIngestion, + func(job *jobs.Job, + settings *cluster.Settings) jobs.Resumer { + return &streamIngestionResumer{job: job} + }) } diff --git a/pkg/ccl/streamingccl/stream_ingestion_processor_planning.go b/pkg/ccl/streamingccl/stream_ingestion_processor_planning.go new file mode 100644 index 000000000000..7d4c138a91e7 --- /dev/null +++ b/pkg/ccl/streamingccl/stream_ingestion_processor_planning.go @@ -0,0 +1,109 @@ +// 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 streamingccl + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/streamclient" + "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/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/logtags" +) + +// TODO(adityamaru): Figure out what the processors will return. +var streamIngestionResultTypes = []*types.T{} + +func distStreamIngestionPlanSpecs( + topology streamclient.Topology, nodes []roachpb.NodeID, +) ([]*execinfrapb.StreamIngestionDataSpec, error) { + + // For each stream partition in the topology, assign it to a node. + streamIngestionSpecs := make([]*execinfrapb.StreamIngestionDataSpec, 0, len(nodes)) + + 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 + // the partition addresses. + if i < len(nodes) { + spec := &execinfrapb.StreamIngestionDataSpec{ + PartitionAddress: make([]streamclient.PartitionAddress, 0), + } + streamIngestionSpecs = append(streamIngestionSpecs, spec) + } + n := i % len(nodes) + streamIngestionSpecs[n].PartitionAddress = append(streamIngestionSpecs[n].PartitionAddress, partition) + } + + return streamIngestionSpecs, nil +} + +func distStreamIngest( + ctx context.Context, + execCtx sql.JobExecContext, + nodes []roachpb.NodeID, + planCtx *sql.PlanningCtx, + dsp *sql.DistSQLPlanner, + streamIngestionSpecs []*execinfrapb.StreamIngestionDataSpec, +) error { + ctx = logtags.AddTag(ctx, "stream-ingest-distsql", nil) + evalCtx := execCtx.ExtendedEvalContext() + var noTxn *kv.Txn + + if len(streamIngestionSpecs) == 0 { + return nil + } + + // Setup a one-stage plan with one proc per input spec. + corePlacement := make([]physicalplan.ProcessorCorePlacement, len(streamIngestionSpecs)) + for i := range streamIngestionSpecs { + corePlacement[i].NodeID = nodes[i] + corePlacement[i].Core.StreamIngestionData = streamIngestionSpecs[i] + } + + p := planCtx.NewPhysicalPlan() + p.AddNoInputStage( + corePlacement, + execinfrapb.PostProcessSpec{}, + streamIngestionResultTypes, + execinfrapb.Ordering{}, + ) + + // 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. + + // TODO(adityamaru): Once result types are updated, add PlanToStreamColMap. + dsp.FinalizePlan(planCtx, p) + + recv := sql.MakeDistSQLReceiver( + ctx, + // TODO(adityamaru): Are there any results we want to surface to the user? + nil, /* resultWriter */ + tree.Rows, + nil, /* rangeCache */ + noTxn, + nil, /* clockUpdater */ + evalCtx.Tracing, + ) + defer recv.Release() + + // Copy the evalCtx, as dsp.Run() might change it. + evalCtxCopy := *evalCtx + dsp.Run(planCtx, noTxn, p, recv, &evalCtxCopy, nil /* finishedSetupFn */) + return nil +} diff --git a/pkg/ccl/streamingccl/streamclient/BUILD.bazel b/pkg/ccl/streamingccl/streamclient/BUILD.bazel index d1b6e4da4732..0ca9ddd0576c 100644 --- a/pkg/ccl/streamingccl/streamclient/BUILD.bazel +++ b/pkg/ccl/streamingccl/streamclient/BUILD.bazel @@ -5,6 +5,7 @@ go_library( srcs = [ "client.go", "event.go", + "stream_client.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/streamclient", visibility = ["//visibility:public"], diff --git a/pkg/ccl/streamingccl/streamclient/client_test.go b/pkg/ccl/streamingccl/streamclient/client_test.go index c5ebbe59684b..01d055d34d9c 100644 --- a/pkg/ccl/streamingccl/streamclient/client_test.go +++ b/pkg/ccl/streamingccl/streamclient/client_test.go @@ -18,12 +18,12 @@ import ( "github.com/stretchr/testify/require" ) -type mockStreamClient struct{} +type testStreamClient struct{} -var _ Client = mockStreamClient{} +var _ Client = testStreamClient{} // GetTopology implements the Client interface. -func (sc mockStreamClient) GetTopology(_ StreamAddress) (Topology, error) { +func (sc testStreamClient) GetTopology(_ StreamAddress) (Topology, error) { return Topology{Partitions: []PartitionAddress{ "s3://my_bucket/my_stream/partition_1", "s3://my_bucket/my_stream/partition_2", @@ -31,7 +31,7 @@ func (sc mockStreamClient) GetTopology(_ StreamAddress) (Topology, error) { } // ConsumePartition implements the Client interface. -func (sc mockStreamClient) ConsumePartition(_ PartitionAddress, _ time.Time) (chan Event, error) { +func (sc testStreamClient) ConsumePartition(_ PartitionAddress, _ time.Time) (chan Event, error) { sampleKV := roachpb.KeyValue{ Key: []byte("key_1"), Value: roachpb.Value{ @@ -51,7 +51,7 @@ func (sc mockStreamClient) ConsumePartition(_ PartitionAddress, _ time.Time) (ch // TestExampleClientUsage serves as documentation to indicate how a stream // client could be used. func TestExampleClientUsage(t *testing.T) { - client := mockStreamClient{} + client := testStreamClient{} sa := StreamAddress("s3://my_bucket/my_stream") topology, err := client.GetTopology(sa) require.NoError(t, err) diff --git a/pkg/ccl/streamingccl/streamclient/stream_client.go b/pkg/ccl/streamingccl/streamclient/stream_client.go new file mode 100644 index 000000000000..72a971953091 --- /dev/null +++ b/pkg/ccl/streamingccl/streamclient/stream_client.go @@ -0,0 +1,33 @@ +// 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 streamclient + +import "time" + +// client is a mock stream client. +type client struct{} + +var _ Client = &client{} + +// NewStreamClient returns a new mock stream client. +func NewStreamClient() Client { + return &client{} +} + +// GetTopology implements the Client interface. +func (m *client) GetTopology(address StreamAddress) (Topology, error) { + panic("unimplemented mock method") +} + +// ConsumePartition implements the Client interface. +func (m *client) ConsumePartition( + address PartitionAddress, startTime time.Time, +) (chan Event, error) { + panic("unimplemented mock method") +} diff --git a/pkg/jobs/jobspb/BUILD.bazel b/pkg/jobs/jobspb/BUILD.bazel index 87b1fd67bffc..879a13e4ff57 100644 --- a/pkg/jobs/jobspb/BUILD.bazel +++ b/pkg/jobs/jobspb/BUILD.bazel @@ -10,6 +10,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/jobs/jobspb", visibility = ["//visibility:public"], deps = [ + "//pkg/ccl/streamingccl/streamclient", "//pkg/roachpb", "//pkg/security", "//pkg/sql/catalog/descpb", diff --git a/pkg/jobs/jobspb/jobs.pb.go b/pkg/jobs/jobspb/jobs.pb.go index 22096fa0178b..6eefa434c85f 100644 --- a/pkg/jobs/jobspb/jobs.pb.go +++ b/pkg/jobs/jobspb/jobs.pb.go @@ -12,6 +12,7 @@ import descpb "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import github_com_cockroachdb_cockroach_pkg_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" +import github_com_cockroachdb_cockroach_pkg_ccl_streamingccl_streamclient "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/streamclient" import github_com_cockroachdb_cockroach_pkg_util_uuid "github.com/cockroachdb/cockroach/pkg/util/uuid" import github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" import github_com_cockroachdb_cockroach_pkg_sql_sem_tree "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -55,7 +56,7 @@ func (x EncryptionMode) String() string { return proto.EnumName(EncryptionMode_name, int32(x)) } func (EncryptionMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{0} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{0} } type Status int32 @@ -84,7 +85,7 @@ func (x Status) String() string { return proto.EnumName(Status_name, int32(x)) } func (Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{1} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{1} } type Type int32 @@ -102,19 +103,21 @@ const ( // We can't name this TYPE_SCHEMA_CHANGE due to how proto generates actual // names for this enum, which cause a conflict with the SCHEMA_CHANGE entry. TypeTypeSchemaChange Type = 9 + TypeStreamIngestion Type = 10 ) var Type_name = map[int32]string{ - 0: "UNSPECIFIED", - 1: "BACKUP", - 2: "RESTORE", - 3: "SCHEMA_CHANGE", - 4: "IMPORT", - 5: "CHANGEFEED", - 6: "CREATE_STATS", - 7: "AUTO_CREATE_STATS", - 8: "SCHEMA_CHANGE_GC", - 9: "TYPEDESC_SCHEMA_CHANGE", + 0: "UNSPECIFIED", + 1: "BACKUP", + 2: "RESTORE", + 3: "SCHEMA_CHANGE", + 4: "IMPORT", + 5: "CHANGEFEED", + 6: "CREATE_STATS", + 7: "AUTO_CREATE_STATS", + 8: "SCHEMA_CHANGE_GC", + 9: "TYPEDESC_SCHEMA_CHANGE", + 10: "STREAM_INGESTION", } var Type_value = map[string]int32{ "UNSPECIFIED": 0, @@ -127,10 +130,11 @@ var Type_value = map[string]int32{ "AUTO_CREATE_STATS": 7, "SCHEMA_CHANGE_GC": 8, "TYPEDESC_SCHEMA_CHANGE": 9, + "STREAM_INGESTION": 10, } func (Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{2} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{2} } type EncryptionInfo_Scheme int32 @@ -150,7 +154,7 @@ func (x EncryptionInfo_Scheme) String() string { return proto.EnumName(EncryptionInfo_Scheme_name, int32(x)) } func (EncryptionInfo_Scheme) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{2, 0} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{2, 0} } type SchemaChangeGCProgress_Status int32 @@ -180,7 +184,7 @@ func (x SchemaChangeGCProgress_Status) String() string { return proto.EnumName(SchemaChangeGCProgress_Status_name, int32(x)) } func (SchemaChangeGCProgress_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{18, 0} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{19, 0} } type Lease struct { @@ -194,7 +198,7 @@ func (m *Lease) Reset() { *m = Lease{} } func (m *Lease) String() string { return proto.CompactTextString(m) } func (*Lease) ProtoMessage() {} func (*Lease) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{0} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{0} } func (m *Lease) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -235,7 +239,7 @@ func (m *BackupEncryptionOptions) Reset() { *m = BackupEncryptionOptions func (m *BackupEncryptionOptions) String() string { return proto.CompactTextString(m) } func (*BackupEncryptionOptions) ProtoMessage() {} func (*BackupEncryptionOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{1} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{1} } func (m *BackupEncryptionOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -269,7 +273,7 @@ func (m *BackupEncryptionOptions_KMSInfo) Reset() { *m = BackupEncryptio func (m *BackupEncryptionOptions_KMSInfo) String() string { return proto.CompactTextString(m) } func (*BackupEncryptionOptions_KMSInfo) ProtoMessage() {} func (*BackupEncryptionOptions_KMSInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{1, 0} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{1, 0} } func (m *BackupEncryptionOptions_KMSInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -309,7 +313,7 @@ func (m *EncryptionInfo) Reset() { *m = EncryptionInfo{} } func (m *EncryptionInfo) String() string { return proto.CompactTextString(m) } func (*EncryptionInfo) ProtoMessage() {} func (*EncryptionInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{2} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{2} } func (m *EncryptionInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -334,6 +338,41 @@ func (m *EncryptionInfo) XXX_DiscardUnknown() { var xxx_messageInfo_EncryptionInfo proto.InternalMessageInfo +type StreamIngestionDetails struct { + // StreamAddress is the location of the stream which the ingestion job will + // read from. + StreamAddress github_com_cockroachdb_cockroach_pkg_ccl_streamingccl_streamclient.StreamAddress `protobuf:"bytes,1,opt,name=stream_address,json=streamAddress,proto3,casttype=github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/streamclient.StreamAddress" json:"stream_address,omitempty"` +} + +func (m *StreamIngestionDetails) Reset() { *m = StreamIngestionDetails{} } +func (m *StreamIngestionDetails) String() string { return proto.CompactTextString(m) } +func (*StreamIngestionDetails) ProtoMessage() {} +func (*StreamIngestionDetails) Descriptor() ([]byte, []int) { + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{3} +} +func (m *StreamIngestionDetails) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StreamIngestionDetails) 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 *StreamIngestionDetails) XXX_Merge(src proto.Message) { + xxx_messageInfo_StreamIngestionDetails.Merge(dst, src) +} +func (m *StreamIngestionDetails) XXX_Size() int { + return m.Size() +} +func (m *StreamIngestionDetails) XXX_DiscardUnknown() { + xxx_messageInfo_StreamIngestionDetails.DiscardUnknown(m) +} + +var xxx_messageInfo_StreamIngestionDetails proto.InternalMessageInfo + type BackupDetails struct { StartTime hlc.Timestamp `protobuf:"bytes,1,opt,name=start_time,json=startTime,proto3" json:"start_time"` EndTime hlc.Timestamp `protobuf:"bytes,2,opt,name=end_time,json=endTime,proto3" json:"end_time"` @@ -364,7 +403,7 @@ func (m *BackupDetails) Reset() { *m = BackupDetails{} } func (m *BackupDetails) String() string { return proto.CompactTextString(m) } func (*BackupDetails) ProtoMessage() {} func (*BackupDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{3} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{4} } func (m *BackupDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -396,7 +435,7 @@ func (m *BackupProgress) Reset() { *m = BackupProgress{} } func (m *BackupProgress) String() string { return proto.CompactTextString(m) } func (*BackupProgress) ProtoMessage() {} func (*BackupProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{4} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{5} } func (m *BackupProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -458,7 +497,7 @@ func (m *RestoreDetails) Reset() { *m = RestoreDetails{} } func (m *RestoreDetails) String() string { return proto.CompactTextString(m) } func (*RestoreDetails) ProtoMessage() {} func (*RestoreDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{5} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{6} } func (m *RestoreDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -495,7 +534,7 @@ func (m *RestoreDetails_DescriptorRewrite) Reset() { *m = RestoreDetails func (m *RestoreDetails_DescriptorRewrite) String() string { return proto.CompactTextString(m) } func (*RestoreDetails_DescriptorRewrite) ProtoMessage() {} func (*RestoreDetails_DescriptorRewrite) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{5, 0} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{6, 0} } func (m *RestoreDetails_DescriptorRewrite) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -528,7 +567,7 @@ func (m *RestoreDetails_BackupLocalityInfo) Reset() { *m = RestoreDetail func (m *RestoreDetails_BackupLocalityInfo) String() string { return proto.CompactTextString(m) } func (*RestoreDetails_BackupLocalityInfo) ProtoMessage() {} func (*RestoreDetails_BackupLocalityInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{5, 1} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{6, 1} } func (m *RestoreDetails_BackupLocalityInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -561,7 +600,7 @@ func (m *RestoreProgress) Reset() { *m = RestoreProgress{} } func (m *RestoreProgress) String() string { return proto.CompactTextString(m) } func (*RestoreProgress) ProtoMessage() {} func (*RestoreProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{6} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{7} } func (m *RestoreProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -622,7 +661,7 @@ func (m *ImportDetails) Reset() { *m = ImportDetails{} } func (m *ImportDetails) String() string { return proto.CompactTextString(m) } func (*ImportDetails) ProtoMessage() {} func (*ImportDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{7} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{8} } func (m *ImportDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -660,7 +699,7 @@ func (m *ImportDetails_Table) Reset() { *m = ImportDetails_Table{} } func (m *ImportDetails_Table) String() string { return proto.CompactTextString(m) } func (*ImportDetails_Table) ProtoMessage() {} func (*ImportDetails_Table) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{7, 0} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{8, 0} } func (m *ImportDetails_Table) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -702,7 +741,7 @@ func (m *SequenceValChunk) Reset() { *m = SequenceValChunk{} } func (m *SequenceValChunk) String() string { return proto.CompactTextString(m) } func (*SequenceValChunk) ProtoMessage() {} func (*SequenceValChunk) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{8} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{9} } func (m *SequenceValChunk) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -738,7 +777,7 @@ func (m *SequenceDetails) Reset() { *m = SequenceDetails{} } func (m *SequenceDetails) String() string { return proto.CompactTextString(m) } func (*SequenceDetails) ProtoMessage() {} func (*SequenceDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{9} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{10} } func (m *SequenceDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -773,7 +812,7 @@ func (m *SequenceDetails_SequenceChunks) Reset() { *m = SequenceDetails_ func (m *SequenceDetails_SequenceChunks) String() string { return proto.CompactTextString(m) } func (*SequenceDetails_SequenceChunks) ProtoMessage() {} func (*SequenceDetails_SequenceChunks) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{9, 0} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{10, 0} } func (m *SequenceDetails_SequenceChunks) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -819,7 +858,7 @@ func (m *ImportProgress) Reset() { *m = ImportProgress{} } func (m *ImportProgress) String() string { return proto.CompactTextString(m) } func (*ImportProgress) ProtoMessage() {} func (*ImportProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{10} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{11} } func (m *ImportProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -853,7 +892,7 @@ func (m *TypeSchemaChangeDetails) Reset() { *m = TypeSchemaChangeDetails func (m *TypeSchemaChangeDetails) String() string { return proto.CompactTextString(m) } func (*TypeSchemaChangeDetails) ProtoMessage() {} func (*TypeSchemaChangeDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{11} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{12} } func (m *TypeSchemaChangeDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -886,7 +925,7 @@ func (m *TypeSchemaChangeProgress) Reset() { *m = TypeSchemaChangeProgre func (m *TypeSchemaChangeProgress) String() string { return proto.CompactTextString(m) } func (*TypeSchemaChangeProgress) ProtoMessage() {} func (*TypeSchemaChangeProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{12} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{13} } func (m *TypeSchemaChangeProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -919,7 +958,7 @@ func (m *ResumeSpanList) Reset() { *m = ResumeSpanList{} } func (m *ResumeSpanList) String() string { return proto.CompactTextString(m) } func (*ResumeSpanList) ProtoMessage() {} func (*ResumeSpanList) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{13} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{14} } func (m *ResumeSpanList) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -954,7 +993,7 @@ func (m *DroppedTableDetails) Reset() { *m = DroppedTableDetails{} } func (m *DroppedTableDetails) String() string { return proto.CompactTextString(m) } func (*DroppedTableDetails) ProtoMessage() {} func (*DroppedTableDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{14} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{15} } func (m *DroppedTableDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1012,7 +1051,7 @@ func (m *SchemaChangeGCDetails) Reset() { *m = SchemaChangeGCDetails{} } func (m *SchemaChangeGCDetails) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCDetails) ProtoMessage() {} func (*SchemaChangeGCDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{15} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{16} } func (m *SchemaChangeGCDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1046,7 +1085,7 @@ func (m *SchemaChangeGCDetails_DroppedIndex) Reset() { *m = SchemaChange func (m *SchemaChangeGCDetails_DroppedIndex) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCDetails_DroppedIndex) ProtoMessage() {} func (*SchemaChangeGCDetails_DroppedIndex) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{15, 0} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{16, 0} } func (m *SchemaChangeGCDetails_DroppedIndex) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1080,7 +1119,7 @@ func (m *SchemaChangeGCDetails_DroppedID) Reset() { *m = SchemaChangeGCD func (m *SchemaChangeGCDetails_DroppedID) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCDetails_DroppedID) ProtoMessage() {} func (*SchemaChangeGCDetails_DroppedID) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{15, 1} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{16, 1} } func (m *SchemaChangeGCDetails_DroppedID) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1144,7 +1183,7 @@ func (m *SchemaChangeDetails) Reset() { *m = SchemaChangeDetails{} } func (m *SchemaChangeDetails) String() string { return proto.CompactTextString(m) } func (*SchemaChangeDetails) ProtoMessage() {} func (*SchemaChangeDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{16} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{17} } func (m *SchemaChangeDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1176,7 +1215,7 @@ func (m *SchemaChangeProgress) Reset() { *m = SchemaChangeProgress{} } func (m *SchemaChangeProgress) String() string { return proto.CompactTextString(m) } func (*SchemaChangeProgress) ProtoMessage() {} func (*SchemaChangeProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{17} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{18} } func (m *SchemaChangeProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1212,7 +1251,7 @@ func (m *SchemaChangeGCProgress) Reset() { *m = SchemaChangeGCProgress{} func (m *SchemaChangeGCProgress) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCProgress) ProtoMessage() {} func (*SchemaChangeGCProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{18} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{19} } func (m *SchemaChangeGCProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1246,7 +1285,7 @@ func (m *SchemaChangeGCProgress_IndexProgress) Reset() { *m = SchemaChan func (m *SchemaChangeGCProgress_IndexProgress) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCProgress_IndexProgress) ProtoMessage() {} func (*SchemaChangeGCProgress_IndexProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{18, 0} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{19, 0} } func (m *SchemaChangeGCProgress_IndexProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1280,7 +1319,7 @@ func (m *SchemaChangeGCProgress_TableProgress) Reset() { *m = SchemaChan func (m *SchemaChangeGCProgress_TableProgress) String() string { return proto.CompactTextString(m) } func (*SchemaChangeGCProgress_TableProgress) ProtoMessage() {} func (*SchemaChangeGCProgress_TableProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{18, 1} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{19, 1} } func (m *SchemaChangeGCProgress_TableProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1313,7 +1352,7 @@ func (m *ChangefeedTarget) Reset() { *m = ChangefeedTarget{} } func (m *ChangefeedTarget) String() string { return proto.CompactTextString(m) } func (*ChangefeedTarget) ProtoMessage() {} func (*ChangefeedTarget) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{19} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{20} } func (m *ChangefeedTarget) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1366,7 +1405,7 @@ func (m *ChangefeedDetails) Reset() { *m = ChangefeedDetails{} } func (m *ChangefeedDetails) String() string { return proto.CompactTextString(m) } func (*ChangefeedDetails) ProtoMessage() {} func (*ChangefeedDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{20} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{21} } func (m *ChangefeedDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1401,7 +1440,7 @@ func (m *ResolvedSpan) Reset() { *m = ResolvedSpan{} } func (m *ResolvedSpan) String() string { return proto.CompactTextString(m) } func (*ResolvedSpan) ProtoMessage() {} func (*ResolvedSpan) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{21} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{22} } func (m *ResolvedSpan) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1444,7 +1483,7 @@ func (m *ChangefeedProgress) Reset() { *m = ChangefeedProgress{} } func (m *ChangefeedProgress) String() string { return proto.CompactTextString(m) } func (*ChangefeedProgress) ProtoMessage() {} func (*ChangefeedProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{22} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{23} } func (m *ChangefeedProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1488,7 +1527,7 @@ func (m *CreateStatsDetails) Reset() { *m = CreateStatsDetails{} } func (m *CreateStatsDetails) String() string { return proto.CompactTextString(m) } func (*CreateStatsDetails) ProtoMessage() {} func (*CreateStatsDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{23} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{24} } func (m *CreateStatsDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1529,7 +1568,7 @@ func (m *CreateStatsDetails_ColStat) Reset() { *m = CreateStatsDetails_C func (m *CreateStatsDetails_ColStat) String() string { return proto.CompactTextString(m) } func (*CreateStatsDetails_ColStat) ProtoMessage() {} func (*CreateStatsDetails_ColStat) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{23, 0} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{24, 0} } func (m *CreateStatsDetails_ColStat) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1561,7 +1600,7 @@ func (m *CreateStatsProgress) Reset() { *m = CreateStatsProgress{} } func (m *CreateStatsProgress) String() string { return proto.CompactTextString(m) } func (*CreateStatsProgress) ProtoMessage() {} func (*CreateStatsProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{24} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{25} } func (m *CreateStatsProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1628,7 +1667,7 @@ func (m *Payload) Reset() { *m = Payload{} } func (m *Payload) String() string { return proto.CompactTextString(m) } func (*Payload) ProtoMessage() {} func (*Payload) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{25} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{26} } func (m *Payload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1967,7 +2006,7 @@ func (m *Progress) Reset() { *m = Progress{} } func (m *Progress) String() string { return proto.CompactTextString(m) } func (*Progress) ProtoMessage() {} func (*Progress) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{26} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{27} } func (m *Progress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2373,7 +2412,7 @@ func (m *Job) Reset() { *m = Job{} } func (m *Job) String() string { return proto.CompactTextString(m) } func (*Job) ProtoMessage() {} func (*Job) Descriptor() ([]byte, []int) { - return fileDescriptor_jobs_98971d9474e7d77d, []int{27} + return fileDescriptor_jobs_8868ebe50bd1b6b1, []int{28} } func (m *Job) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2404,6 +2443,7 @@ func init() { proto.RegisterType((*BackupEncryptionOptions_KMSInfo)(nil), "cockroach.sql.jobs.jobspb.BackupEncryptionOptions.KMSInfo") proto.RegisterType((*EncryptionInfo)(nil), "cockroach.sql.jobs.jobspb.EncryptionInfo") proto.RegisterMapType((map[string][]byte)(nil), "cockroach.sql.jobs.jobspb.EncryptionInfo.EncryptedDataKeyByKMSMasterKeyIDEntry") + proto.RegisterType((*StreamIngestionDetails)(nil), "cockroach.sql.jobs.jobspb.StreamIngestionDetails") proto.RegisterType((*BackupDetails)(nil), "cockroach.sql.jobs.jobspb.BackupDetails") proto.RegisterMapType((map[string]string)(nil), "cockroach.sql.jobs.jobspb.BackupDetails.UrisByLocalityKvEntry") proto.RegisterType((*BackupProgress)(nil), "cockroach.sql.jobs.jobspb.BackupProgress") @@ -2723,6 +2763,30 @@ func (m *EncryptionInfo) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func (m *StreamIngestionDetails) 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 *StreamIngestionDetails) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.StreamAddress) > 0 { + dAtA[i] = 0xa + i++ + i = encodeVarintJobs(dAtA, i, uint64(len(m.StreamAddress))) + i += copy(dAtA[i:], m.StreamAddress) + } + return i, nil +} + func (m *BackupDetails) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -4937,6 +5001,19 @@ func (m *EncryptionInfo) Size() (n int) { return n } +func (m *StreamIngestionDetails) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.StreamAddress) + if l > 0 { + n += 1 + l + sovJobs(uint64(l)) + } + return n +} + func (m *BackupDetails) Size() (n int) { if m == nil { return 0 @@ -6572,6 +6649,85 @@ func (m *EncryptionInfo) Unmarshal(dAtA []byte) error { } return nil } +func (m *StreamIngestionDetails) 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 ErrIntOverflowJobs + } + 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: StreamIngestionDetails: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: StreamIngestionDetails: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StreamAddress", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthJobs + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.StreamAddress = github_com_cockroachdb_cockroach_pkg_ccl_streamingccl_streamclient.StreamAddress(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipJobs(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthJobs + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *BackupDetails) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -13066,282 +13222,287 @@ var ( ErrIntOverflowJobs = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("jobs/jobspb/jobs.proto", fileDescriptor_jobs_98971d9474e7d77d) } - -var fileDescriptor_jobs_98971d9474e7d77d = []byte{ - // 4369 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5a, 0x4b, 0x6c, 0x23, 0x47, - 0x7a, 0x16, 0x1f, 0x22, 0x9b, 0x3f, 0x45, 0xb2, 0x59, 0xd2, 0xcc, 0x70, 0x19, 0x5b, 0x54, 0xe8, - 0xd7, 0xcc, 0xd8, 0xa6, 0xbc, 0x72, 0xd6, 0x6b, 0x4f, 0xec, 0xf1, 0x8a, 0x0f, 0x49, 0xa4, 0x46, - 0x0f, 0x37, 0xa5, 0xf1, 0xda, 0x0b, 0xa7, 0xd3, 0xec, 0x2e, 0x49, 0x1d, 0x91, 0xdd, 0x3d, 0x5d, - 0xcd, 0x99, 0xd1, 0x26, 0x48, 0x82, 0x0d, 0x12, 0x2c, 0xe6, 0x94, 0x00, 0x9b, 0x1c, 0x92, 0x0c, - 0x10, 0x20, 0x59, 0x20, 0x87, 0x00, 0x01, 0x8c, 0x20, 0xc9, 0x21, 0xb7, 0x5c, 0x7c, 0x48, 0x80, - 0xbd, 0x04, 0x58, 0xe4, 0xc0, 0x4d, 0xe4, 0x4b, 0x2e, 0x01, 0x16, 0xc9, 0x6d, 0x4e, 0x41, 0x3d, - 0xba, 0xd9, 0xa4, 0x5e, 0xd4, 0xc8, 0xde, 0x5c, 0x24, 0xd6, 0x5f, 0x7f, 0x7d, 0x55, 0xf5, 0xd7, - 0xff, 0xac, 0x6a, 0xb8, 0xfe, 0x1b, 0x76, 0x87, 0x2c, 0xd2, 0x3f, 0x4e, 0x87, 0xfd, 0xab, 0x38, - 0xae, 0xed, 0xd9, 0xe8, 0x1b, 0xba, 0xad, 0x1f, 0xba, 0xb6, 0xa6, 0x1f, 0x54, 0xc8, 0x83, 0x6e, - 0x85, 0xf5, 0x70, 0xae, 0xe2, 0x35, 0xec, 0xba, 0xb6, 0x4b, 0xf9, 0xf9, 0x0f, 0x3e, 0xa2, 0x38, - 0xb7, 0x6f, 0xef, 0xdb, 0xec, 0xe7, 0x22, 0xfd, 0x25, 0xa8, 0x88, 0x61, 0x38, 0x9d, 0x45, 0x43, - 0xf3, 0x34, 0x41, 0x2b, 0xf8, 0x34, 0xd3, 0x7e, 0x73, 0xcf, 0x76, 0x7b, 0x9a, 0xe7, 0x63, 0xbc, - 0x44, 0x1e, 0x74, 0x17, 0x75, 0xcd, 0xd3, 0xba, 0xf6, 0xfe, 0xa2, 0x81, 0x89, 0xee, 0x74, 0x16, - 0x89, 0xe7, 0xf6, 0x75, 0xaf, 0xef, 0x62, 0x43, 0x30, 0x95, 0x4e, 0x61, 0xf2, 0xb0, 0xa5, 0x59, - 0x9e, 0x8f, 0xdf, 0xf7, 0xcc, 0xee, 0xe2, 0x41, 0x57, 0x5f, 0xf4, 0xcc, 0x1e, 0x26, 0x9e, 0xd6, - 0x73, 0x78, 0x4f, 0xf9, 0x77, 0x60, 0xfa, 0x1e, 0xd6, 0x08, 0x46, 0x9f, 0x42, 0xd2, 0xb2, 0x0d, - 0xac, 0x9a, 0x46, 0x21, 0xb2, 0x10, 0xb9, 0x99, 0xa9, 0x2e, 0x1f, 0x0f, 0x4a, 0x89, 0x4d, 0xdb, - 0xc0, 0xcd, 0xfa, 0xb3, 0x41, 0xe9, 0xed, 0x7d, 0xd3, 0x3b, 0xe8, 0x77, 0x2a, 0xba, 0xdd, 0x5b, - 0x0c, 0x04, 0x61, 0x74, 0x86, 0xbf, 0x17, 0x9d, 0xc3, 0xfd, 0x45, 0xb1, 0x8d, 0x0a, 0x1f, 0xa6, - 0x24, 0x28, 0x62, 0xd3, 0x40, 0x73, 0x30, 0x8d, 0x1d, 0x5b, 0x3f, 0x28, 0x44, 0x17, 0x22, 0x37, - 0x63, 0x0a, 0x6f, 0xdc, 0x89, 0xff, 0xd7, 0x5f, 0x94, 0x22, 0xe5, 0x1f, 0x47, 0xe1, 0x46, 0x55, - 0xd3, 0x0f, 0xfb, 0x4e, 0xc3, 0xd2, 0xdd, 0x23, 0xc7, 0x33, 0x6d, 0x6b, 0x8b, 0xfd, 0x25, 0x48, - 0x86, 0xd8, 0x21, 0x3e, 0x62, 0xeb, 0x99, 0x51, 0xe8, 0x4f, 0xf4, 0x01, 0xc4, 0x7b, 0xb6, 0x81, - 0x19, 0x50, 0x76, 0xe9, 0x56, 0xe5, 0xcc, 0x33, 0xa9, 0x0c, 0xd1, 0x36, 0x6c, 0x03, 0x2b, 0x6c, - 0x18, 0xea, 0x80, 0x74, 0xd8, 0x23, 0xaa, 0x69, 0xed, 0xd9, 0x85, 0xd8, 0x42, 0xe4, 0x66, 0x7a, - 0xe9, 0xce, 0x39, 0x10, 0x67, 0x2c, 0xab, 0xb2, 0xbe, 0xd1, 0x6e, 0x5a, 0x7b, 0x76, 0x35, 0x7d, - 0x3c, 0x28, 0x25, 0x45, 0x43, 0x49, 0x1e, 0xf6, 0x08, 0xfd, 0x51, 0xdc, 0x02, 0x9f, 0x46, 0xd7, - 0xdf, 0x77, 0x4d, 0xb6, 0xfe, 0x94, 0x42, 0x7f, 0xa2, 0x37, 0x00, 0x61, 0x8e, 0x87, 0x0d, 0x95, - 0x2a, 0x80, 0x4a, 0x37, 0x18, 0x65, 0x1b, 0x94, 0x83, 0x9e, 0xba, 0xe6, 0x69, 0xeb, 0xf8, 0x88, - 0x4b, 0x48, 0xc8, 0xe9, 0x77, 0x63, 0x90, 0x1d, 0x2e, 0x85, 0xc1, 0xaf, 0x41, 0x82, 0xe8, 0x07, - 0xb8, 0x87, 0xd9, 0x0c, 0xd9, 0xa5, 0xb7, 0x26, 0x12, 0x07, 0x1d, 0x5a, 0x69, 0xb3, 0x71, 0x8a, - 0x18, 0x8f, 0x10, 0xc4, 0x89, 0xd6, 0xf5, 0xc4, 0x42, 0xd8, 0x6f, 0xf4, 0x67, 0x11, 0x58, 0x18, - 0x5f, 0x51, 0xf5, 0x68, 0x7d, 0xa3, 0xbd, 0xa1, 0x11, 0x0f, 0xbb, 0xeb, 0xf8, 0xa8, 0x59, 0x2f, - 0xc4, 0x16, 0x62, 0x37, 0xd3, 0x4b, 0x5b, 0x93, 0x4f, 0xdc, 0xb8, 0x00, 0xb1, 0x61, 0x79, 0xee, - 0x91, 0x72, 0xe1, 0xc4, 0xc5, 0x36, 0xbc, 0x32, 0x11, 0x54, 0x58, 0x87, 0x52, 0x5c, 0x87, 0xe6, - 0x60, 0xfa, 0xa1, 0xd6, 0xed, 0x63, 0xb1, 0x5b, 0xde, 0xb8, 0x13, 0x7d, 0x37, 0x52, 0xbe, 0x01, - 0x09, 0x2e, 0x18, 0x94, 0x81, 0xd4, 0x72, 0xa3, 0xbd, 0xf4, 0xad, 0x77, 0x56, 0x6b, 0x1b, 0xf2, - 0x94, 0x38, 0x82, 0x7f, 0x4a, 0x40, 0x86, 0xeb, 0x44, 0x1d, 0x7b, 0x9a, 0xd9, 0x25, 0xa8, 0x0a, - 0x40, 0x3c, 0xcd, 0xf5, 0x54, 0x6a, 0x56, 0x6c, 0x8e, 0xf4, 0xd2, 0x8b, 0x21, 0x61, 0x50, 0xb3, - 0xab, 0x1c, 0x74, 0xf5, 0xca, 0x8e, 0x6f, 0x76, 0xd5, 0xf8, 0x17, 0x83, 0xd2, 0x94, 0x92, 0x62, - 0xc3, 0x28, 0x15, 0xdd, 0x05, 0x09, 0x5b, 0x06, 0x47, 0x88, 0x4e, 0x8e, 0x90, 0xc4, 0x96, 0xc1, - 0xc6, 0x7f, 0x83, 0x2b, 0x19, 0x55, 0xe7, 0x54, 0x35, 0x79, 0x3c, 0x28, 0xc5, 0x76, 0x95, 0x26, - 0xd7, 0xb6, 0xf7, 0xa1, 0x68, 0x60, 0xc7, 0xc5, 0xba, 0x46, 0xd5, 0xad, 0xc3, 0x96, 0xae, 0xf6, - 0x34, 0xcb, 0xdc, 0xc3, 0xc4, 0x2b, 0xc4, 0xd9, 0xf6, 0x0b, 0x43, 0x0e, 0xbe, 0xb7, 0x0d, 0xd1, - 0x8f, 0x7e, 0x2f, 0x02, 0xb3, 0x7d, 0xd7, 0x24, 0x6a, 0xe7, 0x48, 0xed, 0xda, 0xba, 0xd6, 0x35, - 0xbd, 0x23, 0xf5, 0xf0, 0x61, 0x61, 0x9a, 0x9d, 0xf9, 0xdd, 0x0b, 0x0d, 0x47, 0x08, 0xa9, 0xb2, - 0xeb, 0x9a, 0xa4, 0x7a, 0x74, 0x4f, 0x20, 0xac, 0x3f, 0x64, 0xe7, 0x52, 0x9d, 0x3b, 0x1e, 0x94, - 0xe4, 0x5d, 0xa5, 0x19, 0xee, 0xba, 0xaf, 0xc8, 0xfd, 0x31, 0x66, 0xa4, 0x05, 0x16, 0x63, 0xda, - 0x96, 0x6a, 0x73, 0x13, 0x2c, 0x24, 0x98, 0xa0, 0x96, 0x2e, 0x6f, 0xbc, 0x4a, 0x1e, 0x9f, 0x70, - 0x33, 0x7f, 0x14, 0x81, 0x22, 0xf5, 0x86, 0x58, 0xa7, 0x62, 0x0a, 0x3c, 0xa4, 0xea, 0x62, 0xdd, - 0x76, 0x8d, 0x42, 0x92, 0xca, 0xa9, 0xda, 0xfe, 0xf7, 0x49, 0x9d, 0x20, 0xf3, 0xb5, 0xfd, 0xbe, - 0x69, 0x54, 0x76, 0x77, 0x9b, 0xf5, 0xe3, 0x41, 0xa9, 0xb0, 0xed, 0x83, 0x07, 0x87, 0xa8, 0x30, - 0x68, 0xa5, 0xe0, 0x9c, 0xd1, 0x83, 0xde, 0x85, 0xac, 0x6e, 0x77, 0xbb, 0x58, 0x67, 0xdb, 0xde, - 0x55, 0x9a, 0x05, 0x89, 0x1d, 0x70, 0xfe, 0x78, 0x50, 0xca, 0xd4, 0x82, 0x1e, 0x7a, 0xd4, 0x19, - 0x3d, 0xdc, 0x44, 0x0a, 0xe4, 0x42, 0x02, 0x63, 0xae, 0x2e, 0xc5, 0xa4, 0x75, 0x6b, 0x62, 0x2b, - 0x55, 0xb2, 0x78, 0xa4, 0x5d, 0xac, 0xc1, 0xb5, 0x53, 0x4f, 0xf1, 0x22, 0xeb, 0x4a, 0x85, 0xad, - 0x4b, 0x86, 0x2c, 0x3f, 0x94, 0x6d, 0xd7, 0xde, 0x77, 0x31, 0x21, 0xe5, 0xcf, 0xb3, 0x90, 0x55, - 0x30, 0xf1, 0x6c, 0x17, 0xfb, 0x16, 0xf5, 0x79, 0x04, 0x66, 0x69, 0x04, 0x73, 0x4d, 0xc7, 0xb3, - 0x5d, 0xd5, 0xc5, 0x8f, 0x5c, 0xd3, 0xc3, 0xa4, 0x10, 0x65, 0x4a, 0xb7, 0x7c, 0xce, 0x16, 0x46, - 0x81, 0x2a, 0xf5, 0x00, 0x44, 0x11, 0x18, 0x5c, 0xef, 0xee, 0xfe, 0xe0, 0x67, 0xa5, 0x3b, 0x13, - 0x9d, 0xe3, 0xc9, 0xa0, 0x5a, 0x69, 0xd6, 0x15, 0x64, 0x9c, 0x00, 0x46, 0x2f, 0x40, 0x9c, 0xea, - 0x2d, 0xf3, 0x86, 0xa9, 0xaa, 0x74, 0x3c, 0x28, 0xc5, 0xa9, 0x66, 0x2b, 0x8c, 0x3a, 0x62, 0xe0, - 0xf1, 0xe7, 0x30, 0xf0, 0x55, 0x48, 0x7b, 0x5a, 0xa7, 0x8b, 0x55, 0x3a, 0x33, 0x11, 0xe6, 0xf7, - 0xea, 0x98, 0x24, 0xc8, 0x83, 0x6e, 0x47, 0x23, 0xb8, 0xb2, 0x43, 0x39, 0x43, 0x7b, 0x07, 0xcf, - 0x27, 0x10, 0xb4, 0x08, 0x69, 0xfb, 0x21, 0x76, 0x5d, 0xd3, 0xc0, 0xaa, 0xd1, 0x61, 0x36, 0x94, - 0xaa, 0x66, 0x8f, 0x07, 0x25, 0xd8, 0x12, 0xe4, 0x7a, 0x55, 0x01, 0x9f, 0xa5, 0xde, 0x41, 0x1e, - 0xcc, 0x09, 0xa7, 0x11, 0xd8, 0x3f, 0xd3, 0xa7, 0x24, 0x5b, 0xc2, 0xfb, 0x93, 0x1f, 0x06, 0x3f, - 0x77, 0x5f, 0x79, 0x58, 0xf0, 0xe4, 0x9b, 0x44, 0x9d, 0x13, 0x3d, 0xe8, 0x75, 0xc8, 0x3b, 0x2e, - 0x76, 0x34, 0x17, 0xab, 0xba, 0xdd, 0x73, 0xba, 0xd8, 0xc3, 0x06, 0xd3, 0x7e, 0x49, 0x91, 0x45, - 0x47, 0xcd, 0xa7, 0xa3, 0x57, 0x20, 0x4b, 0x3c, 0xcd, 0xa3, 0x31, 0x9d, 0x60, 0x97, 0x72, 0xa6, - 0x18, 0x67, 0x86, 0x51, 0x9b, 0x82, 0x88, 0xde, 0x86, 0x6b, 0xc3, 0x73, 0x23, 0xaa, 0xd3, 0xef, - 0x74, 0x4d, 0x72, 0x80, 0x8d, 0x02, 0x30, 0xee, 0xb9, 0x50, 0xe7, 0xb6, 0xdf, 0x87, 0x8e, 0x46, - 0x54, 0x51, 0xa7, 0x82, 0xd1, 0xf6, 0x71, 0x21, 0xbd, 0x10, 0xb9, 0x39, 0x5d, 0x5d, 0x7b, 0x36, - 0x28, 0xd5, 0x27, 0xd6, 0x23, 0x82, 0x7b, 0x8b, 0x9e, 0x8b, 0x71, 0x48, 0x2d, 0x6b, 0x02, 0x2f, - 0xac, 0x51, 0x3e, 0x0d, 0x29, 0x00, 0x43, 0x13, 0x2c, 0xcc, 0x3c, 0xb7, 0xb7, 0x0b, 0xa1, 0xa0, - 0x65, 0x48, 0xf2, 0xa4, 0x90, 0x14, 0x32, 0xec, 0x00, 0x7f, 0xf9, 0x2c, 0x1d, 0x62, 0x5c, 0xa1, - 0x53, 0xf2, 0xc7, 0xa1, 0x3a, 0x80, 0x77, 0xe4, 0xf8, 0x9a, 0x98, 0x65, 0x28, 0xaf, 0x9c, 0x85, - 0x72, 0xe4, 0x84, 0x15, 0x31, 0xe5, 0x89, 0x36, 0x41, 0x2d, 0x98, 0x61, 0x79, 0x87, 0x26, 0x70, - 0x72, 0x0c, 0xe7, 0xb5, 0x33, 0x70, 0x58, 0x44, 0xd6, 0x42, 0x48, 0x69, 0x12, 0x50, 0x08, 0xda, - 0x86, 0x2c, 0x4d, 0xa3, 0x28, 0xa7, 0x40, 0x93, 0x19, 0xda, 0xad, 0x33, 0xd0, 0xea, 0x82, 0x39, - 0x84, 0x97, 0x31, 0x42, 0x34, 0x52, 0xfc, 0xdf, 0x08, 0xe4, 0x4f, 0x38, 0x0f, 0xb4, 0x03, 0xd1, - 0x20, 0x33, 0xa6, 0x3e, 0x3d, 0xca, 0xb2, 0xe2, 0xab, 0x38, 0x92, 0xa8, 0x69, 0xa0, 0x7d, 0x48, - 0x51, 0x75, 0xb6, 0x3c, 0x9a, 0x76, 0x47, 0x19, 0x78, 0xeb, 0x78, 0x50, 0x92, 0xb6, 0x19, 0xf1, - 0xca, 0x53, 0x48, 0x1c, 0xbc, 0x69, 0xa0, 0x12, 0xa4, 0x3d, 0x5b, 0xc5, 0x8f, 0x4d, 0xe2, 0x99, - 0xd6, 0x3e, 0x4b, 0x16, 0x24, 0x05, 0x3c, 0xbb, 0x21, 0x28, 0xc5, 0x3f, 0x8f, 0x02, 0x3a, 0x69, - 0xa5, 0xe8, 0x1f, 0x23, 0xf0, 0x82, 0x9f, 0x03, 0xd8, 0xae, 0xb9, 0x6f, 0x5a, 0x5a, 0x77, 0x24, - 0x19, 0x88, 0x30, 0x69, 0x7f, 0x7a, 0x15, 0x57, 0x20, 0x12, 0x84, 0x2d, 0x01, 0x3f, 0x9e, 0x28, - 0xbc, 0x40, 0x23, 0x28, 0x4f, 0x14, 0x4e, 0xb0, 0xdc, 0x57, 0x0a, 0xfd, 0x33, 0x06, 0x17, 0xd7, - 0xe1, 0xc5, 0x73, 0x81, 0x2f, 0x13, 0xbb, 0x8a, 0x3f, 0x88, 0xc0, 0x8d, 0x33, 0x22, 0x4a, 0x18, - 0x27, 0xc3, 0x71, 0x3e, 0x0a, 0xe3, 0xa4, 0x97, 0x7e, 0xf5, 0x0a, 0x51, 0x2b, 0xb4, 0x88, 0x56, - 0x5c, 0x8a, 0xc8, 0xd1, 0xf2, 0x5b, 0x90, 0x13, 0x83, 0xfc, 0x38, 0x8a, 0x5e, 0x04, 0x38, 0x30, - 0xf7, 0x0f, 0xd4, 0x47, 0x9a, 0x87, 0x5d, 0x51, 0x2e, 0xa5, 0x28, 0xe5, 0x63, 0x4a, 0x28, 0xff, - 0x9b, 0x04, 0x99, 0x66, 0xcf, 0xb1, 0x5d, 0xcf, 0x8f, 0xb2, 0xf7, 0x20, 0xc1, 0xe2, 0x02, 0x11, - 0xe7, 0x57, 0x39, 0x67, 0x85, 0x23, 0x23, 0x79, 0x7c, 0x11, 0x6e, 0x41, 0x60, 0x04, 0xe1, 0x2f, - 0x7a, 0x6a, 0xf8, 0xfb, 0x00, 0x12, 0xbc, 0xa4, 0x15, 0x15, 0x57, 0x29, 0x34, 0x97, 0x5f, 0x2f, - 0x36, 0xb7, 0x56, 0xcc, 0x2e, 0x5e, 0x61, 0x6c, 0x3e, 0x38, 0x1f, 0x84, 0x5e, 0x05, 0x89, 0x10, - 0x4f, 0x25, 0xe6, 0xf7, 0x79, 0xf4, 0x8c, 0xf1, 0xb2, 0xab, 0xdd, 0xde, 0x69, 0x9b, 0xdf, 0xc7, - 0x4a, 0x92, 0x10, 0x8f, 0xfe, 0x40, 0x45, 0x90, 0x1e, 0x69, 0xdd, 0x2e, 0x8b, 0xb2, 0xd3, 0xac, - 0xcc, 0x0c, 0xda, 0xa3, 0x66, 0x96, 0xf8, 0x7a, 0xcd, 0x4c, 0x04, 0x4c, 0x47, 0xf3, 0x0e, 0x58, - 0xe6, 0x98, 0x52, 0x80, 0x93, 0xb6, 0x35, 0xef, 0x00, 0x15, 0x20, 0x49, 0x34, 0x1a, 0xbb, 0x48, - 0x41, 0x5a, 0x88, 0xdd, 0x9c, 0x51, 0xfc, 0x26, 0x9a, 0x07, 0x16, 0x79, 0x79, 0x93, 0x05, 0xb1, - 0x98, 0x12, 0xa2, 0x30, 0x39, 0x1c, 0x9a, 0x8e, 0xba, 0x77, 0x48, 0x78, 0xd0, 0x12, 0x72, 0x38, - 0x34, 0x9d, 0x95, 0x75, 0xa2, 0x24, 0x69, 0xe7, 0xca, 0x21, 0x41, 0xaf, 0x41, 0xce, 0xb4, 0xf6, - 0x31, 0xf1, 0x54, 0xc3, 0x74, 0xb1, 0xee, 0x75, 0x8f, 0x58, 0xc0, 0x92, 0x94, 0x2c, 0x27, 0xd7, - 0x05, 0x15, 0xdd, 0x02, 0x79, 0x3c, 0xcc, 0xb2, 0x40, 0x23, 0x29, 0xb9, 0xb1, 0x28, 0x4b, 0x59, - 0xf9, 0x51, 0x87, 0x02, 0x67, 0x86, 0xb3, 0x72, 0xfa, 0x30, 0x66, 0x56, 0x60, 0xd6, 0xd1, 0x5c, - 0x82, 0xd5, 0x4e, 0xdf, 0x32, 0xba, 0x58, 0xe5, 0xbe, 0xba, 0x90, 0x65, 0xdc, 0x79, 0xd6, 0x55, - 0x65, 0x3d, 0xdc, 0xad, 0x5f, 0x94, 0x7b, 0x5f, 0xff, 0x7f, 0xc8, 0xbd, 0x8b, 0x3f, 0x8e, 0xc2, - 0x34, 0xd3, 0x73, 0x74, 0x07, 0xe2, 0xf4, 0x98, 0x45, 0x65, 0x37, 0x69, 0xce, 0xc5, 0xc6, 0xd0, - 0x9a, 0xda, 0xd2, 0x7a, 0xb8, 0x80, 0x98, 0x12, 0xb0, 0xdf, 0xe8, 0x06, 0x24, 0x09, 0x7e, 0xa0, - 0x3e, 0xd4, 0xba, 0x85, 0x59, 0x76, 0xc2, 0x09, 0x82, 0x1f, 0xdc, 0xd7, 0xba, 0xe8, 0x1a, 0x24, - 0x4c, 0xa2, 0x5a, 0xf8, 0x51, 0x61, 0x8e, 0x49, 0x6a, 0xda, 0x24, 0x9b, 0xf8, 0x11, 0x73, 0xdb, - 0x9a, 0xbb, 0x8f, 0x3d, 0x55, 0xb7, 0xbb, 0xa4, 0x70, 0x8d, 0x1a, 0x18, 0x4d, 0xe9, 0x28, 0xa9, - 0x66, 0x77, 0x09, 0xfa, 0x25, 0x48, 0x3d, 0xd2, 0x88, 0x8a, 0x7b, 0x8e, 0x77, 0xc4, 0x84, 0x25, - 0x51, 0xb5, 0x27, 0x0d, 0xda, 0x6e, 0xc5, 0xa5, 0xa8, 0x1c, 0x6b, 0xc5, 0xa5, 0x98, 0x1c, 0x6f, - 0xc5, 0xa5, 0xb8, 0x3c, 0xdd, 0x8a, 0x4b, 0xd3, 0x72, 0xa2, 0x15, 0x97, 0x12, 0x72, 0xb2, 0x15, - 0x97, 0x92, 0xb2, 0xd4, 0x8a, 0x4b, 0x92, 0x9c, 0x6a, 0xc5, 0xa5, 0x94, 0x0c, 0xad, 0xb8, 0x04, - 0x72, 0xba, 0x15, 0x97, 0xd2, 0xf2, 0x4c, 0x2b, 0x2e, 0xcd, 0xc8, 0x99, 0x56, 0x5c, 0xca, 0xc8, - 0xd9, 0x56, 0x5c, 0xca, 0xca, 0xb9, 0x56, 0x5c, 0xca, 0xc9, 0x72, 0x2b, 0x2e, 0xc9, 0x72, 0xbe, - 0x15, 0x97, 0xf2, 0x32, 0x2a, 0x7f, 0x1e, 0x01, 0xb9, 0x8d, 0x1f, 0xf4, 0xb1, 0xa5, 0xe3, 0xfb, - 0x5a, 0xb7, 0x76, 0xd0, 0xb7, 0x0e, 0xd1, 0xab, 0x90, 0xd3, 0xe9, 0x0f, 0x95, 0x17, 0xc6, 0x74, - 0xab, 0x11, 0xb6, 0xd5, 0x0c, 0x23, 0xb7, 0x29, 0x95, 0xee, 0xf8, 0x45, 0x00, 0xc1, 0x47, 0x2d, - 0x9b, 0x5f, 0x0c, 0xa5, 0x38, 0x0b, 0x35, 0xe7, 0x31, 0x18, 0xd7, 0x7e, 0xc4, 0xdc, 0xc7, 0x08, - 0x8c, 0x62, 0x3f, 0x42, 0x8b, 0x30, 0x67, 0xe1, 0xc7, 0x9e, 0x3a, 0xce, 0xcc, 0x5c, 0x85, 0x92, - 0xa7, 0x7d, 0xb5, 0xf0, 0x80, 0xf2, 0xbf, 0x46, 0x21, 0xe7, 0x2f, 0xda, 0x77, 0x87, 0x7b, 0x20, - 0xd3, 0x63, 0x31, 0x0d, 0xd5, 0xb3, 0x39, 0x92, 0xef, 0x18, 0x3f, 0x38, 0xc7, 0x31, 0x8e, 0xa1, - 0xd0, 0x76, 0xd3, 0xd8, 0xb1, 0xd9, 0x74, 0x3c, 0x34, 0x28, 0x19, 0x12, 0xa6, 0x15, 0x77, 0x21, - 0xeb, 0x0f, 0xe2, 0x14, 0x54, 0x83, 0xc4, 0xc8, 0x7c, 0xaf, 0x4f, 0x30, 0x9f, 0x2f, 0x6a, 0x45, - 0x0c, 0x2d, 0xfe, 0x26, 0xa0, 0x93, 0x73, 0x87, 0xc3, 0xd2, 0x34, 0x0f, 0x4b, 0x5b, 0xa3, 0x61, - 0xe9, 0xbd, 0xcb, 0xed, 0x2d, 0xb4, 0xec, 0x70, 0x55, 0xf7, 0xcf, 0x51, 0xc8, 0xf2, 0x10, 0x11, - 0x84, 0xa3, 0xd7, 0x21, 0xcf, 0x9c, 0x96, 0x69, 0xed, 0xab, 0x8e, 0x20, 0xb2, 0xfd, 0x45, 0x15, - 0xd9, 0xef, 0x08, 0x98, 0x5f, 0x82, 0x8c, 0x8b, 0x35, 0x63, 0xc8, 0x18, 0x65, 0x8c, 0x33, 0x94, - 0x18, 0x30, 0xbd, 0x02, 0x59, 0x16, 0x0d, 0x87, 0x5c, 0x31, 0xc6, 0x95, 0x61, 0xd4, 0x80, 0xad, - 0x0a, 0x19, 0xe2, 0x68, 0xd6, 0x90, 0x2b, 0xce, 0x84, 0x7a, 0xe3, 0x94, 0x88, 0xd3, 0x76, 0x34, - 0x4b, 0x44, 0x9a, 0x19, 0x3a, 0x26, 0x1c, 0x4b, 0x5d, 0x4c, 0xfa, 0x3d, 0xac, 0x3a, 0x36, 0x2f, - 0xb6, 0x62, 0x4a, 0x8a, 0x53, 0xb6, 0x6d, 0x82, 0x76, 0x99, 0xaa, 0x30, 0x59, 0xa8, 0x06, 0x17, - 0x4e, 0x21, 0xc1, 0x66, 0xb9, 0x3d, 0xb9, 0x38, 0x95, 0x1c, 0x19, 0x25, 0x94, 0x7f, 0x0b, 0x6e, - 0xd0, 0x7c, 0x99, 0x3b, 0xc5, 0xda, 0x81, 0x66, 0xed, 0x07, 0xca, 0xa9, 0x41, 0x92, 0xe5, 0xdc, - 0x41, 0xfa, 0xb9, 0x76, 0x3c, 0x28, 0x25, 0x28, 0xf7, 0x95, 0x03, 0x57, 0x82, 0x02, 0x37, 0x8d, - 0x72, 0x11, 0x0a, 0xe3, 0xb3, 0x07, 0x35, 0xba, 0xc2, 0x4a, 0xf4, 0x7e, 0x0f, 0x53, 0x89, 0xdd, - 0x33, 0x89, 0x87, 0xbe, 0x03, 0x33, 0x42, 0x42, 0x54, 0x70, 0xbe, 0xe6, 0x5e, 0x20, 0xe4, 0xb4, - 0x1b, 0x80, 0x90, 0xf2, 0xdf, 0x45, 0x60, 0xb6, 0xee, 0xda, 0x8e, 0x83, 0x0d, 0xe1, 0x3b, 0xf9, - 0x56, 0x7d, 0x97, 0x19, 0x09, 0xb9, 0xcc, 0x4d, 0x88, 0x36, 0xeb, 0x22, 0x37, 0xbe, 0x7b, 0xd5, - 0x94, 0xbb, 0x59, 0x47, 0xef, 0x41, 0x82, 0x96, 0x86, 0x7d, 0xc2, 0xfc, 0x49, 0xf6, 0x44, 0x11, - 0x34, 0x72, 0x6c, 0x8c, 0x51, 0x11, 0x03, 0xca, 0x7f, 0x90, 0x80, 0x6b, 0x61, 0x19, 0xad, 0xd6, - 0xfc, 0x85, 0x7f, 0x06, 0x49, 0xd3, 0x32, 0xf0, 0x63, 0x3c, 0x91, 0xdf, 0x38, 0x0d, 0xa2, 0x22, - 0xe4, 0xd1, 0xa4, 0x30, 0x7e, 0xd9, 0x25, 0x30, 0xd1, 0x77, 0x83, 0x74, 0x8d, 0x5f, 0x83, 0xdc, - 0x79, 0x6e, 0xf4, 0xfa, 0x58, 0xea, 0x36, 0x92, 0x19, 0x31, 0x07, 0xfb, 0x35, 0x65, 0x46, 0x6d, - 0xc8, 0x9b, 0x96, 0x87, 0xdd, 0x2e, 0xd6, 0x1e, 0xd2, 0x40, 0x4f, 0xa7, 0x17, 0xb7, 0x21, 0x93, - 0x86, 0x55, 0x39, 0x04, 0xc0, 0xc3, 0xf3, 0x67, 0x30, 0x1b, 0x06, 0xf5, 0x8f, 0xe0, 0xfc, 0x1b, - 0x12, 0x26, 0xe1, 0x21, 0xac, 0x7f, 0x11, 0x11, 0x02, 0x6a, 0x72, 0x9c, 0xe2, 0x9f, 0x44, 0x60, - 0x26, 0x7c, 0x2c, 0xc8, 0x04, 0x89, 0xcd, 0xe1, 0xdb, 0x62, 0xac, 0xba, 0x49, 0x73, 0x30, 0xd6, - 0xc9, 0x64, 0xf5, 0xe1, 0x73, 0xcb, 0x8a, 0x43, 0x88, 0x23, 0x6f, 0x1a, 0x34, 0xb0, 0x1b, 0xae, - 0xed, 0x0c, 0xaf, 0x85, 0x63, 0x8a, 0x44, 0x09, 0x34, 0x57, 0x29, 0xfe, 0x36, 0xa4, 0x82, 0x03, - 0x0d, 0x55, 0xa6, 0xb1, 0xaf, 0xb0, 0x32, 0x3d, 0x6f, 0xfe, 0xf2, 0xcf, 0x12, 0x30, 0x7b, 0x9a, - 0xab, 0xfa, 0x04, 0xe4, 0x90, 0x67, 0x50, 0xbb, 0x26, 0xf1, 0x84, 0xc6, 0xde, 0x3a, 0xbf, 0x04, - 0x0a, 0xb9, 0x17, 0x71, 0x1e, 0x59, 0x77, 0xd4, 0xe9, 0x7c, 0x0f, 0xb2, 0x06, 0xdf, 0xb2, 0x2a, - 0x4c, 0x21, 0x76, 0x61, 0xe5, 0x72, 0x8a, 0x8b, 0x11, 0xe8, 0x19, 0x23, 0xd4, 0x45, 0xd8, 0x4d, - 0xb7, 0x8f, 0x1e, 0xdc, 0x26, 0x98, 0x06, 0xd3, 0xcf, 0x4c, 0xb5, 0x7d, 0x3c, 0x28, 0xe5, 0x05, - 0x96, 0x7f, 0x7d, 0x70, 0x65, 0x19, 0xe7, 0x8d, 0x31, 0x40, 0x83, 0x3a, 0x7a, 0xda, 0x4f, 0x27, - 0x9e, 0x1e, 0x3a, 0x7a, 0xaa, 0xa9, 0x57, 0x77, 0xf4, 0xf4, 0x67, 0xd3, 0x40, 0xbf, 0x1f, 0x81, - 0x3c, 0xbf, 0x4b, 0xec, 0xf5, 0x3d, 0x8d, 0x5f, 0x10, 0xfb, 0x15, 0xd1, 0x27, 0xc7, 0x83, 0x52, - 0x8e, 0x09, 0x64, 0x43, 0xf4, 0xb1, 0x69, 0xab, 0xcf, 0x3b, 0xed, 0x10, 0x45, 0x54, 0x09, 0x01, - 0xc1, 0x40, 0xeb, 0x90, 0xe5, 0xe5, 0x9d, 0x4a, 0x0b, 0x1c, 0xd3, 0xb6, 0x58, 0xa9, 0x94, 0xa9, - 0xbe, 0xfc, 0x6c, 0x50, 0x5a, 0x38, 0x45, 0xb3, 0x78, 0x65, 0x78, 0x9f, 0xf3, 0x2a, 0x99, 0xbd, - 0x70, 0x13, 0xe9, 0x90, 0x09, 0x54, 0xe3, 0xc8, 0x11, 0x95, 0xd5, 0xd5, 0x83, 0xc5, 0x8c, 0xaf, - 0x23, 0x14, 0x13, 0xed, 0x43, 0xce, 0x9f, 0x84, 0x97, 0x34, 0xa4, 0x90, 0xfa, 0x4a, 0xa6, 0xf1, - 0xd5, 0x9a, 0xef, 0x9a, 0x88, 0x22, 0xff, 0x3a, 0xcc, 0x9d, 0x1a, 0x8d, 0xff, 0x74, 0x1a, 0xae, - 0x8f, 0x7a, 0xf8, 0x20, 0x71, 0x51, 0xc7, 0x63, 0xd0, 0x87, 0x13, 0x47, 0x09, 0x1f, 0x83, 0xbb, - 0x21, 0xbf, 0x35, 0x1e, 0x85, 0x3e, 0x1b, 0x8b, 0x42, 0xcf, 0x81, 0xcf, 0xd4, 0x6b, 0x0c, 0x5f, - 0x80, 0x16, 0xff, 0x25, 0x02, 0x99, 0x91, 0xf9, 0x7f, 0x91, 0xee, 0x76, 0x3b, 0xc8, 0x0a, 0xf8, - 0xcb, 0xf2, 0xbb, 0x97, 0xdf, 0xdb, 0x68, 0xb2, 0x50, 0xfc, 0x87, 0x08, 0x64, 0x46, 0xb6, 0xfb, - 0x35, 0x39, 0xea, 0xaf, 0x7c, 0xe5, 0xe5, 0x6f, 0x43, 0x82, 0x53, 0x10, 0x82, 0xec, 0xc7, 0xcb, - 0xcd, 0x9d, 0xe6, 0xe6, 0xaa, 0xba, 0xb2, 0xa5, 0xa8, 0xab, 0x35, 0x79, 0x0a, 0xcd, 0x80, 0x54, - 0x6f, 0xdc, 0x6b, 0x50, 0xa2, 0x1c, 0x41, 0x69, 0x48, 0xb2, 0x56, 0xa3, 0x2e, 0x47, 0xcb, 0x55, - 0x90, 0x39, 0xf6, 0x1e, 0xa6, 0xae, 0x95, 0x16, 0xa9, 0xa8, 0x02, 0xb3, 0x14, 0x16, 0xf7, 0x68, - 0x8e, 0x41, 0x83, 0x89, 0x1a, 0xca, 0xf0, 0xf2, 0x41, 0x17, 0x0d, 0x2b, 0x9b, 0x5a, 0x0f, 0x97, - 0xff, 0x3e, 0x0e, 0xf9, 0x21, 0x88, 0x1f, 0x58, 0x5e, 0x05, 0x89, 0x98, 0xd6, 0xa1, 0x3a, 0x7c, - 0xe8, 0xe4, 0x97, 0x1f, 0xa6, 0x75, 0xb8, 0xab, 0x34, 0x95, 0x24, 0xed, 0xdc, 0x75, 0x4d, 0xd4, - 0x82, 0xb8, 0xed, 0x78, 0x7e, 0xde, 0xff, 0xce, 0x39, 0xa2, 0x38, 0x31, 0x47, 0x65, 0xcb, 0xf1, - 0x44, 0xd5, 0xc6, 0x30, 0xd0, 0xdf, 0x44, 0x20, 0xc9, 0x2b, 0x6d, 0x3f, 0xc3, 0x7f, 0xef, 0x52, - 0x78, 0x5c, 0x00, 0xe2, 0xd5, 0xe9, 0x63, 0xaa, 0xea, 0xcf, 0x06, 0xa5, 0xfc, 0xb8, 0x80, 0xc8, - 0x15, 0x9f, 0xa3, 0xfc, 0x25, 0xa2, 0x16, 0x7f, 0x08, 0x19, 0x0a, 0x9a, 0xb9, 0xd4, 0x09, 0xdf, - 0x9a, 0x32, 0x23, 0x07, 0x51, 0xdc, 0x87, 0x99, 0xf0, 0xea, 0x4f, 0xb9, 0xe1, 0x5c, 0x1e, 0x2d, - 0x25, 0x5f, 0x9f, 0x48, 0x32, 0x1c, 0x33, 0x7c, 0xad, 0xfa, 0x6d, 0x48, 0x05, 0x62, 0xbf, 0xcc, - 0x7d, 0x2c, 0xf7, 0x92, 0xc1, 0x05, 0xc7, 0xb4, 0x9c, 0x28, 0xff, 0x6d, 0x04, 0x66, 0x14, 0x4c, - 0xec, 0xee, 0x43, 0x6c, 0xd0, 0xac, 0x01, 0x7d, 0x13, 0xe2, 0x34, 0x0b, 0x11, 0x77, 0x37, 0x17, - 0x94, 0x27, 0x8c, 0x15, 0x2d, 0x43, 0x2a, 0xb8, 0x81, 0xba, 0xcc, 0x5b, 0xfc, 0x70, 0x14, 0xba, - 0x05, 0x72, 0xc7, 0xee, 0x5b, 0x86, 0xe6, 0x1e, 0xa9, 0x2e, 0xd6, 0xf4, 0x03, 0x6c, 0x88, 0xdb, - 0xf6, 0x9c, 0x4f, 0x57, 0x38, 0xb9, 0xfc, 0xc3, 0x28, 0xa0, 0xa1, 0x70, 0x42, 0x6e, 0x82, 0xe6, - 0x3e, 0x6c, 0x1f, 0xa2, 0xc0, 0x8a, 0x9e, 0xfa, 0x3e, 0x32, 0x96, 0x42, 0x05, 0x1b, 0xf7, 0x8f, - 0xd4, 0x0d, 0xd1, 0x08, 0xfa, 0xe3, 0xf3, 0xef, 0xd9, 0x62, 0xec, 0x9e, 0x8d, 0x69, 0xe9, 0x2f, - 0xf4, 0xae, 0x4d, 0x84, 0xbb, 0xff, 0x89, 0x03, 0xaa, 0xb9, 0x58, 0xf3, 0x30, 0xf5, 0x3c, 0xe4, - 0xbc, 0x7a, 0xb0, 0x0a, 0xd3, 0xbc, 0x78, 0x88, 0x5e, 0xa6, 0x78, 0x10, 0x42, 0xe1, 0x43, 0xd1, - 0xaf, 0xc1, 0x8c, 0x6e, 0x77, 0xfb, 0x3d, 0x4b, 0x65, 0xaf, 0x84, 0x22, 0x95, 0xfc, 0xd6, 0x79, - 0x4a, 0x7c, 0x62, 0x71, 0x95, 0x9a, 0xdd, 0xa5, 0x6d, 0xbf, 0xbe, 0xe5, 0x80, 0x8c, 0x03, 0xbd, - 0x00, 0xa9, 0xc0, 0xa0, 0x58, 0x12, 0x99, 0x52, 0x86, 0x04, 0xb4, 0x04, 0xd3, 0x1a, 0x51, 0xed, - 0x3d, 0x96, 0xe5, 0x5d, 0xa4, 0x61, 0x4a, 0x5c, 0x23, 0x5b, 0x7b, 0xe8, 0x6d, 0xc8, 0xec, 0x3d, - 0xe0, 0x99, 0x2f, 0x77, 0xa0, 0xfc, 0xf1, 0x36, 0x77, 0x3c, 0x28, 0xa5, 0x57, 0x3e, 0x62, 0x9b, - 0xa5, 0xee, 0x53, 0x49, 0xef, 0x3d, 0x08, 0x1a, 0xe8, 0x36, 0xe4, 0x7b, 0xda, 0x63, 0x75, 0xcf, - 0xd5, 0x74, 0x91, 0xea, 0x75, 0xb9, 0x57, 0x88, 0x28, 0xb9, 0x9e, 0xf6, 0x78, 0x45, 0xd0, 0x9b, - 0x46, 0x17, 0x17, 0x7f, 0x1e, 0x81, 0xa4, 0xd8, 0x11, 0x72, 0x00, 0x84, 0x78, 0x4c, 0x83, 0x27, - 0x13, 0x99, 0xea, 0x47, 0xc7, 0x83, 0x52, 0xaa, 0xc6, 0xa8, 0xcd, 0x3a, 0x79, 0x36, 0x28, 0x7d, - 0xe7, 0x79, 0x9d, 0x96, 0x0f, 0xa2, 0xa4, 0xf8, 0x24, 0x4d, 0x83, 0x5d, 0x02, 0x1d, 0x68, 0x44, - 0x3d, 0x30, 0x89, 0x67, 0xef, 0xbb, 0x5a, 0x8f, 0x1d, 0xae, 0xa4, 0xcc, 0x1c, 0x68, 0x64, 0xcd, - 0xa7, 0xa1, 0x22, 0x4d, 0x07, 0x1e, 0xf2, 0x47, 0x5e, 0x6e, 0x52, 0x41, 0x1b, 0x2d, 0xc1, 0xb5, - 0x60, 0xb0, 0x4a, 0x37, 0xdd, 0xe9, 0xeb, 0x87, 0x98, 0x45, 0x02, 0xea, 0xb3, 0x66, 0x83, 0xce, - 0x0d, 0xed, 0x71, 0x95, 0x77, 0x95, 0xaf, 0xc1, 0x6c, 0xe8, 0x58, 0x83, 0x14, 0xeb, 0xbf, 0x01, - 0x92, 0xdb, 0xda, 0x51, 0xd7, 0xd6, 0x0c, 0xb4, 0x00, 0x69, 0xff, 0x71, 0x96, 0x26, 0xa9, 0x5c, - 0x0f, 0xc3, 0x24, 0x64, 0x42, 0xb6, 0x4f, 0xb0, 0x4b, 0xcf, 0x44, 0x65, 0x5f, 0xd4, 0x71, 0x5f, - 0x55, 0xad, 0x3e, 0x1b, 0x94, 0xee, 0x4e, 0x26, 0x22, 0xac, 0xf7, 0x5d, 0xd3, 0x3b, 0xaa, 0xb4, - 0x3f, 0xba, 0xb7, 0x2b, 0xa0, 0xa8, 0x21, 0xd9, 0x4a, 0xa6, 0x1f, 0x6e, 0x8a, 0xa7, 0x6e, 0xba, - 0x5d, 0xb5, 0x67, 0xea, 0xae, 0x4d, 0xfc, 0x1b, 0x51, 0x41, 0xdd, 0x60, 0x44, 0xf4, 0x1a, 0xe4, - 0xf6, 0x4c, 0x8b, 0xdd, 0xc6, 0xfb, 0x7c, 0xfc, 0x32, 0x34, 0xeb, 0x93, 0x05, 0xe3, 0x43, 0xc8, - 0x86, 0x9e, 0xb7, 0xe9, 0x51, 0x27, 0xd8, 0x51, 0x6f, 0x1d, 0x0f, 0x4a, 0x99, 0xa1, 0xe9, 0xf0, - 0xe3, 0xbe, 0x4a, 0x8c, 0xca, 0x0c, 0xa7, 0xa1, 0x87, 0x3d, 0x07, 0xd3, 0xec, 0x33, 0x49, 0xfe, - 0x45, 0x8b, 0xc2, 0x1b, 0xe8, 0x1d, 0x98, 0xee, 0x62, 0x8d, 0x60, 0xf1, 0xb1, 0xca, 0xc2, 0x39, - 0xc6, 0xc8, 0x3e, 0x58, 0x54, 0x38, 0x3b, 0xaa, 0x42, 0x82, 0xbf, 0xaf, 0xb0, 0x57, 0x91, 0xf4, - 0xd2, 0xcd, 0x49, 0xbf, 0x4b, 0x5a, 0x9b, 0x52, 0xc4, 0x48, 0xd4, 0x80, 0xa4, 0xcb, 0x9f, 0xd4, - 0xd8, 0x5b, 0xc9, 0x85, 0xe5, 0x6a, 0xe8, 0xc5, 0x6e, 0x6d, 0x4a, 0xf1, 0xc7, 0xa2, 0x1d, 0xff, - 0x5d, 0x9b, 0x7b, 0x75, 0xf1, 0x6c, 0x5f, 0x99, 0x30, 0x21, 0x1b, 0x02, 0x8e, 0xa0, 0xd0, 0x0d, - 0x9a, 0xec, 0x7e, 0x95, 0x3d, 0xb9, 0x9c, 0xbf, 0xc1, 0x91, 0xb7, 0x3a, 0xba, 0x41, 0x3e, 0x12, - 0x6d, 0x02, 0xe8, 0x41, 0xa4, 0x61, 0x8f, 0x31, 0xe9, 0xa5, 0x37, 0x2e, 0x93, 0xcd, 0xac, 0x4d, - 0x29, 0x21, 0x04, 0xf4, 0x11, 0xa4, 0xf5, 0xa1, 0xe9, 0x14, 0x72, 0x0c, 0xf0, 0xcd, 0x4b, 0xf9, - 0xcf, 0x35, 0xea, 0x33, 0x87, 0xd4, 0x51, 0x9f, 0x29, 0x8f, 0xfb, 0xcc, 0x06, 0x64, 0xc4, 0xcd, - 0x02, 0xff, 0xc2, 0xb6, 0x90, 0x67, 0x2e, 0x3b, 0xac, 0x25, 0xfe, 0x37, 0xb8, 0x95, 0x86, 0xa5, - 0xdb, 0x06, 0x36, 0x1a, 0xb4, 0xad, 0x88, 0xab, 0x4a, 0xd6, 0x20, 0x68, 0x15, 0xb2, 0x7a, 0x17, - 0x6b, 0x56, 0xdf, 0xf1, 0x71, 0xd0, 0x84, 0x38, 0x19, 0x31, 0x4e, 0x00, 0x6d, 0x02, 0xda, 0x63, - 0x6f, 0xe1, 0xe1, 0x55, 0xb1, 0x37, 0x9d, 0x49, 0xc0, 0x64, 0x36, 0x56, 0x19, 0xae, 0x0c, 0xbd, - 0x0c, 0x19, 0xcb, 0xb6, 0x74, 0xcd, 0xd2, 0x71, 0x97, 0x45, 0x37, 0xfe, 0x0c, 0x34, 0x4a, 0x44, - 0x9f, 0x42, 0x96, 0x8c, 0xa4, 0xf0, 0x85, 0x6b, 0x6c, 0xc6, 0xb7, 0x2e, 0x7b, 0x1f, 0xb8, 0x36, - 0xa5, 0x8c, 0x21, 0xa1, 0x5f, 0x07, 0xd9, 0x1b, 0xbb, 0x03, 0x66, 0x0f, 0x4a, 0xe7, 0x7f, 0x77, - 0x72, 0xc6, 0xa5, 0xf5, 0xda, 0x94, 0x72, 0x02, 0xad, 0x9a, 0x82, 0xa4, 0xb8, 0x31, 0x0f, 0xde, - 0xa1, 0x92, 0xb2, 0x54, 0xfe, 0x79, 0x02, 0xa4, 0x20, 0xf9, 0x59, 0x04, 0x14, 0x84, 0xab, 0xe1, - 0xc7, 0x3f, 0xd4, 0xef, 0x46, 0xd7, 0xa6, 0x94, 0xbc, 0xdf, 0x37, 0xfc, 0xfe, 0xe7, 0x35, 0xc8, - 0xf5, 0x6c, 0xc3, 0xdc, 0x33, 0x87, 0xde, 0x8e, 0xdf, 0x56, 0x65, 0x7d, 0xb2, 0xf0, 0x76, 0x77, - 0x47, 0xde, 0xc8, 0x63, 0x13, 0x84, 0xde, 0xb5, 0xa9, 0xd0, 0x23, 0x3a, 0xf5, 0xbe, 0x6e, 0xdf, - 0xb2, 0x4c, 0x6b, 0x5f, 0x15, 0xf5, 0x16, 0x0f, 0xec, 0x19, 0x41, 0x15, 0x25, 0x53, 0x6d, 0xcc, - 0x1d, 0xdd, 0xba, 0xd0, 0x1d, 0xf9, 0x7b, 0x5f, 0x8b, 0x04, 0xfe, 0x68, 0x65, 0xdc, 0x1f, 0xdd, - 0xbe, 0xd8, 0x1f, 0x85, 0x60, 0x02, 0x87, 0xb4, 0x7b, 0xaa, 0x43, 0x5a, 0x9c, 0x50, 0x5b, 0x42, - 0x88, 0xa3, 0x1e, 0xa9, 0x36, 0xe6, 0x91, 0x6e, 0x5d, 0xe8, 0x91, 0xc2, 0x7b, 0x14, 0x2e, 0x69, - 0xeb, 0x14, 0x97, 0xf4, 0xe6, 0x44, 0x2e, 0x29, 0x04, 0x16, 0xf6, 0x49, 0xca, 0x69, 0x3e, 0xa9, - 0x32, 0x99, 0x4f, 0x0a, 0x41, 0x8e, 0x38, 0xa5, 0xef, 0x9d, 0x30, 0x38, 0x99, 0xc1, 0x7e, 0xf3, - 0xd2, 0x45, 0xf6, 0x5a, 0xe4, 0x84, 0xc5, 0x69, 0xa7, 0x58, 0x5c, 0x9e, 0xc1, 0xbf, 0x7d, 0x09, - 0x8b, 0x0b, 0x4d, 0x70, 0xd2, 0xe4, 0x00, 0x24, 0xff, 0x2d, 0x2c, 0x64, 0x7e, 0xe5, 0x1f, 0x45, - 0x20, 0xd6, 0xb2, 0x3b, 0x28, 0x3b, 0xbc, 0x91, 0x60, 0x77, 0x09, 0x1f, 0x0e, 0xd9, 0x45, 0x7a, - 0xfd, 0xd2, 0x39, 0x2b, 0xf1, 0x67, 0x56, 0x82, 0x41, 0xe8, 0x7d, 0x48, 0x3a, 0x3c, 0x75, 0x12, - 0x16, 0x56, 0x3e, 0x6f, 0x3c, 0xe7, 0x54, 0xfc, 0x21, 0xb7, 0x6f, 0x85, 0xbf, 0x70, 0xdf, 0xb0, - 0x0d, 0x8c, 0xb2, 0x00, 0xdb, 0x1a, 0x21, 0xce, 0x81, 0xab, 0x11, 0x2c, 0x4f, 0xa1, 0x24, 0xc4, - 0xd6, 0x37, 0xda, 0x72, 0xe4, 0xf6, 0x77, 0xc3, 0x77, 0x14, 0x75, 0x65, 0xb9, 0xb9, 0xd9, 0xdc, - 0x5c, 0x55, 0x37, 0x97, 0x37, 0x1a, 0x6d, 0x79, 0x0a, 0x15, 0x60, 0xee, 0xe3, 0xe5, 0xe6, 0x8e, - 0xb8, 0xb4, 0x50, 0x9b, 0x9b, 0x3b, 0x0d, 0xe5, 0xfe, 0xf2, 0x3d, 0x39, 0x82, 0xae, 0x03, 0x52, - 0xb6, 0x6a, 0xeb, 0xed, 0x7a, 0x55, 0xad, 0x6d, 0x6d, 0x6c, 0x2f, 0xd7, 0x76, 0x9a, 0x5b, 0x9b, - 0x72, 0x14, 0x49, 0x10, 0xaf, 0x6f, 0x6d, 0x36, 0x64, 0xb8, 0xfd, 0xa3, 0x18, 0xc4, 0xa9, 0x8c, - 0xd1, 0xcb, 0x90, 0xde, 0xdd, 0x6c, 0x6f, 0x37, 0x6a, 0xcd, 0x95, 0x66, 0xa3, 0x2e, 0x4f, 0x15, - 0x67, 0x9f, 0x3c, 0x5d, 0xc8, 0xd1, 0xae, 0x5d, 0x8b, 0x38, 0x58, 0x67, 0xce, 0x05, 0x15, 0x21, - 0x51, 0x5d, 0xae, 0xad, 0xef, 0x6e, 0xcb, 0x91, 0x62, 0xf6, 0xc9, 0xd3, 0x05, 0xa0, 0x0c, 0xdc, - 0xb0, 0xd1, 0x0b, 0x90, 0x54, 0x1a, 0xed, 0x9d, 0x2d, 0xa5, 0x21, 0x47, 0x8b, 0xb9, 0x27, 0x4f, - 0x17, 0xd2, 0xb4, 0x53, 0xd8, 0x2b, 0x7a, 0x0d, 0x32, 0xed, 0xda, 0x5a, 0x63, 0x63, 0x59, 0xad, - 0xad, 0x2d, 0x6f, 0xae, 0x36, 0xe4, 0x58, 0x71, 0xee, 0xc9, 0xd3, 0x05, 0x79, 0xfc, 0x80, 0xe9, - 0x14, 0xcd, 0x8d, 0xed, 0x2d, 0x65, 0x47, 0x8e, 0x0f, 0xa7, 0xe0, 0x76, 0x85, 0xca, 0x00, 0x7c, - 0xf4, 0x4a, 0xa3, 0x51, 0x97, 0xa7, 0x8b, 0xe8, 0xc9, 0xd3, 0x85, 0x2c, 0xed, 0x1f, 0x9a, 0x0b, - 0x7a, 0x05, 0x66, 0x6a, 0x4a, 0x63, 0x79, 0xa7, 0xa1, 0xb6, 0x77, 0x96, 0x77, 0xda, 0x72, 0x62, - 0xb8, 0x93, 0x90, 0x09, 0xa0, 0x0a, 0xe4, 0x97, 0x77, 0x77, 0xb6, 0xd4, 0x11, 0xde, 0x64, 0xf1, - 0xc6, 0x93, 0xa7, 0x0b, 0xb3, 0x94, 0x77, 0xb9, 0xef, 0xd9, 0x61, 0xfe, 0x37, 0x40, 0x1e, 0x59, - 0xbf, 0xba, 0x5a, 0x93, 0xa5, 0xe2, 0xf5, 0x27, 0x4f, 0x17, 0xd0, 0xf8, 0x16, 0x56, 0x6b, 0xe8, - 0x57, 0xe0, 0xfa, 0xce, 0x27, 0xdb, 0x8d, 0x7a, 0xa3, 0x5d, 0x53, 0x47, 0xb7, 0x9d, 0x2a, 0x16, - 0x9e, 0x3c, 0x5d, 0x98, 0xa3, 0x63, 0xc6, 0xc7, 0x15, 0xa5, 0x1f, 0xfe, 0xe5, 0xfc, 0xd4, 0x5f, - 0xff, 0xd5, 0xfc, 0x54, 0xf5, 0xe6, 0x17, 0xff, 0x39, 0x3f, 0xf5, 0xc5, 0xf1, 0x7c, 0xe4, 0x27, - 0xc7, 0xf3, 0x91, 0x9f, 0x1e, 0xcf, 0x47, 0xfe, 0xe3, 0x78, 0x3e, 0xf2, 0x87, 0x5f, 0xce, 0x4f, - 0xfd, 0xe4, 0xcb, 0xf9, 0xa9, 0x9f, 0x7e, 0x39, 0x3f, 0xf5, 0x69, 0x82, 0xab, 0x55, 0x27, 0xc1, - 0xd2, 0xf0, 0xb7, 0xff, 0x2f, 0x00, 0x00, 0xff, 0xff, 0x90, 0x45, 0x7d, 0x1b, 0xc8, 0x33, 0x00, - 0x00, +func init() { proto.RegisterFile("jobs/jobspb/jobs.proto", fileDescriptor_jobs_8868ebe50bd1b6b1) } + +var fileDescriptor_jobs_8868ebe50bd1b6b1 = []byte{ + // 4452 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5b, 0x4b, 0x6c, 0x23, 0x47, + 0x7a, 0x56, 0x93, 0x14, 0xd9, 0xfc, 0x29, 0x52, 0xcd, 0x92, 0x66, 0x86, 0xcb, 0xd8, 0xa2, 0xc2, + 0xf5, 0x63, 0x66, 0x6c, 0x53, 0x5e, 0x39, 0xeb, 0xb5, 0x27, 0xf6, 0x78, 0xf9, 0x1a, 0x89, 0xd4, + 0xe8, 0xe1, 0xa6, 0x34, 0x5e, 0x7b, 0xe1, 0x74, 0x5a, 0xdd, 0x25, 0xa9, 0xa3, 0x66, 0x37, 0xa7, + 0xab, 0x39, 0x33, 0xda, 0x0d, 0x92, 0x60, 0x83, 0x04, 0x8b, 0x39, 0x25, 0x40, 0x36, 0x87, 0x24, + 0x03, 0x04, 0x48, 0x16, 0xc8, 0x21, 0x40, 0x00, 0x23, 0x48, 0x72, 0xc8, 0x6d, 0x2f, 0x3e, 0x24, + 0xc0, 0x5e, 0x02, 0x2c, 0x72, 0xe0, 0x26, 0xf2, 0x25, 0x97, 0x00, 0x8b, 0xe4, 0x36, 0xa7, 0xa0, + 0x1e, 0xdd, 0x6c, 0x52, 0x1a, 0x89, 0x1a, 0xd9, 0x9b, 0x8b, 0x86, 0xf5, 0xd7, 0x5f, 0x5f, 0x55, + 0xfd, 0xf5, 0x3f, 0xab, 0x7a, 0xe0, 0xea, 0x6f, 0xb9, 0xbb, 0x64, 0x89, 0xfe, 0xe9, 0xed, 0xb2, + 0x7f, 0x2a, 0x3d, 0xcf, 0xf5, 0x5d, 0xf4, 0x35, 0xc3, 0x35, 0x0e, 0x3d, 0x57, 0x37, 0x0e, 0x2a, + 0xe4, 0xbe, 0x5d, 0x61, 0x3d, 0x9c, 0xab, 0x78, 0x05, 0x7b, 0x9e, 0xeb, 0x51, 0x7e, 0xfe, 0x83, + 0x8f, 0x28, 0xce, 0xef, 0xbb, 0xfb, 0x2e, 0xfb, 0xb9, 0x44, 0x7f, 0x09, 0x2a, 0x62, 0x18, 0xbd, + 0xdd, 0x25, 0x53, 0xf7, 0x75, 0x41, 0x2b, 0x04, 0x34, 0xcb, 0x7d, 0x63, 0xcf, 0xf5, 0xba, 0xba, + 0x1f, 0x60, 0x7c, 0x9d, 0xdc, 0xb7, 0x97, 0x0c, 0xdd, 0xd7, 0x6d, 0x77, 0x7f, 0xc9, 0xc4, 0xc4, + 0xe8, 0xed, 0x2e, 0x11, 0xdf, 0xeb, 0x1b, 0x7e, 0xdf, 0xc3, 0xa6, 0x60, 0x2a, 0x9d, 0xc2, 0xe4, + 0x63, 0x47, 0x77, 0xfc, 0x00, 0xbf, 0xef, 0x5b, 0xf6, 0xd2, 0x81, 0x6d, 0x2c, 0xf9, 0x56, 0x17, + 0x13, 0x5f, 0xef, 0xf6, 0x78, 0x4f, 0xf9, 0x77, 0x61, 0xfa, 0x2e, 0xd6, 0x09, 0x46, 0x9f, 0x40, + 0xca, 0x71, 0x4d, 0xac, 0x59, 0x66, 0x41, 0x5a, 0x94, 0xae, 0x67, 0x6b, 0xd5, 0xe3, 0x41, 0x29, + 0xb9, 0xe1, 0x9a, 0xb8, 0xd5, 0x78, 0x3a, 0x28, 0xbd, 0xb5, 0x6f, 0xf9, 0x07, 0xfd, 0xdd, 0x8a, + 0xe1, 0x76, 0x97, 0x42, 0x41, 0x98, 0xbb, 0xc3, 0xdf, 0x4b, 0xbd, 0xc3, 0xfd, 0x25, 0xb1, 0x8d, + 0x0a, 0x1f, 0xa6, 0x26, 0x29, 0x62, 0xcb, 0x44, 0xf3, 0x30, 0x8d, 0x7b, 0xae, 0x71, 0x50, 0x88, + 0x2d, 0x4a, 0xd7, 0xe3, 0x2a, 0x6f, 0xdc, 0x4a, 0xfc, 0xd7, 0x5f, 0x96, 0xa4, 0xf2, 0x8f, 0x63, + 0x70, 0xad, 0xa6, 0x1b, 0x87, 0xfd, 0x5e, 0xd3, 0x31, 0xbc, 0xa3, 0x9e, 0x6f, 0xb9, 0xce, 0x26, + 0xfb, 0x4b, 0x90, 0x02, 0xf1, 0x43, 0x7c, 0xc4, 0xd6, 0x33, 0xa3, 0xd2, 0x9f, 0xe8, 0x7d, 0x48, + 0x74, 0x5d, 0x13, 0x33, 0xa0, 0xdc, 0xf2, 0x8d, 0xca, 0x33, 0xcf, 0xa4, 0x32, 0x44, 0x5b, 0x77, + 0x4d, 0xac, 0xb2, 0x61, 0x68, 0x17, 0xe4, 0xc3, 0x2e, 0xd1, 0x2c, 0x67, 0xcf, 0x2d, 0xc4, 0x17, + 0xa5, 0xeb, 0x99, 0xe5, 0x5b, 0x67, 0x40, 0x3c, 0x63, 0x59, 0x95, 0xb5, 0xf5, 0x4e, 0xcb, 0xd9, + 0x73, 0x6b, 0x99, 0xe3, 0x41, 0x29, 0x25, 0x1a, 0x6a, 0xea, 0xb0, 0x4b, 0xe8, 0x8f, 0xe2, 0x26, + 0x04, 0x34, 0xba, 0xfe, 0xbe, 0x67, 0xb1, 0xf5, 0xa7, 0x55, 0xfa, 0x13, 0xbd, 0x0e, 0x08, 0x73, + 0x3c, 0x6c, 0x6a, 0x54, 0x01, 0x34, 0xba, 0xc1, 0x18, 0xdb, 0xa0, 0x12, 0xf6, 0x34, 0x74, 0x5f, + 0x5f, 0xc3, 0x47, 0x5c, 0x42, 0x42, 0x4e, 0xbf, 0x17, 0x87, 0xdc, 0x70, 0x29, 0x0c, 0x7e, 0x15, + 0x92, 0xc4, 0x38, 0xc0, 0x5d, 0xcc, 0x66, 0xc8, 0x2d, 0xbf, 0x39, 0x91, 0x38, 0xe8, 0xd0, 0x4a, + 0x87, 0x8d, 0x53, 0xc5, 0x78, 0x84, 0x20, 0x41, 0x74, 0xdb, 0x17, 0x0b, 0x61, 0xbf, 0xd1, 0x9f, + 0x4b, 0xb0, 0x38, 0xbe, 0xa2, 0xda, 0xd1, 0xda, 0x7a, 0x67, 0x5d, 0x27, 0x3e, 0xf6, 0xd6, 0xf0, + 0x51, 0xab, 0x51, 0x88, 0x2f, 0xc6, 0xaf, 0x67, 0x96, 0x37, 0x27, 0x9f, 0xb8, 0x79, 0x0e, 0x62, + 0xd3, 0xf1, 0xbd, 0x23, 0xf5, 0xdc, 0x89, 0x8b, 0x1d, 0x78, 0x79, 0x22, 0xa8, 0xa8, 0x0e, 0xa5, + 0xb9, 0x0e, 0xcd, 0xc3, 0xf4, 0x03, 0xdd, 0xee, 0x63, 0xb1, 0x5b, 0xde, 0xb8, 0x15, 0x7b, 0x47, + 0x2a, 0x5f, 0x83, 0x24, 0x17, 0x0c, 0xca, 0x42, 0xba, 0xda, 0xec, 0x2c, 0x7f, 0xf3, 0xed, 0x95, + 0xfa, 0xba, 0x32, 0x25, 0x8e, 0xe0, 0x47, 0x12, 0x5c, 0xed, 0xf8, 0x1e, 0xd6, 0xbb, 0x2d, 0x67, + 0x1f, 0x13, 0xba, 0xa7, 0x06, 0xf6, 0x75, 0xcb, 0x26, 0xe8, 0xfb, 0x90, 0x23, 0xac, 0x47, 0xd3, + 0x4d, 0xd3, 0xc3, 0x84, 0xf0, 0x09, 0x6b, 0xdb, 0x4f, 0x07, 0xa5, 0xad, 0x89, 0x4c, 0xc7, 0x30, + 0xec, 0x25, 0x0e, 0x61, 0x39, 0xfb, 0xc3, 0x86, 0x61, 0x5b, 0xd8, 0xf1, 0x2b, 0x7c, 0xda, 0x2a, + 0xc7, 0x56, 0xb3, 0x24, 0xda, 0x2c, 0xff, 0x73, 0x12, 0xb2, 0x5c, 0x57, 0x83, 0xe5, 0xd4, 0x00, + 0x88, 0xaf, 0x7b, 0xbe, 0x46, 0xcd, 0x9d, 0x2d, 0x25, 0xb3, 0xfc, 0x62, 0xe4, 0x90, 0xa8, 0x3b, + 0xa8, 0x1c, 0xd8, 0x46, 0x65, 0x3b, 0x70, 0x07, 0xb5, 0xc4, 0xe7, 0x83, 0xd2, 0x94, 0x9a, 0x66, + 0xc3, 0x28, 0x15, 0xdd, 0x06, 0x19, 0x3b, 0x26, 0x47, 0x88, 0x4d, 0x8e, 0x90, 0xc2, 0x8e, 0xc9, + 0xc6, 0x7f, 0x8d, 0x2b, 0x7f, 0x9c, 0xc9, 0x21, 0x75, 0x3c, 0x28, 0xc5, 0x77, 0xd4, 0x16, 0xb7, + 0x82, 0xf7, 0xa0, 0x68, 0xe2, 0x9e, 0x87, 0x0d, 0x9d, 0x9a, 0xc1, 0x2e, 0x5b, 0xba, 0xd6, 0xd5, + 0x1d, 0x6b, 0x0f, 0x13, 0xbf, 0x90, 0x60, 0xc7, 0x52, 0x18, 0x72, 0xf0, 0xbd, 0xad, 0x8b, 0x7e, + 0xf4, 0xfb, 0x12, 0xcc, 0xf5, 0x3d, 0x8b, 0x68, 0xbb, 0x47, 0x9a, 0xed, 0x1a, 0xba, 0x6d, 0xf9, + 0x47, 0xda, 0xe1, 0x83, 0xc2, 0x34, 0xd3, 0xc5, 0xdb, 0xe7, 0x1a, 0xb4, 0x10, 0x52, 0x65, 0xc7, + 0xb3, 0x48, 0xed, 0xe8, 0xae, 0x40, 0x58, 0x7b, 0xc0, 0xf4, 0xa5, 0x36, 0x7f, 0x3c, 0x28, 0x29, + 0x3b, 0x6a, 0x2b, 0xda, 0x75, 0x4f, 0x55, 0xfa, 0x63, 0xcc, 0x48, 0x0f, 0x2d, 0xd9, 0x72, 0x1d, + 0xcd, 0xe5, 0xae, 0xa1, 0x90, 0x64, 0x82, 0x5a, 0xbe, 0xb8, 0x53, 0x51, 0xf3, 0xf8, 0x84, 0xfb, + 0xfb, 0x63, 0x09, 0x8a, 0xd4, 0x4b, 0x63, 0x83, 0x8a, 0x29, 0xf4, 0xdc, 0x9a, 0x87, 0x0d, 0xd7, + 0x33, 0x0b, 0x29, 0x2a, 0xa7, 0x5a, 0xe7, 0xdf, 0x27, 0x75, 0xce, 0x2c, 0x06, 0xf4, 0xfb, 0x96, + 0x59, 0xd9, 0xd9, 0x69, 0x35, 0x8e, 0x07, 0xa5, 0xc2, 0x56, 0x00, 0x1e, 0x1e, 0xa2, 0xca, 0xa0, + 0xd5, 0x42, 0xef, 0x19, 0x3d, 0xe8, 0x1d, 0xc8, 0x19, 0xae, 0x6d, 0x63, 0x83, 0x6d, 0x7b, 0x47, + 0x6d, 0x15, 0x64, 0x76, 0xc0, 0xf9, 0xe3, 0x41, 0x29, 0x5b, 0x0f, 0x7b, 0xe8, 0x51, 0x67, 0x8d, + 0x68, 0x13, 0xa9, 0x30, 0x1b, 0x11, 0x18, 0x73, 0xc1, 0x69, 0x26, 0xad, 0x1b, 0x13, 0x7b, 0x0f, + 0x35, 0x87, 0x47, 0xda, 0xc5, 0x3a, 0x5c, 0x39, 0xf5, 0x14, 0xcf, 0xb3, 0xfa, 0x74, 0xd4, 0xea, + 0x15, 0xc8, 0xf1, 0x43, 0xd9, 0xf2, 0xdc, 0x7d, 0x66, 0x50, 0x9f, 0xe5, 0x20, 0xa7, 0x62, 0xe2, + 0xbb, 0x1e, 0x0e, 0x2c, 0xea, 0x33, 0x09, 0xe6, 0x68, 0x64, 0xf5, 0xac, 0x9e, 0xef, 0x7a, 0x9a, + 0x87, 0x1f, 0x7a, 0x96, 0x8f, 0x49, 0x21, 0xc6, 0x94, 0xae, 0x7a, 0xc6, 0x16, 0x46, 0x81, 0x2a, + 0x8d, 0x10, 0x44, 0x15, 0x18, 0x5c, 0xef, 0x6e, 0xff, 0xe0, 0xe7, 0xa5, 0x5b, 0x13, 0x9d, 0xe3, + 0xc9, 0x60, 0x5f, 0x69, 0x35, 0x54, 0x64, 0x9e, 0x00, 0x46, 0x2f, 0x40, 0x82, 0xea, 0x2d, 0xf3, + 0xd2, 0xe9, 0x9a, 0x7c, 0x3c, 0x28, 0x25, 0xa8, 0x66, 0xab, 0x8c, 0x3a, 0x62, 0xe0, 0x89, 0xe7, + 0x30, 0xf0, 0x15, 0xc8, 0xf8, 0xfa, 0xae, 0x8d, 0x35, 0x3a, 0x33, 0x11, 0xe6, 0xf7, 0xca, 0x98, + 0x24, 0xc8, 0x7d, 0x7b, 0x57, 0x27, 0xb8, 0xb2, 0x4d, 0x39, 0x23, 0x7b, 0x07, 0x3f, 0x20, 0x10, + 0xb4, 0x04, 0x19, 0xf7, 0x01, 0xf6, 0x3c, 0xcb, 0xc4, 0x9a, 0xb9, 0xcb, 0x6c, 0x28, 0x5d, 0xcb, + 0x1d, 0x0f, 0x4a, 0xb0, 0x29, 0xc8, 0x8d, 0x9a, 0x0a, 0x01, 0x4b, 0x63, 0x17, 0xf9, 0x30, 0x2f, + 0x9c, 0x46, 0x68, 0xff, 0x4c, 0x9f, 0x52, 0x6c, 0x09, 0xef, 0x4d, 0x7e, 0x18, 0xfc, 0xdc, 0x03, + 0xe5, 0x61, 0x41, 0x9d, 0x6f, 0x12, 0xed, 0x9e, 0xe8, 0x41, 0xaf, 0x41, 0xbe, 0xe7, 0xe1, 0x9e, + 0xee, 0x61, 0xcd, 0x70, 0xbb, 0x3d, 0x1b, 0xfb, 0xd8, 0x64, 0xda, 0x2f, 0xab, 0x8a, 0xe8, 0xa8, + 0x07, 0x74, 0xf4, 0x32, 0x0d, 0x08, 0xba, 0x4f, 0x73, 0x0d, 0x82, 0x3d, 0xca, 0x99, 0x66, 0x9c, + 0x59, 0x46, 0x6d, 0x09, 0x22, 0x7a, 0x0b, 0xae, 0x0c, 0xcf, 0x8d, 0x68, 0xbd, 0xfe, 0xae, 0x6d, + 0x91, 0x03, 0x6c, 0x16, 0x80, 0x71, 0xcf, 0x47, 0x3a, 0xb7, 0x82, 0x3e, 0x74, 0x34, 0xa2, 0x8a, + 0x06, 0x15, 0x8c, 0xbe, 0x8f, 0x0b, 0x99, 0x45, 0xe9, 0xfa, 0x74, 0x6d, 0xf5, 0xe9, 0xa0, 0xd4, + 0x98, 0x58, 0x8f, 0x08, 0xee, 0x2e, 0xf9, 0x1e, 0xc6, 0x11, 0xb5, 0xac, 0x0b, 0xbc, 0xa8, 0x46, + 0x05, 0x34, 0xa4, 0x02, 0x0c, 0x4d, 0xb0, 0x30, 0xf3, 0xdc, 0xde, 0x2e, 0x82, 0x82, 0xaa, 0x90, + 0xe2, 0xc9, 0x2a, 0x29, 0x64, 0xd9, 0x01, 0xfe, 0xea, 0xb3, 0x74, 0x88, 0x71, 0x45, 0x4e, 0x29, + 0x18, 0x87, 0x1a, 0x00, 0xfe, 0x51, 0x2f, 0xd0, 0xc4, 0x1c, 0x43, 0x79, 0xf9, 0x59, 0x28, 0x47, + 0xbd, 0xa8, 0x22, 0xa6, 0x7d, 0xd1, 0x26, 0xa8, 0x0d, 0x33, 0x2c, 0x1f, 0xd2, 0x05, 0xce, 0x2c, + 0xc3, 0x79, 0xf5, 0x19, 0x38, 0x2c, 0x53, 0xd0, 0x23, 0x48, 0x19, 0x12, 0x52, 0x08, 0xda, 0x82, + 0x1c, 0x4d, 0xef, 0x28, 0xa7, 0x40, 0x53, 0x18, 0xda, 0x8d, 0x67, 0xa0, 0x35, 0x04, 0x73, 0x04, + 0x2f, 0x6b, 0x46, 0x68, 0xa4, 0xf8, 0xbf, 0x12, 0xe4, 0x4f, 0x38, 0x0f, 0xb4, 0x0d, 0xb1, 0x30, + 0x63, 0xa7, 0x3e, 0x3d, 0xc6, 0xb2, 0xf5, 0xcb, 0x38, 0x92, 0x98, 0x65, 0xa2, 0x7d, 0x48, 0x53, + 0x75, 0x76, 0x7c, 0x5a, 0x0e, 0xc4, 0x18, 0x78, 0xfb, 0x78, 0x50, 0x92, 0xb7, 0x18, 0xf1, 0xd2, + 0x53, 0xc8, 0x1c, 0xbc, 0x65, 0xa2, 0x12, 0x64, 0x7c, 0x57, 0xc3, 0x8f, 0x2c, 0xe2, 0x5b, 0xce, + 0x3e, 0x4b, 0x16, 0x64, 0x15, 0x7c, 0xb7, 0x29, 0x28, 0xc5, 0xbf, 0x88, 0x01, 0x3a, 0x69, 0xa5, + 0xe8, 0x9f, 0x24, 0x78, 0x21, 0xc8, 0x01, 0x5c, 0xcf, 0xda, 0xb7, 0x1c, 0xdd, 0x1e, 0x49, 0x06, + 0x24, 0x26, 0xed, 0x4f, 0x2e, 0xe3, 0x0a, 0x44, 0x82, 0xb0, 0x29, 0xe0, 0xc7, 0x13, 0x85, 0x17, + 0x68, 0x04, 0xe5, 0x89, 0xc2, 0x09, 0x96, 0x7b, 0x6a, 0xa1, 0xff, 0x8c, 0xc1, 0xc5, 0x35, 0x78, + 0xf1, 0x4c, 0xe0, 0x8b, 0xc4, 0xae, 0xe2, 0x0f, 0x24, 0xb8, 0xf6, 0x8c, 0x88, 0x12, 0xc5, 0xc9, + 0x72, 0x9c, 0x0f, 0xa3, 0x38, 0x99, 0xe5, 0x5f, 0xbf, 0x44, 0xd4, 0x8a, 0x2c, 0xa2, 0x9d, 0x90, + 0x25, 0x25, 0x56, 0x7e, 0x13, 0x66, 0xc5, 0xa0, 0x20, 0x8e, 0xa2, 0x17, 0x01, 0x0e, 0xac, 0xfd, + 0x03, 0xed, 0xa1, 0xee, 0x63, 0x4f, 0x94, 0x71, 0x69, 0x4a, 0xf9, 0x88, 0x12, 0xca, 0xff, 0x26, + 0x43, 0xb6, 0xd5, 0xed, 0xb9, 0x9e, 0x1f, 0x44, 0xd9, 0xbb, 0x90, 0x64, 0x71, 0x81, 0x88, 0xf3, + 0xab, 0x9c, 0xb1, 0xc2, 0x91, 0x91, 0x3c, 0xbe, 0x08, 0xb7, 0x20, 0x30, 0xc2, 0xf0, 0x17, 0x3b, + 0x35, 0xfc, 0xbd, 0x0f, 0x49, 0x5e, 0x6a, 0x8b, 0x4a, 0xb0, 0x14, 0x99, 0x2b, 0xa8, 0x63, 0x5b, + 0x9b, 0x77, 0x2c, 0x1b, 0xdf, 0x61, 0x6c, 0x01, 0x38, 0x1f, 0x84, 0x5e, 0x01, 0x99, 0x10, 0x5f, + 0x23, 0xd6, 0xf7, 0x78, 0xf4, 0x8c, 0xf3, 0x72, 0xb0, 0xd3, 0xd9, 0xee, 0x58, 0xdf, 0xc3, 0x6a, + 0x8a, 0x10, 0x9f, 0xfe, 0x40, 0x45, 0x90, 0x1f, 0xea, 0xb6, 0xcd, 0xa2, 0xec, 0x34, 0x2b, 0x7f, + 0xc3, 0xf6, 0xa8, 0x99, 0x25, 0xbf, 0x5a, 0x33, 0x13, 0x01, 0xb3, 0xa7, 0xfb, 0x07, 0x2c, 0x73, + 0x4c, 0xab, 0xc0, 0x49, 0x5b, 0xba, 0x7f, 0x80, 0x0a, 0x90, 0x22, 0x3a, 0x8d, 0x5d, 0xa4, 0x20, + 0x2f, 0xc6, 0xaf, 0xcf, 0xa8, 0x41, 0x13, 0x2d, 0x00, 0x8b, 0xbc, 0xbc, 0xc9, 0x82, 0x58, 0x5c, + 0x8d, 0x50, 0x98, 0x1c, 0x0e, 0xad, 0x9e, 0xb6, 0x77, 0x48, 0x78, 0xd0, 0x12, 0x72, 0x38, 0xb4, + 0x7a, 0x77, 0xd6, 0x88, 0x9a, 0xa2, 0x9d, 0x77, 0x0e, 0x09, 0x7a, 0x15, 0x66, 0x2d, 0x56, 0x35, + 0x69, 0xa6, 0xe5, 0x61, 0xc3, 0xb7, 0x8f, 0x58, 0xc0, 0x92, 0xd5, 0x1c, 0x27, 0x37, 0x04, 0x15, + 0xdd, 0x00, 0x65, 0x3c, 0xcc, 0xb2, 0x40, 0x23, 0xab, 0xb3, 0x63, 0x51, 0x96, 0xb2, 0xf2, 0xa3, + 0x8e, 0x04, 0xce, 0x2c, 0x67, 0xe5, 0xf4, 0x61, 0xcc, 0xac, 0xc0, 0x5c, 0x4f, 0xf7, 0x08, 0xd6, + 0x76, 0xfb, 0x8e, 0x69, 0x63, 0x8d, 0xfb, 0xea, 0x42, 0x8e, 0x71, 0xe7, 0x59, 0x57, 0x8d, 0xf5, + 0x70, 0xb7, 0x7e, 0x5e, 0xee, 0x7d, 0xf5, 0xff, 0x21, 0xf7, 0x2e, 0xfe, 0x38, 0x06, 0xd3, 0x4c, + 0xcf, 0xd1, 0x2d, 0x48, 0xd0, 0x63, 0x16, 0x95, 0xdd, 0xa4, 0x39, 0x17, 0x1b, 0x43, 0x6b, 0x7d, + 0x47, 0xef, 0xe2, 0x02, 0x62, 0x4a, 0xc0, 0x7e, 0xa3, 0x6b, 0x90, 0x22, 0xf8, 0xbe, 0xf6, 0x40, + 0xb7, 0x0b, 0x73, 0xec, 0x84, 0x93, 0x04, 0xdf, 0xbf, 0xa7, 0xdb, 0xe8, 0x0a, 0x24, 0x2d, 0xa2, + 0x39, 0xf8, 0x61, 0x61, 0x9e, 0x49, 0x6a, 0xda, 0x22, 0x1b, 0xf8, 0x21, 0x73, 0xdb, 0xba, 0xb7, + 0x8f, 0x7d, 0xcd, 0x70, 0x6d, 0x52, 0xb8, 0x42, 0x0d, 0x8c, 0xa6, 0x74, 0x94, 0x54, 0x77, 0x6d, + 0x82, 0x7e, 0x05, 0xd2, 0x0f, 0x75, 0xa2, 0xe1, 0x6e, 0xcf, 0x3f, 0x62, 0xc2, 0x92, 0xa9, 0xda, + 0x93, 0x26, 0x6d, 0xb7, 0x13, 0x72, 0x4c, 0x89, 0xb7, 0x13, 0x72, 0x5c, 0x49, 0xb4, 0x13, 0x72, + 0x42, 0x99, 0x6e, 0x27, 0xe4, 0x69, 0x25, 0xd9, 0x4e, 0xc8, 0x49, 0x25, 0xd5, 0x4e, 0xc8, 0x29, + 0x45, 0x6e, 0x27, 0x64, 0x59, 0x49, 0xb7, 0x13, 0x72, 0x5a, 0x81, 0x76, 0x42, 0x06, 0x25, 0xd3, + 0x4e, 0xc8, 0x19, 0x65, 0xa6, 0x9d, 0x90, 0x67, 0x94, 0x6c, 0x3b, 0x21, 0x67, 0x95, 0x5c, 0x3b, + 0x21, 0xe7, 0x94, 0xd9, 0x76, 0x42, 0x9e, 0x55, 0x94, 0x76, 0x42, 0x56, 0x94, 0x7c, 0x3b, 0x21, + 0xe7, 0x15, 0x54, 0xfe, 0x4c, 0x02, 0xa5, 0x83, 0xef, 0xf7, 0xb1, 0x63, 0xe0, 0x7b, 0xba, 0x5d, + 0x3f, 0xe8, 0x3b, 0x87, 0xe8, 0x15, 0x98, 0x35, 0xe8, 0x0f, 0x8d, 0x17, 0xc6, 0x74, 0xab, 0x12, + 0xdb, 0x6a, 0x96, 0x91, 0x3b, 0x94, 0x4a, 0x77, 0xfc, 0x22, 0x80, 0xe0, 0xa3, 0x96, 0xcd, 0x2f, + 0xac, 0xd2, 0x9c, 0x85, 0x9a, 0xf3, 0x18, 0x8c, 0xe7, 0x3e, 0x64, 0xee, 0x63, 0x04, 0x46, 0x75, + 0x1f, 0xa2, 0x25, 0x98, 0x77, 0xf0, 0x23, 0x5f, 0x1b, 0x67, 0x66, 0xae, 0x42, 0xcd, 0xd3, 0xbe, + 0x7a, 0x74, 0x40, 0xf9, 0x5f, 0x63, 0x30, 0x1b, 0x2c, 0x3a, 0x70, 0x87, 0x7b, 0xa0, 0xd0, 0x63, + 0xb1, 0x4c, 0xcd, 0x77, 0x39, 0x52, 0xe0, 0x18, 0xdf, 0x3f, 0xc3, 0x31, 0x8e, 0xa1, 0xd0, 0x76, + 0xcb, 0xdc, 0x76, 0xd9, 0x74, 0x3c, 0x34, 0xa8, 0x59, 0x12, 0xa5, 0x15, 0x77, 0x20, 0x17, 0x0c, + 0xe2, 0x14, 0x54, 0x87, 0xe4, 0xc8, 0x7c, 0xaf, 0x4d, 0x30, 0x5f, 0x20, 0x6a, 0x55, 0x0c, 0x2d, + 0x7e, 0x1f, 0xd0, 0xc9, 0xb9, 0xa3, 0x61, 0x69, 0x9a, 0x87, 0xa5, 0xcd, 0xd1, 0xb0, 0xf4, 0xee, + 0xc5, 0xf6, 0x16, 0x59, 0x76, 0xb4, 0xaa, 0xfb, 0x49, 0x0c, 0x72, 0x3c, 0x44, 0x84, 0xe1, 0xe8, + 0x35, 0xc8, 0x33, 0xa7, 0x65, 0x39, 0xfb, 0x5a, 0x4f, 0x10, 0xd9, 0xfe, 0x62, 0xaa, 0x12, 0x74, + 0x84, 0xcc, 0x5f, 0x87, 0xac, 0x87, 0x75, 0x73, 0xc8, 0x18, 0x63, 0x8c, 0x33, 0x94, 0x18, 0x32, + 0xbd, 0x0c, 0x39, 0x16, 0x0d, 0x87, 0x5c, 0x71, 0xc6, 0x95, 0x65, 0xd4, 0x90, 0xad, 0x06, 0x59, + 0xd2, 0xd3, 0x9d, 0x21, 0x57, 0x82, 0x09, 0xf5, 0xda, 0x29, 0x11, 0xa7, 0xd3, 0xd3, 0x1d, 0x11, + 0x69, 0x66, 0xe8, 0x98, 0x68, 0x2c, 0xf5, 0x30, 0xe9, 0x77, 0xb1, 0xd6, 0x73, 0x79, 0xb1, 0x15, + 0x57, 0xd3, 0x9c, 0xb2, 0xe5, 0x12, 0xb4, 0xc3, 0x54, 0x85, 0xc9, 0x42, 0x33, 0xb9, 0x70, 0x0a, + 0x49, 0x36, 0xcb, 0xcd, 0xc9, 0xc5, 0xa9, 0xce, 0x92, 0x51, 0x42, 0xf9, 0xb7, 0xe1, 0x1a, 0xcd, + 0x97, 0xb9, 0x53, 0xac, 0x1f, 0xe8, 0xce, 0x7e, 0xa8, 0x9c, 0x3a, 0xa4, 0x58, 0xce, 0x1d, 0xa6, + 0x9f, 0xab, 0xc7, 0x83, 0x52, 0x92, 0x72, 0x5f, 0x3a, 0x70, 0x25, 0x29, 0x70, 0xcb, 0x2c, 0x17, + 0xa1, 0x30, 0x3e, 0x7b, 0x58, 0xa3, 0xab, 0xac, 0x44, 0xef, 0x77, 0x31, 0x95, 0xd8, 0x5d, 0x8b, + 0xf8, 0xe8, 0xdb, 0x30, 0x23, 0x24, 0x44, 0x05, 0x17, 0x68, 0xee, 0x39, 0x42, 0xce, 0x78, 0x21, + 0x08, 0x29, 0xff, 0xbd, 0x04, 0x73, 0x0d, 0xcf, 0xed, 0xf5, 0xb0, 0x29, 0x7c, 0x27, 0xdf, 0x6a, + 0xe0, 0x32, 0xa5, 0x88, 0xcb, 0xdc, 0x80, 0x58, 0xab, 0x21, 0x72, 0xe3, 0xdb, 0x97, 0x4d, 0xb9, + 0x5b, 0x0d, 0xf4, 0x2e, 0x24, 0x69, 0x69, 0xd8, 0x27, 0xcc, 0x9f, 0xe4, 0x4e, 0x14, 0x41, 0x23, + 0xc7, 0xc6, 0x18, 0x55, 0x31, 0xa0, 0xfc, 0x87, 0x49, 0xb8, 0x12, 0x95, 0xd1, 0x4a, 0x3d, 0x58, + 0xf8, 0xa7, 0x90, 0xb2, 0x1c, 0x13, 0x3f, 0xc2, 0x13, 0xf9, 0x8d, 0xd3, 0x20, 0x2a, 0x42, 0x1e, + 0x2d, 0x0a, 0x13, 0x94, 0x5d, 0x02, 0x13, 0x7d, 0x27, 0x4c, 0xd7, 0xf8, 0x35, 0xc8, 0xad, 0xe7, + 0x46, 0x6f, 0x8c, 0xa5, 0x6e, 0x23, 0x99, 0x11, 0x73, 0xb0, 0x5f, 0x51, 0x66, 0xd4, 0x81, 0xbc, + 0xe5, 0xf8, 0xd8, 0xb3, 0xb1, 0xfe, 0x80, 0x06, 0x7a, 0x3a, 0xbd, 0xb8, 0x0d, 0x99, 0x34, 0xac, + 0x2a, 0x11, 0x00, 0x1e, 0x9e, 0x3f, 0x85, 0xb9, 0x28, 0x68, 0x70, 0x04, 0x67, 0xdf, 0x90, 0x30, + 0x09, 0x0f, 0x61, 0x83, 0x8b, 0x88, 0x08, 0x50, 0x8b, 0xe3, 0x14, 0xff, 0x54, 0x82, 0x99, 0xe8, + 0xb1, 0x20, 0x0b, 0x64, 0x36, 0x47, 0x60, 0x8b, 0xf1, 0xda, 0x06, 0xcd, 0xc1, 0x58, 0x27, 0x93, + 0xd5, 0x07, 0xcf, 0x2d, 0x2b, 0x0e, 0x21, 0x8e, 0xbc, 0x65, 0xd2, 0xc0, 0x6e, 0x7a, 0x6e, 0x6f, + 0x78, 0x2d, 0x1c, 0x57, 0x65, 0x4a, 0xa0, 0xb9, 0x4a, 0xf1, 0x77, 0x20, 0x1d, 0x1e, 0x68, 0xa4, + 0x32, 0x8d, 0x7f, 0x89, 0x95, 0xe9, 0x59, 0xf3, 0x97, 0x7f, 0x9e, 0x84, 0xb9, 0xd3, 0x5c, 0xd5, + 0xc7, 0xa0, 0x44, 0x3c, 0x83, 0x66, 0x5b, 0xc4, 0x17, 0x1a, 0x7b, 0xe3, 0xec, 0x12, 0x28, 0xe2, + 0x5e, 0xc4, 0x79, 0xe4, 0xbc, 0x51, 0xa7, 0xf3, 0x5d, 0xc8, 0x99, 0x7c, 0xcb, 0x9a, 0x30, 0x85, + 0xf8, 0xb9, 0x95, 0xcb, 0x29, 0x2e, 0x46, 0xa0, 0x67, 0xcd, 0x48, 0x17, 0x61, 0x37, 0xdd, 0x01, + 0x7a, 0x78, 0x9b, 0x60, 0x99, 0x4c, 0x3f, 0xb3, 0xb5, 0xce, 0xf1, 0xa0, 0x94, 0x17, 0x58, 0xc1, + 0xf5, 0xc1, 0xa5, 0x65, 0x9c, 0x37, 0xc7, 0x00, 0x4d, 0xea, 0xe8, 0x69, 0x3f, 0x9d, 0x78, 0x7a, + 0xe8, 0xe8, 0xa9, 0xa6, 0x5e, 0xde, 0xd1, 0xd3, 0x9f, 0x2d, 0x13, 0xfd, 0x81, 0x04, 0x79, 0x7e, + 0x97, 0xd8, 0xed, 0xfb, 0x3a, 0xbf, 0x20, 0x0e, 0x2a, 0xa2, 0x8f, 0x8f, 0x07, 0xa5, 0x59, 0x26, + 0x90, 0x75, 0xd1, 0xc7, 0xa6, 0xad, 0x3d, 0xef, 0xb4, 0x43, 0x14, 0x51, 0x25, 0x84, 0x04, 0x13, + 0xad, 0x41, 0x8e, 0x97, 0x77, 0x1a, 0x2d, 0x70, 0x2c, 0xd7, 0x61, 0xa5, 0x52, 0xb6, 0xf6, 0xd2, + 0xd3, 0x41, 0x69, 0xf1, 0x14, 0xcd, 0xe2, 0x95, 0xe1, 0x3d, 0xce, 0xab, 0x66, 0xf7, 0xa2, 0x4d, + 0x64, 0x40, 0x36, 0x54, 0x8d, 0xa3, 0x9e, 0xa8, 0xac, 0x2e, 0x1f, 0x2c, 0x66, 0x02, 0x1d, 0xa1, + 0x98, 0x68, 0x1f, 0x66, 0x83, 0x49, 0x78, 0x49, 0x43, 0x0a, 0xe9, 0x2f, 0x65, 0x9a, 0x40, 0xad, + 0xf9, 0xae, 0x89, 0x28, 0xf2, 0xaf, 0xc2, 0xfc, 0xa9, 0xd1, 0xf8, 0xcf, 0xa6, 0xe1, 0xea, 0xa8, + 0x87, 0x0f, 0x13, 0x17, 0x6d, 0x3c, 0x06, 0x7d, 0x30, 0x71, 0x94, 0x08, 0x30, 0xb8, 0x1b, 0x0a, + 0x5a, 0xe3, 0x51, 0xe8, 0xd3, 0xb1, 0x28, 0xf4, 0x1c, 0xf8, 0x4c, 0xbd, 0xc6, 0xf0, 0x05, 0x68, + 0xf1, 0x5f, 0x24, 0xc8, 0x8e, 0xcc, 0xff, 0xcb, 0x74, 0xb7, 0x5b, 0x61, 0x56, 0xc0, 0x5f, 0xbc, + 0xdf, 0xb9, 0xf8, 0xde, 0x46, 0x93, 0x85, 0xe2, 0x3f, 0x4a, 0x90, 0x1d, 0xd9, 0xee, 0x57, 0xe4, + 0xa8, 0xbf, 0xf4, 0x95, 0x97, 0xbf, 0x05, 0x49, 0x4e, 0x41, 0x08, 0x72, 0x1f, 0x55, 0x5b, 0xdb, + 0xad, 0x8d, 0x15, 0xed, 0xce, 0xa6, 0xaa, 0xad, 0xd4, 0x95, 0x29, 0x34, 0x03, 0x72, 0xa3, 0x79, + 0xb7, 0x49, 0x89, 0x8a, 0x84, 0x32, 0x90, 0x62, 0xad, 0x66, 0x43, 0x89, 0x95, 0x6b, 0xa0, 0x70, + 0xec, 0x3d, 0x4c, 0x5d, 0x2b, 0x2d, 0x52, 0x51, 0x05, 0xe6, 0x28, 0x2c, 0xee, 0xd2, 0x1c, 0x83, + 0x06, 0x13, 0x2d, 0x92, 0xe1, 0xe5, 0xc3, 0x2e, 0x1a, 0x56, 0x36, 0xf4, 0x2e, 0x2e, 0xff, 0x43, + 0x02, 0xf2, 0x43, 0x90, 0x20, 0xb0, 0xbc, 0x02, 0x32, 0xb1, 0x9c, 0x43, 0x6d, 0xf8, 0xd0, 0xc9, + 0x2f, 0x3f, 0x2c, 0xe7, 0x70, 0x47, 0x6d, 0xa9, 0x29, 0xda, 0xb9, 0xe3, 0x59, 0xa8, 0x0d, 0x09, + 0xb7, 0xe7, 0x07, 0x79, 0xff, 0xdb, 0x67, 0x88, 0xe2, 0xc4, 0x1c, 0x95, 0xcd, 0x9e, 0x2f, 0xaa, + 0x36, 0x86, 0x81, 0xfe, 0x56, 0x82, 0x14, 0xaf, 0xb4, 0x83, 0x0c, 0xff, 0xdd, 0x0b, 0xe1, 0x71, + 0x01, 0x88, 0x57, 0xa7, 0x8f, 0xa8, 0xaa, 0x3f, 0x1d, 0x94, 0xf2, 0xe3, 0x02, 0x22, 0x97, 0x7c, + 0x8e, 0x0a, 0x96, 0x88, 0xda, 0xfc, 0x21, 0x64, 0x28, 0x68, 0xe6, 0x52, 0x27, 0x7c, 0x6b, 0xca, + 0x8e, 0x1c, 0x44, 0x71, 0x1f, 0x66, 0xa2, 0xab, 0x3f, 0xe5, 0x86, 0xb3, 0x3a, 0x5a, 0x4a, 0xbe, + 0x36, 0x91, 0x64, 0x38, 0x66, 0xf4, 0x5a, 0xf5, 0x5b, 0x90, 0x0e, 0xc5, 0x7e, 0x91, 0xfb, 0x58, + 0xee, 0x25, 0xc3, 0x0b, 0x8e, 0x69, 0x25, 0x59, 0xfe, 0x3b, 0x09, 0x66, 0x54, 0x4c, 0x5c, 0xfb, + 0x01, 0x36, 0x69, 0xd6, 0x80, 0xbe, 0x01, 0x09, 0x9a, 0x85, 0x88, 0xbb, 0x9b, 0x73, 0xca, 0x13, + 0xc6, 0x8a, 0xaa, 0x90, 0x0e, 0x6f, 0xa0, 0x2e, 0xf2, 0x16, 0x3f, 0x1c, 0x85, 0x6e, 0x80, 0xb2, + 0xeb, 0xf6, 0x1d, 0x53, 0xf7, 0x8e, 0x34, 0x0f, 0xeb, 0xc6, 0x01, 0x36, 0xc5, 0x6d, 0xfb, 0x6c, + 0x40, 0x57, 0x39, 0xb9, 0xfc, 0xc3, 0x18, 0xa0, 0xa1, 0x70, 0x22, 0x6e, 0x82, 0xe6, 0x3e, 0x6c, + 0x1f, 0xa2, 0xc0, 0x8a, 0x9d, 0xfa, 0x3e, 0x32, 0x96, 0x42, 0x85, 0x1b, 0x0f, 0x8e, 0xd4, 0x8b, + 0xd0, 0x08, 0xfa, 0xd1, 0xd9, 0xf7, 0x6c, 0x71, 0x76, 0xcf, 0xc6, 0xb4, 0xf4, 0x97, 0x7a, 0xd7, + 0x26, 0xc2, 0xdd, 0xff, 0x24, 0x00, 0xd5, 0x3d, 0xac, 0xfb, 0x98, 0x7a, 0x1e, 0x72, 0x56, 0x3d, + 0x58, 0x83, 0x69, 0x5e, 0x3c, 0xc4, 0x2e, 0x52, 0x3c, 0x08, 0xa1, 0xf0, 0xa1, 0xe8, 0x37, 0x60, + 0xc6, 0x70, 0xed, 0x7e, 0xd7, 0xd1, 0xd8, 0x2b, 0xa1, 0x48, 0x25, 0xbf, 0x79, 0x96, 0x12, 0x9f, + 0x58, 0x5c, 0xa5, 0xee, 0xda, 0xb4, 0x1d, 0xd4, 0xb7, 0x1c, 0x90, 0x71, 0xa0, 0x17, 0x20, 0x1d, + 0x1a, 0x14, 0x4b, 0x22, 0xd3, 0xea, 0x90, 0x80, 0x96, 0x61, 0x5a, 0x27, 0x9a, 0xbb, 0xc7, 0xb2, + 0xbc, 0xf3, 0x34, 0x4c, 0x4d, 0xe8, 0x64, 0x73, 0x0f, 0xbd, 0x05, 0xd9, 0xbd, 0xfb, 0x3c, 0xf3, + 0xe5, 0x0e, 0x94, 0x3f, 0xde, 0xce, 0x1e, 0x0f, 0x4a, 0x99, 0x3b, 0x1f, 0xb2, 0xcd, 0x52, 0xf7, + 0xa9, 0x66, 0xf6, 0xee, 0x87, 0x0d, 0x74, 0x13, 0xf2, 0x5d, 0xfd, 0x91, 0xb6, 0xe7, 0xe9, 0x86, + 0x48, 0xf5, 0x6c, 0xee, 0x15, 0x24, 0x75, 0xb6, 0xab, 0x3f, 0xba, 0x23, 0xe8, 0x2d, 0xd3, 0xc6, + 0xc5, 0x5f, 0x48, 0x90, 0x12, 0x3b, 0x42, 0x3d, 0x00, 0x21, 0x1e, 0xcb, 0xe4, 0xc9, 0x44, 0xb6, + 0xf6, 0xe1, 0xf1, 0xa0, 0x94, 0xae, 0x33, 0x6a, 0xab, 0x41, 0x9e, 0x0e, 0x4a, 0xdf, 0x7e, 0x5e, + 0xa7, 0x15, 0x80, 0xa8, 0x69, 0x3e, 0x49, 0xcb, 0x64, 0x97, 0x40, 0x07, 0x3a, 0xd1, 0x0e, 0x2c, + 0xe2, 0xbb, 0xfb, 0x9e, 0xde, 0x65, 0x87, 0x2b, 0xab, 0x33, 0x07, 0x3a, 0x59, 0x0d, 0x68, 0xa8, + 0x48, 0xd3, 0x81, 0x07, 0xfc, 0x91, 0x97, 0x9b, 0x54, 0xd8, 0x46, 0xcb, 0x70, 0x25, 0x1c, 0xac, + 0xd1, 0x4d, 0xef, 0xf6, 0x8d, 0x43, 0xcc, 0x22, 0x01, 0xf5, 0x59, 0x73, 0x61, 0xe7, 0xba, 0xfe, + 0xa8, 0xc6, 0xbb, 0xca, 0x57, 0x60, 0x2e, 0x72, 0xac, 0x61, 0x8a, 0xf5, 0xdf, 0x00, 0xa9, 0x2d, + 0xfd, 0xc8, 0x76, 0x75, 0x13, 0x2d, 0x42, 0x26, 0x78, 0x9c, 0xa5, 0x49, 0x2a, 0xd7, 0xc3, 0x28, + 0x09, 0x59, 0x90, 0xeb, 0x13, 0xec, 0xd1, 0x33, 0xd1, 0xd8, 0x97, 0x7e, 0xdc, 0x57, 0xd5, 0x6a, + 0x4f, 0x07, 0xa5, 0xdb, 0x93, 0x89, 0x08, 0x1b, 0x7d, 0xcf, 0xf2, 0x8f, 0x2a, 0x9d, 0x0f, 0xef, + 0xee, 0x08, 0x28, 0x6a, 0x48, 0xae, 0x9a, 0xed, 0x47, 0x9b, 0xe2, 0xa9, 0x9b, 0x6e, 0x57, 0xeb, + 0x5a, 0x86, 0xe7, 0x92, 0xe0, 0x46, 0x54, 0x50, 0xd7, 0x19, 0x11, 0xbd, 0x0a, 0xb3, 0x7b, 0x96, + 0xc3, 0x6e, 0xe3, 0x03, 0x3e, 0x7e, 0x19, 0x9a, 0x0b, 0xc8, 0x82, 0xf1, 0x01, 0xe4, 0x22, 0xcf, + 0xdb, 0xf4, 0xa8, 0x93, 0xec, 0xa8, 0x37, 0x8f, 0x07, 0xa5, 0xec, 0xd0, 0x74, 0xf8, 0x71, 0x5f, + 0x26, 0x46, 0x65, 0x87, 0xd3, 0xd0, 0xc3, 0x9e, 0x87, 0x69, 0xf6, 0xf9, 0x26, 0xff, 0xa2, 0x45, + 0xe5, 0x0d, 0xf4, 0x36, 0x4c, 0xdb, 0x58, 0x27, 0x58, 0x7c, 0xac, 0xb2, 0x78, 0x86, 0x31, 0xb2, + 0x0f, 0x29, 0x55, 0xce, 0x8e, 0x6a, 0x90, 0xe4, 0xef, 0x2b, 0xec, 0x55, 0x24, 0xb3, 0x7c, 0x7d, + 0xd2, 0xef, 0x92, 0x56, 0xa7, 0x54, 0x31, 0x12, 0x35, 0x21, 0xe5, 0xf1, 0x27, 0x35, 0xf6, 0x56, + 0x72, 0x6e, 0xb9, 0x1a, 0x79, 0xb1, 0x5b, 0x9d, 0x52, 0x83, 0xb1, 0x68, 0x3b, 0x78, 0xd7, 0xe6, + 0x5e, 0x5d, 0x3c, 0xdb, 0x57, 0x26, 0x4c, 0xc8, 0x86, 0x80, 0x23, 0x28, 0x74, 0x83, 0x16, 0xbb, + 0x5f, 0x65, 0x4f, 0x2e, 0x67, 0x6f, 0x70, 0xe4, 0xad, 0x8e, 0x6e, 0x90, 0x8f, 0x44, 0x1b, 0x00, + 0x46, 0x18, 0x69, 0xd8, 0x63, 0x4c, 0x66, 0xf9, 0xf5, 0x8b, 0x64, 0x33, 0xab, 0x53, 0x6a, 0x04, + 0x01, 0x7d, 0x08, 0x19, 0x63, 0x68, 0x3a, 0x85, 0x59, 0x06, 0xf8, 0xc6, 0x85, 0xfc, 0xe7, 0x2a, + 0xf5, 0x99, 0x43, 0xea, 0xa8, 0xcf, 0x54, 0xc6, 0x7d, 0x66, 0x13, 0xb2, 0xe2, 0x66, 0x81, 0x7f, + 0xf9, 0x5b, 0xc8, 0x33, 0x97, 0x1d, 0xd5, 0x92, 0xe0, 0xdb, 0xe0, 0x4a, 0xd3, 0x31, 0x5c, 0x13, + 0x9b, 0x4d, 0xda, 0x56, 0xc5, 0x55, 0x25, 0x6b, 0x10, 0xb4, 0x02, 0x39, 0xc3, 0xc6, 0xba, 0xd3, + 0xef, 0x05, 0x38, 0x68, 0x42, 0x9c, 0xac, 0x18, 0x27, 0x80, 0x36, 0x00, 0xed, 0xb1, 0xb7, 0xf0, + 0xe8, 0xaa, 0xd8, 0x9b, 0xce, 0x24, 0x60, 0x0a, 0x1b, 0xab, 0x0e, 0x57, 0x86, 0x5e, 0x82, 0xac, + 0xe3, 0x3a, 0x86, 0xee, 0x18, 0xd8, 0x66, 0xd1, 0x8d, 0x3f, 0x03, 0x8d, 0x12, 0xd1, 0x27, 0x90, + 0x23, 0x23, 0x29, 0x7c, 0xe1, 0x0a, 0x9b, 0xf1, 0xcd, 0x8b, 0xde, 0x07, 0xae, 0x4e, 0xa9, 0x63, + 0x48, 0xe8, 0x37, 0x41, 0xf1, 0xc7, 0xee, 0x80, 0xd9, 0x83, 0xd2, 0xd9, 0xdf, 0x9d, 0x3c, 0xe3, + 0xd2, 0x7a, 0x75, 0x4a, 0x3d, 0x81, 0x56, 0x4b, 0x43, 0x4a, 0xdc, 0x98, 0x87, 0xef, 0x50, 0x29, + 0x45, 0x2e, 0xff, 0x22, 0x09, 0x72, 0x98, 0xfc, 0x2c, 0x01, 0x0a, 0xc3, 0xd5, 0xf0, 0xe3, 0x1f, + 0xea, 0x77, 0x63, 0xab, 0x53, 0x6a, 0x3e, 0xe8, 0x1b, 0x7e, 0xff, 0xf3, 0x2a, 0xcc, 0x76, 0x5d, + 0xd3, 0xda, 0xb3, 0x86, 0xde, 0x8e, 0xdf, 0x56, 0xe5, 0x02, 0xb2, 0xf0, 0x76, 0xb7, 0x47, 0xde, + 0xc8, 0xe3, 0x13, 0x84, 0xde, 0xd5, 0xa9, 0xc8, 0x23, 0x3a, 0xf5, 0xbe, 0x5e, 0xdf, 0x71, 0x2c, + 0x67, 0x5f, 0x13, 0xf5, 0x16, 0x0f, 0xec, 0x59, 0x41, 0x15, 0x25, 0x53, 0x7d, 0xcc, 0x1d, 0xdd, + 0x38, 0xd7, 0x1d, 0x05, 0x7b, 0x5f, 0x95, 0x42, 0x7f, 0x74, 0x67, 0xdc, 0x1f, 0xdd, 0x3c, 0xdf, + 0x1f, 0x45, 0x60, 0x42, 0x87, 0xb4, 0x73, 0xaa, 0x43, 0x5a, 0x9a, 0x50, 0x5b, 0x22, 0x88, 0xa3, + 0x1e, 0xa9, 0x3e, 0xe6, 0x91, 0x6e, 0x9c, 0xeb, 0x91, 0xa2, 0x7b, 0x14, 0x2e, 0x69, 0xf3, 0x14, + 0x97, 0xf4, 0xc6, 0x44, 0x2e, 0x29, 0x02, 0x16, 0xf5, 0x49, 0xea, 0x69, 0x3e, 0xa9, 0x32, 0x99, + 0x4f, 0x8a, 0x40, 0x8e, 0x38, 0xa5, 0xef, 0x9e, 0x30, 0x38, 0x85, 0xc1, 0x7e, 0xe3, 0xc2, 0x45, + 0xf6, 0xaa, 0x74, 0xc2, 0xe2, 0xf4, 0x53, 0x2c, 0x2e, 0xcf, 0xe0, 0xdf, 0xba, 0x80, 0xc5, 0x45, + 0x26, 0x38, 0x69, 0x72, 0x00, 0x72, 0xf0, 0x16, 0x16, 0x31, 0xbf, 0xf2, 0x9f, 0x48, 0x10, 0x6f, + 0xbb, 0xbb, 0x28, 0x37, 0xbc, 0x91, 0x60, 0x77, 0x09, 0x1f, 0x0c, 0xd9, 0x45, 0x7a, 0xfd, 0xf5, + 0x33, 0x56, 0x12, 0xcc, 0xac, 0x86, 0x83, 0xd0, 0x7b, 0x90, 0xea, 0xf1, 0xd4, 0x49, 0x58, 0x58, + 0xf9, 0xac, 0xf1, 0x9c, 0x53, 0x0d, 0x86, 0xdc, 0xbc, 0x11, 0xfd, 0xf2, 0x7e, 0xdd, 0x35, 0x31, + 0xca, 0x01, 0x6c, 0xe9, 0x84, 0xf4, 0x0e, 0x3c, 0x9d, 0x60, 0x65, 0x0a, 0xa5, 0x20, 0xbe, 0xb6, + 0xde, 0x51, 0xa4, 0x9b, 0xdf, 0x89, 0xde, 0x51, 0x34, 0xd4, 0x6a, 0x6b, 0xa3, 0xb5, 0xb1, 0xa2, + 0x6d, 0x54, 0xd7, 0x9b, 0x1d, 0x65, 0x0a, 0x15, 0x60, 0xfe, 0xa3, 0x6a, 0x6b, 0x5b, 0x5c, 0x5a, + 0x68, 0xad, 0x8d, 0xed, 0xa6, 0x7a, 0xaf, 0x7a, 0x57, 0x91, 0xd0, 0x55, 0x40, 0xea, 0x66, 0x7d, + 0xad, 0xd3, 0xa8, 0x69, 0xf5, 0xcd, 0xf5, 0xad, 0x6a, 0x7d, 0xbb, 0xb5, 0xb9, 0xa1, 0xc4, 0x90, + 0x0c, 0x89, 0xc6, 0xe6, 0x46, 0x53, 0x81, 0x9b, 0x3f, 0x89, 0x43, 0x82, 0xca, 0x18, 0xbd, 0x04, + 0x99, 0x9d, 0x8d, 0xce, 0x56, 0xb3, 0xde, 0xba, 0xd3, 0x6a, 0x36, 0x94, 0xa9, 0xe2, 0xdc, 0xe3, + 0x27, 0x8b, 0xb3, 0xb4, 0x6b, 0xc7, 0x21, 0x3d, 0x6c, 0x30, 0xe7, 0x82, 0x8a, 0x90, 0xac, 0x55, + 0xeb, 0x6b, 0x3b, 0x5b, 0x8a, 0x54, 0xcc, 0x3d, 0x7e, 0xb2, 0x08, 0x94, 0x81, 0x1b, 0x36, 0x7a, + 0x01, 0x52, 0x6a, 0xb3, 0xb3, 0xbd, 0xa9, 0x36, 0x95, 0x58, 0x71, 0xf6, 0xf1, 0x93, 0xc5, 0x0c, + 0xed, 0x14, 0xf6, 0x8a, 0x5e, 0x85, 0x6c, 0xa7, 0xbe, 0xda, 0x5c, 0xaf, 0x6a, 0xf5, 0xd5, 0xea, + 0xc6, 0x4a, 0x53, 0x89, 0x17, 0xe7, 0x1f, 0x3f, 0x59, 0x54, 0xc6, 0x0f, 0x98, 0x4e, 0xd1, 0x5a, + 0xdf, 0xda, 0x54, 0xb7, 0x95, 0xc4, 0x70, 0x0a, 0x6e, 0x57, 0xa8, 0x0c, 0xc0, 0x47, 0xdf, 0x69, + 0x36, 0x1b, 0xca, 0x74, 0x11, 0x3d, 0x7e, 0xb2, 0x98, 0xa3, 0xfd, 0x43, 0x73, 0x41, 0x2f, 0xc3, + 0x4c, 0x5d, 0x6d, 0x56, 0xb7, 0x9b, 0x5a, 0x67, 0xbb, 0xba, 0xdd, 0x51, 0x92, 0xc3, 0x9d, 0x44, + 0x4c, 0x00, 0x55, 0x20, 0x5f, 0xdd, 0xd9, 0xde, 0xd4, 0x46, 0x78, 0x53, 0xc5, 0x6b, 0x8f, 0x9f, + 0x2c, 0xce, 0x51, 0xde, 0x6a, 0xdf, 0x77, 0xa3, 0xfc, 0xaf, 0x83, 0x32, 0xb2, 0x7e, 0x6d, 0xa5, + 0xae, 0xc8, 0xc5, 0xab, 0x8f, 0x9f, 0x2c, 0xa2, 0xf1, 0x2d, 0xac, 0xd4, 0xd1, 0xaf, 0xc1, 0xd5, + 0xed, 0x8f, 0xb7, 0x9a, 0x8d, 0x66, 0xa7, 0xae, 0x8d, 0x6e, 0x3b, 0x5d, 0x2c, 0x3c, 0x7e, 0xb2, + 0x38, 0x4f, 0xc7, 0x9c, 0xd8, 0xfa, 0x1b, 0xa0, 0x74, 0xb6, 0xd5, 0x66, 0x75, 0x5d, 0x6b, 0x6d, + 0xac, 0x34, 0x3b, 0xec, 0xb0, 0x60, 0xb8, 0xa4, 0xb1, 0xff, 0x24, 0x50, 0x94, 0x7f, 0xf8, 0x57, + 0x0b, 0x53, 0x7f, 0xf3, 0xd7, 0x0b, 0x53, 0xb5, 0xeb, 0x9f, 0xff, 0xe7, 0xc2, 0xd4, 0xe7, 0xc7, + 0x0b, 0xd2, 0x4f, 0x8f, 0x17, 0xa4, 0x9f, 0x1d, 0x2f, 0x48, 0xff, 0x71, 0xbc, 0x20, 0xfd, 0xd1, + 0x17, 0x0b, 0x53, 0x3f, 0xfd, 0x62, 0x61, 0xea, 0x67, 0x5f, 0x2c, 0x4c, 0x7d, 0x92, 0xe4, 0x5a, + 0xb8, 0x9b, 0x64, 0x59, 0xfb, 0x5b, 0xff, 0x17, 0x00, 0x00, 0xff, 0xff, 0x18, 0xc0, 0x43, 0xa9, + 0x8f, 0x34, 0x00, 0x00, } diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index 5c7f7e03ce8f..c004bac4bc66 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -77,6 +77,11 @@ message EncryptionInfo { map encryptedDataKeyByKMSMasterKeyID = 3; } +message StreamIngestionDetails { + // StreamAddress is the location of the stream which the ingestion job will + // read from. + string stream_address = 1 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/streamclient.StreamAddress"]; +} message BackupDetails { util.hlc.Timestamp start_time = 1 [(gogoproto.nullable) = false]; @@ -601,6 +606,7 @@ enum Type { // We can't name this TYPE_SCHEMA_CHANGE due to how proto generates actual // names for this enum, which cause a conflict with the SCHEMA_CHANGE entry. TYPEDESC_SCHEMA_CHANGE = 9 [(gogoproto.enumvalue_customname) = "TypeTypeSchemaChange"]; + STREAM_INGESTION = 10 [(gogoproto.enumvalue_customname) = "TypeStreamIngestion"]; } message Job { diff --git a/pkg/jobs/jobspb/wrap.go b/pkg/jobs/jobspb/wrap.go index 0c0ef06ec16e..ed0513a29286 100644 --- a/pkg/jobs/jobspb/wrap.go +++ b/pkg/jobs/jobspb/wrap.go @@ -220,7 +220,7 @@ const ( func (Type) SafeValue() {} // NumJobTypes is the number of jobs types. -const NumJobTypes = 10 +const NumJobTypes = 11 func init() { if len(Type_name) != NumJobTypes { diff --git a/pkg/sql/execinfrapb/BUILD.bazel b/pkg/sql/execinfrapb/BUILD.bazel index f9376021f163..b46a4b7980ae 100644 --- a/pkg/sql/execinfrapb/BUILD.bazel +++ b/pkg/sql/execinfrapb/BUILD.bazel @@ -47,6 +47,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/base", + "//pkg/ccl/streamingccl/streamclient", "//pkg/jobs/jobspb", "//pkg/roachpb", "//pkg/rpc", diff --git a/pkg/sql/execinfrapb/processors.pb.go b/pkg/sql/execinfrapb/processors.pb.go index 02b50a6fee55..1d939e464e8f 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_7bc79cdd8886d88f, []int{0} + return fileDescriptor_processors_29d19df42cdc68ae, []int{0} } func (m *ProcessorSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -133,13 +133,14 @@ type ProcessorCoreUnion struct { 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"` } 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_7bc79cdd8886d88f, []int{1} + return fileDescriptor_processors_29d19df42cdc68ae, []int{1} } func (m *ProcessorCoreUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -174,7 +175,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_7bc79cdd8886d88f, []int{2} + return fileDescriptor_processors_29d19df42cdc68ae, []int{2} } func (m *NoopCoreSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -216,7 +217,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_7bc79cdd8886d88f, []int{3} + return fileDescriptor_processors_29d19df42cdc68ae, []int{3} } func (m *LocalPlanNodeSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -249,7 +250,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_7bc79cdd8886d88f, []int{4} + return fileDescriptor_processors_29d19df42cdc68ae, []int{4} } func (m *MetadataTestSenderSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -282,7 +283,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_7bc79cdd8886d88f, []int{5} + return fileDescriptor_processors_29d19df42cdc68ae, []int{5} } func (m *MetadataTestReceiverSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -742,6 +743,18 @@ func (m *ProcessorCoreUnion) MarshalTo(dAtA []byte) (int, error) { } i += n32 } + if m.StreamIngestionData != nil { + dAtA[i] = 0x9a + i++ + dAtA[i] = 0x2 + i++ + i = encodeVarintProcessors(dAtA, i, uint64(m.StreamIngestionData.Size())) + n33, err := m.StreamIngestionData.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n33 + } return i, nil } @@ -1014,6 +1027,10 @@ func (m *ProcessorCoreUnion) Size() (n int) { l = m.Filterer.Size() n += 2 + l + sovProcessors(uint64(l)) } + if m.StreamIngestionData != nil { + l = m.StreamIngestionData.Size() + n += 2 + l + sovProcessors(uint64(l)) + } return n } @@ -1169,6 +1186,9 @@ func (this *ProcessorCoreUnion) GetValue() interface{} { if this.Filterer != nil { return this.Filterer } + if this.StreamIngestionData != nil { + return this.StreamIngestionData + } return nil } @@ -1234,6 +1254,8 @@ func (this *ProcessorCoreUnion) SetValue(value interface{}) bool { this.RestoreData = vt case *FiltererSpec: this.Filterer = vt + case *StreamIngestionDataSpec: + this.StreamIngestionData = vt default: return false } @@ -2499,6 +2521,39 @@ func (m *ProcessorCoreUnion) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 35: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StreamIngestionData", 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.StreamIngestionData == nil { + m.StreamIngestionData = &StreamIngestionDataSpec{} + } + if err := m.StreamIngestionData.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipProcessors(dAtA[iNdEx:]) @@ -2951,85 +3006,87 @@ var ( ) func init() { - proto.RegisterFile("sql/execinfrapb/processors.proto", fileDescriptor_processors_7bc79cdd8886d88f) + proto.RegisterFile("sql/execinfrapb/processors.proto", fileDescriptor_processors_29d19df42cdc68ae) } -var fileDescriptor_processors_7bc79cdd8886d88f = []byte{ - // 1214 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x97, 0xd1, 0x52, 0x1b, 0x37, - 0x14, 0x86, 0x31, 0x18, 0x6c, 0xcb, 0x36, 0x38, 0x0a, 0x49, 0x55, 0xb7, 0x35, 0xd4, 0x93, 0xb4, - 0x6e, 0x4a, 0x4c, 0x87, 0xe9, 0xf4, 0x22, 0xd3, 0x4e, 0x1b, 0xdb, 0x65, 0x58, 0x9a, 0x12, 0xba, - 0xa6, 0x61, 0x86, 0x1b, 0x57, 0xec, 0x0a, 0xb3, 0x61, 0xbd, 0x5a, 0x24, 0x19, 0x42, 0xfa, 0x12, - 0x7d, 0x84, 0x3e, 0x0e, 0x57, 0x9d, 0x5c, 0x75, 0x72, 0xc5, 0xb4, 0xe6, 0x45, 0x32, 0x92, 0xd6, - 0xbb, 0x6b, 0x83, 0xbd, 0xdc, 0x30, 0xb6, 0xfc, 0xff, 0x9f, 0x8e, 0x8e, 0x8e, 0xa4, 0x03, 0x58, - 0xe5, 0xa7, 0xee, 0x3a, 0x79, 0x43, 0x2c, 0xc7, 0x3b, 0x62, 0xd8, 0x3f, 0x5c, 0xf7, 0x19, 0xb5, - 0x08, 0xe7, 0x94, 0xf1, 0xba, 0xcf, 0xa8, 0xa0, 0x10, 0x59, 0xd4, 0x3a, 0x61, 0x14, 0x5b, 0xc7, - 0x75, 0x7e, 0xea, 0xd6, 0x6d, 0x87, 0x0b, 0x7e, 0xea, 0xb2, 0xbe, 0x57, 0x2e, 0x8f, 0x7b, 0x6d, - 0x2c, 0xb0, 0x76, 0x95, 0x1f, 0x4f, 0xe6, 0x76, 0x0e, 0x31, 0x27, 0x81, 0xec, 0xd1, 0x14, 0x99, - 0x9c, 0x4d, 0xab, 0x6a, 0xd3, 0x60, 0x7d, 0xf7, 0xa4, 0xe3, 0xd0, 0x40, 0xb9, 0x36, 0x45, 0x69, - 0x1d, 0x63, 0xaf, 0x4b, 0x8e, 0x08, 0xb1, 0xf9, 0x1d, 0xd4, 0x02, 0x1f, 0xba, 0xa4, 0xc3, 0x05, - 0x16, 0x43, 0xf5, 0x03, 0xa9, 0x16, 0x17, 0x3e, 0xe1, 0xfa, 0x6f, 0x30, 0xbc, 0xdc, 0xa5, 0x5d, - 0xaa, 0x3e, 0xae, 0xcb, 0x4f, 0x7a, 0xb4, 0xfa, 0xcf, 0x1c, 0x28, 0xee, 0x0e, 0x69, 0x6d, 0x9f, - 0x58, 0xb0, 0x09, 0xe6, 0x1d, 0xcf, 0xef, 0x0b, 0x94, 0x5a, 0x9d, 0xab, 0xe5, 0x37, 0xbe, 0xac, - 0x4f, 0xca, 0x6b, 0xdd, 0x90, 0xb2, 0xf6, 0x85, 0x67, 0x49, 0x5f, 0x23, 0x7d, 0x79, 0xb5, 0x32, - 0x63, 0x6a, 0x2f, 0xdc, 0x04, 0x69, 0x8b, 0x32, 0x82, 0x66, 0x57, 0x53, 0xb5, 0xfc, 0xc6, 0xda, - 0x64, 0x46, 0x38, 0x77, 0x93, 0x32, 0xf2, 0xbb, 0xe7, 0x50, 0x2f, 0x00, 0x29, 0x3f, 0xdc, 0x02, - 0x0b, 0xb4, 0x2f, 0x64, 0x34, 0x73, 0x2a, 0x9a, 0x27, 0x93, 0x49, 0x2f, 0x95, 0xce, 0xa4, 0x7d, - 0x41, 0x58, 0x2c, 0xa0, 0xc0, 0x0f, 0x9b, 0x20, 0xed, 0x53, 0x2e, 0x50, 0x5a, 0x45, 0xf4, 0xd5, - 0x94, 0x88, 0x28, 0x17, 0x41, 0x54, 0x31, 0x8c, 0x32, 0xc3, 0x27, 0x20, 0xcb, 0x05, 0xee, 0x92, - 0x8e, 0x63, 0xa3, 0xf9, 0xd5, 0x54, 0x6d, 0xbe, 0xb1, 0x24, 0x7f, 0x1d, 0x5c, 0xad, 0x64, 0xda, - 0x72, 0xdc, 0x68, 0x99, 0x19, 0x25, 0x30, 0x6c, 0xf8, 0x1d, 0x28, 0x84, 0xdb, 0x24, 0xf5, 0x0b, - 0x4a, 0x7f, 0x3f, 0xd0, 0xe7, 0xc3, 0x85, 0x1b, 0x2d, 0x33, 0x1f, 0x0a, 0x0d, 0x1b, 0xfe, 0x00, - 0x0a, 0x8c, 0xf0, 0xbe, 0x2b, 0x3a, 0x6a, 0xf7, 0x50, 0x46, 0x2d, 0xbc, 0x3c, 0x16, 0x30, 0x27, - 0xbd, 0xba, 0xde, 0xdd, 0x3d, 0x33, 0xaf, 0xf5, 0x7b, 0xf2, 0x6b, 0xf5, 0x5f, 0x08, 0xe0, 0xcd, - 0xa4, 0xc2, 0x67, 0x20, 0xed, 0x51, 0xea, 0xa3, 0x94, 0x5a, 0xfe, 0x17, 0x93, 0x97, 0xbf, 0x43, - 0xa9, 0x2f, 0x6d, 0x72, 0xed, 0xa6, 0xf2, 0xc0, 0x5f, 0x40, 0x5e, 0x55, 0x99, 0x49, 0xb0, 0x4d, - 0x58, 0xb0, 0xa7, 0x53, 0x32, 0xb8, 0x17, 0x89, 0x15, 0x25, 0xee, 0x86, 0x5b, 0x00, 0xbc, 0xa6, - 0x8e, 0x17, 0xb0, 0xe6, 0x14, 0xab, 0x36, 0x99, 0xb5, 0x1d, 0x6a, 0x15, 0x2a, 0xe6, 0x85, 0xdf, - 0x83, 0x05, 0x4e, 0x99, 0x20, 0x2c, 0xd8, 0xd3, 0x47, 0x93, 0x29, 0x6d, 0xa5, 0x53, 0x84, 0xc0, - 0x23, 0xe3, 0xc0, 0xdd, 0x2e, 0x23, 0x5d, 0x2c, 0x28, 0x53, 0x9b, 0x39, 0x35, 0x8e, 0xe7, 0xa1, - 0x56, 0xc7, 0x11, 0x79, 0x61, 0x03, 0x64, 0xa5, 0xd0, 0xf1, 0x2c, 0x81, 0x32, 0x49, 0xe9, 0x6d, - 0x05, 0x4a, 0x45, 0x09, 0x7d, 0x32, 0xc5, 0x3d, 0xc2, 0xba, 0x44, 0x2e, 0x97, 0x30, 0x94, 0x4d, - 0x4a, 0xf1, 0xaf, 0x91, 0x58, 0xa7, 0x38, 0xe6, 0x96, 0x4b, 0x3b, 0xc6, 0xfc, 0x38, 0x60, 0xe5, - 0x92, 0x96, 0xb6, 0x15, 0x6a, 0xf5, 0xd2, 0x22, 0x2f, 0xfc, 0x09, 0x2c, 0x9c, 0x61, 0xb7, 0x4f, - 0x38, 0x02, 0x49, 0x94, 0x57, 0x4a, 0x17, 0x56, 0x4e, 0xe0, 0x93, 0xb1, 0x1c, 0x62, 0xeb, 0xe4, - 0xc8, 0x71, 0x5d, 0xc2, 0x50, 0x3e, 0x89, 0xd2, 0x08, 0xb5, 0x3a, 0x96, 0xc8, 0x0b, 0x5f, 0x00, - 0xc0, 0x08, 0xb6, 0x8d, 0x9e, 0x4f, 0x99, 0x40, 0xc5, 0xa4, 0x8b, 0xc5, 0x0c, 0xb5, 0x2d, 0x2c, - 0xb0, 0xa6, 0x45, 0x7e, 0xf8, 0x23, 0xc8, 0xb4, 0x71, 0xcf, 0x97, 0x41, 0x2d, 0x29, 0xd4, 0xe3, - 0x29, 0xd5, 0xa3, 0x85, 0x8a, 0x31, 0x74, 0xc1, 0x03, 0x50, 0xd2, 0x1f, 0xa3, 0xca, 0x40, 0x25, - 0x45, 0xaa, 0x27, 0x91, 0xc6, 0x6a, 0xe9, 0x06, 0x07, 0xfe, 0x01, 0x60, 0x8f, 0x08, 0x2c, 0x9f, - 0xa9, 0x3d, 0xc2, 0x45, 0x9b, 0x78, 0xf2, 0xac, 0x40, 0x45, 0xff, 0x66, 0x5a, 0x51, 0x8c, 0x7b, - 0x14, 0xff, 0x16, 0x16, 0x3c, 0x02, 0xcb, 0xf1, 0x51, 0x93, 0x58, 0xc4, 0x39, 0x23, 0x0c, 0xdd, - 0x57, 0x73, 0x6c, 0xdc, 0x6d, 0x8e, 0xa1, 0x4b, 0xcd, 0x72, 0x2b, 0x0f, 0xfe, 0x0c, 0x72, 0xcd, - 0xf6, 0xab, 0x7d, 0xe6, 0xc8, 0x63, 0xba, 0xac, 0xe0, 0x53, 0x1e, 0x94, 0x50, 0xaa, 0x88, 0x91, - 0x13, 0xee, 0x80, 0xc2, 0x5b, 0xa7, 0xfb, 0x16, 0x77, 0x83, 0x9a, 0x7e, 0xa0, 0x48, 0x53, 0x1e, - 0x83, 0x83, 0x98, 0x5a, 0xc1, 0x46, 0xfc, 0xb2, 0x2a, 0x7d, 0x46, 0x5f, 0x13, 0x4b, 0xb4, 0x89, - 0x40, 0x0f, 0x93, 0xaa, 0x72, 0x37, 0xd4, 0xea, 0x3a, 0x8a, 0xbc, 0xf2, 0xf0, 0x9f, 0x3b, 0x9e, - 0x4d, 0xcf, 0x09, 0x43, 0x1f, 0x25, 0x1d, 0xfe, 0xfd, 0x40, 0xa9, 0x0f, 0xff, 0xd0, 0x07, 0x7f, - 0x03, 0x45, 0x97, 0x5a, 0xd8, 0xdd, 0x75, 0xb1, 0xb7, 0x43, 0x6d, 0x82, 0x90, 0x02, 0x7d, 0x3d, - 0x19, 0xf4, 0x22, 0x2e, 0x57, 0xb4, 0x51, 0x82, 0xac, 0x4e, 0xdd, 0x46, 0xc4, 0xaa, 0xf3, 0xe3, - 0xa4, 0xea, 0x6c, 0x8e, 0x39, 0x74, 0x75, 0x8e, 0x73, 0xe0, 0x1e, 0x58, 0xd4, 0x63, 0x9b, 0x8c, - 0x7a, 0xc2, 0x21, 0x0c, 0x95, 0x93, 0x0e, 0x63, 0x73, 0x44, 0xaf, 0xb8, 0x63, 0x0c, 0xb9, 0x25, - 0x94, 0xd9, 0x8e, 0x87, 0x5d, 0x47, 0x5c, 0xa0, 0x4f, 0x92, 0xb6, 0xe4, 0x65, 0xa8, 0xd5, 0x5b, - 0x12, 0x79, 0x65, 0x3a, 0x65, 0xb3, 0x65, 0xd2, 0xf3, 0xa0, 0xee, 0x3e, 0x4d, 0x4a, 0x67, 0x23, - 0x2e, 0xd7, 0xe9, 0x1c, 0x21, 0xc8, 0x74, 0x3a, 0xde, 0x19, 0x61, 0x82, 0xd8, 0x9b, 0x8e, 0x2b, - 0x08, 0x23, 0x0c, 0x7d, 0x96, 0x94, 0x4e, 0x63, 0xcc, 0xa1, 0xd3, 0x39, 0xce, 0x91, 0xe9, 0x1c, - 0x8e, 0x05, 0xd5, 0x5d, 0x49, 0x4a, 0xa7, 0x31, 0xa2, 0xd7, 0xe9, 0x1c, 0x65, 0x0c, 0xef, 0xdd, - 0xbe, 0x2f, 0x6f, 0x3f, 0xb4, 0x72, 0x97, 0x7b, 0x57, 0x6b, 0xa3, 0x7b, 0x57, 0x7f, 0x87, 0xfb, - 0x60, 0x89, 0xfb, 0xae, 0x23, 0x9e, 0x7b, 0x76, 0xdb, 0xc2, 0x42, 0x26, 0x74, 0x55, 0xe1, 0x9e, - 0x4e, 0xb9, 0xe7, 0x46, 0x0d, 0x8a, 0x39, 0x4e, 0x91, 0x6f, 0x1e, 0x23, 0x5c, 0x50, 0x46, 0x54, - 0x8c, 0x9f, 0x27, 0xbd, 0x79, 0x66, 0x24, 0xd6, 0x6f, 0x5e, 0xcc, 0x2d, 0xcf, 0xe1, 0xd1, 0x70, - 0x67, 0xaa, 0x49, 0xe7, 0x70, 0x64, 0x47, 0x42, 0xdf, 0xb3, 0xf4, 0xe5, 0xdf, 0x2b, 0xa9, 0xed, - 0x74, 0x76, 0xb1, 0xb4, 0xb4, 0x9d, 0xce, 0xde, 0x2b, 0xc1, 0xed, 0x74, 0x76, 0xa1, 0x94, 0xd9, - 0x4e, 0x67, 0x0b, 0xa5, 0x62, 0x75, 0x11, 0x14, 0xe2, 0xbd, 0x51, 0xf5, 0x4f, 0x70, 0xef, 0xc6, - 0x31, 0x84, 0x35, 0x50, 0x30, 0xe9, 0x79, 0x9b, 0xf6, 0x99, 0x45, 0x0c, 0xfb, 0x8d, 0x6a, 0xb7, - 0x8a, 0x41, 0x0b, 0x39, 0xf2, 0x0b, 0xac, 0x82, 0xdc, 0x4e, 0xbf, 0xa7, 0x5a, 0x68, 0xae, 0x5a, - 0xaa, 0xa1, 0x2c, 0x1a, 0x86, 0x08, 0xa4, 0x77, 0x70, 0x8f, 0xa8, 0x2e, 0x29, 0x37, 0x6c, 0x44, - 0xe5, 0x48, 0xf5, 0x5b, 0xf0, 0xf0, 0xf6, 0xdb, 0x1e, 0x96, 0xc1, 0xac, 0x63, 0xab, 0x79, 0x73, - 0x0d, 0x10, 0x34, 0x9b, 0xb3, 0x46, 0xcb, 0x9c, 0x75, 0xec, 0xea, 0x16, 0x40, 0x93, 0xee, 0x6f, - 0xb8, 0x06, 0x00, 0x57, 0x94, 0x8e, 0x63, 0x73, 0xd5, 0xfb, 0xe7, 0x1a, 0xc5, 0xc1, 0xd5, 0x4a, - 0x4e, 0xb3, 0x8d, 0x16, 0x37, 0x73, 0x5a, 0x60, 0xd8, 0xbc, 0xf1, 0xf4, 0xf2, 0xff, 0xca, 0xcc, - 0xe5, 0xa0, 0x92, 0x7a, 0x37, 0xa8, 0xa4, 0xde, 0x0f, 0x2a, 0xa9, 0xff, 0x06, 0x95, 0xd4, 0x5f, - 0xd7, 0x95, 0x99, 0x77, 0xd7, 0x95, 0x99, 0xf7, 0xd7, 0x95, 0x99, 0x83, 0x7c, 0xec, 0xff, 0x95, - 0x0f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x05, 0xbc, 0xaf, 0x7b, 0xbe, 0x0d, 0x00, 0x00, +var fileDescriptor_processors_29d19df42cdc68ae = []byte{ + // 1245 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, } diff --git a/pkg/sql/execinfrapb/processors.proto b/pkg/sql/execinfrapb/processors.proto index 286bdaedaf1e..222ef2e97ea6 100644 --- a/pkg/sql/execinfrapb/processors.proto +++ b/pkg/sql/execinfrapb/processors.proto @@ -117,6 +117,7 @@ message ProcessorCoreUnion { optional SplitAndScatterSpec splitAndScatter = 32; optional RestoreDataSpec restoreData = 33; optional FiltererSpec filterer = 34; + optional StreamIngestionDataSpec streamIngestionData = 35; reserved 6, 12; } diff --git a/pkg/sql/execinfrapb/processors_bulk_io.pb.go b/pkg/sql/execinfrapb/processors_bulk_io.pb.go index 7dfb4741fdbc..03097fe0f52f 100644 --- a/pkg/sql/execinfrapb/processors_bulk_io.pb.go +++ b/pkg/sql/execinfrapb/processors_bulk_io.pb.go @@ -18,6 +18,7 @@ import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import time "time" import github_com_cockroachdb_cockroach_pkg_security "github.com/cockroachdb/cockroach/pkg/security" +import github_com_cockroachdb_cockroach_pkg_ccl_streamingccl_streamclient "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/streamclient" import github_com_gogo_protobuf_sortkeys "github.com/gogo/protobuf/sortkeys" import encoding_binary "encoding/binary" @@ -71,7 +72,7 @@ func (x *FileCompression) UnmarshalJSON(data []byte) error { return nil } func (FileCompression) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_bf46a169ea516277, []int{0} + return fileDescriptor_processors_bulk_io_a1c1a7c66115c204, []int{0} } type BackfillerSpec_Type int32 @@ -110,7 +111,7 @@ func (x *BackfillerSpec_Type) UnmarshalJSON(data []byte) error { return nil } func (BackfillerSpec_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_bf46a169ea516277, []int{0, 0} + return fileDescriptor_processors_bulk_io_a1c1a7c66115c204, []int{0, 0} } // BackfillerSpec is the specification for a "schema change backfiller". @@ -142,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_bf46a169ea516277, []int{0} + return fileDescriptor_processors_bulk_io_a1c1a7c66115c204, []int{0} } func (m *BackfillerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -182,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_bf46a169ea516277, []int{1} + return fileDescriptor_processors_bulk_io_a1c1a7c66115c204, []int{1} } func (m *JobProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -245,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_bf46a169ea516277, []int{2} + return fileDescriptor_processors_bulk_io_a1c1a7c66115c204, []int{2} } func (m *ReadImportDataSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -283,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_bf46a169ea516277, []int{2, 0} + return fileDescriptor_processors_bulk_io_a1c1a7c66115c204, []int{2, 0} } func (m *ReadImportDataSpec_ImportTable) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -308,6 +309,39 @@ func (m *ReadImportDataSpec_ImportTable) XXX_DiscardUnknown() { var xxx_messageInfo_ReadImportDataSpec_ImportTable proto.InternalMessageInfo +type StreamIngestionDataSpec struct { + PartitionAddress []github_com_cockroachdb_cockroach_pkg_ccl_streamingccl_streamclient.PartitionAddress `protobuf:"bytes,1,rep,name=partition_address,json=partitionAddress,customtype=github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/streamclient.PartitionAddress" json:"partition_address"` +} + +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_a1c1a7c66115c204, []int{3} +} +func (m *StreamIngestionDataSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StreamIngestionDataSpec) 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 *StreamIngestionDataSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_StreamIngestionDataSpec.Merge(dst, src) +} +func (m *StreamIngestionDataSpec) XXX_Size() int { + return m.Size() +} +func (m *StreamIngestionDataSpec) XXX_DiscardUnknown() { + xxx_messageInfo_StreamIngestionDataSpec.DiscardUnknown(m) +} + +var xxx_messageInfo_StreamIngestionDataSpec 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"` @@ -329,7 +363,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_bf46a169ea516277, []int{3} + return fileDescriptor_processors_bulk_io_a1c1a7c66115c204, []int{4} } func (m *BackupDataSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -368,7 +402,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_bf46a169ea516277, []int{4} + return fileDescriptor_processors_bulk_io_a1c1a7c66115c204, []int{5} } func (m *RestoreSpanEntry) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -406,7 +440,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_bf46a169ea516277, []int{5} + return fileDescriptor_processors_bulk_io_a1c1a7c66115c204, []int{6} } func (m *RestoreDataSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -440,7 +474,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_bf46a169ea516277, []int{6} + return fileDescriptor_processors_bulk_io_a1c1a7c66115c204, []int{7} } func (m *SplitAndScatterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -473,7 +507,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_bf46a169ea516277, []int{6, 0} + return fileDescriptor_processors_bulk_io_a1c1a7c66115c204, []int{7, 0} } func (m *SplitAndScatterSpec_RestoreEntryChunk) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -520,7 +554,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_bf46a169ea516277, []int{7} + return fileDescriptor_processors_bulk_io_a1c1a7c66115c204, []int{8} } func (m *CSVWriterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -555,7 +589,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_bf46a169ea516277, []int{8} + return fileDescriptor_processors_bulk_io_a1c1a7c66115c204, []int{9} } func (m *BulkRowWriterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -588,6 +622,7 @@ func init() { proto.RegisterMapType((map[string]*ReadImportDataSpec_ImportTable)(nil), "cockroach.sql.distsqlrun.ReadImportDataSpec.TablesEntry") 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((*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") @@ -856,6 +891,39 @@ func (m *ReadImportDataSpec_ImportTable) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func (m *StreamIngestionDataSpec) 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 *StreamIngestionDataSpec) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.PartitionAddress) > 0 { + for _, s := range m.PartitionAddress { + dAtA[i] = 0xa + i++ + l = len(s) + for l >= 1<<7 { + dAtA[i] = uint8(uint64(l)&0x7f | 0x80) + l >>= 7 + i++ + } + dAtA[i] = uint8(l) + i++ + i += copy(dAtA[i:], s) + } + } + return i, nil +} + func (m *BackupDataSpec) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -1348,6 +1416,21 @@ func (m *ReadImportDataSpec_ImportTable) Size() (n int) { return n } +func (m *StreamIngestionDataSpec) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.PartitionAddress) > 0 { + for _, s := range m.PartitionAddress { + l = len(s) + n += 1 + l + sovProcessorsBulkIo(uint64(l)) + } + } + return n +} + func (m *BackupDataSpec) Size() (n int) { if m == nil { return 0 @@ -2475,6 +2558,85 @@ func (m *ReadImportDataSpec_ImportTable) Unmarshal(dAtA []byte) error { } return nil } +func (m *StreamIngestionDataSpec) 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: StreamIngestionDataSpec: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: StreamIngestionDataSpec: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PartitionAddress", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProcessorsBulkIo + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthProcessorsBulkIo + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.PartitionAddress = append(m.PartitionAddress, github_com_cockroachdb_cockroach_pkg_ccl_streamingccl_streamclient.PartitionAddress(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipProcessorsBulkIo(dAtA[iNdEx:]) + if err != nil { + return err + } + if 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 @@ -3929,114 +4091,119 @@ var ( ) func init() { - proto.RegisterFile("sql/execinfrapb/processors_bulk_io.proto", fileDescriptor_processors_bulk_io_bf46a169ea516277) -} - -var fileDescriptor_processors_bulk_io_bf46a169ea516277 = []byte{ - // 1673 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x58, 0x4f, 0x6f, 0x1b, 0xc7, - 0x15, 0xd7, 0x92, 0x4b, 0x8a, 0x7c, 0xb4, 0x24, 0x6a, 0xe2, 0xa4, 0x5b, 0x01, 0x95, 0x04, 0x26, - 0x72, 0x59, 0x17, 0x26, 0x11, 0xbb, 0x2d, 0x8c, 0xb6, 0x89, 0x6b, 0x52, 0x96, 0x43, 0x39, 0xb1, - 0xd5, 0xa5, 0xa5, 0x00, 0x41, 0x8b, 0xc5, 0x72, 0x77, 0x44, 0x8d, 0xb9, 0xdc, 0x59, 0xcd, 0xcc, - 0xca, 0xa6, 0x4f, 0x05, 0x7a, 0xea, 0xad, 0x1f, 0xa1, 0xdf, 0xa0, 0xfd, 0x0e, 0xbd, 0xf8, 0x98, - 0x63, 0x4e, 0x46, 0x2b, 0x7f, 0x8b, 0x9e, 0x8a, 0xf9, 0xb3, 0xd4, 0x4a, 0xa6, 0x64, 0x29, 0x86, - 0x2f, 0xf4, 0x7a, 0xde, 0xfc, 0x7e, 0xf3, 0xe6, 0xcd, 0x7b, 0xbf, 0x37, 0x23, 0x68, 0xf2, 0xc3, - 0xa8, 0x8d, 0x5f, 0xe0, 0x80, 0xc4, 0xfb, 0xcc, 0x4f, 0x06, 0xed, 0x84, 0xd1, 0x00, 0x73, 0x4e, - 0x19, 0xf7, 0x06, 0x69, 0x34, 0xf2, 0x08, 0x6d, 0x25, 0x8c, 0x0a, 0x8a, 0x9c, 0x80, 0x06, 0x23, - 0x46, 0xfd, 0xe0, 0xa0, 0xc5, 0x0f, 0xa3, 0x56, 0x48, 0xb8, 0xe0, 0x87, 0x11, 0x4b, 0xe3, 0x95, - 0x4f, 0x9e, 0xd1, 0x01, 0x6f, 0xcb, 0x9f, 0x64, 0xa0, 0xfe, 0xd1, 0x88, 0x15, 0x47, 0xcd, 0x4e, - 0x06, 0x6d, 0x42, 0x6f, 0xed, 0x53, 0x36, 0xf6, 0x45, 0x66, 0xf9, 0x54, 0xae, 0x1a, 0xf8, 0xc2, - 0x8f, 0xe8, 0xb0, 0x1d, 0x62, 0x1e, 0x24, 0x83, 0x36, 0x17, 0x2c, 0x0d, 0x44, 0xca, 0x70, 0x68, - 0x26, 0x6d, 0x5c, 0xe4, 0x9a, 0xcf, 0x71, 0xb6, 0x4a, 0x2a, 0x48, 0xd4, 0x3e, 0x88, 0x82, 0xb6, - 0x20, 0x63, 0xcc, 0x85, 0x3f, 0x4e, 0x8c, 0xe5, 0xfa, 0x90, 0x0e, 0xa9, 0xfa, 0x6c, 0xcb, 0x2f, - 0x33, 0x8a, 0x32, 0xaf, 0x42, 0x5f, 0xf8, 0x66, 0x6c, 0x39, 0x1b, 0xf3, 0x13, 0xa2, 0x87, 0x1a, - 0xff, 0x2c, 0xc2, 0x62, 0xc7, 0x0f, 0x46, 0xfb, 0x24, 0x8a, 0x30, 0xeb, 0x27, 0x38, 0x40, 0x0f, - 0xc1, 0x16, 0x93, 0x04, 0x3b, 0xd6, 0xba, 0xd5, 0x5c, 0xbc, 0x7d, 0xab, 0x75, 0x5e, 0x40, 0x5a, - 0xa7, 0x71, 0xad, 0xa7, 0x93, 0x04, 0x77, 0xec, 0x57, 0xaf, 0xd7, 0xe6, 0x5c, 0x45, 0x80, 0x3a, - 0x50, 0x12, 0xfe, 0x20, 0xc2, 0x4e, 0x61, 0xdd, 0x6a, 0xd6, 0x6e, 0xdf, 0x38, 0xc3, 0xc4, 0x0f, - 0x23, 0xb5, 0xbf, 0xa7, 0x72, 0xce, 0x26, 0xe6, 0x01, 0x23, 0x89, 0xa0, 0xcc, 0x50, 0x68, 0x28, - 0x7a, 0x00, 0x25, 0x9e, 0xf8, 0x31, 0x77, 0x8a, 0xeb, 0xc5, 0x66, 0xed, 0xf6, 0x2f, 0xce, 0xf7, - 0x46, 0xd1, 0xb8, 0xd8, 0x0f, 0xa5, 0x3b, 0x7e, 0x9c, 0xd1, 0x28, 0x34, 0xfa, 0x1c, 0x2a, 0x61, - 0xca, 0x7c, 0x41, 0x68, 0xec, 0xd8, 0xeb, 0x56, 0xb3, 0xd8, 0xf9, 0x58, 0x9a, 0xff, 0xf7, 0x7a, - 0x6d, 0x41, 0x86, 0xb3, 0xb5, 0x69, 0x8c, 0xee, 0x74, 0x1a, 0xfa, 0x14, 0x20, 0x38, 0x48, 0xe3, - 0x91, 0xc7, 0xc9, 0x4b, 0xec, 0x94, 0x14, 0x48, 0x73, 0x56, 0xd5, 0x78, 0x9f, 0xbc, 0xc4, 0xe8, - 0x1e, 0x54, 0x18, 0xf6, 0xc3, 0xfb, 0xfc, 0xc9, 0xbe, 0x33, 0xaf, 0x76, 0xf9, 0xb3, 0x9c, 0x87, - 0xf2, 0xc8, 0x5a, 0x07, 0x51, 0xd0, 0x7a, 0x9a, 0x1d, 0x99, 0x61, 0x98, 0x82, 0x1a, 0x37, 0xc1, - 0x96, 0x71, 0x43, 0x35, 0x98, 0xef, 0xc5, 0x47, 0x7e, 0x44, 0xc2, 0xfa, 0x1c, 0x02, 0x28, 0x77, - 0x69, 0x94, 0x8e, 0xe3, 0xba, 0x85, 0xaa, 0x50, 0xea, 0xc5, 0x21, 0x7e, 0x51, 0x2f, 0x6c, 0xdb, - 0x95, 0x72, 0x7d, 0xbe, 0xf1, 0x1c, 0x6a, 0xdb, 0x74, 0xb0, 0xc3, 0xe8, 0x90, 0x61, 0xce, 0xd1, - 0x67, 0x50, 0x7e, 0x46, 0x07, 0x1e, 0x09, 0xd5, 0x79, 0x15, 0x3b, 0x0b, 0x72, 0x81, 0xe3, 0xd7, - 0x6b, 0xa5, 0x6d, 0x3a, 0xe8, 0x6d, 0xba, 0xa5, 0x67, 0x74, 0xd0, 0x0b, 0x51, 0x13, 0xae, 0x05, - 0x34, 0x16, 0x8c, 0x0c, 0x52, 0x15, 0x03, 0x79, 0x22, 0x05, 0xe3, 0xcc, 0x29, 0x0b, 0x72, 0xc0, - 0xe6, 0x11, 0x15, 0x4e, 0x71, 0xdd, 0x6a, 0x96, 0xb2, 0xe3, 0x94, 0x23, 0x8d, 0x57, 0x15, 0x40, - 0x32, 0xbe, 0xbd, 0x71, 0x42, 0x99, 0xd8, 0xf4, 0x85, 0xaf, 0xd2, 0x65, 0x03, 0x6a, 0xdc, 0x1f, - 0x27, 0x11, 0xd6, 0x81, 0x2a, 0xe4, 0x70, 0xa0, 0x0d, 0x2a, 0x52, 0x0f, 0xa1, 0x92, 0x18, 0x9f, - 0x9d, 0xb2, 0x8a, 0xd4, 0xc6, 0xf9, 0x67, 0x99, 0xdb, 0x60, 0x16, 0xb1, 0x0c, 0x8c, 0x1e, 0x42, - 0x31, 0x65, 0xc4, 0x99, 0x57, 0xf9, 0xf0, 0xeb, 0xf3, 0x39, 0xde, 0x76, 0xb5, 0xb5, 0xcb, 0xc8, - 0x83, 0x58, 0xb0, 0x89, 0x2b, 0x19, 0xd0, 0x17, 0x50, 0xd6, 0xe5, 0xea, 0x54, 0x94, 0x3f, 0x6b, - 0x39, 0x2e, 0x53, 0x28, 0xad, 0xde, 0x93, 0x2d, 0x12, 0xe1, 0x2d, 0x35, 0xcd, 0x78, 0x62, 0x40, - 0x68, 0x0f, 0xca, 0x2a, 0x45, 0xb9, 0x53, 0x55, 0xae, 0xdc, 0xbd, 0x92, 0x2b, 0x2a, 0x5b, 0xb9, - 0xf2, 0x46, 0xf1, 0x5a, 0xae, 0x61, 0x43, 0xf7, 0xe0, 0xa7, 0x7c, 0x44, 0x12, 0x6f, 0x4c, 0x38, - 0x27, 0xf1, 0xd0, 0xdb, 0xa7, 0x0c, 0x93, 0x61, 0xec, 0x8d, 0xf0, 0x84, 0x3b, 0xb0, 0x6e, 0x35, - 0x2b, 0xc6, 0x91, 0x4f, 0xe4, 0xb4, 0x6f, 0xf4, 0xac, 0x2d, 0x3d, 0xe9, 0x11, 0x9e, 0x70, 0x74, - 0x13, 0x16, 0x9e, 0xfb, 0x51, 0x24, 0xf3, 0xfa, 0xb1, 0x1f, 0x53, 0xee, 0xd4, 0x72, 0xb9, 0x7b, - 0xda, 0x84, 0x6e, 0xc3, 0x32, 0x53, 0x25, 0xb3, 0xe3, 0x33, 0x3f, 0x8a, 0x70, 0x44, 0xf8, 0xd8, - 0x59, 0xc8, 0x1d, 0xe1, 0xdb, 0x66, 0xf4, 0x1d, 0x00, 0xc3, 0x3c, 0x1d, 0x63, 0x2f, 0xa1, 0xdc, - 0x59, 0x54, 0x9b, 0xff, 0xdd, 0x95, 0x36, 0xef, 0x2a, 0xf8, 0x0e, 0xd5, 0xfb, 0x77, 0xab, 0x2c, - 0xfb, 0x3f, 0xc2, 0x00, 0x29, 0xc7, 0xcc, 0x53, 0xe2, 0xe4, 0x2c, 0xad, 0x5b, 0xcd, 0x6a, 0x67, - 0xcb, 0x54, 0xea, 0x97, 0x43, 0x22, 0x0e, 0xd2, 0x41, 0x2b, 0xa0, 0xe3, 0xf6, 0x74, 0xb5, 0x70, - 0x70, 0xf2, 0xdd, 0x4e, 0x46, 0xc3, 0x36, 0xc7, 0x41, 0xca, 0x88, 0x98, 0xb4, 0xfa, 0x7f, 0xfc, - 0x7a, 0x97, 0x63, 0x16, 0xfb, 0x63, 0xbc, 0x23, 0xd9, 0xdc, 0xaa, 0x64, 0x56, 0x9f, 0x2b, 0x29, - 0xd4, 0xb4, 0x4b, 0xea, 0x18, 0xd0, 0x1f, 0xc0, 0x96, 0xea, 0xac, 0x2a, 0xe8, 0x6a, 0x3a, 0x65, - 0xb9, 0x0a, 0x89, 0x3e, 0x03, 0x10, 0x3e, 0x1b, 0x62, 0xd1, 0xa5, 0x11, 0x77, 0x0a, 0xeb, 0xc5, - 0x66, 0xd5, 0xd8, 0x73, 0xe3, 0x2b, 0x1c, 0x6a, 0xb9, 0x73, 0x47, 0x75, 0x28, 0x8e, 0xf0, 0x44, - 0xad, 0x5a, 0x75, 0xe5, 0x27, 0x7a, 0x0c, 0xa5, 0x23, 0x3f, 0x4a, 0x33, 0xc5, 0xbc, 0x5a, 0x4a, - 0xe5, 0x76, 0xe4, 0x6a, 0x9a, 0xdf, 0x16, 0xee, 0x5a, 0x2b, 0xbf, 0x81, 0x4a, 0x96, 0xf7, 0xf9, - 0x15, 0x4b, 0x7a, 0xc5, 0xeb, 0xf9, 0x15, 0xab, 0x79, 0xdc, 0xef, 0x61, 0xf1, 0xf4, 0x39, 0xbd, - 0x0b, 0x5d, 0xcc, 0xa1, 0xb7, 0xed, 0x8a, 0xa5, 0x14, 0xab, 0x58, 0xb7, 0xb7, 0xed, 0x8a, 0x5d, - 0x2f, 0x6d, 0xdb, 0x95, 0x52, 0xbd, 0xbc, 0x6d, 0x57, 0xae, 0xd5, 0x17, 0x1a, 0xff, 0x9e, 0xd7, - 0x5d, 0x27, 0x4d, 0xa6, 0x32, 0x72, 0x27, 0x13, 0x7a, 0x4b, 0x25, 0xd4, 0x4f, 0x66, 0x14, 0xe3, - 0xdb, 0xb2, 0xfe, 0x15, 0xd4, 0x49, 0x2c, 0x18, 0x0d, 0xd3, 0x00, 0x87, 0x9e, 0xc6, 0x17, 0x2e, - 0x83, 0x5f, 0x3a, 0x81, 0xf5, 0x15, 0xd3, 0x1d, 0xa8, 0x85, 0x78, 0xdf, 0x4f, 0x23, 0xe1, 0x49, - 0x75, 0x29, 0xaa, 0xcc, 0x43, 0x46, 0x4b, 0x61, 0x53, 0x9b, 0x76, 0xdd, 0x9e, 0x0b, 0x66, 0xda, - 0x2e, 0x23, 0xe8, 0xaf, 0x16, 0x7c, 0x94, 0x32, 0xc2, 0xbd, 0xc1, 0xc4, 0x8b, 0x68, 0xe0, 0x47, - 0x44, 0x4c, 0xbc, 0xd1, 0x91, 0x63, 0x2b, 0x17, 0xbe, 0xbc, 0xb8, 0x73, 0x9e, 0xec, 0x5d, 0xea, - 0x12, 0xef, 0x4c, 0xbe, 0x36, 0x0c, 0x8f, 0x8e, 0xb4, 0x2c, 0x5c, 0x3f, 0x7e, 0xbd, 0x56, 0xdf, - 0x75, 0x7b, 0x79, 0xd3, 0x9e, 0x5b, 0x4f, 0xcf, 0x4c, 0x46, 0x2e, 0xd4, 0xc6, 0x47, 0x41, 0xe0, - 0xed, 0x93, 0x48, 0x60, 0xa6, 0x3a, 0xd5, 0xe2, 0xa9, 0x36, 0x94, 0xed, 0xff, 0x9b, 0xbd, 0x6e, - 0x77, 0x4b, 0x4d, 0x3a, 0xd9, 0xd9, 0xc9, 0x98, 0x0b, 0x92, 0x45, 0x7f, 0xa3, 0xaf, 0x00, 0x70, - 0x1c, 0xb0, 0x49, 0xa2, 0xba, 0x85, 0xd6, 0xeb, 0xe6, 0x0c, 0x4a, 0xa9, 0x8e, 0x0f, 0xa6, 0x13, - 0x9f, 0xa8, 0x5f, 0xee, 0xe6, 0xb0, 0xe8, 0x09, 0x2c, 0x0f, 0xd4, 0x6e, 0x3d, 0x2e, 0x7c, 0x26, - 0x3c, 0xa9, 0x3d, 0x57, 0x69, 0x95, 0x4b, 0x1a, 0xdd, 0x97, 0x60, 0x69, 0x43, 0x8f, 0xc0, 0x0c, - 0x79, 0x38, 0x0e, 0x35, 0x5d, 0xe5, 0xf2, 0x74, 0x0b, 0x1a, 0xfb, 0x20, 0x0e, 0x15, 0xd9, 0x2e, - 0x94, 0x93, 0x91, 0x47, 0xc2, 0x4c, 0xc4, 0xef, 0x5c, 0xfa, 0xcc, 0x76, 0x46, 0xbd, 0xd0, 0xe8, - 0x77, 0x55, 0xb6, 0xdb, 0x9d, 0x47, 0xbd, 0x4d, 0xee, 0x96, 0x12, 0x39, 0x7c, 0x46, 0xc6, 0xe0, - 0x43, 0xc9, 0x58, 0x17, 0x3e, 0x9e, 0x99, 0x3a, 0x33, 0x94, 0xe5, 0xfc, 0x3a, 0xbf, 0x0b, 0x70, - 0xb2, 0x97, 0x3c, 0xd2, 0x9e, 0x81, 0xac, 0xe4, 0x90, 0x8d, 0x7f, 0x59, 0x50, 0x77, 0x31, 0x17, - 0x94, 0x61, 0x59, 0x44, 0x9a, 0xe0, 0x73, 0xb0, 0x65, 0x1d, 0x1a, 0x2d, 0x7d, 0x47, 0x19, 0xaa, - 0xa9, 0xe8, 0x3e, 0x94, 0xf6, 0x89, 0x6c, 0xa4, 0xba, 0x74, 0x37, 0x66, 0xf5, 0x61, 0xa5, 0x6d, - 0x2e, 0x3e, 0x4c, 0x31, 0x17, 0x2a, 0xeb, 0x32, 0x21, 0x50, 0x48, 0x74, 0x03, 0x6a, 0xd9, 0x05, - 0xa1, 0x17, 0xbe, 0x50, 0xe5, 0x9b, 0x75, 0xbc, 0xbc, 0xa1, 0xf1, 0x97, 0x22, 0x2c, 0x19, 0x97, - 0xa7, 0xca, 0xb3, 0x05, 0xd7, 0x98, 0x1e, 0xd2, 0xd9, 0x64, 0x5d, 0x3e, 0x9b, 0x6a, 0x06, 0xa8, - 0x72, 0xe9, 0x74, 0xcd, 0x14, 0xde, 0xa3, 0x66, 0x7a, 0x50, 0x66, 0x58, 0xf5, 0x7b, 0x7d, 0xeb, - 0xfd, 0xe5, 0x3b, 0x23, 0x62, 0x2e, 0xbf, 0x23, 0x3c, 0xc9, 0x6e, 0x29, 0x9a, 0x40, 0xde, 0x52, - 0x4c, 0x82, 0x6b, 0x51, 0xfa, 0xd5, 0x45, 0x2d, 0xe5, 0x54, 0x5c, 0x2e, 0xcc, 0xf0, 0xf7, 0xc8, - 0x9a, 0x7f, 0x14, 0xe0, 0xa3, 0x7e, 0x12, 0x11, 0x71, 0x3f, 0x0e, 0xfb, 0x81, 0x2f, 0x84, 0x79, - 0x76, 0xfc, 0x19, 0xca, 0xea, 0x5e, 0x9d, 0x75, 0x80, 0x7b, 0xe7, 0x7b, 0x3a, 0x03, 0x9e, 0x79, - 0xaf, 0xfc, 0xe9, 0x4a, 0x9e, 0x2c, 0x10, 0x9a, 0x34, 0x17, 0xd3, 0xc2, 0x7b, 0xc6, 0x74, 0xc5, - 0x83, 0xe5, 0xb7, 0x56, 0x43, 0xdb, 0x30, 0x8f, 0xe5, 0x35, 0x1a, 0x67, 0xfe, 0xdf, 0x7c, 0x67, - 0xa4, 0xa7, 0x45, 0x63, 0xf8, 0x33, 0x82, 0xc6, 0xdf, 0x8a, 0xb0, 0xd0, 0xed, 0xef, 0x7d, 0xcb, - 0x48, 0x16, 0x9c, 0x1b, 0xb2, 0x3d, 0x71, 0x41, 0x62, 0xfd, 0x84, 0x51, 0x85, 0x9d, 0xe5, 0x60, - 0xce, 0x80, 0x7e, 0x0e, 0xd7, 0xa4, 0x52, 0x78, 0x89, 0x0a, 0x8c, 0xce, 0xc2, 0xe9, 0x44, 0xa5, - 0x21, 0xda, 0x80, 0xbe, 0x80, 0x79, 0xaa, 0x33, 0x4f, 0x15, 0x4b, 0x6d, 0x66, 0xc3, 0xe8, 0xf6, - 0xf7, 0x4c, 0x7a, 0x66, 0x1e, 0x1a, 0xcc, 0xc9, 0xe3, 0x88, 0xd1, 0xe7, 0xdc, 0xbc, 0xa8, 0xf2, - 0x8f, 0x23, 0x97, 0x3e, 0xe7, 0xe8, 0x4f, 0xb0, 0x1c, 0xd0, 0x71, 0x22, 0x6b, 0x8f, 0xd0, 0xd8, - 0x0b, 0x68, 0x88, 0x03, 0xd3, 0x9e, 0x2e, 0x78, 0xc7, 0xc9, 0xf2, 0xe8, 0x9e, 0xc0, 0x0c, 0x6d, - 0x3d, 0xc7, 0xd4, 0x95, 0x44, 0x67, 0x34, 0xb6, 0xfc, 0x81, 0x34, 0xb6, 0xf1, 0x2d, 0x2c, 0x77, - 0xd2, 0x48, 0x6e, 0x28, 0x77, 0x1c, 0xd3, 0x97, 0xad, 0xf5, 0xa3, 0x5f, 0xb6, 0x37, 0x37, 0x60, - 0xe9, 0xcc, 0x56, 0x51, 0x05, 0xec, 0xc7, 0x34, 0xc6, 0xf5, 0x39, 0xf9, 0xf5, 0xf0, 0x25, 0x49, - 0xea, 0x56, 0xe7, 0xd6, 0xab, 0xff, 0xae, 0xce, 0xbd, 0x3a, 0x5e, 0xb5, 0xbe, 0x3f, 0x5e, 0xb5, - 0x7e, 0x38, 0x5e, 0xb5, 0xfe, 0x73, 0xbc, 0x6a, 0xfd, 0xfd, 0xcd, 0xea, 0xdc, 0xf7, 0x6f, 0x56, - 0xe7, 0x7e, 0x78, 0xb3, 0x3a, 0xf7, 0x5d, 0x2d, 0xf7, 0xc7, 0x83, 0xff, 0x07, 0x00, 0x00, 0xff, - 0xff, 0x3b, 0xa7, 0xba, 0x6f, 0xe9, 0x10, 0x00, 0x00, + proto.RegisterFile("sql/execinfrapb/processors_bulk_io.proto", fileDescriptor_processors_bulk_io_a1c1a7c66115c204) +} + +var fileDescriptor_processors_bulk_io_a1c1a7c66115c204 = []byte{ + // 1747 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x58, 0x4f, 0x6f, 0x1b, 0xb9, + 0x15, 0xf7, 0xe8, 0x9f, 0xa5, 0xa7, 0xd8, 0x1e, 0x73, 0xb3, 0xbb, 0x53, 0x03, 0xb5, 0x0d, 0xed, + 0x3a, 0x55, 0x53, 0x44, 0xc2, 0x26, 0x6d, 0x11, 0xb4, 0xdd, 0x4d, 0x63, 0x39, 0xce, 0xca, 0xd9, + 0x4d, 0xdc, 0x51, 0xec, 0x05, 0xb6, 0x2d, 0x06, 0xd4, 0x0c, 0x2d, 0x33, 0x1a, 0x0d, 0xc7, 0x24, + 0xc7, 0x89, 0x72, 0x2a, 0xd0, 0x4b, 0x7b, 0xeb, 0x47, 0xe8, 0xa1, 0xf7, 0xf6, 0x3b, 0xf4, 0x92, + 0xe3, 0x1e, 0x17, 0x3d, 0x18, 0xad, 0xf3, 0x2d, 0x7a, 0x2a, 0x48, 0xce, 0x48, 0x63, 0xc7, 0x76, + 0xec, 0x06, 0xb9, 0x38, 0x23, 0x3e, 0xfe, 0x7e, 0x7c, 0x7c, 0x7c, 0xef, 0xf7, 0xc8, 0x40, 0x53, + 0x1c, 0x84, 0x6d, 0xf2, 0x82, 0xf8, 0x34, 0xda, 0xe3, 0x38, 0xee, 0xb7, 0x63, 0xce, 0x7c, 0x22, + 0x04, 0xe3, 0xc2, 0xeb, 0x27, 0xe1, 0xd0, 0xa3, 0xac, 0x15, 0x73, 0x26, 0x19, 0x72, 0x7c, 0xe6, + 0x0f, 0x39, 0xc3, 0xfe, 0x7e, 0x4b, 0x1c, 0x84, 0xad, 0x80, 0x0a, 0x29, 0x0e, 0x42, 0x9e, 0x44, + 0x4b, 0x1f, 0x3d, 0x63, 0x7d, 0xd1, 0x56, 0x7f, 0xe2, 0xbe, 0xfe, 0xc7, 0x20, 0x96, 0x1c, 0x3d, + 0x3b, 0xee, 0xb7, 0x29, 0xbb, 0xb5, 0xc7, 0xf8, 0x08, 0xcb, 0xcc, 0xf2, 0x89, 0x5a, 0xd5, 0xc7, + 0x12, 0x87, 0x6c, 0xd0, 0x0e, 0x88, 0xf0, 0xe3, 0x7e, 0x5b, 0x48, 0x9e, 0xf8, 0x32, 0xe1, 0x24, + 0x48, 0x27, 0xad, 0x5d, 0xe4, 0x1a, 0x16, 0x24, 0x5b, 0x25, 0x91, 0x34, 0x6c, 0xef, 0x87, 0x7e, + 0x5b, 0xd2, 0x11, 0x11, 0x12, 0x8f, 0xe2, 0xd4, 0x72, 0x7d, 0xc0, 0x06, 0x4c, 0x7f, 0xb6, 0xd5, + 0x57, 0x3a, 0x8a, 0x32, 0xaf, 0x02, 0x2c, 0x71, 0x3a, 0xb6, 0x98, 0x8d, 0xe1, 0x98, 0x9a, 0xa1, + 0xc6, 0xdf, 0x8b, 0x30, 0xbf, 0x8e, 0xfd, 0xe1, 0x1e, 0x0d, 0x43, 0xc2, 0x7b, 0x31, 0xf1, 0xd1, + 0x43, 0x28, 0xc9, 0x71, 0x4c, 0x1c, 0x6b, 0xd5, 0x6a, 0xce, 0xdf, 0xbe, 0xd5, 0x3a, 0x2f, 0x20, + 0xad, 0x93, 0xb8, 0xd6, 0xd3, 0x71, 0x4c, 0xd6, 0x4b, 0xaf, 0x8e, 0x56, 0x66, 0x5c, 0x4d, 0x80, + 0xd6, 0xa1, 0x2c, 0x71, 0x3f, 0x24, 0x4e, 0x61, 0xd5, 0x6a, 0xd6, 0x6f, 0xdf, 0x38, 0xc5, 0x24, + 0x0e, 0x42, 0xbd, 0xbf, 0xa7, 0x6a, 0xce, 0x06, 0x11, 0x3e, 0xa7, 0xb1, 0x64, 0x3c, 0xa5, 0x30, + 0x50, 0xf4, 0x00, 0xca, 0x22, 0xc6, 0x91, 0x70, 0x8a, 0xab, 0xc5, 0x66, 0xfd, 0xf6, 0x8f, 0xcf, + 0xf7, 0x46, 0xd3, 0xb8, 0x04, 0x07, 0xca, 0x1d, 0x1c, 0x65, 0x34, 0x1a, 0x8d, 0x3e, 0x83, 0x6a, + 0x90, 0x70, 0x2c, 0x29, 0x8b, 0x9c, 0xd2, 0xaa, 0xd5, 0x2c, 0xae, 0x7f, 0xa8, 0xcc, 0xff, 0x3d, + 0x5a, 0x99, 0x53, 0xe1, 0x6c, 0x6d, 0xa4, 0x46, 0x77, 0x32, 0x0d, 0x7d, 0x02, 0xe0, 0xef, 0x27, + 0xd1, 0xd0, 0x13, 0xf4, 0x25, 0x71, 0xca, 0x1a, 0x64, 0x38, 0x6b, 0x7a, 0xbc, 0x47, 0x5f, 0x12, + 0x74, 0x0f, 0xaa, 0x9c, 0xe0, 0xe0, 0xbe, 0x78, 0xb2, 0xe7, 0xcc, 0xea, 0x5d, 0xfe, 0x30, 0xe7, + 0xa1, 0x3a, 0xb2, 0xd6, 0x7e, 0xe8, 0xb7, 0x9e, 0x66, 0x47, 0x96, 0x32, 0x4c, 0x40, 0x8d, 0x9b, + 0x50, 0x52, 0x71, 0x43, 0x75, 0x98, 0xed, 0x46, 0x87, 0x38, 0xa4, 0x81, 0x3d, 0x83, 0x00, 0x2a, + 0x1d, 0x16, 0x26, 0xa3, 0xc8, 0xb6, 0x50, 0x0d, 0xca, 0xdd, 0x28, 0x20, 0x2f, 0xec, 0xc2, 0x56, + 0xa9, 0x5a, 0xb1, 0x67, 0x1b, 0xcf, 0xa1, 0xbe, 0xc5, 0xfa, 0xdb, 0x9c, 0x0d, 0x38, 0x11, 0x02, + 0x7d, 0x0a, 0x95, 0x67, 0xac, 0xef, 0xd1, 0x40, 0x9f, 0x57, 0x71, 0x7d, 0x4e, 0x2d, 0x70, 0x7c, + 0xb4, 0x52, 0xde, 0x62, 0xfd, 0xee, 0x86, 0x5b, 0x7e, 0xc6, 0xfa, 0xdd, 0x00, 0x35, 0xe1, 0x9a, + 0xcf, 0x22, 0xc9, 0x69, 0x3f, 0xd1, 0x31, 0x50, 0x27, 0x52, 0x48, 0x9d, 0x39, 0x61, 0x41, 0x0e, + 0x94, 0x44, 0xc8, 0xa4, 0x53, 0x5c, 0xb5, 0x9a, 0xe5, 0xec, 0x38, 0xd5, 0x48, 0xe3, 0x55, 0x15, + 0x90, 0x8a, 0x6f, 0x77, 0x14, 0x33, 0x2e, 0x37, 0xb0, 0xc4, 0x3a, 0x5d, 0xd6, 0xa0, 0x2e, 0xf0, + 0x28, 0x0e, 0x89, 0x09, 0x54, 0x21, 0x87, 0x03, 0x63, 0xd0, 0x91, 0x7a, 0x08, 0xd5, 0x38, 0xf5, + 0xd9, 0xa9, 0xe8, 0x48, 0xad, 0x9d, 0x7f, 0x96, 0xb9, 0x0d, 0x66, 0x11, 0xcb, 0xc0, 0xe8, 0x21, + 0x14, 0x13, 0x4e, 0x9d, 0x59, 0x9d, 0x0f, 0x3f, 0x3b, 0x9f, 0xe3, 0x4d, 0x57, 0x5b, 0x3b, 0x9c, + 0x3e, 0x88, 0x24, 0x1f, 0xbb, 0x8a, 0x01, 0x7d, 0x0e, 0x15, 0x53, 0xae, 0x4e, 0x55, 0xfb, 0xb3, + 0x92, 0xe3, 0x4a, 0x0b, 0xa5, 0xd5, 0x7d, 0xb2, 0x49, 0x43, 0xb2, 0xa9, 0xa7, 0xa5, 0x9e, 0xa4, + 0x20, 0xb4, 0x0b, 0x15, 0x9d, 0xa2, 0xc2, 0xa9, 0x69, 0x57, 0xee, 0x5e, 0xc9, 0x15, 0x9d, 0xad, + 0x42, 0x7b, 0xa3, 0x79, 0x2d, 0x37, 0x65, 0x43, 0xf7, 0xe0, 0x07, 0x62, 0x48, 0x63, 0x6f, 0x44, + 0x85, 0xa0, 0xd1, 0xc0, 0xdb, 0x63, 0x9c, 0xd0, 0x41, 0xe4, 0x0d, 0xc9, 0x58, 0x38, 0xb0, 0x6a, + 0x35, 0xab, 0xa9, 0x23, 0x1f, 0xa9, 0x69, 0x5f, 0x9b, 0x59, 0x9b, 0x66, 0xd2, 0x23, 0x32, 0x16, + 0xe8, 0x26, 0xcc, 0x3d, 0xc7, 0x61, 0xa8, 0xf2, 0xfa, 0x31, 0x8e, 0x98, 0x70, 0xea, 0xb9, 0xdc, + 0x3d, 0x69, 0x42, 0xb7, 0x61, 0x91, 0xeb, 0x92, 0xd9, 0xc6, 0x1c, 0x87, 0x21, 0x09, 0xa9, 0x18, + 0x39, 0x73, 0xb9, 0x23, 0x7c, 0xd3, 0x8c, 0xbe, 0x05, 0xe0, 0x44, 0x24, 0x23, 0xe2, 0xc5, 0x4c, + 0x38, 0xf3, 0x7a, 0xf3, 0xbf, 0xbc, 0xd2, 0xe6, 0x5d, 0x0d, 0xdf, 0x66, 0x66, 0xff, 0x6e, 0x8d, + 0x67, 0xbf, 0x11, 0x01, 0x48, 0x04, 0xe1, 0x9e, 0x16, 0x27, 0x67, 0x61, 0xd5, 0x6a, 0xd6, 0xd6, + 0x37, 0xd3, 0x4a, 0xfd, 0x62, 0x40, 0xe5, 0x7e, 0xd2, 0x6f, 0xf9, 0x6c, 0xd4, 0x9e, 0xac, 0x16, + 0xf4, 0xa7, 0xdf, 0xed, 0x78, 0x38, 0x68, 0x0b, 0xe2, 0x27, 0x9c, 0xca, 0x71, 0xab, 0xf7, 0x9b, + 0xaf, 0x76, 0x04, 0xe1, 0x11, 0x1e, 0x91, 0x6d, 0xc5, 0xe6, 0xd6, 0x14, 0xb3, 0xfe, 0x5c, 0x4a, + 0xa0, 0x6e, 0x5c, 0xd2, 0xc7, 0x80, 0x7e, 0x0d, 0x25, 0xa5, 0xce, 0xba, 0x82, 0xae, 0xa6, 0x53, + 0x96, 0xab, 0x91, 0xe8, 0x53, 0x00, 0x89, 0xf9, 0x80, 0xc8, 0x0e, 0x0b, 0x85, 0x53, 0x58, 0x2d, + 0x36, 0x6b, 0xa9, 0x3d, 0x37, 0xbe, 0x24, 0xa0, 0x9e, 0x3b, 0x77, 0x64, 0x43, 0x71, 0x48, 0xc6, + 0x7a, 0xd5, 0x9a, 0xab, 0x3e, 0xd1, 0x63, 0x28, 0x1f, 0xe2, 0x30, 0xc9, 0x14, 0xf3, 0x6a, 0x29, + 0x95, 0xdb, 0x91, 0x6b, 0x68, 0x7e, 0x51, 0xb8, 0x6b, 0x2d, 0xfd, 0x1c, 0xaa, 0x59, 0xde, 0xe7, + 0x57, 0x2c, 0x9b, 0x15, 0xaf, 0xe7, 0x57, 0xac, 0xe5, 0x71, 0xbf, 0x82, 0xf9, 0x93, 0xe7, 0xf4, + 0x36, 0x74, 0x31, 0x87, 0xde, 0x2a, 0x55, 0x2d, 0xad, 0x58, 0x45, 0xbb, 0xb4, 0x55, 0xaa, 0x96, + 0xec, 0xf2, 0x56, 0xa9, 0x5a, 0xb6, 0x2b, 0x5b, 0xa5, 0xea, 0x35, 0x7b, 0xae, 0xf1, 0x37, 0x0b, + 0x3e, 0xee, 0x49, 0x4e, 0xf0, 0xa8, 0x1b, 0x0d, 0x88, 0x50, 0xc2, 0x33, 0xd1, 0x93, 0x3f, 0x59, + 0xb0, 0x18, 0x63, 0x2e, 0xa9, 0x1a, 0xf5, 0x70, 0x10, 0x68, 0xc9, 0xb0, 0x74, 0x48, 0x7f, 0xab, + 0x52, 0xe1, 0x5f, 0x47, 0x2b, 0xbd, 0x4b, 0xa5, 0x82, 0xef, 0x87, 0xaa, 0xdd, 0x12, 0x3c, 0xa2, + 0xd1, 0x60, 0xfa, 0xc3, 0x0f, 0x29, 0x89, 0x64, 0x6b, 0x3b, 0x5b, 0xe3, 0xbe, 0x59, 0xc2, 0xb5, + 0xe3, 0x53, 0x23, 0x8d, 0x7f, 0xce, 0x9a, 0xe6, 0x98, 0xc4, 0x13, 0xef, 0xee, 0x64, 0xfd, 0xc8, + 0xd2, 0x79, 0xff, 0xf1, 0x19, 0x9a, 0xf1, 0x66, 0xf7, 0xf9, 0x12, 0x6c, 0x1a, 0x49, 0xce, 0x82, + 0xc4, 0x27, 0x81, 0x67, 0xf0, 0x85, 0xcb, 0xe0, 0x17, 0xa6, 0xb0, 0x9e, 0x66, 0xba, 0x03, 0xf5, + 0x80, 0xec, 0xe1, 0x24, 0x94, 0x9e, 0x12, 0xc1, 0xa2, 0x2e, 0x10, 0x94, 0x4a, 0x3e, 0x6c, 0x18, + 0xd3, 0x8e, 0xdb, 0x75, 0x21, 0x9d, 0xb6, 0xc3, 0x29, 0xfa, 0xa3, 0x05, 0x1f, 0x24, 0x9c, 0x0a, + 0xaf, 0x3f, 0xf6, 0x42, 0xe6, 0xe3, 0x90, 0xca, 0xb1, 0x37, 0x3c, 0x74, 0x4a, 0xda, 0x85, 0x2f, + 0x2e, 0x6e, 0xf0, 0xd3, 0xbd, 0x2b, 0xf9, 0x14, 0xeb, 0xe3, 0xaf, 0x52, 0x86, 0x47, 0x87, 0x46, + 0xbd, 0xae, 0x1f, 0x1f, 0xad, 0xd8, 0x3b, 0x6e, 0x37, 0x6f, 0xda, 0x75, 0xed, 0xe4, 0xd4, 0x64, + 0xe4, 0x42, 0x7d, 0x74, 0xe8, 0xfb, 0xde, 0x1e, 0x0d, 0x25, 0xe1, 0xba, 0xa1, 0xce, 0x9f, 0xe8, + 0x96, 0xd9, 0xfe, 0xbf, 0xde, 0xed, 0x74, 0x36, 0xf5, 0xa4, 0xe9, 0xce, 0xa6, 0x63, 0x2e, 0x28, + 0x16, 0xf3, 0x8d, 0xbe, 0x04, 0x20, 0x91, 0xcf, 0xc7, 0xb1, 0x6e, 0x6a, 0xa6, 0xad, 0x34, 0xcf, + 0xa0, 0x54, 0x22, 0xfe, 0x60, 0x32, 0xf1, 0x89, 0xfe, 0x2b, 0xdc, 0x1c, 0x16, 0x3d, 0x81, 0xc5, + 0xbe, 0xde, 0xad, 0x27, 0x24, 0xe6, 0xd2, 0x53, 0x12, 0x79, 0x95, 0x8e, 0xbe, 0x60, 0xd0, 0x3d, + 0x05, 0x56, 0x36, 0xf4, 0x08, 0xd2, 0x21, 0x8f, 0x44, 0x81, 0xa1, 0xab, 0x5e, 0x9e, 0x6e, 0xce, + 0x60, 0x1f, 0x44, 0x81, 0x26, 0xdb, 0x81, 0x4a, 0x3c, 0xf4, 0x68, 0x90, 0xf5, 0x9a, 0x3b, 0x97, + 0x3e, 0xb3, 0xed, 0x61, 0x37, 0x48, 0xdb, 0x4c, 0x4d, 0xdd, 0x0a, 0xb6, 0x1f, 0x75, 0x37, 0x84, + 0x5b, 0x8e, 0xd5, 0xf0, 0x29, 0xb5, 0x85, 0xf7, 0xa5, 0xb6, 0x1d, 0xf8, 0xf0, 0xcc, 0xd4, 0x39, + 0x43, 0x00, 0xcf, 0x97, 0xa3, 0xbb, 0x00, 0xd3, 0xbd, 0xe4, 0x91, 0xa5, 0x33, 0x90, 0xd5, 0x1c, + 0xb2, 0xf1, 0x0f, 0x0b, 0x6c, 0x97, 0x08, 0xc9, 0x38, 0x51, 0x45, 0x64, 0x08, 0x3e, 0x83, 0x92, + 0xaa, 0xc3, 0x54, 0xf2, 0xdf, 0x52, 0x86, 0x7a, 0x2a, 0xba, 0x0f, 0xe5, 0x3d, 0xaa, 0xfa, 0xbd, + 0x29, 0xdd, 0xb5, 0xb3, 0xae, 0x0b, 0x5a, 0x82, 0x5d, 0x72, 0x90, 0x10, 0x21, 0x75, 0xd6, 0x65, + 0x42, 0xa0, 0x91, 0xe8, 0x06, 0xd4, 0xb3, 0x7b, 0x4c, 0x37, 0x78, 0xa1, 0xcb, 0x37, 0x6b, 0xcc, + 0x79, 0x43, 0xe3, 0x0f, 0x45, 0x58, 0x48, 0x5d, 0x9e, 0x28, 0xcf, 0x26, 0x5c, 0xe3, 0x66, 0xc8, + 0x64, 0x93, 0x75, 0xf9, 0x6c, 0xaa, 0xa7, 0x40, 0x9d, 0x4b, 0x27, 0x6b, 0xa6, 0xf0, 0x0e, 0x35, + 0xd3, 0x85, 0x0a, 0x27, 0xfa, 0x5a, 0x62, 0x2e, 0xe7, 0x3f, 0x79, 0x6b, 0x44, 0xd2, 0x3b, 0xfa, + 0x90, 0x8c, 0xb3, 0xcb, 0x94, 0x21, 0x50, 0x97, 0xa9, 0x34, 0xc1, 0x8d, 0x28, 0xfd, 0xf4, 0xa2, + 0xce, 0x77, 0x22, 0x2e, 0x17, 0x66, 0xf8, 0x3b, 0x64, 0xcd, 0x5f, 0x0b, 0xf0, 0x41, 0x2f, 0x0e, + 0xa9, 0xbc, 0x1f, 0x05, 0x3d, 0x1f, 0x4b, 0x99, 0xbe, 0x8e, 0x7e, 0x0f, 0x15, 0x7d, 0xfd, 0xcf, + 0x3a, 0xc0, 0xbd, 0xf3, 0x3d, 0x3d, 0x03, 0x9e, 0x79, 0xaf, 0xfd, 0xe9, 0x28, 0x9e, 0x2c, 0x10, + 0x86, 0x34, 0x17, 0xd3, 0xc2, 0x3b, 0xc6, 0x74, 0xc9, 0x83, 0xc5, 0x37, 0x56, 0x43, 0x5b, 0x30, + 0x4b, 0xd4, 0x6d, 0x9f, 0x64, 0xfe, 0xdf, 0x7c, 0x6b, 0xa4, 0x27, 0x45, 0x93, 0xf2, 0x67, 0x04, + 0x8d, 0x3f, 0x17, 0x61, 0xae, 0xd3, 0xdb, 0xfd, 0x86, 0xd3, 0x2c, 0x38, 0x37, 0x54, 0x7b, 0x12, + 0x92, 0x46, 0xe6, 0xa5, 0xa5, 0x0b, 0x3b, 0xcb, 0xc1, 0x9c, 0x01, 0xfd, 0x08, 0xae, 0x29, 0xa5, + 0xf0, 0x62, 0x1d, 0x18, 0x93, 0x85, 0x93, 0x89, 0x5a, 0x43, 0x8c, 0x01, 0x7d, 0x0e, 0xb3, 0xcc, + 0x64, 0x9e, 0x2e, 0x96, 0xfa, 0x99, 0x0d, 0xa3, 0xd3, 0xdb, 0x4d, 0xd3, 0x33, 0xf3, 0x30, 0xc5, + 0x4c, 0xdf, 0x70, 0x9c, 0x3d, 0x17, 0xe9, 0xc3, 0x2f, 0xff, 0x86, 0x73, 0xd9, 0x73, 0x81, 0x7e, + 0x07, 0x8b, 0x3e, 0x1b, 0xc5, 0xaa, 0xf6, 0xd4, 0x8d, 0xc3, 0x67, 0x01, 0xf1, 0xd3, 0xf6, 0x74, + 0xc1, 0x73, 0x53, 0x95, 0x47, 0x67, 0x0a, 0x4b, 0x69, 0xed, 0x1c, 0x53, 0x47, 0x11, 0x9d, 0xd2, + 0xd8, 0xca, 0x7b, 0xd2, 0xd8, 0xc6, 0x37, 0xb0, 0xb8, 0x9e, 0x84, 0x6a, 0x43, 0xb9, 0xe3, 0x98, + 0x3c, 0xc0, 0xad, 0xff, 0xfb, 0x01, 0x7e, 0x73, 0x0d, 0x16, 0x4e, 0x6d, 0x15, 0x55, 0xa1, 0xf4, + 0x98, 0x45, 0xc4, 0x9e, 0x51, 0x5f, 0x0f, 0x5f, 0xd2, 0xd8, 0xb6, 0xd6, 0x6f, 0xbd, 0xfa, 0xcf, + 0xf2, 0xcc, 0xab, 0xe3, 0x65, 0xeb, 0xbb, 0xe3, 0x65, 0xeb, 0xfb, 0xe3, 0x65, 0xeb, 0xdf, 0xc7, + 0xcb, 0xd6, 0x5f, 0x5e, 0x2f, 0xcf, 0x7c, 0xf7, 0x7a, 0x79, 0xe6, 0xfb, 0xd7, 0xcb, 0x33, 0xdf, + 0xd6, 0x73, 0xff, 0xc7, 0xf1, 0xbf, 0x00, 0x00, 0x00, 0xff, 0xff, 0x51, 0xaf, 0x2a, 0x1e, 0x90, + 0x11, 0x00, 0x00, } diff --git a/pkg/sql/execinfrapb/processors_bulk_io.proto b/pkg/sql/execinfrapb/processors_bulk_io.proto index 0ae146ea4fd2..0f19bf7019e0 100644 --- a/pkg/sql/execinfrapb/processors_bulk_io.proto +++ b/pkg/sql/execinfrapb/processors_bulk_io.proto @@ -134,6 +134,10 @@ message ReadImportDataSpec { // NEXTID: 16 } +message StreamIngestionDataSpec { + repeated string partition_address = 1 [(gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/streamclient.PartitionAddress",(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/ts/catalog/chart_catalog.go b/pkg/ts/catalog/chart_catalog.go index 8dca196c74ea..6234f76741bf 100644 --- a/pkg/ts/catalog/chart_catalog.go +++ b/pkg/ts/catalog/chart_catalog.go @@ -2291,6 +2291,7 @@ var charts = []sectionDescription{ "jobs.schema_change.currently_running", "jobs.schema_change_gc.currently_running", "jobs.typedesc_schema_change.currently_running", + "jobs.stream_ingestion.currently_running", }, }, { @@ -2401,6 +2402,17 @@ var charts = []sectionDescription{ }, Rate: DescribeDerivative_NON_NEGATIVE_DERIVATIVE, }, + { + Title: "Stream Ingestion", + Metrics: []string{ + "jobs.stream_ingestion.fail_or_cancel_completed", + "jobs.stream_ingestion.fail_or_cancel_failed", + "jobs.stream_ingestion.fail_or_cancel_retry_error", + "jobs.stream_ingestion.resume_completed", + "jobs.stream_ingestion.resume_failed", + "jobs.stream_ingestion.resume_retry_error", + }, + }, }, }, }