From ba1a92f4bc33b5bce6ffd90d7ab527b48712b1d7 Mon Sep 17 00:00:00 2001 From: Daniel Harrison Date: Thu, 30 Aug 2018 13:55:21 -0700 Subject: [PATCH 1/2] changefeedccl,jobspb: select initial scan timestamp on gateway Fixes a buglet introduced in #28984 where each node in a distsql changefeed flow would pick its own timestamp to do the initial scan at. I thought I got all the backward-incompatible changes out of the way with #27996 but I didn't forsee this one. While I'm making a backward-incompatible change, switch targets from a `id->string` map to an `id->proto` map for future expansion. Row and partition watches, for example, will need this. Release note (backward-incompatible change): CHANGEFEEDs created with previous betas and alphas will not work with this version. --- pkg/ccl/changefeedccl/changefeed_dist.go | 14 +- .../changefeedccl/changefeed_processors.go | 2 +- pkg/ccl/changefeedccl/changefeed_stmt.go | 28 +- pkg/ccl/changefeedccl/helpers_test.go | 7 +- pkg/ccl/changefeedccl/poller.go | 54 +- pkg/ccl/changefeedccl/sink.go | 15 +- pkg/ccl/changefeedccl/sink_test.go | 7 +- pkg/jobs/jobspb/jobs.pb.go | 557 ++++++++++++------ pkg/jobs/jobspb/jobs.proto | 15 +- pkg/jobs/jobspb/wrap.go | 5 + 10 files changed, 455 insertions(+), 249 deletions(-) diff --git a/pkg/ccl/changefeedccl/changefeed_dist.go b/pkg/ccl/changefeedccl/changefeed_dist.go index 04cb38ff3980..01a6a1a7bd84 100644 --- a/pkg/ccl/changefeedccl/changefeed_dist.go +++ b/pkg/ccl/changefeedccl/changefeed_dist.go @@ -73,13 +73,17 @@ func distChangefeedFlow( return err } - var highWater hlc.Timestamp - if h := progress.GetHighWater(); h != nil { - highWater = *h + spansTS := details.StatementTime + var initialHighWater hlc.Timestamp + if h := progress.GetHighWater(); h != nil && *h != (hlc.Timestamp{}) { + initialHighWater = *h + // If we have a high-water set, use it to compute the spans, since the + // ones at the statement time may have been garbage collected by now. + spansTS = initialHighWater } execCfg := phs.ExecCfg() - trackedSpans, err := fetchSpansForTargets(ctx, execCfg.DB, details.Targets, highWater) + trackedSpans, err := fetchSpansForTargets(ctx, execCfg.DB, details.Targets, spansTS) if err != nil { return err } @@ -107,7 +111,7 @@ func distChangefeedFlow( for i, nodeSpan := range nodeSpans { watches[i] = distsqlrun.ChangeAggregatorSpec_Watch{ Span: nodeSpan, - InitialResolved: highWater, + InitialResolved: initialHighWater, } } diff --git a/pkg/ccl/changefeedccl/changefeed_processors.go b/pkg/ccl/changefeedccl/changefeed_processors.go index a089af1204a9..db9f10363b6b 100644 --- a/pkg/ccl/changefeedccl/changefeed_processors.go +++ b/pkg/ccl/changefeedccl/changefeed_processors.go @@ -115,7 +115,7 @@ func newChangeAggregatorProcessor( buf := makeBuffer() ca.poller = makePoller( flowCtx.Settings, flowCtx.ClientDB, flowCtx.ClientDB.Clock(), flowCtx.Gossip, spans, - spec.Feed.Targets, initialHighWater, buf) + spec.Feed, initialHighWater, buf) rowsFn := kvsToRows(flowCtx.LeaseManager.(*sql.LeaseManager), spec.Feed, buf.Get) ca.tickFn = emitEntries(spec.Feed, ca.sink, rowsFn) diff --git a/pkg/ccl/changefeedccl/changefeed_stmt.go b/pkg/ccl/changefeedccl/changefeed_stmt.go index 5de50d3bc608..aaeab223b834 100644 --- a/pkg/ccl/changefeedccl/changefeed_stmt.go +++ b/pkg/ccl/changefeedccl/changefeed_stmt.go @@ -129,14 +129,15 @@ func changefeedPlanHook( return err } - now := p.ExecCfg().Clock.Now() - var highWater hlc.Timestamp + statementTime := p.ExecCfg().Clock.Now() + var initialHighWater hlc.Timestamp if cursor, ok := opts[optCursor]; ok { asOf := tree.AsOfClause{Expr: tree.NewStrVal(cursor)} var err error - if highWater, err = p.EvalAsOfTimestamp(asOf, now); err != nil { + if initialHighWater, err = p.EvalAsOfTimestamp(asOf, statementTime); err != nil { return err } + statementTime = initialHighWater } // For now, disallow targeting a database or wildcard table selection. @@ -157,32 +158,31 @@ func changefeedPlanHook( } // This grabs table descriptors once to get their ids. - descriptorTime := now - if highWater != (hlc.Timestamp{}) { - descriptorTime = highWater - } targetDescs, _, err := backupccl.ResolveTargetsToDescriptors( - ctx, p, descriptorTime, changefeedStmt.Targets) + ctx, p, statementTime, changefeedStmt.Targets) if err != nil { return err } - targets := make(map[sqlbase.ID]string, len(targetDescs)) + targets := make(jobspb.ChangefeedTargets, len(targetDescs)) for _, desc := range targetDescs { if tableDesc := desc.GetTable(); tableDesc != nil { if err := validateChangefeedTable(tableDesc); err != nil { return err } - targets[tableDesc.ID] = tableDesc.Name + targets[tableDesc.ID] = jobspb.ChangefeedTarget{ + StatementTimeName: tableDesc.Name, + } } } details := jobspb.ChangefeedDetails{ - Targets: targets, - Opts: opts, - SinkURI: sinkURI, + Targets: targets, + Opts: opts, + SinkURI: sinkURI, + StatementTime: statementTime, } progress := jobspb.Progress{ - Progress: &jobspb.Progress_HighWater{HighWater: &highWater}, + Progress: &jobspb.Progress_HighWater{HighWater: &initialHighWater}, Details: &jobspb.Progress_Changefeed{ Changefeed: &jobspb.ChangefeedProgress{}, }, diff --git a/pkg/ccl/changefeedccl/helpers_test.go b/pkg/ccl/changefeedccl/helpers_test.go index dba9dbd06b20..7b9d74aec819 100644 --- a/pkg/ccl/changefeedccl/helpers_test.go +++ b/pkg/ccl/changefeedccl/helpers_test.go @@ -94,7 +94,9 @@ func createBenchmarkChangefeed( tableDesc := sqlbase.GetTableDescriptor(s.DB(), database, table) spans := []roachpb.Span{tableDesc.PrimaryIndexSpan()} details := jobspb.ChangefeedDetails{ - Targets: map[sqlbase.ID]string{tableDesc.ID: tableDesc.Name}, + Targets: jobspb.ChangefeedTargets{tableDesc.ID: jobspb.ChangefeedTarget{ + StatementTimeName: tableDesc.Name, + }}, Opts: map[string]string{ optEnvelope: string(optEnvelopeRow), }, @@ -104,8 +106,7 @@ func createBenchmarkChangefeed( buf := makeBuffer() poller := makePoller( - s.ClusterSettings(), s.DB(), feedClock, s.Gossip(), spans, details.Targets, - initialHighWater, buf) + s.ClusterSettings(), s.DB(), feedClock, s.Gossip(), spans, details, initialHighWater, buf) rowsFn := kvsToRows(s.LeaseManager().(*sql.LeaseManager), details, buf.Get) tickFn := emitEntries(details, sink, rowsFn) diff --git a/pkg/ccl/changefeedccl/poller.go b/pkg/ccl/changefeedccl/poller.go index 5df31ca4d313..bcfff30d9081 100644 --- a/pkg/ccl/changefeedccl/poller.go +++ b/pkg/ccl/changefeedccl/poller.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/utilccl/intervalccl" "github.com/cockroachdb/cockroach/pkg/gossip" "github.com/cockroachdb/cockroach/pkg/internal/client" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -44,7 +45,7 @@ type poller struct { clock *hlc.Clock gossip *gossip.Gossip spans []roachpb.Span - targets map[sqlbase.ID]string + details jobspb.ChangefeedDetails buf *buffer highWater hlc.Timestamp @@ -56,8 +57,8 @@ func makePoller( clock *hlc.Clock, gossip *gossip.Gossip, spans []roachpb.Span, - targets map[sqlbase.ID]string, - startTime hlc.Timestamp, + details jobspb.ChangefeedDetails, + highWater hlc.Timestamp, buf *buffer, ) *poller { return &poller{ @@ -65,36 +66,35 @@ func makePoller( db: db, clock: clock, gossip: gossip, - highWater: startTime, + highWater: highWater, spans: spans, - targets: targets, + details: details, buf: buf, } } func fetchSpansForTargets( - ctx context.Context, db *client.DB, targets map[sqlbase.ID]string, ts hlc.Timestamp, + ctx context.Context, db *client.DB, targets jobspb.ChangefeedTargets, ts hlc.Timestamp, ) ([]roachpb.Span, error) { var spans []roachpb.Span err := db.Txn(ctx, func(ctx context.Context, txn *client.Txn) error { spans = nil - if ts != (hlc.Timestamp{}) { - txn.SetFixedTimestamp(ctx, ts) - } + txn.SetFixedTimestamp(ctx, ts) // Note that all targets are currently guaranteed to be tables. - for tableID, origName := range targets { + for tableID, t := range targets { tableDesc, err := sqlbase.GetTableDescFromID(ctx, txn, tableID) if err != nil { if errors.Cause(err) == sqlbase.ErrDescriptorNotFound { - return errors.Errorf(`"%s" was dropped or truncated`, origName) + return errors.Errorf(`"%s" was dropped or truncated`, t.StatementTimeName) } return err } if tableDesc.State == sqlbase.TableDescriptor_DROP { - return errors.Errorf(`"%s" was dropped or truncated`, origName) + return errors.Errorf(`"%s" was dropped or truncated`, t.StatementTimeName) } - if tableDesc.Name != origName { - return errors.Errorf(`"%s" was renamed to "%s"`, origName, tableDesc.Name) + if tableDesc.Name != t.StatementTimeName { + return errors.Errorf( + `"%s" was renamed to "%s"`, t.StatementTimeName, tableDesc.Name) } if err := validateChangefeedTable(tableDesc); err != nil { return err @@ -137,7 +137,7 @@ func (p *poller) Run(ctx context.Context) error { log.VEventf(ctx, 1, `changefeed poll [%s,%s): %s`, p.highWater, nextHighWater, time.Duration(nextHighWater.WallTime-p.highWater.WallTime)) - _, err := fetchSpansForTargets(ctx, p.db, p.targets, nextHighWater) + _, err := fetchSpansForTargets(ctx, p.db, p.details.Targets, nextHighWater) if err != nil { return err } @@ -301,22 +301,15 @@ func (p *poller) slurpSST(ctx context.Context, sst []byte) error { // TODO(nvanbenschoten): this should probably be a whole different type that // shares a common interface with poller. func (p *poller) runUsingRangefeeds(ctx context.Context) error { - startTS := p.highWater - spans, err := fetchSpansForTargets(ctx, p.db, p.targets, startTS) - if err != nil { - return err - } - g := ctxgroup.WithContext(ctx) sender := p.db.NonTransactionalSender() - if startTS == (hlc.Timestamp{}) { - startTS = p.clock.Now() - + if p.highWater == (hlc.Timestamp{}) { // TODO(nvanbenschoten/danhhz): This should be replaced by a series of // ScanRequests and this structure should be completely reworked. Right // now it's copied verbatim from above. var ranges []roachpb.RangeDescriptor if err := p.db.Txn(ctx, func(ctx context.Context, txn *client.Txn) error { + txn.SetFixedTimestamp(ctx, p.details.StatementTime) var err error ranges, err = allRangeDescriptors(ctx, txn) return err @@ -378,7 +371,7 @@ func (p *poller) runUsingRangefeeds(ctx context.Context) error { if log.V(2) { log.Infof(ctx, `sending ExportRequest [%s,%s)`, span.Key, span.EndKey) } - header := roachpb.Header{Timestamp: startTS} + header := roachpb.Header{Timestamp: p.details.StatementTime} req := &roachpb.ExportRequest{ RequestHeader: roachpb.RequestHeaderFromSpan(span), StartTime: hlc.Timestamp{}, @@ -401,20 +394,25 @@ func (p *poller) runUsingRangefeeds(ctx context.Context) error { return err } } - return p.buf.AddResolved(ctx, span, startTS) + return p.buf.AddResolved(ctx, span, p.details.StatementTime) }) } return nil }) } + rangeFeedTS := p.details.StatementTime + if rangeFeedTS.Less(p.highWater) { + rangeFeedTS = p.highWater + } + // TODO(nvanbenschoten): This is horrible. ds := sender.(*client.CrossRangeTxnWrapperSender).Wrapped().(*kv.DistSender) eventC := make(chan *roachpb.RangeFeedEvent, 128) - for _, span := range spans { + for _, span := range p.spans { req := &roachpb.RangeFeedRequest{ Header: roachpb.Header{ - Timestamp: startTS, + Timestamp: rangeFeedTS, }, Span: span, } diff --git a/pkg/ccl/changefeedccl/sink.go b/pkg/ccl/changefeedccl/sink.go index f1ab7374a09b..9ffb9cf4713d 100644 --- a/pkg/ccl/changefeedccl/sink.go +++ b/pkg/ccl/changefeedccl/sink.go @@ -19,6 +19,7 @@ import ( "sync" "github.com/Shopify/sarama" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -47,7 +48,7 @@ type Sink interface { Close() error } -func getSink(sinkURI string, targets map[sqlbase.ID]string) (Sink, error) { +func getSink(sinkURI string, targets jobspb.ChangefeedTargets) (Sink, error) { u, err := url.Parse(sinkURI) if err != nil { return nil, err @@ -96,14 +97,14 @@ type kafkaSink struct { } func getKafkaSink( - kafkaTopicPrefix string, bootstrapServers string, targets map[sqlbase.ID]string, + kafkaTopicPrefix string, bootstrapServers string, targets jobspb.ChangefeedTargets, ) (Sink, error) { sink := &kafkaSink{ kafkaTopicPrefix: kafkaTopicPrefix, } sink.topics = make(map[string]struct{}) - for _, tableName := range targets { - sink.topics[kafkaTopicPrefix+tableName] = struct{}{} + for _, t := range targets { + sink.topics[kafkaTopicPrefix+t.StatementTimeName] = struct{}{} } config := sarama.NewConfig() @@ -354,7 +355,7 @@ type sqlSink struct { rowBuf []interface{} } -func makeSQLSink(uri, tableName string, targets map[sqlbase.ID]string) (*sqlSink, error) { +func makeSQLSink(uri, tableName string, targets jobspb.ChangefeedTargets) (*sqlSink, error) { if u, err := url.Parse(uri); err != nil { return nil, err } else if u.Path == `` { @@ -374,8 +375,8 @@ func makeSQLSink(uri, tableName string, targets map[sqlbase.ID]string) (*sqlSink topics: make(map[string]struct{}), hasher: fnv.New32a(), } - for _, tableName := range targets { - s.topics[tableName] = struct{}{} + for _, t := range targets { + s.topics[t.StatementTimeName] = struct{}{} } return s, nil } diff --git a/pkg/ccl/changefeedccl/sink_test.go b/pkg/ccl/changefeedccl/sink_test.go index 720a11746097..a98a1e1f8dd6 100644 --- a/pkg/ccl/changefeedccl/sink_test.go +++ b/pkg/ccl/changefeedccl/sink_test.go @@ -19,8 +19,8 @@ import ( "github.com/Shopify/sarama" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/security" - "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" @@ -141,7 +141,10 @@ func TestSQLSink(t *testing.T) { defer cleanup() sinkURL.Path = `d` - targets := map[sqlbase.ID]string{0: `foo`, 1: `bar`} + targets := jobspb.ChangefeedTargets{ + 0: jobspb.ChangefeedTarget{StatementTimeName: `foo`}, + 1: jobspb.ChangefeedTarget{StatementTimeName: `bar`}, + } sink, err := makeSQLSink(sinkURL.String(), `sink`, targets) require.NoError(t, err) defer func() { require.NoError(t, sink.Close()) }() diff --git a/pkg/jobs/jobspb/jobs.pb.go b/pkg/jobs/jobspb/jobs.pb.go index 0334cf3546c1..a8a0a2d3c492 100644 --- a/pkg/jobs/jobspb/jobs.pb.go +++ b/pkg/jobs/jobspb/jobs.pb.go @@ -18,6 +18,7 @@ ResumeSpanList SchemaChangeDetails SchemaChangeProgress + ChangefeedTarget ChangefeedDetails ResolvedSpan ChangefeedProgress @@ -230,6 +231,15 @@ func (m *SchemaChangeProgress) String() string { return proto.Compact func (*SchemaChangeProgress) ProtoMessage() {} func (*SchemaChangeProgress) Descriptor() ([]byte, []int) { return fileDescriptorJobs, []int{9} } +type ChangefeedTarget struct { + StatementTimeName string `protobuf:"bytes,1,opt,name=statement_time_name,json=statementTimeName,proto3" json:"statement_time_name,omitempty"` +} + +func (m *ChangefeedTarget) Reset() { *m = ChangefeedTarget{} } +func (m *ChangefeedTarget) String() string { return proto.CompactTextString(m) } +func (*ChangefeedTarget) ProtoMessage() {} +func (*ChangefeedTarget) Descriptor() ([]byte, []int) { return fileDescriptorJobs, []int{10} } + type ChangefeedDetails struct { // Targets contains the user-specified tables and databases to watch, mapping // the descriptor id to the name at the time of changefeed creating. There is @@ -248,15 +258,16 @@ type ChangefeedDetails struct { // renames can be detected. They are also used to construct an error message // if the descriptor id no longer exists when the jobs is unpaused (which can // happen if it was dropped or truncated). - Targets map[github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID]string `protobuf:"bytes,5,rep,name=targets,castkey=github.com/cockroachdb/cockroach/pkg/sql/sqlbase.ID" json:"targets,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - SinkURI string `protobuf:"bytes,3,opt,name=sink_uri,json=sinkUri,proto3" json:"sink_uri,omitempty"` - Opts map[string]string `protobuf:"bytes,4,rep,name=opts" json:"opts,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Targets ChangefeedTargets `protobuf:"bytes,6,rep,name=targets,casttype=ChangefeedTargets,castkey=github.com/cockroachdb/cockroach/pkg/sql/sqlbase.ID" json:"targets" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value"` + SinkURI string `protobuf:"bytes,3,opt,name=sink_uri,json=sinkUri,proto3" json:"sink_uri,omitempty"` + Opts map[string]string `protobuf:"bytes,4,rep,name=opts" json:"opts,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + StatementTime cockroach_util_hlc.Timestamp `protobuf:"bytes,7,opt,name=statement_time,json=statementTime" json:"statement_time"` } func (m *ChangefeedDetails) Reset() { *m = ChangefeedDetails{} } func (m *ChangefeedDetails) String() string { return proto.CompactTextString(m) } func (*ChangefeedDetails) ProtoMessage() {} -func (*ChangefeedDetails) Descriptor() ([]byte, []int) { return fileDescriptorJobs, []int{10} } +func (*ChangefeedDetails) Descriptor() ([]byte, []int) { return fileDescriptorJobs, []int{11} } type ResolvedSpan struct { Span cockroach_roachpb1.Span `protobuf:"bytes,1,opt,name=span" json:"span"` @@ -266,7 +277,7 @@ type ResolvedSpan struct { func (m *ResolvedSpan) Reset() { *m = ResolvedSpan{} } func (m *ResolvedSpan) String() string { return proto.CompactTextString(m) } func (*ResolvedSpan) ProtoMessage() {} -func (*ResolvedSpan) Descriptor() ([]byte, []int) { return fileDescriptorJobs, []int{11} } +func (*ResolvedSpan) Descriptor() ([]byte, []int) { return fileDescriptorJobs, []int{12} } type ChangefeedProgress struct { ResolvedSpans []ResolvedSpan `protobuf:"bytes,2,rep,name=resolved_spans,json=resolvedSpans" json:"resolved_spans"` @@ -275,7 +286,7 @@ type ChangefeedProgress struct { func (m *ChangefeedProgress) Reset() { *m = ChangefeedProgress{} } func (m *ChangefeedProgress) String() string { return proto.CompactTextString(m) } func (*ChangefeedProgress) ProtoMessage() {} -func (*ChangefeedProgress) Descriptor() ([]byte, []int) { return fileDescriptorJobs, []int{12} } +func (*ChangefeedProgress) Descriptor() ([]byte, []int) { return fileDescriptorJobs, []int{13} } type Payload struct { Description string `protobuf:"bytes,1,opt,name=description,proto3" json:"description,omitempty"` @@ -301,7 +312,7 @@ type Payload struct { func (m *Payload) Reset() { *m = Payload{} } func (m *Payload) String() string { return proto.CompactTextString(m) } func (*Payload) ProtoMessage() {} -func (*Payload) Descriptor() ([]byte, []int) { return fileDescriptorJobs, []int{13} } +func (*Payload) Descriptor() ([]byte, []int) { return fileDescriptorJobs, []int{14} } type isPayload_Details interface { isPayload_Details() @@ -522,7 +533,7 @@ type Progress struct { func (m *Progress) Reset() { *m = Progress{} } func (m *Progress) String() string { return proto.CompactTextString(m) } func (*Progress) ProtoMessage() {} -func (*Progress) Descriptor() ([]byte, []int) { return fileDescriptorJobs, []int{14} } +func (*Progress) Descriptor() ([]byte, []int) { return fileDescriptorJobs, []int{15} } type isProgress_Progress interface { isProgress_Progress() @@ -816,6 +827,7 @@ func init() { proto.RegisterType((*ResumeSpanList)(nil), "cockroach.sql.jobs.jobspb.ResumeSpanList") proto.RegisterType((*SchemaChangeDetails)(nil), "cockroach.sql.jobs.jobspb.SchemaChangeDetails") proto.RegisterType((*SchemaChangeProgress)(nil), "cockroach.sql.jobs.jobspb.SchemaChangeProgress") + proto.RegisterType((*ChangefeedTarget)(nil), "cockroach.sql.jobs.jobspb.ChangefeedTarget") proto.RegisterType((*ChangefeedDetails)(nil), "cockroach.sql.jobs.jobspb.ChangefeedDetails") proto.RegisterType((*ResolvedSpan)(nil), "cockroach.sql.jobs.jobspb.ResolvedSpan") proto.RegisterType((*ChangefeedProgress)(nil), "cockroach.sql.jobs.jobspb.ChangefeedProgress") @@ -1371,6 +1383,30 @@ func (m *SchemaChangeProgress) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func (m *ChangefeedTarget) 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 *ChangefeedTarget) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.StatementTimeName) > 0 { + dAtA[i] = 0xa + i++ + i = encodeVarintJobs(dAtA, i, uint64(len(m.StatementTimeName))) + i += copy(dAtA[i:], m.StatementTimeName) + } + return i, nil +} + func (m *ChangefeedDetails) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -1421,20 +1457,37 @@ func (m *ChangefeedDetails) MarshalTo(dAtA []byte) (int, error) { } sortkeys.Uint32s(keysForTargets) for _, k := range keysForTargets { - dAtA[i] = 0x2a + dAtA[i] = 0x32 i++ v := m.Targets[github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID(k)] - mapSize := 1 + sovJobs(uint64(k)) + 1 + len(v) + sovJobs(uint64(len(v))) + msgSize := 0 + if (&v) != nil { + msgSize = (&v).Size() + msgSize += 1 + sovJobs(uint64(msgSize)) + } + mapSize := 1 + sovJobs(uint64(k)) + msgSize i = encodeVarintJobs(dAtA, i, uint64(mapSize)) dAtA[i] = 0x8 i++ i = encodeVarintJobs(dAtA, i, uint64(k)) dAtA[i] = 0x12 i++ - i = encodeVarintJobs(dAtA, i, uint64(len(v))) - i += copy(dAtA[i:], v) + i = encodeVarintJobs(dAtA, i, uint64((&v).Size())) + n11, err := (&v).MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n11 } } + dAtA[i] = 0x3a + i++ + i = encodeVarintJobs(dAtA, i, uint64(m.StatementTime.Size())) + n12, err := m.StatementTime.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n12 return i, nil } @@ -1456,19 +1509,19 @@ func (m *ResolvedSpan) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintJobs(dAtA, i, uint64(m.Span.Size())) - n11, err := m.Span.MarshalTo(dAtA[i:]) + n13, err := m.Span.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n11 + i += n13 dAtA[i] = 0x12 i++ i = encodeVarintJobs(dAtA, i, uint64(m.Timestamp.Size())) - n12, err := m.Timestamp.MarshalTo(dAtA[i:]) + n14, err := m.Timestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n12 + i += n14 return i, nil } @@ -1540,21 +1593,21 @@ func (m *Payload) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintJobs(dAtA, i, uint64(m.FinishedMicros)) } if len(m.DescriptorIDs) > 0 { - dAtA14 := make([]byte, len(m.DescriptorIDs)*10) - var j13 int + dAtA16 := make([]byte, len(m.DescriptorIDs)*10) + var j15 int for _, num := range m.DescriptorIDs { for num >= 1<<7 { - dAtA14[j13] = uint8(uint64(num)&0x7f | 0x80) + dAtA16[j15] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j13++ + j15++ } - dAtA14[j13] = uint8(num) - j13++ + dAtA16[j15] = uint8(num) + j15++ } dAtA[i] = 0x32 i++ - i = encodeVarintJobs(dAtA, i, uint64(j13)) - i += copy(dAtA[i:], dAtA14[:j13]) + i = encodeVarintJobs(dAtA, i, uint64(j15)) + i += copy(dAtA[i:], dAtA16[:j15]) } if len(m.Error) > 0 { dAtA[i] = 0x42 @@ -1566,18 +1619,18 @@ func (m *Payload) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x4a i++ i = encodeVarintJobs(dAtA, i, uint64(m.Lease.Size())) - n15, err := m.Lease.MarshalTo(dAtA[i:]) + n17, err := m.Lease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n15 + i += n17 } if m.Details != nil { - nn16, err := m.Details.MarshalTo(dAtA[i:]) + nn18, err := m.Details.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn16 + i += nn18 } return i, nil } @@ -1588,11 +1641,11 @@ func (m *Payload_Backup) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x52 i++ i = encodeVarintJobs(dAtA, i, uint64(m.Backup.Size())) - n17, err := m.Backup.MarshalTo(dAtA[i:]) + n19, err := m.Backup.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n17 + i += n19 } return i, nil } @@ -1602,11 +1655,11 @@ func (m *Payload_Restore) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x5a i++ i = encodeVarintJobs(dAtA, i, uint64(m.Restore.Size())) - n18, err := m.Restore.MarshalTo(dAtA[i:]) + n20, err := m.Restore.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n18 + i += n20 } return i, nil } @@ -1616,11 +1669,11 @@ func (m *Payload_SchemaChange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x62 i++ i = encodeVarintJobs(dAtA, i, uint64(m.SchemaChange.Size())) - n19, err := m.SchemaChange.MarshalTo(dAtA[i:]) + n21, err := m.SchemaChange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n19 + i += n21 } return i, nil } @@ -1630,11 +1683,11 @@ func (m *Payload_Import) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x6a i++ i = encodeVarintJobs(dAtA, i, uint64(m.Import.Size())) - n20, err := m.Import.MarshalTo(dAtA[i:]) + n22, err := m.Import.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n20 + i += n22 } return i, nil } @@ -1644,11 +1697,11 @@ func (m *Payload_Changefeed) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x72 i++ i = encodeVarintJobs(dAtA, i, uint64(m.Changefeed.Size())) - n21, err := m.Changefeed.MarshalTo(dAtA[i:]) + n23, err := m.Changefeed.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n21 + i += n23 } return i, nil } @@ -1668,11 +1721,11 @@ func (m *Progress) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if m.Progress != nil { - nn22, err := m.Progress.MarshalTo(dAtA[i:]) + nn24, err := m.Progress.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn22 + i += nn24 } if m.ModifiedMicros != 0 { dAtA[i] = 0x10 @@ -1680,11 +1733,11 @@ func (m *Progress) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintJobs(dAtA, i, uint64(m.ModifiedMicros)) } if m.Details != nil { - nn23, err := m.Details.MarshalTo(dAtA[i:]) + nn25, err := m.Details.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn23 + i += nn25 } return i, nil } @@ -1703,11 +1756,11 @@ func (m *Progress_HighWater) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintJobs(dAtA, i, uint64(m.HighWater.Size())) - n24, err := m.HighWater.MarshalTo(dAtA[i:]) + n26, err := m.HighWater.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n24 + i += n26 } return i, nil } @@ -1717,11 +1770,11 @@ func (m *Progress_Backup) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x52 i++ i = encodeVarintJobs(dAtA, i, uint64(m.Backup.Size())) - n25, err := m.Backup.MarshalTo(dAtA[i:]) + n27, err := m.Backup.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n25 + i += n27 } return i, nil } @@ -1731,11 +1784,11 @@ func (m *Progress_Restore) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x5a i++ i = encodeVarintJobs(dAtA, i, uint64(m.Restore.Size())) - n26, err := m.Restore.MarshalTo(dAtA[i:]) + n28, err := m.Restore.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n26 + i += n28 } return i, nil } @@ -1745,11 +1798,11 @@ func (m *Progress_SchemaChange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x62 i++ i = encodeVarintJobs(dAtA, i, uint64(m.SchemaChange.Size())) - n27, err := m.SchemaChange.MarshalTo(dAtA[i:]) + n29, err := m.SchemaChange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n27 + i += n29 } return i, nil } @@ -1759,11 +1812,11 @@ func (m *Progress_Import) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x6a i++ i = encodeVarintJobs(dAtA, i, uint64(m.Import.Size())) - n28, err := m.Import.MarshalTo(dAtA[i:]) + n30, err := m.Import.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n28 + i += n30 } return i, nil } @@ -1773,11 +1826,11 @@ func (m *Progress_Changefeed) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x72 i++ i = encodeVarintJobs(dAtA, i, uint64(m.Changefeed.Size())) - n29, err := m.Changefeed.MarshalTo(dAtA[i:]) + n31, err := m.Changefeed.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n29 + i += n31 } return i, nil } @@ -2000,6 +2053,16 @@ func (m *SchemaChangeProgress) Size() (n int) { return n } +func (m *ChangefeedTarget) Size() (n int) { + var l int + _ = l + l = len(m.StatementTimeName) + if l > 0 { + n += 1 + l + sovJobs(uint64(l)) + } + return n +} + func (m *ChangefeedDetails) Size() (n int) { var l int _ = l @@ -2019,10 +2082,13 @@ func (m *ChangefeedDetails) Size() (n int) { for k, v := range m.Targets { _ = k _ = v - mapEntrySize := 1 + sovJobs(uint64(k)) + 1 + len(v) + sovJobs(uint64(len(v))) + l = v.Size() + mapEntrySize := 1 + sovJobs(uint64(k)) + 1 + l + sovJobs(uint64(l)) n += mapEntrySize + 1 + sovJobs(uint64(mapEntrySize)) } } + l = m.StatementTime.Size() + n += 1 + l + sovJobs(uint64(l)) return n } @@ -3864,6 +3930,85 @@ func (m *SchemaChangeProgress) Unmarshal(dAtA []byte) error { } return nil } +func (m *ChangefeedTarget) 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: ChangefeedTarget: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ChangefeedTarget: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StatementTimeName", 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.StatementTimeName = string(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 *ChangefeedDetails) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -4040,7 +4185,7 @@ func (m *ChangefeedDetails) Unmarshal(dAtA []byte) error { } m.Opts[mapkey] = mapvalue iNdEx = postIndex - case 5: + case 6: if wireType != 2 { return fmt.Errorf("proto: wrong wireType = %d for field Targets", wireType) } @@ -4067,10 +4212,10 @@ func (m *ChangefeedDetails) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Targets == nil { - m.Targets = make(map[github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID]string) + m.Targets = make(ChangefeedTargets) } var mapkey uint32 - var mapvalue string + mapvalue := &ChangefeedTarget{} for iNdEx < postIndex { entryPreIndex := iNdEx var wire uint64 @@ -4105,7 +4250,7 @@ func (m *ChangefeedDetails) Unmarshal(dAtA []byte) error { } } } else if fieldNum == 2 { - var stringLenmapvalue uint64 + var mapmsglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowJobs @@ -4115,21 +4260,26 @@ func (m *ChangefeedDetails) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLenmapvalue |= (uint64(b) & 0x7F) << shift + mapmsglen |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - intStringLenmapvalue := int(stringLenmapvalue) - if intStringLenmapvalue < 0 { + if mapmsglen < 0 { return ErrInvalidLengthJobs } - postStringIndexmapvalue := iNdEx + intStringLenmapvalue - if postStringIndexmapvalue > l { + postmsgIndex := iNdEx + mapmsglen + if mapmsglen < 0 { + return ErrInvalidLengthJobs + } + if postmsgIndex > l { return io.ErrUnexpectedEOF } - mapvalue = string(dAtA[iNdEx:postStringIndexmapvalue]) - iNdEx = postStringIndexmapvalue + mapvalue = &ChangefeedTarget{} + if err := mapvalue.Unmarshal(dAtA[iNdEx:postmsgIndex]); err != nil { + return err + } + iNdEx = postmsgIndex } else { iNdEx = entryPreIndex skippy, err := skipJobs(dAtA[iNdEx:]) @@ -4145,7 +4295,37 @@ func (m *ChangefeedDetails) Unmarshal(dAtA []byte) error { iNdEx += skippy } } - m.Targets[github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID(mapkey)] = mapvalue + m.Targets[github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID(mapkey)] = *mapvalue + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field StatementTime", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowJobs + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthJobs + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.StatementTime.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex default: iNdEx = preIndex @@ -5169,123 +5349,128 @@ var ( func init() { proto.RegisterFile("jobs/jobspb/jobs.proto", fileDescriptorJobs) } var fileDescriptorJobs = []byte{ - // 1888 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x58, 0xdf, 0x6f, 0xe3, 0x58, - 0xf5, 0x8f, 0x63, 0x27, 0x76, 0x4e, 0x7e, 0xd4, 0xbd, 0x53, 0xed, 0x7a, 0xa3, 0xd9, 0x26, 0xca, - 0xf7, 0xcb, 0x4e, 0x66, 0x97, 0x4d, 0xa0, 0x2b, 0xed, 0xc2, 0x08, 0x56, 0x34, 0x6d, 0xba, 0x89, - 0x77, 0x66, 0x5a, 0xb9, 0x2d, 0x88, 0x95, 0x90, 0x71, 0xe2, 0xdb, 0xc6, 0xc4, 0x89, 0x5d, 0x5f, - 0x67, 0x86, 0xd9, 0x17, 0x24, 0xc4, 0x03, 0xaa, 0x78, 0xe0, 0x8d, 0x17, 0x2a, 0x21, 0xc1, 0x03, - 0xef, 0x08, 0xfe, 0x86, 0x79, 0x41, 0xec, 0x23, 0x2b, 0xa4, 0x00, 0xe1, 0x85, 0xbf, 0x81, 0x27, - 0x74, 0xef, 0xb5, 0x1d, 0xa7, 0x5b, 0xfa, 0x63, 0x10, 0x2f, 0xad, 0x7d, 0xee, 0x39, 0x1f, 0x9f, - 0x73, 0xfc, 0x39, 0x3f, 0x1c, 0x78, 0xed, 0x07, 0xde, 0x80, 0xb4, 0xe9, 0x1f, 0x7f, 0xc0, 0xfe, - 0xb5, 0xfc, 0xc0, 0x0b, 0x3d, 0xf4, 0xc6, 0xd0, 0x1b, 0x8e, 0x03, 0xcf, 0x1a, 0x8e, 0x5a, 0xe4, - 0xcc, 0x6d, 0xb1, 0x13, 0xae, 0x55, 0xdd, 0x38, 0xf5, 0x4e, 0x3d, 0xa6, 0xd5, 0xa6, 0x57, 0xdc, - 0xa0, 0x8a, 0x98, 0xb2, 0x3f, 0x68, 0xdb, 0x56, 0x68, 0x45, 0x32, 0x2d, 0x96, 0x39, 0xde, 0xbb, - 0x27, 0x5e, 0x30, 0xb1, 0xc2, 0x08, 0xbe, 0x7a, 0x9f, 0x9c, 0xb9, 0x6d, 0x72, 0xe6, 0x0e, 0x2c, - 0x82, 0xdb, 0x24, 0x0c, 0x66, 0xc3, 0x70, 0x16, 0x60, 0x3b, 0xb6, 0x9b, 0x85, 0x8e, 0xdb, 0x1e, - 0xb9, 0xc3, 0x76, 0xe8, 0x4c, 0x30, 0x09, 0xad, 0x89, 0xcf, 0x4f, 0x1a, 0x3f, 0x82, 0xdc, 0x63, - 0x6c, 0x11, 0x8c, 0x3e, 0x01, 0x79, 0xea, 0xd9, 0xd8, 0x74, 0x6c, 0x4d, 0xa8, 0x0b, 0xcd, 0x72, - 0x67, 0x7b, 0x31, 0xaf, 0xe5, 0x9f, 0x7a, 0x36, 0xee, 0xef, 0xfe, 0x6b, 0x5e, 0x7b, 0xef, 0xd4, - 0x09, 0x47, 0xb3, 0x41, 0x6b, 0xe8, 0x4d, 0xda, 0x49, 0x24, 0xf6, 0x60, 0x79, 0xdd, 0xf6, 0xc7, - 0xa7, 0xed, 0xc8, 0xbd, 0x16, 0x37, 0x33, 0xf2, 0x14, 0xb1, 0x6f, 0xa3, 0x0d, 0xc8, 0x61, 0xdf, - 0x1b, 0x8e, 0xb4, 0x6c, 0x5d, 0x68, 0x8a, 0x06, 0xbf, 0x79, 0x24, 0xfd, 0xf3, 0x57, 0x35, 0xa1, - 0xf1, 0x17, 0x01, 0xca, 0x1d, 0x6b, 0x38, 0x9e, 0xf9, 0xbb, 0x38, 0xb4, 0x1c, 0x97, 0xa0, 0x0e, - 0x00, 0x09, 0xad, 0x20, 0x34, 0xa9, 0xaf, 0xcc, 0x99, 0xe2, 0xd6, 0x9b, 0xad, 0x65, 0xfa, 0x68, - 0x2c, 0xad, 0x91, 0x3b, 0x6c, 0x1d, 0xc5, 0xb1, 0x74, 0xa4, 0x97, 0xf3, 0x5a, 0xc6, 0x28, 0x30, - 0x33, 0x2a, 0x45, 0x1f, 0x82, 0x82, 0xa7, 0x36, 0x47, 0xc8, 0xde, 0x1e, 0x41, 0xc6, 0x53, 0x9b, - 0xd9, 0xbf, 0x01, 0xe2, 0x2c, 0x70, 0x34, 0xb1, 0x2e, 0x34, 0x0b, 0x1d, 0x79, 0x31, 0xaf, 0x89, - 0xc7, 0x46, 0xdf, 0xa0, 0x32, 0xf4, 0x0e, 0xac, 0x0f, 0x98, 0xbf, 0xa6, 0x8d, 0xc9, 0x30, 0x70, - 0xfc, 0xd0, 0x0b, 0x34, 0xa9, 0x2e, 0x34, 0x4b, 0x86, 0x3a, 0x88, 0x02, 0x89, 0xe5, 0x0d, 0x15, - 0x2a, 0x3c, 0xb8, 0x83, 0xc0, 0x3b, 0x0d, 0x30, 0x21, 0x8d, 0xcf, 0x73, 0x50, 0x31, 0x30, 0x09, - 0xbd, 0x00, 0xc7, 0x01, 0xff, 0x52, 0x80, 0x4a, 0x68, 0x0d, 0x5c, 0x6c, 0x06, 0xf8, 0x79, 0xe0, - 0x84, 0x98, 0x68, 0xd9, 0xba, 0xd8, 0x2c, 0x6e, 0x7d, 0xa3, 0xf5, 0x1f, 0x49, 0xd3, 0x5a, 0xc5, - 0x68, 0x1d, 0x51, 0x7b, 0x23, 0x32, 0xef, 0x4e, 0xc3, 0xe0, 0x45, 0xe7, 0x83, 0x1f, 0xff, 0xf5, - 0x96, 0xaf, 0x2d, 0xc5, 0x9d, 0x56, 0x7f, 0xd7, 0x28, 0x87, 0x69, 0x30, 0x74, 0x1f, 0xa4, 0x59, - 0xe0, 0x10, 0x4d, 0xac, 0x8b, 0xcd, 0x42, 0x47, 0x59, 0xcc, 0x6b, 0xd2, 0xb1, 0xd1, 0x27, 0x06, - 0x93, 0xae, 0x64, 0x5a, 0x7a, 0x85, 0x4c, 0x7f, 0x04, 0x45, 0x1e, 0x3b, 0xcd, 0x26, 0xd1, 0x72, - 0x2c, 0xf0, 0xb7, 0x2e, 0x05, 0x1e, 0x3b, 0xc7, 0xa2, 0x5c, 0xa6, 0xd7, 0x80, 0x30, 0x16, 0x10, - 0xd4, 0x86, 0xa2, 0xf7, 0x0c, 0x07, 0x81, 0x63, 0x63, 0xd3, 0x1e, 0x68, 0x79, 0xf6, 0xea, 0x2a, - 0x8b, 0x79, 0x0d, 0xf6, 0x23, 0xf1, 0x6e, 0xc7, 0x80, 0x58, 0x65, 0x77, 0x50, 0xfd, 0x93, 0x00, - 0xa5, 0x74, 0xda, 0xd0, 0xf7, 0x40, 0xe1, 0xae, 0x24, 0x35, 0xd0, 0x59, 0xcc, 0x6b, 0x32, 0xd3, - 0xb9, 0x43, 0x11, 0x5c, 0xca, 0xa6, 0xcc, 0x30, 0xfb, 0x36, 0xfa, 0x3e, 0x14, 0x7c, 0x2b, 0xc0, - 0xd3, 0x90, 0xe2, 0x67, 0x19, 0xfe, 0xce, 0x62, 0x5e, 0x53, 0x0e, 0x98, 0xf0, 0xd5, 0x1f, 0xa0, - 0x70, 0xd4, 0xbe, 0x5d, 0xfd, 0x21, 0xa0, 0x2f, 0xf2, 0x00, 0xa9, 0x20, 0x8e, 0xf1, 0x0b, 0x1e, - 0x91, 0x41, 0x2f, 0xd1, 0x63, 0xc8, 0x3d, 0xb3, 0xdc, 0x59, 0x5c, 0x1a, 0xef, 0xbf, 0x1a, 0xcd, - 0x0c, 0x0e, 0xf2, 0x28, 0xfb, 0x35, 0x41, 0x97, 0x14, 0x41, 0xcd, 0x36, 0xbe, 0x02, 0x6b, 0x91, - 0x7e, 0x4c, 0x77, 0xf4, 0x26, 0xc0, 0xc8, 0x39, 0x1d, 0x99, 0xcf, 0xad, 0x10, 0x07, 0xcc, 0x87, - 0x92, 0x51, 0xa0, 0x92, 0xef, 0x50, 0x41, 0xe3, 0x0f, 0x39, 0x28, 0xf7, 0x27, 0xbe, 0x17, 0x84, - 0x71, 0x31, 0x3c, 0x86, 0x3c, 0x4b, 0x18, 0xd1, 0x04, 0x46, 0x85, 0xd6, 0x35, 0xce, 0xad, 0x58, - 0x72, 0xdf, 0x22, 0x7a, 0x45, 0x18, 0x09, 0x77, 0xb3, 0x57, 0x72, 0xf7, 0x9b, 0x90, 0xe7, 0x5d, - 0x94, 0x15, 0x7a, 0x71, 0xab, 0x96, 0x7a, 0x56, 0xdc, 0xca, 0xfa, 0xfb, 0x7b, 0x8e, 0x8b, 0xf7, - 0x98, 0x5a, 0x0c, 0xce, 0x8d, 0xd0, 0x5b, 0xa0, 0x10, 0x12, 0x9a, 0xc4, 0xf9, 0x94, 0x53, 0x5f, - 0xec, 0x14, 0x29, 0x5f, 0x0e, 0x0f, 0x8f, 0x0e, 0x9d, 0x4f, 0xb1, 0x21, 0x13, 0x12, 0xd2, 0x0b, - 0x54, 0x05, 0xe5, 0xb9, 0xe5, 0xba, 0xac, 0x44, 0x72, 0xac, 0x03, 0x26, 0xf7, 0xab, 0xa4, 0xc8, - 0xff, 0x0f, 0x48, 0x81, 0x6a, 0x50, 0x8c, 0xfa, 0x95, 0x6f, 0x85, 0x23, 0x4d, 0xa6, 0x75, 0x61, - 0x00, 0x17, 0x1d, 0x58, 0xe1, 0x08, 0x69, 0x20, 0x13, 0x6b, 0xe2, 0xd3, 0x94, 0x2b, 0x75, 0xb1, - 0x59, 0x32, 0xe2, 0x5b, 0xb4, 0x09, 0xac, 0x5e, 0xf8, 0xad, 0x56, 0x60, 0xae, 0xa7, 0x24, 0x2c, - 0x01, 0x63, 0xc7, 0x37, 0x4f, 0xc6, 0x44, 0x83, 0xba, 0xd0, 0x54, 0xa2, 0x04, 0x8c, 0x1d, 0x7f, - 0xef, 0x63, 0x62, 0xc8, 0xf4, 0x70, 0x6f, 0x4c, 0xaa, 0x9f, 0x09, 0x90, 0x63, 0x6f, 0x07, 0x3d, - 0x02, 0x89, 0xd6, 0x79, 0xd4, 0xd5, 0x6f, 0x5b, 0xe6, 0xcc, 0x06, 0x21, 0x90, 0xa6, 0xd6, 0x04, - 0x6b, 0x88, 0x45, 0xc0, 0xae, 0xd1, 0xeb, 0x20, 0x13, 0x7c, 0x66, 0x3e, 0xb3, 0x5c, 0xed, 0x1e, - 0x73, 0x2f, 0x4f, 0xf0, 0xd9, 0xb7, 0x2d, 0x57, 0x97, 0x94, 0xac, 0x2a, 0xea, 0x92, 0x22, 0xaa, - 0x92, 0x2e, 0x29, 0x92, 0x9a, 0xd3, 0x25, 0x25, 0xa7, 0xe6, 0x75, 0x49, 0xc9, 0xab, 0xb2, 0x2e, - 0x29, 0xb2, 0xaa, 0xe8, 0x92, 0xa2, 0xa8, 0x05, 0x5d, 0x52, 0x0a, 0x2a, 0xe8, 0x92, 0x02, 0x6a, - 0x51, 0x97, 0x94, 0xa2, 0x5a, 0xd2, 0x25, 0xa5, 0xa4, 0x96, 0x75, 0x49, 0x29, 0xab, 0x15, 0x5d, - 0x52, 0x2a, 0xea, 0x9a, 0x2e, 0x29, 0x6b, 0xaa, 0xaa, 0x4b, 0x8a, 0xaa, 0xae, 0xeb, 0x92, 0xb2, - 0xae, 0xa2, 0xc6, 0x1f, 0x05, 0xa8, 0x70, 0xfa, 0x25, 0x54, 0x7f, 0x07, 0xd6, 0x59, 0x5e, 0x9c, - 0xe9, 0xa9, 0xe9, 0x47, 0x42, 0x46, 0xe2, 0xac, 0xa1, 0xc6, 0x07, 0x89, 0xf2, 0xff, 0x41, 0x39, - 0xc0, 0x96, 0xbd, 0x54, 0xcc, 0x32, 0xc5, 0x12, 0x15, 0x26, 0x4a, 0x5f, 0x82, 0x0a, 0xab, 0xb4, - 0xa5, 0x96, 0xc8, 0xb4, 0xca, 0x4c, 0x9a, 0xa8, 0x75, 0xa0, 0x4c, 0x7c, 0x6b, 0xba, 0xd4, 0x92, - 0x58, 0xe5, 0xbc, 0x7e, 0x05, 0x9b, 0x0f, 0x7d, 0x6b, 0x1a, 0xb1, 0xb8, 0x44, 0x6d, 0x92, 0xb1, - 0x64, 0xb0, 0xa9, 0x34, 0x9b, 0x60, 0xaa, 0xf1, 0xd8, 0x21, 0x21, 0xfa, 0x16, 0x94, 0x02, 0x26, - 0x31, 0xa9, 0x62, 0x5c, 0x8e, 0x37, 0x80, 0x16, 0x83, 0x04, 0x84, 0x34, 0x7e, 0x27, 0xc0, 0xbd, - 0xc3, 0xe1, 0x08, 0x4f, 0xac, 0x9d, 0x91, 0x35, 0x3d, 0x4d, 0xe6, 0xdd, 0x36, 0x00, 0x8b, 0xdd, - 0x22, 0xa6, 0x77, 0x72, 0x97, 0x01, 0xaf, 0x50, 0xb3, 0x6d, 0xb2, 0x7f, 0x82, 0xbe, 0x0b, 0x6a, - 0xca, 0x39, 0xd3, 0x75, 0x48, 0x18, 0xcd, 0xcc, 0x87, 0xd7, 0x37, 0xb3, 0x54, 0x84, 0x11, 0x68, - 0x25, 0x58, 0x91, 0x36, 0x5e, 0x83, 0x8d, 0xb4, 0xd3, 0x49, 0x86, 0x7e, 0x21, 0xc2, 0x3a, 0x17, - 0x9d, 0x60, 0x6c, 0xc7, 0xb1, 0xd0, 0x12, 0x70, 0xa6, 0x63, 0x73, 0xb9, 0x2d, 0xf0, 0x12, 0x70, - 0xa6, 0x63, 0xba, 0x31, 0xc8, 0xf4, 0xf0, 0x38, 0x70, 0x90, 0x0e, 0x92, 0xe7, 0x87, 0xf1, 0xab, - 0xb9, 0xae, 0xe3, 0x7e, 0xe1, 0x19, 0xad, 0x7d, 0x3f, 0xe4, 0xad, 0xdc, 0x60, 0x18, 0xe8, 0x67, - 0x02, 0xc8, 0xa1, 0x15, 0x9c, 0xe2, 0x30, 0x9e, 0x97, 0x5f, 0xbf, 0x13, 0xde, 0x11, 0xb7, 0xfd, - 0x2f, 0xb7, 0x84, 0xd8, 0x85, 0xea, 0x23, 0x3a, 0x46, 0x97, 0x88, 0x57, 0xcc, 0x9b, 0x8d, 0xf4, - 0xbc, 0x29, 0xa4, 0xe6, 0x46, 0xf5, 0x03, 0x28, 0x24, 0xd1, 0xa5, 0x0d, 0x0b, 0x37, 0x18, 0xf2, - 0x81, 0xc3, 0xab, 0xbc, 0xf1, 0x13, 0x01, 0x4a, 0x06, 0x26, 0x9e, 0xfb, 0x0c, 0xdb, 0xf4, 0x35, - 0xa2, 0xaf, 0x82, 0x44, 0x69, 0x11, 0x51, 0xeb, 0x06, 0xca, 0x32, 0x55, 0xb4, 0x0d, 0x85, 0x64, - 0x35, 0xbe, 0xcb, 0xc6, 0xb8, 0xb4, 0x6a, 0xf8, 0x80, 0x96, 0xb9, 0x4e, 0x8a, 0xf3, 0x08, 0x28, - 0xc1, 0x98, 0x6f, 0x51, 0x21, 0x71, 0x9e, 0x3e, 0xb8, 0x9e, 0xa7, 0x49, 0x30, 0xd1, 0x73, 0xca, - 0x41, 0x4a, 0x46, 0xa2, 0x79, 0xbb, 0xc8, 0x81, 0x7c, 0x60, 0xbd, 0x70, 0x3d, 0xcb, 0x46, 0x75, - 0x28, 0xc6, 0xfb, 0xa8, 0xe3, 0x4d, 0xa3, 0x24, 0xa6, 0x45, 0x74, 0x0c, 0xcd, 0x08, 0x0e, 0x58, - 0x0f, 0xe5, 0xf9, 0x4c, 0xee, 0x69, 0xa7, 0x61, 0xcb, 0x33, 0xb6, 0xcd, 0x89, 0x33, 0x0c, 0x3c, - 0xc2, 0xc8, 0x2c, 0x1a, 0xe5, 0x48, 0xfa, 0x84, 0x09, 0xd1, 0x03, 0x58, 0x3b, 0x71, 0xa6, 0x0e, - 0x19, 0x2d, 0xf5, 0xd8, 0xe0, 0x33, 0x2a, 0xb1, 0x38, 0x52, 0xf4, 0xa0, 0xb2, 0xdc, 0x8e, 0x4d, - 0xc7, 0x26, 0x5a, 0xbe, 0x2e, 0x36, 0xcb, 0x9d, 0xde, 0x62, 0x5e, 0x2b, 0x2f, 0x3b, 0x7b, 0x7f, - 0x97, 0xbc, 0xea, 0x80, 0x2b, 0x2f, 0xf1, 0xfb, 0x36, 0x61, 0x9f, 0x18, 0x41, 0xe0, 0x05, 0x9a, - 0xc2, 0x99, 0xc2, 0x6e, 0xd0, 0xfb, 0x90, 0x73, 0xe9, 0xd7, 0x0d, 0x9b, 0x5d, 0xc5, 0xad, 0xfa, - 0x35, 0x39, 0x67, 0x5f, 0x41, 0x06, 0x57, 0x47, 0x1d, 0xc8, 0xf3, 0x01, 0xc9, 0xc6, 0x5a, 0x71, - 0xab, 0x79, 0x8d, 0xe1, 0xca, 0xc7, 0x4b, 0x2f, 0x63, 0x44, 0x96, 0xa8, 0x0b, 0x72, 0xc0, 0x97, - 0x21, 0xad, 0xc8, 0x40, 0x1e, 0xde, 0x7a, 0xcd, 0xea, 0x65, 0x8c, 0xd8, 0x16, 0x1d, 0x41, 0x89, - 0xa4, 0xda, 0x91, 0x56, 0x62, 0x58, 0xd7, 0x6d, 0x45, 0x57, 0xb4, 0xdc, 0x1e, 0x6d, 0xf7, 0x29, - 0x31, 0x0d, 0xd0, 0x61, 0xd3, 0x4b, 0x2b, 0xdf, 0x18, 0xe0, 0xca, 0x96, 0x45, 0x03, 0xe4, 0x96, - 0xe8, 0x29, 0xc0, 0x30, 0xe1, 0xbb, 0x56, 0x61, 0x38, 0x5f, 0xbe, 0x4b, 0x23, 0xea, 0x65, 0x8c, - 0x14, 0x42, 0xa7, 0x00, 0xb2, 0xcd, 0x0f, 0x92, 0x59, 0x2d, 0xab, 0x4a, 0xe3, 0xf7, 0x12, 0x28, - 0x49, 0x35, 0xb5, 0x01, 0x9d, 0x04, 0xd6, 0x90, 0xf2, 0xd9, 0x1c, 0x7a, 0x74, 0x09, 0x09, 0x31, - 0x5f, 0xd6, 0xb3, 0xbd, 0x8c, 0xb1, 0x1e, 0x9f, 0xed, 0xc4, 0x47, 0x94, 0xb1, 0x13, 0xcf, 0x76, - 0x4e, 0x9c, 0x25, 0x63, 0xf9, 0x47, 0x68, 0x25, 0x16, 0x47, 0x8c, 0xfd, 0x70, 0x65, 0x51, 0x15, - 0x6f, 0xd1, 0x01, 0x7a, 0x99, 0xd4, 0x26, 0x8b, 0x76, 0x2e, 0x51, 0xe6, 0xe1, 0x8d, 0x94, 0x89, - 0x83, 0xea, 0x09, 0x09, 0x67, 0xf6, 0x2e, 0x73, 0xe6, 0xed, 0x9b, 0x39, 0x93, 0x82, 0x49, 0x48, - 0x73, 0x7c, 0x25, 0x69, 0xda, 0xb7, 0x24, 0x4d, 0x0a, 0x71, 0x95, 0x35, 0x3b, 0x97, 0x58, 0xf3, - 0xf0, 0x46, 0xd6, 0xa4, 0x63, 0x8c, 0x68, 0xb3, 0x7f, 0x05, 0x6d, 0xde, 0xbd, 0x15, 0x6d, 0x52, - 0x60, 0x69, 0xde, 0x00, 0x28, 0xf1, 0xe6, 0x93, 0xe2, 0xd0, 0xdb, 0x9f, 0x0b, 0x20, 0x1d, 0xbd, - 0xf0, 0x31, 0xfa, 0x7f, 0x28, 0x1e, 0x3f, 0x3d, 0x3c, 0xe8, 0xee, 0xf4, 0xf7, 0xfa, 0xdd, 0x5d, - 0x35, 0x53, 0xbd, 0x77, 0x7e, 0x51, 0x5f, 0xa3, 0x47, 0xc7, 0x53, 0xe2, 0xe3, 0x21, 0x63, 0x01, - 0xaa, 0x42, 0xbe, 0xb3, 0xbd, 0xf3, 0xf1, 0xf1, 0x81, 0x2a, 0x54, 0x2b, 0xe7, 0x17, 0x75, 0xa0, - 0x0a, 0xfc, 0x45, 0xa1, 0xfb, 0x20, 0x1b, 0xdd, 0xc3, 0xa3, 0x7d, 0xa3, 0xab, 0x66, 0xab, 0x6b, - 0xe7, 0x17, 0xf5, 0x22, 0x3d, 0x8c, 0xf2, 0x8f, 0x1e, 0x40, 0xf9, 0x70, 0xa7, 0xd7, 0x7d, 0xb2, - 0x6d, 0xee, 0xf4, 0xb6, 0x9f, 0x7e, 0xd4, 0x55, 0xc5, 0xea, 0xc6, 0xf9, 0x45, 0x5d, 0xa5, 0x3a, - 0xe9, 0xb4, 0xd2, 0x47, 0xf4, 0x9f, 0x1c, 0xec, 0x1b, 0x47, 0xaa, 0xb4, 0x7c, 0x04, 0xcf, 0x13, - 0x6a, 0x00, 0x70, 0xeb, 0xbd, 0x6e, 0x77, 0x57, 0xcd, 0x55, 0xd1, 0xf9, 0x45, 0xbd, 0x42, 0xcf, - 0x97, 0xe1, 0x57, 0x95, 0x9f, 0xfe, 0x7a, 0x33, 0xf3, 0xdb, 0xdf, 0x6c, 0x66, 0x3a, 0xf5, 0x97, - 0x7f, 0xdf, 0xcc, 0xbc, 0x5c, 0x6c, 0x0a, 0x9f, 0x2d, 0x36, 0x85, 0x3f, 0x2f, 0x36, 0x85, 0xbf, - 0x2d, 0x36, 0x85, 0x9f, 0xff, 0x63, 0x33, 0xf3, 0x49, 0x9e, 0x67, 0x6d, 0x90, 0x67, 0x3f, 0xef, - 0xbc, 0xf7, 0xef, 0x00, 0x00, 0x00, 0xff, 0xff, 0x43, 0x6d, 0x36, 0xa2, 0x8f, 0x12, 0x00, 0x00, + // 1956 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x58, 0xdd, 0x6f, 0xe3, 0x58, + 0x15, 0x8f, 0x13, 0x27, 0x76, 0x4e, 0x3e, 0xea, 0xde, 0xa9, 0x76, 0xbd, 0xd1, 0x6c, 0x13, 0x05, + 0xd8, 0xe9, 0xec, 0xb0, 0x09, 0x74, 0xa5, 0x5d, 0x18, 0xc1, 0x8a, 0xa6, 0x4d, 0x37, 0xf1, 0xce, + 0xb4, 0x95, 0xdb, 0x82, 0x58, 0x09, 0x19, 0x27, 0xbe, 0x4d, 0x4c, 0x3e, 0xec, 0xfa, 0x3a, 0x33, + 0xcc, 0xbe, 0x20, 0x21, 0x1e, 0x50, 0x9f, 0xf8, 0x03, 0xa8, 0x84, 0x04, 0x48, 0xbc, 0xf1, 0x80, + 0xe0, 0x6f, 0x98, 0x17, 0xc4, 0x3e, 0xb2, 0x42, 0x0a, 0x10, 0x5e, 0xf8, 0x1b, 0xe6, 0x09, 0xdd, + 0x7b, 0xfd, 0x95, 0x4e, 0xe9, 0x97, 0xb4, 0x2f, 0x33, 0xf1, 0xb9, 0xe7, 0xfc, 0xee, 0x39, 0xc7, + 0xbf, 0xf3, 0xe1, 0xc2, 0x1b, 0x3f, 0x71, 0x7a, 0xa4, 0x49, 0xff, 0x71, 0x7b, 0xec, 0xbf, 0x86, + 0xeb, 0x39, 0xbe, 0x83, 0xde, 0xea, 0x3b, 0xfd, 0x91, 0xe7, 0x98, 0xfd, 0x61, 0x83, 0x9c, 0x8e, + 0x1b, 0xec, 0x84, 0x6b, 0x55, 0xd6, 0x06, 0xce, 0xc0, 0x61, 0x5a, 0x4d, 0xfa, 0x8b, 0x1b, 0x54, + 0x10, 0x53, 0x76, 0x7b, 0x4d, 0xcb, 0xf4, 0xcd, 0x40, 0xa6, 0x86, 0x32, 0xdb, 0x79, 0xef, 0xc4, + 0xf1, 0x26, 0xa6, 0x1f, 0xc0, 0x57, 0xee, 0x93, 0xd3, 0x71, 0x93, 0x9c, 0x8e, 0x7b, 0x26, 0xc1, + 0x4d, 0xe2, 0x7b, 0xb3, 0xbe, 0x3f, 0xf3, 0xb0, 0x15, 0xda, 0xcd, 0x7c, 0x7b, 0xdc, 0x1c, 0x8e, + 0xfb, 0x4d, 0xdf, 0x9e, 0x60, 0xe2, 0x9b, 0x13, 0x97, 0x9f, 0xd4, 0x7f, 0x06, 0xd9, 0x27, 0xd8, + 0x24, 0x18, 0x7d, 0x0a, 0xd2, 0xd4, 0xb1, 0xb0, 0x61, 0x5b, 0xaa, 0x50, 0x13, 0x36, 0x4a, 0xad, + 0xad, 0xc5, 0xbc, 0x9a, 0xdb, 0x73, 0x2c, 0xdc, 0xdd, 0x79, 0x35, 0xaf, 0xbe, 0x3f, 0xb0, 0xfd, + 0xe1, 0xac, 0xd7, 0xe8, 0x3b, 0x93, 0x66, 0x14, 0x89, 0xd5, 0x8b, 0x7f, 0x37, 0xdd, 0xd1, 0xa0, + 0x19, 0xb8, 0xd7, 0xe0, 0x66, 0x7a, 0x8e, 0x22, 0x76, 0x2d, 0xb4, 0x06, 0x59, 0xec, 0x3a, 0xfd, + 0xa1, 0x9a, 0xae, 0x09, 0x1b, 0x19, 0x9d, 0x3f, 0x3c, 0x16, 0xff, 0xfb, 0x9b, 0xaa, 0x50, 0xff, + 0x87, 0x00, 0xa5, 0x96, 0xd9, 0x1f, 0xcd, 0xdc, 0x1d, 0xec, 0x9b, 0xf6, 0x98, 0xa0, 0x16, 0x00, + 0xf1, 0x4d, 0xcf, 0x37, 0xa8, 0xaf, 0xcc, 0x99, 0xc2, 0xe6, 0xdb, 0x8d, 0x38, 0x7d, 0x34, 0x96, + 0xc6, 0x70, 0xdc, 0x6f, 0x1c, 0x85, 0xb1, 0xb4, 0xc4, 0x97, 0xf3, 0x6a, 0x4a, 0xcf, 0x33, 0x33, + 0x2a, 0x45, 0x1f, 0x81, 0x8c, 0xa7, 0x16, 0x47, 0x48, 0xdf, 0x1c, 0x41, 0xc2, 0x53, 0x8b, 0xd9, + 0xbf, 0x05, 0x99, 0x99, 0x67, 0xab, 0x99, 0x9a, 0xb0, 0x91, 0x6f, 0x49, 0x8b, 0x79, 0x35, 0x73, + 0xac, 0x77, 0x75, 0x2a, 0x43, 0x8f, 0x60, 0xb5, 0xc7, 0xfc, 0x35, 0x2c, 0x4c, 0xfa, 0x9e, 0xed, + 0xfa, 0x8e, 0xa7, 0x8a, 0x35, 0x61, 0xa3, 0xa8, 0x2b, 0xbd, 0x20, 0x90, 0x50, 0x5e, 0x57, 0xa0, + 0xcc, 0x83, 0x3b, 0xf0, 0x9c, 0x81, 0x87, 0x09, 0xa9, 0x7f, 0x91, 0x85, 0xb2, 0x8e, 0x89, 0xef, + 0x78, 0x38, 0x0c, 0xf8, 0xd7, 0x02, 0x94, 0x7d, 0xb3, 0x37, 0xc6, 0x86, 0x87, 0x9f, 0x7b, 0xb6, + 0x8f, 0x89, 0x9a, 0xae, 0x65, 0x36, 0x0a, 0x9b, 0xdf, 0x69, 0xfc, 0x5f, 0xd2, 0x34, 0x96, 0x31, + 0x1a, 0x47, 0xd4, 0x5e, 0x0f, 0xcc, 0xdb, 0x53, 0xdf, 0x7b, 0xd1, 0xfa, 0xf0, 0xe7, 0xff, 0xbc, + 0xe1, 0x6b, 0x4b, 0x70, 0xa7, 0xd1, 0xdd, 0xd1, 0x4b, 0x7e, 0x12, 0x0c, 0xdd, 0x07, 0x71, 0xe6, + 0xd9, 0x44, 0xcd, 0xd4, 0x32, 0x1b, 0xf9, 0x96, 0xbc, 0x98, 0x57, 0xc5, 0x63, 0xbd, 0x4b, 0x74, + 0x26, 0x5d, 0xca, 0xb4, 0x78, 0x87, 0x4c, 0x7f, 0x0c, 0x05, 0x1e, 0x3b, 0xcd, 0x26, 0x51, 0xb3, + 0x2c, 0xf0, 0x77, 0x2e, 0x04, 0x1e, 0x3a, 0xc7, 0xa2, 0x8c, 0xd3, 0xab, 0x83, 0x1f, 0x0a, 0x08, + 0x6a, 0x42, 0xc1, 0x79, 0x86, 0x3d, 0xcf, 0xb6, 0xb0, 0x61, 0xf5, 0xd4, 0x1c, 0x7b, 0x75, 0xe5, + 0xc5, 0xbc, 0x0a, 0xfb, 0x81, 0x78, 0xa7, 0xa5, 0x43, 0xa8, 0xb2, 0xd3, 0xab, 0xfc, 0x4d, 0x80, + 0x62, 0x32, 0x6d, 0xe8, 0x47, 0x20, 0x73, 0x57, 0xa2, 0x1a, 0x68, 0x2d, 0xe6, 0x55, 0x89, 0xe9, + 0xdc, 0xa2, 0x08, 0x2e, 0x64, 0x53, 0x62, 0x98, 0x5d, 0x0b, 0xfd, 0x18, 0xf2, 0xae, 0xe9, 0xe1, + 0xa9, 0x4f, 0xf1, 0xd3, 0x0c, 0x7f, 0x7b, 0x31, 0xaf, 0xca, 0x07, 0x4c, 0x78, 0xf7, 0x0b, 0x64, + 0x8e, 0xda, 0xb5, 0x2a, 0x3f, 0x05, 0xf4, 0x3a, 0x0f, 0x90, 0x02, 0x99, 0x11, 0x7e, 0xc1, 0x23, + 0xd2, 0xe9, 0x4f, 0xf4, 0x04, 0xb2, 0xcf, 0xcc, 0xf1, 0x2c, 0x2c, 0x8d, 0x0f, 0xee, 0x46, 0x33, + 0x9d, 0x83, 0x3c, 0x4e, 0x7f, 0x4b, 0xd0, 0x44, 0x59, 0x50, 0xd2, 0xf5, 0x6f, 0xc0, 0x4a, 0xa0, + 0x1f, 0xd2, 0x1d, 0xbd, 0x0d, 0x30, 0xb4, 0x07, 0x43, 0xe3, 0xb9, 0xe9, 0x63, 0x8f, 0xf9, 0x50, + 0xd4, 0xf3, 0x54, 0xf2, 0x03, 0x2a, 0xa8, 0xff, 0x25, 0x0b, 0xa5, 0xee, 0xc4, 0x75, 0x3c, 0x3f, + 0x2c, 0x86, 0x27, 0x90, 0x63, 0x09, 0x23, 0xaa, 0xc0, 0xa8, 0xd0, 0xb8, 0xc2, 0xb9, 0x25, 0x4b, + 0xee, 0x5b, 0x40, 0xaf, 0x00, 0x23, 0xe2, 0x6e, 0xfa, 0x52, 0xee, 0x7e, 0x17, 0x72, 0xbc, 0x8b, + 0xb2, 0x42, 0x2f, 0x6c, 0x56, 0x13, 0x77, 0x85, 0xad, 0xac, 0xbb, 0xbf, 0x6b, 0x8f, 0xf1, 0x2e, + 0x53, 0x0b, 0xc1, 0xb9, 0x11, 0x7a, 0x07, 0x64, 0x42, 0x7c, 0x83, 0xd8, 0x9f, 0x71, 0xea, 0x67, + 0x5a, 0x05, 0xca, 0x97, 0xc3, 0xc3, 0xa3, 0x43, 0xfb, 0x33, 0xac, 0x4b, 0x84, 0xf8, 0xf4, 0x07, + 0xaa, 0x80, 0xfc, 0xdc, 0x1c, 0x8f, 0x59, 0x89, 0x64, 0x59, 0x07, 0x8c, 0x9e, 0x97, 0x49, 0x91, + 0xfb, 0x12, 0x48, 0x81, 0xaa, 0x50, 0x08, 0xfa, 0x95, 0x6b, 0xfa, 0x43, 0x55, 0xa2, 0x75, 0xa1, + 0x03, 0x17, 0x1d, 0x98, 0xfe, 0x10, 0xa9, 0x20, 0x11, 0x73, 0xe2, 0xd2, 0x94, 0xcb, 0xb5, 0xcc, + 0x46, 0x51, 0x0f, 0x1f, 0xd1, 0x3a, 0xb0, 0x7a, 0xe1, 0x8f, 0x6a, 0x9e, 0xb9, 0x9e, 0x90, 0xb0, + 0x04, 0x8c, 0x6c, 0xd7, 0x38, 0x19, 0x11, 0x15, 0x6a, 0xc2, 0x86, 0x1c, 0x24, 0x60, 0x64, 0xbb, + 0xbb, 0x9f, 0x10, 0x5d, 0xa2, 0x87, 0xbb, 0x23, 0x52, 0xf9, 0x5c, 0x80, 0x2c, 0x7b, 0x3b, 0xe8, + 0x31, 0x88, 0xb4, 0xce, 0x83, 0xae, 0x7e, 0xd3, 0x32, 0x67, 0x36, 0x08, 0x81, 0x38, 0x35, 0x27, + 0x58, 0x45, 0x2c, 0x02, 0xf6, 0x1b, 0xbd, 0x09, 0x12, 0xc1, 0xa7, 0xc6, 0x33, 0x73, 0xac, 0xde, + 0x63, 0xee, 0xe5, 0x08, 0x3e, 0xfd, 0xbe, 0x39, 0xd6, 0x44, 0x39, 0xad, 0x64, 0x34, 0x51, 0xce, + 0x28, 0xa2, 0x26, 0xca, 0xa2, 0x92, 0xd5, 0x44, 0x39, 0xab, 0xe4, 0x34, 0x51, 0xce, 0x29, 0x92, + 0x26, 0xca, 0x92, 0x22, 0x6b, 0xa2, 0x2c, 0x2b, 0x79, 0x4d, 0x94, 0xf3, 0x0a, 0x68, 0xa2, 0x0c, + 0x4a, 0x41, 0x13, 0xe5, 0x82, 0x52, 0xd4, 0x44, 0xb9, 0xa8, 0x94, 0x34, 0x51, 0x2e, 0x29, 0x65, + 0x4d, 0x94, 0xcb, 0xca, 0x8a, 0x26, 0xca, 0x2b, 0x8a, 0xa2, 0x89, 0xb2, 0xa2, 0xac, 0x6a, 0xa2, + 0xbc, 0xaa, 0xa0, 0xfa, 0x5f, 0x05, 0x28, 0x73, 0xfa, 0x45, 0x54, 0x7f, 0x04, 0xab, 0x2c, 0x2f, + 0xf6, 0x74, 0x60, 0xb8, 0x81, 0x90, 0x91, 0x38, 0xad, 0x2b, 0xe1, 0x41, 0xa4, 0xfc, 0x15, 0x28, + 0x79, 0xd8, 0xb4, 0x62, 0xc5, 0x34, 0x53, 0x2c, 0x52, 0x61, 0xa4, 0xf4, 0x35, 0x28, 0xb3, 0x4a, + 0x8b, 0xb5, 0x32, 0x4c, 0xab, 0xc4, 0xa4, 0x91, 0x5a, 0x0b, 0x4a, 0xc4, 0x35, 0xa7, 0xb1, 0x96, + 0xc8, 0x2a, 0xe7, 0xcd, 0x4b, 0xd8, 0x7c, 0xe8, 0x9a, 0xd3, 0x80, 0xc5, 0x45, 0x6a, 0x13, 0x8d, + 0x25, 0x9d, 0x4d, 0xa5, 0xd9, 0x04, 0x53, 0x8d, 0x27, 0x36, 0xf1, 0xd1, 0xf7, 0xa0, 0xe8, 0x31, + 0x89, 0x41, 0x15, 0xc3, 0x72, 0xbc, 0x06, 0xb4, 0xe0, 0x45, 0x20, 0xa4, 0xfe, 0x27, 0x01, 0xee, + 0x1d, 0xf6, 0x87, 0x78, 0x62, 0x6e, 0x0f, 0xcd, 0xe9, 0x20, 0x9a, 0x77, 0x5b, 0x00, 0x2c, 0x76, + 0x93, 0x18, 0xce, 0xc9, 0x6d, 0x06, 0xbc, 0x4c, 0xcd, 0xb6, 0xc8, 0xfe, 0x09, 0xfa, 0x21, 0x28, + 0x09, 0xe7, 0x8c, 0xb1, 0x4d, 0xfc, 0x60, 0x66, 0x3e, 0xbc, 0xba, 0x99, 0x25, 0x22, 0x0c, 0x40, + 0xcb, 0xde, 0x92, 0xb4, 0xfe, 0x06, 0xac, 0x25, 0x9d, 0x8e, 0x32, 0xd4, 0x02, 0x85, 0x4b, 0x4e, + 0x30, 0xb6, 0x8e, 0x4c, 0x6f, 0x80, 0x7d, 0xd4, 0x80, 0x7b, 0xc4, 0x37, 0x7d, 0x3c, 0xa1, 0x05, + 0x4c, 0xeb, 0xd9, 0x60, 0x0c, 0x15, 0x18, 0x43, 0x57, 0xa3, 0x23, 0x1a, 0xc7, 0x9e, 0x39, 0xc1, + 0xf5, 0x3f, 0x8a, 0xb0, 0x1a, 0x83, 0x84, 0xf9, 0xa0, 0x65, 0x64, 0x4f, 0x47, 0x46, 0xbc, 0x71, + 0xf0, 0x32, 0xb2, 0xa7, 0x23, 0xba, 0x75, 0x48, 0xf4, 0xf0, 0xd8, 0xb3, 0x91, 0x06, 0xa2, 0xe3, + 0xfa, 0xe1, 0xeb, 0xbd, 0xaa, 0x6b, 0xbf, 0x76, 0x47, 0x63, 0xdf, 0xf5, 0xf9, 0x38, 0xd0, 0x19, + 0x06, 0xfa, 0xbd, 0x00, 0x92, 0xcf, 0x82, 0x20, 0x6a, 0x8e, 0xe1, 0x7d, 0xfb, 0x56, 0x78, 0x3c, + 0x01, 0xc1, 0xa6, 0x71, 0x40, 0x13, 0xf9, 0x6a, 0x5e, 0x5d, 0xbd, 0x98, 0x20, 0x72, 0xd7, 0x15, + 0x24, 0xf4, 0x0d, 0x69, 0x50, 0x5e, 0xce, 0x30, 0x6b, 0x60, 0x37, 0xe4, 0x4b, 0x69, 0xe9, 0x0d, + 0x54, 0x06, 0x74, 0xde, 0xc7, 0x6e, 0x5f, 0x32, 0x18, 0xb7, 0x96, 0x07, 0xe3, 0xa3, 0x1b, 0xa5, + 0x84, 0x63, 0x26, 0xa6, 0x61, 0xe5, 0x43, 0xc8, 0x47, 0xf9, 0x4e, 0xde, 0x92, 0xe7, 0xb7, 0xac, + 0x25, 0x6f, 0xc9, 0xbf, 0x36, 0x46, 0xa3, 0xde, 0x95, 0x55, 0x72, 0xf5, 0x5f, 0x08, 0x50, 0xd4, + 0x31, 0x71, 0xc6, 0xcf, 0xb0, 0x45, 0x29, 0x8a, 0xbe, 0x09, 0x22, 0xa5, 0x7c, 0x50, 0x36, 0xd7, + 0x94, 0x23, 0x53, 0x45, 0x5b, 0x90, 0x8f, 0xd6, 0xfe, 0xdb, 0x6c, 0xc3, 0xb1, 0x55, 0xdd, 0x05, + 0x14, 0x07, 0x1c, 0x35, 0x9e, 0x23, 0xa0, 0xc5, 0xc3, 0x7c, 0x0b, 0x9a, 0x04, 0xaf, 0xc1, 0x07, + 0x57, 0xd7, 0x60, 0x14, 0x4c, 0xf8, 0x9a, 0xbc, 0x84, 0x8c, 0x04, 0xbb, 0xc4, 0x22, 0x0b, 0xd2, + 0x81, 0xf9, 0x62, 0xec, 0x98, 0x16, 0xaa, 0x41, 0x21, 0xdc, 0xb5, 0x6d, 0x67, 0x1a, 0xa4, 0x32, + 0x29, 0xa2, 0x23, 0x76, 0x46, 0xb0, 0xc7, 0xaa, 0x8f, 0x67, 0x35, 0x7a, 0xa6, 0x5d, 0x94, 0x7d, + 0x18, 0x60, 0xcb, 0x98, 0xd8, 0x7d, 0xcf, 0x21, 0xac, 0xc8, 0x32, 0x8c, 0x1d, 0x54, 0xfa, 0x94, + 0x09, 0xd1, 0x03, 0x58, 0x39, 0xb1, 0xa7, 0x36, 0x19, 0xc6, 0x7a, 0x6c, 0xa8, 0xeb, 0xe5, 0x50, + 0x1c, 0x28, 0x3a, 0x50, 0x8e, 0x37, 0x7f, 0xc3, 0xb6, 0x78, 0x01, 0x95, 0x5a, 0x9d, 0xc5, 0xbc, + 0x5a, 0x8a, 0xa7, 0x56, 0x77, 0x87, 0xdc, 0x75, 0x78, 0x97, 0x62, 0xfc, 0xae, 0x45, 0xd8, 0xe7, + 0x93, 0xe7, 0x39, 0x9e, 0x2a, 0x73, 0xbe, 0xb0, 0x07, 0xf4, 0x01, 0x64, 0xc7, 0xf4, 0xcb, 0x8d, + 0xcd, 0xe5, 0xc2, 0x66, 0xed, 0x8a, 0x9c, 0xb3, 0x2f, 0x3c, 0x9d, 0xab, 0xa3, 0x16, 0xe4, 0xf8, + 0xf0, 0x67, 0x23, 0xbb, 0xb0, 0xb9, 0x71, 0x85, 0xe1, 0xd2, 0x87, 0x59, 0x27, 0xa5, 0x07, 0x96, + 0xa8, 0x0d, 0x92, 0xc7, 0x17, 0x3d, 0xb5, 0xc0, 0x40, 0x1e, 0xde, 0x78, 0x85, 0xec, 0xa4, 0xf4, + 0xd0, 0x16, 0x1d, 0x41, 0x91, 0x24, 0x5a, 0xad, 0x5a, 0x64, 0x58, 0x57, 0x6d, 0x7c, 0x97, 0x8c, + 0x93, 0x0e, 0x1d, 0x65, 0x09, 0x31, 0x0d, 0xd0, 0x66, 0x93, 0x59, 0x2d, 0x5d, 0x1b, 0xe0, 0xd2, + 0x06, 0x49, 0x03, 0xe4, 0x96, 0x68, 0x0f, 0xa0, 0x1f, 0xf1, 0x5d, 0x2d, 0x33, 0x9c, 0xaf, 0xdf, + 0xa6, 0x41, 0x76, 0x52, 0x7a, 0x02, 0xa1, 0x95, 0x07, 0xc9, 0xe2, 0x07, 0xd1, 0x1e, 0x22, 0x29, + 0x72, 0xfd, 0xcf, 0x22, 0xc8, 0x51, 0x35, 0x35, 0x01, 0x9d, 0x78, 0x66, 0x9f, 0xf2, 0xd9, 0xe8, + 0x3b, 0x74, 0xc1, 0xf2, 0x31, 0xff, 0x10, 0x49, 0x77, 0x52, 0xfa, 0x6a, 0x78, 0xb6, 0x1d, 0x1e, + 0x51, 0xc6, 0x4e, 0x1c, 0xcb, 0x3e, 0xb1, 0x63, 0xc6, 0xf2, 0x0f, 0xec, 0x72, 0x28, 0x0e, 0x18, + 0xfb, 0xd1, 0xd2, 0x12, 0x9e, 0xb9, 0x41, 0x07, 0xe8, 0xa4, 0x12, 0x5b, 0x3a, 0xda, 0xbe, 0x40, + 0x99, 0x87, 0xd7, 0x52, 0x26, 0x0c, 0xaa, 0x23, 0x44, 0x9c, 0xd9, 0xbd, 0xc8, 0x99, 0x77, 0xaf, + 0xe7, 0x4c, 0x02, 0x26, 0x22, 0xcd, 0xf1, 0xa5, 0xa4, 0x69, 0xde, 0x90, 0x34, 0x09, 0xc4, 0x65, + 0xd6, 0x6c, 0x5f, 0x60, 0xcd, 0xc3, 0x6b, 0x59, 0x93, 0x8c, 0x31, 0xa0, 0xcd, 0xfe, 0x25, 0xb4, + 0x79, 0xef, 0x46, 0xb4, 0x49, 0x80, 0x25, 0x79, 0x03, 0x20, 0x87, 0x5b, 0x5d, 0x82, 0x43, 0xef, + 0x7e, 0x21, 0x80, 0x78, 0xf4, 0xc2, 0xc5, 0xe8, 0xab, 0x50, 0x38, 0xde, 0x3b, 0x3c, 0x68, 0x6f, + 0x77, 0x77, 0xbb, 0xed, 0x1d, 0x25, 0x55, 0xb9, 0x77, 0x76, 0x5e, 0x5b, 0xa1, 0x47, 0xc7, 0x53, + 0xe2, 0xe2, 0x3e, 0x63, 0x01, 0xaa, 0x40, 0xae, 0xb5, 0xb5, 0xfd, 0xc9, 0xf1, 0x81, 0x22, 0x54, + 0xca, 0x67, 0xe7, 0x35, 0xa0, 0x0a, 0xfc, 0x45, 0xa1, 0xfb, 0x20, 0xe9, 0xed, 0xc3, 0xa3, 0x7d, + 0xbd, 0xad, 0xa4, 0x2b, 0x2b, 0x67, 0xe7, 0xb5, 0x02, 0x3d, 0x0c, 0xf2, 0x8f, 0x1e, 0x40, 0xe9, + 0x70, 0xbb, 0xd3, 0x7e, 0xba, 0x65, 0x6c, 0x77, 0xb6, 0xf6, 0x3e, 0x6e, 0x2b, 0x99, 0xca, 0xda, + 0xd9, 0x79, 0x4d, 0xa1, 0x3a, 0xc9, 0xb4, 0xd2, 0x2b, 0xba, 0x4f, 0x0f, 0xf6, 0xf5, 0x23, 0x45, + 0x8c, 0xaf, 0xe0, 0x79, 0x42, 0x75, 0x00, 0x6e, 0xbd, 0xdb, 0x6e, 0xef, 0x28, 0xd9, 0x0a, 0x3a, + 0x3b, 0xaf, 0x95, 0xe9, 0x79, 0x1c, 0x7e, 0x45, 0xfe, 0xe5, 0x6f, 0xd7, 0x53, 0x7f, 0xf8, 0xdd, + 0x7a, 0xaa, 0x55, 0x7b, 0xf9, 0xef, 0xf5, 0xd4, 0xcb, 0xc5, 0xba, 0xf0, 0xf9, 0x62, 0x5d, 0xf8, + 0xfb, 0x62, 0x5d, 0xf8, 0xd7, 0x62, 0x5d, 0xf8, 0xd5, 0x7f, 0xd6, 0x53, 0x9f, 0xe6, 0x78, 0xd6, + 0x7a, 0x39, 0xf6, 0xa7, 0xab, 0xf7, 0xff, 0x17, 0x00, 0x00, 0xff, 0xff, 0x8f, 0xfd, 0x1b, 0xc6, + 0x6b, 0x13, 0x00, 0x00, } diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index 4feed1b813af..1a488d355b25 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -127,6 +127,12 @@ message SchemaChangeProgress { } +message ChangefeedTarget { + string statement_time_name = 1; + + // TODO(dan): Add partition name, ranges of primary keys. +} + message ChangefeedDetails { // Targets contains the user-specified tables and databases to watch, mapping // the descriptor id to the name at the time of changefeed creating. There is @@ -145,13 +151,16 @@ message ChangefeedDetails { // renames can be detected. They are also used to construct an error message // if the descriptor id no longer exists when the jobs is unpaused (which can // happen if it was dropped or truncated). - map targets = 5 [ - (gogoproto.castkey) = "github.com/cockroachdb/cockroach/pkg/sql/sqlbase.ID" + map targets = 6 [ + (gogoproto.castkey) = "github.com/cockroachdb/cockroach/pkg/sql/sqlbase.ID", + (gogoproto.casttype) = "ChangefeedTargets", + (gogoproto.nullable) = false ]; string sink_uri = 3 [(gogoproto.customname) = "SinkURI"]; map opts = 4; + util.hlc.Timestamp statement_time = 7 [(gogoproto.nullable) = false]; - reserved 1, 2; + reserved 1, 2, 5; } message ResolvedSpan { diff --git a/pkg/jobs/jobspb/wrap.go b/pkg/jobs/jobspb/wrap.go index 623318ec85d8..a4f1f1334798 100644 --- a/pkg/jobs/jobspb/wrap.go +++ b/pkg/jobs/jobspb/wrap.go @@ -17,6 +17,8 @@ package jobspb import ( "fmt" "strings" + + "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" ) // Details is a marker interface for job details proto structs. @@ -192,3 +194,6 @@ func (d ImportProgress) Completed() float32 { } return completed } + +// ChangefeedTargets is a set of id targets with metadata. +type ChangefeedTargets map[sqlbase.ID]ChangefeedTarget From a004ddbaa37a8997ba913d9eab736f8c3373f159 Mon Sep 17 00:00:00 2001 From: Andrew Couch Date: Tue, 28 Aug 2018 15:30:05 -0400 Subject: [PATCH 2/2] ui: allow long table rows to wrap Fixes: #29132 Release note (admin ui change): Allow long table rows to wrap, if necessary. --- pkg/ui/src/views/shared/util/table.styl | 1 - 1 file changed, 1 deletion(-) diff --git a/pkg/ui/src/views/shared/util/table.styl b/pkg/ui/src/views/shared/util/table.styl index 64a4519b1a09..7e7acfcd2c26 100644 --- a/pkg/ui/src/views/shared/util/table.styl +++ b/pkg/ui/src/views/shared/util/table.styl @@ -16,7 +16,6 @@ $table-base font-weight 300 font-size 14px vertical-align top - white-space nowrap color $stats-table-td--fg &__row