diff --git a/pkg/kv/kvserver/batcheval/cmd_get.go b/pkg/kv/kvserver/batcheval/cmd_get.go index 7f778606a332..ffa57985415f 100644 --- a/pkg/kv/kvserver/batcheval/cmd_get.go +++ b/pkg/kv/kvserver/batcheval/cmd_get.go @@ -46,7 +46,11 @@ func Get( reply.Value = val if h.ReadConsistency == roachpb.READ_UNCOMMITTED { var intentVals []roachpb.KeyValue - intentVals, err = CollectIntentRows(ctx, reader, cArgs, intents) + // NOTE: MVCCGet uses a Prefix iterator, so we want to use one in + // CollectIntentRows as well so that we're guaranteed to use the same + // cached iterator and observe a consistent snapshot of the engine. + const usePrefixIter = true + intentVals, err = CollectIntentRows(ctx, reader, usePrefixIter, intents) if err == nil { switch len(intentVals) { case 0: diff --git a/pkg/kv/kvserver/batcheval/cmd_reverse_scan.go b/pkg/kv/kvserver/batcheval/cmd_reverse_scan.go index 5edc2b2c698b..1757c90f1774 100644 --- a/pkg/kv/kvserver/batcheval/cmd_reverse_scan.go +++ b/pkg/kv/kvserver/batcheval/cmd_reverse_scan.go @@ -76,7 +76,11 @@ func ReverseScan( } if h.ReadConsistency == roachpb.READ_UNCOMMITTED { - reply.IntentRows, err = CollectIntentRows(ctx, reader, cArgs, scanRes.Intents) + // NOTE: MVCCScan doesn't use a Prefix iterator, so we don't want to use + // one in CollectIntentRows either so that we're guaranteed to use the + // same cached iterator and observe a consistent snapshot of the engine. + const usePrefixIter = false + reply.IntentRows, err = CollectIntentRows(ctx, reader, usePrefixIter, scanRes.Intents) if err != nil { return result.Result{}, err } diff --git a/pkg/kv/kvserver/batcheval/cmd_scan.go b/pkg/kv/kvserver/batcheval/cmd_scan.go index fd10986f6d07..090946f91b86 100644 --- a/pkg/kv/kvserver/batcheval/cmd_scan.go +++ b/pkg/kv/kvserver/batcheval/cmd_scan.go @@ -76,7 +76,11 @@ func Scan( } if h.ReadConsistency == roachpb.READ_UNCOMMITTED { - reply.IntentRows, err = CollectIntentRows(ctx, reader, cArgs, scanRes.Intents) + // NOTE: MVCCScan doesn't use a Prefix iterator, so we don't want to use + // one in CollectIntentRows either so that we're guaranteed to use the + // same cached iterator and observe a consistent snapshot of the engine. + const usePrefixIter = false + reply.IntentRows, err = CollectIntentRows(ctx, reader, usePrefixIter, scanRes.Intents) if err != nil { return result.Result{}, err } diff --git a/pkg/kv/kvserver/batcheval/intent.go b/pkg/kv/kvserver/batcheval/intent.go index 418ac41f444e..e93e2743c57b 100644 --- a/pkg/kv/kvserver/batcheval/intent.go +++ b/pkg/kv/kvserver/batcheval/intent.go @@ -17,39 +17,91 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/util/log" ) -// CollectIntentRows collects the key-value pairs for each intent provided. It -// also verifies that the ReturnIntents option is allowed. +// CollectIntentRows collects the provisional key-value pairs for each intent +// provided. // -// TODO(nvanbenschoten): mvccGetInternal should return the intent values directly -// when ReturnIntents is true. Since this will initially only be used for -// RangeLookups and since this is how they currently collect intent values, this -// is ok for now. +// The method accepts a reader and flag indicating whether a prefix iterator +// should be used when creating an iterator from the reader. This flexibility +// works around a limitation of the Engine.NewReadOnly interface where prefix +// iterators and non-prefix iterators pulled from the same read-only engine are +// not guaranteed to provide a consistent snapshot of the underlying engine. +// This function expects to be able to retrieve the corresponding provisional +// value for each of the provided intents. As such, it is critical that it +// observes the engine in the same state that it was in when the intent keys +// were originally collected. Because of this, callers are tasked with +// indicating whether the intents were originally collected using a prefix +// iterator or not. +// +// TODO(nvanbenschoten): remove the usePrefixIter complexity when we're fully on +// Pebble and can guarantee that all iterators created from a read-only engine +// are consistent. +// +// TODO(nvanbenschoten): mvccGetInternal should return the intent values +// directly when reading at the READ_UNCOMMITTED consistency level. Since this +// is only currently used for range lookups and when watching for a merge (both +// of which are off the hot path), this is ok for now. func CollectIntentRows( - ctx context.Context, reader storage.Reader, cArgs CommandArgs, intents []roachpb.Intent, + ctx context.Context, reader storage.Reader, usePrefixIter bool, intents []roachpb.Intent, ) ([]roachpb.KeyValue, error) { if len(intents) == 0 { return nil, nil } res := make([]roachpb.KeyValue, 0, len(intents)) - for _, intent := range intents { + for i := range intents { + kv, err := readProvisionalVal(ctx, reader, usePrefixIter, &intents[i]) + if err != nil { + switch t := err.(type) { + case *roachpb.WriteIntentError: + log.Fatalf(ctx, "unexpected %T in CollectIntentRows: %+v", t, t) + case *roachpb.ReadWithinUncertaintyIntervalError: + log.Fatalf(ctx, "unexpected %T in CollectIntentRows: %+v", t, t) + } + return nil, err + } + if kv.Value.IsPresent() { + res = append(res, kv) + } + } + return res, nil +} + +// readProvisionalVal retrieves the provisional value for the provided intent +// using the reader and the specified access method (i.e. with or without the +// use of a prefix iterator). The function returns an empty KeyValue if the +// intent is found to contain a deletion tombstone as its provisional value. +func readProvisionalVal( + ctx context.Context, reader storage.Reader, usePrefixIter bool, intent *roachpb.Intent, +) (roachpb.KeyValue, error) { + if usePrefixIter { val, _, err := storage.MVCCGetAsTxn( ctx, reader, intent.Key, intent.Txn.WriteTimestamp, intent.Txn, ) if err != nil { - return nil, err + return roachpb.KeyValue{}, err } if val == nil { // Intent is a deletion. - continue + return roachpb.KeyValue{}, nil } - res = append(res, roachpb.KeyValue{ - Key: intent.Key, - Value: *val, - }) + return roachpb.KeyValue{Key: intent.Key, Value: *val}, nil } - return res, nil + res, err := storage.MVCCScanAsTxn( + ctx, reader, intent.Key, intent.Key.Next(), intent.Txn.WriteTimestamp, intent.Txn, + ) + if err != nil { + return roachpb.KeyValue{}, err + } + if len(res.KVs) > 1 { + log.Fatalf(ctx, "multiple key-values returned from single-key scan: %+v", res.KVs) + } else if len(res.KVs) == 0 { + // Intent is a deletion. + return roachpb.KeyValue{}, nil + } + return res.KVs[0], nil + } // acquireUnreplicatedLocksOnKeys adds an unreplicated lock acquisition by the diff --git a/pkg/kv/kvserver/batcheval/intent_test.go b/pkg/kv/kvserver/batcheval/intent_test.go new file mode 100644 index 000000000000..6e97230f71d4 --- /dev/null +++ b/pkg/kv/kvserver/batcheval/intent_test.go @@ -0,0 +1,163 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package batcheval + +import ( + "context" + "testing" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/stretchr/testify/require" +) + +// instrumentedEngine wraps a storage.Engine and allows for various methods in +// the interface to be instrumented for testing purposes. +type instrumentedEngine struct { + storage.Engine + + onNewIterator func(storage.IterOptions) + // ... can be extended ... +} + +func (ie *instrumentedEngine) NewIterator(opts storage.IterOptions) storage.Iterator { + if ie.onNewIterator != nil { + ie.onNewIterator(opts) + } + return ie.Engine.NewIterator(opts) +} + +// TestCollectIntentsUsesSameIterator tests that all uses of CollectIntents +// (currently only by READ_UNCOMMITTED Gets, Scans, and ReverseScans) use the +// same cached iterator (prefix or non-prefix) for their initial read and their +// provisional value collection for any intents they find. +func TestCollectIntentsUsesSameIterator(t *testing.T) { + defer leaktest.AfterTest(t)() + + ctx := context.Background() + key := roachpb.Key("key") + ts := hlc.Timestamp{WallTime: 123} + header := roachpb.Header{ + Timestamp: ts, + ReadConsistency: roachpb.READ_UNCOMMITTED, + } + + testCases := []struct { + name string + run func(*testing.T, storage.ReadWriter) (intents []roachpb.KeyValue, _ error) + expPrefixIters int + expNonPrefixIters int + }{ + { + name: "get", + run: func(t *testing.T, db storage.ReadWriter) ([]roachpb.KeyValue, error) { + req := &roachpb.GetRequest{ + RequestHeader: roachpb.RequestHeader{Key: key}, + } + var resp roachpb.GetResponse + if _, err := Get(ctx, db, CommandArgs{Args: req, Header: header}, &resp); err != nil { + return nil, err + } + if resp.IntentValue == nil { + return nil, nil + } + return []roachpb.KeyValue{{Key: key, Value: *resp.IntentValue}}, nil + }, + expPrefixIters: 2, + expNonPrefixIters: 0, + }, + { + name: "scan", + run: func(t *testing.T, db storage.ReadWriter) ([]roachpb.KeyValue, error) { + req := &roachpb.ScanRequest{ + RequestHeader: roachpb.RequestHeader{Key: key, EndKey: key.Next()}, + } + var resp roachpb.ScanResponse + if _, err := Scan(ctx, db, CommandArgs{Args: req, Header: header}, &resp); err != nil { + return nil, err + } + return resp.IntentRows, nil + }, + expPrefixIters: 0, + expNonPrefixIters: 2, + }, + { + name: "reverse scan", + run: func(t *testing.T, db storage.ReadWriter) ([]roachpb.KeyValue, error) { + req := &roachpb.ReverseScanRequest{ + RequestHeader: roachpb.RequestHeader{Key: key, EndKey: key.Next()}, + } + var resp roachpb.ReverseScanResponse + if _, err := ReverseScan(ctx, db, CommandArgs{Args: req, Header: header}, &resp); err != nil { + return nil, err + } + return resp.IntentRows, nil + }, + expPrefixIters: 0, + expNonPrefixIters: 2, + }, + } + for _, c := range testCases { + t.Run(c.name, func(t *testing.T) { + // Test with and without deletion intents. If a READ_UNCOMMITTED request + // encounters an intent whose provisional value is a deletion tombstone, + // the request should ignore the intent and should not return any + // corresponding intent row. + testutils.RunTrueAndFalse(t, "deletion intent", func(t *testing.T, delete bool) { + db := &instrumentedEngine{Engine: storage.NewDefaultInMem()} + defer db.Close() + + // Write an intent. + val := roachpb.MakeValueFromBytes([]byte("val")) + txn := roachpb.MakeTransaction("test", key, roachpb.NormalUserPriority, ts, 0) + var err error + if delete { + err = storage.MVCCDelete(ctx, db, nil, key, ts, &txn) + } else { + err = storage.MVCCPut(ctx, db, nil, key, ts, val, &txn) + } + require.NoError(t, err) + + // Instrument iterator creation, count prefix vs. non-prefix iters. + var prefixIters, nonPrefixIters int + db.onNewIterator = func(opts storage.IterOptions) { + if opts.Prefix { + prefixIters++ + } else { + nonPrefixIters++ + } + } + + intents, err := c.run(t, db) + require.NoError(t, err) + + // Assert proper intent values. + if delete { + require.Len(t, intents, 0) + } else { + expIntentVal := val + expIntentVal.Timestamp = ts + expIntentKeyVal := roachpb.KeyValue{Key: key, Value: expIntentVal} + require.Len(t, intents, 1) + require.Equal(t, expIntentKeyVal, intents[0]) + } + + // Assert proper iterator use. + require.Equal(t, c.expPrefixIters, prefixIters) + require.Equal(t, c.expNonPrefixIters, nonPrefixIters) + require.Equal(t, c.expNonPrefixIters, nonPrefixIters) + }) + }) + } +} diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index c833540631e8..4cd38b3c0930 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -72,7 +72,7 @@ func (x ReadConsistencyType) String() string { return proto.EnumName(ReadConsistencyType_name, int32(x)) } func (ReadConsistencyType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_a9d9e49af26caba8, []int{0} + return fileDescriptor_api_953726689c67e522, []int{0} } // ScanFormat is an enumeration of the available response formats for MVCCScan @@ -100,7 +100,7 @@ func (x ScanFormat) String() string { return proto.EnumName(ScanFormat_name, int32(x)) } func (ScanFormat) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_a9d9e49af26caba8, []int{1} + return fileDescriptor_api_953726689c67e522, []int{1} } type ChecksumMode int32 @@ -147,7 +147,7 @@ func (x ChecksumMode) String() string { return proto.EnumName(ChecksumMode_name, int32(x)) } func (ChecksumMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_a9d9e49af26caba8, []int{2} + return fileDescriptor_api_953726689c67e522, []int{2} } // PushTxnType determines what action to take when pushing a transaction. @@ -178,7 +178,7 @@ func (x PushTxnType) String() string { return proto.EnumName(PushTxnType_name, int32(x)) } func (PushTxnType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_a9d9e49af26caba8, []int{3} + return fileDescriptor_api_953726689c67e522, []int{3} } type ExternalStorageProvider int32 @@ -216,7 +216,7 @@ func (x ExternalStorageProvider) String() string { return proto.EnumName(ExternalStorageProvider_name, int32(x)) } func (ExternalStorageProvider) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_a9d9e49af26caba8, []int{4} + return fileDescriptor_api_953726689c67e522, []int{4} } type MVCCFilter int32 @@ -239,7 +239,7 @@ func (x MVCCFilter) String() string { return proto.EnumName(MVCCFilter_name, int32(x)) } func (MVCCFilter) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_a9d9e49af26caba8, []int{5} + return fileDescriptor_api_953726689c67e522, []int{5} } type ResponseHeader_ResumeReason int32 @@ -265,7 +265,7 @@ func (x ResponseHeader_ResumeReason) String() string { return proto.EnumName(ResponseHeader_ResumeReason_name, int32(x)) } func (ResponseHeader_ResumeReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_a9d9e49af26caba8, []int{2, 0} + return fileDescriptor_api_953726689c67e522, []int{2, 0} } type CheckConsistencyResponse_Status int32 @@ -307,7 +307,7 @@ func (x CheckConsistencyResponse_Status) String() string { return proto.EnumName(CheckConsistencyResponse_Status_name, int32(x)) } func (CheckConsistencyResponse_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_a9d9e49af26caba8, []int{26, 0} + return fileDescriptor_api_953726689c67e522, []int{26, 0} } // RangeInfo describes a range which executed a request. It contains @@ -321,7 +321,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_a9d9e49af26caba8, []int{0} + return fileDescriptor_api_953726689c67e522, []int{0} } func (m *RangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -364,7 +364,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_a9d9e49af26caba8, []int{1} + return fileDescriptor_api_953726689c67e522, []int{1} } func (m *RequestHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -431,7 +431,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_a9d9e49af26caba8, []int{2} + return fileDescriptor_api_953726689c67e522, []int{2} } func (m *ResponseHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -465,7 +465,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_a9d9e49af26caba8, []int{3} + return fileDescriptor_api_953726689c67e522, []int{3} } func (m *GetRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -496,6 +496,11 @@ type GetResponse struct { ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` Value *Value `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"` // The intent seen, if any, when using the READ_UNCOMMITTED consistency level. + // + // NOTE: this field is not currently populated with intents for deletion + // tombstones. It probably should be because the value field may contain a + // value that is being deleted by a corresponding intent. We should revisit + // this decision if this ever becomes a problem. IntentValue *Value `protobuf:"bytes,3,opt,name=intent_value,json=intentValue,proto3" json:"intent_value,omitempty"` } @@ -503,7 +508,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_a9d9e49af26caba8, []int{4} + return fileDescriptor_api_953726689c67e522, []int{4} } func (m *GetResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -546,7 +551,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_a9d9e49af26caba8, []int{5} + return fileDescriptor_api_953726689c67e522, []int{5} } func (m *PutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -580,7 +585,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_a9d9e49af26caba8, []int{6} + return fileDescriptor_api_953726689c67e522, []int{6} } func (m *PutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -634,7 +639,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_a9d9e49af26caba8, []int{7} + return fileDescriptor_api_953726689c67e522, []int{7} } func (m *ConditionalPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -669,7 +674,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_a9d9e49af26caba8, []int{8} + return fileDescriptor_api_953726689c67e522, []int{8} } func (m *ConditionalPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -715,7 +720,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_a9d9e49af26caba8, []int{9} + return fileDescriptor_api_953726689c67e522, []int{9} } func (m *InitPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -749,7 +754,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_a9d9e49af26caba8, []int{10} + return fileDescriptor_api_953726689c67e522, []int{10} } func (m *InitPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -789,7 +794,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_a9d9e49af26caba8, []int{11} + return fileDescriptor_api_953726689c67e522, []int{11} } func (m *IncrementRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -826,7 +831,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_a9d9e49af26caba8, []int{12} + return fileDescriptor_api_953726689c67e522, []int{12} } func (m *IncrementResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -860,7 +865,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_a9d9e49af26caba8, []int{13} + return fileDescriptor_api_953726689c67e522, []int{13} } func (m *DeleteRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -894,7 +899,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_a9d9e49af26caba8, []int{14} + return fileDescriptor_api_953726689c67e522, []int{14} } func (m *DeleteResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -946,7 +951,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_a9d9e49af26caba8, []int{15} + return fileDescriptor_api_953726689c67e522, []int{15} } func (m *DeleteRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -983,7 +988,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_a9d9e49af26caba8, []int{16} + return fileDescriptor_api_953726689c67e522, []int{16} } func (m *DeleteRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1030,7 +1035,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_a9d9e49af26caba8, []int{17} + return fileDescriptor_api_953726689c67e522, []int{17} } func (m *ClearRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1064,7 +1069,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_a9d9e49af26caba8, []int{18} + return fileDescriptor_api_953726689c67e522, []int{18} } func (m *ClearRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1105,7 +1110,7 @@ func (m *RevertRangeRequest) Reset() { *m = RevertRangeRequest{} } func (m *RevertRangeRequest) String() string { return proto.CompactTextString(m) } func (*RevertRangeRequest) ProtoMessage() {} func (*RevertRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a9d9e49af26caba8, []int{19} + return fileDescriptor_api_953726689c67e522, []int{19} } func (m *RevertRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1139,7 +1144,7 @@ func (m *RevertRangeResponse) Reset() { *m = RevertRangeResponse{} } func (m *RevertRangeResponse) String() string { return proto.CompactTextString(m) } func (*RevertRangeResponse) ProtoMessage() {} func (*RevertRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a9d9e49af26caba8, []int{20} + return fileDescriptor_api_953726689c67e522, []int{20} } func (m *RevertRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1190,7 +1195,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_a9d9e49af26caba8, []int{21} + return fileDescriptor_api_953726689c67e522, []int{21} } func (m *ScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1223,6 +1228,11 @@ type ScanResponse struct { // The intent rows seen when performing a scan at the READ_UNCOMMITTED // consistency level. These rows do not count against the MaxSpanRequestKeys // count. + // + // NOTE: this field is not currently populated with intents for deletion + // tombstones. It probably should be because the rows field may contain + // key-values that are being deleted by corresponding intents. We should + // revisit this decision if this ever becomes a problem. IntentRows []KeyValue `protobuf:"bytes,3,rep,name=intent_rows,json=intentRows,proto3" json:"intent_rows"` // If set, each item in this repeated bytes field contains part of the results // in batch format - the key/value pairs are a buffer of varint-prefixed @@ -1237,7 +1247,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_a9d9e49af26caba8, []int{22} + return fileDescriptor_api_953726689c67e522, []int{22} } func (m *ScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1288,7 +1298,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_a9d9e49af26caba8, []int{23} + return fileDescriptor_api_953726689c67e522, []int{23} } func (m *ReverseScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1321,6 +1331,11 @@ type ReverseScanResponse struct { // The intent rows seen when performing a scan at the READ_UNCOMMITTED // consistency level. These rows do not count against the MaxSpanRequestKeys // count. + // + // NOTE: this field is not currently populated with intents for deletion + // tombstones. It probably should be because the rows field may contain + // key-values that are being deleted by corresponding intents. We should + // revisit this decision if this ever becomes a problem. IntentRows []KeyValue `protobuf:"bytes,3,rep,name=intent_rows,json=intentRows,proto3" json:"intent_rows"` // If set, each item in this repeated bytes field contains part of the results // in batch format - the key/value pairs are a buffer of varint-prefixed @@ -1335,7 +1350,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_a9d9e49af26caba8, []int{24} + return fileDescriptor_api_953726689c67e522, []int{24} } func (m *ReverseScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1388,7 +1403,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_a9d9e49af26caba8, []int{25} + return fileDescriptor_api_953726689c67e522, []int{25} } func (m *CheckConsistencyRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1425,7 +1440,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_a9d9e49af26caba8, []int{26} + return fileDescriptor_api_953726689c67e522, []int{26} } func (m *CheckConsistencyResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1469,7 +1484,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_a9d9e49af26caba8, []int{26, 0} + return fileDescriptor_api_953726689c67e522, []int{26, 0} } func (m *CheckConsistencyResponse_Result) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1517,7 +1532,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_a9d9e49af26caba8, []int{27} + return fileDescriptor_api_953726689c67e522, []int{27} } func (m *RecomputeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1553,7 +1568,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_a9d9e49af26caba8, []int{28} + return fileDescriptor_api_953726689c67e522, []int{28} } func (m *RecomputeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1650,7 +1665,7 @@ func (m *EndTxnRequest) Reset() { *m = EndTxnRequest{} } func (m *EndTxnRequest) String() string { return proto.CompactTextString(m) } func (*EndTxnRequest) ProtoMessage() {} func (*EndTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a9d9e49af26caba8, []int{29} + return fileDescriptor_api_953726689c67e522, []int{29} } func (m *EndTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1696,7 +1711,7 @@ func (m *EndTxnResponse) Reset() { *m = EndTxnResponse{} } func (m *EndTxnResponse) String() string { return proto.CompactTextString(m) } func (*EndTxnResponse) ProtoMessage() {} func (*EndTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a9d9e49af26caba8, []int{30} + return fileDescriptor_api_953726689c67e522, []int{30} } func (m *EndTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1757,7 +1772,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_a9d9e49af26caba8, []int{31} + return fileDescriptor_api_953726689c67e522, []int{31} } func (m *AdminSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1792,7 +1807,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_a9d9e49af26caba8, []int{32} + return fileDescriptor_api_953726689c67e522, []int{32} } func (m *AdminSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1831,7 +1846,7 @@ func (m *AdminUnsplitRequest) Reset() { *m = AdminUnsplitRequest{} } func (m *AdminUnsplitRequest) String() string { return proto.CompactTextString(m) } func (*AdminUnsplitRequest) ProtoMessage() {} func (*AdminUnsplitRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a9d9e49af26caba8, []int{33} + return fileDescriptor_api_953726689c67e522, []int{33} } func (m *AdminUnsplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1866,7 +1881,7 @@ func (m *AdminUnsplitResponse) Reset() { *m = AdminUnsplitResponse{} } func (m *AdminUnsplitResponse) String() string { return proto.CompactTextString(m) } func (*AdminUnsplitResponse) ProtoMessage() {} func (*AdminUnsplitResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a9d9e49af26caba8, []int{34} + return fileDescriptor_api_953726689c67e522, []int{34} } func (m *AdminUnsplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1909,7 +1924,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_a9d9e49af26caba8, []int{35} + return fileDescriptor_api_953726689c67e522, []int{35} } func (m *AdminMergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1944,7 +1959,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_a9d9e49af26caba8, []int{36} + return fileDescriptor_api_953726689c67e522, []int{36} } func (m *AdminMergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1982,7 +1997,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_a9d9e49af26caba8, []int{37} + return fileDescriptor_api_953726689c67e522, []int{37} } func (m *AdminTransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2015,7 +2030,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_a9d9e49af26caba8, []int{38} + return fileDescriptor_api_953726689c67e522, []int{38} } func (m *AdminTransferLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2050,7 +2065,7 @@ func (m *ReplicationChange) Reset() { *m = ReplicationChange{} } func (m *ReplicationChange) String() string { return proto.CompactTextString(m) } func (*ReplicationChange) ProtoMessage() {} func (*ReplicationChange) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a9d9e49af26caba8, []int{39} + return fileDescriptor_api_953726689c67e522, []int{39} } func (m *ReplicationChange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2108,7 +2123,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_a9d9e49af26caba8, []int{40} + return fileDescriptor_api_953726689c67e522, []int{40} } func (m *AdminChangeReplicasRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2143,7 +2158,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_a9d9e49af26caba8, []int{41} + return fileDescriptor_api_953726689c67e522, []int{41} } func (m *AdminChangeReplicasResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2180,7 +2195,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_a9d9e49af26caba8, []int{42} + return fileDescriptor_api_953726689c67e522, []int{42} } func (m *AdminRelocateRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2213,7 +2228,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_a9d9e49af26caba8, []int{43} + return fileDescriptor_api_953726689c67e522, []int{43} } func (m *AdminRelocateRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2252,7 +2267,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_a9d9e49af26caba8, []int{44} + return fileDescriptor_api_953726689c67e522, []int{44} } func (m *HeartbeatTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2289,7 +2304,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_a9d9e49af26caba8, []int{45} + return fileDescriptor_api_953726689c67e522, []int{45} } func (m *HeartbeatTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2327,7 +2342,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_a9d9e49af26caba8, []int{46} + return fileDescriptor_api_953726689c67e522, []int{46} } func (m *GCRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2361,7 +2376,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_a9d9e49af26caba8, []int{46, 0} + return fileDescriptor_api_953726689c67e522, []int{46, 0} } func (m *GCRequest_GCKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2395,7 +2410,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_a9d9e49af26caba8, []int{47} + return fileDescriptor_api_953726689c67e522, []int{47} } func (m *GCResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2464,7 +2479,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_a9d9e49af26caba8, []int{48} + return fileDescriptor_api_953726689c67e522, []int{48} } func (m *PushTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2507,7 +2522,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_a9d9e49af26caba8, []int{49} + return fileDescriptor_api_953726689c67e522, []int{49} } func (m *PushTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2554,7 +2569,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_a9d9e49af26caba8, []int{50} + return fileDescriptor_api_953726689c67e522, []int{50} } func (m *RecoverTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2590,7 +2605,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_a9d9e49af26caba8, []int{51} + return fileDescriptor_api_953726689c67e522, []int{51} } func (m *RecoverTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2634,7 +2649,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_a9d9e49af26caba8, []int{52} + return fileDescriptor_api_953726689c67e522, []int{52} } func (m *QueryTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2673,7 +2688,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_a9d9e49af26caba8, []int{53} + return fileDescriptor_api_953726689c67e522, []int{53} } func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2733,7 +2748,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_a9d9e49af26caba8, []int{54} + return fileDescriptor_api_953726689c67e522, []int{54} } func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2769,7 +2784,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_a9d9e49af26caba8, []int{55} + return fileDescriptor_api_953726689c67e522, []int{55} } func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2815,7 +2830,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_a9d9e49af26caba8, []int{56} + return fileDescriptor_api_953726689c67e522, []int{56} } func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2850,7 +2865,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_a9d9e49af26caba8, []int{57} + return fileDescriptor_api_953726689c67e522, []int{57} } func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2900,7 +2915,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_a9d9e49af26caba8, []int{58} + return fileDescriptor_api_953726689c67e522, []int{58} } func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2935,7 +2950,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_a9d9e49af26caba8, []int{59} + return fileDescriptor_api_953726689c67e522, []int{59} } func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2972,7 +2987,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_a9d9e49af26caba8, []int{60} + return fileDescriptor_api_953726689c67e522, []int{60} } func (m *MergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3006,7 +3021,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_a9d9e49af26caba8, []int{61} + return fileDescriptor_api_953726689c67e522, []int{61} } func (m *MergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3051,7 +3066,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_a9d9e49af26caba8, []int{62} + return fileDescriptor_api_953726689c67e522, []int{62} } func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3085,7 +3100,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_a9d9e49af26caba8, []int{63} + return fileDescriptor_api_953726689c67e522, []int{63} } func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3129,7 +3144,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_a9d9e49af26caba8, []int{64} + return fileDescriptor_api_953726689c67e522, []int{64} } func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3178,7 +3193,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_a9d9e49af26caba8, []int{65} + return fileDescriptor_api_953726689c67e522, []int{65} } func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3215,7 +3230,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_a9d9e49af26caba8, []int{66} + return fileDescriptor_api_953726689c67e522, []int{66} } func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3252,7 +3267,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_a9d9e49af26caba8, []int{67} + return fileDescriptor_api_953726689c67e522, []int{67} } func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3287,7 +3302,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_a9d9e49af26caba8, []int{68} + return fileDescriptor_api_953726689c67e522, []int{68} } func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3342,7 +3357,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_a9d9e49af26caba8, []int{69} + return fileDescriptor_api_953726689c67e522, []int{69} } func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3379,7 +3394,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_a9d9e49af26caba8, []int{70} + return fileDescriptor_api_953726689c67e522, []int{70} } func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3418,7 +3433,7 @@ func (m *ExternalStorage) Reset() { *m = ExternalStorage{} } func (m *ExternalStorage) String() string { return proto.CompactTextString(m) } func (*ExternalStorage) ProtoMessage() {} func (*ExternalStorage) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a9d9e49af26caba8, []int{71} + return fileDescriptor_api_953726689c67e522, []int{71} } func (m *ExternalStorage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3452,7 +3467,7 @@ func (m *ExternalStorage_LocalFilePath) Reset() { *m = ExternalStorage_L func (m *ExternalStorage_LocalFilePath) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_LocalFilePath) ProtoMessage() {} func (*ExternalStorage_LocalFilePath) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a9d9e49af26caba8, []int{71, 0} + return fileDescriptor_api_953726689c67e522, []int{71, 0} } func (m *ExternalStorage_LocalFilePath) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3485,7 +3500,7 @@ func (m *ExternalStorage_Http) Reset() { *m = ExternalStorage_Http{} } func (m *ExternalStorage_Http) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Http) ProtoMessage() {} func (*ExternalStorage_Http) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a9d9e49af26caba8, []int{71, 1} + return fileDescriptor_api_953726689c67e522, []int{71, 1} } func (m *ExternalStorage_Http) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3525,7 +3540,7 @@ func (m *ExternalStorage_S3) Reset() { *m = ExternalStorage_S3{} } func (m *ExternalStorage_S3) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_S3) ProtoMessage() {} func (*ExternalStorage_S3) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a9d9e49af26caba8, []int{71, 2} + return fileDescriptor_api_953726689c67e522, []int{71, 2} } func (m *ExternalStorage_S3) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3564,7 +3579,7 @@ func (m *ExternalStorage_GCS) Reset() { *m = ExternalStorage_GCS{} } func (m *ExternalStorage_GCS) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_GCS) ProtoMessage() {} func (*ExternalStorage_GCS) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a9d9e49af26caba8, []int{71, 3} + return fileDescriptor_api_953726689c67e522, []int{71, 3} } func (m *ExternalStorage_GCS) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3600,7 +3615,7 @@ func (m *ExternalStorage_Azure) Reset() { *m = ExternalStorage_Azure{} } func (m *ExternalStorage_Azure) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Azure) ProtoMessage() {} func (*ExternalStorage_Azure) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a9d9e49af26caba8, []int{71, 4} + return fileDescriptor_api_953726689c67e522, []int{71, 4} } func (m *ExternalStorage_Azure) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3639,7 +3654,7 @@ func (m *ExternalStorage_Workload) Reset() { *m = ExternalStorage_Worklo func (m *ExternalStorage_Workload) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Workload) ProtoMessage() {} func (*ExternalStorage_Workload) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a9d9e49af26caba8, []int{71, 5} + return fileDescriptor_api_953726689c67e522, []int{71, 5} } func (m *ExternalStorage_Workload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3679,7 +3694,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_a9d9e49af26caba8, []int{72} + return fileDescriptor_api_953726689c67e522, []int{72} } func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3713,7 +3728,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_a9d9e49af26caba8, []int{73} + return fileDescriptor_api_953726689c67e522, []int{73} } func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3747,7 +3762,7 @@ func (m *FileEncryptionOptions) Reset() { *m = FileEncryptionOptions{} } func (m *FileEncryptionOptions) String() string { return proto.CompactTextString(m) } func (*FileEncryptionOptions) ProtoMessage() {} func (*FileEncryptionOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a9d9e49af26caba8, []int{74} + return fileDescriptor_api_953726689c67e522, []int{74} } func (m *FileEncryptionOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3817,7 +3832,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_a9d9e49af26caba8, []int{75} + return fileDescriptor_api_953726689c67e522, []int{75} } func (m *ExportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3869,7 +3884,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_a9d9e49af26caba8, []int{76} + return fileDescriptor_api_953726689c67e522, []int{76} } func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3905,7 +3920,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_a9d9e49af26caba8, []int{77} + return fileDescriptor_api_953726689c67e522, []int{77} } func (m *ExportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3945,7 +3960,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_a9d9e49af26caba8, []int{77, 0} + return fileDescriptor_api_953726689c67e522, []int{77, 0} } func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3996,7 +4011,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_a9d9e49af26caba8, []int{78} + return fileDescriptor_api_953726689c67e522, []int{78} } func (m *ImportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4031,7 +4046,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_a9d9e49af26caba8, []int{78, 0} + return fileDescriptor_api_953726689c67e522, []int{78, 0} } func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4067,7 +4082,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_a9d9e49af26caba8, []int{78, 1} + return fileDescriptor_api_953726689c67e522, []int{78, 1} } func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4102,7 +4117,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_a9d9e49af26caba8, []int{79} + return fileDescriptor_api_953726689c67e522, []int{79} } func (m *ImportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4140,7 +4155,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_a9d9e49af26caba8, []int{80} + return fileDescriptor_api_953726689c67e522, []int{80} } func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4175,7 +4190,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_a9d9e49af26caba8, []int{81} + return fileDescriptor_api_953726689c67e522, []int{81} } func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4208,7 +4223,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_a9d9e49af26caba8, []int{81, 0} + return fileDescriptor_api_953726689c67e522, []int{81, 0} } func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4253,7 +4268,7 @@ func (m *AdminVerifyProtectedTimestampRequest) Reset() { *m = AdminVerif func (m *AdminVerifyProtectedTimestampRequest) String() string { return proto.CompactTextString(m) } func (*AdminVerifyProtectedTimestampRequest) ProtoMessage() {} func (*AdminVerifyProtectedTimestampRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a9d9e49af26caba8, []int{82} + return fileDescriptor_api_953726689c67e522, []int{82} } func (m *AdminVerifyProtectedTimestampRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4291,7 +4306,7 @@ func (m *AdminVerifyProtectedTimestampResponse) Reset() { *m = AdminVeri func (m *AdminVerifyProtectedTimestampResponse) String() string { return proto.CompactTextString(m) } func (*AdminVerifyProtectedTimestampResponse) ProtoMessage() {} func (*AdminVerifyProtectedTimestampResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a9d9e49af26caba8, []int{83} + return fileDescriptor_api_953726689c67e522, []int{83} } func (m *AdminVerifyProtectedTimestampResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4344,7 +4359,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_a9d9e49af26caba8, []int{84} + return fileDescriptor_api_953726689c67e522, []int{84} } func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4378,7 +4393,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_a9d9e49af26caba8, []int{85} + return fileDescriptor_api_953726689c67e522, []int{85} } func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4422,7 +4437,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_a9d9e49af26caba8, []int{86} + return fileDescriptor_api_953726689c67e522, []int{86} } func (m *RefreshRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4456,7 +4471,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_a9d9e49af26caba8, []int{87} + return fileDescriptor_api_953726689c67e522, []int{87} } func (m *RefreshResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4495,7 +4510,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_a9d9e49af26caba8, []int{88} + return fileDescriptor_api_953726689c67e522, []int{88} } func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4529,7 +4544,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_a9d9e49af26caba8, []int{89} + return fileDescriptor_api_953726689c67e522, []int{89} } func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4578,7 +4593,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_a9d9e49af26caba8, []int{90} + return fileDescriptor_api_953726689c67e522, []int{90} } func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4627,7 +4642,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_a9d9e49af26caba8, []int{91} + return fileDescriptor_api_953726689c67e522, []int{91} } func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4662,7 +4677,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_a9d9e49af26caba8, []int{92} + return fileDescriptor_api_953726689c67e522, []int{92} } func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4701,7 +4716,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_a9d9e49af26caba8, []int{93} + return fileDescriptor_api_953726689c67e522, []int{93} } func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4784,7 +4799,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_a9d9e49af26caba8, []int{94} + return fileDescriptor_api_953726689c67e522, []int{94} } func (m *RequestUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6234,7 +6249,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_a9d9e49af26caba8, []int{95} + return fileDescriptor_api_953726689c67e522, []int{95} } func (m *ResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7749,7 +7764,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_a9d9e49af26caba8, []int{96} + return fileDescriptor_api_953726689c67e522, []int{96} } func (m *Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7785,7 +7800,7 @@ type BatchRequest struct { func (m *BatchRequest) Reset() { *m = BatchRequest{} } func (*BatchRequest) ProtoMessage() {} func (*BatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a9d9e49af26caba8, []int{97} + return fileDescriptor_api_953726689c67e522, []int{97} } func (m *BatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7822,7 +7837,7 @@ type BatchResponse struct { func (m *BatchResponse) Reset() { *m = BatchResponse{} } func (*BatchResponse) ProtoMessage() {} func (*BatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a9d9e49af26caba8, []int{98} + return fileDescriptor_api_953726689c67e522, []int{98} } func (m *BatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7880,7 +7895,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_a9d9e49af26caba8, []int{98, 0} + return fileDescriptor_api_953726689c67e522, []int{98, 0} } func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7919,7 +7934,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_a9d9e49af26caba8, []int{99} + return fileDescriptor_api_953726689c67e522, []int{99} } func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7960,7 +7975,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_a9d9e49af26caba8, []int{100} + return fileDescriptor_api_953726689c67e522, []int{100} } func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8001,7 +8016,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_a9d9e49af26caba8, []int{101} + return fileDescriptor_api_953726689c67e522, []int{101} } func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8038,7 +8053,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_a9d9e49af26caba8, []int{102} + return fileDescriptor_api_953726689c67e522, []int{102} } func (m *RangeFeedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8075,7 +8090,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_a9d9e49af26caba8, []int{103} + return fileDescriptor_api_953726689c67e522, []int{103} } func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -37489,9 +37504,9 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_a9d9e49af26caba8) } +func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_953726689c67e522) } -var fileDescriptor_api_a9d9e49af26caba8 = []byte{ +var fileDescriptor_api_953726689c67e522 = []byte{ // 7362 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5d, 0x6c, 0x23, 0xc9, 0x75, 0xae, 0x9a, 0xa4, 0x24, 0xf2, 0x90, 0xa2, 0x5a, 0xa5, 0xf9, 0xe1, 0x68, 0x66, 0x25, 0x0d, diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 5faa0dfe9d97..c8a85da17f01 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -129,6 +129,11 @@ message GetResponse { Value value = 2; // The intent seen, if any, when using the READ_UNCOMMITTED consistency level. + // + // NOTE: this field is not currently populated with intents for deletion + // tombstones. It probably should be because the value field may contain a + // value that is being deleted by a corresponding intent. We should revisit + // this decision if this ever becomes a problem. Value intent_value = 3; } @@ -372,6 +377,11 @@ message ScanResponse { // The intent rows seen when performing a scan at the READ_UNCOMMITTED // consistency level. These rows do not count against the MaxSpanRequestKeys // count. + // + // NOTE: this field is not currently populated with intents for deletion + // tombstones. It probably should be because the rows field may contain + // key-values that are being deleted by corresponding intents. We should + // revisit this decision if this ever becomes a problem. repeated KeyValue intent_rows = 3 [(gogoproto.nullable) = false]; // If set, each item in this repeated bytes field contains part of the results @@ -419,6 +429,11 @@ message ReverseScanResponse { // The intent rows seen when performing a scan at the READ_UNCOMMITTED // consistency level. These rows do not count against the MaxSpanRequestKeys // count. + // + // NOTE: this field is not currently populated with intents for deletion + // tombstones. It probably should be because the rows field may contain + // key-values that are being deleted by corresponding intents. We should + // revisit this decision if this ever becomes a problem. repeated KeyValue intent_rows = 3 [(gogoproto.nullable) = false]; // If set, each item in this repeated bytes field contains part of the results diff --git a/pkg/storage/engine.go b/pkg/storage/engine.go index de80c58fca8b..8767c8df8b6b 100644 --- a/pkg/storage/engine.go +++ b/pkg/storage/engine.go @@ -364,10 +364,23 @@ type Engine interface { // this engine. Batched engines accumulate all mutations and apply // them atomically on a call to Commit(). NewBatch() Batch - // NewReadOnly returns a new instance of a ReadWriter that wraps - // this engine. This wrapper panics when unexpected operations (e.g., write + // NewReadOnly returns a new instance of a ReadWriter that wraps this + // engine. This wrapper panics when unexpected operations (e.g., write // operations) are executed on it and caches iterators to avoid the overhead // of creating multiple iterators for batched reads. + // + // All iterators created from a read-only engine with the same "Prefix" + // option are guaranteed to provide a consistent snapshot of the underlying + // engine. For instance, two prefix iterators created from a read-only + // engine will provide a consistent snapshot. Similarly, two non-prefix + // iterators created from a read-only engine will provide a consistent + // snapshot. However, a prefix iterator and a non-prefix iterator created + // from a read-only engine are not guaranteed to provide a consistent view + // of the underlying engine. + // + // TODO(nvanbenschoten): remove this complexity when we're fully on Pebble + // and can guarantee that all iterators created from a read-only engine are + // consistent. To do this, we will want to add an Iterator.Clone method. NewReadOnly() ReadWriter // NewWriteOnlyBatch returns a new instance of a batched engine which wraps // this engine. A write-only batch accumulates all mutations and applies them diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 233235d60722..bb56340a28fd 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -2544,6 +2544,28 @@ func MVCCScanToBytes( return mvccScanToBytes(ctx, iter, key, endKey, timestamp, opts) } +// MVCCScanAsTxn constructs a temporary transaction from the given transaction +// metadata and calls MVCCScan as that transaction. This method is required only +// for reading intents of a transaction when only its metadata is known and +// should rarely be used. +// +// The read is carried out without the chance of uncertainty restarts. +func MVCCScanAsTxn( + ctx context.Context, + reader Reader, + key, endKey roachpb.Key, + timestamp hlc.Timestamp, + txnMeta enginepb.TxnMeta, +) (MVCCScanResult, error) { + return MVCCScan(ctx, reader, key, endKey, timestamp, MVCCScanOptions{ + Txn: &roachpb.Transaction{ + TxnMeta: txnMeta, + Status: roachpb.PENDING, + ReadTimestamp: txnMeta.WriteTimestamp, + MaxTimestamp: txnMeta.WriteTimestamp, + }}) +} + // MVCCIterate iterates over the key range [start,end). At each step of the // iteration, f() is invoked with the current key/value pair. If f returns // true (done) or an error, the iteration stops and the error is propagated.