diff --git a/pkg/ccl/changefeedccl/changefeed_processors.go b/pkg/ccl/changefeedccl/changefeed_processors.go index e046705c7f74..10fa7670fb0e 100644 --- a/pkg/ccl/changefeedccl/changefeed_processors.go +++ b/pkg/ccl/changefeedccl/changefeed_processors.go @@ -337,6 +337,10 @@ type changeFrontier struct { // jobProgressedFn, if non-nil, is called to checkpoint the changefeed's // progress in the corresponding system job entry. jobProgressedFn func(context.Context, jobs.HighWaterProgressedFn) error + // highWaterAtStart is the greater of the job high-water and the timestamp the + // CHANGEFEED statement was run at. It's used in an assertion that we never + // regress the job high-water. + highWaterAtStart hlc.Timestamp // passthroughBuf, in some but not all flows, contains changed row data to // pass through unchanged to the gateway node. passthroughBuf encDatumRowBuffer @@ -441,6 +445,7 @@ func (cf *changeFrontier) Start(ctx context.Context) context.Context { cf.sink = makeMetricsSink(cf.metrics, cf.sink) cf.sink = &errorWrapperSink{wrapped: cf.sink} + cf.highWaterAtStart = cf.spec.Feed.StatementTime if cf.spec.JobID != 0 { job, err := cf.flowCtx.JobRegistry.LoadJob(ctx, cf.spec.JobID) if err != nil { @@ -448,6 +453,11 @@ func (cf *changeFrontier) Start(ctx context.Context) context.Context { return ctx } cf.jobProgressedFn = job.HighWaterProgressed + + p := job.Progress() + if ts := p.GetHighWater(); ts != nil { + cf.highWaterAtStart.Forward(*ts) + } } cf.metrics.mu.Lock() @@ -535,6 +545,20 @@ func (cf *changeFrontier) noteResolvedSpan(d sqlbase.EncDatum) error { `unmarshalling resolved span: %x`, raw) } + // Inserting a timestamp less than the one the changefeed flow started at + // could potentially regress the job progress. This is not expected, but it + // was a bug at one point, so assert to prevent regressions. + // + // TODO(dan): This is much more naturally expressed as an assert inside the + // job progress update closure, but it currently doesn't pass along the info + // we'd need to do it that way. + if !resolved.Timestamp.IsEmpty() && resolved.Timestamp.Less(cf.highWaterAtStart) { + log.ReportOrPanic(cf.Ctx, &cf.flowCtx.Settings.SV, + `got a span level timestamp %s for %s that is less than the initial high-water %s`, + log.Safe(resolved.Timestamp), resolved.Span, log.Safe(cf.highWaterAtStart)) + return nil + } + frontierChanged := cf.sf.Forward(resolved.Span, resolved.Timestamp) if frontierChanged { newResolved := cf.sf.Frontier() diff --git a/pkg/ccl/changefeedccl/changefeed_stmt.go b/pkg/ccl/changefeedccl/changefeed_stmt.go index be38449b48fe..b3f216d26322 100644 --- a/pkg/ccl/changefeedccl/changefeed_stmt.go +++ b/pkg/ccl/changefeedccl/changefeed_stmt.go @@ -456,11 +456,11 @@ func (b *changefeedResumer) Resume( return nil } if !IsRetryableError(err) { - log.Warningf(ctx, `CHANGEFEED job %d returning with error: %v`, jobID, err) + log.Warningf(ctx, `CHANGEFEED job %d returning with error: %+v`, jobID, err) return err } - log.Warningf(ctx, `CHANGEFEED job %d encountered retryable error: %+v`, jobID, err) + log.Warningf(ctx, `CHANGEFEED job %d encountered retryable error: %v`, jobID, err) if metrics, ok := execCfg.JobRegistry.MetricsStruct().Changefeed.(*Metrics); ok { metrics.ErrorRetries.Inc(1) } diff --git a/pkg/ccl/changefeedccl/poller.go b/pkg/ccl/changefeedccl/poller.go index 4782091cc5a1..98a8cd9b4adf 100644 --- a/pkg/ccl/changefeedccl/poller.go +++ b/pkg/ccl/changefeedccl/poller.go @@ -279,16 +279,12 @@ func (p *poller) rangefeedImpl(ctx context.Context) error { // the faster-to-implement solution for now. frontier := makeSpanFrontier(spans...) + rangeFeedStartTS := lastHighwater for _, span := range p.spans { - req := &roachpb.RangeFeedRequest{ - Header: roachpb.Header{ - Timestamp: lastHighwater, - }, - Span: span, - } - frontier.Forward(span, lastHighwater) + span := span + frontier.Forward(span, rangeFeedStartTS) g.GoCtx(func(ctx context.Context) error { - return ds.RangeFeed(ctx, req, eventC) + return ds.RangeFeed(ctx, span, rangeFeedStartTS, eventC) }) } g.GoCtx(func(ctx context.Context) error { @@ -302,6 +298,12 @@ func (p *poller) rangefeedImpl(ctx context.Context) error { return err } case *roachpb.RangeFeedCheckpoint: + if !t.ResolvedTS.IsEmpty() && t.ResolvedTS.Less(rangeFeedStartTS) { + // RangeFeed happily forwards any closed timestamps it receives as + // soon as there are no outstanding intents under them. + // Changefeeds don't care about these at all, so throw them out. + continue + } if err := memBuf.AddResolved(ctx, t.Span, t.ResolvedTS); err != nil { return err } diff --git a/pkg/kv/dist_sender_rangefeed.go b/pkg/kv/dist_sender_rangefeed.go index 5645dc246f55..864c3a0d914f 100644 --- a/pkg/kv/dist_sender_rangefeed.go +++ b/pkg/kv/dist_sender_rangefeed.go @@ -39,18 +39,21 @@ type singleRangeInfo struct { // RangeFeed divides a RangeFeed request on range boundaries and establishes a // RangeFeed to each of the individual ranges. It streams back results on the // provided channel. +// +// Note that the timestamps in RangeFeedCheckpoint events that are streamed back +// may be lower than the timestamp given here. func (ds *DistSender) RangeFeed( - ctx context.Context, args *roachpb.RangeFeedRequest, eventCh chan<- *roachpb.RangeFeedEvent, + ctx context.Context, span roachpb.Span, ts hlc.Timestamp, eventCh chan<- *roachpb.RangeFeedEvent, ) error { ctx = ds.AnnotateCtx(ctx) ctx, sp := tracing.EnsureChildSpan(ctx, ds.AmbientContext.Tracer, "dist sender") defer sp.Finish() - startRKey, err := keys.Addr(args.Span.Key) + startRKey, err := keys.Addr(span.Key) if err != nil { return err } - endRKey, err := keys.Addr(args.Span.EndKey) + endRKey, err := keys.Addr(span.EndKey) if err != nil { return err } @@ -76,7 +79,7 @@ func (ds *DistSender) RangeFeed( // Kick off the initial set of ranges. g.GoCtx(func(ctx context.Context) error { - return ds.divideAndSendRangeFeedToRanges(ctx, rs, args.Timestamp, rangeCh) + return ds.divideAndSendRangeFeedToRanges(ctx, rs, ts, rangeCh) }) return g.Wait() diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index ab0dd6a7171a..4cd46aaa574b 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -70,7 +70,7 @@ func (x ReadConsistencyType) String() string { return proto.EnumName(ReadConsistencyType_name, int32(x)) } func (ReadConsistencyType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{0} + return fileDescriptor_api_c281d1b5fba7c11f, []int{0} } // ScanFormat is an enumeration of the available response formats for MVCCScan @@ -98,7 +98,7 @@ func (x ScanFormat) String() string { return proto.EnumName(ScanFormat_name, int32(x)) } func (ScanFormat) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{1} + return fileDescriptor_api_c281d1b5fba7c11f, []int{1} } type ChecksumMode int32 @@ -145,7 +145,7 @@ func (x ChecksumMode) String() string { return proto.EnumName(ChecksumMode_name, int32(x)) } func (ChecksumMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{2} + return fileDescriptor_api_c281d1b5fba7c11f, []int{2} } // PushTxnType determines what action to take when pushing a transaction. @@ -176,7 +176,7 @@ func (x PushTxnType) String() string { return proto.EnumName(PushTxnType_name, int32(x)) } func (PushTxnType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{3} + return fileDescriptor_api_c281d1b5fba7c11f, []int{3} } type ExportStorageProvider int32 @@ -214,7 +214,7 @@ func (x ExportStorageProvider) String() string { return proto.EnumName(ExportStorageProvider_name, int32(x)) } func (ExportStorageProvider) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{4} + return fileDescriptor_api_c281d1b5fba7c11f, []int{4} } type MVCCFilter int32 @@ -237,7 +237,7 @@ func (x MVCCFilter) String() string { return proto.EnumName(MVCCFilter_name, int32(x)) } func (MVCCFilter) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{5} + return fileDescriptor_api_c281d1b5fba7c11f, []int{5} } type ResponseHeader_ResumeReason int32 @@ -269,7 +269,7 @@ func (x ResponseHeader_ResumeReason) String() string { return proto.EnumName(ResponseHeader_ResumeReason_name, int32(x)) } func (ResponseHeader_ResumeReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{2, 0} + return fileDescriptor_api_c281d1b5fba7c11f, []int{2, 0} } type CheckConsistencyResponse_Status int32 @@ -311,7 +311,7 @@ func (x CheckConsistencyResponse_Status) String() string { return proto.EnumName(CheckConsistencyResponse_Status_name, int32(x)) } func (CheckConsistencyResponse_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{25, 0} + return fileDescriptor_api_c281d1b5fba7c11f, []int{25, 0} } type QueryIntentRequest_IfMissingBehavior int32 @@ -345,7 +345,7 @@ func (x QueryIntentRequest_IfMissingBehavior) String() string { return proto.EnumName(QueryIntentRequest_IfMissingBehavior_name, int32(x)) } func (QueryIntentRequest_IfMissingBehavior) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{52, 0} + return fileDescriptor_api_c281d1b5fba7c11f, []int{52, 0} } // RangeInfo describes a range which executed a request. It contains @@ -361,7 +361,7 @@ func (m *RangeInfo) Reset() { *m = RangeInfo{} } func (m *RangeInfo) String() string { return proto.CompactTextString(m) } func (*RangeInfo) ProtoMessage() {} func (*RangeInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{0} + return fileDescriptor_api_c281d1b5fba7c11f, []int{0} } func (m *RangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -406,7 +406,7 @@ func (m *RequestHeader) Reset() { *m = RequestHeader{} } func (m *RequestHeader) String() string { return proto.CompactTextString(m) } func (*RequestHeader) ProtoMessage() {} func (*RequestHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{1} + return fileDescriptor_api_c281d1b5fba7c11f, []int{1} } func (m *RequestHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -466,7 +466,7 @@ func (m *ResponseHeader) Reset() { *m = ResponseHeader{} } func (m *ResponseHeader) String() string { return proto.CompactTextString(m) } func (*ResponseHeader) ProtoMessage() {} func (*ResponseHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{2} + return fileDescriptor_api_c281d1b5fba7c11f, []int{2} } func (m *ResponseHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -502,7 +502,7 @@ func (m *GetRequest) Reset() { *m = GetRequest{} } func (m *GetRequest) String() string { return proto.CompactTextString(m) } func (*GetRequest) ProtoMessage() {} func (*GetRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{3} + return fileDescriptor_api_c281d1b5fba7c11f, []int{3} } func (m *GetRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -542,7 +542,7 @@ func (m *GetResponse) Reset() { *m = GetResponse{} } func (m *GetResponse) String() string { return proto.CompactTextString(m) } func (*GetResponse) ProtoMessage() {} func (*GetResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{4} + return fileDescriptor_api_c281d1b5fba7c11f, []int{4} } func (m *GetResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -587,7 +587,7 @@ func (m *PutRequest) Reset() { *m = PutRequest{} } func (m *PutRequest) String() string { return proto.CompactTextString(m) } func (*PutRequest) ProtoMessage() {} func (*PutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{5} + return fileDescriptor_api_c281d1b5fba7c11f, []int{5} } func (m *PutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -623,7 +623,7 @@ func (m *PutResponse) Reset() { *m = PutResponse{} } func (m *PutResponse) String() string { return proto.CompactTextString(m) } func (*PutResponse) ProtoMessage() {} func (*PutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{6} + return fileDescriptor_api_c281d1b5fba7c11f, []int{6} } func (m *PutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -679,7 +679,7 @@ func (m *ConditionalPutRequest) Reset() { *m = ConditionalPutRequest{} } func (m *ConditionalPutRequest) String() string { return proto.CompactTextString(m) } func (*ConditionalPutRequest) ProtoMessage() {} func (*ConditionalPutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{7} + return fileDescriptor_api_c281d1b5fba7c11f, []int{7} } func (m *ConditionalPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -716,7 +716,7 @@ func (m *ConditionalPutResponse) Reset() { *m = ConditionalPutResponse{} func (m *ConditionalPutResponse) String() string { return proto.CompactTextString(m) } func (*ConditionalPutResponse) ProtoMessage() {} func (*ConditionalPutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{8} + return fileDescriptor_api_c281d1b5fba7c11f, []int{8} } func (m *ConditionalPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -764,7 +764,7 @@ func (m *InitPutRequest) Reset() { *m = InitPutRequest{} } func (m *InitPutRequest) String() string { return proto.CompactTextString(m) } func (*InitPutRequest) ProtoMessage() {} func (*InitPutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{9} + return fileDescriptor_api_c281d1b5fba7c11f, []int{9} } func (m *InitPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -800,7 +800,7 @@ func (m *InitPutResponse) Reset() { *m = InitPutResponse{} } func (m *InitPutResponse) String() string { return proto.CompactTextString(m) } func (*InitPutResponse) ProtoMessage() {} func (*InitPutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{10} + return fileDescriptor_api_c281d1b5fba7c11f, []int{10} } func (m *InitPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -842,7 +842,7 @@ func (m *IncrementRequest) Reset() { *m = IncrementRequest{} } func (m *IncrementRequest) String() string { return proto.CompactTextString(m) } func (*IncrementRequest) ProtoMessage() {} func (*IncrementRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{11} + return fileDescriptor_api_c281d1b5fba7c11f, []int{11} } func (m *IncrementRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -881,7 +881,7 @@ func (m *IncrementResponse) Reset() { *m = IncrementResponse{} } func (m *IncrementResponse) String() string { return proto.CompactTextString(m) } func (*IncrementResponse) ProtoMessage() {} func (*IncrementResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{12} + return fileDescriptor_api_c281d1b5fba7c11f, []int{12} } func (m *IncrementResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -917,7 +917,7 @@ func (m *DeleteRequest) Reset() { *m = DeleteRequest{} } func (m *DeleteRequest) String() string { return proto.CompactTextString(m) } func (*DeleteRequest) ProtoMessage() {} func (*DeleteRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{13} + return fileDescriptor_api_c281d1b5fba7c11f, []int{13} } func (m *DeleteRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -953,7 +953,7 @@ func (m *DeleteResponse) Reset() { *m = DeleteResponse{} } func (m *DeleteResponse) String() string { return proto.CompactTextString(m) } func (*DeleteResponse) ProtoMessage() {} func (*DeleteResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{14} + return fileDescriptor_api_c281d1b5fba7c11f, []int{14} } func (m *DeleteResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1004,7 +1004,7 @@ func (m *DeleteRangeRequest) Reset() { *m = DeleteRangeRequest{} } func (m *DeleteRangeRequest) String() string { return proto.CompactTextString(m) } func (*DeleteRangeRequest) ProtoMessage() {} func (*DeleteRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{15} + return fileDescriptor_api_c281d1b5fba7c11f, []int{15} } func (m *DeleteRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1043,7 +1043,7 @@ func (m *DeleteRangeResponse) Reset() { *m = DeleteRangeResponse{} } func (m *DeleteRangeResponse) String() string { return proto.CompactTextString(m) } func (*DeleteRangeResponse) ProtoMessage() {} func (*DeleteRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{16} + return fileDescriptor_api_c281d1b5fba7c11f, []int{16} } func (m *DeleteRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1092,7 +1092,7 @@ func (m *ClearRangeRequest) Reset() { *m = ClearRangeRequest{} } func (m *ClearRangeRequest) String() string { return proto.CompactTextString(m) } func (*ClearRangeRequest) ProtoMessage() {} func (*ClearRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{17} + return fileDescriptor_api_c281d1b5fba7c11f, []int{17} } func (m *ClearRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1128,7 +1128,7 @@ func (m *ClearRangeResponse) Reset() { *m = ClearRangeResponse{} } func (m *ClearRangeResponse) String() string { return proto.CompactTextString(m) } func (*ClearRangeResponse) ProtoMessage() {} func (*ClearRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{18} + return fileDescriptor_api_c281d1b5fba7c11f, []int{18} } func (m *ClearRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1199,7 +1199,7 @@ func (m *ScanOptions) Reset() { *m = ScanOptions{} } func (m *ScanOptions) String() string { return proto.CompactTextString(m) } func (*ScanOptions) ProtoMessage() {} func (*ScanOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{19} + return fileDescriptor_api_c281d1b5fba7c11f, []int{19} } func (m *ScanOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1241,7 +1241,7 @@ func (m *ScanRequest) Reset() { *m = ScanRequest{} } func (m *ScanRequest) String() string { return proto.CompactTextString(m) } func (*ScanRequest) ProtoMessage() {} func (*ScanRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{20} + return fileDescriptor_api_c281d1b5fba7c11f, []int{20} } func (m *ScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1290,7 +1290,7 @@ func (m *ScanResponse) Reset() { *m = ScanResponse{} } func (m *ScanResponse) String() string { return proto.CompactTextString(m) } func (*ScanResponse) ProtoMessage() {} func (*ScanResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{21} + return fileDescriptor_api_c281d1b5fba7c11f, []int{21} } func (m *ScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1332,7 +1332,7 @@ func (m *ReverseScanRequest) Reset() { *m = ReverseScanRequest{} } func (m *ReverseScanRequest) String() string { return proto.CompactTextString(m) } func (*ReverseScanRequest) ProtoMessage() {} func (*ReverseScanRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{22} + return fileDescriptor_api_c281d1b5fba7c11f, []int{22} } func (m *ReverseScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1381,7 +1381,7 @@ func (m *ReverseScanResponse) Reset() { *m = ReverseScanResponse{} } func (m *ReverseScanResponse) String() string { return proto.CompactTextString(m) } func (*ReverseScanResponse) ProtoMessage() {} func (*ReverseScanResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{23} + return fileDescriptor_api_c281d1b5fba7c11f, []int{23} } func (m *ReverseScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1429,7 +1429,7 @@ func (m *CheckConsistencyRequest) Reset() { *m = CheckConsistencyRequest func (m *CheckConsistencyRequest) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyRequest) ProtoMessage() {} func (*CheckConsistencyRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{24} + return fileDescriptor_api_c281d1b5fba7c11f, []int{24} } func (m *CheckConsistencyRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1468,7 +1468,7 @@ func (m *CheckConsistencyResponse) Reset() { *m = CheckConsistencyRespon func (m *CheckConsistencyResponse) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyResponse) ProtoMessage() {} func (*CheckConsistencyResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{25} + return fileDescriptor_api_c281d1b5fba7c11f, []int{25} } func (m *CheckConsistencyResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1514,7 +1514,7 @@ func (m *CheckConsistencyResponse_Result) Reset() { *m = CheckConsistenc func (m *CheckConsistencyResponse_Result) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyResponse_Result) ProtoMessage() {} func (*CheckConsistencyResponse_Result) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{25, 0} + return fileDescriptor_api_c281d1b5fba7c11f, []int{25, 0} } func (m *CheckConsistencyResponse_Result) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1564,7 +1564,7 @@ func (m *RecomputeStatsRequest) Reset() { *m = RecomputeStatsRequest{} } func (m *RecomputeStatsRequest) String() string { return proto.CompactTextString(m) } func (*RecomputeStatsRequest) ProtoMessage() {} func (*RecomputeStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{26} + return fileDescriptor_api_c281d1b5fba7c11f, []int{26} } func (m *RecomputeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1602,7 +1602,7 @@ func (m *RecomputeStatsResponse) Reset() { *m = RecomputeStatsResponse{} func (m *RecomputeStatsResponse) String() string { return proto.CompactTextString(m) } func (*RecomputeStatsResponse) ProtoMessage() {} func (*RecomputeStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{27} + return fileDescriptor_api_c281d1b5fba7c11f, []int{27} } func (m *RecomputeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1638,7 +1638,7 @@ func (m *BeginTransactionRequest) Reset() { *m = BeginTransactionRequest func (m *BeginTransactionRequest) String() string { return proto.CompactTextString(m) } func (*BeginTransactionRequest) ProtoMessage() {} func (*BeginTransactionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{28} + return fileDescriptor_api_c281d1b5fba7c11f, []int{28} } func (m *BeginTransactionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1674,7 +1674,7 @@ func (m *BeginTransactionResponse) Reset() { *m = BeginTransactionRespon func (m *BeginTransactionResponse) String() string { return proto.CompactTextString(m) } func (*BeginTransactionResponse) ProtoMessage() {} func (*BeginTransactionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{29} + return fileDescriptor_api_c281d1b5fba7c11f, []int{29} } func (m *BeginTransactionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1744,7 +1744,7 @@ func (m *EndTransactionRequest) Reset() { *m = EndTransactionRequest{} } func (m *EndTransactionRequest) String() string { return proto.CompactTextString(m) } func (*EndTransactionRequest) ProtoMessage() {} func (*EndTransactionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{30} + return fileDescriptor_api_c281d1b5fba7c11f, []int{30} } func (m *EndTransactionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1790,7 +1790,7 @@ func (m *EndTransactionResponse) Reset() { *m = EndTransactionResponse{} func (m *EndTransactionResponse) String() string { return proto.CompactTextString(m) } func (*EndTransactionResponse) ProtoMessage() {} func (*EndTransactionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{31} + return fileDescriptor_api_c281d1b5fba7c11f, []int{31} } func (m *EndTransactionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1846,7 +1846,7 @@ func (m *AdminSplitRequest) Reset() { *m = AdminSplitRequest{} } func (m *AdminSplitRequest) String() string { return proto.CompactTextString(m) } func (*AdminSplitRequest) ProtoMessage() {} func (*AdminSplitRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{32} + return fileDescriptor_api_c281d1b5fba7c11f, []int{32} } func (m *AdminSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1883,7 +1883,7 @@ func (m *AdminSplitResponse) Reset() { *m = AdminSplitResponse{} } func (m *AdminSplitResponse) String() string { return proto.CompactTextString(m) } func (*AdminSplitResponse) ProtoMessage() {} func (*AdminSplitResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{33} + return fileDescriptor_api_c281d1b5fba7c11f, []int{33} } func (m *AdminSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1927,7 +1927,7 @@ func (m *AdminMergeRequest) Reset() { *m = AdminMergeRequest{} } func (m *AdminMergeRequest) String() string { return proto.CompactTextString(m) } func (*AdminMergeRequest) ProtoMessage() {} func (*AdminMergeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{34} + return fileDescriptor_api_c281d1b5fba7c11f, []int{34} } func (m *AdminMergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1964,7 +1964,7 @@ func (m *AdminMergeResponse) Reset() { *m = AdminMergeResponse{} } func (m *AdminMergeResponse) String() string { return proto.CompactTextString(m) } func (*AdminMergeResponse) ProtoMessage() {} func (*AdminMergeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{35} + return fileDescriptor_api_c281d1b5fba7c11f, []int{35} } func (m *AdminMergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2004,7 +2004,7 @@ func (m *AdminTransferLeaseRequest) Reset() { *m = AdminTransferLeaseReq func (m *AdminTransferLeaseRequest) String() string { return proto.CompactTextString(m) } func (*AdminTransferLeaseRequest) ProtoMessage() {} func (*AdminTransferLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{36} + return fileDescriptor_api_c281d1b5fba7c11f, []int{36} } func (m *AdminTransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2039,7 +2039,7 @@ func (m *AdminTransferLeaseResponse) Reset() { *m = AdminTransferLeaseRe func (m *AdminTransferLeaseResponse) String() string { return proto.CompactTextString(m) } func (*AdminTransferLeaseResponse) ProtoMessage() {} func (*AdminTransferLeaseResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{37} + return fileDescriptor_api_c281d1b5fba7c11f, []int{37} } func (m *AdminTransferLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2093,7 +2093,7 @@ func (m *AdminChangeReplicasRequest) Reset() { *m = AdminChangeReplicasR func (m *AdminChangeReplicasRequest) String() string { return proto.CompactTextString(m) } func (*AdminChangeReplicasRequest) ProtoMessage() {} func (*AdminChangeReplicasRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{38} + return fileDescriptor_api_c281d1b5fba7c11f, []int{38} } func (m *AdminChangeReplicasRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2132,7 +2132,7 @@ func (m *AdminChangeReplicasResponse) Reset() { *m = AdminChangeReplicas func (m *AdminChangeReplicasResponse) String() string { return proto.CompactTextString(m) } func (*AdminChangeReplicasResponse) ProtoMessage() {} func (*AdminChangeReplicasResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{39} + return fileDescriptor_api_c281d1b5fba7c11f, []int{39} } func (m *AdminChangeReplicasResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2171,7 +2171,7 @@ func (m *AdminRelocateRangeRequest) Reset() { *m = AdminRelocateRangeReq func (m *AdminRelocateRangeRequest) String() string { return proto.CompactTextString(m) } func (*AdminRelocateRangeRequest) ProtoMessage() {} func (*AdminRelocateRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{40} + return fileDescriptor_api_c281d1b5fba7c11f, []int{40} } func (m *AdminRelocateRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2206,7 +2206,7 @@ func (m *AdminRelocateRangeResponse) Reset() { *m = AdminRelocateRangeRe func (m *AdminRelocateRangeResponse) String() string { return proto.CompactTextString(m) } func (*AdminRelocateRangeResponse) ProtoMessage() {} func (*AdminRelocateRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{41} + return fileDescriptor_api_c281d1b5fba7c11f, []int{41} } func (m *AdminRelocateRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2247,7 +2247,7 @@ func (m *HeartbeatTxnRequest) Reset() { *m = HeartbeatTxnRequest{} } func (m *HeartbeatTxnRequest) String() string { return proto.CompactTextString(m) } func (*HeartbeatTxnRequest) ProtoMessage() {} func (*HeartbeatTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{42} + return fileDescriptor_api_c281d1b5fba7c11f, []int{42} } func (m *HeartbeatTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2286,7 +2286,7 @@ func (m *HeartbeatTxnResponse) Reset() { *m = HeartbeatTxnResponse{} } func (m *HeartbeatTxnResponse) String() string { return proto.CompactTextString(m) } func (*HeartbeatTxnResponse) ProtoMessage() {} func (*HeartbeatTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{43} + return fileDescriptor_api_c281d1b5fba7c11f, []int{43} } func (m *HeartbeatTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2331,7 +2331,7 @@ func (m *GCRequest) Reset() { *m = GCRequest{} } func (m *GCRequest) String() string { return proto.CompactTextString(m) } func (*GCRequest) ProtoMessage() {} func (*GCRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{44} + return fileDescriptor_api_c281d1b5fba7c11f, []int{44} } func (m *GCRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2367,7 +2367,7 @@ func (m *GCRequest_GCKey) Reset() { *m = GCRequest_GCKey{} } func (m *GCRequest_GCKey) String() string { return proto.CompactTextString(m) } func (*GCRequest_GCKey) ProtoMessage() {} func (*GCRequest_GCKey) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{44, 0} + return fileDescriptor_api_c281d1b5fba7c11f, []int{44, 0} } func (m *GCRequest_GCKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2403,7 +2403,7 @@ func (m *GCResponse) Reset() { *m = GCResponse{} } func (m *GCResponse) String() string { return proto.CompactTextString(m) } func (*GCResponse) ProtoMessage() {} func (*GCResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{45} + return fileDescriptor_api_c281d1b5fba7c11f, []int{45} } func (m *GCResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2487,7 +2487,7 @@ func (m *PushTxnRequest) Reset() { *m = PushTxnRequest{} } func (m *PushTxnRequest) String() string { return proto.CompactTextString(m) } func (*PushTxnRequest) ProtoMessage() {} func (*PushTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{46} + return fileDescriptor_api_c281d1b5fba7c11f, []int{46} } func (m *PushTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2532,7 +2532,7 @@ func (m *PushTxnResponse) Reset() { *m = PushTxnResponse{} } func (m *PushTxnResponse) String() string { return proto.CompactTextString(m) } func (*PushTxnResponse) ProtoMessage() {} func (*PushTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{47} + return fileDescriptor_api_c281d1b5fba7c11f, []int{47} } func (m *PushTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2581,7 +2581,7 @@ func (m *RecoverTxnRequest) Reset() { *m = RecoverTxnRequest{} } func (m *RecoverTxnRequest) String() string { return proto.CompactTextString(m) } func (*RecoverTxnRequest) ProtoMessage() {} func (*RecoverTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{48} + return fileDescriptor_api_c281d1b5fba7c11f, []int{48} } func (m *RecoverTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2619,7 +2619,7 @@ func (m *RecoverTxnResponse) Reset() { *m = RecoverTxnResponse{} } func (m *RecoverTxnResponse) String() string { return proto.CompactTextString(m) } func (*RecoverTxnResponse) ProtoMessage() {} func (*RecoverTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{49} + return fileDescriptor_api_c281d1b5fba7c11f, []int{49} } func (m *RecoverTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2665,7 +2665,7 @@ func (m *QueryTxnRequest) Reset() { *m = QueryTxnRequest{} } func (m *QueryTxnRequest) String() string { return proto.CompactTextString(m) } func (*QueryTxnRequest) ProtoMessage() {} func (*QueryTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{50} + return fileDescriptor_api_c281d1b5fba7c11f, []int{50} } func (m *QueryTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2706,7 +2706,7 @@ func (m *QueryTxnResponse) Reset() { *m = QueryTxnResponse{} } func (m *QueryTxnResponse) String() string { return proto.CompactTextString(m) } func (*QueryTxnResponse) ProtoMessage() {} func (*QueryTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{51} + return fileDescriptor_api_c281d1b5fba7c11f, []int{51} } func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2768,7 +2768,7 @@ func (m *QueryIntentRequest) Reset() { *m = QueryIntentRequest{} } func (m *QueryIntentRequest) String() string { return proto.CompactTextString(m) } func (*QueryIntentRequest) ProtoMessage() {} func (*QueryIntentRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{52} + return fileDescriptor_api_c281d1b5fba7c11f, []int{52} } func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2806,7 +2806,7 @@ func (m *QueryIntentResponse) Reset() { *m = QueryIntentResponse{} } func (m *QueryIntentResponse) String() string { return proto.CompactTextString(m) } func (*QueryIntentResponse) ProtoMessage() {} func (*QueryIntentResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{53} + return fileDescriptor_api_c281d1b5fba7c11f, []int{53} } func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2852,7 +2852,7 @@ func (m *ResolveIntentRequest) Reset() { *m = ResolveIntentRequest{} } func (m *ResolveIntentRequest) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRequest) ProtoMessage() {} func (*ResolveIntentRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{54} + return fileDescriptor_api_c281d1b5fba7c11f, []int{54} } func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2889,7 +2889,7 @@ func (m *ResolveIntentResponse) Reset() { *m = ResolveIntentResponse{} } func (m *ResolveIntentResponse) String() string { return proto.CompactTextString(m) } func (*ResolveIntentResponse) ProtoMessage() {} func (*ResolveIntentResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{55} + return fileDescriptor_api_c281d1b5fba7c11f, []int{55} } func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2939,7 +2939,7 @@ func (m *ResolveIntentRangeRequest) Reset() { *m = ResolveIntentRangeReq func (m *ResolveIntentRangeRequest) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRangeRequest) ProtoMessage() {} func (*ResolveIntentRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{56} + return fileDescriptor_api_c281d1b5fba7c11f, []int{56} } func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2976,7 +2976,7 @@ func (m *ResolveIntentRangeResponse) Reset() { *m = ResolveIntentRangeRe func (m *ResolveIntentRangeResponse) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRangeResponse) ProtoMessage() {} func (*ResolveIntentRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{57} + return fileDescriptor_api_c281d1b5fba7c11f, []int{57} } func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3015,7 +3015,7 @@ func (m *MergeRequest) Reset() { *m = MergeRequest{} } func (m *MergeRequest) String() string { return proto.CompactTextString(m) } func (*MergeRequest) ProtoMessage() {} func (*MergeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{58} + return fileDescriptor_api_c281d1b5fba7c11f, []int{58} } func (m *MergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3051,7 +3051,7 @@ func (m *MergeResponse) Reset() { *m = MergeResponse{} } func (m *MergeResponse) String() string { return proto.CompactTextString(m) } func (*MergeResponse) ProtoMessage() {} func (*MergeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{59} + return fileDescriptor_api_c281d1b5fba7c11f, []int{59} } func (m *MergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3098,7 +3098,7 @@ func (m *TruncateLogRequest) Reset() { *m = TruncateLogRequest{} } func (m *TruncateLogRequest) String() string { return proto.CompactTextString(m) } func (*TruncateLogRequest) ProtoMessage() {} func (*TruncateLogRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{60} + return fileDescriptor_api_c281d1b5fba7c11f, []int{60} } func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3134,7 +3134,7 @@ func (m *TruncateLogResponse) Reset() { *m = TruncateLogResponse{} } func (m *TruncateLogResponse) String() string { return proto.CompactTextString(m) } func (*TruncateLogResponse) ProtoMessage() {} func (*TruncateLogResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{61} + return fileDescriptor_api_c281d1b5fba7c11f, []int{61} } func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3180,7 +3180,7 @@ func (m *RequestLeaseRequest) Reset() { *m = RequestLeaseRequest{} } func (m *RequestLeaseRequest) String() string { return proto.CompactTextString(m) } func (*RequestLeaseRequest) ProtoMessage() {} func (*RequestLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{62} + return fileDescriptor_api_c281d1b5fba7c11f, []int{62} } func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3231,7 +3231,7 @@ func (m *TransferLeaseRequest) Reset() { *m = TransferLeaseRequest{} } func (m *TransferLeaseRequest) String() string { return proto.CompactTextString(m) } func (*TransferLeaseRequest) ProtoMessage() {} func (*TransferLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{63} + return fileDescriptor_api_c281d1b5fba7c11f, []int{63} } func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3270,7 +3270,7 @@ func (m *LeaseInfoRequest) Reset() { *m = LeaseInfoRequest{} } func (m *LeaseInfoRequest) String() string { return proto.CompactTextString(m) } func (*LeaseInfoRequest) ProtoMessage() {} func (*LeaseInfoRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{64} + return fileDescriptor_api_c281d1b5fba7c11f, []int{64} } func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3309,7 +3309,7 @@ func (m *LeaseInfoResponse) Reset() { *m = LeaseInfoResponse{} } func (m *LeaseInfoResponse) String() string { return proto.CompactTextString(m) } func (*LeaseInfoResponse) ProtoMessage() {} func (*LeaseInfoResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{65} + return fileDescriptor_api_c281d1b5fba7c11f, []int{65} } func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3346,7 +3346,7 @@ func (m *RequestLeaseResponse) Reset() { *m = RequestLeaseResponse{} } func (m *RequestLeaseResponse) String() string { return proto.CompactTextString(m) } func (*RequestLeaseResponse) ProtoMessage() {} func (*RequestLeaseResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{66} + return fileDescriptor_api_c281d1b5fba7c11f, []int{66} } func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3398,7 +3398,7 @@ func (m *ComputeChecksumRequest) Reset() { *m = ComputeChecksumRequest{} func (m *ComputeChecksumRequest) String() string { return proto.CompactTextString(m) } func (*ComputeChecksumRequest) ProtoMessage() {} func (*ComputeChecksumRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{67} + return fileDescriptor_api_c281d1b5fba7c11f, []int{67} } func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3437,7 +3437,7 @@ func (m *ComputeChecksumResponse) Reset() { *m = ComputeChecksumResponse func (m *ComputeChecksumResponse) String() string { return proto.CompactTextString(m) } func (*ComputeChecksumResponse) ProtoMessage() {} func (*ComputeChecksumResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{68} + return fileDescriptor_api_c281d1b5fba7c11f, []int{68} } func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3478,7 +3478,7 @@ func (m *ExportStorage) Reset() { *m = ExportStorage{} } func (m *ExportStorage) String() string { return proto.CompactTextString(m) } func (*ExportStorage) ProtoMessage() {} func (*ExportStorage) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{69} + return fileDescriptor_api_c281d1b5fba7c11f, []int{69} } func (m *ExportStorage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3514,7 +3514,7 @@ func (m *ExportStorage_LocalFilePath) Reset() { *m = ExportStorage_Local func (m *ExportStorage_LocalFilePath) String() string { return proto.CompactTextString(m) } func (*ExportStorage_LocalFilePath) ProtoMessage() {} func (*ExportStorage_LocalFilePath) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{69, 0} + return fileDescriptor_api_c281d1b5fba7c11f, []int{69, 0} } func (m *ExportStorage_LocalFilePath) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3549,7 +3549,7 @@ func (m *ExportStorage_Http) Reset() { *m = ExportStorage_Http{} } func (m *ExportStorage_Http) String() string { return proto.CompactTextString(m) } func (*ExportStorage_Http) ProtoMessage() {} func (*ExportStorage_Http) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{69, 1} + return fileDescriptor_api_c281d1b5fba7c11f, []int{69, 1} } func (m *ExportStorage_Http) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3590,7 +3590,7 @@ func (m *ExportStorage_S3) Reset() { *m = ExportStorage_S3{} } func (m *ExportStorage_S3) String() string { return proto.CompactTextString(m) } func (*ExportStorage_S3) ProtoMessage() {} func (*ExportStorage_S3) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{69, 2} + return fileDescriptor_api_c281d1b5fba7c11f, []int{69, 2} } func (m *ExportStorage_S3) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3631,7 +3631,7 @@ func (m *ExportStorage_GCS) Reset() { *m = ExportStorage_GCS{} } func (m *ExportStorage_GCS) String() string { return proto.CompactTextString(m) } func (*ExportStorage_GCS) ProtoMessage() {} func (*ExportStorage_GCS) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{69, 3} + return fileDescriptor_api_c281d1b5fba7c11f, []int{69, 3} } func (m *ExportStorage_GCS) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3669,7 +3669,7 @@ func (m *ExportStorage_Azure) Reset() { *m = ExportStorage_Azure{} } func (m *ExportStorage_Azure) String() string { return proto.CompactTextString(m) } func (*ExportStorage_Azure) ProtoMessage() {} func (*ExportStorage_Azure) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{69, 4} + return fileDescriptor_api_c281d1b5fba7c11f, []int{69, 4} } func (m *ExportStorage_Azure) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3710,7 +3710,7 @@ func (m *ExportStorage_Workload) Reset() { *m = ExportStorage_Workload{} func (m *ExportStorage_Workload) String() string { return proto.CompactTextString(m) } func (*ExportStorage_Workload) ProtoMessage() {} func (*ExportStorage_Workload) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{69, 5} + return fileDescriptor_api_c281d1b5fba7c11f, []int{69, 5} } func (m *ExportStorage_Workload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3752,7 +3752,7 @@ func (m *WriteBatchRequest) Reset() { *m = WriteBatchRequest{} } func (m *WriteBatchRequest) String() string { return proto.CompactTextString(m) } func (*WriteBatchRequest) ProtoMessage() {} func (*WriteBatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{70} + return fileDescriptor_api_c281d1b5fba7c11f, []int{70} } func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3788,7 +3788,7 @@ func (m *WriteBatchResponse) Reset() { *m = WriteBatchResponse{} } func (m *WriteBatchResponse) String() string { return proto.CompactTextString(m) } func (*WriteBatchResponse) ProtoMessage() {} func (*WriteBatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{71} + return fileDescriptor_api_c281d1b5fba7c11f, []int{71} } func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3848,7 +3848,7 @@ func (m *ExportRequest) Reset() { *m = ExportRequest{} } func (m *ExportRequest) String() string { return proto.CompactTextString(m) } func (*ExportRequest) ProtoMessage() {} func (*ExportRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{72} + return fileDescriptor_api_c281d1b5fba7c11f, []int{72} } func (m *ExportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3886,7 +3886,7 @@ func (m *BulkOpSummary) Reset() { *m = BulkOpSummary{} } func (m *BulkOpSummary) String() string { return proto.CompactTextString(m) } func (*BulkOpSummary) ProtoMessage() {} func (*BulkOpSummary) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{73} + return fileDescriptor_api_c281d1b5fba7c11f, []int{73} } func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3924,7 +3924,7 @@ func (m *ExportResponse) Reset() { *m = ExportResponse{} } func (m *ExportResponse) String() string { return proto.CompactTextString(m) } func (*ExportResponse) ProtoMessage() {} func (*ExportResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{74} + return fileDescriptor_api_c281d1b5fba7c11f, []int{74} } func (m *ExportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3965,7 +3965,7 @@ func (m *ExportResponse_File) Reset() { *m = ExportResponse_File{} } func (m *ExportResponse_File) String() string { return proto.CompactTextString(m) } func (*ExportResponse_File) ProtoMessage() {} func (*ExportResponse_File) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{74, 0} + return fileDescriptor_api_c281d1b5fba7c11f, []int{74, 0} } func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4017,7 +4017,7 @@ func (m *ImportRequest) Reset() { *m = ImportRequest{} } func (m *ImportRequest) String() string { return proto.CompactTextString(m) } func (*ImportRequest) ProtoMessage() {} func (*ImportRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{75} + return fileDescriptor_api_c281d1b5fba7c11f, []int{75} } func (m *ImportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4054,7 +4054,7 @@ func (m *ImportRequest_File) Reset() { *m = ImportRequest_File{} } func (m *ImportRequest_File) String() string { return proto.CompactTextString(m) } func (*ImportRequest_File) ProtoMessage() {} func (*ImportRequest_File) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{75, 0} + return fileDescriptor_api_c281d1b5fba7c11f, []int{75, 0} } func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4092,7 +4092,7 @@ func (m *ImportRequest_TableRekey) Reset() { *m = ImportRequest_TableRek func (m *ImportRequest_TableRekey) String() string { return proto.CompactTextString(m) } func (*ImportRequest_TableRekey) ProtoMessage() {} func (*ImportRequest_TableRekey) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{75, 1} + return fileDescriptor_api_c281d1b5fba7c11f, []int{75, 1} } func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4129,7 +4129,7 @@ func (m *ImportResponse) Reset() { *m = ImportResponse{} } func (m *ImportResponse) String() string { return proto.CompactTextString(m) } func (*ImportResponse) ProtoMessage() {} func (*ImportResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{76} + return fileDescriptor_api_c281d1b5fba7c11f, []int{76} } func (m *ImportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4169,7 +4169,7 @@ func (m *AdminScatterRequest) Reset() { *m = AdminScatterRequest{} } func (m *AdminScatterRequest) String() string { return proto.CompactTextString(m) } func (*AdminScatterRequest) ProtoMessage() {} func (*AdminScatterRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{77} + return fileDescriptor_api_c281d1b5fba7c11f, []int{77} } func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4206,7 +4206,7 @@ func (m *AdminScatterResponse) Reset() { *m = AdminScatterResponse{} } func (m *AdminScatterResponse) String() string { return proto.CompactTextString(m) } func (*AdminScatterResponse) ProtoMessage() {} func (*AdminScatterResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{78} + return fileDescriptor_api_c281d1b5fba7c11f, []int{78} } func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4241,7 +4241,7 @@ func (m *AdminScatterResponse_Range) Reset() { *m = AdminScatterResponse func (m *AdminScatterResponse_Range) String() string { return proto.CompactTextString(m) } func (*AdminScatterResponse_Range) ProtoMessage() {} func (*AdminScatterResponse_Range) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{78, 0} + return fileDescriptor_api_c281d1b5fba7c11f, []int{78, 0} } func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4279,7 +4279,7 @@ func (m *AddSSTableRequest) Reset() { *m = AddSSTableRequest{} } func (m *AddSSTableRequest) String() string { return proto.CompactTextString(m) } func (*AddSSTableRequest) ProtoMessage() {} func (*AddSSTableRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{79} + return fileDescriptor_api_c281d1b5fba7c11f, []int{79} } func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4315,7 +4315,7 @@ func (m *AddSSTableResponse) Reset() { *m = AddSSTableResponse{} } func (m *AddSSTableResponse) String() string { return proto.CompactTextString(m) } func (*AddSSTableResponse) ProtoMessage() {} func (*AddSSTableResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{80} + return fileDescriptor_api_c281d1b5fba7c11f, []int{80} } func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4361,7 +4361,7 @@ func (m *RefreshRequest) Reset() { *m = RefreshRequest{} } func (m *RefreshRequest) String() string { return proto.CompactTextString(m) } func (*RefreshRequest) ProtoMessage() {} func (*RefreshRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{81} + return fileDescriptor_api_c281d1b5fba7c11f, []int{81} } func (m *RefreshRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4397,7 +4397,7 @@ func (m *RefreshResponse) Reset() { *m = RefreshResponse{} } func (m *RefreshResponse) String() string { return proto.CompactTextString(m) } func (*RefreshResponse) ProtoMessage() {} func (*RefreshResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{82} + return fileDescriptor_api_c281d1b5fba7c11f, []int{82} } func (m *RefreshResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4439,7 +4439,7 @@ func (m *RefreshRangeRequest) Reset() { *m = RefreshRangeRequest{} } func (m *RefreshRangeRequest) String() string { return proto.CompactTextString(m) } func (*RefreshRangeRequest) ProtoMessage() {} func (*RefreshRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{83} + return fileDescriptor_api_c281d1b5fba7c11f, []int{83} } func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4475,7 +4475,7 @@ func (m *RefreshRangeResponse) Reset() { *m = RefreshRangeResponse{} } func (m *RefreshRangeResponse) String() string { return proto.CompactTextString(m) } func (*RefreshRangeResponse) ProtoMessage() {} func (*RefreshRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{84} + return fileDescriptor_api_c281d1b5fba7c11f, []int{84} } func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4529,7 +4529,7 @@ func (m *SubsumeRequest) Reset() { *m = SubsumeRequest{} } func (m *SubsumeRequest) String() string { return proto.CompactTextString(m) } func (*SubsumeRequest) ProtoMessage() {} func (*SubsumeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{85} + return fileDescriptor_api_c281d1b5fba7c11f, []int{85} } func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4576,7 +4576,7 @@ func (m *SubsumeResponse) Reset() { *m = SubsumeResponse{} } func (m *SubsumeResponse) String() string { return proto.CompactTextString(m) } func (*SubsumeResponse) ProtoMessage() {} func (*SubsumeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{86} + return fileDescriptor_api_c281d1b5fba7c11f, []int{86} } func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4613,7 +4613,7 @@ func (m *RangeStatsRequest) Reset() { *m = RangeStatsRequest{} } func (m *RangeStatsRequest) String() string { return proto.CompactTextString(m) } func (*RangeStatsRequest) ProtoMessage() {} func (*RangeStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{87} + return fileDescriptor_api_c281d1b5fba7c11f, []int{87} } func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4654,7 +4654,7 @@ func (m *RangeStatsResponse) Reset() { *m = RangeStatsResponse{} } func (m *RangeStatsResponse) String() string { return proto.CompactTextString(m) } func (*RangeStatsResponse) ProtoMessage() {} func (*RangeStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{88} + return fileDescriptor_api_c281d1b5fba7c11f, []int{88} } func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4737,7 +4737,7 @@ func (m *RequestUnion) Reset() { *m = RequestUnion{} } func (m *RequestUnion) String() string { return proto.CompactTextString(m) } func (*RequestUnion) ProtoMessage() {} func (*RequestUnion) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{89} + return fileDescriptor_api_c281d1b5fba7c11f, []int{89} } func (m *RequestUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6127,7 +6127,7 @@ func (m *ResponseUnion) Reset() { *m = ResponseUnion{} } func (m *ResponseUnion) String() string { return proto.CompactTextString(m) } func (*ResponseUnion) ProtoMessage() {} func (*ResponseUnion) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{90} + return fileDescriptor_api_c281d1b5fba7c11f, []int{90} } func (m *ResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7506,7 +7506,7 @@ func (m *Header) Reset() { *m = Header{} } func (m *Header) String() string { return proto.CompactTextString(m) } func (*Header) ProtoMessage() {} func (*Header) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{91} + return fileDescriptor_api_c281d1b5fba7c11f, []int{91} } func (m *Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7544,7 +7544,7 @@ type BatchRequest struct { func (m *BatchRequest) Reset() { *m = BatchRequest{} } func (*BatchRequest) ProtoMessage() {} func (*BatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{92} + return fileDescriptor_api_c281d1b5fba7c11f, []int{92} } func (m *BatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7583,7 +7583,7 @@ type BatchResponse struct { func (m *BatchResponse) Reset() { *m = BatchResponse{} } func (*BatchResponse) ProtoMessage() {} func (*BatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{93} + return fileDescriptor_api_c281d1b5fba7c11f, []int{93} } func (m *BatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7644,7 +7644,7 @@ func (m *BatchResponse_Header) Reset() { *m = BatchResponse_Header{} } func (m *BatchResponse_Header) String() string { return proto.CompactTextString(m) } func (*BatchResponse_Header) ProtoMessage() {} func (*BatchResponse_Header) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{93, 0} + return fileDescriptor_api_c281d1b5fba7c11f, []int{93, 0} } func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7682,7 +7682,7 @@ func (m *RangeFeedRequest) Reset() { *m = RangeFeedRequest{} } func (m *RangeFeedRequest) String() string { return proto.CompactTextString(m) } func (*RangeFeedRequest) ProtoMessage() {} func (*RangeFeedRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{94} + return fileDescriptor_api_c281d1b5fba7c11f, []int{94} } func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7720,7 +7720,7 @@ func (m *RangeFeedValue) Reset() { *m = RangeFeedValue{} } func (m *RangeFeedValue) String() string { return proto.CompactTextString(m) } func (*RangeFeedValue) ProtoMessage() {} func (*RangeFeedValue) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{95} + return fileDescriptor_api_c281d1b5fba7c11f, []int{95} } func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7749,6 +7749,9 @@ var xxx_messageInfo_RangeFeedValue proto.InternalMessageInfo // promise that no more RangeFeedValue events with keys in the specified span // with timestamps less than or equal to the specified resolved timestamp will // be emitted on the RangeFeed response stream. +// +// Note that these resolved timestamps may be lower than the timestamp used in +// the RangeFeedRequest used to start the RangeFeed. type RangeFeedCheckpoint struct { Span Span `protobuf:"bytes,1,opt,name=span,proto3" json:"span"` ResolvedTS hlc.Timestamp `protobuf:"bytes,2,opt,name=resolved_ts,json=resolvedTs,proto3" json:"resolved_ts"` @@ -7760,7 +7763,7 @@ func (m *RangeFeedCheckpoint) Reset() { *m = RangeFeedCheckpoint{} } func (m *RangeFeedCheckpoint) String() string { return proto.CompactTextString(m) } func (*RangeFeedCheckpoint) ProtoMessage() {} func (*RangeFeedCheckpoint) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{96} + return fileDescriptor_api_c281d1b5fba7c11f, []int{96} } func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7799,7 +7802,7 @@ func (m *RangeFeedError) Reset() { *m = RangeFeedError{} } func (m *RangeFeedError) String() string { return proto.CompactTextString(m) } func (*RangeFeedError) ProtoMessage() {} func (*RangeFeedError) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{97} + return fileDescriptor_api_c281d1b5fba7c11f, []int{97} } func (m *RangeFeedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7838,7 +7841,7 @@ func (m *RangeFeedEvent) Reset() { *m = RangeFeedEvent{} } func (m *RangeFeedEvent) String() string { return proto.CompactTextString(m) } func (*RangeFeedEvent) ProtoMessage() {} func (*RangeFeedEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_9e6916522f7d2776, []int{98} + return fileDescriptor_api_c281d1b5fba7c11f, []int{98} } func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -34997,9 +35000,9 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_9e6916522f7d2776) } +func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_c281d1b5fba7c11f) } -var fileDescriptor_api_9e6916522f7d2776 = []byte{ +var fileDescriptor_api_c281d1b5fba7c11f = []byte{ // 6655 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x5d, 0x5b, 0x8c, 0x23, 0xd9, 0x59, 0xee, 0xb2, 0xdd, 0xdd, 0xf6, 0xef, 0x4b, 0xbb, 0x4f, 0xcf, 0xc5, 0xd3, 0xbb, 0x3b, 0xdd, diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index c7f0944f335a..b2e09313a390 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -1761,6 +1761,9 @@ message RangeFeedValue { // promise that no more RangeFeedValue events with keys in the specified span // with timestamps less than or equal to the specified resolved timestamp will // be emitted on the RangeFeed response stream. +// +// Note that these resolved timestamps may be lower than the timestamp used in +// the RangeFeedRequest used to start the RangeFeed. message RangeFeedCheckpoint { Span span = 1 [(gogoproto.nullable) = false]; util.hlc.Timestamp resolved_ts = 2 [ diff --git a/pkg/storage/rangefeed/registry.go b/pkg/storage/rangefeed/registry.go index 4208d8b668f6..e13e4d19ac61 100644 --- a/pkg/storage/rangefeed/registry.go +++ b/pkg/storage/rangefeed/registry.go @@ -365,8 +365,11 @@ func (reg *registry) PublishToOverlapping(span roachpb.Span, event *roachpb.Rang // timestamps equal to or greater than the value's timestamp. minTS = t.Value.Timestamp case *roachpb.RangeFeedCheckpoint: - // Always publish checkpoint notifications, regardless - // of a registration's starting timestamp. + // Always publish checkpoint notifications, regardless of a registration's + // starting timestamp. + // + // TODO(dan): It's unclear if this is the right contract, it's certainly + // surprising. Revisit this once RangeFeed has more users. minTS = hlc.MaxTimestamp default: panic(fmt.Sprintf("unexpected RangeFeedEvent variant: %v", event)) diff --git a/pkg/storage/replica_rangefeed_test.go b/pkg/storage/replica_rangefeed_test.go index 345f4c108bd9..35a642e4f730 100644 --- a/pkg/storage/replica_rangefeed_test.go +++ b/pkg/storage/replica_rangefeed_test.go @@ -699,13 +699,10 @@ func TestReplicaRangefeedNudgeSlowClosedTimestamp(t *testing.T) { rangeFeedCh := make(chan *roachpb.RangeFeedEvent) rangeFeedChs[i] = rangeFeedCh go func() { - req := roachpb.RangeFeedRequest{ - Header: roachpb.Header{Timestamp: ts1}, - Span: roachpb.Span{ - Key: desc.StartKey.AsRawKey(), EndKey: desc.EndKey.AsRawKey(), - }, + span := roachpb.Span{ + Key: desc.StartKey.AsRawKey(), EndKey: desc.EndKey.AsRawKey(), } - rangeFeedErrC <- ds.RangeFeed(rangeFeedCtx, &req, rangeFeedCh) + rangeFeedErrC <- ds.RangeFeed(rangeFeedCtx, span, ts1, rangeFeedCh) }() }